Skip to content
180 changes: 115 additions & 65 deletions openeo_processes_dask/process_implementations/cubes/_filter.py
Original file line number Diff line number Diff line change
@@ -1,7 +1,7 @@
import json
import logging
import warnings
from typing import Any, Callable, Optional
from typing import Any, Callable, Optional, Union

import dask.array as da
import geopandas as gpd
Expand All @@ -12,11 +12,15 @@
import shapely
import xarray as xr
from openeo_pg_parser_networkx.pg_schema import BoundingBox, TemporalInterval
from shapely.geometry import box

from openeo_processes_dask.process_implementations.cubes.mask_polygon import (
mask_polygon,
)
from openeo_processes_dask.process_implementations.data_model import RasterCube
from openeo_processes_dask.process_implementations.data_model import (
RasterCube,
VectorCube,
)
from openeo_processes_dask.process_implementations.exceptions import (
BandFilterParameterMissing,
DimensionMissing,
Expand Down Expand Up @@ -168,76 +172,122 @@ def filter_spatial(data: RasterCube, geometries) -> RasterCube:
return data


def filter_bbox(data: RasterCube, extent: BoundingBox) -> RasterCube:
try:
input_crs = str(data.rio.crs)
except Exception as e:
raise Exception(f"Not possible to estimate the input data projection! {e}")
if not pyproj.crs.CRS(extent.crs).equals(input_crs):
reprojected_extent = _reproject_bbox(extent, input_crs)
else:
reprojected_extent = extent
y_dim = data.openeo.y_dim
x_dim = data.openeo.x_dim

# Check first if the data has some spatial dimensions:
if y_dim is None and x_dim is None:
raise DimensionNotAvailable(
"No spatial dimensions available, can't apply filter_bbox."
def filter_bbox(
data: Union[VectorCube, RasterCube], extent: BoundingBox
) -> Union[VectorCube, RasterCube]:
if not isinstance(data, (RasterCube, VectorCube)):
raise TypeError(
f"filter_bbox expects a RasterCube or VectorCube, but got {type(data)}."
)

if y_dim is not None:
# Check if the coordinates are increasing or decreasing
if len(data[y_dim]) > 1:
if data[y_dim][0] > data[y_dim][1]:
y_slice = slice(reprojected_extent.north, reprojected_extent.south)
else:
y_slice = slice(reprojected_extent.south, reprojected_extent.north)
if isinstance(data, RasterCube):
try:
input_crs = str(data.rio.crs)
except Exception as e:
raise Exception(f"Not possible to estimate the input data projection! {e}")
if not pyproj.crs.CRS(extent.crs).equals(input_crs):
reprojected_extent = _reproject_bbox(extent, input_crs)
else:
# We need to check if the bbox crosses this single coordinate
# if data[y_dim][0] < reprojected_extent.north and data[y_dim][0] > reprojected_extent.south:
# # bbox crosses the single coordinate
# y_slice = data[y_dim][0]
# else:
# # bbox doesn't cross the single coordinate: return empty data or error?
raise NotImplementedError(
f"filter_bbox can't filter data with a single coordinate on {y_dim} yet."
reprojected_extent = extent
y_dim = data.openeo.y_dim
x_dim = data.openeo.x_dim

# Check first if the data has some spatial dimensions:
if y_dim is None and x_dim is None:
raise DimensionNotAvailable(
"No spatial dimensions available, can't apply filter_bbox."
)

if x_dim is not None:
if len(data[x_dim]) > 1:
if data[x_dim][0] > data[x_dim][1]:
x_slice = slice(reprojected_extent.east, reprojected_extent.west)
if y_dim is not None:
# Check if the coordinates are increasing or decreasing
if len(data[y_dim]) > 1:
if data[y_dim][0] > data[y_dim][1]:
y_slice = slice(reprojected_extent.north, reprojected_extent.south)
else:
y_slice = slice(reprojected_extent.south, reprojected_extent.north)
else:
x_slice = slice(reprojected_extent.west, reprojected_extent.east)
# We need to check if the bbox crosses this single coordinate
# if data[y_dim][0] < reprojected_extent.north and data[y_dim][0] > reprojected_extent.south:
# # bbox crosses the single coordinate
# y_slice = data[y_dim][0]
# else:
# # bbox doesn't cross the single coordinate: return empty data or error?
raise NotImplementedError(
f"filter_bbox can't filter data with a single coordinate on {y_dim} yet."
)

if x_dim is not None:
if len(data[x_dim]) > 1:
if data[x_dim][0] > data[x_dim][1]:
x_slice = slice(reprojected_extent.east, reprojected_extent.west)
else:
x_slice = slice(reprojected_extent.west, reprojected_extent.east)
else:
# We need to check if the bbox crosses this single coordinate. How to do this correctly?
# if data[x_dim][0] < reprojected_extent.east and data[x_dim][0] > reprojected_extent.west:
# # bbox crosses the single coordinate
# y_slice = data[x_dim][0]
# else:
# # bbox doesn't cross the single coordinate: return empty data or error?
raise NotImplementedError(
f"filter_bbox can't filter data with a single coordinate on {x_dim} yet."
)

if y_dim is not None and x_dim is not None:
aoi = data.loc[{y_dim: y_slice, x_dim: x_slice}]
elif x_dim is None:
aoi = data.loc[{y_dim: y_slice}]
else:
# We need to check if the bbox crosses this single coordinate. How to do this correctly?
# if data[x_dim][0] < reprojected_extent.east and data[x_dim][0] > reprojected_extent.west:
# # bbox crosses the single coordinate
# y_slice = data[x_dim][0]
# else:
# # bbox doesn't cross the single coordinate: return empty data or error?
raise NotImplementedError(
f"filter_bbox can't filter data with a single coordinate on {x_dim} yet."
)
aoi = data.loc[{x_dim: x_slice}]

if y_dim is not None and x_dim is not None:
aoi = data.loc[{y_dim: y_slice, x_dim: x_slice}]
elif x_dim is None:
aoi = data.loc[{y_dim: y_slice}]
else:
aoi = data.loc[{x_dim: x_slice}]
return aoi

elif isinstance(data, VectorCube):
if hasattr(data, "crs"):
input_crs = str(data.crs)
elif hasattr(data, "attrs") and "crs" in data.attrs:
input_crs = str(data.attrs["crs"])
else:
raise ValueError("Cannot determine CRS of VectorCube.")

if not pyproj.crs.CRS(extent.crs).equals(input_crs):
reprojected_extent = _reproject_bbox(extent, input_crs)
else:
reprojected_extent = extent

return aoi
bbox_geom = box(
reprojected_extent.west,
reprojected_extent.south,
reprojected_extent.east,
reprojected_extent.north,
)

# GeoDataFrame or Dask GeoDataFrame
if hasattr(data, "geometry"):
# Use intersects() instead of within() to include geometries touching the boundary
filtered = data[
data.geometry.intersects(bbox_geom) & (~data.geometry.is_empty)
]
return filtered

# xarray.Dataset with geometry variable
elif isinstance(data, xr.Dataset) and "geometry" in data:
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we do not have a test for this, yet, do we?

# Use intersects() for consistent spatial filtering behavior
mask = data["geometry"].intersects(bbox_geom) & (~data["geometry"].is_empty)
return data.where(mask, drop=True)
else:
raise TypeError("Unsupported VectorCube type for filter_bbox")


def _reproject_bbox(extent: BoundingBox, target_crs: str) -> BoundingBox:
# Create corner points of the bounding box (x, y order for consistency)
bbox_points = [
[extent.south, extent.west],
[extent.south, extent.east],
[extent.north, extent.east],
[extent.north, extent.west],
[extent.west, extent.south], # bottom-left
[extent.east, extent.south], # bottom-right
[extent.east, extent.north], # top-right
[extent.west, extent.north], # top-left
]

if extent.crs is not None:
source_crs = extent.crs
else:
Expand All @@ -247,17 +297,17 @@ def _reproject_bbox(extent: BoundingBox, target_crs: str) -> BoundingBox:

x_reprojected = []
y_reprojected = []
for p in bbox_points:
x1, y1 = p
x2, y2 = transformer.transform(y1, x1)
x_reprojected.append(x2)
y_reprojected.append(y2)
for point in bbox_points:
x, y = point
# transformer.transform expects (x, y) when always_xy=True
x_transformed, y_transformed = transformer.transform(x, y)
x_reprojected.append(x_transformed)
y_reprojected.append(y_transformed)

x_reprojected = np.array(x_reprojected)
y_reprojected = np.array(y_reprojected)

reprojected_extent = {}

# Create the reprojected bounding box from the transformed coordinates
reprojected_extent = BoundingBox(
west=x_reprojected.min(),
east=x_reprojected.max(),
Expand Down
Loading