Skip to content

Commit

Permalink
Merge branch 'Open-EO:main' into main
Browse files Browse the repository at this point in the history
  • Loading branch information
clausmichele authored Aug 9, 2023
2 parents 0fbfc61 + ede97a8 commit f6f493f
Show file tree
Hide file tree
Showing 10 changed files with 179 additions and 65 deletions.
7 changes: 0 additions & 7 deletions openeo_processes_dask/process_implementations/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -15,13 +15,6 @@
"Did not load machine learning processes due to missing dependencies: Install them like this: `pip install openeo-processes-dask[implementations, ml]`"
)

try:
from .experimental import *
except ImportError as e:
logger.warning(
"Did not load experimental processes due to missing dependencies. Install them like this: `pip install openeo-processes-dask[implementations, experimental]`"
)

import rioxarray as rio # Required for the .rio accessor on xarrays.

import openeo_processes_dask.process_implementations.cubes._xr_interop
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@
from .aggregate import *
from .apply import *
from .general import *
from .indices import *
from .load import *
from .merge import *
from .reduce import *
59 changes: 59 additions & 0 deletions openeo_processes_dask/process_implementations/cubes/indices.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
import xarray as xr

from openeo_processes_dask.process_implementations.data_model import RasterCube
from openeo_processes_dask.process_implementations.exceptions import (
BandExists,
DimensionAmbiguous,
NirBandAmbiguous,
RedBandAmbiguous,
)
from openeo_processes_dask.process_implementations.math import normalized_difference

__all__ = ["ndvi"]


def ndvi(data: RasterCube, nir="nir", red="red", target_band=None):
if len(data.openeo.band_dims) == 0:
raise DimensionAmbiguous(
"Dimension of type `bands` is not available or is ambiguous."
)
band_dim = data.openeo.band_dims[0]
available_bands = data.coords[band_dim]

if nir not in available_bands or red not in available_bands:
try:
data = data.set_xindex("common_name")
except (ValueError, KeyError):
pass

if (
nir not in available_bands
and "common_name" in data.xindexes._coord_name_id.keys()
and nir not in data.coords["common_name"].data
):
raise NirBandAmbiguous(
"The NIR band can't be resolved, please specify the specific NIR band name."
)
elif (
red not in available_bands
and "common_name" in data.xindexes._coord_name_id.keys()
and red not in data.coords["common_name"].data
):
raise RedBandAmbiguous(
"The Red band can't be resolved, please specify the specific Red band name."
)

nir_band_dim = "common_name" if nir not in available_bands else band_dim
red_band_dim = "common_name" if red not in available_bands else band_dim

nir_band = data.sel({nir_band_dim: nir})
red_band = data.sel({red_band_dim: red})

nd = normalized_difference(nir_band, red_band)
if target_band is not None:
if target_band in data.coords:
raise BandExists("A band with the specified target name exists.")
nd = nd.expand_dims(band_dim).assign_coords({band_dim: [target_band]})
nd = xr.merge([data, nd])
nd.attrs = data.attrs
return nd
16 changes: 16 additions & 0 deletions openeo_processes_dask/process_implementations/exceptions.py
Original file line number Diff line number Diff line change
Expand Up @@ -64,3 +64,19 @@ class NoDataAvailable(OpenEOException):

class TemporalExtentEmpty(OpenEOException):
pass


class DimensionAmbiguous(OpenEOException):
pass


class NirBandAmbiguous(OpenEOException):
pass


class RedBandAmbiguous(OpenEOException):
pass


class BandExists(OpenEOException):
pass
Original file line number Diff line number Diff line change
@@ -1,4 +1,3 @@
import odc.algo
import rioxarray # needs to be imported to set .rio accessor on xarray objects.

from openeo_processes_dask.process_implementations.data_model import RasterCube
Expand All @@ -9,13 +8,6 @@
def resample_cube_spatial(
data: RasterCube, target: RasterCube, method="near", options=None
) -> RasterCube:
# NOTE: Using the odc-algo library is not great, because it is only a random collection of experimental opendatacube features
# but we've investigated all other available alternatives for resampling that are currently available with dask and none do the job.
# We've tested pyresample (did not work at all and requires loads of helper code),
# rasterio.reproject_match (loads all the data into memory to do gdal.warp) and odc-geo (doesn't support dask yet).
# Github issue tracking this feature in rioxarray: https://github.com/corteva/rioxarray/issues/119
# Github issue tracking this feature in odc-geo: https://github.com/opendatacube/odc-geo/issues/26

