Skip to content

Commit

Permalink
feat: add filter_temporal process (#76)
Browse files Browse the repository at this point in the history
* partial fix of sggregate_temporal

* Revert "partial fix of sggregate_temporal"

This reverts commit bfd7a7e.

* implemented filter_temporal

* added filter_temporal test

* added filter_temporal test

* added filter_temporal test

* fixed little things

* removed unused import

* merge main

* add spec for filter_temporal

* add test with open interval

---------

Co-authored-by: Lukas Weidenholzer <[email protected]>
  • Loading branch information
clausmichele and Lukas Weidenholzer authored Mar 20, 2023
1 parent f782ec8 commit e5b84d8
Show file tree
Hide file tree
Showing 3 changed files with 96 additions and 8 deletions.
49 changes: 42 additions & 7 deletions openeo_processes_dask/process_implementations/cubes/_filter.py
Original file line number Diff line number Diff line change
@@ -1,27 +1,62 @@
import logging
from typing import Callable

from openeo_pg_parser_networkx.pg_schema import BoundingBox, GeoJson, TemporalInterval
import numpy as np
from openeo_pg_parser_networkx.pg_schema import TemporalInterval

from openeo_processes_dask.process_implementations.data_model import RasterCube
from openeo_processes_dask.process_implementations.exceptions import (
DimensionNotAvailable,
TooManyDimensions,
)

logger = logging.getLogger(__name__)

__all__ = ["filter_labels"]
__all__ = ["filter_labels", "filter_temporal"]


def filter_temporal(
data: RasterCube, extent: TemporalInterval, dimension: str
data: RasterCube, extent: TemporalInterval, dimension: str = None
) -> RasterCube:
raise NotImplementedError()
temporal_dims = data.openeo.temporal_dims

if dimension is not None:
if dimension not in data.dims:
raise DimensionNotAvailable(
f"A dimension with the specified name: {dimension} does not exist."
)
applicable_temporal_dimension = dimension
if dimension not in temporal_dims:
logger.warning(
f"The selected dimension {dimension} exists but it is not labeled as a temporal dimension. Available temporal diemnsions are {temporal_dims}."
)
else:
if not temporal_dims:
raise DimensionNotAvailable(
f"No temporal dimension detected on dataset. Available dimensions: {data.dims}"
)
if len(temporal_dims) > 1:
raise TooManyDimensions(
f"The data cube contains multiple temporal dimensions: {temporal_dims}. The parameter `dimension` must be specified."
)
applicable_temporal_dimension = temporal_dims[0]

def filter_labels(
data: RasterCube, condition: Callable, dimension: str, **kwargs
) -> RasterCube:
start_time = extent[0]
if start_time is not None:
start_time = start_time.to_numpy()
end_time = extent[1]
if end_time is not None:
end_time = extent[1].to_numpy() - np.timedelta64(1, "ms")
# The second element is the end of the temporal interval.
# The specified instance in time is excluded from the interval.
# See https://processes.openeo.org/#filter_temporal

filtered = data.loc[{applicable_temporal_dimension: slice(start_time, end_time)}]

return filtered


def filter_labels(data: RasterCube, condition: Callable, dimension: str) -> RasterCube:
if dimension not in data.dims:
raise DimensionNotAvailable(
f"Provided dimension ({dimension}) not found in data.dims: {data.dims}"
Expand Down
2 changes: 1 addition & 1 deletion openeo_processes_dask/specs/openeo-processes
53 changes: 53 additions & 0 deletions tests/test_filter.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
import numpy as np
import pytest
import xarray as xr
from openeo_pg_parser_networkx.pg_schema import TemporalInterval

from openeo_processes_dask.process_implementations.cubes._filter import filter_temporal
from openeo_processes_dask.process_implementations.exceptions import (
DimensionNotAvailable,
)
from tests.general_checks import general_output_checks
from tests.mockdata import create_fake_rastercube


@pytest.mark.parametrize("size", [(30, 30, 30, 1)])
@pytest.mark.parametrize("dtype", [np.uint8])
def test_filter_temporal(temporal_interval, bounding_box, random_raster_data):
input_cube = create_fake_rastercube(
data=random_raster_data,
spatial_extent=bounding_box,
temporal_extent=temporal_interval,
bands=["B02"],
backend="dask",
)

temporal_interval_part = TemporalInterval.parse_obj(
["2018-05-15T00:00:00", "2018-06-01T00:00:00"]
)
output_cube = filter_temporal(data=input_cube, extent=temporal_interval_part)

general_output_checks(
input_cube=input_cube,
output_cube=output_cube,
verify_attrs=False,
verify_crs=True,
)

xr.testing.assert_equal(
output_cube,
input_cube.loc[dict(t=slice("2018-05-15T00:00:00", "2018-05-31T23:59:59"))],
)

with pytest.raises(DimensionNotAvailable):
filter_temporal(
data=input_cube, extent=temporal_interval_part, dimension="immissing"
)

temporal_interval_open = TemporalInterval.parse_obj([None, "2018-05-03T00:00:00"])
output_cube = filter_temporal(data=input_cube, extent=temporal_interval_open)

xr.testing.assert_equal(
output_cube,
input_cube.loc[dict(t=slice("2018-05-01T00:00:00", "2018-05-02T23:59:59"))],
)

0 comments on commit e5b84d8

Please sign in to comment.