methods_list = [
"near",
"bilinear",
Expand All @@ -31,8 +23,12 @@ def resample_cube_spatial(
"q3",
]

# ODC reproject requires y to be before x
required_dim_order = (
data.openeo.band_dims + data.openeo.temporal_dims + data.openeo.spatial_dims
data.openeo.band_dims
+ data.openeo.temporal_dims
+ tuple(data.openeo.y_dim)
+ tuple(data.openeo.x_dim)
)

data_reordered = data.transpose(*required_dim_order, missing_dims="ignore")
Expand All @@ -47,13 +43,14 @@ def resample_cube_spatial(
f"[{', '.join(methods_list)}]"
)

resampled_data = odc.algo._warp.xr_reproject(
data_reordered, target_reordered.geobox, resampling=method
resampled_data = data_reordered.odc.reproject(
target_reordered.odc.geobox, resampling=method
)

resampled_data.rio.write_crs(target_reordered.rio.crs, inplace=True)

try:
# xr_reproject renames the coordinates according to the geobox, this undoes that.
# odc.reproject renames the coordinates according to the geobox, this undoes that.
resampled_data = resampled_data.rename(
{"longitude": data.openeo.x_dim, "latitude": data.openeo.y_dim}
)
Expand Down
39 changes: 0 additions & 39 deletions openeo_processes_dask/process_implementations/math.py
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,6 @@
"quantiles",
"product",
"normalized_difference",
"ndvi",
]


Expand Down Expand Up @@ -326,41 +325,3 @@ def product(data, ignore_nodata=True, axis=None, keepdims=False):
def normalized_difference(x, y):
nd = (x - y) / (x + y)
return nd


def ndvi(data, nir="nir", red="red", target_band=None):
r = np.nan
n = np.nan
if "bands" in data.dims:
if red == "red":
if "B04" in data["bands"].values:
r = data.sel(bands="B04")
elif red == "rededge":
if "B05" in data["bands"].values:
r = data.sel(bands="B05")
elif "B06" in data["bands"].values:
r = data.sel(bands="B06")
elif "B07" in data["bands"].values:
r = data.sel(bands="B07")
if nir == "nir":
n = data.sel(bands="B08")
elif nir == "nir08":
if "B8a" in data["bands"].values:
n = data.sel(bands="B8a")
elif "B8A" in data["bands"].values:
n = data.sel(bands="B8A")
elif "B05" in data["bands"].values:
n = data.sel(bands="B05")
elif nir == "nir09":
if "B09" in data["bands"].values:
n = data.sel(bands="B09")
if red in data["bands"].values:
r = data.sel(bands=red)
if nir in data["bands"].values:
n = data.sel(bands=nir)
nd = normalized_difference(n, r)
if target_band is not None:
nd = nd.assign_coords(bands=target_band)
# TODO: Remove this once we have the .openeo accessor
nd.attrs = data.attrs
return nd
2 changes: 1 addition & 1 deletion openeo_processes_dask/specs/openeo-processes
6 changes: 2 additions & 4 deletions pyproject.toml
Original file line number Diff line number Diff line change
@@ -1,6 +1,6 @@
[tool.poetry]
name = "openeo-processes-dask"
version = "2023.7.1"
version = "2023.7.5"
description = "Python implementations of many OpenEO processes, dask-friendly by default."
authors = ["Lukas Weidenholzer <[email protected]>", "Sean Hoyal <[email protected]>", "Valentina Hutter <[email protected]>"]
maintainers = ["EODC Staff <[email protected]>"]
Expand Down Expand Up @@ -31,9 +31,8 @@ rasterio = { version = "^1.3.4", optional = true }
dask-geopandas = { version = ">=0.2.0,<1", optional = true }
xgboost = { version = "^1.5.1", optional = true }
rioxarray = { version = ">=0.12.0,<1", optional = true }
odc-algo = { version = "==0.2.3", optional = true }
openeo-pg-parser-networkx = { version = ">=2023.5.1", optional = true }
odc-geo = { version = "^0.3.2", optional = true }
odc-geo = { version = ">=0.3.2,<1", optional = true }
stac_validator = { version = ">=3.3.1", optional = true }
stackstac = { version = ">=0.4.3", optional = true }
pystac_client = { version = ">=0.6.1", optional = true }
Expand All @@ -50,7 +49,6 @@ pytest-cov = "^4.0.0"

[tool.poetry.extras]
implementations = ["geopandas", "xarray", "dask", "rasterio", "dask-geopandas", "rioxarray", "openeo-pg-parser-networkx", "odc-geo", "stackstac", "planetary_computer", "pystac_client", "stac_validator"]
experimental = ["odc-algo"]
ml = ["xgboost"]

[tool.pytest.ini_options]
Expand Down
8 changes: 6 additions & 2 deletions tests/conftest.py
Original file line number Diff line number Diff line change
Expand Up @@ -28,14 +28,18 @@ def dask_client():
client.shutdown()


@pytest.fixture
def random_raster_data(size, dtype, seed=42):
def _random_raster_data(size, dtype, seed=42):
rng = np.random.default_rng(seed)
data = rng.integers(-100, 100, size=size)
data = data.astype(dtype)
return data


@pytest.fixture
def random_raster_data(size, dtype, seed=42):
return _random_raster_data(size, dtype, seed=seed)


@pytest.fixture
def bounding_box(
west=10.45, east=10.5, south=46.1, north=46.2, crs="EPSG:4326"
Expand Down
85 changes: 85 additions & 0 deletions tests/test_indices.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,85 @@
import numpy as np
import pytest

from openeo_processes_dask.process_implementations.cubes.indices import ndvi
from openeo_processes_dask.process_implementations.cubes.load import load_stac
from openeo_processes_dask.process_implementations.exceptions import (
BandExists,
DimensionAmbiguous,
NirBandAmbiguous,
RedBandAmbiguous,
)
from tests.conftest import _random_raster_data
from tests.general_checks import general_output_checks


def test_ndvi(bounding_box):
url = "./tests/data/stac/s2_l2a_test_item.json"
input_cube = load_stac(
url=url,
spatial_extent=bounding_box,
bands=["red", "nir"],
).isel({"x": slice(0, 20), "y": slice(0, 20)})

# Test whether this works with different band names
input_cube = input_cube.rename({"band": "b"})

import dask.array as da

numpy_data = _random_raster_data(input_cube.data.shape, dtype=np.float64)

input_cube.data = da.from_array(numpy_data, chunks=("auto", "auto", "auto", -1))

output = ndvi(input_cube)

band_dim = input_cube.openeo.band_dims[0]
assert band_dim not in output.dims

expected_results = (
input_cube.sel({band_dim: "nir"}) - input_cube.sel({band_dim: "red"})
) / (input_cube.sel({band_dim: "nir"}) + input_cube.sel({band_dim: "red"}))

general_output_checks(
input_cube=input_cube, output_cube=output, expected_results=expected_results
)

cube_with_resolvable_coords = input_cube.assign_coords(
{band_dim: ["blue", "yellow"]}
)
output = ndvi(cube_with_resolvable_coords)
general_output_checks(
input_cube=cube_with_resolvable_coords,
output_cube=output,
expected_results=expected_results,
)

with pytest.raises(DimensionAmbiguous):
ndvi(output)

cube_with_nir_unresolvable = cube_with_resolvable_coords
cube_with_nir_unresolvable.common_name.data = np.array(["blue", "red"])

with pytest.raises(NirBandAmbiguous):
ndvi(cube_with_nir_unresolvable)

cube_with_red_unresolvable = cube_with_resolvable_coords
cube_with_red_unresolvable.common_name.data = np.array(["nir", "yellow"])

with pytest.raises(RedBandAmbiguous):
ndvi(cube_with_red_unresolvable)

cube_with_nothing_resolvable = cube_with_resolvable_coords
cube_with_nothing_resolvable = cube_with_nothing_resolvable.drop_vars("common_name")
with pytest.raises(KeyError):
ndvi(cube_with_nothing_resolvable)

target_band = "yay"
output_with_extra_dim = ndvi(input_cube, target_band=target_band)
assert len(output_with_extra_dim.dims) == len(output.dims) + 1
assert (
len(output_with_extra_dim.coords[band_dim])
== len(input_cube.coords[band_dim]) + 1
)

with pytest.raises(BandExists):
output_with_extra_dim = ndvi(input_cube, target_band="time")

0 comments on commit f6f493f

Please sign in to comment.