From 2a01bfa83579786604f2e2f7afe0b26027a8c7cc Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Tue, 4 Feb 2025 14:38:40 -0800 Subject: [PATCH 01/78] Added zarray_to_v3metadata and test --- virtualizarr/tests/test_zarr.py | 36 +++++++++++++++++++- virtualizarr/zarr.py | 60 +++++++++++++++------------------ 2 files changed, 63 insertions(+), 33 deletions(-) diff --git a/virtualizarr/tests/test_zarr.py b/virtualizarr/tests/test_zarr.py index 95dbf55f..421b6937 100644 --- a/virtualizarr/tests/test_zarr.py +++ b/virtualizarr/tests/test_zarr.py @@ -1,6 +1,7 @@ import numpy as np +from zarr.core.metadata.v3 import ArrayV3Metadata -from virtualizarr.zarr import ZArray +from virtualizarr.zarr import ZArray, zarray_to_v3metadata def test_replace_partial(): @@ -27,3 +28,36 @@ def test_replace_total(): result = arr.replace(**kwargs) expected = ZArray(**kwargs) assert result == expected + + +def test_zarray_to_v3metadata(): + from zarr.codecs import BytesCodec + + zarray = ZArray( + shape=(5, 20), + chunks=(5, 10), + dtype=np.dtype("int32"), + fill_value=0, + order="C", + compressor={"id": "zlib", "level": 1}, + filters=None, + zarr_format=3, + ) + + metadata = zarray_to_v3metadata(zarray) + + assert isinstance(metadata, ArrayV3Metadata) + assert metadata.shape == (5, 20) + assert metadata.data_type.value == "int32" + chunk_grid_dict = metadata.chunk_grid.to_dict() + assert chunk_grid_dict["name"] == "regular" + assert chunk_grid_dict["configuration"]["chunk_shape"] == (5, 10) + assert metadata.chunk_key_encoding.name == "default" + assert metadata.fill_value == np.int32(0) + assert type(metadata.codecs[0]) is BytesCodec + metadata_codec_dict = metadata.codecs[1].to_dict() + assert metadata_codec_dict["name"] == "numcodecs.zlib" + assert metadata_codec_dict["configuration"]["level"] == 1 + assert metadata.attributes == {} + assert metadata.dimension_names is None + assert metadata.storage_transformers == () diff --git a/virtualizarr/zarr.py b/virtualizarr/zarr.py index 4c36e7dc..6e9454d2 100644 --- a/virtualizarr/zarr.py +++ b/virtualizarr/zarr.py @@ -2,6 +2,7 @@ from typing import TYPE_CHECKING, Any, Literal, NewType, cast import numpy as np +from zarr.core.metadata.v3 import ArrayV3Metadata if TYPE_CHECKING: pass @@ -157,39 +158,17 @@ def _v3_codec_pipeline(self) -> Any: ``` """ try: - from zarr.core.metadata.v3 import ( # type: ignore[import-untyped] - parse_codecs, - ) + from zarr.abc.codec import Codec as ZarrCodec + from zarr.core.array import _parse_chunk_encoding_v3 except ImportError: - raise ImportError("zarr v3 is required to generate v3 codec pipelines") - - codec_configs = [] - - # https://zarr-specs.readthedocs.io/en/latest/v3/codecs/transpose/v1.0.html#transpose-codec-v1 - # Either "C" or "F", defining the layout of bytes within each chunk of the array. - # "C" means row-major order, i.e., the last dimension varies fastest; - # "F" means column-major order, i.e., the first dimension varies fastest. - # For now, we only need transpose if the order is not "C" - if self.order == "F": - order = tuple(reversed(range(len(self.shape)))) - transpose = dict(name="transpose", configuration=dict(order=order)) - codec_configs.append(transpose) - - # Noting here that zarr v3 has very few codecs specificed in the official spec, - # and that there are far more codecs in `numcodecs`. We take a gamble and assume - # that the codec names and configuration are simply mapped into zarrv3 "configurables". - if self.filters: - codec_configs.extend( - [_num_codec_config_to_configurable(filter) for filter in self.filters] - ) - - if self.compressor: - codec_configs.append(_num_codec_config_to_configurable(self.compressor)) - - # convert the pipeline repr into actual v3 codec objects - codec_pipeline = parse_codecs(codec_configs) - - return codec_pipeline + raise ImportError("zarr v3 is required to generate v3 codecs") + codecs = _parse_chunk_encoding_v3( # returns tuple[tuple[ArrayArrayCodec, ...], ArrayBytesCodec, tuple[BytesBytesCodec, ...]] + compressors=[_num_codec_config_to_configurable(self.compressor)], + filters=self.filters, + dtype=self.dtype, + serializer="auto", + ) + return cast(tuple["ZarrCodec", ...], (*codecs[0], codecs[1], *codecs[2])) def serializer(self) -> Any: """ @@ -239,3 +218,20 @@ def _num_codec_config_to_configurable(num_codec: dict) -> dict: num_codec_copy = num_codec.copy() name = "numcodecs." + num_codec_copy.pop("id") return {"name": name, "configuration": num_codec_copy} + + +def zarray_to_v3metadata(zarray: ZArray) -> ArrayV3Metadata: + """ + Convert a ZArray to a zarr v3 metadata object. + """ + return ArrayV3Metadata( + shape=zarray.shape, + data_type=zarray.dtype, + chunk_grid={"name": "regular", "configuration": {"chunk_shape": zarray.chunks}}, + chunk_key_encoding={"name": "default"}, + fill_value=zarray.fill_value, + codecs=zarray._v3_codec_pipeline(), + attributes={}, + dimension_names=None, + storage_transformers=None, + ) From 17fd5473b1aaf8f408f0555f50a30bd4d12500e3 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Tue, 4 Feb 2025 18:00:36 -0800 Subject: [PATCH 02/78] Working on manifest array tests --- virtualizarr/codecs.py | 6 +- virtualizarr/manifests/array.py | 45 +++-- virtualizarr/manifests/array_api.py | 14 +- .../tests/test_manifests/test_array.py | 183 ++++++++++-------- virtualizarr/zarr.py | 65 ++++--- 5 files changed, 186 insertions(+), 127 deletions(-) diff --git a/virtualizarr/codecs.py b/virtualizarr/codecs.py index 94f6d1aa..d8eb4691 100644 --- a/virtualizarr/codecs.py +++ b/virtualizarr/codecs.py @@ -54,10 +54,8 @@ def _get_manifestarray_codecs( normalize_to_zarr_v3: bool = False, ) -> Union[Codec, tuple["ArrayArrayCodec | ArrayBytesCodec | BytesBytesCodec", ...]]: """Get codecs for a ManifestArray based on its zarr_format.""" - if normalize_to_zarr_v3 or array.zarray.zarr_format == 3: - return (array.zarray.serializer(),) + array.zarray._v3_codec_pipeline() - elif array.zarray.zarr_format == 2: - return array.zarray.codec + if normalize_to_zarr_v3 or array.metadata.zarr_format == 3: + return array.metadata.codecs else: raise ValueError("Unsupported zarr_format for ManifestArray.") diff --git a/virtualizarr/manifests/array.py b/virtualizarr/manifests/array.py index 89f648aa..763bea20 100644 --- a/virtualizarr/manifests/array.py +++ b/virtualizarr/manifests/array.py @@ -2,13 +2,13 @@ from typing import Any, Callable, Union import numpy as np +from zarr.core.metadata.v3 import ArrayV3Metadata, RegularChunkGrid from virtualizarr.manifests.array_api import ( MANIFESTARRAY_HANDLED_ARRAY_FUNCTIONS, _isnan, ) from virtualizarr.manifests.manifest import ChunkManifest -from virtualizarr.zarr import ZArray class ManifestArray: @@ -24,11 +24,11 @@ class ManifestArray: """ _manifest: ChunkManifest - _zarray: ZArray + _metadata: ArrayV3Metadata def __init__( self, - zarray: ZArray | dict, + metadata: ArrayV3Metadata | dict, chunkmanifest: dict | ChunkManifest, ) -> None: """ @@ -36,15 +36,15 @@ def __init__( Parameters ---------- - zarray : dict or ZArray + metadata : dict or ArrayV3Metadata chunkmanifest : dict or ChunkManifest """ - if isinstance(zarray, ZArray): - _zarray = zarray + if isinstance(metadata, ArrayV3Metadata): + _metadata = metadata else: # try unpacking the dict - _zarray = ZArray(**zarray) + _metadata = ArrayV3Metadata(**metadata) if isinstance(chunkmanifest, ChunkManifest): _chunkmanifest = chunkmanifest @@ -55,10 +55,10 @@ def __init__( f"chunkmanifest arg must be of type ChunkManifest or dict, but got type {type(chunkmanifest)}" ) - # TODO check that the zarray shape and chunkmanifest shape are consistent with one another + # TODO check that the metadata shape and chunkmanifest shape are consistent with one another # TODO also cover the special case of scalar arrays - self._zarray = _zarray + self._metadata = _metadata self._manifest = _chunkmanifest @property @@ -66,21 +66,32 @@ def manifest(self) -> ChunkManifest: return self._manifest @property - def zarray(self) -> ZArray: - return self._zarray + def metadata(self) -> ArrayV3Metadata: + return self._metadata @property def chunks(self) -> tuple[int, ...]: - return tuple(self.zarray.chunks) + """ + Individual chunk size by number of elements. + """ + if isinstance(self._metadata.chunk_grid, RegularChunkGrid): + return self._metadata.chunk_grid.chunk_shape + else: + raise NotImplementedError( + "Only RegularChunkGrid is currently supported for chunk size" + ) @property def dtype(self) -> np.dtype: - dtype_str = self.zarray.dtype - return np.dtype(dtype_str) + dtype_str = self.metadata.data_type + return dtype_str.to_numpy() @property def shape(self) -> tuple[int, ...]: - return tuple(int(length) for length in list(self.zarray.shape)) + """ + Array shape by number of elements along each dimension. + """ + return tuple(int(length) for length in list(self.metadata.shape)) @property def ndim(self) -> int: @@ -155,7 +166,7 @@ def __eq__( # type: ignore[override] if self.shape != other.shape: raise NotImplementedError("Unsure how to handle broadcasting like this") - if self.zarray != other.zarray: + if self.metadata != other.metadata: return np.full(shape=self.shape, fill_value=False, dtype=np.dtype(bool)) else: if self.manifest == other.manifest: @@ -263,7 +274,7 @@ def rename_paths( ChunkManifest.rename_paths """ renamed_manifest = self.manifest.rename_paths(new) - return ManifestArray(zarray=self.zarray, chunkmanifest=renamed_manifest) + return ManifestArray(metadata=self.metadata, chunkmanifest=renamed_manifest) def _possibly_expand_trailing_ellipsis(key, ndim: int): diff --git a/virtualizarr/manifests/array_api.py b/virtualizarr/manifests/array_api.py index 4950b48c..bf5ce44e 100644 --- a/virtualizarr/manifests/array_api.py +++ b/virtualizarr/manifests/array_api.py @@ -100,12 +100,14 @@ def concatenate( lengths=concatenated_lengths, ) - # chunk shape has not changed, there are just now more chunks along the concatenation axis - new_zarray = first_arr.zarray.replace( - shape=tuple(new_shape), - ) - - return ManifestArray(chunkmanifest=concatenated_manifest, zarray=new_zarray) + # FIXME(aimee): there is probably a more elegant way to do this + new_metadata = first_arr.metadata.to_dict().copy() + new_metadata["shape"] = tuple(new_shape) + new_metadata["dimension_names"] = new_metadata.get("dimension_names", None) + new_metadata.pop("zarr_format", None) + new_metadata.pop("node_type", None) + + return ManifestArray(chunkmanifest=concatenated_manifest, metadata=new_metadata) @implements(np.stack) diff --git a/virtualizarr/tests/test_manifests/test_array.py b/virtualizarr/tests/test_manifests/test_array.py index 2444e4ea..e89b9242 100644 --- a/virtualizarr/tests/test_manifests/test_array.py +++ b/virtualizarr/tests/test_manifests/test_array.py @@ -1,9 +1,10 @@ import numpy as np import pytest +from zarr.core.metadata.v3 import ArrayV3Metadata from virtualizarr.manifests import ChunkManifest, ManifestArray from virtualizarr.tests import create_manifestarray -from virtualizarr.zarr import ZArray +from virtualizarr.zarr import ZArray, convert_to_codec_pipeline class TestManifestArray: @@ -17,18 +18,58 @@ def test_create_manifestarray(self): manifest = ChunkManifest(entries=chunks_dict) chunks = (5, 1, 10) shape = (5, 2, 20) - zarray = ZArray( - chunks=chunks, - compressor={"id": "zlib", "level": 1}, - dtype=np.dtype("int32"), - fill_value=0.0, - filters=None, - order="C", + metadata = ArrayV3Metadata( shape=shape, - zarr_format=2, + data_type="int32", + chunk_grid={"name": "regular", "configuration": {"chunk_shape": chunks}}, + chunk_key_encoding={"name": "default"}, + fill_value=0, + codecs=convert_to_codec_pipeline( + compressor={"id": "zlib", "level": 1}, + filters=None, + dtype=np.dtype("int32"), + ), + attributes={}, + dimension_names=None, + storage_transformers=None, ) - marr = ManifestArray(zarray=zarray, chunkmanifest=manifest) + marr = ManifestArray(metadata=metadata, chunkmanifest=manifest) + assert marr.chunks == chunks + assert marr.dtype == np.dtype("int32") + assert marr.shape == shape + assert marr.size == 5 * 2 * 20 + assert marr.ndim == 3 + + def test_create_manifestarray_dict_v3_metadata(self): + chunks_dict = { + "0.0.0": {"path": "s3://bucket/foo.nc", "offset": 100, "length": 100}, + "0.0.1": {"path": "s3://bucket/foo.nc", "offset": 200, "length": 100}, + "0.1.0": {"path": "s3://bucket/foo.nc", "offset": 300, "length": 100}, + "0.1.1": {"path": "s3://bucket/foo.nc", "offset": 400, "length": 100}, + } + manifest = ChunkManifest(entries=chunks_dict) + chunks = (5, 1, 10) + shape = (5, 2, 20) + metadata_dict = { + "shape": shape, + "data_type": "int32", + "chunk_grid": {"name": "regular", "configuration": {"chunk_shape": chunks}}, + "chunk_key_encoding": {"name": "default"}, + "fill_value": 0, + "codecs": [ + {"configuration": {"endian": "little"}, "name": "bytes"}, + { + "name": "numcodecs.zlib", + "configuration": {"level": 1}, + }, + ], + "attributes": {}, + "dimension_names": None, + "storage_transformers": None, + } + + marr = ManifestArray(metadata=metadata_dict, chunkmanifest=manifest) assert marr.chunks == chunks assert marr.dtype == np.dtype("int32") assert marr.shape == shape @@ -213,85 +254,73 @@ def test_broadcast_empty(self, shape, chunks, grid_shape, target_shape): # The hard part is having an alternative way to get to the expected result of concatenation class TestConcat: def test_concat(self): - # both manifest arrays in this example have the same zarray properties - zarray = ZArray( - chunks=(5, 1, 10), - compressor={"id": "zlib", "level": 1}, - dtype=np.dtype("int32"), - fill_value=0.0, - filters=None, - order="C", - shape=(5, 1, 20), - zarr_format=2, - ) - + # both manifest arrays in this example have the same metadata properties chunks_dict1 = { - "0.0.0": {"path": "/foo.nc", "offset": 100, "length": 100}, - "0.0.1": {"path": "/foo.nc", "offset": 200, "length": 100}, + "0.0.0": {"path": "/foo1.nc", "offset": 100, "length": 100}, } - manifest1 = ChunkManifest(entries=chunks_dict1) - marr1 = ManifestArray(zarray=zarray, chunkmanifest=manifest1) - chunks_dict2 = { - "0.0.0": {"path": "/foo.nc", "offset": 300, "length": 100}, - "0.0.1": {"path": "/foo.nc", "offset": 400, "length": 100}, + "0.0.0": {"path": "/foo2.nc", "offset": 200, "length": 100}, } + manifest1 = ChunkManifest(entries=chunks_dict1) manifest2 = ChunkManifest(entries=chunks_dict2) - marr2 = ManifestArray(zarray=zarray, chunkmanifest=manifest2) + chunks = (5, 1, 10) + shape = (5, 2, 20) + metadata = ArrayV3Metadata( + shape=shape, + data_type="int32", + chunk_grid={"name": "regular", "configuration": {"chunk_shape": chunks}}, + chunk_key_encoding={"name": "default"}, + fill_value=0, + codecs=convert_to_codec_pipeline( + compressor={"id": "zlib", "level": 1}, + filters=None, + dtype=np.dtype("int32"), + ), + attributes={}, + dimension_names=None, + storage_transformers=None, + ) - result = np.concatenate([marr1, marr2], axis=1) + marr1 = ManifestArray(metadata=metadata, chunkmanifest=manifest1) + marr2 = ManifestArray(metadata=metadata, chunkmanifest=manifest2) - assert result.shape == (5, 2, 20) - assert result.chunks == (5, 1, 10) - assert result.manifest.dict() == { - "0.0.0": {"path": "file:///foo.nc", "offset": 100, "length": 100}, - "0.0.1": {"path": "file:///foo.nc", "offset": 200, "length": 100}, - "0.1.0": {"path": "file:///foo.nc", "offset": 300, "length": 100}, - "0.1.1": {"path": "file:///foo.nc", "offset": 400, "length": 100}, - } - assert result.zarray.compressor == zarray.compressor - assert result.zarray.filters == zarray.filters - assert result.zarray.fill_value == zarray.fill_value - assert result.zarray.order == zarray.order - assert result.zarray.zarr_format == zarray.zarr_format + # Concatenate along the first axis + concatenated = np.concatenate([marr1, marr2], axis=0) + assert concatenated.shape == (10, 2, 20) + assert concatenated.dtype == np.dtype("int32") + # FAILING: TypeError: no implementation found for 'numpy.concatenate' on types that implement __array_function__: [, ] def test_concat_empty(self): - # both manifest arrays in this example have the same zarray properties - zarray = ZArray( - chunks=(5, 1, 10), - compressor={"id": "zlib", "level": 1}, - dtype=np.dtype("int32"), - fill_value=0.0, - filters=None, - order="C", - shape=(5, 1, 20), - zarr_format=2, - ) - - chunks_dict1 = {} - manifest1 = ChunkManifest(entries=chunks_dict1, shape=(1, 1, 2)) - marr1 = ManifestArray(zarray=zarray, chunkmanifest=manifest1) - - chunks_dict2 = { - "0.0.0": {"path": "/foo.nc", "offset": 300, "length": 100}, - "0.0.1": {"path": "/foo.nc", "offset": 400, "length": 100}, + chunks_dict1 = { + "0.0.0": {"path": "/foo1.nc", "offset": 100, "length": 100}, } - manifest2 = ChunkManifest(entries=chunks_dict2) - marr2 = ManifestArray(zarray=zarray, chunkmanifest=manifest2) + manifest1 = ChunkManifest(entries=chunks_dict1) + chunks = (5, 1, 10) + shape = (5, 2, 20) + metadata = ArrayV3Metadata( + shape=shape, + data_type="int32", + chunk_grid={"name": "regular", "configuration": {"chunk_shape": chunks}}, + chunk_key_encoding={"name": "default"}, + fill_value=0, + codecs=convert_to_codec_pipeline( + compressor={"id": "zlib", "level": 1}, + filters=None, + dtype=np.dtype("int32"), + ), + attributes={}, + dimension_names=None, + storage_transformers=None, + ) - result = np.concatenate([marr1, marr2], axis=1) + marr1 = ManifestArray(metadata=metadata, chunkmanifest=manifest1) - assert result.shape == (5, 2, 20) - assert result.chunks == (5, 1, 10) - assert result.manifest.dict() == { - "0.1.0": {"path": "file:///foo.nc", "offset": 300, "length": 100}, - "0.1.1": {"path": "file:///foo.nc", "offset": 400, "length": 100}, - } - assert result.zarray.compressor == zarray.compressor - assert result.zarray.filters == zarray.filters - assert result.zarray.fill_value == zarray.fill_value - assert result.zarray.order == zarray.order - assert result.zarray.zarr_format == zarray.zarr_format + # Concatenate with an empty array + concatenated = np.concatenate( + [marr1, np.empty((0, 2, 20), dtype="int32")], axis=0 + ) + assert concatenated.shape == (5, 2, 20) + assert concatenated.dtype == np.dtype("int32") class TestStack: diff --git a/virtualizarr/zarr.py b/virtualizarr/zarr.py index 6e9454d2..3fee2df0 100644 --- a/virtualizarr/zarr.py +++ b/virtualizarr/zarr.py @@ -2,10 +2,11 @@ from typing import TYPE_CHECKING, Any, Literal, NewType, cast import numpy as np +from zarr.abc.codec import Codec as ZarrCodec from zarr.core.metadata.v3 import ArrayV3Metadata if TYPE_CHECKING: - pass + from zarr.core.array import CompressorsLike, FiltersLike, SerializerLike # TODO replace these with classes imported directly from Zarr? (i.e. Zarr Object Models) ZAttrs = NewType( @@ -144,31 +145,13 @@ def replace( replacements["zarr_format"] = zarr_format return dataclasses.replace(self, **replacements) - def _v3_codec_pipeline(self) -> Any: + def _v3_codec_pipeline(self) -> tuple["ZarrCodec", ...]: """ - VirtualiZarr internally uses the `filters`, `compressor`, and `order` attributes - from zarr v2, but to create conformant zarr v3 metadata those 3 must be turned into `codecs` objects. - Not all codecs are created equal though: https://github.com/zarr-developers/zarr-python/issues/1943 - An array _must_ declare a single ArrayBytes codec, and 0 or more ArrayArray, BytesBytes codecs. - Roughly, this is the mapping: - ``` - filters: Iterable[ArrayArrayCodec] #optional - compressor: ArrayBytesCodec #mandatory - post_compressor: Iterable[BytesBytesCodec] #optional - ``` + Convert the compressor, filters, and dtype to a pipeline of ZarrCodecs. """ - try: - from zarr.abc.codec import Codec as ZarrCodec - from zarr.core.array import _parse_chunk_encoding_v3 - except ImportError: - raise ImportError("zarr v3 is required to generate v3 codecs") - codecs = _parse_chunk_encoding_v3( # returns tuple[tuple[ArrayArrayCodec, ...], ArrayBytesCodec, tuple[BytesBytesCodec, ...]] - compressors=[_num_codec_config_to_configurable(self.compressor)], - filters=self.filters, - dtype=self.dtype, - serializer="auto", + return convert_to_codec_pipeline( + compressor=self.compressor, filters=self.filters, dtype=self.dtype ) - return cast(tuple["ZarrCodec", ...], (*codecs[0], codecs[1], *codecs[2])) def serializer(self) -> Any: """ @@ -235,3 +218,39 @@ def zarray_to_v3metadata(zarray: ZArray) -> ArrayV3Metadata: dimension_names=None, storage_transformers=None, ) + + +def convert_to_codec_pipeline( + compressor: "CompressorsLike", + dtype: np.dtype[Any], + filters: "FiltersLike" = None, + serializer: "SerializerLike" = "auto", +) -> tuple[ZarrCodec, ...]: + """ + Convert compressor, filters, serializer, and dtype to a pipeline of ZarrCodecs. + + Parameters + ---------- + compressor : Any + The compressor configuration. + filters : Any + The filters configuration. + dtype : Any + The data type. + serializer : str, optional + The serializer to use, by default "auto". + + Returns + ------- + Tuple[ZarrCodec, ...] + A tuple of ZarrCodecs. + """ + from zarr.core.array import _parse_chunk_encoding_v3 + + codecs = _parse_chunk_encoding_v3( + compressors=[_num_codec_config_to_configurable(compressor)], + filters=filters, + dtype=dtype, + serializer=serializer, + ) + return cast(tuple[ZarrCodec, ...], (*codecs[0], codecs[1], *codecs[2])) From e5666ab99eaa50d9324ba354acecd68cc9507ed9 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Wed, 5 Feb 2025 12:32:19 -0800 Subject: [PATCH 03/78] Fix test_manifests/test_array#TestConcat tests --- virtualizarr/manifests/array_api.py | 12 +++--- .../tests/test_manifests/test_array.py | 39 +++++++++++++------ 2 files changed, 34 insertions(+), 17 deletions(-) diff --git a/virtualizarr/manifests/array_api.py b/virtualizarr/manifests/array_api.py index bf5ce44e..c1cc0a9a 100644 --- a/virtualizarr/manifests/array_api.py +++ b/virtualizarr/manifests/array_api.py @@ -53,6 +53,8 @@ def concatenate( The signature of this function is array API compliant, so that it can be called by `xarray.concat`. """ + from zarr.core.metadata.v3 import ArrayV3Metadata + from .array import ManifestArray if axis is None: @@ -100,12 +102,10 @@ def concatenate( lengths=concatenated_lengths, ) - # FIXME(aimee): there is probably a more elegant way to do this - new_metadata = first_arr.metadata.to_dict().copy() - new_metadata["shape"] = tuple(new_shape) - new_metadata["dimension_names"] = new_metadata.get("dimension_names", None) - new_metadata.pop("zarr_format", None) - new_metadata.pop("node_type", None) + metadata_copy = first_arr.metadata.to_dict().copy() + metadata_copy["shape"] = tuple(new_shape) + # ArrayV3Metadata.from_dict removes extra keys zarr_format and node_type + new_metadata = ArrayV3Metadata.from_dict(metadata_copy) return ManifestArray(chunkmanifest=concatenated_manifest, metadata=new_metadata) diff --git a/virtualizarr/tests/test_manifests/test_array.py b/virtualizarr/tests/test_manifests/test_array.py index e89b9242..fd463b14 100644 --- a/virtualizarr/tests/test_manifests/test_array.py +++ b/virtualizarr/tests/test_manifests/test_array.py @@ -291,12 +291,9 @@ def test_concat(self): # FAILING: TypeError: no implementation found for 'numpy.concatenate' on types that implement __array_function__: [, ] def test_concat_empty(self): - chunks_dict1 = { - "0.0.0": {"path": "/foo1.nc", "offset": 100, "length": 100}, - } - manifest1 = ChunkManifest(entries=chunks_dict1) chunks = (5, 1, 10) - shape = (5, 2, 20) + shape = (5, 1, 20) + compressor = {"id": "zlib", "level": 1} metadata = ArrayV3Metadata( shape=shape, data_type="int32", @@ -304,7 +301,7 @@ def test_concat_empty(self): chunk_key_encoding={"name": "default"}, fill_value=0, codecs=convert_to_codec_pipeline( - compressor={"id": "zlib", "level": 1}, + compressor=compressor, filters=None, dtype=np.dtype("int32"), ), @@ -312,15 +309,35 @@ def test_concat_empty(self): dimension_names=None, storage_transformers=None, ) + empty_chunks_dict = {} + empty_chunk_manifest = ChunkManifest(entries=empty_chunks_dict, shape=(1, 1, 2)) + manifest_array_with_empty_chunks = ManifestArray( + metadata=metadata, chunkmanifest=empty_chunk_manifest + ) - marr1 = ManifestArray(metadata=metadata, chunkmanifest=manifest1) + chunks_dict = { + "0.0.0": {"path": "/foo.nc", "offset": 300, "length": 100}, + "0.0.1": {"path": "/foo.nc", "offset": 400, "length": 100}, + } + manifest = ChunkManifest(entries=chunks_dict) + manifest_array = ManifestArray(metadata=metadata, chunkmanifest=manifest) # Concatenate with an empty array - concatenated = np.concatenate( - [marr1, np.empty((0, 2, 20), dtype="int32")], axis=0 + result = np.concatenate( + [manifest_array_with_empty_chunks, manifest_array], axis=1 ) - assert concatenated.shape == (5, 2, 20) - assert concatenated.dtype == np.dtype("int32") + assert result.shape == (5, 2, 20) + assert result.chunks == (5, 1, 10) + assert result.manifest.dict() == { + "0.1.0": {"path": "file:///foo.nc", "offset": 300, "length": 100}, + "0.1.1": {"path": "file:///foo.nc", "offset": 400, "length": 100}, + } + codec_dict = result.metadata.codecs[1].to_dict() + assert codec_dict["name"] == "numcodecs.zlib" + # Strange? The result.metadata.codecs[1].to_dict() adds "id": "zlib" to the configuration + # e.g. it's not present in the codec itself: Zlib(codec_name='numcodecs.zlib', codec_config={'level': 1}) + assert codec_dict["configuration"] == compressor + assert result.metadata.fill_value == metadata.fill_value class TestStack: From 5a8cc4cf74fe2180ec7655325063edf712a1b5a4 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Wed, 5 Feb 2025 13:25:13 -0800 Subject: [PATCH 04/78] Passing TestStack tests and add fixture --- virtualizarr/manifests/array_api.py | 13 +- .../tests/test_manifests/test_array.py | 151 +++++++----------- 2 files changed, 66 insertions(+), 98 deletions(-) diff --git a/virtualizarr/manifests/array_api.py b/virtualizarr/manifests/array_api.py index c1cc0a9a..596277b0 100644 --- a/virtualizarr/manifests/array_api.py +++ b/virtualizarr/manifests/array_api.py @@ -122,6 +122,8 @@ def stack( The signature of this function is array API compliant, so that it can be called by `xarray.stack`. """ + from zarr.core.metadata.v3 import ArrayV3Metadata + from .array import ManifestArray if not isinstance(axis, int): @@ -172,12 +174,13 @@ def stack( new_chunks = list(old_chunks) new_chunks.insert(axis, 1) - new_zarray = first_arr.zarray.replace( - chunks=tuple(new_chunks), - shape=tuple(new_shape), - ) + metadata_copy = first_arr.metadata.to_dict().copy() + metadata_copy["shape"] = tuple(new_shape) + metadata_copy["chunk_grid"]["configuration"]["chunk_shape"] = tuple(new_chunks) + # ArrayV3Metadata.from_dict removes extra keys zarr_format and node_type + new_metadata = ArrayV3Metadata.from_dict(metadata_copy) - return ManifestArray(chunkmanifest=stacked_manifest, zarray=new_zarray) + return ManifestArray(chunkmanifest=stacked_manifest, metadata=new_metadata) @implements(np.expand_dims) diff --git a/virtualizarr/tests/test_manifests/test_array.py b/virtualizarr/tests/test_manifests/test_array.py index fd463b14..b804e7ba 100644 --- a/virtualizarr/tests/test_manifests/test_array.py +++ b/virtualizarr/tests/test_manifests/test_array.py @@ -7,25 +7,19 @@ from virtualizarr.zarr import ZArray, convert_to_codec_pipeline -class TestManifestArray: - def test_create_manifestarray(self): - chunks_dict = { - "0.0.0": {"path": "s3://bucket/foo.nc", "offset": 100, "length": 100}, - "0.0.1": {"path": "s3://bucket/foo.nc", "offset": 200, "length": 100}, - "0.1.0": {"path": "s3://bucket/foo.nc", "offset": 300, "length": 100}, - "0.1.1": {"path": "s3://bucket/foo.nc", "offset": 400, "length": 100}, - } - manifest = ChunkManifest(entries=chunks_dict) - chunks = (5, 1, 10) - shape = (5, 2, 20) - metadata = ArrayV3Metadata( +@pytest.fixture +def array_v3_metadata(): + def _create_metadata( + shape: tuple, chunks: tuple, compressor: dict = {"id": "zlib", "level": 1} + ): + return ArrayV3Metadata( shape=shape, data_type="int32", chunk_grid={"name": "regular", "configuration": {"chunk_shape": chunks}}, chunk_key_encoding={"name": "default"}, fill_value=0, codecs=convert_to_codec_pipeline( - compressor={"id": "zlib", "level": 1}, + compressor=compressor, filters=None, dtype=np.dtype("int32"), ), @@ -34,6 +28,22 @@ def test_create_manifestarray(self): storage_transformers=None, ) + return _create_metadata + + +class TestManifestArray: + def test_create_manifestarray(self, array_v3_metadata): + chunks_dict = { + "0.0.0": {"path": "s3://bucket/foo.nc", "offset": 100, "length": 100}, + "0.0.1": {"path": "s3://bucket/foo.nc", "offset": 200, "length": 100}, + "0.1.0": {"path": "s3://bucket/foo.nc", "offset": 300, "length": 100}, + "0.1.1": {"path": "s3://bucket/foo.nc", "offset": 400, "length": 100}, + } + manifest = ChunkManifest(entries=chunks_dict) + chunks = (5, 1, 10) + shape = (5, 2, 20) + metadata = array_v3_metadata(shape=shape, chunks=chunks) + marr = ManifestArray(metadata=metadata, chunkmanifest=manifest) assert marr.chunks == chunks assert marr.dtype == np.dtype("int32") @@ -78,7 +88,7 @@ def test_create_manifestarray_dict_v3_metadata(self): class TestEquals: - def test_equals(self): + def test_equals(self, array_v3_metadata): chunks_dict = { "0.0.0": {"path": "s3://bucket/foo.nc", "offset": 100, "length": 100}, "0.0.1": {"path": "s3://bucket/foo.nc", "offset": 200, "length": 100}, @@ -88,51 +98,35 @@ def test_equals(self): manifest = ChunkManifest(entries=chunks_dict) chunks = (5, 1, 10) shape = (5, 2, 20) - zarray = ZArray( - chunks=chunks, - compressor={"id": "zlib", "level": 1}, - dtype=np.dtype("int32"), - fill_value=0.0, - filters=None, - order="C", - shape=shape, - zarr_format=2, - ) + metadata = array_v3_metadata(shape=shape, chunks=chunks) - marr1 = ManifestArray(zarray=zarray, chunkmanifest=manifest) - marr2 = ManifestArray(zarray=zarray, chunkmanifest=manifest) + marr1 = ManifestArray(zarray=metadata, chunkmanifest=manifest) + marr2 = ManifestArray(zarray=metadata, chunkmanifest=manifest) result = marr1 == marr2 assert isinstance(result, np.ndarray) assert result.shape == shape assert result.dtype == np.dtype(bool) assert result.all() - def test_not_equal_chunk_entries(self): + def test_not_equal_chunk_entries(self, array_v3_metadata): # both manifest arrays in this example have the same zarray properties - zarray = ZArray( - chunks=(5, 1, 10), - compressor={"id": "zlib", "level": 1}, - dtype=np.dtype("int32"), - fill_value=0.0, - filters=None, - order="C", - shape=(5, 1, 20), - zarr_format=2, - ) + chunks = (5, 1, 10) + shape = (5, 2, 20) + metadata = array_v3_metadata(shape=shape, chunks=chunks) chunks_dict1 = { "0.0.0": {"path": "/oo.nc", "offset": 100, "length": 100}, "0.0.1": {"path": "/oo.nc", "offset": 200, "length": 100}, } manifest1 = ChunkManifest(entries=chunks_dict1) - marr1 = ManifestArray(zarray=zarray, chunkmanifest=manifest1) + marr1 = ManifestArray(metadata=metadata, chunkmanifest=manifest1) chunks_dict2 = { "0.0.0": {"path": "/oo.nc", "offset": 300, "length": 100}, "0.0.1": {"path": "/oo.nc", "offset": 400, "length": 100}, } manifest2 = ChunkManifest(entries=chunks_dict2) - marr2 = ManifestArray(zarray=zarray, chunkmanifest=manifest2) + marr2 = ManifestArray(metadata=metadata, chunkmanifest=manifest2) assert not (marr1 == marr2).all() @pytest.mark.skip(reason="Not Implemented") @@ -253,7 +247,7 @@ def test_broadcast_empty(self, shape, chunks, grid_shape, target_shape): # TODO we really need some kind of fixtures to generate useful example data # The hard part is having an alternative way to get to the expected result of concatenation class TestConcat: - def test_concat(self): + def test_concat(self, array_v3_metadata): # both manifest arrays in this example have the same metadata properties chunks_dict1 = { "0.0.0": {"path": "/foo1.nc", "offset": 100, "length": 100}, @@ -265,21 +259,7 @@ def test_concat(self): manifest2 = ChunkManifest(entries=chunks_dict2) chunks = (5, 1, 10) shape = (5, 2, 20) - metadata = ArrayV3Metadata( - shape=shape, - data_type="int32", - chunk_grid={"name": "regular", "configuration": {"chunk_shape": chunks}}, - chunk_key_encoding={"name": "default"}, - fill_value=0, - codecs=convert_to_codec_pipeline( - compressor={"id": "zlib", "level": 1}, - filters=None, - dtype=np.dtype("int32"), - ), - attributes={}, - dimension_names=None, - storage_transformers=None, - ) + metadata = array_v3_metadata(shape=shape, chunks=chunks) marr1 = ManifestArray(metadata=metadata, chunkmanifest=manifest1) marr2 = ManifestArray(metadata=metadata, chunkmanifest=manifest2) @@ -341,32 +321,25 @@ def test_concat_empty(self): class TestStack: - def test_stack(self): + def test_stack(self, array_v3_metadata): # both manifest arrays in this example have the same zarray properties - zarray = ZArray( - chunks=(5, 10), - compressor={"id": "zlib", "level": 1}, - dtype=np.dtype("int32"), - fill_value=0.0, - filters=None, - order="C", - shape=(5, 20), - zarr_format=2, - ) - + chunks = (5, 10) + shape = (5, 20) + compressor = {"id": "zlib", "level": 1} + metadata = array_v3_metadata(shape=shape, chunks=chunks, compressor=compressor) chunks_dict1 = { "0.0": {"path": "/foo.nc", "offset": 100, "length": 100}, "0.1": {"path": "/foo.nc", "offset": 200, "length": 100}, } manifest1 = ChunkManifest(entries=chunks_dict1) - marr1 = ManifestArray(zarray=zarray, chunkmanifest=manifest1) + marr1 = ManifestArray(metadata=metadata, chunkmanifest=manifest1) chunks_dict2 = { "0.0": {"path": "/foo.nc", "offset": 300, "length": 100}, "0.1": {"path": "/foo.nc", "offset": 400, "length": 100}, } manifest2 = ChunkManifest(entries=chunks_dict2) - marr2 = ManifestArray(zarray=zarray, chunkmanifest=manifest2) + marr2 = ManifestArray(metadata=metadata, chunkmanifest=manifest2) result = np.stack([marr1, marr2], axis=1) @@ -378,35 +351,28 @@ def test_stack(self): "0.1.0": {"path": "file:///foo.nc", "offset": 300, "length": 100}, "0.1.1": {"path": "file:///foo.nc", "offset": 400, "length": 100}, } - assert result.zarray.compressor == zarray.compressor - assert result.zarray.filters == zarray.filters - assert result.zarray.fill_value == zarray.fill_value - assert result.zarray.order == zarray.order - assert result.zarray.zarr_format == zarray.zarr_format + codec_dict = result.metadata.codecs[1].to_dict() + assert codec_dict["name"] == "numcodecs.zlib" + assert codec_dict["configuration"] == compressor + assert result.metadata.fill_value == metadata.fill_value - def test_stack_empty(self): - # both manifest arrays in this example have the same zarray properties - zarray = ZArray( - chunks=(5, 10), - compressor={"id": "zlib", "level": 1}, - dtype=np.dtype("int32"), - fill_value=0.0, - filters=None, - order="C", - shape=(5, 20), - zarr_format=2, - ) + def test_stack_empty(self, array_v3_metadata): + # both manifest arrays in this example have the same metadata properties + chunks = (5, 10) + shape = (5, 20) + compressor = {"id": "zlib", "level": 1} + metadata = array_v3_metadata(shape=shape, chunks=chunks, compressor=compressor) chunks_dict1 = {} manifest1 = ChunkManifest(entries=chunks_dict1, shape=(1, 2)) - marr1 = ManifestArray(zarray=zarray, chunkmanifest=manifest1) + marr1 = ManifestArray(metadata=metadata, chunkmanifest=manifest1) chunks_dict2 = { "0.0": {"path": "/foo.nc", "offset": 300, "length": 100}, "0.1": {"path": "/foo.nc", "offset": 400, "length": 100}, } manifest2 = ChunkManifest(entries=chunks_dict2) - marr2 = ManifestArray(zarray=zarray, chunkmanifest=manifest2) + marr2 = ManifestArray(metadata=metadata, chunkmanifest=manifest2) result = np.stack([marr1, marr2], axis=1) @@ -416,11 +382,10 @@ def test_stack_empty(self): "0.1.0": {"path": "file:///foo.nc", "offset": 300, "length": 100}, "0.1.1": {"path": "file:///foo.nc", "offset": 400, "length": 100}, } - assert result.zarray.compressor == zarray.compressor - assert result.zarray.filters == zarray.filters - assert result.zarray.fill_value == zarray.fill_value - assert result.zarray.order == zarray.order - assert result.zarray.zarr_format == zarray.zarr_format + codec_dict = result.metadata.codecs[1].to_dict() + assert codec_dict["name"] == "numcodecs.zlib" + assert codec_dict["configuration"] == compressor + assert result.metadata.fill_value == metadata.fill_value def test_refuse_combine(): From 4c0b616e68f661caec90c9835d2e74a26965e520 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Wed, 5 Feb 2025 17:27:35 -0800 Subject: [PATCH 05/78] All test_manifests/test_array tests passing --- virtualizarr/manifests/array_api.py | 12 +- virtualizarr/tests/__init__.py | 26 ++-- .../tests/test_manifests/test_array.py | 112 +++++++++--------- 3 files changed, 80 insertions(+), 70 deletions(-) diff --git a/virtualizarr/manifests/array_api.py b/virtualizarr/manifests/array_api.py index 596277b0..ace9136f 100644 --- a/virtualizarr/manifests/array_api.py +++ b/virtualizarr/manifests/array_api.py @@ -195,6 +195,7 @@ def broadcast_to(x: "ManifestArray", /, shape: tuple[int, ...]) -> "ManifestArra """ Broadcasts a ManifestArray to a specified shape, by either adjusting chunk keys or copying chunk manifest entries. """ + from zarr.core.metadata.v3 import ArrayV3Metadata from .array import ManifestArray @@ -241,12 +242,13 @@ def broadcast_to(x: "ManifestArray", /, shape: tuple[int, ...]) -> "ManifestArra lengths=broadcasted_lengths, ) - new_zarray = x.zarray.replace( - chunks=new_chunk_shape, - shape=new_shape, - ) + metadata_copy = x.metadata.to_dict().copy() + metadata_copy["shape"] = tuple(new_shape) + metadata_copy["chunk_grid"]["configuration"]["chunk_shape"] = tuple(new_chunk_shape) + # ArrayV3Metadata.from_dict removes extra keys zarr_format and node_type + new_metadata = ArrayV3Metadata.from_dict(metadata_copy) - return ManifestArray(chunkmanifest=broadcasted_manifest, zarray=new_zarray) + return ManifestArray(chunkmanifest=broadcasted_manifest, metadata=new_metadata) def _prepend_singleton_dimensions(shape: tuple[int, ...], ndim: int) -> tuple[int, ...]: diff --git a/virtualizarr/tests/__init__.py b/virtualizarr/tests/__init__.py index 258a9112..41cfb68d 100644 --- a/virtualizarr/tests/__init__.py +++ b/virtualizarr/tests/__init__.py @@ -9,7 +9,7 @@ from virtualizarr.manifests.manifest import join from virtualizarr.readers import HDF5VirtualBackend from virtualizarr.readers.hdf import HDFVirtualBackend -from virtualizarr.zarr import ZArray, ceildiv +from virtualizarr.zarr import ceildiv, convert_to_codec_pipeline requires_network = pytest.mark.network @@ -60,16 +60,22 @@ def create_manifestarray( The manifest is populated with a (somewhat) unique path, offset, and length for each key. """ - - zarray = ZArray( - chunks=chunks, - compressor={"id": "blosc", "clevel": 5, "cname": "lz4", "shuffle": 1}, - dtype=np.dtype("float32"), + from zarr.core.metadata.v3 import ArrayV3Metadata + + metadata = ArrayV3Metadata( + chunk_grid={"name": "regular", "configuration": {"chunk_shape": chunks}}, + chunk_key_encoding={"name": "default"}, + codecs=convert_to_codec_pipeline( + compressor={"id": "blosc", "clevel": 5, "cname": "lz4", "shuffle": 1}, + filters=None, + dtype=np.dtype("float32"), + ), + data_type=np.dtype("float32"), fill_value=0.0, - filters=None, - order="C", shape=shape, - zarr_format=2, + attributes={}, + dimension_names=None, + # storage_transformers=None, ) chunk_grid_shape = tuple( @@ -88,7 +94,7 @@ def create_manifestarray( chunkmanifest = ChunkManifest(entries=d) - return ManifestArray(chunkmanifest=chunkmanifest, zarray=zarray) + return ManifestArray(chunkmanifest=chunkmanifest, metadata=metadata) def entry_from_chunk_key(ind: tuple[int, ...]) -> dict[str, str | int]: diff --git a/virtualizarr/tests/test_manifests/test_array.py b/virtualizarr/tests/test_manifests/test_array.py index b804e7ba..060b84a5 100644 --- a/virtualizarr/tests/test_manifests/test_array.py +++ b/virtualizarr/tests/test_manifests/test_array.py @@ -4,7 +4,7 @@ from virtualizarr.manifests import ChunkManifest, ManifestArray from virtualizarr.tests import create_manifestarray -from virtualizarr.zarr import ZArray, convert_to_codec_pipeline +from virtualizarr.zarr import convert_to_codec_pipeline @pytest.fixture @@ -31,6 +31,34 @@ def _create_metadata( return _create_metadata +@pytest.fixture +def array_v3_metadata_dict(): + def _create_metadata_dict( + shape: tuple, + chunks: tuple, + codecs: list[dict] = [ + {"configuration": {"endian": "little"}, "name": "bytes"}, + { + "name": "numcodecs.zlib", + "configuration": {"level": 1}, + }, + ], + ): + return { + "shape": shape, + "data_type": "int32", + "chunk_grid": {"name": "regular", "configuration": {"chunk_shape": chunks}}, + "chunk_key_encoding": {"name": "default"}, + "fill_value": 0, + "codecs": codecs, + "attributes": {}, + "dimension_names": None, + "storage_transformers": None, + } + + return _create_metadata_dict + + class TestManifestArray: def test_create_manifestarray(self, array_v3_metadata): chunks_dict = { @@ -51,7 +79,7 @@ def test_create_manifestarray(self, array_v3_metadata): assert marr.size == 5 * 2 * 20 assert marr.ndim == 3 - def test_create_manifestarray_dict_v3_metadata(self): + def test_create_manifestarray_dict_v3_metadata(self, array_v3_metadata_dict): chunks_dict = { "0.0.0": {"path": "s3://bucket/foo.nc", "offset": 100, "length": 100}, "0.0.1": {"path": "s3://bucket/foo.nc", "offset": 200, "length": 100}, @@ -61,23 +89,7 @@ def test_create_manifestarray_dict_v3_metadata(self): manifest = ChunkManifest(entries=chunks_dict) chunks = (5, 1, 10) shape = (5, 2, 20) - metadata_dict = { - "shape": shape, - "data_type": "int32", - "chunk_grid": {"name": "regular", "configuration": {"chunk_shape": chunks}}, - "chunk_key_encoding": {"name": "default"}, - "fill_value": 0, - "codecs": [ - {"configuration": {"endian": "little"}, "name": "bytes"}, - { - "name": "numcodecs.zlib", - "configuration": {"level": 1}, - }, - ], - "attributes": {}, - "dimension_names": None, - "storage_transformers": None, - } + metadata_dict = array_v3_metadata_dict(shape=shape, chunks=chunks) marr = ManifestArray(metadata=metadata_dict, chunkmanifest=manifest) assert marr.chunks == chunks @@ -100,8 +112,8 @@ def test_equals(self, array_v3_metadata): shape = (5, 2, 20) metadata = array_v3_metadata(shape=shape, chunks=chunks) - marr1 = ManifestArray(zarray=metadata, chunkmanifest=manifest) - marr2 = ManifestArray(zarray=metadata, chunkmanifest=manifest) + marr1 = ManifestArray(metadata=metadata, chunkmanifest=manifest) + marr2 = ManifestArray(metadata=metadata, chunkmanifest=manifest) result = marr1 == marr2 assert isinstance(result, np.ndarray) assert result.shape == shape @@ -220,19 +232,12 @@ def test_broadcast_any_shape(self, shape, chunks, target_shape): ((3, 1), (2, 1), (2, 1), (2, 3, 4)), ], ) - def test_broadcast_empty(self, shape, chunks, grid_shape, target_shape): - zarray = ZArray( - chunks=chunks, - compressor={"id": "zlib", "level": 1}, - dtype=np.dtype("int32"), - fill_value=0.0, - filters=None, - order="C", - shape=shape, - zarr_format=2, - ) + def test_broadcast_empty( + self, shape, chunks, grid_shape, target_shape, array_v3_metadata + ): + metadata = array_v3_metadata(chunks=chunks, shape=shape) manifest = ChunkManifest(entries={}, shape=grid_shape) - marr = ManifestArray(zarray, manifest) + marr = ManifestArray(metadata=metadata, chunkmanifest=manifest) expanded = np.broadcast_to(marr, shape=target_shape) assert expanded.shape == target_shape @@ -388,19 +393,11 @@ def test_stack_empty(self, array_v3_metadata): assert result.metadata.fill_value == metadata.fill_value -def test_refuse_combine(): +def test_refuse_combine(array_v3_metadata_dict): # TODO test refusing to concatenate arrays that have conflicting shapes / chunk sizes - - zarray_common = { - "chunks": (5, 1, 10), - "compressor": {"id": "zlib", "level": 1}, - "dtype": np.dtype("int32"), - "fill_value": 0.0, - "filters": None, - "order": "C", - "shape": (5, 1, 10), - "zarr_format": 2, - } + chunks = (5, 1, 10) + shape = (5, 1, 20) + metadata_common = array_v3_metadata_dict(shape=shape, chunks=chunks) chunks_dict1 = { "0.0.0": {"path": "/foo.nc", "offset": 100, "length": 100}, } @@ -409,25 +406,30 @@ def test_refuse_combine(): "0.0.0": {"path": "/foo.nc", "offset": 300, "length": 100}, } chunkmanifest2 = ChunkManifest(entries=chunks_dict2) - marr1 = ManifestArray(zarray=zarray_common, chunkmanifest=chunkmanifest1) + marr1 = ManifestArray(metadata=metadata_common, chunkmanifest=chunkmanifest1) - zarray_wrong_compressor = zarray_common.copy() - zarray_wrong_compressor["compressor"] = None - marr2 = ManifestArray(zarray=zarray_wrong_compressor, chunkmanifest=chunkmanifest2) + metadata_wrong_compressor = array_v3_metadata_dict( + shape=shape, + chunks=chunks, + codecs=[{"name": "bytes", "configuration": {"endian": "little"}}], + ) + marr2 = ManifestArray( + metadata=metadata_wrong_compressor, chunkmanifest=chunkmanifest2 + ) for func in [np.concatenate, np.stack]: with pytest.raises(NotImplementedError, match="different codecs"): func([marr1, marr2], axis=0) - zarray_wrong_dtype = zarray_common.copy() - zarray_wrong_dtype["dtype"] = np.dtype("int64") - marr2 = ManifestArray(zarray=zarray_wrong_dtype, chunkmanifest=chunkmanifest2) + metadata_wrong_dtype = metadata_common.copy() + metadata_wrong_dtype["data_type"] = np.dtype("int64") + marr2 = ManifestArray(metadata=metadata_wrong_dtype, chunkmanifest=chunkmanifest2) for func in [np.concatenate, np.stack]: with pytest.raises(ValueError, match="inconsistent dtypes"): func([marr1, marr2], axis=0) - zarray_wrong_dtype = zarray_common.copy() - zarray_wrong_dtype["dtype"] = np.dtype("int64") - marr2 = ManifestArray(zarray=zarray_wrong_dtype, chunkmanifest=chunkmanifest2) + metadata_wrong_dtype = metadata_common.copy() + metadata_wrong_dtype["data_type"] = np.dtype("int64") + marr2 = ManifestArray(metadata=metadata_wrong_dtype, chunkmanifest=chunkmanifest2) for func in [np.concatenate, np.stack]: with pytest.raises(ValueError, match="inconsistent dtypes"): func([marr1, marr2], axis=0) From ac2f787b64475bee157f1f025bac4612746e8683 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Wed, 5 Feb 2025 17:55:30 -0800 Subject: [PATCH 06/78] Compressors should be list --- virtualizarr/tests/__init__.py | 2 +- .../tests/test_manifests/test_array.py | 32 ++++++++----------- virtualizarr/tests/test_zarr.py | 2 +- virtualizarr/zarr.py | 10 +++--- 4 files changed, 21 insertions(+), 25 deletions(-) diff --git a/virtualizarr/tests/__init__.py b/virtualizarr/tests/__init__.py index 41cfb68d..72335ff1 100644 --- a/virtualizarr/tests/__init__.py +++ b/virtualizarr/tests/__init__.py @@ -66,7 +66,7 @@ def create_manifestarray( chunk_grid={"name": "regular", "configuration": {"chunk_shape": chunks}}, chunk_key_encoding={"name": "default"}, codecs=convert_to_codec_pipeline( - compressor={"id": "blosc", "clevel": 5, "cname": "lz4", "shuffle": 1}, + compressors=[{"id": "blosc", "clevel": 5, "cname": "lz4", "shuffle": 1}], filters=None, dtype=np.dtype("float32"), ), diff --git a/virtualizarr/tests/test_manifests/test_array.py b/virtualizarr/tests/test_manifests/test_array.py index 060b84a5..30f6fba6 100644 --- a/virtualizarr/tests/test_manifests/test_array.py +++ b/virtualizarr/tests/test_manifests/test_array.py @@ -10,7 +10,9 @@ @pytest.fixture def array_v3_metadata(): def _create_metadata( - shape: tuple, chunks: tuple, compressor: dict = {"id": "zlib", "level": 1} + shape: tuple, + chunks: tuple, + compressors: list[dict] = [{"id": "zlib", "level": 1}], ): return ArrayV3Metadata( shape=shape, @@ -19,7 +21,7 @@ def _create_metadata( chunk_key_encoding={"name": "default"}, fill_value=0, codecs=convert_to_codec_pipeline( - compressor=compressor, + compressors=compressors, filters=None, dtype=np.dtype("int32"), ), @@ -275,24 +277,12 @@ def test_concat(self, array_v3_metadata): assert concatenated.dtype == np.dtype("int32") # FAILING: TypeError: no implementation found for 'numpy.concatenate' on types that implement __array_function__: [, ] - def test_concat_empty(self): + def test_concat_empty(self, array_v3_metadata): chunks = (5, 1, 10) shape = (5, 1, 20) compressor = {"id": "zlib", "level": 1} - metadata = ArrayV3Metadata( - shape=shape, - data_type="int32", - chunk_grid={"name": "regular", "configuration": {"chunk_shape": chunks}}, - chunk_key_encoding={"name": "default"}, - fill_value=0, - codecs=convert_to_codec_pipeline( - compressor=compressor, - filters=None, - dtype=np.dtype("int32"), - ), - attributes={}, - dimension_names=None, - storage_transformers=None, + metadata = array_v3_metadata( + shape=shape, chunks=chunks, compressors=[compressor] ) empty_chunks_dict = {} empty_chunk_manifest = ChunkManifest(entries=empty_chunks_dict, shape=(1, 1, 2)) @@ -331,7 +321,9 @@ def test_stack(self, array_v3_metadata): chunks = (5, 10) shape = (5, 20) compressor = {"id": "zlib", "level": 1} - metadata = array_v3_metadata(shape=shape, chunks=chunks, compressor=compressor) + metadata = array_v3_metadata( + shape=shape, chunks=chunks, compressors=[compressor] + ) chunks_dict1 = { "0.0": {"path": "/foo.nc", "offset": 100, "length": 100}, "0.1": {"path": "/foo.nc", "offset": 200, "length": 100}, @@ -366,7 +358,9 @@ def test_stack_empty(self, array_v3_metadata): chunks = (5, 10) shape = (5, 20) compressor = {"id": "zlib", "level": 1} - metadata = array_v3_metadata(shape=shape, chunks=chunks, compressor=compressor) + metadata = array_v3_metadata( + shape=shape, chunks=chunks, compressors=[compressor] + ) chunks_dict1 = {} manifest1 = ChunkManifest(entries=chunks_dict1, shape=(1, 2)) diff --git a/virtualizarr/tests/test_zarr.py b/virtualizarr/tests/test_zarr.py index 421b6937..ed3ea17f 100644 --- a/virtualizarr/tests/test_zarr.py +++ b/virtualizarr/tests/test_zarr.py @@ -21,7 +21,7 @@ def test_replace_total(): dtype=np.dtype(" tuple["ZarrCodec", ...]: Convert the compressor, filters, and dtype to a pipeline of ZarrCodecs. """ return convert_to_codec_pipeline( - compressor=self.compressor, filters=self.filters, dtype=self.dtype + compressors=[self.compressor], filters=self.filters, dtype=self.dtype ) def serializer(self) -> Any: @@ -221,7 +221,7 @@ def zarray_to_v3metadata(zarray: ZArray) -> ArrayV3Metadata: def convert_to_codec_pipeline( - compressor: "CompressorsLike", + compressors: "CompressorsLike", dtype: np.dtype[Any], filters: "FiltersLike" = None, serializer: "SerializerLike" = "auto", @@ -248,7 +248,9 @@ def convert_to_codec_pipeline( from zarr.core.array import _parse_chunk_encoding_v3 codecs = _parse_chunk_encoding_v3( - compressors=[_num_codec_config_to_configurable(compressor)], + compressors=[ + _num_codec_config_to_configurable(compressor) for compressor in compressors + ], filters=filters, dtype=dtype, serializer=serializer, From 5503c60b4c244c35f6e37074a4c20e38e395ada6 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Wed, 5 Feb 2025 18:18:50 -0800 Subject: [PATCH 07/78] Passing dmrpp tests --- virtualizarr/readers/dmrpp.py | 32 +++++++++++++------ virtualizarr/tests/test_readers/test_dmrpp.py | 6 ++-- virtualizarr/zarr.py | 8 +++-- 3 files changed, 32 insertions(+), 14 deletions(-) diff --git a/virtualizarr/readers/dmrpp.py b/virtualizarr/readers/dmrpp.py index 74e097ad..cf179f0c 100644 --- a/virtualizarr/readers/dmrpp.py +++ b/virtualizarr/readers/dmrpp.py @@ -12,7 +12,6 @@ from virtualizarr.readers.common import VirtualBackend from virtualizarr.types import ChunkKey from virtualizarr.utils import _FsspecFSFromFilepath, check_for_collisions -from virtualizarr.zarr import ZArray class DMRPPVirtualBackend(VirtualBackend): @@ -378,6 +377,10 @@ def _parse_variable(self, var_tag: ET.Element) -> Variable: ------- xr.Variable """ + from zarr.core.metadata.v3 import ArrayV3Metadata + + from virtualizarr.zarr import convert_to_codec_pipeline + # Dimension info dims: dict[str, int] = {} dimension_tags = self._find_dimension_tags(var_tag) @@ -414,16 +417,27 @@ def _parse_variable(self, var_tag: ET.Element) -> Variable: # Fill value is placed in zarr array's fill_value and variable encoding and removed from attributes encoding = {k: attrs.get(k) for k in self._ENCODING_KEYS if k in attrs} fill_value = attrs.pop("_FillValue", None) - # create ManifestArray and ZArray - zarray = ZArray( - chunks=chunks_shape, - dtype=dtype, - fill_value=fill_value, - filters=filters, - order="C", + # create ManifestArray + metadata = ArrayV3Metadata( shape=shape, + data_type=dtype, + chunk_grid={ + "name": "regular", + "configuration": {"chunk_shape": chunks_shape}, + }, + chunk_key_encoding={"name": "default"}, + fill_value=fill_value, + codecs=convert_to_codec_pipeline( + compressors=filters, + dtype=dtype, + filters=None, + serializer="auto", + ), + attributes=attrs, + dimension_names=None, + storage_transformers=None, ) - marr = ManifestArray(zarray=zarray, chunkmanifest=chunkmanifest) + marr = ManifestArray(metadata=metadata, chunkmanifest=chunkmanifest) return Variable(dims=dims.keys(), data=marr, attrs=attrs, encoding=encoding) def _parse_attribute(self, attr_tag: ET.Element) -> dict[str, Any]: diff --git a/virtualizarr/tests/test_readers/test_dmrpp.py b/virtualizarr/tests/test_readers/test_dmrpp.py index 37e45453..23baa51d 100644 --- a/virtualizarr/tests/test_readers/test_dmrpp.py +++ b/virtualizarr/tests/test_readers/test_dmrpp.py @@ -294,8 +294,10 @@ def test_parse_variable(tmp_path): assert var.dtype == "float32" assert var.dims == ("x", "y") assert var.shape == (720, 1440) - assert var.data.zarray.chunks == (360, 720) - assert var.data.zarray.fill_value == -32768 + assert var.data.metadata.to_dict()["chunk_grid"]["configuration"][ + "chunk_shape" + ] == (360, 720) + assert var.data.metadata.fill_value == -32768 assert var.encoding == { "add_offset": 298.15, "scale_factor": 0.001, diff --git a/virtualizarr/zarr.py b/virtualizarr/zarr.py index 62c15d68..13bfeb48 100644 --- a/virtualizarr/zarr.py +++ b/virtualizarr/zarr.py @@ -221,8 +221,8 @@ def zarray_to_v3metadata(zarray: ZArray) -> ArrayV3Metadata: def convert_to_codec_pipeline( - compressors: "CompressorsLike", dtype: np.dtype[Any], + compressors: "CompressorsLike" = None, filters: "FiltersLike" = None, serializer: "SerializerLike" = "auto", ) -> tuple[ZarrCodec, ...]: @@ -231,7 +231,7 @@ def convert_to_codec_pipeline( Parameters ---------- - compressor : Any + compressors : Any The compressor configuration. filters : Any The filters configuration. @@ -250,7 +250,9 @@ def convert_to_codec_pipeline( codecs = _parse_chunk_encoding_v3( compressors=[ _num_codec_config_to_configurable(compressor) for compressor in compressors - ], + ] + if compressors is not None + else None, filters=filters, dtype=dtype, serializer=serializer, From 1f36755b2c82d2404b8100a348238d00b528148b Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Wed, 5 Feb 2025 18:28:04 -0800 Subject: [PATCH 08/78] Passing test_hdf.py tests --- virtualizarr/readers/hdf/hdf.py | 31 ++++++++++++++++++++----------- 1 file changed, 20 insertions(+), 11 deletions(-) diff --git a/virtualizarr/readers/hdf/hdf.py b/virtualizarr/readers/hdf/hdf.py index 2b45cd9a..cc02fdac 100644 --- a/virtualizarr/readers/hdf/hdf.py +++ b/virtualizarr/readers/hdf/hdf.py @@ -28,7 +28,6 @@ from virtualizarr.readers.hdf.filters import cfcodec_from_dataset, codecs_from_dataset from virtualizarr.types import ChunkKey from virtualizarr.utils import _FsspecFSFromFilepath, check_for_collisions, soft_import -from virtualizarr.zarr import ZArray h5py = soft_import("h5py", "For reading hdf files", strict=False) @@ -305,21 +304,31 @@ def _dataset_to_variable( fill_value = float("nan") if isinstance(fill_value, np.generic): fill_value = fill_value.item() - filters = [codec.get_config() for codec in codecs] - zarray = ZArray( - chunks=chunks, # type: ignore - compressor=None, - dtype=dtype, - fill_value=fill_value, - filters=filters, - order="C", + compressors = [codec.get_config() for codec in codecs] + from zarr.core.metadata.v3 import ArrayV3Metadata + + from virtualizarr.zarr import convert_to_codec_pipeline + + metadata = ArrayV3Metadata( shape=dataset.shape, - zarr_format=2, + data_type=dtype, + chunk_grid={"name": "regular", "configuration": {"chunk_shape": chunks}}, + chunk_key_encoding={"name": "default"}, + fill_value=fill_value, + codecs=convert_to_codec_pipeline( + compressors=compressors, + dtype=dtype, + filters=None, + serializer="auto", + ), + attributes=attrs, + dimension_names=None, + storage_transformers=None, ) dims = HDFVirtualBackend._dataset_dims(dataset, group=group) manifest = HDFVirtualBackend._dataset_chunk_manifest(path, dataset) if manifest: - marray = ManifestArray(zarray=zarray, chunkmanifest=manifest) + marray = ManifestArray(metadata=metadata, chunkmanifest=manifest) variable = xr.Variable(data=marray, dims=dims, attrs=attrs) else: variable = xr.Variable(data=np.empty(dataset.shape), dims=dims, attrs=attrs) From 709880362c495b9acfc200ac792fe91fa487e82c Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Wed, 5 Feb 2025 19:21:58 -0800 Subject: [PATCH 09/78] Start to work on kerchunk tests --- virtualizarr/readers/hdf/hdf.py | 6 +- .../tests/test_manifests/test_array.py | 56 ------------------ virtualizarr/tests/test_readers/conftest.py | 59 +++++++++++++++++++ .../tests/test_readers/test_kerchunk.py | 14 +++-- .../tests/test_readers/test_netcdf3.py | 8 +-- virtualizarr/translators/kerchunk.py | 19 +++--- virtualizarr/zarr.py | 26 +++++--- 7 files changed, 103 insertions(+), 85 deletions(-) diff --git a/virtualizarr/readers/hdf/hdf.py b/virtualizarr/readers/hdf/hdf.py index cc02fdac..ad6d2e77 100644 --- a/virtualizarr/readers/hdf/hdf.py +++ b/virtualizarr/readers/hdf/hdf.py @@ -284,6 +284,9 @@ def _dataset_to_variable( """ # This chunk determination logic mirrors zarr-python's create # https://github.com/zarr-developers/zarr-python/blob/main/zarr/creation.py#L62-L66 + from zarr.core.metadata.v3 import ArrayV3Metadata + + from virtualizarr.zarr import convert_to_codec_pipeline chunks = dataset.chunks if dataset.chunks else dataset.shape codecs = codecs_from_dataset(dataset) @@ -305,9 +308,6 @@ def _dataset_to_variable( if isinstance(fill_value, np.generic): fill_value = fill_value.item() compressors = [codec.get_config() for codec in codecs] - from zarr.core.metadata.v3 import ArrayV3Metadata - - from virtualizarr.zarr import convert_to_codec_pipeline metadata = ArrayV3Metadata( shape=dataset.shape, diff --git a/virtualizarr/tests/test_manifests/test_array.py b/virtualizarr/tests/test_manifests/test_array.py index 30f6fba6..09bea269 100644 --- a/virtualizarr/tests/test_manifests/test_array.py +++ b/virtualizarr/tests/test_manifests/test_array.py @@ -1,64 +1,8 @@ import numpy as np import pytest -from zarr.core.metadata.v3 import ArrayV3Metadata from virtualizarr.manifests import ChunkManifest, ManifestArray from virtualizarr.tests import create_manifestarray -from virtualizarr.zarr import convert_to_codec_pipeline - - -@pytest.fixture -def array_v3_metadata(): - def _create_metadata( - shape: tuple, - chunks: tuple, - compressors: list[dict] = [{"id": "zlib", "level": 1}], - ): - return ArrayV3Metadata( - shape=shape, - data_type="int32", - chunk_grid={"name": "regular", "configuration": {"chunk_shape": chunks}}, - chunk_key_encoding={"name": "default"}, - fill_value=0, - codecs=convert_to_codec_pipeline( - compressors=compressors, - filters=None, - dtype=np.dtype("int32"), - ), - attributes={}, - dimension_names=None, - storage_transformers=None, - ) - - return _create_metadata - - -@pytest.fixture -def array_v3_metadata_dict(): - def _create_metadata_dict( - shape: tuple, - chunks: tuple, - codecs: list[dict] = [ - {"configuration": {"endian": "little"}, "name": "bytes"}, - { - "name": "numcodecs.zlib", - "configuration": {"level": 1}, - }, - ], - ): - return { - "shape": shape, - "data_type": "int32", - "chunk_grid": {"name": "regular", "configuration": {"chunk_shape": chunks}}, - "chunk_key_encoding": {"name": "default"}, - "fill_value": 0, - "codecs": codecs, - "attributes": {}, - "dimension_names": None, - "storage_transformers": None, - } - - return _create_metadata_dict class TestManifestArray: diff --git a/virtualizarr/tests/test_readers/conftest.py b/virtualizarr/tests/test_readers/conftest.py index 8d0c1997..dff78004 100644 --- a/virtualizarr/tests/test_readers/conftest.py +++ b/virtualizarr/tests/test_readers/conftest.py @@ -342,3 +342,62 @@ def non_coord_dim(tmpdir): ds = ds.drop_dims("dim3") ds.to_netcdf(filepath, engine="netcdf4") return filepath + + +from zarr.core.metadata.v3 import ArrayV3Metadata + +from virtualizarr.zarr import convert_to_codec_pipeline + + +@pytest.fixture +def array_v3_metadata(): + def _create_metadata( + shape: tuple, + chunks: tuple, + compressors: list[dict] = [{"id": "zlib", "level": 1}], + ): + return ArrayV3Metadata( + shape=shape, + data_type="int32", + chunk_grid={"name": "regular", "configuration": {"chunk_shape": chunks}}, + chunk_key_encoding={"name": "default"}, + fill_value=0, + codecs=convert_to_codec_pipeline( + compressors=compressors, + filters=None, + dtype=np.dtype("int32"), + ), + attributes={}, + dimension_names=None, + storage_transformers=None, + ) + + return _create_metadata + + +@pytest.fixture +def array_v3_metadata_dict(): + def _create_metadata_dict( + shape: tuple, + chunks: tuple, + codecs: list[dict] = [ + {"configuration": {"endian": "little"}, "name": "bytes"}, + { + "name": "numcodecs.zlib", + "configuration": {"level": 1}, + }, + ], + ): + return { + "shape": shape, + "data_type": "int32", + "chunk_grid": {"name": "regular", "configuration": {"chunk_shape": chunks}}, + "chunk_key_encoding": {"name": "default"}, + "fill_value": 0, + "codecs": codecs, + "attributes": {}, + "dimension_names": None, + "storage_transformers": None, + } + + return _create_metadata_dict diff --git a/virtualizarr/tests/test_readers/test_kerchunk.py b/virtualizarr/tests/test_readers/test_kerchunk.py index 83f7999d..665524b1 100644 --- a/virtualizarr/tests/test_readers/test_kerchunk.py +++ b/virtualizarr/tests/test_readers/test_kerchunk.py @@ -73,10 +73,11 @@ def test_dataset_from_df_refs(refs_file_factory): assert vda.chunks == (2, 3) assert vda.dtype == np.dtype("i4"), shape=(3,), chunks=(3,)) - expected_ma = ManifestArray(chunkmanifest=expected_manifest, zarray=expected_zarray) + metadata = array_v3_metadata(shape=(3,), chunks=(3,)) + expected_ma = ManifestArray(chunkmanifest=expected_manifest, metadata=metadata) expected_vds = xr.Dataset({"foo": xr.Variable(data=expected_ma, dims=["x"])}) xrt.assert_identical(vds, expected_vds) diff --git a/virtualizarr/translators/kerchunk.py b/virtualizarr/translators/kerchunk.py index d818870e..aeef5cd6 100644 --- a/virtualizarr/translators/kerchunk.py +++ b/virtualizarr/translators/kerchunk.py @@ -162,23 +162,26 @@ def variable_from_kerchunk_refs( """Create a single xarray Variable by reading specific keys of a kerchunk references dict.""" arr_refs = extract_array_refs(refs, var_name) - chunk_dict, zarray, zattrs = parse_array_refs(arr_refs) + chunk_dict, metadata, zattrs = parse_array_refs(arr_refs) # we want to remove the _ARRAY_DIMENSIONS from the final variables' .attrs dims = zattrs.pop("_ARRAY_DIMENSIONS") if chunk_dict: manifest = manifest_from_kerchunk_chunk_dict(chunk_dict, fs_root=fs_root) - varr = virtual_array_class(zarray=zarray, chunkmanifest=manifest) - elif len(zarray.shape) != 0: + varr = virtual_array_class(metadata=metadata, chunkmanifest=manifest) + elif len(metadata.shape) != 0: # empty variables don't have physical chunks, but zarray shows that the variable # is at least 1D - shape = determine_chunk_grid_shape(zarray.shape, zarray.chunks) + shape = determine_chunk_grid_shape( + metadata.shape, + metadata.to_dict()["chunk_grid"]["configuration"]["chunk_shape"], + ) manifest = ChunkManifest(entries={}, shape=shape) - varr = virtual_array_class(zarray=zarray, chunkmanifest=manifest) + varr = virtual_array_class(metadata=metadata, chunkmanifest=manifest) else: # This means we encountered a scalar variable of dimension 0, # very likely that it actually has no numeric value and its only purpose # is to communicate dataset attributes. - varr = zarray.fill_value + varr = metadata.fill_value return Variable(data=varr, dims=dims, attrs=zattrs) @@ -264,11 +267,11 @@ def extract_array_refs( def parse_array_refs( arr_refs: KerchunkArrRefs, ) -> tuple[dict, ZArray, ZAttrs]: - zarray = ZArray.from_kerchunk_refs(arr_refs.pop(".zarray")) + metadata = ZArray.from_kerchunk_refs(arr_refs.pop(".zarray")) zattrs = arr_refs.pop(".zattrs", {}) chunk_dict = arr_refs - return chunk_dict, zarray, zattrs + return chunk_dict, metadata, zattrs def fully_decode_arr_refs(d: dict) -> KerchunkArrRefs: diff --git a/virtualizarr/zarr.py b/virtualizarr/zarr.py index 13bfeb48..03b4baeb 100644 --- a/virtualizarr/zarr.py +++ b/virtualizarr/zarr.py @@ -72,7 +72,7 @@ def codec(self) -> Codec: return Codec(compressor=self.compressor, filters=self.filters) @classmethod - def from_kerchunk_refs(cls, decoded_arr_refs_zarray) -> "ZArray": + def from_kerchunk_refs(cls, decoded_arr_refs_zarray) -> "ArrayV3Metadata": # coerce type of fill_value as kerchunk can be inconsistent with this dtype = np.dtype(decoded_arr_refs_zarray["dtype"]) fill_value = decoded_arr_refs_zarray["fill_value"] @@ -86,15 +86,25 @@ def from_kerchunk_refs(cls, decoded_arr_refs_zarray) -> "ZArray": if zarr_format not in (2, 3): raise ValueError(f"Zarr format must be 2 or 3, but got {zarr_format}") - return ZArray( - chunks=tuple(decoded_arr_refs_zarray["chunks"]), - compressor=compressor, - dtype=dtype, + return ArrayV3Metadata( + chunk_grid={ + "name": "regular", + "configuration": { + "chunk_shape": tuple(decoded_arr_refs_zarray["chunks"]) + }, + }, + codecs=convert_to_codec_pipeline( + dtype=dtype, + compressors=compressor, + filters=decoded_arr_refs_zarray["filters"], + serializer="auto", + ), + data_type=dtype, fill_value=fill_value, - filters=decoded_arr_refs_zarray["filters"], - order=decoded_arr_refs_zarray["order"], shape=tuple(decoded_arr_refs_zarray["shape"]), - zarr_format=cast(ZARR_FORMAT, zarr_format), + chunk_key_encoding={"name": "default"}, + attributes={}, + dimension_names=None, ) def dict(self) -> dict[str, Any]: From ce2284c117a4050af4b4baf778142464e1a1e9e7 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Thu, 6 Feb 2025 17:10:41 -0800 Subject: [PATCH 10/78] Add method to convert array v3 metadata to v2 metadata for kerchunk (not happy about this) --- virtualizarr/readers/hdf/hdf.py | 6 +-- .../tests/test_readers/test_kerchunk.py | 6 +-- virtualizarr/writers/kerchunk.py | 11 ++--- virtualizarr/zarr.py | 42 ++++++++++++++++++- 4 files changed, 53 insertions(+), 12 deletions(-) diff --git a/virtualizarr/readers/hdf/hdf.py b/virtualizarr/readers/hdf/hdf.py index ad6d2e77..15f62875 100644 --- a/virtualizarr/readers/hdf/hdf.py +++ b/virtualizarr/readers/hdf/hdf.py @@ -307,7 +307,7 @@ def _dataset_to_variable( fill_value = float("nan") if isinstance(fill_value, np.generic): fill_value = fill_value.item() - compressors = [codec.get_config() for codec in codecs] + filters = [codec.get_config() for codec in codecs] metadata = ArrayV3Metadata( shape=dataset.shape, @@ -316,9 +316,9 @@ def _dataset_to_variable( chunk_key_encoding={"name": "default"}, fill_value=fill_value, codecs=convert_to_codec_pipeline( - compressors=compressors, + compressors=None, dtype=dtype, - filters=None, + filters=filters, serializer="auto", ), attributes=attrs, diff --git a/virtualizarr/tests/test_readers/test_kerchunk.py b/virtualizarr/tests/test_readers/test_kerchunk.py index 665524b1..621cfd5e 100644 --- a/virtualizarr/tests/test_readers/test_kerchunk.py +++ b/virtualizarr/tests/test_readers/test_kerchunk.py @@ -85,13 +85,13 @@ def test_dataset_from_df_refs(refs_file_factory): def test_dataset_from_df_refs_with_filters(refs_file_factory): - filters = [{"elementsize": 4, "id": "shuffle"}, {"id": "zlib", "level": 4}] + compressor = [{"elementsize": 4, "id": "shuffle"}, {"id": "zlib", "level": 4}] zarray = { "chunks": [2, 3], - "compressor": None, + "compressor": compressor, "dtype": " KerchunkArrRe for chunk_key, entry in marr.manifest.dict().items() } - zarray = marr.zarray.replace(zarr_format=2) - + array_v2_metadata = convert_v3_to_v2_metadata(marr.metadata) else: try: np_arr = var.to_numpy() @@ -118,7 +117,9 @@ def variable_to_kerchunk_arr_refs(var: Variable, var_name: str) -> KerchunkArrRe # TODO will this fail for a scalar? arr_refs = {join(0 for _ in np_arr.shape): inlined_data} - zarray = ZArray( + from zarr.core.metadata.v2 import ArrayV2Metadata + + array_v2_metadata = ArrayV2Metadata( chunks=np_arr.shape, shape=np_arr.shape, dtype=np_arr.dtype, @@ -126,7 +127,7 @@ def variable_to_kerchunk_arr_refs(var: Variable, var_name: str) -> KerchunkArrRe fill_value=None, ) - zarray_dict = zarray.to_kerchunk_json() + zarray_dict = to_kerchunk_json(array_v2_metadata) arr_refs[".zarray"] = zarray_dict zattrs = {**var.attrs, **var.encoding} diff --git a/virtualizarr/zarr.py b/virtualizarr/zarr.py index 03b4baeb..ed51b5bf 100644 --- a/virtualizarr/zarr.py +++ b/virtualizarr/zarr.py @@ -3,6 +3,7 @@ import numpy as np from zarr.abc.codec import Codec as ZarrCodec +from zarr.core.metadata.v2 import ArrayV2Metadata from zarr.core.metadata.v3 import ArrayV3Metadata if TYPE_CHECKING: @@ -263,8 +264,47 @@ def convert_to_codec_pipeline( ] if compressors is not None else None, - filters=filters, + filters=[_num_codec_config_to_configurable(filter) for filter in filters] + if filters is not None + else None, dtype=dtype, serializer=serializer, ) return cast(tuple[ZarrCodec, ...], (*codecs[0], codecs[1], *codecs[2])) + + +def convert_v3_to_v2_metadata(v3_metadata: ArrayV3Metadata) -> ArrayV2Metadata: + """ + Convert ArrayV3Metadata to ArrayV2Metadata. + + Parameters + ---------- + v3_metadata : ArrayV3Metadata + The metadata object in v3 format. + + Returns + ------- + ArrayV2Metadata + The metadata object in v2 format. + """ + v2_metadata = ArrayV2Metadata( + shape=v3_metadata.shape, + dtype=v3_metadata.data_type.to_numpy(), + chunks=v3_metadata.chunk_grid.chunk_shape, + fill_value=v3_metadata.fill_value, + order="C", # Assuming 'C' order as default + compressor=None, # TODO(aimee): parse compressors + filters=None, # TODO(aimee): parse filters + attributes=v3_metadata.attributes, + dimension_separator=".", # Assuming '.' as default dimension separator + ) + return v2_metadata + + +def to_kerchunk_json(v2_metadata: ArrayV2Metadata) -> str: + import ujson + + zarray_dict = v2_metadata.to_dict() + if np.isnan(zarray_dict["fill_value"]): + zarray_dict["fill_value"] = None + return ujson.dumps(zarray_dict) From c9853d5e05bc39f531ca2ab2efc7b84322c84894 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Thu, 6 Feb 2025 17:19:33 -0800 Subject: [PATCH 11/78] Fix fixtures and mark xfail netcdf3 --- conftest.py | 57 ++++++++++++++++++ virtualizarr/tests/test_readers/conftest.py | 59 ------------------- .../tests/test_readers/test_netcdf3.py | 2 + 3 files changed, 59 insertions(+), 59 deletions(-) diff --git a/conftest.py b/conftest.py index 0781c37e..d80b5553 100644 --- a/conftest.py +++ b/conftest.py @@ -6,6 +6,9 @@ import pytest import xarray as xr from xarray.core.variable import Variable +from zarr.core.metadata.v3 import ArrayV3Metadata + +from virtualizarr.zarr import convert_to_codec_pipeline def pytest_addoption(parser): @@ -150,3 +153,57 @@ def simple_netcdf4(tmp_path: Path) -> str: ds.to_netcdf(filepath) return str(filepath) + + +@pytest.fixture +def array_v3_metadata(): + def _create_metadata( + shape: tuple, + chunks: tuple, + compressors: list[dict] = [{"id": "zlib", "level": 1}], + ): + return ArrayV3Metadata( + shape=shape, + data_type="int32", + chunk_grid={"name": "regular", "configuration": {"chunk_shape": chunks}}, + chunk_key_encoding={"name": "default"}, + fill_value=0, + codecs=convert_to_codec_pipeline( + compressors=compressors, + filters=None, + dtype=np.dtype("int32"), + ), + attributes={}, + dimension_names=None, + storage_transformers=None, + ) + + return _create_metadata + + +@pytest.fixture +def array_v3_metadata_dict(): + def _create_metadata_dict( + shape: tuple, + chunks: tuple, + codecs: list[dict] = [ + {"configuration": {"endian": "little"}, "name": "bytes"}, + { + "name": "numcodecs.zlib", + "configuration": {"level": 1}, + }, + ], + ): + return { + "shape": shape, + "data_type": "int32", + "chunk_grid": {"name": "regular", "configuration": {"chunk_shape": chunks}}, + "chunk_key_encoding": {"name": "default"}, + "fill_value": 0, + "codecs": codecs, + "attributes": {}, + "dimension_names": None, + "storage_transformers": None, + } + + return _create_metadata_dict diff --git a/virtualizarr/tests/test_readers/conftest.py b/virtualizarr/tests/test_readers/conftest.py index dff78004..8d0c1997 100644 --- a/virtualizarr/tests/test_readers/conftest.py +++ b/virtualizarr/tests/test_readers/conftest.py @@ -342,62 +342,3 @@ def non_coord_dim(tmpdir): ds = ds.drop_dims("dim3") ds.to_netcdf(filepath, engine="netcdf4") return filepath - - -from zarr.core.metadata.v3 import ArrayV3Metadata - -from virtualizarr.zarr import convert_to_codec_pipeline - - -@pytest.fixture -def array_v3_metadata(): - def _create_metadata( - shape: tuple, - chunks: tuple, - compressors: list[dict] = [{"id": "zlib", "level": 1}], - ): - return ArrayV3Metadata( - shape=shape, - data_type="int32", - chunk_grid={"name": "regular", "configuration": {"chunk_shape": chunks}}, - chunk_key_encoding={"name": "default"}, - fill_value=0, - codecs=convert_to_codec_pipeline( - compressors=compressors, - filters=None, - dtype=np.dtype("int32"), - ), - attributes={}, - dimension_names=None, - storage_transformers=None, - ) - - return _create_metadata - - -@pytest.fixture -def array_v3_metadata_dict(): - def _create_metadata_dict( - shape: tuple, - chunks: tuple, - codecs: list[dict] = [ - {"configuration": {"endian": "little"}, "name": "bytes"}, - { - "name": "numcodecs.zlib", - "configuration": {"level": 1}, - }, - ], - ): - return { - "shape": shape, - "data_type": "int32", - "chunk_grid": {"name": "regular", "configuration": {"chunk_shape": chunks}}, - "chunk_key_encoding": {"name": "default"}, - "fill_value": 0, - "codecs": codecs, - "attributes": {}, - "dimension_names": None, - "storage_transformers": None, - } - - return _create_metadata_dict diff --git a/virtualizarr/tests/test_readers/test_netcdf3.py b/virtualizarr/tests/test_readers/test_netcdf3.py index 1dad17c3..208d8d8f 100644 --- a/virtualizarr/tests/test_readers/test_netcdf3.py +++ b/virtualizarr/tests/test_readers/test_netcdf3.py @@ -1,3 +1,4 @@ +import pytest import xarray as xr import xarray.testing as xrt @@ -7,6 +8,7 @@ @requires_scipy +@pytest.mark.xfail(reason="zarr-python 3.0 does not support big endian") def test_read_netcdf3(netcdf3_file, array_v3_metadata): filepath = str(netcdf3_file) vds = open_virtual_dataset(filepath) From 209dae3b1b5fa35a4c9cf4865371aed87570a8ef Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Fri, 7 Feb 2025 07:18:38 -0800 Subject: [PATCH 12/78] Test for convert_v3_to_v2_metadata --- virtualizarr/tests/test_zarr.py | 62 +++++++++++++++++++++++++++++++++ virtualizarr/zarr.py | 13 +++++-- 2 files changed, 72 insertions(+), 3 deletions(-) diff --git a/virtualizarr/tests/test_zarr.py b/virtualizarr/tests/test_zarr.py index ed3ea17f..dece5705 100644 --- a/virtualizarr/tests/test_zarr.py +++ b/virtualizarr/tests/test_zarr.py @@ -61,3 +61,65 @@ def test_zarray_to_v3metadata(): assert metadata.attributes == {} assert metadata.dimension_names is None assert metadata.storage_transformers == () + + +import pytest +from zarr.core.metadata.v2 import ArrayV2Metadata + +from virtualizarr.zarr import convert_to_codec_pipeline, convert_v3_to_v2_metadata + + +@pytest.fixture +def array_v3_metadata(): + def _create_metadata( + shape: tuple, + chunks: tuple, + compressors: list[dict] = [ + {"id": "blosc", "cname": "zstd", "clevel": 5, "shuffle": 1} + ], + filters: list[dict] = [{"id": "delta", "dtype": " ArrayV2Metadata: ArrayV2Metadata The metadata object in v2 format. """ + import numcodecs + + # there is no serializer in Zarr v2 + filters, compressor = [ + numcodecs.get_codec(codec.codec_config) + for codec in [v3_metadata.codecs[0], v3_metadata.codecs[2]] + ] v2_metadata = ArrayV2Metadata( shape=v3_metadata.shape, dtype=v3_metadata.data_type.to_numpy(), chunks=v3_metadata.chunk_grid.chunk_shape, fill_value=v3_metadata.fill_value, - order="C", # Assuming 'C' order as default - compressor=None, # TODO(aimee): parse compressors - filters=None, # TODO(aimee): parse filters + compressor=compressor, + filters=filters, + order="C", attributes=v3_metadata.attributes, dimension_separator=".", # Assuming '.' as default dimension separator ) From e7205eff75b896d1a1293ed2a4067ce4895f3fc5 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Fri, 7 Feb 2025 07:21:43 -0800 Subject: [PATCH 13/78] Deduplicate fixture for array v3 metadata --- conftest.py | 3 ++- virtualizarr/tests/test_zarr.py | 38 ++------------------------------- 2 files changed, 4 insertions(+), 37 deletions(-) diff --git a/conftest.py b/conftest.py index d80b5553..b05e94a4 100644 --- a/conftest.py +++ b/conftest.py @@ -161,6 +161,7 @@ def _create_metadata( shape: tuple, chunks: tuple, compressors: list[dict] = [{"id": "zlib", "level": 1}], + filters: list[dict] | None = None, ): return ArrayV3Metadata( shape=shape, @@ -170,7 +171,7 @@ def _create_metadata( fill_value=0, codecs=convert_to_codec_pipeline( compressors=compressors, - filters=None, + filters=filters, dtype=np.dtype("int32"), ), attributes={}, diff --git a/virtualizarr/tests/test_zarr.py b/virtualizarr/tests/test_zarr.py index dece5705..2935e9f1 100644 --- a/virtualizarr/tests/test_zarr.py +++ b/virtualizarr/tests/test_zarr.py @@ -1,7 +1,8 @@ import numpy as np +from zarr.core.metadata.v2 import ArrayV2Metadata from zarr.core.metadata.v3 import ArrayV3Metadata -from virtualizarr.zarr import ZArray, zarray_to_v3metadata +from virtualizarr.zarr import ZArray, convert_v3_to_v2_metadata, zarray_to_v3metadata def test_replace_partial(): @@ -63,41 +64,6 @@ def test_zarray_to_v3metadata(): assert metadata.storage_transformers == () -import pytest -from zarr.core.metadata.v2 import ArrayV2Metadata - -from virtualizarr.zarr import convert_to_codec_pipeline, convert_v3_to_v2_metadata - - -@pytest.fixture -def array_v3_metadata(): - def _create_metadata( - shape: tuple, - chunks: tuple, - compressors: list[dict] = [ - {"id": "blosc", "cname": "zstd", "clevel": 5, "shuffle": 1} - ], - filters: list[dict] = [{"id": "delta", "dtype": " Date: Fri, 7 Feb 2025 07:44:25 -0800 Subject: [PATCH 14/78] Parse filters and compressors from v3 metdata for v2 metadata --- virtualizarr/zarr.py | 39 +++++++++++++++++++++++++++++++++------ 1 file changed, 33 insertions(+), 6 deletions(-) diff --git a/virtualizarr/zarr.py b/virtualizarr/zarr.py index 8848ce6f..ebf9ce88 100644 --- a/virtualizarr/zarr.py +++ b/virtualizarr/zarr.py @@ -273,6 +273,31 @@ def convert_to_codec_pipeline( return cast(tuple[ZarrCodec, ...], (*codecs[0], codecs[1], *codecs[2])) +import numcodecs +from zarr.codecs.bytes import BytesCodec + + +def extract_codecs(v3_metadata): + """Extracts filters and compressor from Zarr v3 metadata.""" + codecs = v3_metadata.codecs or [] # Ensure it's a list, even if None + + if not codecs: # No codecs available + return None, None + + first, *rest = codecs # Unpack the first codec and the rest + + if isinstance(first, BytesCodec): # BytesCodec is first + filters = None + compressor = numcodecs.get_codec(rest[0].codec_config) if rest else None + else: # First codec is a filter + filters = numcodecs.get_codec(first.codec_config) + compressor = ( + numcodecs.get_codec(rest[1].codec_config) if len(rest) > 1 else None + ) + + return filters, compressor + + def convert_v3_to_v2_metadata(v3_metadata: ArrayV3Metadata) -> ArrayV2Metadata: """ Convert ArrayV3Metadata to ArrayV2Metadata. @@ -287,13 +312,9 @@ def convert_v3_to_v2_metadata(v3_metadata: ArrayV3Metadata) -> ArrayV2Metadata: ArrayV2Metadata The metadata object in v2 format. """ - import numcodecs - # there is no serializer in Zarr v2 - filters, compressor = [ - numcodecs.get_codec(codec.codec_config) - for codec in [v3_metadata.codecs[0], v3_metadata.codecs[2]] - ] + filters, compressor = extract_codecs(v3_metadata) + v2_metadata = ArrayV2Metadata( shape=v3_metadata.shape, dtype=v3_metadata.data_type.to_numpy(), @@ -312,6 +333,12 @@ def to_kerchunk_json(v2_metadata: ArrayV2Metadata) -> str: import ujson zarray_dict = v2_metadata.to_dict() + if zarray_dict["filters"]: + zarray_dict["filters"] = [ + codec.get_config() for codec in zarray_dict["filters"] + ] + if zarray_dict["compressor"]: + zarray_dict["compressor"] = zarray_dict["compressor"].get_config() if np.isnan(zarray_dict["fill_value"]): zarray_dict["fill_value"] = None return ujson.dumps(zarray_dict) From 190c20f1a37cbbc76a892053abf3d201ddd4698a Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Fri, 7 Feb 2025 12:15:38 -0800 Subject: [PATCH 15/78] Rewrite extract_codecs --- virtualizarr/zarr.py | 44 +++++++++++++++++++++++++------------------- 1 file changed, 25 insertions(+), 19 deletions(-) diff --git a/virtualizarr/zarr.py b/virtualizarr/zarr.py index ebf9ce88..b48e80bf 100644 --- a/virtualizarr/zarr.py +++ b/virtualizarr/zarr.py @@ -273,29 +273,31 @@ def convert_to_codec_pipeline( return cast(tuple[ZarrCodec, ...], (*codecs[0], codecs[1], *codecs[2])) -import numcodecs -from zarr.codecs.bytes import BytesCodec +from zarr.abc.codec import ArrayArrayCodec, ArrayBytesCodec, BytesBytesCodec + + +def identify_codec(codec): + if isinstance(codec, BytesBytesCodec): + return "compressor" + elif isinstance(codec, ArrayBytesCodec): + return "serializer" + elif isinstance(codec, ArrayArrayCodec): + return "filter" + else: + return "Unknown codec type" def extract_codecs(v3_metadata): """Extracts filters and compressor from Zarr v3 metadata.""" codecs = v3_metadata.codecs or [] # Ensure it's a list, even if None - - if not codecs: # No codecs available - return None, None - - first, *rest = codecs # Unpack the first codec and the rest - - if isinstance(first, BytesCodec): # BytesCodec is first - filters = None - compressor = numcodecs.get_codec(rest[0].codec_config) if rest else None - else: # First codec is a filter - filters = numcodecs.get_codec(first.codec_config) - compressor = ( - numcodecs.get_codec(rest[1].codec_config) if len(rest) > 1 else None - ) - - return filters, compressor + compressors, filters = [], [] + for codec in codecs: + codec_type = identify_codec(codec) + if codec_type == "compressor": + compressors.append(codec.codec_config) + elif codec_type == "filter": + filters.append(codec.codec_config) + return filters, compressors def convert_v3_to_v2_metadata(v3_metadata: ArrayV3Metadata) -> ArrayV2Metadata: @@ -312,8 +314,12 @@ def convert_v3_to_v2_metadata(v3_metadata: ArrayV3Metadata) -> ArrayV2Metadata: ArrayV2Metadata The metadata object in v2 format. """ + import warnings - filters, compressor = extract_codecs(v3_metadata) + filters, compressors = extract_codecs(v3_metadata) + compressor = compressors[0] if compressors else None + if len(compressors) > 1: + warnings.warn("Multiple compressors detected. Only the first one will be used.") v2_metadata = ArrayV2Metadata( shape=v3_metadata.shape, From 47f5ddde3fdc01a407bef5a93596f96f2c2fff51 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Fri, 7 Feb 2025 16:14:06 -0800 Subject: [PATCH 16/78] Refactor convert_to_codec_pipeline --- conftest.py | 8 +- virtualizarr/readers/dmrpp.py | 28 ++-- virtualizarr/readers/hdf/hdf.py | 16 +-- virtualizarr/tests/__init__.py | 12 +- .../tests/test_manifests/test_array.py | 24 +--- virtualizarr/tests/test_readers/test_dmrpp.py | 14 +- virtualizarr/tests/test_zarr.py | 49 ++----- virtualizarr/zarr.py | 135 ++++++++---------- 8 files changed, 124 insertions(+), 162 deletions(-) diff --git a/conftest.py b/conftest.py index b05e94a4..9aeffa4f 100644 --- a/conftest.py +++ b/conftest.py @@ -160,8 +160,9 @@ def array_v3_metadata(): def _create_metadata( shape: tuple, chunks: tuple, - compressors: list[dict] = [{"id": "zlib", "level": 1}], - filters: list[dict] | None = None, + codecs: list[dict] = [ + {"name": "numcodecs.zlib", "configuration": {"level": 1}} + ], ): return ArrayV3Metadata( shape=shape, @@ -170,8 +171,7 @@ def _create_metadata( chunk_key_encoding={"name": "default"}, fill_value=0, codecs=convert_to_codec_pipeline( - compressors=compressors, - filters=filters, + codecs=codecs, dtype=np.dtype("int32"), ), attributes={}, diff --git a/virtualizarr/readers/dmrpp.py b/virtualizarr/readers/dmrpp.py index cf179f0c..6b58dadb 100644 --- a/virtualizarr/readers/dmrpp.py +++ b/virtualizarr/readers/dmrpp.py @@ -393,7 +393,6 @@ def _parse_variable(self, var_tag: ET.Element) -> Variable: self._DAP_NP_DTYPE[var_tag.tag.removeprefix("{" + self._NS["dap"] + "}")] ) # Chunks and Filters - filters = None shape: tuple[int, ...] = tuple(dims.values()) chunks_shape = shape chunks_tag = var_tag.find("dmrpp:chunks", self._NS) @@ -409,7 +408,7 @@ def _parse_variable(self, var_tag: ET.Element) -> Variable: chunks_shape = shape chunkmanifest = self._parse_chunks(chunks_tag, chunks_shape) # Filters - filters = self._parse_filters(chunks_tag, dtype) + codecs = self._parse_filters(chunks_tag, dtype) # Attributes attrs: dict[str, Any] = {} for attr_tag in var_tag.iterfind("dap:Attribute", self._NS): @@ -428,10 +427,8 @@ def _parse_variable(self, var_tag: ET.Element) -> Variable: chunk_key_encoding={"name": "default"}, fill_value=fill_value, codecs=convert_to_codec_pipeline( - compressors=filters, + codecs=codecs, dtype=dtype, - filters=None, - serializer="auto", ), attributes=attrs, dimension_names=None, @@ -504,16 +501,23 @@ def _parse_filters( compression_types = chunks_tag.attrib["compressionType"].split(" ") for c in compression_types: if c == "shuffle": - filters.append({"id": "shuffle", "elementsize": dtype.itemsize}) + filters.append( + { + "name": "numcodecs.shuffle", + "configuration": {"elementsize": dtype.itemsize}, + } + ) elif c == "deflate": filters.append( { - "id": "zlib", - "level": int( - chunks_tag.attrib.get( - "deflateLevel", self._DEFAULT_ZLIB_VALUE - ) - ), + "name": "numcodecs.zlib", + "configuration": { + "level": int( + chunks_tag.attrib.get( + "deflateLevel", self._DEFAULT_ZLIB_VALUE + ) + ), + }, } ) return filters diff --git a/virtualizarr/readers/hdf/hdf.py b/virtualizarr/readers/hdf/hdf.py index 15f62875..e00c570e 100644 --- a/virtualizarr/readers/hdf/hdf.py +++ b/virtualizarr/readers/hdf/hdf.py @@ -286,7 +286,10 @@ def _dataset_to_variable( # https://github.com/zarr-developers/zarr-python/blob/main/zarr/creation.py#L62-L66 from zarr.core.metadata.v3 import ArrayV3Metadata - from virtualizarr.zarr import convert_to_codec_pipeline + from virtualizarr.zarr import ( + _num_codec_config_to_configurable, + convert_to_codec_pipeline, + ) chunks = dataset.chunks if dataset.chunks else dataset.shape codecs = codecs_from_dataset(dataset) @@ -307,7 +310,9 @@ def _dataset_to_variable( fill_value = float("nan") if isinstance(fill_value, np.generic): fill_value = fill_value.item() - filters = [codec.get_config() for codec in codecs] + codec_configs = [ + _num_codec_config_to_configurable(codec.get_config()) for codec in codecs + ] metadata = ArrayV3Metadata( shape=dataset.shape, @@ -315,12 +320,7 @@ def _dataset_to_variable( chunk_grid={"name": "regular", "configuration": {"chunk_shape": chunks}}, chunk_key_encoding={"name": "default"}, fill_value=fill_value, - codecs=convert_to_codec_pipeline( - compressors=None, - dtype=dtype, - filters=filters, - serializer="auto", - ), + codecs=convert_to_codec_pipeline(codecs=codec_configs, dtype=dtype), attributes=attrs, dimension_names=None, storage_transformers=None, diff --git a/virtualizarr/tests/__init__.py b/virtualizarr/tests/__init__.py index 72335ff1..78d3e16a 100644 --- a/virtualizarr/tests/__init__.py +++ b/virtualizarr/tests/__init__.py @@ -66,8 +66,16 @@ def create_manifestarray( chunk_grid={"name": "regular", "configuration": {"chunk_shape": chunks}}, chunk_key_encoding={"name": "default"}, codecs=convert_to_codec_pipeline( - compressors=[{"id": "blosc", "clevel": 5, "cname": "lz4", "shuffle": 1}], - filters=None, + codecs=[ + { + "name": "blosc", + "configuration": { + "clevel": 5, + "cname": "lz4", + "shuffle": "shuffle", + }, + } + ], dtype=np.dtype("float32"), ), data_type=np.dtype("float32"), diff --git a/virtualizarr/tests/test_manifests/test_array.py b/virtualizarr/tests/test_manifests/test_array.py index 09bea269..8f40bdb8 100644 --- a/virtualizarr/tests/test_manifests/test_array.py +++ b/virtualizarr/tests/test_manifests/test_array.py @@ -224,10 +224,8 @@ def test_concat(self, array_v3_metadata): def test_concat_empty(self, array_v3_metadata): chunks = (5, 1, 10) shape = (5, 1, 20) - compressor = {"id": "zlib", "level": 1} - metadata = array_v3_metadata( - shape=shape, chunks=chunks, compressors=[compressor] - ) + compressor = {"name": "numcodecs.zlib", "configuration": {"level": 1}} + metadata = array_v3_metadata(shape=shape, chunks=chunks, codecs=[compressor]) empty_chunks_dict = {} empty_chunk_manifest = ChunkManifest(entries=empty_chunks_dict, shape=(1, 1, 2)) manifest_array_with_empty_chunks = ManifestArray( @@ -253,9 +251,7 @@ def test_concat_empty(self, array_v3_metadata): } codec_dict = result.metadata.codecs[1].to_dict() assert codec_dict["name"] == "numcodecs.zlib" - # Strange? The result.metadata.codecs[1].to_dict() adds "id": "zlib" to the configuration - # e.g. it's not present in the codec itself: Zlib(codec_name='numcodecs.zlib', codec_config={'level': 1}) - assert codec_dict["configuration"] == compressor + assert codec_dict["configuration"] == {"id": "zlib", "level": 1} assert result.metadata.fill_value == metadata.fill_value @@ -264,10 +260,8 @@ def test_stack(self, array_v3_metadata): # both manifest arrays in this example have the same zarray properties chunks = (5, 10) shape = (5, 20) - compressor = {"id": "zlib", "level": 1} - metadata = array_v3_metadata( - shape=shape, chunks=chunks, compressors=[compressor] - ) + compressor = {"name": "numcodecs.zlib", "configuration": {"level": 1}} + metadata = array_v3_metadata(shape=shape, chunks=chunks, codecs=[compressor]) chunks_dict1 = { "0.0": {"path": "/foo.nc", "offset": 100, "length": 100}, "0.1": {"path": "/foo.nc", "offset": 200, "length": 100}, @@ -294,17 +288,14 @@ def test_stack(self, array_v3_metadata): } codec_dict = result.metadata.codecs[1].to_dict() assert codec_dict["name"] == "numcodecs.zlib" - assert codec_dict["configuration"] == compressor + assert codec_dict["configuration"] == {"id": "zlib", "level": 1} assert result.metadata.fill_value == metadata.fill_value def test_stack_empty(self, array_v3_metadata): # both manifest arrays in this example have the same metadata properties chunks = (5, 10) shape = (5, 20) - compressor = {"id": "zlib", "level": 1} - metadata = array_v3_metadata( - shape=shape, chunks=chunks, compressors=[compressor] - ) + metadata = array_v3_metadata(shape=shape, chunks=chunks) chunks_dict1 = {} manifest1 = ChunkManifest(entries=chunks_dict1, shape=(1, 2)) @@ -327,7 +318,6 @@ def test_stack_empty(self, array_v3_metadata): } codec_dict = result.metadata.codecs[1].to_dict() assert codec_dict["name"] == "numcodecs.zlib" - assert codec_dict["configuration"] == compressor assert result.metadata.fill_value == metadata.fill_value diff --git a/virtualizarr/tests/test_readers/test_dmrpp.py b/virtualizarr/tests/test_readers/test_dmrpp.py index 23baa51d..482856f0 100644 --- a/virtualizarr/tests/test_readers/test_dmrpp.py +++ b/virtualizarr/tests/test_readers/test_dmrpp.py @@ -334,14 +334,22 @@ def test_parse_attribute(tmp_path, attr_path, expected): "/data", np.dtype("float32"), [ - {"elementsize": np.dtype("float32").itemsize, "id": "shuffle"}, - {"id": "zlib", "level": 5}, + { + "configuration": {"elementsize": np.dtype("float32").itemsize}, + "name": "numcodecs.shuffle", + }, + {"name": "numcodecs.zlib", "configuration": {"level": 5}}, ], ), ( "/mask", np.dtype("float32"), - [{"elementsize": np.dtype("float32").itemsize, "id": "shuffle"}], + [ + { + "configuration": {"elementsize": np.dtype("float32").itemsize}, + "name": "numcodecs.shuffle", + } + ], ), ], ) diff --git a/virtualizarr/tests/test_zarr.py b/virtualizarr/tests/test_zarr.py index 2935e9f1..cdd9e636 100644 --- a/virtualizarr/tests/test_zarr.py +++ b/virtualizarr/tests/test_zarr.py @@ -1,8 +1,7 @@ import numpy as np from zarr.core.metadata.v2 import ArrayV2Metadata -from zarr.core.metadata.v3 import ArrayV3Metadata -from virtualizarr.zarr import ZArray, convert_v3_to_v2_metadata, zarray_to_v3metadata +from virtualizarr.zarr import ZArray, convert_v3_to_v2_metadata def test_replace_partial(): @@ -31,46 +30,18 @@ def test_replace_total(): assert result == expected -def test_zarray_to_v3metadata(): - from zarr.codecs import BytesCodec - - zarray = ZArray( - shape=(5, 20), - chunks=(5, 10), - dtype=np.dtype("int32"), - fill_value=0, - order="C", - compressor={"id": "zlib", "level": 1}, - filters=None, - zarr_format=3, - ) - - metadata = zarray_to_v3metadata(zarray) - - assert isinstance(metadata, ArrayV3Metadata) - assert metadata.shape == (5, 20) - assert metadata.data_type.value == "int32" - chunk_grid_dict = metadata.chunk_grid.to_dict() - assert chunk_grid_dict["name"] == "regular" - assert chunk_grid_dict["configuration"]["chunk_shape"] == (5, 10) - assert metadata.chunk_key_encoding.name == "default" - assert metadata.fill_value == np.int32(0) - assert type(metadata.codecs[0]) is BytesCodec - metadata_codec_dict = metadata.codecs[1].to_dict() - assert metadata_codec_dict["name"] == "numcodecs.zlib" - assert metadata_codec_dict["configuration"]["level"] == 1 - assert metadata.attributes == {} - assert metadata.dimension_names is None - assert metadata.storage_transformers == () - - def test_convert_v3_to_v2_metadata(array_v3_metadata): shape = (5, 20) chunks = (5, 10) - compressors = [{"id": "blosc", "cname": "zstd", "clevel": 5, "shuffle": 1}] - filters = [{"id": "delta", "dtype": " "ArrayV3Metadata": ): fill_value = np.nan - compressor = decoded_arr_refs_zarray["compressor"] zarr_format = int(decoded_arr_refs_zarray["zarr_format"]) if zarr_format not in (2, 3): raise ValueError(f"Zarr format must be 2 or 3, but got {zarr_format}") - + filters = ( + decoded_arr_refs_zarray.get("filters", []) or [] + ) # Ensure filters is a list + compressor = decoded_arr_refs_zarray.get("compressor") # Might be None + + # Ensure compressor is a list before unpacking + codec_configs = [*filters, *(compressor if compressor is not None else [])] + numcodec_configs = [ + _num_codec_config_to_configurable(config) for config in codec_configs + ] return ArrayV3Metadata( chunk_grid={ "name": "regular", @@ -96,9 +104,7 @@ def from_kerchunk_refs(cls, decoded_arr_refs_zarray) -> "ArrayV3Metadata": }, codecs=convert_to_codec_pipeline( dtype=dtype, - compressors=compressor, - filters=decoded_arr_refs_zarray["filters"], - serializer="auto", + codecs=numcodec_configs, ), data_type=dtype, fill_value=fill_value, @@ -160,8 +166,9 @@ def _v3_codec_pipeline(self) -> tuple["ZarrCodec", ...]: """ Convert the compressor, filters, and dtype to a pipeline of ZarrCodecs. """ + filters = self.filters or [] return convert_to_codec_pipeline( - compressors=[self.compressor], filters=self.filters, dtype=self.dtype + codecs=[*filters, self.compressor], dtype=self.dtype ) def serializer(self) -> Any: @@ -211,93 +218,59 @@ def _num_codec_config_to_configurable(num_codec: dict) -> dict: num_codec_copy = num_codec.copy() name = "numcodecs." + num_codec_copy.pop("id") + # name = num_codec_copy.pop("id") return {"name": name, "configuration": num_codec_copy} -def zarray_to_v3metadata(zarray: ZArray) -> ArrayV3Metadata: - """ - Convert a ZArray to a zarr v3 metadata object. - """ - return ArrayV3Metadata( - shape=zarray.shape, - data_type=zarray.dtype, - chunk_grid={"name": "regular", "configuration": {"chunk_shape": zarray.chunks}}, - chunk_key_encoding={"name": "default"}, - fill_value=zarray.fill_value, - codecs=zarray._v3_codec_pipeline(), - attributes={}, - dimension_names=None, - storage_transformers=None, - ) +def extract_codecs( + codecs: list[ZarrCodec] | None = [], +) -> tuple[list[dict], list[dict]]: + """Extracts filters and compressor from Zarr v3 metadata.""" + from zarr.abc.codec import ArrayArrayCodec, BytesBytesCodec + + arrayarray_codecs, bytesbytes_codecs = [], [] + for codec in codecs: + if isinstance(codec, ArrayArrayCodec): + arrayarray_codecs.append(codec) + if isinstance(codec, BytesBytesCodec): + bytesbytes_codecs.append(codec) + return arrayarray_codecs, bytesbytes_codecs def convert_to_codec_pipeline( - dtype: np.dtype[Any], - compressors: "CompressorsLike" = None, - filters: "FiltersLike" = None, - serializer: "SerializerLike" = "auto", + dtype: np.dtype, + codecs: list[dict] | None = [], ) -> tuple[ZarrCodec, ...]: """ Convert compressor, filters, serializer, and dtype to a pipeline of ZarrCodecs. Parameters ---------- - compressors : Any - The compressor configuration. - filters : Any - The filters configuration. dtype : Any The data type. - serializer : str, optional - The serializer to use, by default "auto". + codecs: list[ZarrCodec] | None Returns ------- Tuple[ZarrCodec, ...] A tuple of ZarrCodecs. """ - from zarr.core.array import _parse_chunk_encoding_v3 + from zarr.core.array import _get_default_chunk_encoding_v3 + from zarr.registry import get_codec_class - codecs = _parse_chunk_encoding_v3( - compressors=[ - _num_codec_config_to_configurable(compressor) for compressor in compressors + if codecs and len(codecs) > 0: + zarr_codecs = [ + get_codec_class(codec["name"]).from_dict(codec) for codec in codecs ] - if compressors is not None - else None, - filters=[_num_codec_config_to_configurable(filter) for filter in filters] - if filters is not None - else None, - dtype=dtype, - serializer=serializer, - ) - return cast(tuple[ZarrCodec, ...], (*codecs[0], codecs[1], *codecs[2])) - - -from zarr.abc.codec import ArrayArrayCodec, ArrayBytesCodec, BytesBytesCodec - - -def identify_codec(codec): - if isinstance(codec, BytesBytesCodec): - return "compressor" - elif isinstance(codec, ArrayBytesCodec): - return "serializer" - elif isinstance(codec, ArrayArrayCodec): - return "filter" + arrayarray_codecs, bytesbytes_codecs = extract_codecs(zarr_codecs) else: - return "Unknown codec type" + arrayarray_codecs, bytesbytes_codecs = [], [] + # FIXME: using private zarr-python function + # we can also use zarr_config.get("array.v3_default_serializer").get("numeric"), but requires rewriting a lot of this function + arraybytes_codecs = _get_default_chunk_encoding_v3(dtype)[1] + codec_pipeline = (*arrayarray_codecs, arraybytes_codecs, *bytesbytes_codecs) - -def extract_codecs(v3_metadata): - """Extracts filters and compressor from Zarr v3 metadata.""" - codecs = v3_metadata.codecs or [] # Ensure it's a list, even if None - compressors, filters = [], [] - for codec in codecs: - codec_type = identify_codec(codec) - if codec_type == "compressor": - compressors.append(codec.codec_config) - elif codec_type == "filter": - filters.append(codec.codec_config) - return filters, compressors + return codec_pipeline def convert_v3_to_v2_metadata(v3_metadata: ArrayV3Metadata) -> ArrayV2Metadata: @@ -316,18 +289,26 @@ def convert_v3_to_v2_metadata(v3_metadata: ArrayV3Metadata) -> ArrayV2Metadata: """ import warnings - filters, compressors = extract_codecs(v3_metadata) - compressor = compressors[0] if compressors else None - if len(compressors) > 1: - warnings.warn("Multiple compressors detected. Only the first one will be used.") + filters, compressors = extract_codecs(v3_metadata.codecs) + if compressors: + compressor = compressors[0] + compressor_config = compressor.codec_config + if len(compressors) > 1: + warnings.warn( + "Multiple compressors detected. Only the first one will be used." + ) + else: + compressor_config = None + + filter_configs = [filter.codec_config for filter in filters] v2_metadata = ArrayV2Metadata( shape=v3_metadata.shape, dtype=v3_metadata.data_type.to_numpy(), chunks=v3_metadata.chunk_grid.chunk_shape, fill_value=v3_metadata.fill_value, - compressor=compressor, - filters=filters, + compressor=compressor_config, + filters=filter_configs, order="C", attributes=v3_metadata.attributes, dimension_separator=".", # Assuming '.' as default dimension separator From 5d1560800aff9294e3fd80099006fca10d3240c1 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Sat, 8 Feb 2025 09:38:56 -0800 Subject: [PATCH 17/78] Fix hdf integration tests --- virtualizarr/zarr.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/virtualizarr/zarr.py b/virtualizarr/zarr.py index 145d8c0f..92042442 100644 --- a/virtualizarr/zarr.py +++ b/virtualizarr/zarr.py @@ -328,4 +328,6 @@ def to_kerchunk_json(v2_metadata: ArrayV2Metadata) -> str: zarray_dict["compressor"] = zarray_dict["compressor"].get_config() if np.isnan(zarray_dict["fill_value"]): zarray_dict["fill_value"] = None + if isinstance(zarray_dict["fill_value"], np.generic): + zarray_dict["fill_value"] = zarray_dict["fill_value"].item() return ujson.dumps(zarray_dict) From 908bc527d394b4ccb69dc3dc09012402d46cfdce Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Sat, 8 Feb 2025 10:12:28 -0800 Subject: [PATCH 18/78] Test for convert_to_codec_pipeline --- virtualizarr/tests/test_zarr.py | 49 ++++++++++++++++++++++++++++++++- 1 file changed, 48 insertions(+), 1 deletion(-) diff --git a/virtualizarr/tests/test_zarr.py b/virtualizarr/tests/test_zarr.py index cdd9e636..6c90ee86 100644 --- a/virtualizarr/tests/test_zarr.py +++ b/virtualizarr/tests/test_zarr.py @@ -1,7 +1,54 @@ import numpy as np +from zarr.codecs import BytesCodec from zarr.core.metadata.v2 import ArrayV2Metadata -from virtualizarr.zarr import ZArray, convert_v3_to_v2_metadata +from virtualizarr.zarr import ( + ZArray, + convert_to_codec_pipeline, + convert_v3_to_v2_metadata, +) + + +def test_convert_to_codec_pipeline(): + expected_default_codecs = (BytesCodec(endian="little"),) + # Test with just dtype (default codec pipeline) + dtype = np.dtype(" Date: Sat, 8 Feb 2025 10:56:12 -0800 Subject: [PATCH 19/78] Refactor get_codecs and its tests --- conftest.py | 60 ++++- virtualizarr/codecs.py | 100 ++++---- virtualizarr/tests/__init__.py | 77 ------- virtualizarr/tests/test_codecs.py | 215 ++++++------------ .../tests/test_manifests/test_array.py | 15 +- 5 files changed, 187 insertions(+), 280 deletions(-) diff --git a/conftest.py b/conftest.py index 9aeffa4f..2012283f 100644 --- a/conftest.py +++ b/conftest.py @@ -1,3 +1,4 @@ +import itertools from pathlib import Path from typing import Any, Callable, Mapping, Optional @@ -8,7 +9,9 @@ from xarray.core.variable import Variable from zarr.core.metadata.v3 import ArrayV3Metadata -from virtualizarr.zarr import convert_to_codec_pipeline +from virtualizarr.manifests import ChunkManifest, ManifestArray +from virtualizarr.manifests.manifest import join +from virtualizarr.zarr import ceildiv, convert_to_codec_pipeline def pytest_addoption(parser): @@ -155,6 +158,61 @@ def simple_netcdf4(tmp_path: Path) -> str: return str(filepath) +def offset_from_chunk_key(ind: tuple[int, ...]) -> int: + return sum(ind) * 10 + + +def length_from_chunk_key(ind: tuple[int, ...]) -> int: + return sum(ind) + 5 + + +def entry_from_chunk_key(ind: tuple[int, ...]) -> dict[str, str | int]: + """Generate a (somewhat) unique manifest entry from a given chunk key""" + entry = { + "path": f"/foo.{str(join(ind))}.nc", + "offset": offset_from_chunk_key(ind), + "length": length_from_chunk_key(ind), + } + return entry # type: ignore[return-value] + + +@pytest.fixture +def create_manifestarray(array_v3_metadata): + """ + Create an example ManifestArray with sensible defaults. + + The manifest is populated with a (somewhat) unique path, offset, and length for each key. + """ + + def _create_manifestarray( + shape: tuple | None = (5, 5), + chunks: tuple | None = (5, 5), + codecs: list[dict] | None = [ + {"name": "numcodecs.zlib", "configuration": {"level": 1}} + ], + ): + metadata = array_v3_metadata(shape, chunks, codecs) + chunk_grid_shape = tuple( + ceildiv(axis_length, chunk_length) + for axis_length, chunk_length in zip(shape, chunks) + ) + + if chunk_grid_shape == (): + d = {"0": entry_from_chunk_key((0,))} + else: + # create every possible combination of keys + all_possible_combos = itertools.product( + *[range(length) for length in chunk_grid_shape] + ) + d = {join(ind): entry_from_chunk_key(ind) for ind in all_possible_combos} + + chunkmanifest = ChunkManifest(entries=d) + + return ManifestArray(chunkmanifest=chunkmanifest, metadata=metadata) + + return _create_manifestarray + + @pytest.fixture def array_v3_metadata(): def _create_metadata( diff --git a/virtualizarr/codecs.py b/virtualizarr/codecs.py index d8eb4691..100755ab 100644 --- a/virtualizarr/codecs.py +++ b/virtualizarr/codecs.py @@ -1,6 +1,4 @@ -from typing import TYPE_CHECKING, Union - -from virtualizarr.zarr import Codec +from typing import TYPE_CHECKING, Tuple, Union if TYPE_CHECKING: from zarr import Array # type: ignore @@ -12,29 +10,39 @@ from .manifests.array import ManifestArray +CodecPipeline = Tuple[ + Union["ArrayArrayCodec", "ArrayBytesCodec", "BytesBytesCodec"], ... +] -def get_codecs( - array: Union["ManifestArray", "Array"], - normalize_to_zarr_v3: bool = False, -) -> Union[Codec, tuple["ArrayArrayCodec | ArrayBytesCodec | BytesBytesCodec", ...]]: - """ - Get the codecs for either a ManifestArray or a Zarr Array. - - Parameters: - array (Union[ManifestArray, ZarrArray]): The input array, either ManifestArray or Zarr Array. - Returns: - List[Optional[Codec]]: A list of codecs or an empty list if no codecs are found. - - Raises: - ImportError: If `zarr` is required but not installed. - ValueError: If the array type is unsupported. +def get_codecs(array: Union["ManifestArray", "Array"]) -> CodecPipeline: + """ + Get the zarr v3 codec pipeline for either a ManifestArray or a Zarr Array. + + Parameters + ---------- + array : Union[ManifestArray, Array] + The input array, either ManifestArray or Zarr Array. + + Returns + ------- + CodecPipeline + A tuple of zarr v3 codecs representing the codec pipeline. + + Raises + ------ + ImportError + If `zarr` is required but not installed. + ValueError + If the array type is unsupported or the array's metadata is not in zarr v3 format. + NotImplementedError + If zarr-python v3 is not installed. """ if _is_manifest_array(array): - return _get_manifestarray_codecs(array, normalize_to_zarr_v3) # type: ignore[arg-type] + return _get_manifestarray_codecs(array) # type: ignore[arg-type] if _is_zarr_array(array): - return _get_zarr_array_codecs(array, normalize_to_zarr_v3) # type: ignore[arg-type] + return _get_zarr_array_codecs(array) # type: ignore[arg-type] raise ValueError("Unsupported array type or zarr is not installed.") @@ -49,15 +57,13 @@ def _is_manifest_array(array: object) -> bool: return False -def _get_manifestarray_codecs( - array: "ManifestArray", - normalize_to_zarr_v3: bool = False, -) -> Union[Codec, tuple["ArrayArrayCodec | ArrayBytesCodec | BytesBytesCodec", ...]]: - """Get codecs for a ManifestArray based on its zarr_format.""" - if normalize_to_zarr_v3 or array.metadata.zarr_format == 3: - return array.metadata.codecs - else: - raise ValueError("Unsupported zarr_format for ManifestArray.") +def _get_manifestarray_codecs(array: "ManifestArray") -> CodecPipeline: + """Get zarr v3 codec pipeline for a ManifestArray.""" + if array.metadata.zarr_format != 3: + raise ValueError( + "Only zarr v3 format is supported. Please convert your array metadata to v3 format." + ) + return array.metadata.codecs def _is_zarr_array(array: object) -> bool: @@ -70,11 +76,8 @@ def _is_zarr_array(array: object) -> bool: return False -def _get_zarr_array_codecs( - array: "Array", - normalize_to_zarr_v3: bool = False, -) -> Union[Codec, tuple["ArrayArrayCodec | ArrayBytesCodec | BytesBytesCodec", ...]]: - """Get codecs for a Zarr Array based on its format.""" +def _get_zarr_array_codecs(array: "Array") -> CodecPipeline: + """Get zarr v3 codec pipeline for a Zarr Array.""" import zarr from packaging import version @@ -85,25 +88,12 @@ def _get_zarr_array_codecs( raise NotImplementedError( f"zarr-python v3 or higher is required, but version {installed_version} is installed." ) - from zarr.core.metadata import ( # type: ignore[import-untyped] - ArrayV2Metadata, - ArrayV3Metadata, - ) - # For zarr format v3 - if isinstance(array.metadata, ArrayV3Metadata): - return tuple(array.metadata.codecs) - # For zarr format v2 - elif isinstance(array.metadata, ArrayV2Metadata): - if normalize_to_zarr_v3: - # we could potentially normalize to v3 using ZArray._v3_codec_pipeline, but we don't have a use case for that. - raise NotImplementedError( - "Normalization to zarr v3 is not supported for zarr v2 array." - ) - else: - return Codec( - compressor=array.metadata.compressor, - filters=list(array.metadata.filters or ()), - ) - else: - raise ValueError("Unsupported zarr_format for Zarr Array.") + from zarr.core.metadata import ArrayV3Metadata # type: ignore[import-untyped] + + if not isinstance(array.metadata, ArrayV3Metadata): + raise ValueError( + "Only zarr v3 format arrays are supported. Please convert your array to v3 format." + ) + + return tuple(array.metadata.codecs) diff --git a/virtualizarr/tests/__init__.py b/virtualizarr/tests/__init__.py index 78d3e16a..09d36d3d 100644 --- a/virtualizarr/tests/__init__.py +++ b/virtualizarr/tests/__init__.py @@ -1,15 +1,10 @@ import importlib -import itertools -import numpy as np import pytest from packaging.version import Version -from virtualizarr.manifests import ChunkManifest, ManifestArray -from virtualizarr.manifests.manifest import join from virtualizarr.readers import HDF5VirtualBackend from virtualizarr.readers.hdf import HDFVirtualBackend -from virtualizarr.zarr import ceildiv, convert_to_codec_pipeline requires_network = pytest.mark.network @@ -44,80 +39,8 @@ def _importorskip( has_imagecodecs, requires_imagecodecs = _importorskip("imagecodecs") has_hdf5plugin, requires_hdf5plugin = _importorskip("hdf5plugin") has_zarr_python, requires_zarr_python = _importorskip("zarr") -has_zarr_python_v3, requires_zarr_python_v3 = _importorskip("zarr", "3.0.0b") parametrize_over_hdf_backends = pytest.mark.parametrize( "hdf_backend", [HDF5VirtualBackend, HDFVirtualBackend] if has_kerchunk else [HDFVirtualBackend], ) - - -def create_manifestarray( - shape: tuple[int, ...], chunks: tuple[int, ...] -) -> ManifestArray: - """ - Create an example ManifestArray with sensible defaults. - - The manifest is populated with a (somewhat) unique path, offset, and length for each key. - """ - from zarr.core.metadata.v3 import ArrayV3Metadata - - metadata = ArrayV3Metadata( - chunk_grid={"name": "regular", "configuration": {"chunk_shape": chunks}}, - chunk_key_encoding={"name": "default"}, - codecs=convert_to_codec_pipeline( - codecs=[ - { - "name": "blosc", - "configuration": { - "clevel": 5, - "cname": "lz4", - "shuffle": "shuffle", - }, - } - ], - dtype=np.dtype("float32"), - ), - data_type=np.dtype("float32"), - fill_value=0.0, - shape=shape, - attributes={}, - dimension_names=None, - # storage_transformers=None, - ) - - chunk_grid_shape = tuple( - ceildiv(axis_length, chunk_length) - for axis_length, chunk_length in zip(shape, chunks) - ) - - if chunk_grid_shape == (): - d = {"0": entry_from_chunk_key((0,))} - else: - # create every possible combination of keys - all_possible_combos = itertools.product( - *[range(length) for length in chunk_grid_shape] - ) - d = {join(ind): entry_from_chunk_key(ind) for ind in all_possible_combos} - - chunkmanifest = ChunkManifest(entries=d) - - return ManifestArray(chunkmanifest=chunkmanifest, metadata=metadata) - - -def entry_from_chunk_key(ind: tuple[int, ...]) -> dict[str, str | int]: - """Generate a (somewhat) unique manifest entry from a given chunk key""" - entry = { - "path": f"/foo.{str(join(ind))}.nc", - "offset": offset_from_chunk_key(ind), - "length": length_from_chunk_key(ind), - } - return entry # type: ignore[return-value] - - -def offset_from_chunk_key(ind: tuple[int, ...]) -> int: - return sum(ind) * 10 - - -def length_from_chunk_key(ind: tuple[int, ...]) -> int: - return sum(ind) + 5 diff --git a/virtualizarr/tests/test_codecs.py b/virtualizarr/tests/test_codecs.py index 23cb494e..ef2715ed 100644 --- a/virtualizarr/tests/test_codecs.py +++ b/virtualizarr/tests/test_codecs.py @@ -1,161 +1,94 @@ -from unittest.mock import patch - import numpy as np import pytest -from numcodecs import Blosc, Delta +from zarr.codecs import BytesCodec +from zarr.registry import get_codec_class -from virtualizarr import ChunkManifest, ManifestArray from virtualizarr.codecs import get_codecs -from virtualizarr.tests import ( - requires_zarr_python, - requires_zarr_python_v3, -) -from virtualizarr.zarr import Codec - - -class TestCodecs: - def create_manifest_array(self, compressor=None, filters=None, zarr_format=2): - return ManifestArray( - chunkmanifest=ChunkManifest( - entries={"0.0": dict(path="/test.nc", offset=6144, length=48)} - ), - zarray=dict( - shape=(2, 3), - dtype=np.dtype(" Date: Sun, 9 Feb 2025 11:10:37 -0800 Subject: [PATCH 20/78] Fix most integration tests and writer tests --- conftest.py | 117 +++++++++- virtualizarr/tests/test_integration.py | 30 +-- .../tests/test_manifests/test_array.py | 6 +- virtualizarr/tests/test_writers/conftest.py | 11 +- .../tests/test_writers/test_icechunk.py | 210 +++++------------- .../tests/test_writers/test_kerchunk.py | 52 ++--- virtualizarr/writers/icechunk.py | 22 +- virtualizarr/writers/kerchunk.py | 2 +- virtualizarr/zarr.py | 22 +- 9 files changed, 230 insertions(+), 242 deletions(-) diff --git a/conftest.py b/conftest.py index 2012283f..0c3989a9 100644 --- a/conftest.py +++ b/conftest.py @@ -1,4 +1,5 @@ import itertools +from itertools import product from pathlib import Path from typing import Any, Callable, Mapping, Optional @@ -191,7 +192,7 @@ def _create_manifestarray( {"name": "numcodecs.zlib", "configuration": {"level": 1}} ], ): - metadata = array_v3_metadata(shape, chunks, codecs) + metadata = array_v3_metadata(shape=shape, chunks=chunks, codecs=codecs) chunk_grid_shape = tuple( ceildiv(axis_length, chunk_length) for axis_length, chunk_length in zip(shape, chunks) @@ -218,16 +219,16 @@ def array_v3_metadata(): def _create_metadata( shape: tuple, chunks: tuple, - codecs: list[dict] = [ - {"name": "numcodecs.zlib", "configuration": {"level": 1}} - ], + data_type: str = "int32", + codecs: list[dict] | None = None, + fill_value: int = None, ): return ArrayV3Metadata( shape=shape, - data_type="int32", + data_type=data_type, chunk_grid={"name": "regular", "configuration": {"chunk_shape": chunks}}, chunk_key_encoding={"name": "default"}, - fill_value=0, + fill_value=fill_value, codecs=convert_to_codec_pipeline( codecs=codecs, dtype=np.dtype("int32"), @@ -266,3 +267,107 @@ def _create_metadata_dict( } return _create_metadata_dict + + +def generate_chunk_manifest( + netcdf4_file: str, + shape: tuple[int, ...], + chunks: tuple[int, ...], + offset=6144, + length=48, +) -> ChunkManifest: + chunk_dict = {} + num_chunks = [shape[i] // chunks[i] for i in range(len(shape))] + offset = offset + + # Generate all possible chunk indices using Cartesian product + for chunk_indices in product(*[range(n) for n in num_chunks]): + chunk_index = ".".join(map(str, chunk_indices)) + chunk_dict[chunk_index] = { + "path": netcdf4_file, + "offset": offset, + "length": length, + } + offset += length # Increase offset for each chunk + + return ChunkManifest(chunk_dict) + + +@pytest.fixture +def gen_virtual_variable(array_v3_metadata: Callable) -> Callable: + def _gen_virtual_variable( + file_uri: str, + shape: tuple[int, ...] = (3, 4), + chunk_shape: tuple[int, ...] = (3, 4), + dtype: np.dtype = np.dtype("int32"), + codecs: Optional[list[dict[Any, Any]]] = None, + fill_value: Optional[str] = None, + encoding: Optional[dict] = None, + offset: int = 6144, + length: int = 48, + dims: list[str] = [], + attrs: dict[str, Any] = {}, + ) -> xr.Variable: + manifest = generate_chunk_manifest( + file_uri, + shape=shape, + chunks=chunk_shape, + offset=offset, + length=length, + ) + metadata = array_v3_metadata( + shape=shape, + chunks=chunk_shape, + codecs=codecs, + data_type=dtype, + fill_value=fill_value, + ) + ma = ManifestArray(chunkmanifest=manifest, metadata=metadata) + return xr.Variable( + data=ma, + dims=dims, + encoding=encoding, + attrs=attrs, + ) + + return _gen_virtual_variable + + +@pytest.fixture +def gen_virtual_dataset(gen_virtual_variable: Callable) -> Callable: + def _gen_virtual_dataset( + file_uri: str, + shape: tuple[int, ...] = (3, 4), + chunk_shape: tuple[int, ...] = (3, 4), + dtype: np.dtype = np.dtype("int32"), + codecs: Optional[list[dict[Any, Any]]] = None, + fill_value: Optional[str] = None, + encoding: Optional[dict] = None, + variable_name: str = "foo", + offset: int = 6144, + length: int = 48, + dims: Optional[list[str]] = None, + coords: Optional[xr.Coordinates] = None, + ) -> xr.Dataset: + with xr.open_dataset(file_uri) as ds: + var = gen_virtual_variable( + file_uri=file_uri, + shape=shape, + chunk_shape=chunk_shape, + dtype=dtype, + codecs=codecs, + fill_value=fill_value, + encoding=encoding, + offset=offset, + length=length, + dims=dims or [str(name) for name in ds.dims], + attrs=ds[variable_name].attrs, + ) + + return xr.Dataset( + {variable_name: var}, + coords=coords, + attrs=ds.attrs, + ) + + return _gen_virtual_dataset diff --git a/virtualizarr/tests/test_integration.py b/virtualizarr/tests/test_integration.py index 14cc8d3d..ed127fb2 100644 --- a/virtualizarr/tests/test_integration.py +++ b/virtualizarr/tests/test_integration.py @@ -18,10 +18,9 @@ from virtualizarr.translators.kerchunk import ( dataset_from_kerchunk_refs, ) -from virtualizarr.zarr import ZArray -def test_kerchunk_roundtrip_in_memory_no_concat(): +def test_kerchunk_roundtrip_in_memory_no_concat(array_v3_metadata): # Set up example xarray dataset chunks_dict = { "0.0": {"path": "/foo.nc", "offset": 100, "length": 100}, @@ -29,15 +28,7 @@ def test_kerchunk_roundtrip_in_memory_no_concat(): } manifest = ChunkManifest(entries=chunks_dict) marr = ManifestArray( - zarray=dict( - shape=(2, 4), - dtype=np.dtype(" Dataset: +def vds_with_manifest_arrays(array_v3_metadata) -> Dataset: arr = ManifestArray( chunkmanifest=ChunkManifest( entries={"0.0": dict(path="/test.nc", offset=6144, length=48)} ), - zarray=dict( + metadata=array_v3_metadata( shape=(2, 3), - dtype=np.dtype(" ChunkManifest: - chunk_dict = {} - num_chunks = [shape[i] // chunks[i] for i in range(len(shape))] - offset = offset - - # Generate all possible chunk indices using Cartesian product - for chunk_indices in product(*[range(n) for n in num_chunks]): - chunk_index = ".".join(map(str, chunk_indices)) - chunk_dict[chunk_index] = { - "path": netcdf4_file, - "offset": offset, - "length": length, - } - offset += length # Increase offset for each chunk - - return ChunkManifest(chunk_dict) - - -def gen_virtual_variable( - file_uri: str, - shape: tuple[int, ...] = (3, 4), - chunk_shape: tuple[int, ...] = (3, 4), - dtype: np.dtype = np.dtype("int32"), - compressor: Optional[dict] = None, - filters: Optional[list[dict[Any, Any]]] = None, - fill_value: Optional[str] = None, - encoding: Optional[dict] = None, - offset: int = 6144, - length: int = 48, - dims: list[str] = [], - zarr_format: Literal[2, 3] = 2, - attrs: dict[str, Any] = {}, -) -> xr.Variable: - manifest = generate_chunk_manifest( - file_uri, - shape=shape, - chunks=chunk_shape, - offset=offset, - length=length, - ) - zarray = ZArray( - shape=shape, - chunks=chunk_shape, - dtype=dtype, - compressor=compressor, - filters=filters, - fill_value=fill_value, - zarr_format=zarr_format, - ) - ma = ManifestArray(chunkmanifest=manifest, zarray=zarray) - return xr.Variable( - data=ma, - dims=dims, - encoding=encoding, - attrs=attrs, - ) - - -def gen_virtual_dataset( - file_uri: str, - shape: tuple[int, ...] = (3, 4), - chunk_shape: tuple[int, ...] = (3, 4), - dtype: np.dtype = np.dtype("int32"), - compressor: Optional[dict] = None, - filters: Optional[list[dict[Any, Any]]] = None, - fill_value: Optional[str] = None, - encoding: Optional[dict] = None, - variable_name: str = "foo", - offset: int = 6144, - length: int = 48, - dims: Optional[list[str]] = None, - zarr_format: Literal[2, 3] = 2, - coords: Optional[xr.Coordinates] = None, -) -> xr.Dataset: - with xr.open_dataset(file_uri) as ds: - var = gen_virtual_variable( - file_uri, - shape=shape, - chunk_shape=chunk_shape, - dtype=dtype, - compressor=compressor, - filters=filters, - fill_value=fill_value, - encoding=encoding, - offset=offset, - length=length, - dims=dims or [str(name) for name in ds.dims], - zarr_format=zarr_format, - attrs=ds[variable_name].attrs, - ) - - return xr.Dataset( - {variable_name: var}, - coords=coords, - attrs=ds.attrs, - ) - - class TestAppend: """ Tests for appending to existing icechunk store. @@ -546,7 +430,10 @@ class TestAppend: # Success cases ## When appending to a single virtual ref without encoding, it succeeds def test_append_virtual_ref_without_encoding( - self, icechunk_repo: "Repository", simple_netcdf4: str + self, + icechunk_repo: "Repository", + simple_netcdf4: str, + gen_virtual_dataset: Callable, ): import xarray.testing as xrt @@ -579,14 +466,16 @@ def test_append_virtual_ref_without_encoding( xrt.assert_identical(array, expected_array) def test_append_virtual_ref_with_encoding( - self, icechunk_repo: "Repository", netcdf4_files_factory: Callable + self, + icechunk_repo: "Repository", + netcdf4_files_factory: Callable, + gen_virtual_dataset: Callable, ): import xarray.testing as xrt scale_factor = 0.01 encoding = {"air": {"scale_factor": scale_factor}} filepath1, filepath2 = netcdf4_files_factory(encoding=encoding) - vds1, vds2 = ( gen_virtual_dataset( file_uri=filepath1, @@ -639,7 +528,11 @@ def test_append_virtual_ref_with_encoding( ## When appending to a virtual ref with encoding, it succeeds @pytest.mark.asyncio async def test_append_with_multiple_root_arrays( - self, icechunk_repo: "Repository", netcdf4_files_factory: Callable + self, + icechunk_repo: "Repository", + netcdf4_files_factory: Callable, + gen_virtual_variable: Callable, + gen_virtual_dataset: Callable, ): import xarray.testing as xrt from zarr.core.buffer import default_buffer_prototype @@ -748,12 +641,11 @@ async def test_append_with_multiple_root_arrays( xrt.assert_equal(ds, expected_ds) # When appending to a virtual ref with compression, it succeeds - @pytest.mark.parametrize("zarr_format", [2, 3]) def test_append_with_compression_succeeds( self, icechunk_repo: "Repository", netcdf4_files_factory: Callable, - zarr_format: Literal[2, 3], + gen_virtual_dataset: Callable, ): import xarray.testing as xrt @@ -772,25 +664,23 @@ def test_append_with_compression_succeeds( file_uri=file1, shape=(1460, 25, 53), chunk_shape=(1460, 25, 53), - compressor={"id": "zlib", "level": 4}, + codecs=[{"name": "numcodecs.zlib", "configuration": {"level": 4}}], dims=["time", "lat", "lon"], dtype=np.dtype("float64"), variable_name="air", offset=18043, length=3936114, - zarr_format=zarr_format, ), gen_virtual_dataset( file_uri=file2, shape=(1460, 25, 53), chunk_shape=(1460, 25, 53), - compressor={"id": "zlib", "level": 4}, + codecs=[{"name": "numcodecs.zlib", "configuration": {"level": 4}}], dims=["time", "lat", "lon"], dtype=np.dtype("float64"), variable_name="air", offset=18043, length=3938672, - zarr_format=zarr_format, ), ) @@ -816,7 +706,10 @@ def test_append_with_compression_succeeds( ## When chunk shapes are different it fails def test_append_with_different_chunking_fails( - self, icechunk_repo: "Repository", simple_netcdf4: str + self, + icechunk_repo: "Repository", + simple_netcdf4: str, + gen_virtual_dataset: Callable, ): # Generate a virtual dataset with specific chunking vds = gen_virtual_dataset(file_uri=simple_netcdf4, chunk_shape=(3, 4)) @@ -840,7 +733,10 @@ def test_append_with_different_chunking_fails( ## When encoding is different it fails def test_append_with_different_encoding_fails( - self, icechunk_repo: "Repository", simple_netcdf4: str + self, + icechunk_repo: "Repository", + simple_netcdf4: str, + gen_virtual_dataset: Callable, ): # Generate datasets with different encoding vds1 = gen_virtual_dataset( @@ -864,7 +760,10 @@ def test_append_with_different_encoding_fails( vds2.virtualize.to_icechunk(icechunk_filestore_append.store, append_dim="x") def test_dimensions_do_not_align( - self, icechunk_repo: "Repository", simple_netcdf4: str + self, + icechunk_repo: "Repository", + simple_netcdf4: str, + gen_virtual_dataset: Callable, ): # Generate datasets with different lengths on the non-append dimension (x) vds1 = gen_virtual_dataset( @@ -889,7 +788,10 @@ def test_dimensions_do_not_align( vds2.virtualize.to_icechunk(icechunk_filestore_append.store, append_dim="y") def test_append_dim_not_in_dims_raises_error( - self, icechunk_repo: "Repository", simple_netcdf4: str + self, + icechunk_repo: "Repository", + simple_netcdf4: str, + gen_virtual_dataset: Callable, ): """ Test that attempting to append with an append_dim not present in dims raises a ValueError. diff --git a/virtualizarr/tests/test_writers/test_kerchunk.py b/virtualizarr/tests/test_writers/test_kerchunk.py index 1e9b240c..34305b90 100644 --- a/virtualizarr/tests/test_writers/test_kerchunk.py +++ b/virtualizarr/tests/test_writers/test_kerchunk.py @@ -8,20 +8,18 @@ @requires_kerchunk class TestAccessor: - def test_accessor_to_kerchunk_dict(self): + def test_accessor_to_kerchunk_dict(self, array_v3_metadata): manifest = ChunkManifest( entries={"0.0": dict(path="file:///test.nc", offset=6144, length=48)} ) arr = ManifestArray( chunkmanifest=manifest, - zarray=dict( + metadata=array_v3_metadata( shape=(2, 3), - dtype=np.dtype(" KerchunkStoreRefs: prepended_with_var_name = { f"{var_name}/{key}": val for key, val in arr_refs.items() } - all_arr_refs.update(prepended_with_var_name) zattrs = ds.attrs @@ -124,6 +123,7 @@ def variable_to_kerchunk_arr_refs(var: Variable, var_name: str) -> KerchunkArrRe shape=np_arr.shape, dtype=np_arr.dtype, order="C", + # TODO: should this be None? fill_value=None, ) diff --git a/virtualizarr/zarr.py b/virtualizarr/zarr.py index 92042442..f52b6ee4 100644 --- a/virtualizarr/zarr.py +++ b/virtualizarr/zarr.py @@ -116,7 +116,6 @@ def from_kerchunk_refs(cls, decoded_arr_refs_zarray) -> "ArrayV3Metadata": def dict(self) -> dict[str, Any]: zarray_dict = dataclasses.asdict(self) - zarray_dict["dtype"] = encode_dtype(zarray_dict["dtype"]) return zarray_dict def to_kerchunk_json(self) -> str: @@ -189,11 +188,6 @@ def serializer(self) -> Any: return parse_codecs([bytes])[0] -def encode_dtype(dtype: np.dtype) -> str: - # TODO not sure if there is a better way to get the ' int: """ Ceiling division operator for integers. @@ -273,7 +267,9 @@ def convert_to_codec_pipeline( return codec_pipeline -def convert_v3_to_v2_metadata(v3_metadata: ArrayV3Metadata) -> ArrayV2Metadata: +def convert_v3_to_v2_metadata( + v3_metadata: ArrayV3Metadata, fill_value: Any = None +) -> ArrayV2Metadata: """ Convert ArrayV3Metadata to ArrayV2Metadata. @@ -306,7 +302,7 @@ def convert_v3_to_v2_metadata(v3_metadata: ArrayV3Metadata) -> ArrayV2Metadata: shape=v3_metadata.shape, dtype=v3_metadata.data_type.to_numpy(), chunks=v3_metadata.chunk_grid.chunk_shape, - fill_value=v3_metadata.fill_value, + fill_value=fill_value or v3_metadata.fill_value, compressor=compressor_config, filters=filter_configs, order="C", @@ -326,8 +322,10 @@ def to_kerchunk_json(v2_metadata: ArrayV2Metadata) -> str: ] if zarray_dict["compressor"]: zarray_dict["compressor"] = zarray_dict["compressor"].get_config() - if np.isnan(zarray_dict["fill_value"]): - zarray_dict["fill_value"] = None - if isinstance(zarray_dict["fill_value"], np.generic): - zarray_dict["fill_value"] = zarray_dict["fill_value"].item() + fill_value = zarray_dict["fill_value"] + if fill_value is not None: + if np.isnan(fill_value): # np.isnan returns False for non-float types + zarray_dict["fill_value"] = None + elif isinstance(fill_value, np.generic): + zarray_dict["fill_value"] = fill_value.item() return ujson.dumps(zarray_dict) From ff23eeb28ecc6109123f190a49b83691ee0c8a23 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Sun, 9 Feb 2025 13:18:14 -0800 Subject: [PATCH 21/78] Fix xarray tests --- conftest.py | 4 +- virtualizarr/tests/test_xarray.py | 121 +++++++++--------------------- 2 files changed, 36 insertions(+), 89 deletions(-) diff --git a/conftest.py b/conftest.py index 0c3989a9..5caadf2f 100644 --- a/conftest.py +++ b/conftest.py @@ -217,8 +217,8 @@ def _create_manifestarray( @pytest.fixture def array_v3_metadata(): def _create_metadata( - shape: tuple, - chunks: tuple, + shape: tuple = (5, 5), + chunks: tuple = (5, 5), data_type: str = "int32", codecs: list[dict] | None = None, fill_value: int = None, diff --git a/virtualizarr/tests/test_xarray.py b/virtualizarr/tests/test_xarray.py index fd856c06..c643cd78 100644 --- a/virtualizarr/tests/test_xarray.py +++ b/virtualizarr/tests/test_xarray.py @@ -12,30 +12,21 @@ requires_hdf5plugin, requires_imagecodecs, ) -from virtualizarr.zarr import ZArray -def test_wrapping(): +def test_wrapping(array_v3_metadata): chunks = (5, 10) shape = (5, 20) dtype = np.dtype("int32") - zarray = ZArray( - chunks=chunks, - compressor={"id": "zlib", "level": 1}, - dtype=dtype, - fill_value=0.0, - filters=None, - order="C", - shape=shape, - zarr_format=2, - ) chunks_dict = { "0.0": {"path": "/foo.nc", "offset": 100, "length": 100}, "0.1": {"path": "/foo.nc", "offset": 200, "length": 100}, } manifest = ChunkManifest(entries=chunks_dict) - marr = ManifestArray(zarray=zarray, chunkmanifest=manifest) + marr = ManifestArray( + metadata=array_v3_metadata(chunks=chunks, shape=shape), chunkmanifest=manifest + ) ds = xr.Dataset({"a": (["x", "y"], marr)}) assert isinstance(ds["a"].data, ManifestArray) @@ -46,29 +37,16 @@ def test_wrapping(): class TestEquals: # regression test for GH29 https://github.com/TomNicholas/VirtualiZarr/issues/29 - def test_equals(self): - chunks = (5, 10) - shape = (5, 20) - zarray = ZArray( - chunks=chunks, - compressor={"id": "zlib", "level": 1}, - dtype=np.dtype("int32"), - fill_value=0.0, - filters=None, - order="C", - shape=shape, - zarr_format=2, - ) - + def test_equals(self, array_v3_metadata): chunks_dict1 = { "0.0": {"path": "/foo.nc", "offset": 100, "length": 100}, "0.1": {"path": "/foo.nc", "offset": 200, "length": 100}, } manifest1 = ChunkManifest(entries=chunks_dict1) - marr1 = ManifestArray(zarray=zarray, chunkmanifest=manifest1) + marr1 = ManifestArray(metadata=array_v3_metadata(), chunkmanifest=manifest1) ds1 = xr.Dataset({"a": (["x", "y"], marr1)}) - marr2 = ManifestArray(zarray=zarray, chunkmanifest=manifest1) + marr2 = ManifestArray(metadata=array_v3_metadata(), chunkmanifest=manifest1) ds2 = xr.Dataset({"a": (["x", "y"], marr2)}) assert ds1.equals(ds2) @@ -77,32 +55,23 @@ def test_equals(self): "0.1": {"path": "/foo.nc", "offset": 400, "length": 100}, } manifest3 = ChunkManifest(entries=chunks_dict3) - marr3 = ManifestArray(zarray=zarray, chunkmanifest=manifest3) + marr3 = ManifestArray(metadata=array_v3_metadata(), chunkmanifest=manifest3) ds3 = xr.Dataset({"a": (["x", "y"], marr3)}) assert not ds1.equals(ds3) # TODO refactor these tests by making some fixtures class TestConcat: - def test_concat_along_existing_dim(self): - # both manifest arrays in this example have the same zarray properties - zarray = ZArray( - chunks=(1, 10), - compressor={"id": "zlib", "level": 1}, - dtype=np.dtype("int32"), - fill_value=0.0, - filters=None, - order="C", - shape=(1, 20), - zarr_format=2, - ) + def test_concat_along_existing_dim(self, array_v3_metadata): + # both manifest arrays in this example have the same metadata properties + metadata = array_v3_metadata(chunks=(1, 10), shape=(1, 20)) chunks_dict1 = { "0.0": {"path": "/foo.nc", "offset": 100, "length": 100}, "0.1": {"path": "/foo.nc", "offset": 200, "length": 100}, } manifest1 = ChunkManifest(entries=chunks_dict1) - marr1 = ManifestArray(zarray=zarray, chunkmanifest=manifest1) + marr1 = ManifestArray(metadata=metadata, chunkmanifest=manifest1) ds1 = xr.Dataset({"a": (["x", "y"], marr1)}) chunks_dict2 = { @@ -110,7 +79,7 @@ def test_concat_along_existing_dim(self): "0.1": {"path": "/foo.nc", "offset": 400, "length": 100}, } manifest2 = ChunkManifest(entries=chunks_dict2) - marr2 = ManifestArray(zarray=zarray, chunkmanifest=manifest2) + marr2 = ManifestArray(metadata=metadata, chunkmanifest=manifest2) ds2 = xr.Dataset({"a": (["x", "y"], marr2)}) result = xr.concat([ds1, ds2], dim="x")["a"] @@ -124,32 +93,21 @@ def test_concat_along_existing_dim(self): "1.0": {"path": "file:///foo.nc", "offset": 300, "length": 100}, "1.1": {"path": "file:///foo.nc", "offset": 400, "length": 100}, } - assert result.data.zarray.compressor == zarray.compressor - assert result.data.zarray.filters == zarray.filters - assert result.data.zarray.fill_value == zarray.fill_value - assert result.data.zarray.order == zarray.order - assert result.data.zarray.zarr_format == zarray.zarr_format + metadata_copy = metadata.to_dict().copy() + metadata_copy["shape"] = (2, 20) + assert result.data.metadata.to_dict() == metadata_copy - def test_concat_along_new_dim(self): + def test_concat_along_new_dim(self, array_v3_metadata): # this calls np.stack internally - # both manifest arrays in this example have the same zarray properties - zarray = ZArray( - chunks=(5, 10), - compressor={"id": "zlib", "level": 1}, - dtype=np.dtype("int32"), - fill_value=0.0, - filters=None, - order="C", - shape=(5, 20), - zarr_format=2, - ) + # both manifest arrays in this example have the same metadata properties chunks_dict1 = { "0.0": {"path": "/foo.nc", "offset": 100, "length": 100}, "0.1": {"path": "/foo.nc", "offset": 200, "length": 100}, } manifest1 = ChunkManifest(entries=chunks_dict1) - marr1 = ManifestArray(zarray=zarray, chunkmanifest=manifest1) + metadata = array_v3_metadata(chunks=(5, 10), shape=(5, 20)) + marr1 = ManifestArray(metadata=metadata, chunkmanifest=manifest1) ds1 = xr.Dataset({"a": (["x", "y"], marr1)}) chunks_dict2 = { @@ -157,7 +115,7 @@ def test_concat_along_new_dim(self): "0.1": {"path": "/foo.nc", "offset": 400, "length": 100}, } manifest2 = ChunkManifest(entries=chunks_dict2) - marr2 = ManifestArray(zarray=zarray, chunkmanifest=manifest2) + marr2 = ManifestArray(metadata=metadata, chunkmanifest=manifest2) ds2 = xr.Dataset({"a": (["x", "y"], marr2)}) result = xr.concat([ds1, ds2], dim="z")["a"] @@ -172,34 +130,24 @@ def test_concat_along_new_dim(self): "1.0.0": {"path": "file:///foo.nc", "offset": 300, "length": 100}, "1.0.1": {"path": "file:///foo.nc", "offset": 400, "length": 100}, } - assert result.data.zarray.compressor == zarray.compressor - assert result.data.zarray.filters == zarray.filters - assert result.data.zarray.fill_value == zarray.fill_value - assert result.data.zarray.order == zarray.order - assert result.data.zarray.zarr_format == zarray.zarr_format + metadata_copy = metadata.to_dict().copy() + metadata_copy["shape"] = (2, 5, 20) + metadata_copy["chunk_grid"]["configuration"]["chunk_shape"] = (1, 5, 10) + assert result.data.metadata.to_dict() == metadata_copy - def test_concat_dim_coords_along_existing_dim(self): + def test_concat_dim_coords_along_existing_dim(self, array_v3_metadata): # Tests that dimension coordinates don't automatically get new indexes on concat # See https://github.com/pydata/xarray/issues/8871 - # both manifest arrays in this example have the same zarray properties - zarray = ZArray( - chunks=(10,), - compressor={"id": "zlib", "level": 1}, - dtype=np.dtype("int32"), - fill_value=0.0, - filters=None, - order="C", - shape=(20,), - zarr_format=2, - ) + # both manifest arrays in this example have the same metadata properties chunks_dict1 = { "0": {"path": "/foo.nc", "offset": 100, "length": 100}, "1": {"path": "/foo.nc", "offset": 200, "length": 100}, } manifest1 = ChunkManifest(entries=chunks_dict1) - marr1 = ManifestArray(zarray=zarray, chunkmanifest=manifest1) + metadata = array_v3_metadata(chunks=(10,), shape=(20,)) + marr1 = ManifestArray(metadata=metadata, chunkmanifest=manifest1) coords = xr.Coordinates({"t": (["t"], marr1)}, indexes={}) ds1 = xr.Dataset(coords=coords) @@ -208,7 +156,7 @@ def test_concat_dim_coords_along_existing_dim(self): "1": {"path": "/foo.nc", "offset": 400, "length": 100}, } manifest2 = ChunkManifest(entries=chunks_dict2) - marr2 = ManifestArray(zarray=zarray, chunkmanifest=manifest2) + marr2 = ManifestArray(metadata=metadata, chunkmanifest=manifest2) coords = xr.Coordinates({"t": (["t"], marr2)}, indexes={}) ds2 = xr.Dataset(coords=coords) @@ -223,11 +171,10 @@ def test_concat_dim_coords_along_existing_dim(self): "2": {"path": "file:///foo.nc", "offset": 300, "length": 100}, "3": {"path": "file:///foo.nc", "offset": 400, "length": 100}, } - assert result.data.zarray.compressor == zarray.compressor - assert result.data.zarray.filters == zarray.filters - assert result.data.zarray.fill_value == zarray.fill_value - assert result.data.zarray.order == zarray.order - assert result.data.zarray.zarr_format == zarray.zarr_format + metadata_copy = metadata.to_dict().copy() + metadata_copy["shape"] = (40,) + metadata_copy["chunk_grid"]["configuration"]["chunk_shape"] = (10,) + assert result.data.metadata.to_dict() == metadata_copy @requires_hdf5plugin From 8560f2d085b066cd3acd755c2bdc70831de00864 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Sun, 9 Feb 2025 14:12:22 -0800 Subject: [PATCH 22/78] Working on integration tests --- virtualizarr/writers/kerchunk.py | 37 ++++++++++++++++++++++++++------ virtualizarr/zarr.py | 11 +++++----- 2 files changed, 37 insertions(+), 11 deletions(-) diff --git a/virtualizarr/writers/kerchunk.py b/virtualizarr/writers/kerchunk.py index c1a4f28a..cc49d145 100644 --- a/virtualizarr/writers/kerchunk.py +++ b/virtualizarr/writers/kerchunk.py @@ -13,7 +13,17 @@ class NumpyEncoder(json.JSONEncoder): - # TODO I don't understand how kerchunk gets around this problem of encoding numpy types (in the zattrs) whilst only using ujson + """JSON encoder that handles common scientific Python types found in attributes. + + This encoder converts various Python types to JSON-serializable formats: + - NumPy arrays and scalars to Python lists and native types + - NumPy dtypes to strings + - Sets to lists + - Other objects that implement __array__ to lists + - Objects with to_dict method (like pandas objects) + - Objects with __str__ method as fallback + """ + def default(self, obj): if isinstance(obj, np.ndarray): return obj.tolist() # Convert NumPy array to Python list @@ -21,7 +31,19 @@ def default(self, obj): return obj.item() # Convert NumPy scalar to Python scalar elif isinstance(obj, np.dtype): return str(obj) - return json.JSONEncoder.default(self, obj) + elif isinstance(obj, set): + return list(obj) # Convert sets to lists + elif hasattr(obj, "__array__"): + return np.asarray(obj).tolist() # Handle array-like objects + elif hasattr(obj, "to_dict"): + return obj.to_dict() # Handle objects with to_dict method + + try: + return json.JSONEncoder.default(self, obj) + except TypeError: + if hasattr(obj, "__str__"): + return str(obj) + raise def dataset_to_kerchunk_refs(ds: Dataset) -> KerchunkStoreRefs: @@ -85,8 +107,12 @@ def variable_to_kerchunk_arr_refs(var: Variable, var_name: str) -> KerchunkArrRe ] for chunk_key, entry in marr.manifest.dict().items() } - - array_v2_metadata = convert_v3_to_v2_metadata(marr.metadata) + # attrs = json.loads(json.dumps(var.attrs, cls=NumpyEncoder)) + # marr.metadata.update_attributes(attrs) + # import pdb; pdb.set_trace() + array_v2_metadata = convert_v3_to_v2_metadata( + marr.metadata + ) # .update_attributes(attrs)) else: try: np_arr = var.to_numpy() @@ -123,8 +149,7 @@ def variable_to_kerchunk_arr_refs(var: Variable, var_name: str) -> KerchunkArrRe shape=np_arr.shape, dtype=np_arr.dtype, order="C", - # TODO: should this be None? - fill_value=None, + fill_value=var.encoding.get("fill_value", None), ) zarray_dict = to_kerchunk_json(array_v2_metadata) diff --git a/virtualizarr/zarr.py b/virtualizarr/zarr.py index f52b6ee4..e16d1c63 100644 --- a/virtualizarr/zarr.py +++ b/virtualizarr/zarr.py @@ -322,10 +322,11 @@ def to_kerchunk_json(v2_metadata: ArrayV2Metadata) -> str: ] if zarray_dict["compressor"]: zarray_dict["compressor"] = zarray_dict["compressor"].get_config() + + # TODO: should we use NumpyEncoder here? fill_value = zarray_dict["fill_value"] - if fill_value is not None: - if np.isnan(fill_value): # np.isnan returns False for non-float types - zarray_dict["fill_value"] = None - elif isinstance(fill_value, np.generic): - zarray_dict["fill_value"] = fill_value.item() + if np.isnan(fill_value): + zarray_dict["fill_value"] = None + if isinstance(fill_value, np.generic): + zarray_dict["fill_value"] = fill_value.item() return ujson.dumps(zarray_dict) From 97d0a710a731328f0dbaa0400f89ee8b3f8962c1 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Sun, 9 Feb 2025 20:34:40 -0800 Subject: [PATCH 23/78] Add expected type --- conftest.py | 4 ++-- virtualizarr/tests/test_integration.py | 3 +++ virtualizarr/zarr.py | 4 ++-- 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/conftest.py b/conftest.py index 5caadf2f..1d8112e8 100644 --- a/conftest.py +++ b/conftest.py @@ -219,7 +219,7 @@ def array_v3_metadata(): def _create_metadata( shape: tuple = (5, 5), chunks: tuple = (5, 5), - data_type: str = "int32", + data_type: str = np.dtype("int32"), codecs: list[dict] | None = None, fill_value: int = None, ): @@ -231,7 +231,7 @@ def _create_metadata( fill_value=fill_value, codecs=convert_to_codec_pipeline( codecs=codecs, - dtype=np.dtype("int32"), + dtype=data_type, ), attributes={}, dimension_names=None, diff --git a/virtualizarr/tests/test_integration.py b/virtualizarr/tests/test_integration.py index ed127fb2..784f6a61 100644 --- a/virtualizarr/tests/test_integration.py +++ b/virtualizarr/tests/test_integration.py @@ -221,6 +221,7 @@ def test_non_dimension_coordinates(self, tmpdir, roundtrip_func, hdf_backend): for coord in ds.coords: assert ds.coords[coord].attrs == roundtrip.coords[coord].attrs + # In Zarr V3 we cannot use a data type of dat def test_datetime64_dtype_fill_value( self, tmpdir, roundtrip_func, array_v3_metadata ): @@ -234,6 +235,8 @@ def test_datetime64_dtype_fill_value( shape=shape, chunks=chunks, codecs=[{"name": "numcodecs.zlib", "configuration": {"level": 1}}], + # Cannot do in Zarr v3 + data_type=np.dtype("M8[ns]"), ) marr1 = ManifestArray(metadata=metadata, chunkmanifest=manifest) vds = xr.Dataset( diff --git a/virtualizarr/zarr.py b/virtualizarr/zarr.py index e16d1c63..8e978fff 100644 --- a/virtualizarr/zarr.py +++ b/virtualizarr/zarr.py @@ -323,9 +323,9 @@ def to_kerchunk_json(v2_metadata: ArrayV2Metadata) -> str: if zarray_dict["compressor"]: zarray_dict["compressor"] = zarray_dict["compressor"].get_config() - # TODO: should we use NumpyEncoder here? + # TODO: Use NumpyEncoder here? fill_value = zarray_dict["fill_value"] - if np.isnan(fill_value): + if fill_value is not None and np.isnan(fill_value): zarray_dict["fill_value"] = None if isinstance(fill_value, np.generic): zarray_dict["fill_value"] = fill_value.item() From 669ce521b5754387924390077e1231d634f1885f Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Sun, 9 Feb 2025 20:49:35 -0800 Subject: [PATCH 24/78] Mark datetime tests xfail --- virtualizarr/tests/test_integration.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/virtualizarr/tests/test_integration.py b/virtualizarr/tests/test_integration.py index 784f6a61..797d1f81 100644 --- a/virtualizarr/tests/test_integration.py +++ b/virtualizarr/tests/test_integration.py @@ -221,7 +221,9 @@ def test_non_dimension_coordinates(self, tmpdir, roundtrip_func, hdf_backend): for coord in ds.coords: assert ds.coords[coord].attrs == roundtrip.coords[coord].attrs - # In Zarr V3 we cannot use a data type of dat + @pytest.mark.xfail( + reason="zarr-python 3.0 does not support datetime and timedelta data types" + ) def test_datetime64_dtype_fill_value( self, tmpdir, roundtrip_func, array_v3_metadata ): @@ -235,7 +237,6 @@ def test_datetime64_dtype_fill_value( shape=shape, chunks=chunks, codecs=[{"name": "numcodecs.zlib", "configuration": {"level": 1}}], - # Cannot do in Zarr v3 data_type=np.dtype("M8[ns]"), ) marr1 = ManifestArray(metadata=metadata, chunkmanifest=manifest) From b794dab6198d878e37ab1c7c78a27d23e1dd160f Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Sun, 9 Feb 2025 21:03:21 -0800 Subject: [PATCH 25/78] Upgrade xarray for tests --- ci/min-deps.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ci/min-deps.yml b/ci/min-deps.yml index a0a0cf20..c1288465 100644 --- a/ci/min-deps.yml +++ b/ci/min-deps.yml @@ -6,7 +6,7 @@ dependencies: - h5py - hdf5 - netcdf4 - - xarray>=2024.10.0 + - xarray>=2025.1.1 - numpy>=2.0.0 - numcodecs - packaging From 825142daa1ab9803e497c0f8a92047248659a097 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Sun, 9 Feb 2025 21:11:05 -0800 Subject: [PATCH 26/78] xfail some unsupported zarr-python 3 data types --- virtualizarr/tests/test_backend.py | 8 +++----- virtualizarr/tests/test_readers/test_fits.py | 1 + 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/virtualizarr/tests/test_backend.py b/virtualizarr/tests/test_backend.py index 1b27b0f2..23af780f 100644 --- a/virtualizarr/tests/test_backend.py +++ b/virtualizarr/tests/test_backend.py @@ -14,7 +14,6 @@ from virtualizarr.readers import HDF5VirtualBackend from virtualizarr.readers.hdf import HDFVirtualBackend from virtualizarr.tests import ( - has_astropy, parametrize_over_hdf_backends, requires_hdf5plugin, requires_imagecodecs, @@ -230,9 +229,10 @@ class TestReadFromURL: "grib", "https://github.com/pydata/xarray-data/raw/master/era5-2mt-2019-03-uk.grib", ), - ( + pytest.param( "netcdf3", "https://github.com/pydata/xarray-data/raw/master/air_temperature.nc", + marks=pytest.mark.xfail(reason="Not supported by zarr-python 3.0"), ), ( "netcdf4", @@ -258,9 +258,7 @@ class TestReadFromURL: pytest.param( "fits", "https://fits.gsfc.nasa.gov/samples/WFPC2u5780205r_c0fx.fits", - marks=pytest.mark.skipif( - not has_astropy, reason="package astropy is not available" - ), + marks=pytest.mark.xfail(reason="Not supported by zarr-python 3.0"), ), ( "jpg", diff --git a/virtualizarr/tests/test_readers/test_fits.py b/virtualizarr/tests/test_readers/test_fits.py index cdaeeab8..65cb7422 100644 --- a/virtualizarr/tests/test_readers/test_fits.py +++ b/virtualizarr/tests/test_readers/test_fits.py @@ -9,6 +9,7 @@ @requires_kerchunk @requires_network +@pytest.mark.xfail(reason="zarr-python 3.0 does not yet support big endian") def test_open_hubble_data(): # data from https://registry.opendata.aws/hst/ vds = open_virtual_dataset( From 6684125d7f3592f691fb2ef55a5a2c5eb5cda0fb Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Sun, 9 Feb 2025 21:15:12 -0800 Subject: [PATCH 27/78] Require zarr --- ci/min-deps.yml | 1 + pyproject.toml | 1 + 2 files changed, 2 insertions(+) diff --git a/ci/min-deps.yml b/ci/min-deps.yml index c1288465..8a961a5b 100644 --- a/ci/min-deps.yml +++ b/ci/min-deps.yml @@ -23,3 +23,4 @@ dependencies: - pytest - pooch - fsspec + - zarr>=3.0.2 diff --git a/pyproject.toml b/pyproject.toml index e6256aff..e46069b7 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -26,6 +26,7 @@ dependencies = [ "numcodecs", "ujson", "packaging", + "zarr">=3.0.2" ] [project.optional-dependencies] From 5e82de4be7550cd2fa7299b3c02f493bfc6ddbcb Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Sun, 9 Feb 2025 21:16:53 -0800 Subject: [PATCH 28/78] Remove zarr dep --- ci/min-deps.yml | 1 - pyproject.toml | 1 - 2 files changed, 2 deletions(-) diff --git a/ci/min-deps.yml b/ci/min-deps.yml index 8a961a5b..c1288465 100644 --- a/ci/min-deps.yml +++ b/ci/min-deps.yml @@ -23,4 +23,3 @@ dependencies: - pytest - pooch - fsspec - - zarr>=3.0.2 diff --git a/pyproject.toml b/pyproject.toml index e46069b7..e6256aff 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -26,7 +26,6 @@ dependencies = [ "numcodecs", "ujson", "packaging", - "zarr">=3.0.2" ] [project.optional-dependencies] From f57b48d284c1a04f0b0858ccd20e40505f456cc2 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Mon, 10 Feb 2025 09:04:14 -0800 Subject: [PATCH 29/78] import zarr, explicit dependency Co-authored-by: Tom Nicholas --- virtualizarr/codecs.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/virtualizarr/codecs.py b/virtualizarr/codecs.py index 100755ab..a0b66810 100644 --- a/virtualizarr/codecs.py +++ b/virtualizarr/codecs.py @@ -15,7 +15,9 @@ ] -def get_codecs(array: Union["ManifestArray", "Array"]) -> CodecPipeline: +import zarr + +def get_codecs(array: Union["ManifestArray", "zarr.Array"]) -> CodecPipeline: """ Get the zarr v3 codec pipeline for either a ManifestArray or a Zarr Array. From b811959146bbd3cd0f768d32d92d7744a7755550 Mon Sep 17 00:00:00 2001 From: "pre-commit-ci[bot]" <66853113+pre-commit-ci[bot]@users.noreply.github.com> Date: Mon, 10 Feb 2025 17:04:28 +0000 Subject: [PATCH 30/78] [pre-commit.ci] auto fixes from pre-commit.com hooks for more information, see https://pre-commit.ci --- virtualizarr/codecs.py | 1 + 1 file changed, 1 insertion(+) diff --git a/virtualizarr/codecs.py b/virtualizarr/codecs.py index a0b66810..58055a3c 100644 --- a/virtualizarr/codecs.py +++ b/virtualizarr/codecs.py @@ -17,6 +17,7 @@ import zarr + def get_codecs(array: Union["ManifestArray", "zarr.Array"]) -> CodecPipeline: """ Get the zarr v3 codec pipeline for either a ManifestArray or a Zarr Array. From 8c5139b5970044a22d1af4bdf6b447b971be5338 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Tue, 11 Feb 2025 13:26:20 -0800 Subject: [PATCH 31/78] Add zarr as a dependency --- ci/min-deps.yml | 1 + pyproject.toml | 1 + virtualizarr/tests/test_manifests/test_array.py | 4 ++-- virtualizarr/tests/test_readers/test_kerchunk.py | 2 +- 4 files changed, 5 insertions(+), 3 deletions(-) diff --git a/ci/min-deps.yml b/ci/min-deps.yml index c1288465..8a961a5b 100644 --- a/ci/min-deps.yml +++ b/ci/min-deps.yml @@ -23,3 +23,4 @@ dependencies: - pytest - pooch - fsspec + - zarr>=3.0.2 diff --git a/pyproject.toml b/pyproject.toml index e6256aff..45b592af 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -26,6 +26,7 @@ dependencies = [ "numcodecs", "ujson", "packaging", + "zarr>=3.0.2", ] [project.optional-dependencies] diff --git a/virtualizarr/tests/test_manifests/test_array.py b/virtualizarr/tests/test_manifests/test_array.py index b1ad83de..fca94ef0 100644 --- a/virtualizarr/tests/test_manifests/test_array.py +++ b/virtualizarr/tests/test_manifests/test_array.py @@ -254,7 +254,7 @@ def test_concat_empty(self, array_v3_metadata): } codec_dict = result.metadata.codecs[1].to_dict() assert codec_dict["name"] == "numcodecs.zlib" - assert codec_dict["configuration"] == {"id": "zlib", "level": 1} + assert codec_dict["configuration"] == {"level": 1} assert result.metadata.fill_value == metadata.fill_value @@ -291,7 +291,7 @@ def test_stack(self, array_v3_metadata): } codec_dict = result.metadata.codecs[1].to_dict() assert codec_dict["name"] == "numcodecs.zlib" - assert codec_dict["configuration"] == {"id": "zlib", "level": 1} + assert codec_dict["configuration"] == {"level": 1} assert result.metadata.fill_value == metadata.fill_value def test_stack_empty(self, array_v3_metadata): diff --git a/virtualizarr/tests/test_readers/test_kerchunk.py b/virtualizarr/tests/test_readers/test_kerchunk.py index 621cfd5e..b9f82453 100644 --- a/virtualizarr/tests/test_readers/test_kerchunk.py +++ b/virtualizarr/tests/test_readers/test_kerchunk.py @@ -103,7 +103,7 @@ def test_dataset_from_df_refs_with_filters(refs_file_factory): vda = vds["a"] assert vda.data.metadata.codecs[1].to_dict() == { "name": "numcodecs.shuffle", - "configuration": {"id": "shuffle", "elementsize": 8}, + "configuration": {"elementsize": 4}, } From 53597624f569ecdbd987eee427eea61ecd694385 Mon Sep 17 00:00:00 2001 From: "pre-commit-ci[bot]" <66853113+pre-commit-ci[bot]@users.noreply.github.com> Date: Tue, 11 Feb 2025 21:32:10 +0000 Subject: [PATCH 32/78] [pre-commit.ci] auto fixes from pre-commit.com hooks for more information, see https://pre-commit.ci --- virtualizarr/codecs.py | 4 +--- virtualizarr/tests/test_codecs.py | 1 + 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/virtualizarr/codecs.py b/virtualizarr/codecs.py index 3a065725..dd62a3b8 100644 --- a/virtualizarr/codecs.py +++ b/virtualizarr/codecs.py @@ -2,9 +2,6 @@ if TYPE_CHECKING: from zarr import Array # type: ignore - from zarr.core.abc.codec import ( # type: ignore - Codec as ZarrCodec, - ) from .manifests.array import ManifestArray @@ -56,6 +53,7 @@ def _is_manifest_array(array: object) -> bool: except ImportError: return False + def _get_manifestarray_codecs(array: "ManifestArray") -> CodecPipeline: """Get zarr v3 codec pipeline for a ManifestArray.""" if array.metadata.zarr_format != 3: diff --git a/virtualizarr/tests/test_codecs.py b/virtualizarr/tests/test_codecs.py index 517ded39..ef2715ed 100644 --- a/virtualizarr/tests/test_codecs.py +++ b/virtualizarr/tests/test_codecs.py @@ -17,6 +17,7 @@ }, } + class TestCodecs: def create_zarr_array(self, codecs=None, zarr_format=3): """Create a test Zarr array with the specified codecs.""" From c808351f58ea3a4135965fc83c2942228f12360d Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Tue, 11 Feb 2025 21:14:15 -0800 Subject: [PATCH 33/78] Min numcodecs version --- ci/min-deps.yml | 2 +- pyproject.toml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/ci/min-deps.yml b/ci/min-deps.yml index 8a961a5b..02921b1e 100644 --- a/ci/min-deps.yml +++ b/ci/min-deps.yml @@ -8,7 +8,7 @@ dependencies: - netcdf4 - xarray>=2025.1.1 - numpy>=2.0.0 - - numcodecs + - numcodecs>=0.15.1 - packaging - ujson - universal_pathlib diff --git a/pyproject.toml b/pyproject.toml index 45b592af..b68c919b 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -23,7 +23,7 @@ dependencies = [ "xarray>=2025.1.1", "numpy>=2.0.0", "universal-pathlib", - "numcodecs", + "numcodecs>=0.15.1", "ujson", "packaging", "zarr>=3.0.2", From bd50167c2da817ac55601d59fbbd0f0b1e8c9ef7 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Tue, 11 Feb 2025 21:17:06 -0800 Subject: [PATCH 34/78] numcodecs>=0.15.1 in environment and upstream.yml conda env files --- ci/environment.yml | 4 ++-- ci/upstream.yml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/ci/environment.yml b/ci/environment.yml index a114b659..df444d95 100644 --- a/ci/environment.yml +++ b/ci/environment.yml @@ -14,7 +14,7 @@ dependencies: - packaging - universal_pathlib - hdf5plugin - - numcodecs + - numcodecs>=0.15.1 - icechunk>=0.1.1 # Testing - codecov[toml] @@ -37,4 +37,4 @@ dependencies: - astropy - pip - pip: - - imagecodecs-numcodecs==2024.6.1 + - imagecodecs-numcodecs==2024.6.1 diff --git a/ci/upstream.yml b/ci/upstream.yml index f312de2c..2da63c9b 100644 --- a/ci/upstream.yml +++ b/ci/upstream.yml @@ -13,7 +13,7 @@ dependencies: - ujson - universal_pathlib - hdf5plugin - - numcodecs + - numcodecs>=0.15.1 - imagecodecs>=2024.6.1 # Testing - codecov[toml] From ed977044aa8b0a5e6cf73f3022a27b5991818275 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Tue, 11 Feb 2025 22:31:46 -0800 Subject: [PATCH 35/78] Working on mypy errors --- virtualizarr/codecs.py | 9 +- virtualizarr/manifests/array_api.py | 10 +- virtualizarr/translators/kerchunk.py | 14 +- virtualizarr/writers/icechunk.py | 2 +- virtualizarr/zarr.py | 188 ++++++++++----------------- 5 files changed, 95 insertions(+), 128 deletions(-) diff --git a/virtualizarr/codecs.py b/virtualizarr/codecs.py index dd62a3b8..eff6d090 100644 --- a/virtualizarr/codecs.py +++ b/virtualizarr/codecs.py @@ -1,16 +1,15 @@ from typing import TYPE_CHECKING, Tuple, Union -if TYPE_CHECKING: - from zarr import Array # type: ignore +import zarr +from zarr.abc.codec import ArrayArrayCodec, ArrayBytesCodec, BytesBytesCodec +if TYPE_CHECKING: from .manifests.array import ManifestArray CodecPipeline = Tuple[ Union["ArrayArrayCodec", "ArrayBytesCodec", "BytesBytesCodec"], ... ] -import zarr - def get_codecs(array: Union["ManifestArray", "zarr.Array"]) -> CodecPipeline: """ @@ -73,7 +72,7 @@ def _is_zarr_array(array: object) -> bool: return False -def _get_zarr_array_codecs(array: "Array") -> CodecPipeline: +def _get_zarr_array_codecs(array: "zarr.Array") -> CodecPipeline: """Get zarr v3 codec pipeline for a Zarr Array.""" from zarr.core.metadata import ArrayV3Metadata # type: ignore[import-untyped] diff --git a/virtualizarr/manifests/array_api.py b/virtualizarr/manifests/array_api.py index ace9136f..92c4022b 100644 --- a/virtualizarr/manifests/array_api.py +++ b/virtualizarr/manifests/array_api.py @@ -176,7 +176,10 @@ def stack( metadata_copy = first_arr.metadata.to_dict().copy() metadata_copy["shape"] = tuple(new_shape) - metadata_copy["chunk_grid"]["configuration"]["chunk_shape"] = tuple(new_chunks) + metadata_copy["chunk_grid"] = { + "name": "regular", + "configuration": {"chunk_shape": tuple(new_chunks)}, + } # ArrayV3Metadata.from_dict removes extra keys zarr_format and node_type new_metadata = ArrayV3Metadata.from_dict(metadata_copy) @@ -244,7 +247,10 @@ def broadcast_to(x: "ManifestArray", /, shape: tuple[int, ...]) -> "ManifestArra metadata_copy = x.metadata.to_dict().copy() metadata_copy["shape"] = tuple(new_shape) - metadata_copy["chunk_grid"]["configuration"]["chunk_shape"] = tuple(new_chunk_shape) + metadata_copy["chunk_grid"] = { + "name": "regular", + "configuration": {"chunk_shape": tuple(new_chunk_shape)}, + } # ArrayV3Metadata.from_dict removes extra keys zarr_format and node_type new_metadata = ArrayV3Metadata.from_dict(metadata_copy) diff --git a/virtualizarr/translators/kerchunk.py b/virtualizarr/translators/kerchunk.py index aeef5cd6..d30832b9 100644 --- a/virtualizarr/translators/kerchunk.py +++ b/virtualizarr/translators/kerchunk.py @@ -11,7 +11,7 @@ KerchunkArrRefs, KerchunkStoreRefs, ) -from virtualizarr.zarr import ZArray, ZAttrs, determine_chunk_grid_shape +from virtualizarr.zarr import ZAttrs, determine_chunk_grid_shape def virtual_vars_and_metadata_from_kerchunk_refs( @@ -171,9 +171,10 @@ def variable_from_kerchunk_refs( elif len(metadata.shape) != 0: # empty variables don't have physical chunks, but zarray shows that the variable # is at least 1D + shape = determine_chunk_grid_shape( metadata.shape, - metadata.to_dict()["chunk_grid"]["configuration"]["chunk_shape"], + metadata.chunks, ) manifest = ChunkManifest(entries={}, shape=shape) varr = virtual_array_class(metadata=metadata, chunkmanifest=manifest) @@ -264,10 +265,15 @@ def extract_array_refs( ) +from zarr.core.metadata import ArrayV3Metadata + +from virtualizarr.zarr import from_kerchunk_refs + + def parse_array_refs( arr_refs: KerchunkArrRefs, -) -> tuple[dict, ZArray, ZAttrs]: - metadata = ZArray.from_kerchunk_refs(arr_refs.pop(".zarray")) +) -> tuple[dict, ArrayV3Metadata, ZAttrs]: + metadata = from_kerchunk_refs(arr_refs.pop(".zarray")) zattrs = arr_refs.pop(".zattrs", {}) chunk_dict = arr_refs diff --git a/virtualizarr/writers/icechunk.py b/virtualizarr/writers/icechunk.py index 75305cf7..5872bcd2 100644 --- a/virtualizarr/writers/icechunk.py +++ b/virtualizarr/writers/icechunk.py @@ -244,7 +244,7 @@ def write_virtual_variable_to_icechunk( else: append_axis = None # TODO: Should codecs be an argument to zarr's AsyncrGroup.create_array? - filters, compressors = extract_codecs(metadata.codecs) + filters, _, compressors = extract_codecs(metadata.codecs) arr = group.require_array( name=name, shape=metadata.shape, diff --git a/virtualizarr/zarr.py b/virtualizarr/zarr.py index 7216c2ad..665d00a1 100644 --- a/virtualizarr/zarr.py +++ b/virtualizarr/zarr.py @@ -2,6 +2,7 @@ from typing import TYPE_CHECKING, Any, Literal, NewType import numpy as np +from zarr.abc.codec import ArrayArrayCodec, ArrayBytesCodec, BytesBytesCodec from zarr.abc.codec import Codec as ZarrCodec from zarr.core.metadata.v2 import ArrayV2Metadata from zarr.core.metadata.v3 import ArrayV3Metadata @@ -9,7 +10,6 @@ if TYPE_CHECKING: try: from zarr.abc.codec import Codec as ZarrCodec - from zarr.core.array import CompressorLike, FiltersLike, SerializerLike except ImportError: pass @@ -36,34 +36,6 @@ """ -@dataclasses.dataclass -class Codec: - compressors: list[dict] | None = None - filters: list[dict] | None = None - - -@dataclasses.dataclass -class ZarrV3Codecs: - filters: "FiltersLike" - compressors: "CompressorLike" - serializer: "SerializerLike" - - def into_v3_codecs(self, dtype: np.dtype) -> tuple["ZarrCodec", ...]: - try: - from zarr.core.array import _parse_chunk_encoding_v3 - except ImportError: - raise ImportError("zarr v3 is required to generate v3 codecs") - - codecs = _parse_chunk_encoding_v3( - serializer=self.serializer, - compressors=self.compressors, - filters=self.filters, - dtype=dtype, - ) - - return cast(tuple["ZarrCodec", ...], (*codecs[0], codecs[1], *codecs[2])) - - @dataclasses.dataclass class ZArray: """Just the .zarray information""" @@ -93,53 +65,6 @@ def __post_init__(self) -> None: if self.fill_value is None: self.fill_value = ZARR_DEFAULT_FILL_VALUE.get(self.dtype.kind, 0.0) - @property - def codec(self) -> Codec: - """For comparison against other arrays.""" - return Codec(compressor=self.compressor, filters=self.filters) - - @classmethod - def from_kerchunk_refs(cls, decoded_arr_refs_zarray) -> "ArrayV3Metadata": - # coerce type of fill_value as kerchunk can be inconsistent with this - dtype = np.dtype(decoded_arr_refs_zarray["dtype"]) - fill_value = decoded_arr_refs_zarray["fill_value"] - if np.issubdtype(dtype, np.floating) and ( - fill_value is None or fill_value == "NaN" or fill_value == "nan" - ): - fill_value = np.nan - - zarr_format = int(decoded_arr_refs_zarray["zarr_format"]) - if zarr_format not in (2, 3): - raise ValueError(f"Zarr format must be 2 or 3, but got {zarr_format}") - filters = ( - decoded_arr_refs_zarray.get("filters", []) or [] - ) # Ensure filters is a list - compressor = decoded_arr_refs_zarray.get("compressor") # Might be None - - # Ensure compressor is a list before unpacking - codec_configs = [*filters, *(compressor if compressor is not None else [])] - numcodec_configs = [ - _num_codec_config_to_configurable(config) for config in codec_configs - ] - return ArrayV3Metadata( - chunk_grid={ - "name": "regular", - "configuration": { - "chunk_shape": tuple(decoded_arr_refs_zarray["chunks"]) - }, - }, - codecs=convert_to_codec_pipeline( - dtype=dtype, - codecs=numcodec_configs, - ), - data_type=dtype, - fill_value=fill_value, - shape=tuple(decoded_arr_refs_zarray["shape"]), - chunk_key_encoding={"name": "default"}, - attributes={}, - dimension_names=None, - ) - def dict(self) -> dict[str, Any]: zarray_dict = dataclasses.asdict(self) return zarray_dict @@ -187,15 +112,6 @@ def replace( replacements["zarr_format"] = zarr_format return dataclasses.replace(self, **replacements) - def _v3_codec_pipeline(self) -> tuple["ZarrCodec", ...]: - """ - Convert the compressor, filters, and dtype to a pipeline of ZarrCodecs. - """ - filters = self.filters or [] - return convert_to_codec_pipeline( - codecs=[*filters, self.compressor], dtype=self.dtype - ) - def serializer(self) -> Any: """ testing @@ -242,19 +158,22 @@ def _num_codec_config_to_configurable(num_codec: dict) -> dict: return {"name": name, "configuration": num_codec_copy} +from virtualizarr.codecs import CodecPipeline + + def extract_codecs( - codecs: list[ZarrCodec] | None = [], -) -> tuple[list[dict], list[dict]]: + codecs: CodecPipeline, +) -> tuple[list[ArrayArrayCodec], ArrayBytesCodec | None, list[BytesBytesCodec]]: """Extracts filters and compressor from Zarr v3 metadata.""" - from zarr.abc.codec import ArrayArrayCodec, BytesBytesCodec - - arrayarray_codecs, bytesbytes_codecs = [], [] + arrayarray_codecs, arraybytes_codec, bytesbytes_codecs = [], None, [] for codec in codecs: if isinstance(codec, ArrayArrayCodec): arrayarray_codecs.append(codec) + if isinstance(codec, ArrayBytesCodec): + arraybytes_codec = codec if isinstance(codec, BytesBytesCodec): bytesbytes_codecs.append(codec) - return arrayarray_codecs, bytesbytes_codecs + return arrayarray_codecs, arraybytes_codec, bytesbytes_codecs def convert_to_codec_pipeline( @@ -278,16 +197,18 @@ def convert_to_codec_pipeline( from zarr.core.array import _get_default_chunk_encoding_v3 from zarr.registry import get_codec_class + zarr_codecs = () if codecs and len(codecs) > 0: - zarr_codecs = [ - get_codec_class(codec["name"]).from_dict(codec) for codec in codecs - ] - arrayarray_codecs, bytesbytes_codecs = extract_codecs(zarr_codecs) - else: - arrayarray_codecs, bytesbytes_codecs = [], [] - # FIXME: using private zarr-python function - # we can also use zarr_config.get("array.v3_default_serializer").get("numeric"), but requires rewriting a lot of this function - arraybytes_codecs = _get_default_chunk_encoding_v3(dtype)[1] + zarr_codecs = tuple( + [get_codec_class(codec["name"]).from_dict(codec) for codec in codecs] + ) + + arrayarray_codecs, arraybytes_codecs, bytesbytes_codecs = extract_codecs( + zarr_codecs + ) + + if arraybytes_codecs is None: + arraybytes_codecs = _get_default_chunk_encoding_v3(dtype)[1] codec_pipeline = (*arrayarray_codecs, arraybytes_codecs, *bytesbytes_codecs) return codec_pipeline @@ -311,7 +232,7 @@ def convert_v3_to_v2_metadata( """ import warnings - filters, compressors = extract_codecs(v3_metadata.codecs) + filters, _, compressors = extract_codecs(v3_metadata.codecs) if compressors: compressor = compressors[0] compressor_config = compressor.codec_config @@ -323,11 +244,10 @@ def convert_v3_to_v2_metadata( compressor_config = None filter_configs = [filter.codec_config for filter in filters] - v2_metadata = ArrayV2Metadata( shape=v3_metadata.shape, dtype=v3_metadata.data_type.to_numpy(), - chunks=v3_metadata.chunk_grid.chunk_shape, + chunks=v3_metadata.chunks, fill_value=fill_value or v3_metadata.fill_value, compressor=compressor_config, filters=filter_configs, @@ -339,20 +259,56 @@ def convert_v3_to_v2_metadata( def to_kerchunk_json(v2_metadata: ArrayV2Metadata) -> str: - import ujson + import json + + from virtualizarr.writers.kerchunk import NumpyEncoder zarray_dict = v2_metadata.to_dict() - if zarray_dict["filters"]: + if v2_metadata.filters: zarray_dict["filters"] = [ - codec.get_config() for codec in zarray_dict["filters"] + codec.get_config() for codec in v2_metadata.filters if codec is not None ] - if zarray_dict["compressor"]: - zarray_dict["compressor"] = zarray_dict["compressor"].get_config() - - # TODO: Use NumpyEncoder here? - fill_value = zarray_dict["fill_value"] - if fill_value is not None and np.isnan(fill_value): - zarray_dict["fill_value"] = None - if isinstance(fill_value, np.generic): - zarray_dict["fill_value"] = fill_value.item() - return ujson.dumps(zarray_dict) + if v2_metadata.compressor: + zarray_dict["compressor"] = v2_metadata.compressor.get_config() + + return json.dumps(zarray_dict, separators=(",", ":"), cls=NumpyEncoder) + + +def from_kerchunk_refs(decoded_arr_refs_zarray) -> "ArrayV3Metadata": + # coerce type of fill_value as kerchunk can be inconsistent with this + dtype = np.dtype(decoded_arr_refs_zarray["dtype"]) + fill_value = decoded_arr_refs_zarray["fill_value"] + if np.issubdtype(dtype, np.floating) and ( + fill_value is None or fill_value == "NaN" or fill_value == "nan" + ): + fill_value = np.nan + + zarr_format = int(decoded_arr_refs_zarray["zarr_format"]) + if zarr_format not in (2, 3): + raise ValueError(f"Zarr format must be 2 or 3, but got {zarr_format}") + filters = ( + decoded_arr_refs_zarray.get("filters", []) or [] + ) # Ensure filters is a list + compressor = decoded_arr_refs_zarray.get("compressor") # Might be None + + # Ensure compressor is a list before unpacking + codec_configs = [*filters, *(compressor if compressor is not None else [])] + numcodec_configs = [ + _num_codec_config_to_configurable(config) for config in codec_configs + ] + return ArrayV3Metadata( + chunk_grid={ + "name": "regular", + "configuration": {"chunk_shape": tuple(decoded_arr_refs_zarray["chunks"])}, + }, + codecs=convert_to_codec_pipeline( + dtype=dtype, + codecs=numcodec_configs, + ), + data_type=dtype, + fill_value=fill_value, + shape=tuple(decoded_arr_refs_zarray["shape"]), + chunk_key_encoding={"name": "default"}, + attributes={}, + dimension_names=None, + ) From a3c190e3199b8a3e1c3293b3990da4d82782337d Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Wed, 12 Feb 2025 09:43:16 -0800 Subject: [PATCH 36/78] Fix mypy errors and tests --- virtualizarr/tests/test_zarr.py | 23 ++++--- virtualizarr/zarr.py | 109 +++++++++++++++++++++++--------- 2 files changed, 93 insertions(+), 39 deletions(-) diff --git a/virtualizarr/tests/test_zarr.py b/virtualizarr/tests/test_zarr.py index 6c90ee86..929dbe27 100644 --- a/virtualizarr/tests/test_zarr.py +++ b/virtualizarr/tests/test_zarr.py @@ -1,5 +1,6 @@ import numpy as np from zarr.codecs import BytesCodec +from zarr.core.codec_pipeline import BatchedCodecPipeline from zarr.core.metadata.v2 import ArrayV2Metadata from virtualizarr.zarr import ( @@ -10,7 +11,12 @@ def test_convert_to_codec_pipeline(): - expected_default_codecs = (BytesCodec(endian="little"),) + expected_default_codecs = BatchedCodecPipeline( + array_array_codecs=(), + array_bytes_codec=BytesCodec(endian="little"), + bytes_bytes_codecs=(), + batch_size=1, + ) # Test with just dtype (default codec pipeline) dtype = np.dtype(" dict: def extract_codecs( codecs: CodecPipeline, -) -> tuple[list[ArrayArrayCodec], ArrayBytesCodec | None, list[BytesBytesCodec]]: - """Extracts filters and compressor from Zarr v3 metadata.""" - arrayarray_codecs, arraybytes_codec, bytesbytes_codecs = [], None, [] +) -> tuple[ + tuple[ArrayArrayCodec, ...], ArrayBytesCodec | None, tuple[BytesBytesCodec, ...] +]: + """Extracts various codec types.""" + arrayarray_codecs: tuple[ArrayArrayCodec, ...] = () + arraybytes_codec: ArrayBytesCodec | None = None + bytesbytes_codecs: tuple[BytesBytesCodec, ...] = () for codec in codecs: if isinstance(codec, ArrayArrayCodec): - arrayarray_codecs.append(codec) + arrayarray_codecs += (codec,) if isinstance(codec, ArrayBytesCodec): arraybytes_codec = codec if isinstance(codec, BytesBytesCodec): - bytesbytes_codecs.append(codec) - return arrayarray_codecs, arraybytes_codec, bytesbytes_codecs + bytesbytes_codecs += (codec,) + return (arrayarray_codecs, arraybytes_codec, bytesbytes_codecs) + + +from zarr.core.codec_pipeline import BatchedCodecPipeline def convert_to_codec_pipeline( dtype: np.dtype, codecs: list[dict] | None = [], -) -> tuple[ZarrCodec, ...]: +) -> BatchedCodecPipeline: """ Convert compressor, filters, serializer, and dtype to a pipeline of ZarrCodecs. @@ -191,29 +199,53 @@ def convert_to_codec_pipeline( Returns ------- - Tuple[ZarrCodec, ...] - A tuple of ZarrCodecs. + BatchedCodecPipeline """ from zarr.core.array import _get_default_chunk_encoding_v3 from zarr.registry import get_codec_class - zarr_codecs = () + zarr_codecs: tuple[ArrayArrayCodec | ArrayBytesCodec | BytesBytesCodec, ...] = () if codecs and len(codecs) > 0: zarr_codecs = tuple( - [get_codec_class(codec["name"]).from_dict(codec) for codec in codecs] + get_codec_class(codec["name"]).from_dict(codec) for codec in codecs ) - arrayarray_codecs, arraybytes_codecs, bytesbytes_codecs = extract_codecs( - zarr_codecs - ) + # (aimee): I would like to use zarr.core.codec_pipeline.codecs_from_list here but it requires array array codecs before array bytes codecs, + # which I don't think is actually a requirement. + arrayarray_codecs, arraybytes_codec, bytesbytes_codecs = extract_codecs(zarr_codecs) - if arraybytes_codecs is None: - arraybytes_codecs = _get_default_chunk_encoding_v3(dtype)[1] - codec_pipeline = (*arrayarray_codecs, arraybytes_codecs, *bytesbytes_codecs) + if arraybytes_codec is None: + arraybytes_codec = _get_default_chunk_encoding_v3(dtype)[1] + + codec_pipeline = BatchedCodecPipeline( + array_array_codecs=arrayarray_codecs, + array_bytes_codec=arraybytes_codec, + bytes_bytes_codecs=bytesbytes_codecs, + batch_size=1, + ) return codec_pipeline +def _get_codec_config(codec: ZarrCodec) -> dict[str, JSON]: + """ + Extract configuration from a codec, handling both zarr-python and numcodecs codecs. + """ + if hasattr(codec, "codec_config"): + return codec.codec_config + elif hasattr(codec, "get_config"): + return codec.get_config() + elif hasattr(codec, "codec_name"): + # If we can't get config, try to get the name and configuration directly + # This assumes the codec follows the v3 spec format + return { + "id": codec.codec_name.replace("numcodecs.", ""), + **getattr(codec, "configuration", {}), + } + else: + raise ValueError(f"Unable to parse codec configuration: {codec}") + + def convert_v3_to_v2_metadata( v3_metadata: ArrayV3Metadata, fill_value: Any = None ) -> ArrayV2Metadata: @@ -224,26 +256,40 @@ def convert_v3_to_v2_metadata( ---------- v3_metadata : ArrayV3Metadata The metadata object in v3 format. + fill_value : Any, optional + Override the fill value from v3 metadata. Returns ------- ArrayV2Metadata The metadata object in v2 format. + + Notes + ----- + The conversion handles the following cases: + - Extracts compressor and filter configurations from v3 codecs + - Preserves codec configurations regardless of codec implementation + - Maintains backward compatibility with both zarr-python and numcodecs """ import warnings - filters, _, compressors = extract_codecs(v3_metadata.codecs) - if compressors: - compressor = compressors[0] - compressor_config = compressor.codec_config - if len(compressors) > 1: + array_filters: tuple[ArrayArrayCodec, ...] + bytes_compressors: tuple[BytesBytesCodec, ...] + array_filters, _, bytes_compressors = extract_codecs(v3_metadata.codecs) + + # Handle compressor configuration + compressor_config: dict[str, JSON] | None = None + if bytes_compressors: + if len(bytes_compressors) > 1: warnings.warn( - "Multiple compressors detected. Only the first one will be used." + "Multiple compressors found in v3 metadata. Using the first compressor, " + "others will be ignored. This may affect data compatibility.", + UserWarning, ) - else: - compressor_config = None + compressor_config = _get_codec_config(bytes_compressors[0]) - filter_configs = [filter.codec_config for filter in filters] + # Handle filter configurations + filter_configs = [_get_codec_config(filter_) for filter_ in array_filters] v2_metadata = ArrayV2Metadata( shape=v3_metadata.shape, dtype=v3_metadata.data_type.to_numpy(), @@ -263,13 +309,16 @@ def to_kerchunk_json(v2_metadata: ArrayV2Metadata) -> str: from virtualizarr.writers.kerchunk import NumpyEncoder - zarray_dict = v2_metadata.to_dict() + zarray_dict: dict[str, JSON] = v2_metadata.to_dict() if v2_metadata.filters: zarray_dict["filters"] = [ - codec.get_config() for codec in v2_metadata.filters if codec is not None - ] + # we could also cast to json, but get_config is intended for serialization + codec.get_config() + for codec in v2_metadata.filters + if codec is not None + ] # type: ignore[assignment] if v2_metadata.compressor: - zarray_dict["compressor"] = v2_metadata.compressor.get_config() + zarray_dict["compressor"] = v2_metadata.compressor.get_config() # type: ignore[assignment] return json.dumps(zarray_dict, separators=(",", ":"), cls=NumpyEncoder) From 95886b930f8c4ec43f9cbb20cb2b85da106bc54d Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Wed, 12 Feb 2025 10:14:53 -0800 Subject: [PATCH 37/78] Remove ZArray class --- docs/usage.md | 67 ++++++++++++++++++---- virtualizarr/readers/hdf/hdf.py | 1 - virtualizarr/tests/test_zarr.py | 27 --------- virtualizarr/writers/kerchunk.py | 7 +-- virtualizarr/zarr.py | 95 -------------------------------- 5 files changed, 57 insertions(+), 140 deletions(-) diff --git a/docs/usage.md b/docs/usage.md index c783da9c..b610beb3 100644 --- a/docs/usage.md +++ b/docs/usage.md @@ -1,4 +1,5 @@ (usage)= + # Usage This page explains how to use VirtualiZarr today, by introducing the key concepts one-by-one. @@ -27,7 +28,6 @@ vds = open_virtual_dataset('air.nc') (Notice we did not have to explicitly indicate the file format, as {py:func}`open_virtual_dataset ` will attempt to automatically infer it.) - ```{note} In future we would like for it to be possible to just use `xr.open_dataset`, e.g. @@ -43,6 +43,7 @@ Printing this "virtual dataset" shows that although it is an instance of `xarray ```python vds ``` + ``` Size: 8MB Dimensions: (time: 2920, lat: 25, lon: 53) @@ -69,12 +70,15 @@ As the manifest contains only addresses at which to find large binary chunks, th ```python ds.nbytes ``` + ``` 30975672 ``` + ```python vds.virtualize.nbytes ``` + ``` 128 ``` @@ -119,6 +123,7 @@ marr = vds['air'].data manifest = marr.manifest manifest.dict() ``` + ```python {'0.0.0': {'path': 'file:///work/data/air.nc', 'offset': 15419, 'length': 7738000}} ``` @@ -134,20 +139,49 @@ A Zarr array is defined not just by the location of its constituent chunk data, ```python marr ``` + ``` ManifestArray ``` + ```python marr.manifest ``` + ``` ChunkManifest ``` + ```python -marr.zarray +marr.metadata ``` + ``` -ZArray(shape=(2920, 25, 53), chunks=(2920, 25, 53), dtype=int16, compressor=None, filters=None, fill_value=None) +ArrayV3Metadata(shape=(2920, 25, 53), + data_type=, + chunk_grid=RegularChunkGrid(chunk_shape=(2920, 25, 53)), + chunk_key_encoding=DefaultChunkKeyEncoding(name='default', + separator='/'), + fill_value=np.float64(-327.67), + codecs=(FixedScaleOffset(codec_name='numcodecs.fixedscaleoffset', codec_config={'scale': 100.0, 'offset': 0, 'dtype': ')), + attributes={'GRIB_id': 11, + 'GRIB_name': 'TMP', + 'actual_range': [185.16000366210938, + 322.1000061035156], + 'dataset': 'NMC Reanalysis', + 'level_desc': 'Surface', + 'long_name': '4xDaily Air temperature at sigma ' + 'level 995', + 'parent_stat': 'Other', + 'precision': 2, + 'statistic': 'Individual Obs', + 'units': 'degK', + 'var_desc': 'Air temperature'}, + dimension_names=None, + zarr_format=3, + node_type='array', + storage_transformers=()) ``` A `ManifestArray` can therefore be thought of as a virtualized representation of a single Zarr array. @@ -160,12 +194,15 @@ import numpy as np concatenated = np.concatenate([marr, marr], axis=0) concatenated ``` + ``` ManifestArray ``` + ```python concatenated.manifest.dict() ``` + ``` {'0.0.0': {'path': 'file:///work/data/air.nc', 'offset': 15419, 'length': 7738000}, '1.0.0': {'path': 'file:///work/data/air.nc', 'offset': 15419, 'length': 7738000}} @@ -182,6 +219,7 @@ Remember that you cannot load values from a `ManifestArray` directly. ```python vds['air'].values ``` + ```python NotImplementedError: ManifestArrays can't be converted into numpy arrays or pandas Index objects ``` @@ -207,6 +245,7 @@ Whilst the values of virtual variables (i.e. those backed by `ManifestArray` obj ```python vds = open_virtual_dataset('air.nc', loadable_variables=['air', 'time']) ``` + ```python Size: 31MB Dimensions: (time: 2920, lat: 25, lon: 53) @@ -223,11 +262,13 @@ Attributes: references: http://www.esrl.noaa.gov/psd/data/gridded/data.ncep.reanaly... title: 4x daily NMC reanalysis (1948) ``` + You can see that the dataset contains a mixture of virtual variables backed by `ManifestArray` objects (`lat` and `lon`), and loadable variables backed by (lazy) numpy arrays (`air` and `time`). Loading variables can be useful in a few scenarios: + 1. You need to look at the actual values of a multi-dimensional variable in order to decide what to do next, -2. You want in-memory indexes to use with ``xr.combine_by_coords``, +2. You want in-memory indexes to use with `xr.combine_by_coords`, 3. Storing a variable on-disk as a set of references would be inefficient, e.g. because it's a very small array (saving the values like this is similar to kerchunk's concept of "inlining" data), 4. The variable has encoding, and the simplest way to decode it correctly is to let xarray's standard decoding machinery load it into memory and apply the decoding, 5. Some of your variables have inconsistent-length chunks, and you want to be able to concatenate them together. For example you might have multiple virtual datasets with coordinates of inconsistent length (e.g., leap years within multi-year daily data). @@ -250,6 +291,7 @@ vds = open_virtual_dataset( decode_times=True, ) ``` + ```python Size: 31MB Dimensions: (time: 2920, lat: 25, lon: 53) @@ -300,6 +342,7 @@ As we know the correct order a priori, we can just combine along one dimension u combined_vds = xr.concat([vds1, vds2], dim='time', coords='minimal', compat='override') combined_vds ``` + ``` Size: 8MB Dimensions: (time: 2920, lat: 25, lon: 53) @@ -322,6 +365,7 @@ We can see that the resulting combined manifest has two chunks, as expected. ```python combined_vds['air'].data.manifest.dict() ``` + ``` {'0.0.0': {'path': 'file:///work/data/air1.nc', 'offset': 15419, 'length': 3869000}, '1.0.0': {'path': 'file:///work/data/air2.nc', 'offset': 15419, 'length': 3869000}} @@ -370,11 +414,12 @@ vds2 = open_virtual_dataset('air2.nc', loadable_variables=['time', 'lat', 'lon'] combined_vds = xr.combine_by_coords([vds2, vds1], coords='minimal', compat='override') ``` -Notice we don't have to specify the concatenation dimension explicitly - xarray works out the correct ordering for us. Even though we actually passed in the virtual datasets in the wrong order just now, the manifest still has the chunks listed in the correct order such that the 1-dimensional ``time`` coordinate has ascending values: +Notice we don't have to specify the concatenation dimension explicitly - xarray works out the correct ordering for us. Even though we actually passed in the virtual datasets in the wrong order just now, the manifest still has the chunks listed in the correct order such that the 1-dimensional `time` coordinate has ascending values: ```python combined_vds['air'].data.manifest.dict() ``` + ``` {'0.0.0': {'path': 'file:///work/data/air1.nc', 'offset': 15419, 'length': 3869000}, '1.0.0': {'path': 'file:///work/data/air2.nc', 'offset': 15419, 'length': 3869000}} @@ -382,7 +427,7 @@ combined_vds['air'].data.manifest.dict() ### Ordering using metadata -TODO: Use preprocess to create a new index from the metadata. Requires ``open_virtual_mfdataset`` to be implemented in [PR #349](https://github.com/zarr-developers/VirtualiZarr/pull/349). +TODO: Use preprocess to create a new index from the metadata. Requires `open_virtual_mfdataset` to be implemented in [PR #349](https://github.com/zarr-developers/VirtualiZarr/pull/349). ## Writing virtual stores to disk @@ -456,8 +501,6 @@ session.commit("Appended second dataset") See the [Icechunk documentation](https://icechunk.io/icechunk-python/virtual/#creating-a-virtual-dataset-with-virtualizarr) for more details. - - ## Opening Kerchunk references as virtual datasets You can open existing Kerchunk `json` or `parquet` references as Virtualizarr virtual datasets. This may be useful for converting existing Kerchunk formatted references to storage formats like [Icechunk](https://icechunk.io/). @@ -468,9 +511,9 @@ vds = open_virtual_dataset('combined.json', filetype='kerchunk') vds = open_virtual_dataset('combined.parquet', filetype='kerchunk') ``` -One difference between the kerchunk references format and virtualizarr's internal manifest representation (as well as icechunk's format) is that paths in kerchunk references can be relative paths. Opening kerchunk references that contain relative local filepaths therefore requires supplying another piece of information: the directory of the ``fsspec`` filesystem which the filepath was defined relative to. +One difference between the kerchunk references format and virtualizarr's internal manifest representation (as well as icechunk's format) is that paths in kerchunk references can be relative paths. Opening kerchunk references that contain relative local filepaths therefore requires supplying another piece of information: the directory of the `fsspec` filesystem which the filepath was defined relative to. -You can dis-ambuiguate kerchunk references containing relative paths by passing the ``fs_root`` kwarg to ``virtual_backend_kwargs``. +You can dis-ambuiguate kerchunk references containing relative paths by passing the `fs_root` kwarg to `virtual_backend_kwargs`. ```python # file `relative_refs.json` contains a path like './file.nc' @@ -484,7 +527,7 @@ vds = open_virtual_dataset( # the path in the virtual dataset will now be 'file:///some_directory/file.nc' ``` -Note that as the virtualizarr {py:meth}`vds.virtualize.to_kerchunk ` method only writes absolute paths, the only scenario in which you might come across references containing relative paths is if you are opening references that were previously created using the ``kerchunk`` library alone. +Note that as the virtualizarr {py:meth}`vds.virtualize.to_kerchunk ` method only writes absolute paths, the only scenario in which you might come across references containing relative paths is if you are opening references that were previously created using the `kerchunk` library alone. ## Rewriting existing manifests @@ -505,10 +548,12 @@ def local_to_s3_url(old_local_path: str) -> str: filename = Path(old_local_path).name return str(new_s3_bucket_url / filename) ``` + ```python renamed_vds = vds.virtualize.rename_paths(local_to_s3_url) renamed_vds['air'].data.manifest.dict() ``` + ``` {'0.0.0': {'path': 'http://s3.amazonaws.com/my_bucket/air.nc', 'offset': 15419, 'length': 7738000}} ``` diff --git a/virtualizarr/readers/hdf/hdf.py b/virtualizarr/readers/hdf/hdf.py index e00c570e..d39bf4a8 100644 --- a/virtualizarr/readers/hdf/hdf.py +++ b/virtualizarr/readers/hdf/hdf.py @@ -313,7 +313,6 @@ def _dataset_to_variable( codec_configs = [ _num_codec_config_to_configurable(codec.get_config()) for codec in codecs ] - metadata = ArrayV3Metadata( shape=dataset.shape, data_type=dtype, diff --git a/virtualizarr/tests/test_zarr.py b/virtualizarr/tests/test_zarr.py index 929dbe27..c6a7e6e2 100644 --- a/virtualizarr/tests/test_zarr.py +++ b/virtualizarr/tests/test_zarr.py @@ -4,7 +4,6 @@ from zarr.core.metadata.v2 import ArrayV2Metadata from virtualizarr.zarr import ( - ZArray, convert_to_codec_pipeline, convert_v3_to_v2_metadata, ) @@ -56,32 +55,6 @@ def test_convert_to_codec_pipeline(): assert config["shuffle"] == 1 -def test_replace_partial(): - arr = ZArray(shape=(2, 3), chunks=(1, 1), dtype=np.dtype(" KerchunkArrRe ] for chunk_key, entry in marr.manifest.dict().items() } - # attrs = json.loads(json.dumps(var.attrs, cls=NumpyEncoder)) - # marr.metadata.update_attributes(attrs) - # import pdb; pdb.set_trace() - array_v2_metadata = convert_v3_to_v2_metadata( - marr.metadata - ) # .update_attributes(attrs)) + array_v2_metadata = convert_v3_to_v2_metadata(marr.metadata) else: try: np_arr = var.to_numpy() diff --git a/virtualizarr/zarr.py b/virtualizarr/zarr.py index ff9d1d58..b441853e 100644 --- a/virtualizarr/zarr.py +++ b/virtualizarr/zarr.py @@ -1,4 +1,3 @@ -import dataclasses from typing import TYPE_CHECKING, Any, Literal, NewType import numpy as np @@ -37,100 +36,6 @@ """ -@dataclasses.dataclass -class ZArray: - """Just the .zarray information""" - - # TODO will this work for V3? - - shape: tuple[int, ...] - chunks: tuple[int, ...] - dtype: np.dtype - fill_value: FillValueT = dataclasses.field(default=None) - order: Literal["C", "F"] = "C" - compressor: dict | None = None - filters: list[dict] | None = None - zarr_format: Literal[2, 3] = 2 - - def __post_init__(self) -> None: - if len(self.shape) != len(self.chunks): - raise ValueError( - "Dimension mismatch between array shape and chunk shape. " - f"Array shape {self.shape} has ndim={self.shape} but chunk shape {self.chunks} has ndim={len(self.chunks)}" - ) - - if isinstance(self.dtype, str): - # Convert dtype string to numpy.dtype - self.dtype = np.dtype(self.dtype) - - if self.fill_value is None: - self.fill_value = ZARR_DEFAULT_FILL_VALUE.get(self.dtype.kind, 0.0) - - def dict(self) -> dict[str, Any]: - zarray_dict = dataclasses.asdict(self) - return zarray_dict - - def to_kerchunk_json(self) -> str: - import ujson - - zarray_dict = self.dict() - if zarray_dict["fill_value"] is np.nan: - zarray_dict["fill_value"] = None - return ujson.dumps(zarray_dict) - - # ZArray.dict seems to shadow "dict", so we need the type ignore in - # the signature below. - def replace( - self, - shape: tuple[int, ...] | None = None, - chunks: tuple[int, ...] | None = None, - dtype: np.dtype | str | None = None, - fill_value: FillValueT = None, - order: Literal["C", "F"] | None = None, - compressor: "dict | None" = None, # type: ignore[valid-type] - filters: list[dict] | None = None, # type: ignore[valid-type] - zarr_format: Literal[2, 3] | None = None, - ) -> "ZArray": - """ - Convenience method to create a new ZArray from an existing one by altering only certain attributes. - """ - replacements: dict[str, Any] = {} - if shape is not None: - replacements["shape"] = shape - if chunks is not None: - replacements["chunks"] = chunks - if dtype is not None: - replacements["dtype"] = dtype - if fill_value is not None: - replacements["fill_value"] = fill_value - if order is not None: - replacements["order"] = order - if compressor is not None: - replacements["compressor"] = compressor - if filters is not None: - replacements["filters"] = filters - if zarr_format is not None: - replacements["zarr_format"] = zarr_format - return dataclasses.replace(self, **replacements) - - def serializer(self) -> Any: - """ - testing - """ - try: - from zarr.core.metadata.v3 import ( # type: ignore[import-untyped] - parse_codecs, - ) - except ImportError: - raise ImportError("zarr v3 is required to generate v3 codec pipelines") - # https://github.com/zarr-developers/zarr-python/pull/1944#issuecomment-2151994097 - # "If no ArrayBytesCodec is supplied, we can auto-add a BytesCodec" - bytes = dict( - name="bytes", configuration={} - ) # TODO need to handle endianess configuration - return parse_codecs([bytes])[0] - - def ceildiv(a: int, b: int) -> int: """ Ceiling division operator for integers. From a0f72b2517accd63e7228a146e9a7fdaf87e1c8a Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Wed, 12 Feb 2025 14:07:08 -0800 Subject: [PATCH 38/78] Just return metadata's shape --- virtualizarr/manifests/array.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/virtualizarr/manifests/array.py b/virtualizarr/manifests/array.py index 763bea20..94b9f1f8 100644 --- a/virtualizarr/manifests/array.py +++ b/virtualizarr/manifests/array.py @@ -91,7 +91,7 @@ def shape(self) -> tuple[int, ...]: """ Array shape by number of elements along each dimension. """ - return tuple(int(length) for length in list(self.metadata.shape)) + return self.metadata.shape @property def ndim(self) -> int: From aad511f86ff1e89dd4cf6b37252bf6adf3d3d289 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Wed, 12 Feb 2025 14:15:43 -0800 Subject: [PATCH 39/78] Create update metadata function --- virtualizarr/manifests/array_api.py | 40 ++++++++++++++--------------- 1 file changed, 19 insertions(+), 21 deletions(-) diff --git a/virtualizarr/manifests/array_api.py b/virtualizarr/manifests/array_api.py index 92c4022b..a2116347 100644 --- a/virtualizarr/manifests/array_api.py +++ b/virtualizarr/manifests/array_api.py @@ -1,6 +1,7 @@ from typing import TYPE_CHECKING, Any, Callable, cast import numpy as np +from zarr.core.metadata.v3 import ArrayV3Metadata from virtualizarr.zarr import determine_chunk_grid_shape @@ -53,7 +54,6 @@ def concatenate( The signature of this function is array API compliant, so that it can be called by `xarray.concat`. """ - from zarr.core.metadata.v3 import ArrayV3Metadata from .array import ManifestArray @@ -102,10 +102,7 @@ def concatenate( lengths=concatenated_lengths, ) - metadata_copy = first_arr.metadata.to_dict().copy() - metadata_copy["shape"] = tuple(new_shape) - # ArrayV3Metadata.from_dict removes extra keys zarr_format and node_type - new_metadata = ArrayV3Metadata.from_dict(metadata_copy) + new_metadata = update_metadata(first_arr, new_shape=new_shape) return ManifestArray(chunkmanifest=concatenated_manifest, metadata=new_metadata) @@ -122,7 +119,6 @@ def stack( The signature of this function is array API compliant, so that it can be called by `xarray.stack`. """ - from zarr.core.metadata.v3 import ArrayV3Metadata from .array import ManifestArray @@ -174,16 +170,26 @@ def stack( new_chunks = list(old_chunks) new_chunks.insert(axis, 1) + new_metadata = update_metadata( + first_arr, new_shape=new_shape, new_chunks=new_chunks + ) + + return ManifestArray(chunkmanifest=stacked_manifest, metadata=new_metadata) + + +def update_metadata( + first_arr, new_shape: tuple, new_chunks: tuple | None = None +) -> ArrayV3Metadata: metadata_copy = first_arr.metadata.to_dict().copy() metadata_copy["shape"] = tuple(new_shape) - metadata_copy["chunk_grid"] = { - "name": "regular", - "configuration": {"chunk_shape": tuple(new_chunks)}, - } + if new_chunks is not None: + metadata_copy["chunk_grid"] = { + "name": "regular", + "configuration": {"chunk_shape": tuple(new_chunks)}, + } # ArrayV3Metadata.from_dict removes extra keys zarr_format and node_type new_metadata = ArrayV3Metadata.from_dict(metadata_copy) - - return ManifestArray(chunkmanifest=stacked_manifest, metadata=new_metadata) + return new_metadata @implements(np.expand_dims) @@ -198,7 +204,6 @@ def broadcast_to(x: "ManifestArray", /, shape: tuple[int, ...]) -> "ManifestArra """ Broadcasts a ManifestArray to a specified shape, by either adjusting chunk keys or copying chunk manifest entries. """ - from zarr.core.metadata.v3 import ArrayV3Metadata from .array import ManifestArray @@ -245,14 +250,7 @@ def broadcast_to(x: "ManifestArray", /, shape: tuple[int, ...]) -> "ManifestArra lengths=broadcasted_lengths, ) - metadata_copy = x.metadata.to_dict().copy() - metadata_copy["shape"] = tuple(new_shape) - metadata_copy["chunk_grid"] = { - "name": "regular", - "configuration": {"chunk_shape": tuple(new_chunk_shape)}, - } - # ArrayV3Metadata.from_dict removes extra keys zarr_format and node_type - new_metadata = ArrayV3Metadata.from_dict(metadata_copy) + new_metadata = update_metadata(x, new_shape=new_shape, new_chunks=new_chunk_shape) return ManifestArray(chunkmanifest=broadcasted_manifest, metadata=new_metadata) From b357b045357b06b25558c9ec0653f128fea36a5a Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Wed, 12 Feb 2025 14:20:54 -0800 Subject: [PATCH 40/78] Fix typing for update_metadata --- virtualizarr/manifests/array_api.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/virtualizarr/manifests/array_api.py b/virtualizarr/manifests/array_api.py index a2116347..771c73db 100644 --- a/virtualizarr/manifests/array_api.py +++ b/virtualizarr/manifests/array_api.py @@ -178,7 +178,7 @@ def stack( def update_metadata( - first_arr, new_shape: tuple, new_chunks: tuple | None = None + first_arr, new_shape: list[int], new_chunks: list[int] | None = None ) -> ArrayV3Metadata: metadata_copy = first_arr.metadata.to_dict().copy() metadata_copy["shape"] = tuple(new_shape) @@ -250,7 +250,9 @@ def broadcast_to(x: "ManifestArray", /, shape: tuple[int, ...]) -> "ManifestArra lengths=broadcasted_lengths, ) - new_metadata = update_metadata(x, new_shape=new_shape, new_chunks=new_chunk_shape) + new_metadata = update_metadata( + x, new_shape=list(new_shape), new_chunks=list(new_chunk_shape) + ) return ManifestArray(chunkmanifest=broadcasted_manifest, metadata=new_metadata) From 08e877ac005ac26282465dcc34de4d026acd486b Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Wed, 12 Feb 2025 14:34:45 -0800 Subject: [PATCH 41/78] Check for regular chunk grid in manifest instantiation --- conftest.py | 7 ++++++- virtualizarr/manifests/array.py | 12 ++++++------ .../tests/test_manifests/test_array.py | 19 +++++++++++++++++++ 3 files changed, 31 insertions(+), 7 deletions(-) diff --git a/conftest.py b/conftest.py index 1d8112e8..cf61bd91 100644 --- a/conftest.py +++ b/conftest.py @@ -219,14 +219,19 @@ def array_v3_metadata(): def _create_metadata( shape: tuple = (5, 5), chunks: tuple = (5, 5), + chunk_grid: Any | None = None, data_type: str = np.dtype("int32"), codecs: list[dict] | None = None, fill_value: int = None, ): + chunk_grid = chunk_grid or { + "name": "regular", + "configuration": {"chunk_shape": chunks}, + } return ArrayV3Metadata( shape=shape, data_type=data_type, - chunk_grid={"name": "regular", "configuration": {"chunk_shape": chunks}}, + chunk_grid=chunk_grid, chunk_key_encoding={"name": "default"}, fill_value=fill_value, codecs=convert_to_codec_pipeline( diff --git a/virtualizarr/manifests/array.py b/virtualizarr/manifests/array.py index 94b9f1f8..25399b84 100644 --- a/virtualizarr/manifests/array.py +++ b/virtualizarr/manifests/array.py @@ -46,6 +46,11 @@ def __init__( # try unpacking the dict _metadata = ArrayV3Metadata(**metadata) + if not isinstance(_metadata.chunk_grid, RegularChunkGrid): + raise NotImplementedError( + f"Only RegularChunkGrid is currently supported for chunk size, but got type {type(_metadata.chunk_grid)}" + ) + if isinstance(chunkmanifest, ChunkManifest): _chunkmanifest = chunkmanifest elif isinstance(chunkmanifest, dict): @@ -74,12 +79,7 @@ def chunks(self) -> tuple[int, ...]: """ Individual chunk size by number of elements. """ - if isinstance(self._metadata.chunk_grid, RegularChunkGrid): - return self._metadata.chunk_grid.chunk_shape - else: - raise NotImplementedError( - "Only RegularChunkGrid is currently supported for chunk size" - ) + return self._metadata.chunk_grid.chunk_shape @property def dtype(self) -> np.dtype: diff --git a/virtualizarr/tests/test_manifests/test_array.py b/virtualizarr/tests/test_manifests/test_array.py index fca94ef0..7b6ec472 100644 --- a/virtualizarr/tests/test_manifests/test_array.py +++ b/virtualizarr/tests/test_manifests/test_array.py @@ -43,6 +43,25 @@ def test_create_manifestarray_dict_v3_metadata(self, array_v3_metadata_dict): assert marr.size == 5 * 2 * 20 assert marr.ndim == 3 + def test_manifestarray_notimplementederror(self, array_v3_metadata): + from zarr.core.chunk_grids import ChunkGrid + + class IrregularChunkGrid(ChunkGrid): + pass + + metadata = array_v3_metadata(chunk_grid=IrregularChunkGrid()) + + chunks_dict = { + "0.0.0": {"path": "s3://bucket/foo.nc", "offset": 100, "length": 100}, + } + manifest = ChunkManifest(entries=chunks_dict) + + with pytest.raises( + NotImplementedError, + match="Only RegularChunkGrid is currently supported for chunk size", + ): + ManifestArray(metadata=metadata, chunkmanifest=manifest) + class TestEquals: def test_equals(self, array_v3_metadata): From f040459b0383fe57fe5d826a897df07b24c2cf28 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Wed, 12 Feb 2025 14:46:29 -0800 Subject: [PATCH 42/78] Remove obsolete codecs code --- virtualizarr/codecs.py | 49 ++---------------------------------------- 1 file changed, 2 insertions(+), 47 deletions(-) diff --git a/virtualizarr/codecs.py b/virtualizarr/codecs.py index eff6d090..353d33c9 100644 --- a/virtualizarr/codecs.py +++ b/virtualizarr/codecs.py @@ -2,6 +2,7 @@ import zarr from zarr.abc.codec import ArrayArrayCodec, ArrayBytesCodec, BytesBytesCodec +from zarr.core.metadata.v3 import ArrayV3Metadata if TYPE_CHECKING: from .manifests.array import ManifestArray @@ -27,58 +28,12 @@ def get_codecs(array: Union["ManifestArray", "zarr.Array"]) -> CodecPipeline: Raises ------ - ImportError - If `zarr` is required but not installed. ValueError If the array type is unsupported or the array's metadata is not in zarr v3 format. - NotImplementedError - If zarr-python v3 is not installed. """ - if _is_manifest_array(array): - return _get_manifestarray_codecs(array) # type: ignore[arg-type] - - if _is_zarr_array(array): - return _get_zarr_array_codecs(array) # type: ignore[arg-type] - - raise ValueError("Unsupported array type or zarr is not installed.") - - -def _is_manifest_array(array: object) -> bool: - """Check if the array is an instance of ManifestArray.""" - try: - from .manifests.array import ManifestArray - - return isinstance(array, ManifestArray) - except ImportError: - return False - - -def _get_manifestarray_codecs(array: "ManifestArray") -> CodecPipeline: - """Get zarr v3 codec pipeline for a ManifestArray.""" - if array.metadata.zarr_format != 3: - raise ValueError( - "Only zarr v3 format is supported. Please convert your array metadata to v3 format." - ) - return array.metadata.codecs - - -def _is_zarr_array(array: object) -> bool: - """Check if the array is an instance of Zarr Array.""" - try: - from zarr import Array - - return isinstance(array, Array) - except ImportError: - return False - - -def _get_zarr_array_codecs(array: "zarr.Array") -> CodecPipeline: - """Get zarr v3 codec pipeline for a Zarr Array.""" - from zarr.core.metadata import ArrayV3Metadata # type: ignore[import-untyped] - if not isinstance(array.metadata, ArrayV3Metadata): raise ValueError( "Only zarr v3 format arrays are supported. Please convert your array to v3 format." ) - return tuple(array.metadata.codecs) + return array.metadata.codecs From 495d6606d5980476af5901e53748bbd9658d3d9c Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Wed, 12 Feb 2025 14:51:54 -0800 Subject: [PATCH 43/78] Fix chunks function and add docstring --- virtualizarr/manifests/array.py | 2 +- virtualizarr/zarr.py | 22 +++++++++++++++++++--- 2 files changed, 20 insertions(+), 4 deletions(-) diff --git a/virtualizarr/manifests/array.py b/virtualizarr/manifests/array.py index 25399b84..f1c3c36d 100644 --- a/virtualizarr/manifests/array.py +++ b/virtualizarr/manifests/array.py @@ -79,7 +79,7 @@ def chunks(self) -> tuple[int, ...]: """ Individual chunk size by number of elements. """ - return self._metadata.chunk_grid.chunk_shape + return self._metadata.chunks @property def dtype(self) -> np.dtype: diff --git a/virtualizarr/zarr.py b/virtualizarr/zarr.py index b441853e..597868b7 100644 --- a/virtualizarr/zarr.py +++ b/virtualizarr/zarr.py @@ -3,6 +3,7 @@ import numpy as np from zarr.abc.codec import ArrayArrayCodec, ArrayBytesCodec, BytesBytesCodec from zarr.abc.codec import Codec as ZarrCodec +from zarr.core.codec_pipeline import BatchedCodecPipeline from zarr.core.common import JSON from zarr.core.metadata.v2 import ArrayV2Metadata from zarr.core.metadata.v3 import ArrayV3Metadata @@ -86,9 +87,6 @@ def extract_codecs( return (arrayarray_codecs, arraybytes_codec, bytesbytes_codecs) -from zarr.core.codec_pipeline import BatchedCodecPipeline - - def convert_to_codec_pipeline( dtype: np.dtype, codecs: list[dict] | None = [], @@ -229,6 +227,24 @@ def to_kerchunk_json(v2_metadata: ArrayV2Metadata) -> str: def from_kerchunk_refs(decoded_arr_refs_zarray) -> "ArrayV3Metadata": + """ + Convert a decoded Zarr array reference to an ArrayV3Metadata object. + This function processes the given decoded Zarr array reference dictionary, + to construct and return an ArrayV3Metadata object based on the provided information. + Parameters: + ---------- + decoded_arr_refs_zarray : dict + A dictionary containing the decoded Zarr array reference information. + Expected keys include "dtype", "fill_value", "zarr_format", "filters", + "compressor", "chunks", and "shape". + Returns: + ------- + ArrayV3Metadata + Raises: + ------ + ValueError + If the Zarr format specified in the input dictionary is not 2 or 3. + """ # coerce type of fill_value as kerchunk can be inconsistent with this dtype = np.dtype(decoded_arr_refs_zarray["dtype"]) fill_value = decoded_arr_refs_zarray["fill_value"] From a262f0bae6372851381dd0f3e7d3b1a62d7fbc09 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Wed, 12 Feb 2025 15:02:33 -0800 Subject: [PATCH 44/78] Remove custom zattrs type --- virtualizarr/translators/kerchunk.py | 5 ++-- virtualizarr/zarr.py | 37 +++++----------------------- 2 files changed, 9 insertions(+), 33 deletions(-) diff --git a/virtualizarr/translators/kerchunk.py b/virtualizarr/translators/kerchunk.py index d30832b9..374866d1 100644 --- a/virtualizarr/translators/kerchunk.py +++ b/virtualizarr/translators/kerchunk.py @@ -11,7 +11,7 @@ KerchunkArrRefs, KerchunkStoreRefs, ) -from virtualizarr.zarr import ZAttrs, determine_chunk_grid_shape +from virtualizarr.zarr import determine_chunk_grid_shape def virtual_vars_and_metadata_from_kerchunk_refs( @@ -265,6 +265,7 @@ def extract_array_refs( ) +from zarr.core.common import JSON from zarr.core.metadata import ArrayV3Metadata from virtualizarr.zarr import from_kerchunk_refs @@ -272,7 +273,7 @@ def extract_array_refs( def parse_array_refs( arr_refs: KerchunkArrRefs, -) -> tuple[dict, ArrayV3Metadata, ZAttrs]: +) -> tuple[dict, ArrayV3Metadata, dict[str, JSON]]: metadata = from_kerchunk_refs(arr_refs.pop(".zarray")) zattrs = arr_refs.pop(".zattrs", {}) chunk_dict = arr_refs diff --git a/virtualizarr/zarr.py b/virtualizarr/zarr.py index 597868b7..ec6c68bc 100644 --- a/virtualizarr/zarr.py +++ b/virtualizarr/zarr.py @@ -1,4 +1,4 @@ -from typing import TYPE_CHECKING, Any, Literal, NewType +from typing import Any import numpy as np from zarr.abc.codec import ArrayArrayCodec, ArrayBytesCodec, BytesBytesCodec @@ -6,35 +6,9 @@ from zarr.core.codec_pipeline import BatchedCodecPipeline from zarr.core.common import JSON from zarr.core.metadata.v2 import ArrayV2Metadata -from zarr.core.metadata.v3 import ArrayV3Metadata - -if TYPE_CHECKING: - try: - from zarr.abc.codec import Codec as ZarrCodec - except ImportError: - pass - -# TODO replace these with classes imported directly from Zarr? (i.e. Zarr Object Models) -ZAttrs = NewType( - "ZAttrs", dict[str, Any] -) # just the .zattrs (for one array or for the whole store/group) -FillValueT = bool | str | float | int | list | None -ZARR_FORMAT = Literal[2, 3] - -ZARR_DEFAULT_FILL_VALUE: dict[str, FillValueT] = { - # numpy dtypes's hierarchy lets us avoid checking for all the widths - # https://numpy.org/doc/stable/reference/arrays.scalars.html - np.dtype("bool").kind: False, - np.dtype("int").kind: 0, - np.dtype("float").kind: 0.0, - np.dtype("complex").kind: [0.0, 0.0], - np.dtype("datetime64").kind: 0, -} -""" -The value and format of the fill_value depend on the `data_type` of the array. -See here for spec: -https://zarr-specs.readthedocs.io/en/latest/v3/core/v3.0.html#fill-value -""" +from zarr.core.metadata.v3 import ( + ArrayV3Metadata, # just the .zattrs (for one array or for the whole store/group) +) def ceildiv(a: int, b: int) -> int: @@ -228,9 +202,10 @@ def to_kerchunk_json(v2_metadata: ArrayV2Metadata) -> str: def from_kerchunk_refs(decoded_arr_refs_zarray) -> "ArrayV3Metadata": """ - Convert a decoded Zarr array reference to an ArrayV3Metadata object. + Convert a decoded zarr array (.zarray) reference to an ArrayV3Metadata object. This function processes the given decoded Zarr array reference dictionary, to construct and return an ArrayV3Metadata object based on the provided information. + Parameters: ---------- decoded_arr_refs_zarray : dict From bcd68a00bc95f90169bec8caf9cf74e5f4b07103 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Wed, 12 Feb 2025 15:14:39 -0800 Subject: [PATCH 45/78] Move some imports and make update_metadata a private method --- virtualizarr/manifests/array_api.py | 11 +++++++---- virtualizarr/readers/hdf/hdf.py | 14 +++++--------- virtualizarr/translators/kerchunk.py | 10 +++------- 3 files changed, 15 insertions(+), 20 deletions(-) diff --git a/virtualizarr/manifests/array_api.py b/virtualizarr/manifests/array_api.py index 771c73db..a5701388 100644 --- a/virtualizarr/manifests/array_api.py +++ b/virtualizarr/manifests/array_api.py @@ -102,7 +102,7 @@ def concatenate( lengths=concatenated_lengths, ) - new_metadata = update_metadata(first_arr, new_shape=new_shape) + new_metadata = _update_metadata(first_arr, new_shape=new_shape) return ManifestArray(chunkmanifest=concatenated_manifest, metadata=new_metadata) @@ -170,16 +170,19 @@ def stack( new_chunks = list(old_chunks) new_chunks.insert(axis, 1) - new_metadata = update_metadata( + new_metadata = _update_metadata( first_arr, new_shape=new_shape, new_chunks=new_chunks ) return ManifestArray(chunkmanifest=stacked_manifest, metadata=new_metadata) -def update_metadata( +def _update_metadata( first_arr, new_shape: list[int], new_chunks: list[int] | None = None ) -> ArrayV3Metadata: + """ + Update the metadata of a ManifestArray to reflect a new shape and/or chunk shape. + """ metadata_copy = first_arr.metadata.to_dict().copy() metadata_copy["shape"] = tuple(new_shape) if new_chunks is not None: @@ -250,7 +253,7 @@ def broadcast_to(x: "ManifestArray", /, shape: tuple[int, ...]) -> "ManifestArra lengths=broadcasted_lengths, ) - new_metadata = update_metadata( + new_metadata = _update_metadata( x, new_shape=list(new_shape), new_chunks=list(new_chunk_shape) ) diff --git a/virtualizarr/readers/hdf/hdf.py b/virtualizarr/readers/hdf/hdf.py index d39bf4a8..81d072b1 100644 --- a/virtualizarr/readers/hdf/hdf.py +++ b/virtualizarr/readers/hdf/hdf.py @@ -13,6 +13,7 @@ import numpy as np import xarray as xr +from zarr.core.metadata.v3 import ArrayV3Metadata from virtualizarr.manifests import ( ChunkEntry, @@ -28,6 +29,10 @@ from virtualizarr.readers.hdf.filters import cfcodec_from_dataset, codecs_from_dataset from virtualizarr.types import ChunkKey from virtualizarr.utils import _FsspecFSFromFilepath, check_for_collisions, soft_import +from virtualizarr.zarr import ( + _num_codec_config_to_configurable, + convert_to_codec_pipeline, +) h5py = soft_import("h5py", "For reading hdf files", strict=False) @@ -282,15 +287,6 @@ def _dataset_to_variable( list: xarray.Variable A list of xarray variables. """ - # This chunk determination logic mirrors zarr-python's create - # https://github.com/zarr-developers/zarr-python/blob/main/zarr/creation.py#L62-L66 - from zarr.core.metadata.v3 import ArrayV3Metadata - - from virtualizarr.zarr import ( - _num_codec_config_to_configurable, - convert_to_codec_pipeline, - ) - chunks = dataset.chunks if dataset.chunks else dataset.shape codecs = codecs_from_dataset(dataset) cfcodec = cfcodec_from_dataset(dataset) diff --git a/virtualizarr/translators/kerchunk.py b/virtualizarr/translators/kerchunk.py index 374866d1..dc751dc4 100644 --- a/virtualizarr/translators/kerchunk.py +++ b/virtualizarr/translators/kerchunk.py @@ -3,6 +3,8 @@ from xarray import Dataset from xarray.core.indexes import Index from xarray.core.variable import Variable +from zarr.core.common import JSON +from zarr.core.metadata import ArrayV3Metadata from virtualizarr.manifests import ChunkManifest, ManifestArray from virtualizarr.manifests.manifest import ChunkEntry, ChunkKey @@ -11,7 +13,7 @@ KerchunkArrRefs, KerchunkStoreRefs, ) -from virtualizarr.zarr import determine_chunk_grid_shape +from virtualizarr.zarr import determine_chunk_grid_shape, from_kerchunk_refs def virtual_vars_and_metadata_from_kerchunk_refs( @@ -265,12 +267,6 @@ def extract_array_refs( ) -from zarr.core.common import JSON -from zarr.core.metadata import ArrayV3Metadata - -from virtualizarr.zarr import from_kerchunk_refs - - def parse_array_refs( arr_refs: KerchunkArrRefs, ) -> tuple[dict, ArrayV3Metadata, dict[str, JSON]]: From f0ce77893f872f5ce19406054b6f817ccfa6f411 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Wed, 12 Feb 2025 15:49:38 -0800 Subject: [PATCH 46/78] Remove zarr.py --- conftest.py | 3 +- virtualizarr/codecs.py | 96 ++++++- virtualizarr/manifests/array_api.py | 2 +- virtualizarr/readers/dmrpp.py | 2 +- virtualizarr/readers/hdf/hdf.py | 10 +- virtualizarr/tests/test_codecs.py | 49 +++- .../tests/test_writers/test_kerchunk.py | 34 +++ virtualizarr/tests/test_zarr.py | 87 ------ virtualizarr/translators/kerchunk.py | 87 +++++- virtualizarr/utils.py | 16 ++ virtualizarr/writers/icechunk.py | 2 +- virtualizarr/writers/kerchunk.py | 58 +++- virtualizarr/zarr.py | 259 ------------------ 13 files changed, 345 insertions(+), 360 deletions(-) delete mode 100644 virtualizarr/tests/test_zarr.py delete mode 100644 virtualizarr/zarr.py diff --git a/conftest.py b/conftest.py index cf61bd91..ea59b3b6 100644 --- a/conftest.py +++ b/conftest.py @@ -10,9 +10,10 @@ from xarray.core.variable import Variable from zarr.core.metadata.v3 import ArrayV3Metadata +from virtualizarr.codecs import convert_to_codec_pipeline from virtualizarr.manifests import ChunkManifest, ManifestArray from virtualizarr.manifests.manifest import join -from virtualizarr.zarr import ceildiv, convert_to_codec_pipeline +from virtualizarr.utils import ceildiv def pytest_addoption(parser): diff --git a/virtualizarr/codecs.py b/virtualizarr/codecs.py index 353d33c9..27d2f70e 100644 --- a/virtualizarr/codecs.py +++ b/virtualizarr/codecs.py @@ -1,7 +1,10 @@ -from typing import TYPE_CHECKING, Tuple, Union +from typing import TYPE_CHECKING, Any, Tuple, Union +import numpy as np import zarr from zarr.abc.codec import ArrayArrayCodec, ArrayBytesCodec, BytesBytesCodec +from zarr.abc.codec import Codec as ZarrCodec +from zarr.core.codec_pipeline import BatchedCodecPipeline from zarr.core.metadata.v3 import ArrayV3Metadata if TYPE_CHECKING: @@ -12,6 +15,97 @@ ] +def num_codec_config_to_configurable(num_codec: dict) -> dict: + """ + Convert a numcodecs codec into a zarr v3 configurable. + """ + if num_codec["id"].startswith("numcodecs."): + return num_codec + + num_codec_copy = num_codec.copy() + name = "numcodecs." + num_codec_copy.pop("id") + return {"name": name, "configuration": num_codec_copy} + + +def extract_codecs( + codecs: CodecPipeline, +) -> tuple[ + tuple[ArrayArrayCodec, ...], ArrayBytesCodec | None, tuple[BytesBytesCodec, ...] +]: + """Extracts various codec types.""" + arrayarray_codecs: tuple[ArrayArrayCodec, ...] = () + arraybytes_codec: ArrayBytesCodec | None = None + bytesbytes_codecs: tuple[BytesBytesCodec, ...] = () + for codec in codecs: + if isinstance(codec, ArrayArrayCodec): + arrayarray_codecs += (codec,) + if isinstance(codec, ArrayBytesCodec): + arraybytes_codec = codec + if isinstance(codec, BytesBytesCodec): + bytesbytes_codecs += (codec,) + return (arrayarray_codecs, arraybytes_codec, bytesbytes_codecs) + + +def convert_to_codec_pipeline( + dtype: np.dtype, + codecs: list[dict] | None = [], +) -> BatchedCodecPipeline: + """ + Convert compressor, filters, serializer, and dtype to a pipeline of ZarrCodecs. + + Parameters + ---------- + dtype : Any + The data type. + codecs: list[ZarrCodec] | None + + Returns + ------- + BatchedCodecPipeline + """ + from zarr.core.array import _get_default_chunk_encoding_v3 + from zarr.registry import get_codec_class + + zarr_codecs: tuple[ArrayArrayCodec | ArrayBytesCodec | BytesBytesCodec, ...] = () + if codecs and len(codecs) > 0: + zarr_codecs = tuple( + get_codec_class(codec["name"]).from_dict(codec) for codec in codecs + ) + + arrayarray_codecs, arraybytes_codec, bytesbytes_codecs = extract_codecs(zarr_codecs) + + if arraybytes_codec is None: + arraybytes_codec = _get_default_chunk_encoding_v3(dtype)[1] + + codec_pipeline = BatchedCodecPipeline( + array_array_codecs=arrayarray_codecs, + array_bytes_codec=arraybytes_codec, + bytes_bytes_codecs=bytesbytes_codecs, + batch_size=1, + ) + + return codec_pipeline + + +def _get_codec_config(codec: ZarrCodec) -> dict[str, Any]: + """ + Extract configuration from a codec, handling both zarr-python and numcodecs codecs. + """ + if hasattr(codec, "codec_config"): + return codec.codec_config + elif hasattr(codec, "get_config"): + return codec.get_config() + elif hasattr(codec, "codec_name"): + # If we can't get config, try to get the name and configuration directly + # This assumes the codec follows the v3 spec format + return { + "id": codec.codec_name.replace("numcodecs.", ""), + **getattr(codec, "configuration", {}), + } + else: + raise ValueError(f"Unable to parse codec configuration: {codec}") + + def get_codecs(array: Union["ManifestArray", "zarr.Array"]) -> CodecPipeline: """ Get the zarr v3 codec pipeline for either a ManifestArray or a Zarr Array. diff --git a/virtualizarr/manifests/array_api.py b/virtualizarr/manifests/array_api.py index a5701388..ba489d60 100644 --- a/virtualizarr/manifests/array_api.py +++ b/virtualizarr/manifests/array_api.py @@ -3,7 +3,7 @@ import numpy as np from zarr.core.metadata.v3 import ArrayV3Metadata -from virtualizarr.zarr import determine_chunk_grid_shape +from virtualizarr.utils import determine_chunk_grid_shape from .manifest import ChunkManifest from .utils import ( diff --git a/virtualizarr/readers/dmrpp.py b/virtualizarr/readers/dmrpp.py index 6b58dadb..95989ad6 100644 --- a/virtualizarr/readers/dmrpp.py +++ b/virtualizarr/readers/dmrpp.py @@ -379,7 +379,7 @@ def _parse_variable(self, var_tag: ET.Element) -> Variable: """ from zarr.core.metadata.v3 import ArrayV3Metadata - from virtualizarr.zarr import convert_to_codec_pipeline + from virtualizarr.codecs import convert_to_codec_pipeline # Dimension info dims: dict[str, int] = {} diff --git a/virtualizarr/readers/hdf/hdf.py b/virtualizarr/readers/hdf/hdf.py index 81d072b1..a63ffc2c 100644 --- a/virtualizarr/readers/hdf/hdf.py +++ b/virtualizarr/readers/hdf/hdf.py @@ -15,6 +15,10 @@ import xarray as xr from zarr.core.metadata.v3 import ArrayV3Metadata +from virtualizarr.codecs import ( + convert_to_codec_pipeline, + num_codec_config_to_configurable, +) from virtualizarr.manifests import ( ChunkEntry, ChunkManifest, @@ -29,10 +33,6 @@ from virtualizarr.readers.hdf.filters import cfcodec_from_dataset, codecs_from_dataset from virtualizarr.types import ChunkKey from virtualizarr.utils import _FsspecFSFromFilepath, check_for_collisions, soft_import -from virtualizarr.zarr import ( - _num_codec_config_to_configurable, - convert_to_codec_pipeline, -) h5py = soft_import("h5py", "For reading hdf files", strict=False) @@ -307,7 +307,7 @@ def _dataset_to_variable( if isinstance(fill_value, np.generic): fill_value = fill_value.item() codec_configs = [ - _num_codec_config_to_configurable(codec.get_config()) for codec in codecs + num_codec_config_to_configurable(codec.get_config()) for codec in codecs ] metadata = ArrayV3Metadata( shape=dataset.shape, diff --git a/virtualizarr/tests/test_codecs.py b/virtualizarr/tests/test_codecs.py index ef2715ed..76f2cd12 100644 --- a/virtualizarr/tests/test_codecs.py +++ b/virtualizarr/tests/test_codecs.py @@ -1,9 +1,10 @@ import numpy as np import pytest from zarr.codecs import BytesCodec +from zarr.core.codec_pipeline import BatchedCodecPipeline from zarr.registry import get_codec_class -from virtualizarr.codecs import get_codecs +from virtualizarr.codecs import convert_to_codec_pipeline, get_codecs arrayarray_codec = {"name": "numcodecs.delta", "configuration": {"dtype": " str: + """Convert V2 metadata to kerchunk JSON format.""" + import json + + from virtualizarr.writers.kerchunk import NumpyEncoder + + zarray_dict: dict[str, JSON] = v2_metadata.to_dict() + if v2_metadata.filters: + zarray_dict["filters"] = [ + # we could also cast to json, but get_config is intended for serialization + codec.get_config() + for codec in v2_metadata.filters + if codec is not None + ] # type: ignore[assignment] + if v2_metadata.compressor: + zarray_dict["compressor"] = v2_metadata.compressor.get_config() # type: ignore[assignment] + + return json.dumps(zarray_dict, separators=(",", ":"), cls=NumpyEncoder) + + +def from_kerchunk_refs(decoded_arr_refs_zarray) -> "ArrayV3Metadata": + """ + Convert a decoded zarr array (.zarray) reference to an ArrayV3Metadata object. + This function processes the given decoded Zarr array reference dictionary, + to construct and return an ArrayV3Metadata object based on the provided information. + + Parameters: + ---------- + decoded_arr_refs_zarray : dict + A dictionary containing the decoded Zarr array reference information. + Expected keys include "dtype", "fill_value", "zarr_format", "filters", + "compressor", "chunks", and "shape". + Returns: + ------- + ArrayV3Metadata + Raises: + ------ + ValueError + If the Zarr format specified in the input dictionary is not 2 or 3. + """ + # coerce type of fill_value as kerchunk can be inconsistent with this + dtype = np.dtype(decoded_arr_refs_zarray["dtype"]) + fill_value = decoded_arr_refs_zarray["fill_value"] + if np.issubdtype(dtype, np.floating) and ( + fill_value is None or fill_value == "NaN" or fill_value == "nan" + ): + fill_value = np.nan + + zarr_format = int(decoded_arr_refs_zarray["zarr_format"]) + if zarr_format not in (2, 3): + raise ValueError(f"Zarr format must be 2 or 3, but got {zarr_format}") + filters = ( + decoded_arr_refs_zarray.get("filters", []) or [] + ) # Ensure filters is a list + compressor = decoded_arr_refs_zarray.get("compressor") # Might be None + + # Ensure compressor is a list before unpacking + codec_configs = [*filters, *(compressor if compressor is not None else [])] + numcodec_configs = [ + num_codec_config_to_configurable(config) for config in codec_configs + ] + return ArrayV3Metadata( + chunk_grid={ + "name": "regular", + "configuration": {"chunk_shape": tuple(decoded_arr_refs_zarray["chunks"])}, + }, + codecs=convert_to_codec_pipeline( + dtype=dtype, + codecs=numcodec_configs, + ), + data_type=dtype, + fill_value=fill_value, + shape=tuple(decoded_arr_refs_zarray["shape"]), + chunk_key_encoding={"name": "default"}, + attributes={}, + dimension_names=None, + ) def virtual_vars_and_metadata_from_kerchunk_refs( diff --git a/virtualizarr/utils.py b/virtualizarr/utils.py index b5ae3447..4f6c4773 100644 --- a/virtualizarr/utils.py +++ b/virtualizarr/utils.py @@ -100,3 +100,19 @@ def soft_import(name: str, reason: str, strict: Optional[bool] = True): ) else: return None + + +def ceildiv(a: int, b: int) -> int: + """ + Ceiling division operator for integers. + + See https://stackoverflow.com/questions/14822184/is-there-a-ceiling-equivalent-of-operator-in-python + """ + return -(a // -b) + + +def determine_chunk_grid_shape( + shape: tuple[int, ...], chunks: tuple[int, ...] +) -> tuple[int, ...]: + """Calculate the shape of the chunk grid based on array shape and chunk size.""" + return tuple(ceildiv(length, chunksize) for length, chunksize in zip(shape, chunks)) diff --git a/virtualizarr/writers/icechunk.py b/virtualizarr/writers/icechunk.py index 5872bcd2..1c044465 100644 --- a/virtualizarr/writers/icechunk.py +++ b/virtualizarr/writers/icechunk.py @@ -211,7 +211,7 @@ def write_virtual_variable_to_icechunk( """Write a single virtual variable into an icechunk store""" from zarr import Array - from virtualizarr.zarr import extract_codecs + from virtualizarr.codecs import extract_codecs ma = cast(ManifestArray, var.data) metadata = ma.metadata diff --git a/virtualizarr/writers/kerchunk.py b/virtualizarr/writers/kerchunk.py index 6a6386be..3c01711f 100644 --- a/virtualizarr/writers/kerchunk.py +++ b/virtualizarr/writers/kerchunk.py @@ -1,15 +1,17 @@ import base64 import json -from typing import cast +from typing import Any, cast import numpy as np from xarray import Dataset from xarray.coding.times import CFDatetimeCoder from xarray.core.variable import Variable +from zarr.abc.codec import ArrayArrayCodec, BytesBytesCodec +from zarr.core.metadata import ArrayV2Metadata, ArrayV3Metadata +from virtualizarr.codecs import _get_codec_config, extract_codecs from virtualizarr.manifests.manifest import join from virtualizarr.types.kerchunk import KerchunkArrRefs, KerchunkStoreRefs -from virtualizarr.zarr import convert_v3_to_v2_metadata, to_kerchunk_json class NumpyEncoder(json.JSONEncoder): @@ -88,6 +90,57 @@ def remove_file_uri_prefix(path: str): return path +def convert_v3_to_v2_metadata( + v3_metadata: ArrayV3Metadata, fill_value: Any = None +) -> ArrayV2Metadata: + """ + Convert ArrayV3Metadata to ArrayV2Metadata. + + Parameters + ---------- + v3_metadata : ArrayV3Metadata + The metadata object in v3 format. + fill_value : Any, optional + Override the fill value from v3 metadata. + + Returns + ------- + ArrayV2Metadata + The metadata object in v2 format. + """ + import warnings + + array_filters: tuple[ArrayArrayCodec, ...] + bytes_compressors: tuple[BytesBytesCodec, ...] + array_filters, _, bytes_compressors = extract_codecs(v3_metadata.codecs) + + # Handle compressor configuration + compressor_config: dict[str, Any] | None = None + if bytes_compressors: + if len(bytes_compressors) > 1: + warnings.warn( + "Multiple compressors found in v3 metadata. Using the first compressor, " + "others will be ignored. This may affect data compatibility.", + UserWarning, + ) + compressor_config = _get_codec_config(bytes_compressors[0]) + + # Handle filter configurations + filter_configs = [_get_codec_config(filter_) for filter_ in array_filters] + v2_metadata = ArrayV2Metadata( + shape=v3_metadata.shape, + dtype=v3_metadata.data_type.to_numpy(), + chunks=v3_metadata.chunks, + fill_value=fill_value or v3_metadata.fill_value, + compressor=compressor_config, + filters=filter_configs, + order="C", + attributes=v3_metadata.attributes, + dimension_separator=".", # Assuming '.' as default dimension separator + ) + return v2_metadata + + def variable_to_kerchunk_arr_refs(var: Variable, var_name: str) -> KerchunkArrRefs: """ Create a dictionary containing kerchunk-style array references from a single xarray.Variable (which wraps either a ManifestArray or a numpy array). @@ -95,6 +148,7 @@ def variable_to_kerchunk_arr_refs(var: Variable, var_name: str) -> KerchunkArrRe Partially encodes the inner dicts to json to match kerchunk behaviour (see https://github.com/fsspec/kerchunk/issues/415). """ from virtualizarr.manifests import ManifestArray + from virtualizarr.translators.kerchunk import to_kerchunk_json if isinstance(var.data, ManifestArray): marr = var.data diff --git a/virtualizarr/zarr.py b/virtualizarr/zarr.py deleted file mode 100644 index ec6c68bc..00000000 --- a/virtualizarr/zarr.py +++ /dev/null @@ -1,259 +0,0 @@ -from typing import Any - -import numpy as np -from zarr.abc.codec import ArrayArrayCodec, ArrayBytesCodec, BytesBytesCodec -from zarr.abc.codec import Codec as ZarrCodec -from zarr.core.codec_pipeline import BatchedCodecPipeline -from zarr.core.common import JSON -from zarr.core.metadata.v2 import ArrayV2Metadata -from zarr.core.metadata.v3 import ( - ArrayV3Metadata, # just the .zattrs (for one array or for the whole store/group) -) - - -def ceildiv(a: int, b: int) -> int: - """ - Ceiling division operator for integers. - - See https://stackoverflow.com/questions/14822184/is-there-a-ceiling-equivalent-of-operator-in-python - """ - return -(a // -b) - - -def determine_chunk_grid_shape( - shape: tuple[int, ...], chunks: tuple[int, ...] -) -> tuple[int, ...]: - return tuple(ceildiv(length, chunksize) for length, chunksize in zip(shape, chunks)) - - -def _num_codec_config_to_configurable(num_codec: dict) -> dict: - """ - Convert a numcodecs codec into a zarr v3 configurable. - """ - if num_codec["id"].startswith("numcodecs."): - return num_codec - - num_codec_copy = num_codec.copy() - name = "numcodecs." + num_codec_copy.pop("id") - # name = num_codec_copy.pop("id") - return {"name": name, "configuration": num_codec_copy} - - -from virtualizarr.codecs import CodecPipeline - - -def extract_codecs( - codecs: CodecPipeline, -) -> tuple[ - tuple[ArrayArrayCodec, ...], ArrayBytesCodec | None, tuple[BytesBytesCodec, ...] -]: - """Extracts various codec types.""" - arrayarray_codecs: tuple[ArrayArrayCodec, ...] = () - arraybytes_codec: ArrayBytesCodec | None = None - bytesbytes_codecs: tuple[BytesBytesCodec, ...] = () - for codec in codecs: - if isinstance(codec, ArrayArrayCodec): - arrayarray_codecs += (codec,) - if isinstance(codec, ArrayBytesCodec): - arraybytes_codec = codec - if isinstance(codec, BytesBytesCodec): - bytesbytes_codecs += (codec,) - return (arrayarray_codecs, arraybytes_codec, bytesbytes_codecs) - - -def convert_to_codec_pipeline( - dtype: np.dtype, - codecs: list[dict] | None = [], -) -> BatchedCodecPipeline: - """ - Convert compressor, filters, serializer, and dtype to a pipeline of ZarrCodecs. - - Parameters - ---------- - dtype : Any - The data type. - codecs: list[ZarrCodec] | None - - Returns - ------- - BatchedCodecPipeline - """ - from zarr.core.array import _get_default_chunk_encoding_v3 - from zarr.registry import get_codec_class - - zarr_codecs: tuple[ArrayArrayCodec | ArrayBytesCodec | BytesBytesCodec, ...] = () - if codecs and len(codecs) > 0: - zarr_codecs = tuple( - get_codec_class(codec["name"]).from_dict(codec) for codec in codecs - ) - - # (aimee): I would like to use zarr.core.codec_pipeline.codecs_from_list here but it requires array array codecs before array bytes codecs, - # which I don't think is actually a requirement. - arrayarray_codecs, arraybytes_codec, bytesbytes_codecs = extract_codecs(zarr_codecs) - - if arraybytes_codec is None: - arraybytes_codec = _get_default_chunk_encoding_v3(dtype)[1] - - codec_pipeline = BatchedCodecPipeline( - array_array_codecs=arrayarray_codecs, - array_bytes_codec=arraybytes_codec, - bytes_bytes_codecs=bytesbytes_codecs, - batch_size=1, - ) - - return codec_pipeline - - -def _get_codec_config(codec: ZarrCodec) -> dict[str, JSON]: - """ - Extract configuration from a codec, handling both zarr-python and numcodecs codecs. - """ - if hasattr(codec, "codec_config"): - return codec.codec_config - elif hasattr(codec, "get_config"): - return codec.get_config() - elif hasattr(codec, "codec_name"): - # If we can't get config, try to get the name and configuration directly - # This assumes the codec follows the v3 spec format - return { - "id": codec.codec_name.replace("numcodecs.", ""), - **getattr(codec, "configuration", {}), - } - else: - raise ValueError(f"Unable to parse codec configuration: {codec}") - - -def convert_v3_to_v2_metadata( - v3_metadata: ArrayV3Metadata, fill_value: Any = None -) -> ArrayV2Metadata: - """ - Convert ArrayV3Metadata to ArrayV2Metadata. - - Parameters - ---------- - v3_metadata : ArrayV3Metadata - The metadata object in v3 format. - fill_value : Any, optional - Override the fill value from v3 metadata. - - Returns - ------- - ArrayV2Metadata - The metadata object in v2 format. - - Notes - ----- - The conversion handles the following cases: - - Extracts compressor and filter configurations from v3 codecs - - Preserves codec configurations regardless of codec implementation - - Maintains backward compatibility with both zarr-python and numcodecs - """ - import warnings - - array_filters: tuple[ArrayArrayCodec, ...] - bytes_compressors: tuple[BytesBytesCodec, ...] - array_filters, _, bytes_compressors = extract_codecs(v3_metadata.codecs) - - # Handle compressor configuration - compressor_config: dict[str, JSON] | None = None - if bytes_compressors: - if len(bytes_compressors) > 1: - warnings.warn( - "Multiple compressors found in v3 metadata. Using the first compressor, " - "others will be ignored. This may affect data compatibility.", - UserWarning, - ) - compressor_config = _get_codec_config(bytes_compressors[0]) - - # Handle filter configurations - filter_configs = [_get_codec_config(filter_) for filter_ in array_filters] - v2_metadata = ArrayV2Metadata( - shape=v3_metadata.shape, - dtype=v3_metadata.data_type.to_numpy(), - chunks=v3_metadata.chunks, - fill_value=fill_value or v3_metadata.fill_value, - compressor=compressor_config, - filters=filter_configs, - order="C", - attributes=v3_metadata.attributes, - dimension_separator=".", # Assuming '.' as default dimension separator - ) - return v2_metadata - - -def to_kerchunk_json(v2_metadata: ArrayV2Metadata) -> str: - import json - - from virtualizarr.writers.kerchunk import NumpyEncoder - - zarray_dict: dict[str, JSON] = v2_metadata.to_dict() - if v2_metadata.filters: - zarray_dict["filters"] = [ - # we could also cast to json, but get_config is intended for serialization - codec.get_config() - for codec in v2_metadata.filters - if codec is not None - ] # type: ignore[assignment] - if v2_metadata.compressor: - zarray_dict["compressor"] = v2_metadata.compressor.get_config() # type: ignore[assignment] - - return json.dumps(zarray_dict, separators=(",", ":"), cls=NumpyEncoder) - - -def from_kerchunk_refs(decoded_arr_refs_zarray) -> "ArrayV3Metadata": - """ - Convert a decoded zarr array (.zarray) reference to an ArrayV3Metadata object. - This function processes the given decoded Zarr array reference dictionary, - to construct and return an ArrayV3Metadata object based on the provided information. - - Parameters: - ---------- - decoded_arr_refs_zarray : dict - A dictionary containing the decoded Zarr array reference information. - Expected keys include "dtype", "fill_value", "zarr_format", "filters", - "compressor", "chunks", and "shape". - Returns: - ------- - ArrayV3Metadata - Raises: - ------ - ValueError - If the Zarr format specified in the input dictionary is not 2 or 3. - """ - # coerce type of fill_value as kerchunk can be inconsistent with this - dtype = np.dtype(decoded_arr_refs_zarray["dtype"]) - fill_value = decoded_arr_refs_zarray["fill_value"] - if np.issubdtype(dtype, np.floating) and ( - fill_value is None or fill_value == "NaN" or fill_value == "nan" - ): - fill_value = np.nan - - zarr_format = int(decoded_arr_refs_zarray["zarr_format"]) - if zarr_format not in (2, 3): - raise ValueError(f"Zarr format must be 2 or 3, but got {zarr_format}") - filters = ( - decoded_arr_refs_zarray.get("filters", []) or [] - ) # Ensure filters is a list - compressor = decoded_arr_refs_zarray.get("compressor") # Might be None - - # Ensure compressor is a list before unpacking - codec_configs = [*filters, *(compressor if compressor is not None else [])] - numcodec_configs = [ - _num_codec_config_to_configurable(config) for config in codec_configs - ] - return ArrayV3Metadata( - chunk_grid={ - "name": "regular", - "configuration": {"chunk_shape": tuple(decoded_arr_refs_zarray["chunks"])}, - }, - codecs=convert_to_codec_pipeline( - dtype=dtype, - codecs=numcodec_configs, - ), - data_type=dtype, - fill_value=fill_value, - shape=tuple(decoded_arr_refs_zarray["shape"]), - chunk_key_encoding={"name": "default"}, - attributes={}, - dimension_names=None, - ) From 0518488b48b9ea5c125f01200d7f086d50031398 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Thu, 13 Feb 2025 08:09:35 -0800 Subject: [PATCH 47/78] Add zarr to other ci env files --- ci/environment.yml | 1 + ci/upstream.yml | 1 + 2 files changed, 2 insertions(+) diff --git a/ci/environment.yml b/ci/environment.yml index df444d95..581e275f 100644 --- a/ci/environment.yml +++ b/ci/environment.yml @@ -36,5 +36,6 @@ dependencies: # for opening FITS files - astropy - pip + - zarr>=3.0.2 - pip: - imagecodecs-numcodecs==2024.6.1 diff --git a/ci/upstream.yml b/ci/upstream.yml index 2da63c9b..c6c13d30 100644 --- a/ci/upstream.yml +++ b/ci/upstream.yml @@ -28,6 +28,7 @@ dependencies: - pooch - fsspec - pip + - zarr>=3.0.2 - pip: - git+https://github.com/earth-mover/icechunk.git@main#subdirectory=icechunk-python # Installs zarr-python v3.0.0 as dependency - git+https://github.com/fsspec/kerchunk.git@main From 07129798bc1b5732daaff5822d9a4a12693dd9cd Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Thu, 13 Feb 2025 08:53:34 -0800 Subject: [PATCH 48/78] Fixture array_v3_metadata uses array_v3_metadata_dict --- conftest.py | 82 ++++++++++--------- virtualizarr/codecs.py | 2 +- .../tests/test_manifests/test_array.py | 37 ++++----- virtualizarr/tests/test_writers/conftest.py | 5 +- .../tests/test_writers/test_icechunk.py | 10 ++- .../tests/test_writers/test_kerchunk.py | 1 + virtualizarr/writers/kerchunk.py | 6 +- 7 files changed, 77 insertions(+), 66 deletions(-) diff --git a/conftest.py b/conftest.py index ea59b3b6..074ccd7b 100644 --- a/conftest.py +++ b/conftest.py @@ -10,7 +10,6 @@ from xarray.core.variable import Variable from zarr.core.metadata.v3 import ArrayV3Metadata -from virtualizarr.codecs import convert_to_codec_pipeline from virtualizarr.manifests import ChunkManifest, ManifestArray from virtualizarr.manifests.manifest import join from virtualizarr.utils import ceildiv @@ -190,7 +189,8 @@ def _create_manifestarray( shape: tuple | None = (5, 5), chunks: tuple | None = (5, 5), codecs: list[dict] | None = [ - {"name": "numcodecs.zlib", "configuration": {"level": 1}} + {"configuration": {"endian": "little"}, "name": "bytes"}, + {"name": "numcodecs.zlib", "configuration": {"level": 1}}, ], ): metadata = array_v3_metadata(shape=shape, chunks=chunks, codecs=codecs) @@ -215,43 +215,12 @@ def _create_manifestarray( return _create_manifestarray -@pytest.fixture -def array_v3_metadata(): - def _create_metadata( - shape: tuple = (5, 5), - chunks: tuple = (5, 5), - chunk_grid: Any | None = None, - data_type: str = np.dtype("int32"), - codecs: list[dict] | None = None, - fill_value: int = None, - ): - chunk_grid = chunk_grid or { - "name": "regular", - "configuration": {"chunk_shape": chunks}, - } - return ArrayV3Metadata( - shape=shape, - data_type=data_type, - chunk_grid=chunk_grid, - chunk_key_encoding={"name": "default"}, - fill_value=fill_value, - codecs=convert_to_codec_pipeline( - codecs=codecs, - dtype=data_type, - ), - attributes={}, - dimension_names=None, - storage_transformers=None, - ) - - return _create_metadata - - @pytest.fixture def array_v3_metadata_dict(): def _create_metadata_dict( - shape: tuple, - chunks: tuple, + shape: tuple = (5, 5), + chunks: tuple = (5, 5), + chunk_grid: dict | None = None, codecs: list[dict] = [ {"configuration": {"endian": "little"}, "name": "bytes"}, { @@ -259,13 +228,19 @@ def _create_metadata_dict( "configuration": {"level": 1}, }, ], + data_type: str = "int32", + fill_value: int = 0, ): + chunk_grid = chunk_grid or { + "name": "regular", + "configuration": {"chunk_shape": chunks}, + } return { "shape": shape, - "data_type": "int32", - "chunk_grid": {"name": "regular", "configuration": {"chunk_shape": chunks}}, + "data_type": data_type, + "chunk_grid": chunk_grid, "chunk_key_encoding": {"name": "default"}, - "fill_value": 0, + "fill_value": fill_value, "codecs": codecs, "attributes": {}, "dimension_names": None, @@ -275,6 +250,35 @@ def _create_metadata_dict( return _create_metadata_dict +@pytest.fixture +def array_v3_metadata(array_v3_metadata_dict): + def _create_metadata( + shape: tuple = (5, 5), + chunks: tuple = (5, 5), + chunk_grid: Any | None = None, + data_type: str = np.dtype("int32"), + codecs: list[dict] | None = None, + fill_value: int = None, + ): + chunk_grid = chunk_grid or { + "name": "regular", + "configuration": {"chunk_shape": chunks}, + } + codecs = codecs or [{"configuration": {"endian": "little"}, "name": "bytes"}] + + metadata_dict = array_v3_metadata_dict( + shape=shape, + chunk_grid=chunk_grid, + chunks=chunks, + codecs=codecs, + data_type=data_type, + fill_value=fill_value or 0, + ) + return ArrayV3Metadata(**metadata_dict) + + return _create_metadata + + def generate_chunk_manifest( netcdf4_file: str, shape: tuple[int, ...], diff --git a/virtualizarr/codecs.py b/virtualizarr/codecs.py index 27d2f70e..fec7b22b 100644 --- a/virtualizarr/codecs.py +++ b/virtualizarr/codecs.py @@ -87,7 +87,7 @@ def convert_to_codec_pipeline( return codec_pipeline -def _get_codec_config(codec: ZarrCodec) -> dict[str, Any]: +def get_codec_config(codec: ZarrCodec) -> dict[str, Any]: """ Extract configuration from a codec, handling both zarr-python and numcodecs codecs. """ diff --git a/virtualizarr/tests/test_manifests/test_array.py b/virtualizarr/tests/test_manifests/test_array.py index 7b6ec472..3fd78adc 100644 --- a/virtualizarr/tests/test_manifests/test_array.py +++ b/virtualizarr/tests/test_manifests/test_array.py @@ -44,23 +44,11 @@ def test_create_manifestarray_dict_v3_metadata(self, array_v3_metadata_dict): assert marr.ndim == 3 def test_manifestarray_notimplementederror(self, array_v3_metadata): - from zarr.core.chunk_grids import ChunkGrid - - class IrregularChunkGrid(ChunkGrid): - pass - - metadata = array_v3_metadata(chunk_grid=IrregularChunkGrid()) - - chunks_dict = { - "0.0.0": {"path": "s3://bucket/foo.nc", "offset": 100, "length": 100}, - } - manifest = ChunkManifest(entries=chunks_dict) - with pytest.raises( - NotImplementedError, - match="Only RegularChunkGrid is currently supported for chunk size", + ValueError, + match="Unknown chunk grid. Got irregular.", ): - ManifestArray(metadata=metadata, chunkmanifest=manifest) + array_v3_metadata(chunk_grid={"name": "irregular", "configuration": {}}) class TestEquals: @@ -246,8 +234,11 @@ def test_concat(self, array_v3_metadata): def test_concat_empty(self, array_v3_metadata): chunks = (5, 1, 10) shape = (5, 1, 20) - compressor = {"name": "numcodecs.zlib", "configuration": {"level": 1}} - metadata = array_v3_metadata(shape=shape, chunks=chunks, codecs=[compressor]) + codecs = [ + {"name": "bytes", "configuration": {"endian": "little"}}, + {"name": "numcodecs.zlib", "configuration": {"level": 1}}, + ] + metadata = array_v3_metadata(shape=shape, chunks=chunks, codecs=codecs) empty_chunks_dict = {} empty_chunk_manifest = ChunkManifest(entries=empty_chunks_dict, shape=(1, 1, 2)) manifest_array_with_empty_chunks = ManifestArray( @@ -282,8 +273,11 @@ def test_stack(self, array_v3_metadata): # both manifest arrays in this example have the same zarray properties chunks = (5, 10) shape = (5, 20) - compressor = {"name": "numcodecs.zlib", "configuration": {"level": 1}} - metadata = array_v3_metadata(shape=shape, chunks=chunks, codecs=[compressor]) + codecs = [ + {"name": "bytes", "configuration": {"endian": "little"}}, + {"name": "numcodecs.zlib", "configuration": {"level": 1}}, + ] + metadata = array_v3_metadata(shape=shape, chunks=chunks, codecs=codecs) chunks_dict1 = { "0.0": {"path": "/foo.nc", "offset": 100, "length": 100}, "0.1": {"path": "/foo.nc", "offset": 200, "length": 100}, @@ -320,7 +314,10 @@ def test_stack_empty(self, array_v3_metadata): metadata = array_v3_metadata( shape=shape, chunks=chunks, - codecs=[{"name": "numcodecs.zlib", "configuration": {"level": 1}}], + codecs=[ + {"name": "bytes", "configuration": {"endian": "little"}}, + {"name": "numcodecs.zlib", "configuration": {"level": 1}}, + ], ) chunks_dict1 = {} diff --git a/virtualizarr/tests/test_writers/conftest.py b/virtualizarr/tests/test_writers/conftest.py index 772355a0..8af5d858 100644 --- a/virtualizarr/tests/test_writers/conftest.py +++ b/virtualizarr/tests/test_writers/conftest.py @@ -16,7 +16,10 @@ def vds_with_manifest_arrays(array_v3_metadata) -> Dataset: shape=(2, 3), data_type=np.dtype(" Date: Thu, 13 Feb 2025 09:03:46 -0800 Subject: [PATCH 49/78] No need for union type for CodecPipeline --- virtualizarr/codecs.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/virtualizarr/codecs.py b/virtualizarr/codecs.py index fec7b22b..343fcfba 100644 --- a/virtualizarr/codecs.py +++ b/virtualizarr/codecs.py @@ -10,9 +10,7 @@ if TYPE_CHECKING: from .manifests.array import ManifestArray -CodecPipeline = Tuple[ - Union["ArrayArrayCodec", "ArrayBytesCodec", "BytesBytesCodec"], ... -] +CodecPipeline = Tuple[ArrayArrayCodec | ArrayBytesCodec | BytesBytesCodec, ...] def num_codec_config_to_configurable(num_codec: dict) -> dict: From cdaca53cecd1800a2f6d8616591b3647b202322d Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Thu, 13 Feb 2025 09:13:38 -0800 Subject: [PATCH 50/78] Use type alias --- virtualizarr/codecs.py | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/virtualizarr/codecs.py b/virtualizarr/codecs.py index 343fcfba..d3af62b7 100644 --- a/virtualizarr/codecs.py +++ b/virtualizarr/codecs.py @@ -12,6 +12,12 @@ CodecPipeline = Tuple[ArrayArrayCodec | ArrayBytesCodec | BytesBytesCodec, ...] +DeconstructedCodecPipeline = tuple[ + tuple[ArrayArrayCodec, ...], # Array-to-array transformations + ArrayBytesCodec | None, # Array-to-bytes conversion + tuple[BytesBytesCodec, ...], # Bytes-to-bytes transformations +] + def num_codec_config_to_configurable(num_codec: dict) -> dict: """ @@ -27,9 +33,7 @@ def num_codec_config_to_configurable(num_codec: dict) -> dict: def extract_codecs( codecs: CodecPipeline, -) -> tuple[ - tuple[ArrayArrayCodec, ...], ArrayBytesCodec | None, tuple[BytesBytesCodec, ...] -]: +) -> DeconstructedCodecPipeline: """Extracts various codec types.""" arrayarray_codecs: tuple[ArrayArrayCodec, ...] = () arraybytes_codec: ArrayBytesCodec | None = None From 2415e07f8003e41667832048cb661a18095fd59d Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Thu, 13 Feb 2025 09:17:22 -0800 Subject: [PATCH 51/78] Add comment --- virtualizarr/codecs.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/virtualizarr/codecs.py b/virtualizarr/codecs.py index d3af62b7..95cb8f24 100644 --- a/virtualizarr/codecs.py +++ b/virtualizarr/codecs.py @@ -74,6 +74,8 @@ def convert_to_codec_pipeline( get_codec_class(codec["name"]).from_dict(codec) for codec in codecs ) + # It would be nice to use zarr.core.codec_pipeline.codecs_from_list here but that function requires + # array array codecs and array bytes codecs to already be present in the list and in the correct order. arrayarray_codecs, arraybytes_codec, bytesbytes_codecs = extract_codecs(zarr_codecs) if arraybytes_codec is None: From 9366d69bb0a3af35bba71e4d7c5f91892665d7c7 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Thu, 13 Feb 2025 09:19:47 -0800 Subject: [PATCH 52/78] Update virtualizarr/manifests/array_api.py Co-authored-by: Tom Nicholas --- virtualizarr/manifests/array_api.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/virtualizarr/manifests/array_api.py b/virtualizarr/manifests/array_api.py index ba489d60..b923f134 100644 --- a/virtualizarr/manifests/array_api.py +++ b/virtualizarr/manifests/array_api.py @@ -177,11 +177,11 @@ def stack( return ManifestArray(chunkmanifest=stacked_manifest, metadata=new_metadata) -def _update_metadata( - first_arr, new_shape: list[int], new_chunks: list[int] | None = None +def copy_and_replace( + old_metadata: ArrayV3Metadata, new_shape: list[int] | None = None, new_chunks: list[int] | None = None ) -> ArrayV3Metadata: """ - Update the metadata of a ManifestArray to reflect a new shape and/or chunk shape. + Update metadata to reflect a new shape and/or chunk shape. """ metadata_copy = first_arr.metadata.to_dict().copy() metadata_copy["shape"] = tuple(new_shape) From d590cfc1eff557b01dae88e7fedd8be178c6e67a Mon Sep 17 00:00:00 2001 From: "pre-commit-ci[bot]" <66853113+pre-commit-ci[bot]@users.noreply.github.com> Date: Thu, 13 Feb 2025 17:19:52 +0000 Subject: [PATCH 53/78] [pre-commit.ci] auto fixes from pre-commit.com hooks for more information, see https://pre-commit.ci --- virtualizarr/manifests/array_api.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/virtualizarr/manifests/array_api.py b/virtualizarr/manifests/array_api.py index b923f134..10e2a20b 100644 --- a/virtualizarr/manifests/array_api.py +++ b/virtualizarr/manifests/array_api.py @@ -178,7 +178,9 @@ def stack( def copy_and_replace( - old_metadata: ArrayV3Metadata, new_shape: list[int] | None = None, new_chunks: list[int] | None = None + old_metadata: ArrayV3Metadata, + new_shape: list[int] | None = None, + new_chunks: list[int] | None = None, ) -> ArrayV3Metadata: """ Update metadata to reflect a new shape and/or chunk shape. From 6394207eedd40c205a782e8f000cab6cf7a4d651 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Thu, 13 Feb 2025 09:28:46 -0800 Subject: [PATCH 54/78] Revised copy_and_replace_metadata to be in utils and called correctly --- virtualizarr/manifests/array_api.py | 36 ++++++++--------------------- virtualizarr/manifests/utils.py | 24 +++++++++++++++++-- 2 files changed, 32 insertions(+), 28 deletions(-) diff --git a/virtualizarr/manifests/array_api.py b/virtualizarr/manifests/array_api.py index 10e2a20b..ffc0e95f 100644 --- a/virtualizarr/manifests/array_api.py +++ b/virtualizarr/manifests/array_api.py @@ -1,7 +1,6 @@ from typing import TYPE_CHECKING, Any, Callable, cast import numpy as np -from zarr.core.metadata.v3 import ArrayV3Metadata from virtualizarr.utils import determine_chunk_grid_shape @@ -11,6 +10,7 @@ check_same_ndims, check_same_shapes, check_same_shapes_except_on_concat_axis, + copy_and_replace_metadata, ) if TYPE_CHECKING: @@ -102,7 +102,9 @@ def concatenate( lengths=concatenated_lengths, ) - new_metadata = _update_metadata(first_arr, new_shape=new_shape) + new_metadata = copy_and_replace_metadata( + old_metadata=first_arr.metadata, new_shape=new_shape + ) return ManifestArray(chunkmanifest=concatenated_manifest, metadata=new_metadata) @@ -170,33 +172,13 @@ def stack( new_chunks = list(old_chunks) new_chunks.insert(axis, 1) - new_metadata = _update_metadata( - first_arr, new_shape=new_shape, new_chunks=new_chunks + new_metadata = copy_and_replace_metadata( + old_metadata=first_arr.metadata, new_shape=new_shape, new_chunks=new_chunks ) return ManifestArray(chunkmanifest=stacked_manifest, metadata=new_metadata) -def copy_and_replace( - old_metadata: ArrayV3Metadata, - new_shape: list[int] | None = None, - new_chunks: list[int] | None = None, -) -> ArrayV3Metadata: - """ - Update metadata to reflect a new shape and/or chunk shape. - """ - metadata_copy = first_arr.metadata.to_dict().copy() - metadata_copy["shape"] = tuple(new_shape) - if new_chunks is not None: - metadata_copy["chunk_grid"] = { - "name": "regular", - "configuration": {"chunk_shape": tuple(new_chunks)}, - } - # ArrayV3Metadata.from_dict removes extra keys zarr_format and node_type - new_metadata = ArrayV3Metadata.from_dict(metadata_copy) - return new_metadata - - @implements(np.expand_dims) def expand_dims(x: "ManifestArray", /, *, axis: int = 0) -> "ManifestArray": """Expands the shape of an array by inserting a new axis (dimension) of size one at the position specified by axis.""" @@ -255,8 +237,10 @@ def broadcast_to(x: "ManifestArray", /, shape: tuple[int, ...]) -> "ManifestArra lengths=broadcasted_lengths, ) - new_metadata = _update_metadata( - x, new_shape=list(new_shape), new_chunks=list(new_chunk_shape) + new_metadata = copy_and_replace_metadata( + old_metadata=x.metadata, + new_shape=list(new_shape), + new_chunks=list(new_chunk_shape), ) return ManifestArray(chunkmanifest=broadcasted_manifest, metadata=new_metadata) diff --git a/virtualizarr/manifests/utils.py b/virtualizarr/manifests/utils.py index 07cf2baf..b4698f2c 100644 --- a/virtualizarr/manifests/utils.py +++ b/virtualizarr/manifests/utils.py @@ -1,12 +1,12 @@ from typing import TYPE_CHECKING, Any, Iterable, Union import numpy as np +from zarr import Array +from zarr.core.metadata.v3 import ArrayV3Metadata from virtualizarr.codecs import get_codecs if TYPE_CHECKING: - from zarr import Array # type: ignore - from .array import ManifestArray @@ -116,3 +116,23 @@ def check_compatible_arrays( check_same_ndims([ma.ndim, existing_array.ndim]) arr_shapes = [ma.shape, existing_array.shape] check_same_shapes_except_on_concat_axis(arr_shapes, append_axis) + + +def copy_and_replace_metadata( + old_metadata: ArrayV3Metadata, + new_shape: list[int] | None = None, + new_chunks: list[int] | None = None, +) -> ArrayV3Metadata: + """ + Update metadata to reflect a new shape and/or chunk shape. + """ + metadata_copy = old_metadata.to_dict().copy() + metadata_copy["shape"] = new_shape # type: ignore[assignment] + if new_chunks is not None: + metadata_copy["chunk_grid"] = { + "name": "regular", + "configuration": {"chunk_shape": tuple(new_chunks)}, + } + # ArrayV3Metadata.from_dict removes extra keys zarr_format and node_type + new_metadata = ArrayV3Metadata.from_dict(metadata_copy) + return new_metadata From ea9fd56f3b248eaa0d042e83c9db802471ee4a48 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Thu, 13 Feb 2025 13:34:21 -0800 Subject: [PATCH 55/78] Update virtualizarr/translators/kerchunk.py Co-authored-by: Tom Nicholas --- virtualizarr/translators/kerchunk.py | 1 + 1 file changed, 1 insertion(+) diff --git a/virtualizarr/translators/kerchunk.py b/virtualizarr/translators/kerchunk.py index ca1db7d9..70c44458 100644 --- a/virtualizarr/translators/kerchunk.py +++ b/virtualizarr/translators/kerchunk.py @@ -57,6 +57,7 @@ def from_kerchunk_refs(decoded_arr_refs_zarray) -> "ArrayV3Metadata": Returns: ------- ArrayV3Metadata + Raises: ------ ValueError From 0ee2b488144ee9b11c2fe59fce30580b5d728cec Mon Sep 17 00:00:00 2001 From: "pre-commit-ci[bot]" <66853113+pre-commit-ci[bot]@users.noreply.github.com> Date: Thu, 13 Feb 2025 21:34:26 +0000 Subject: [PATCH 56/78] [pre-commit.ci] auto fixes from pre-commit.com hooks for more information, see https://pre-commit.ci --- virtualizarr/translators/kerchunk.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/virtualizarr/translators/kerchunk.py b/virtualizarr/translators/kerchunk.py index 70c44458..a621f614 100644 --- a/virtualizarr/translators/kerchunk.py +++ b/virtualizarr/translators/kerchunk.py @@ -57,7 +57,7 @@ def from_kerchunk_refs(decoded_arr_refs_zarray) -> "ArrayV3Metadata": Returns: ------- ArrayV3Metadata - + Raises: ------ ValueError From 86d1de5af91501fbdd0b0d8e2f8545645993a7d1 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Thu, 13 Feb 2025 14:02:06 -0800 Subject: [PATCH 57/78] Refactor create v3 array metadata --- conftest.py | 53 ++---------------- virtualizarr/manifests/utils.py | 55 ++++++++++++++++++- virtualizarr/readers/dmrpp.py | 20 ++----- virtualizarr/readers/hdf/hdf.py | 17 ++---- .../tests/test_manifests/test_array.py | 39 ++++++------- 5 files changed, 83 insertions(+), 101 deletions(-) diff --git a/conftest.py b/conftest.py index 074ccd7b..b60cbb59 100644 --- a/conftest.py +++ b/conftest.py @@ -8,10 +8,10 @@ import pytest import xarray as xr from xarray.core.variable import Variable -from zarr.core.metadata.v3 import ArrayV3Metadata from virtualizarr.manifests import ChunkManifest, ManifestArray from virtualizarr.manifests.manifest import join +from virtualizarr.manifests.utils import create_array_metadata from virtualizarr.utils import ceildiv @@ -216,65 +216,22 @@ def _create_manifestarray( @pytest.fixture -def array_v3_metadata_dict(): - def _create_metadata_dict( - shape: tuple = (5, 5), - chunks: tuple = (5, 5), - chunk_grid: dict | None = None, - codecs: list[dict] = [ - {"configuration": {"endian": "little"}, "name": "bytes"}, - { - "name": "numcodecs.zlib", - "configuration": {"level": 1}, - }, - ], - data_type: str = "int32", - fill_value: int = 0, - ): - chunk_grid = chunk_grid or { - "name": "regular", - "configuration": {"chunk_shape": chunks}, - } - return { - "shape": shape, - "data_type": data_type, - "chunk_grid": chunk_grid, - "chunk_key_encoding": {"name": "default"}, - "fill_value": fill_value, - "codecs": codecs, - "attributes": {}, - "dimension_names": None, - "storage_transformers": None, - } - - return _create_metadata_dict - - -@pytest.fixture -def array_v3_metadata(array_v3_metadata_dict): +def array_v3_metadata(): def _create_metadata( shape: tuple = (5, 5), chunks: tuple = (5, 5), - chunk_grid: Any | None = None, data_type: str = np.dtype("int32"), codecs: list[dict] | None = None, fill_value: int = None, ): - chunk_grid = chunk_grid or { - "name": "regular", - "configuration": {"chunk_shape": chunks}, - } codecs = codecs or [{"configuration": {"endian": "little"}, "name": "bytes"}] - - metadata_dict = array_v3_metadata_dict( + return create_array_metadata( shape=shape, - chunk_grid=chunk_grid, - chunks=chunks, - codecs=codecs, + chunk_shape=chunks, data_type=data_type, + codecs=codecs, fill_value=fill_value or 0, ) - return ArrayV3Metadata(**metadata_dict) return _create_metadata diff --git a/virtualizarr/manifests/utils.py b/virtualizarr/manifests/utils.py index b4698f2c..bc1d45ff 100644 --- a/virtualizarr/manifests/utils.py +++ b/virtualizarr/manifests/utils.py @@ -1,15 +1,66 @@ -from typing import TYPE_CHECKING, Any, Iterable, Union +from typing import TYPE_CHECKING, Any, Dict, Iterable, Optional, Union import numpy as np from zarr import Array from zarr.core.metadata.v3 import ArrayV3Metadata -from virtualizarr.codecs import get_codecs +from virtualizarr.codecs import convert_to_codec_pipeline, get_codecs if TYPE_CHECKING: from .array import ManifestArray +def create_array_metadata( + shape: tuple[int, ...], + data_type: np.dtype, + chunk_shape: tuple[int, ...], + fill_value: Any = None, + codecs: Optional[list[Dict[str, Any]]] = None, + attributes: Optional[Dict[str, Any]] = None, +) -> ArrayV3Metadata: + """ + Create an ArrayV3Metadata instance with standard configuration. + This function encapsulates common patterns used across different readers. + + Parameters + ---------- + shape : tuple[int, ...] + The shape of the array + data_type : np.dtype + The numpy dtype of the array + chunk_shape : tuple[int, ...] + The shape of each chunk + fill_value : Any, optional + The fill value for the array + codecs : list[Dict[str, Any]], optional + List of codec configurations + attributes : Dict[str, Any], optional + Additional attributes for the array + + Returns + ------- + ArrayV3Metadata + A configured ArrayV3Metadata instance with standard defaults + """ + return ArrayV3Metadata( + shape=shape, + data_type=data_type, + chunk_grid={ + "name": "regular", + "configuration": {"chunk_shape": chunk_shape}, + }, + chunk_key_encoding={"name": "default"}, + fill_value=fill_value, + codecs=convert_to_codec_pipeline( + codecs=codecs or [], + dtype=data_type, + ), + attributes=attributes or {}, + dimension_names=None, + storage_transformers=None, + ) + + def check_same_dtypes(dtypes: list[np.dtype]) -> None: """Check all the dtypes are the same""" diff --git a/virtualizarr/readers/dmrpp.py b/virtualizarr/readers/dmrpp.py index 95989ad6..5dbc5183 100644 --- a/virtualizarr/readers/dmrpp.py +++ b/virtualizarr/readers/dmrpp.py @@ -9,6 +9,7 @@ from virtualizarr.manifests import ChunkManifest, ManifestArray from virtualizarr.manifests.manifest import validate_and_normalize_path_to_uri +from virtualizarr.manifests.utils import create_array_metadata from virtualizarr.readers.common import VirtualBackend from virtualizarr.types import ChunkKey from virtualizarr.utils import _FsspecFSFromFilepath, check_for_collisions @@ -377,9 +378,6 @@ def _parse_variable(self, var_tag: ET.Element) -> Variable: ------- xr.Variable """ - from zarr.core.metadata.v3 import ArrayV3Metadata - - from virtualizarr.codecs import convert_to_codec_pipeline # Dimension info dims: dict[str, int] = {} @@ -417,22 +415,12 @@ def _parse_variable(self, var_tag: ET.Element) -> Variable: encoding = {k: attrs.get(k) for k in self._ENCODING_KEYS if k in attrs} fill_value = attrs.pop("_FillValue", None) # create ManifestArray - metadata = ArrayV3Metadata( + metadata = create_array_metadata( shape=shape, data_type=dtype, - chunk_grid={ - "name": "regular", - "configuration": {"chunk_shape": chunks_shape}, - }, - chunk_key_encoding={"name": "default"}, + chunk_shape=chunks_shape, fill_value=fill_value, - codecs=convert_to_codec_pipeline( - codecs=codecs, - dtype=dtype, - ), - attributes=attrs, - dimension_names=None, - storage_transformers=None, + codecs=codecs, ) marr = ManifestArray(metadata=metadata, chunkmanifest=chunkmanifest) return Variable(dims=dims.keys(), data=marr, attrs=attrs, encoding=encoding) diff --git a/virtualizarr/readers/hdf/hdf.py b/virtualizarr/readers/hdf/hdf.py index a63ffc2c..0e7022af 100644 --- a/virtualizarr/readers/hdf/hdf.py +++ b/virtualizarr/readers/hdf/hdf.py @@ -13,18 +13,15 @@ import numpy as np import xarray as xr -from zarr.core.metadata.v3 import ArrayV3Metadata -from virtualizarr.codecs import ( - convert_to_codec_pipeline, - num_codec_config_to_configurable, -) +from virtualizarr.codecs import num_codec_config_to_configurable from virtualizarr.manifests import ( ChunkEntry, ChunkManifest, ManifestArray, ) from virtualizarr.manifests.manifest import validate_and_normalize_path_to_uri +from virtualizarr.manifests.utils import create_array_metadata from virtualizarr.readers.common import ( VirtualBackend, construct_virtual_dataset, @@ -309,16 +306,12 @@ def _dataset_to_variable( codec_configs = [ num_codec_config_to_configurable(codec.get_config()) for codec in codecs ] - metadata = ArrayV3Metadata( + metadata = create_array_metadata( shape=dataset.shape, data_type=dtype, - chunk_grid={"name": "regular", "configuration": {"chunk_shape": chunks}}, - chunk_key_encoding={"name": "default"}, + chunk_shape=chunks, fill_value=fill_value, - codecs=convert_to_codec_pipeline(codecs=codec_configs, dtype=dtype), - attributes=attrs, - dimension_names=None, - storage_transformers=None, + codecs=codec_configs, ) dims = HDFVirtualBackend._dataset_dims(dataset, group=group) manifest = HDFVirtualBackend._dataset_chunk_manifest(path, dataset) diff --git a/virtualizarr/tests/test_manifests/test_array.py b/virtualizarr/tests/test_manifests/test_array.py index 3fd78adc..1e890668 100644 --- a/virtualizarr/tests/test_manifests/test_array.py +++ b/virtualizarr/tests/test_manifests/test_array.py @@ -1,5 +1,6 @@ import numpy as np import pytest +from zarr.core.metadata.v3 import ArrayV3Metadata from virtualizarr.manifests import ChunkManifest, ManifestArray @@ -24,7 +25,7 @@ def test_create_manifestarray(self, array_v3_metadata): assert marr.size == 5 * 2 * 20 assert marr.ndim == 3 - def test_create_manifestarray_dict_v3_metadata(self, array_v3_metadata_dict): + def test_create_manifestarray_dict_v3_metadata(self, array_v3_metadata): chunks_dict = { "0.0.0": {"path": "s3://bucket/foo.nc", "offset": 100, "length": 100}, "0.0.1": {"path": "s3://bucket/foo.nc", "offset": 200, "length": 100}, @@ -34,7 +35,8 @@ def test_create_manifestarray_dict_v3_metadata(self, array_v3_metadata_dict): manifest = ChunkManifest(entries=chunks_dict) chunks = (5, 1, 10) shape = (5, 2, 20) - metadata_dict = array_v3_metadata_dict(shape=shape, chunks=chunks) + metadata = array_v3_metadata(shape=shape, chunks=chunks) + metadata_dict = ArrayV3Metadata.from_dict(metadata.to_dict()) marr = ManifestArray(metadata=metadata_dict, chunkmanifest=manifest) assert marr.chunks == chunks @@ -43,13 +45,6 @@ def test_create_manifestarray_dict_v3_metadata(self, array_v3_metadata_dict): assert marr.size == 5 * 2 * 20 assert marr.ndim == 3 - def test_manifestarray_notimplementederror(self, array_v3_metadata): - with pytest.raises( - ValueError, - match="Unknown chunk grid. Got irregular.", - ): - array_v3_metadata(chunk_grid={"name": "irregular", "configuration": {}}) - class TestEquals: def test_equals(self, array_v3_metadata): @@ -344,11 +339,12 @@ def test_stack_empty(self, array_v3_metadata): assert result.metadata.fill_value == metadata.fill_value -def test_refuse_combine(array_v3_metadata_dict): +def test_refuse_combine(array_v3_metadata): # TODO test refusing to concatenate arrays that have conflicting shapes / chunk sizes chunks = (5, 1, 10) shape = (5, 1, 20) - metadata_common = array_v3_metadata_dict(shape=shape, chunks=chunks) + metadata_common = array_v3_metadata(shape=shape, chunks=chunks) + chunks_dict1 = { "0.0.0": {"path": "/foo.nc", "offset": 100, "length": 100}, } @@ -359,27 +355,24 @@ def test_refuse_combine(array_v3_metadata_dict): chunkmanifest2 = ChunkManifest(entries=chunks_dict2) marr1 = ManifestArray(metadata=metadata_common, chunkmanifest=chunkmanifest1) - metadata_wrong_compressor = array_v3_metadata_dict( + metadata_different_codecs = array_v3_metadata( shape=shape, chunks=chunks, - codecs=[{"name": "bytes", "configuration": {"endian": "little"}}], + codecs=[ + {"name": "bytes", "configuration": {"endian": "little"}}, + {"name": "numcodecs.zlib", "configuration": {"level": 1}}, + ], ) marr2 = ManifestArray( - metadata=metadata_wrong_compressor, chunkmanifest=chunkmanifest2 + metadata=metadata_different_codecs, chunkmanifest=chunkmanifest2 ) for func in [np.concatenate, np.stack]: with pytest.raises(NotImplementedError, match="different codecs"): func([marr1, marr2], axis=0) - metadata_wrong_dtype = metadata_common.copy() - metadata_wrong_dtype["data_type"] = np.dtype("int64") - marr2 = ManifestArray(metadata=metadata_wrong_dtype, chunkmanifest=chunkmanifest2) - for func in [np.concatenate, np.stack]: - with pytest.raises(ValueError, match="inconsistent dtypes"): - func([marr1, marr2], axis=0) - - metadata_wrong_dtype = metadata_common.copy() - metadata_wrong_dtype["data_type"] = np.dtype("int64") + metadata_copy = metadata_common.to_dict().copy() + metadata_copy["data_type"] = np.dtype("int64") + metadata_wrong_dtype = ArrayV3Metadata.from_dict(metadata_copy) marr2 = ManifestArray(metadata=metadata_wrong_dtype, chunkmanifest=chunkmanifest2) for func in [np.concatenate, np.stack]: with pytest.raises(ValueError, match="inconsistent dtypes"): From fe8305f6210da3a8dd1f3b1ed226d62a8e2daa6d Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Thu, 13 Feb 2025 14:02:33 -0800 Subject: [PATCH 58/78] Rename to create_v3_array_metadata --- conftest.py | 4 ++-- virtualizarr/manifests/utils.py | 2 +- virtualizarr/readers/dmrpp.py | 4 ++-- virtualizarr/readers/hdf/hdf.py | 4 ++-- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/conftest.py b/conftest.py index b60cbb59..b2ae7d54 100644 --- a/conftest.py +++ b/conftest.py @@ -11,7 +11,7 @@ from virtualizarr.manifests import ChunkManifest, ManifestArray from virtualizarr.manifests.manifest import join -from virtualizarr.manifests.utils import create_array_metadata +from virtualizarr.manifests.utils import create_v3_array_metadata from virtualizarr.utils import ceildiv @@ -225,7 +225,7 @@ def _create_metadata( fill_value: int = None, ): codecs = codecs or [{"configuration": {"endian": "little"}, "name": "bytes"}] - return create_array_metadata( + return create_v3_array_metadata( shape=shape, chunk_shape=chunks, data_type=data_type, diff --git a/virtualizarr/manifests/utils.py b/virtualizarr/manifests/utils.py index bc1d45ff..b4dbfb17 100644 --- a/virtualizarr/manifests/utils.py +++ b/virtualizarr/manifests/utils.py @@ -10,7 +10,7 @@ from .array import ManifestArray -def create_array_metadata( +def create_v3_array_metadata( shape: tuple[int, ...], data_type: np.dtype, chunk_shape: tuple[int, ...], diff --git a/virtualizarr/readers/dmrpp.py b/virtualizarr/readers/dmrpp.py index 5dbc5183..b5d2b020 100644 --- a/virtualizarr/readers/dmrpp.py +++ b/virtualizarr/readers/dmrpp.py @@ -9,7 +9,7 @@ from virtualizarr.manifests import ChunkManifest, ManifestArray from virtualizarr.manifests.manifest import validate_and_normalize_path_to_uri -from virtualizarr.manifests.utils import create_array_metadata +from virtualizarr.manifests.utils import create_v3_array_metadata from virtualizarr.readers.common import VirtualBackend from virtualizarr.types import ChunkKey from virtualizarr.utils import _FsspecFSFromFilepath, check_for_collisions @@ -415,7 +415,7 @@ def _parse_variable(self, var_tag: ET.Element) -> Variable: encoding = {k: attrs.get(k) for k in self._ENCODING_KEYS if k in attrs} fill_value = attrs.pop("_FillValue", None) # create ManifestArray - metadata = create_array_metadata( + metadata = create_v3_array_metadata( shape=shape, data_type=dtype, chunk_shape=chunks_shape, diff --git a/virtualizarr/readers/hdf/hdf.py b/virtualizarr/readers/hdf/hdf.py index 0e7022af..d4d55b67 100644 --- a/virtualizarr/readers/hdf/hdf.py +++ b/virtualizarr/readers/hdf/hdf.py @@ -21,7 +21,7 @@ ManifestArray, ) from virtualizarr.manifests.manifest import validate_and_normalize_path_to_uri -from virtualizarr.manifests.utils import create_array_metadata +from virtualizarr.manifests.utils import create_v3_array_metadata from virtualizarr.readers.common import ( VirtualBackend, construct_virtual_dataset, @@ -306,7 +306,7 @@ def _dataset_to_variable( codec_configs = [ num_codec_config_to_configurable(codec.get_config()) for codec in codecs ] - metadata = create_array_metadata( + metadata = create_v3_array_metadata( shape=dataset.shape, data_type=dtype, chunk_shape=chunks, From 0f5b32dd473f84fbf01e2468328afb780ae43a65 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Thu, 13 Feb 2025 14:25:40 -0800 Subject: [PATCH 59/78] Fix some codecs fixtures --- conftest.py | 35 ++++++++++++++++++- virtualizarr/tests/test_codecs.py | 26 +++++--------- virtualizarr/tests/test_integration.py | 4 +-- .../tests/test_manifests/test_array.py | 24 ++++++------- virtualizarr/tests/test_writers/conftest.py | 8 +++-- 5 files changed, 61 insertions(+), 36 deletions(-) diff --git a/conftest.py b/conftest.py index b2ae7d54..ac64ca5b 100644 --- a/conftest.py +++ b/conftest.py @@ -1,7 +1,7 @@ import itertools from itertools import product from pathlib import Path -from typing import Any, Callable, Mapping, Optional +from typing import Any, Callable, Dict, Mapping, Optional import h5py import numpy as np @@ -338,3 +338,36 @@ def _gen_virtual_dataset( ) return _gen_virtual_dataset + + +# Common codec configurations used across tests +@pytest.fixture +def delta_codec() -> Dict[str, Any]: + """Delta codec configuration for array-to-array transformation.""" + return {"name": "numcodecs.delta", "configuration": {"dtype": " Dict[str, Any]: + """Bytes codec configuration for array-to-bytes transformation.""" + return {"name": "bytes", "configuration": {"endian": "little"}} + + +@pytest.fixture +def blosc_codec() -> Dict[str, Any]: + """Blosc codec configuration for bytes-to-bytes transformation.""" + return { + "name": "blosc", + "configuration": { + "cname": "zstd", + "clevel": 5, + "shuffle": "shuffle", + "typesize": 4, + }, + } + + +@pytest.fixture +def zlib_codec() -> Dict[str, Any]: + """Zlib codec configuration for bytes-to-bytes transformation.""" + return {"name": "numcodecs.zlib", "configuration": {"level": 1}} diff --git a/virtualizarr/tests/test_codecs.py b/virtualizarr/tests/test_codecs.py index 76f2cd12..0e1fc83f 100644 --- a/virtualizarr/tests/test_codecs.py +++ b/virtualizarr/tests/test_codecs.py @@ -6,18 +6,6 @@ from virtualizarr.codecs import convert_to_codec_pipeline, get_codecs -arrayarray_codec = {"name": "numcodecs.delta", "configuration": {"dtype": ", ] - def test_concat_empty(self, array_v3_metadata): + def test_concat_empty(self, array_v3_metadata, arraybytes_codec, zlib_codec): chunks = (5, 1, 10) shape = (5, 1, 20) codecs = [ - {"name": "bytes", "configuration": {"endian": "little"}}, - {"name": "numcodecs.zlib", "configuration": {"level": 1}}, + arraybytes_codec, + zlib_codec, ] metadata = array_v3_metadata(shape=shape, chunks=chunks, codecs=codecs) empty_chunks_dict = {} @@ -264,13 +264,13 @@ def test_concat_empty(self, array_v3_metadata): class TestStack: - def test_stack(self, array_v3_metadata): + def test_stack(self, array_v3_metadata, arraybytes_codec, zlib_codec): # both manifest arrays in this example have the same zarray properties chunks = (5, 10) shape = (5, 20) codecs = [ - {"name": "bytes", "configuration": {"endian": "little"}}, - {"name": "numcodecs.zlib", "configuration": {"level": 1}}, + arraybytes_codec, + zlib_codec, ] metadata = array_v3_metadata(shape=shape, chunks=chunks, codecs=codecs) chunks_dict1 = { @@ -302,7 +302,7 @@ def test_stack(self, array_v3_metadata): assert codec_dict["configuration"] == {"level": 1} assert result.metadata.fill_value == metadata.fill_value - def test_stack_empty(self, array_v3_metadata): + def test_stack_empty(self, array_v3_metadata, arraybytes_codec, zlib_codec): # both manifest arrays in this example have the same metadata properties chunks = (5, 10) shape = (5, 20) @@ -310,8 +310,8 @@ def test_stack_empty(self, array_v3_metadata): shape=shape, chunks=chunks, codecs=[ - {"name": "bytes", "configuration": {"endian": "little"}}, - {"name": "numcodecs.zlib", "configuration": {"level": 1}}, + arraybytes_codec, + zlib_codec, ], ) @@ -339,7 +339,7 @@ def test_stack_empty(self, array_v3_metadata): assert result.metadata.fill_value == metadata.fill_value -def test_refuse_combine(array_v3_metadata): +def test_refuse_combine(array_v3_metadata, arraybytes_codec, zlib_codec): # TODO test refusing to concatenate arrays that have conflicting shapes / chunk sizes chunks = (5, 1, 10) shape = (5, 1, 20) @@ -359,8 +359,8 @@ def test_refuse_combine(array_v3_metadata): shape=shape, chunks=chunks, codecs=[ - {"name": "bytes", "configuration": {"endian": "little"}}, - {"name": "numcodecs.zlib", "configuration": {"level": 1}}, + arraybytes_codec, + zlib_codec, ], ) marr2 = ManifestArray( diff --git a/virtualizarr/tests/test_writers/conftest.py b/virtualizarr/tests/test_writers/conftest.py index 8af5d858..18b67658 100644 --- a/virtualizarr/tests/test_writers/conftest.py +++ b/virtualizarr/tests/test_writers/conftest.py @@ -7,7 +7,9 @@ @pytest.fixture -def vds_with_manifest_arrays(array_v3_metadata) -> Dataset: +def vds_with_manifest_arrays( + array_v3_metadata, arraybytes_codec, zlib_codec +) -> Dataset: arr = ManifestArray( chunkmanifest=ChunkManifest( entries={"0.0": dict(path="/test.nc", offset=6144, length=48)} @@ -17,8 +19,8 @@ def vds_with_manifest_arrays(array_v3_metadata) -> Dataset: data_type=np.dtype(" Date: Thu, 13 Feb 2025 14:45:53 -0800 Subject: [PATCH 60/78] Use global vars and simple fixture for creating codec pipelines --- conftest.py | 71 +++++++++---------- virtualizarr/tests/test_codecs.py | 21 +++--- virtualizarr/tests/test_integration.py | 5 +- .../tests/test_manifests/test_array.py | 32 ++++----- virtualizarr/tests/test_writers/conftest.py | 10 +-- 5 files changed, 62 insertions(+), 77 deletions(-) diff --git a/conftest.py b/conftest.py index ac64ca5b..5eeda8a1 100644 --- a/conftest.py +++ b/conftest.py @@ -1,9 +1,9 @@ import itertools from itertools import product from pathlib import Path -from typing import Any, Callable, Dict, Mapping, Optional +from typing import Any, Callable, Mapping, Optional -import h5py +import h5py # type: ignore[import] import numpy as np import pytest import xarray as xr @@ -186,11 +186,11 @@ def create_manifestarray(array_v3_metadata): """ def _create_manifestarray( - shape: tuple | None = (5, 5), - chunks: tuple | None = (5, 5), + shape: tuple = (5, 5), + chunks: tuple = (5, 5), codecs: list[dict] | None = [ - {"configuration": {"endian": "little"}, "name": "bytes"}, - {"name": "numcodecs.zlib", "configuration": {"level": 1}}, + ARRAYBYTES_CODEC, + ZLIB_CODEC, ], ): metadata = array_v3_metadata(shape=shape, chunks=chunks, codecs=codecs) @@ -220,9 +220,9 @@ def array_v3_metadata(): def _create_metadata( shape: tuple = (5, 5), chunks: tuple = (5, 5), - data_type: str = np.dtype("int32"), + data_type: np.dtype = np.dtype("int32"), codecs: list[dict] | None = None, - fill_value: int = None, + fill_value: int | None = None, ): codecs = codecs or [{"configuration": {"endian": "little"}, "name": "bytes"}] return create_v3_array_metadata( @@ -340,34 +340,33 @@ def _gen_virtual_dataset( return _gen_virtual_dataset -# Common codec configurations used across tests -@pytest.fixture -def delta_codec() -> Dict[str, Any]: - """Delta codec configuration for array-to-array transformation.""" - return {"name": "numcodecs.delta", "configuration": {"dtype": " Dict[str, Any]: - """Bytes codec configuration for array-to-bytes transformation.""" - return {"name": "bytes", "configuration": {"endian": "little"}} - - -@pytest.fixture -def blosc_codec() -> Dict[str, Any]: - """Blosc codec configuration for bytes-to-bytes transformation.""" - return { - "name": "blosc", - "configuration": { - "cname": "zstd", - "clevel": 5, - "shuffle": "shuffle", - "typesize": 4, - }, - } +# Common codec configurations as global variables +DELTA_CODEC = {"name": "numcodecs.delta", "configuration": {"dtype": " Dict[str, Any]: - """Zlib codec configuration for bytes-to-bytes transformation.""" - return {"name": "numcodecs.zlib", "configuration": {"level": 1}} +def create_codec_pipeline(): + """Create a codec pipeline from one or more codecs.""" + + def _create_pipeline(*codecs): + """ + Create a codec pipeline from the provided codecs. + Args: + *codecs: Variable number of codec configurations to include in the pipeline + Returns: + list: List of codec configurations in the correct order + """ + return list(codecs) if codecs else [ARRAYBYTES_CODEC] + + return _create_pipeline diff --git a/virtualizarr/tests/test_codecs.py b/virtualizarr/tests/test_codecs.py index 0e1fc83f..e61aebe1 100644 --- a/virtualizarr/tests/test_codecs.py +++ b/virtualizarr/tests/test_codecs.py @@ -4,6 +4,11 @@ from zarr.core.codec_pipeline import BatchedCodecPipeline from zarr.registry import get_codec_class +from conftest import ( + ARRAYBYTES_CODEC, + BLOSC_CODEC, + DELTA_CODEC, +) from virtualizarr.codecs import convert_to_codec_pipeline, get_codecs @@ -36,14 +41,10 @@ def test_manifest_array_zarr_v3_default(self, create_manifestarray): assert actual_codecs == expected_codecs def test_manifest_array_zarr_v3_with_codecs( - self, create_manifestarray, delta_codec, arraybytes_codec, blosc_codec + self, create_manifestarray, create_codec_pipeline ): """Test get_codecs with ManifestArray using multiple v3 codecs.""" - test_codecs = [ - delta_codec, - arraybytes_codec, - blosc_codec, - ] + test_codecs = create_codec_pipeline(DELTA_CODEC, ARRAYBYTES_CODEC, BLOSC_CODEC) manifest_array = create_manifestarray(codecs=test_codecs) actual_codecs = get_codecs(manifest_array) assert actual_codecs == tuple( @@ -59,13 +60,9 @@ def test_zarr_v3_default(self): actual_codecs = get_codecs(zarr_array) assert isinstance(actual_codecs[0], BytesCodec) - def test_zarr_v3_with_codecs(self, delta_codec, arraybytes_codec, blosc_codec): + def test_zarr_v3_with_codecs(self, create_codec_pipeline): """Test get_codecs with Zarr array using multiple v3 codecs.""" - test_codecs = [ - delta_codec, - arraybytes_codec, - blosc_codec, - ] + test_codecs = create_codec_pipeline(DELTA_CODEC, ARRAYBYTES_CODEC, BLOSC_CODEC) zarr_array = self.create_zarr_array(codecs=test_codecs) actual_codecs = get_codecs(zarr_array) assert actual_codecs == tuple( diff --git a/virtualizarr/tests/test_integration.py b/virtualizarr/tests/test_integration.py index 513a61c5..610b0f41 100644 --- a/virtualizarr/tests/test_integration.py +++ b/virtualizarr/tests/test_integration.py @@ -6,6 +6,7 @@ import xarray as xr import xarray.testing as xrt +from conftest import ARRAYBYTES_CODEC, ZLIB_CODEC from virtualizarr import open_virtual_dataset from virtualizarr.manifests import ChunkManifest, ManifestArray from virtualizarr.tests import ( @@ -242,7 +243,7 @@ def test_non_dimension_coordinates(self, tmpdir, roundtrip_func, hdf_backend): reason="zarr-python 3.0 does not support datetime and timedelta data types" ) def test_datetime64_dtype_fill_value( - self, tmpdir, roundtrip_func, array_v3_metadata, zlib_codec + self, tmpdir, roundtrip_func, array_v3_metadata, create_codec_pipeline ): chunks_dict = { "0.0.0": {"path": "/foo.nc", "offset": 100, "length": 100}, @@ -253,7 +254,7 @@ def test_datetime64_dtype_fill_value( metadata = array_v3_metadata( shape=shape, chunks=chunks, - codecs=[zlib_codec], + codecs=create_codec_pipeline(ARRAYBYTES_CODEC, ZLIB_CODEC), data_type=np.dtype("M8[ns]"), ) marr1 = ManifestArray(metadata=metadata, chunkmanifest=manifest) diff --git a/virtualizarr/tests/test_manifests/test_array.py b/virtualizarr/tests/test_manifests/test_array.py index deabaf64..059ad2dc 100644 --- a/virtualizarr/tests/test_manifests/test_array.py +++ b/virtualizarr/tests/test_manifests/test_array.py @@ -2,6 +2,10 @@ import pytest from zarr.core.metadata.v3 import ArrayV3Metadata +from conftest import ( + ARRAYBYTES_CODEC, + ZLIB_CODEC, +) from virtualizarr.manifests import ChunkManifest, ManifestArray @@ -226,13 +230,10 @@ def test_concat(self, array_v3_metadata): assert concatenated.dtype == np.dtype("int32") # FAILING: TypeError: no implementation found for 'numpy.concatenate' on types that implement __array_function__: [, ] - def test_concat_empty(self, array_v3_metadata, arraybytes_codec, zlib_codec): + def test_concat_empty(self, array_v3_metadata, create_codec_pipeline): chunks = (5, 1, 10) shape = (5, 1, 20) - codecs = [ - arraybytes_codec, - zlib_codec, - ] + codecs = create_codec_pipeline(ARRAYBYTES_CODEC, ZLIB_CODEC) metadata = array_v3_metadata(shape=shape, chunks=chunks, codecs=codecs) empty_chunks_dict = {} empty_chunk_manifest = ChunkManifest(entries=empty_chunks_dict, shape=(1, 1, 2)) @@ -264,14 +265,11 @@ def test_concat_empty(self, array_v3_metadata, arraybytes_codec, zlib_codec): class TestStack: - def test_stack(self, array_v3_metadata, arraybytes_codec, zlib_codec): + def test_stack(self, array_v3_metadata, create_codec_pipeline): # both manifest arrays in this example have the same zarray properties chunks = (5, 10) shape = (5, 20) - codecs = [ - arraybytes_codec, - zlib_codec, - ] + codecs = create_codec_pipeline(ARRAYBYTES_CODEC, ZLIB_CODEC) metadata = array_v3_metadata(shape=shape, chunks=chunks, codecs=codecs) chunks_dict1 = { "0.0": {"path": "/foo.nc", "offset": 100, "length": 100}, @@ -302,17 +300,14 @@ def test_stack(self, array_v3_metadata, arraybytes_codec, zlib_codec): assert codec_dict["configuration"] == {"level": 1} assert result.metadata.fill_value == metadata.fill_value - def test_stack_empty(self, array_v3_metadata, arraybytes_codec, zlib_codec): + def test_stack_empty(self, array_v3_metadata, create_codec_pipeline): # both manifest arrays in this example have the same metadata properties chunks = (5, 10) shape = (5, 20) metadata = array_v3_metadata( shape=shape, chunks=chunks, - codecs=[ - arraybytes_codec, - zlib_codec, - ], + codecs=create_codec_pipeline(ARRAYBYTES_CODEC, ZLIB_CODEC), ) chunks_dict1 = {} @@ -339,7 +334,7 @@ def test_stack_empty(self, array_v3_metadata, arraybytes_codec, zlib_codec): assert result.metadata.fill_value == metadata.fill_value -def test_refuse_combine(array_v3_metadata, arraybytes_codec, zlib_codec): +def test_refuse_combine(array_v3_metadata, create_codec_pipeline): # TODO test refusing to concatenate arrays that have conflicting shapes / chunk sizes chunks = (5, 1, 10) shape = (5, 1, 20) @@ -358,10 +353,7 @@ def test_refuse_combine(array_v3_metadata, arraybytes_codec, zlib_codec): metadata_different_codecs = array_v3_metadata( shape=shape, chunks=chunks, - codecs=[ - arraybytes_codec, - zlib_codec, - ], + codecs=create_codec_pipeline(ARRAYBYTES_CODEC, ZLIB_CODEC), ) marr2 = ManifestArray( metadata=metadata_different_codecs, chunkmanifest=chunkmanifest2 diff --git a/virtualizarr/tests/test_writers/conftest.py b/virtualizarr/tests/test_writers/conftest.py index 18b67658..85f3b375 100644 --- a/virtualizarr/tests/test_writers/conftest.py +++ b/virtualizarr/tests/test_writers/conftest.py @@ -3,13 +3,12 @@ from xarray import Dataset from xarray.core.variable import Variable +from conftest import ARRAYBYTES_CODEC, ZLIB_CODEC from virtualizarr.manifests import ChunkManifest, ManifestArray @pytest.fixture -def vds_with_manifest_arrays( - array_v3_metadata, arraybytes_codec, zlib_codec -) -> Dataset: +def vds_with_manifest_arrays(array_v3_metadata, create_codec_pipeline) -> Dataset: arr = ManifestArray( chunkmanifest=ChunkManifest( entries={"0.0": dict(path="/test.nc", offset=6144, length=48)} @@ -18,10 +17,7 @@ def vds_with_manifest_arrays( shape=(2, 3), data_type=np.dtype(" Date: Thu, 13 Feb 2025 15:20:48 -0800 Subject: [PATCH 61/78] Remove redundant create_codec_pipeline fixture --- conftest.py | 17 --- virtualizarr/tests/test_codecs.py | 101 +++++++++--------- virtualizarr/tests/test_integration.py | 4 +- .../tests/test_manifests/test_array.py | 16 +-- virtualizarr/tests/test_writers/conftest.py | 4 +- 5 files changed, 65 insertions(+), 77 deletions(-) diff --git a/conftest.py b/conftest.py index 5eeda8a1..b45a5ac8 100644 --- a/conftest.py +++ b/conftest.py @@ -353,20 +353,3 @@ def _gen_virtual_dataset( }, } ZLIB_CODEC = {"name": "numcodecs.zlib", "configuration": {"level": 1}} - - -@pytest.fixture -def create_codec_pipeline(): - """Create a codec pipeline from one or more codecs.""" - - def _create_pipeline(*codecs): - """ - Create a codec pipeline from the provided codecs. - Args: - *codecs: Variable number of codec configurations to include in the pipeline - Returns: - list: List of codec configurations in the correct order - """ - return list(codecs) if codecs else [ARRAYBYTES_CODEC] - - return _create_pipeline diff --git a/virtualizarr/tests/test_codecs.py b/virtualizarr/tests/test_codecs.py index e61aebe1..74093d33 100644 --- a/virtualizarr/tests/test_codecs.py +++ b/virtualizarr/tests/test_codecs.py @@ -8,6 +8,7 @@ ARRAYBYTES_CODEC, BLOSC_CODEC, DELTA_CODEC, + ZLIB_CODEC, ) from virtualizarr.codecs import convert_to_codec_pipeline, get_codecs @@ -40,11 +41,9 @@ def test_manifest_array_zarr_v3_default(self, create_manifestarray): expected_codecs = tuple([BytesCodec(endian="little")]) assert actual_codecs == expected_codecs - def test_manifest_array_zarr_v3_with_codecs( - self, create_manifestarray, create_codec_pipeline - ): + def test_manifest_array_zarr_v3_with_codecs(self, create_manifestarray): """Test get_codecs with ManifestArray using multiple v3 codecs.""" - test_codecs = create_codec_pipeline(DELTA_CODEC, ARRAYBYTES_CODEC, BLOSC_CODEC) + test_codecs = [DELTA_CODEC, ARRAYBYTES_CODEC, BLOSC_CODEC] manifest_array = create_manifestarray(codecs=test_codecs) actual_codecs = get_codecs(manifest_array) assert actual_codecs == tuple( @@ -60,9 +59,9 @@ def test_zarr_v3_default(self): actual_codecs = get_codecs(zarr_array) assert isinstance(actual_codecs[0], BytesCodec) - def test_zarr_v3_with_codecs(self, create_codec_pipeline): + def test_zarr_v3_with_codecs(self): """Test get_codecs with Zarr array using multiple v3 codecs.""" - test_codecs = create_codec_pipeline(DELTA_CODEC, ARRAYBYTES_CODEC, BLOSC_CODEC) + test_codecs = [DELTA_CODEC, ARRAYBYTES_CODEC, BLOSC_CODEC] zarr_array = self.create_zarr_array(codecs=test_codecs) actual_codecs = get_codecs(zarr_array) assert actual_codecs == tuple( @@ -82,47 +81,53 @@ def test_zarr_v2_error(self): get_codecs(zarr_array) -def test_convert_to_codec_pipeline(): - expected_default_codecs = BatchedCodecPipeline( - array_array_codecs=(), - array_bytes_codec=BytesCodec(endian="little"), - bytes_bytes_codecs=(), - batch_size=1, - ) - # Test with just dtype (default codec pipeline) +@pytest.mark.parametrize( + "input_codecs,expected_pipeline", + [ + # Case 1: No codecs - should result in just BytesCodec + ( + None, + BatchedCodecPipeline( + array_array_codecs=(), + array_bytes_codec=BytesCodec(endian="little"), + bytes_bytes_codecs=(), + batch_size=1, + ), + ), + # Case 2: Delta codec - should result in DeltaCodec + BytesCodec + ( + [DELTA_CODEC], + BatchedCodecPipeline( + array_array_codecs=( + get_codec_class("numcodecs.delta").from_dict(DELTA_CODEC), + ), # type: ignore[arg-type] + array_bytes_codec=BytesCodec(endian="little"), + bytes_bytes_codecs=(), + batch_size=1, + ), + ), + # Case 3: Delta + Blosc + Zlib - should result in all codecs + BytesCodec + ( + [DELTA_CODEC, BLOSC_CODEC, ZLIB_CODEC], + BatchedCodecPipeline( + array_array_codecs=( + get_codec_class("numcodecs.delta").from_dict(DELTA_CODEC), + ), # type: ignore[arg-type] + array_bytes_codec=BytesCodec(endian="little"), + bytes_bytes_codecs=( + get_codec_class(key="blosc").from_dict(BLOSC_CODEC), # type: ignore[arg-type] + get_codec_class("numcodecs.zlib").from_dict(ZLIB_CODEC), # type: ignore[arg-type] + ), + batch_size=1, + ), + ), + ], +) +def test_convert_to_codec_pipeline_scenarios(input_codecs, expected_pipeline): + """Test different scenarios for convert_to_codec_pipeline function.""" dtype = np.dtype(", ] - def test_concat_empty(self, array_v3_metadata, create_codec_pipeline): + def test_concat_empty(self, array_v3_metadata): chunks = (5, 1, 10) shape = (5, 1, 20) - codecs = create_codec_pipeline(ARRAYBYTES_CODEC, ZLIB_CODEC) + codecs = [ARRAYBYTES_CODEC, ZLIB_CODEC] metadata = array_v3_metadata(shape=shape, chunks=chunks, codecs=codecs) empty_chunks_dict = {} empty_chunk_manifest = ChunkManifest(entries=empty_chunks_dict, shape=(1, 1, 2)) @@ -265,11 +265,11 @@ def test_concat_empty(self, array_v3_metadata, create_codec_pipeline): class TestStack: - def test_stack(self, array_v3_metadata, create_codec_pipeline): + def test_stack(self, array_v3_metadata): # both manifest arrays in this example have the same zarray properties chunks = (5, 10) shape = (5, 20) - codecs = create_codec_pipeline(ARRAYBYTES_CODEC, ZLIB_CODEC) + codecs = [ARRAYBYTES_CODEC, ZLIB_CODEC] metadata = array_v3_metadata(shape=shape, chunks=chunks, codecs=codecs) chunks_dict1 = { "0.0": {"path": "/foo.nc", "offset": 100, "length": 100}, @@ -300,14 +300,14 @@ def test_stack(self, array_v3_metadata, create_codec_pipeline): assert codec_dict["configuration"] == {"level": 1} assert result.metadata.fill_value == metadata.fill_value - def test_stack_empty(self, array_v3_metadata, create_codec_pipeline): + def test_stack_empty(self, array_v3_metadata): # both manifest arrays in this example have the same metadata properties chunks = (5, 10) shape = (5, 20) metadata = array_v3_metadata( shape=shape, chunks=chunks, - codecs=create_codec_pipeline(ARRAYBYTES_CODEC, ZLIB_CODEC), + codecs=[ARRAYBYTES_CODEC, ZLIB_CODEC], ) chunks_dict1 = {} @@ -334,7 +334,7 @@ def test_stack_empty(self, array_v3_metadata, create_codec_pipeline): assert result.metadata.fill_value == metadata.fill_value -def test_refuse_combine(array_v3_metadata, create_codec_pipeline): +def test_refuse_combine(array_v3_metadata): # TODO test refusing to concatenate arrays that have conflicting shapes / chunk sizes chunks = (5, 1, 10) shape = (5, 1, 20) @@ -353,7 +353,7 @@ def test_refuse_combine(array_v3_metadata, create_codec_pipeline): metadata_different_codecs = array_v3_metadata( shape=shape, chunks=chunks, - codecs=create_codec_pipeline(ARRAYBYTES_CODEC, ZLIB_CODEC), + codecs=[ARRAYBYTES_CODEC, ZLIB_CODEC], ) marr2 = ManifestArray( metadata=metadata_different_codecs, chunkmanifest=chunkmanifest2 diff --git a/virtualizarr/tests/test_writers/conftest.py b/virtualizarr/tests/test_writers/conftest.py index 85f3b375..1a8ffdbd 100644 --- a/virtualizarr/tests/test_writers/conftest.py +++ b/virtualizarr/tests/test_writers/conftest.py @@ -8,7 +8,7 @@ @pytest.fixture -def vds_with_manifest_arrays(array_v3_metadata, create_codec_pipeline) -> Dataset: +def vds_with_manifest_arrays(array_v3_metadata) -> Dataset: arr = ManifestArray( chunkmanifest=ChunkManifest( entries={"0.0": dict(path="/test.nc", offset=6144, length=48)} @@ -17,7 +17,7 @@ def vds_with_manifest_arrays(array_v3_metadata, create_codec_pipeline) -> Datase shape=(2, 3), data_type=np.dtype(" Date: Thu, 13 Feb 2025 15:36:04 -0800 Subject: [PATCH 62/78] Fix docstring --- virtualizarr/codecs.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/virtualizarr/codecs.py b/virtualizarr/codecs.py index 95cb8f24..5b5ab024 100644 --- a/virtualizarr/codecs.py +++ b/virtualizarr/codecs.py @@ -53,7 +53,7 @@ def convert_to_codec_pipeline( codecs: list[dict] | None = [], ) -> BatchedCodecPipeline: """ - Convert compressor, filters, serializer, and dtype to a pipeline of ZarrCodecs. + Convert list of codecs to valid BatchedCodecPipeline. Parameters ---------- From 4b555b62c98eacc0029903b512bdb01a48eba2a7 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Thu, 13 Feb 2025 15:40:18 -0800 Subject: [PATCH 63/78] Use create_v3_array_metadata in from_kerchunk_refs --- virtualizarr/tests/test_codecs.py | 8 ++++---- virtualizarr/translators/kerchunk.py | 17 ++++------------- 2 files changed, 8 insertions(+), 17 deletions(-) diff --git a/virtualizarr/tests/test_codecs.py b/virtualizarr/tests/test_codecs.py index 74093d33..aa2490b6 100644 --- a/virtualizarr/tests/test_codecs.py +++ b/virtualizarr/tests/test_codecs.py @@ -99,8 +99,8 @@ def test_zarr_v2_error(self): [DELTA_CODEC], BatchedCodecPipeline( array_array_codecs=( - get_codec_class("numcodecs.delta").from_dict(DELTA_CODEC), - ), # type: ignore[arg-type] + get_codec_class("numcodecs.delta").from_dict(DELTA_CODEC), # type: ignore[arg-type] + ), array_bytes_codec=BytesCodec(endian="little"), bytes_bytes_codecs=(), batch_size=1, @@ -111,8 +111,8 @@ def test_zarr_v2_error(self): [DELTA_CODEC, BLOSC_CODEC, ZLIB_CODEC], BatchedCodecPipeline( array_array_codecs=( - get_codec_class("numcodecs.delta").from_dict(DELTA_CODEC), - ), # type: ignore[arg-type] + get_codec_class("numcodecs.delta").from_dict(DELTA_CODEC), # type: ignore[arg-type] + ), array_bytes_codec=BytesCodec(endian="little"), bytes_bytes_codecs=( get_codec_class(key="blosc").from_dict(BLOSC_CODEC), # type: ignore[arg-type] diff --git a/virtualizarr/translators/kerchunk.py b/virtualizarr/translators/kerchunk.py index a621f614..ecd46914 100644 --- a/virtualizarr/translators/kerchunk.py +++ b/virtualizarr/translators/kerchunk.py @@ -9,11 +9,11 @@ from zarr.core.metadata.v2 import ArrayV2Metadata from virtualizarr.codecs import ( - convert_to_codec_pipeline, num_codec_config_to_configurable, ) from virtualizarr.manifests import ChunkManifest, ManifestArray from virtualizarr.manifests.manifest import ChunkEntry, ChunkKey +from virtualizarr.manifests.utils import create_v3_array_metadata from virtualizarr.readers.common import separate_coords from virtualizarr.types.kerchunk import ( KerchunkArrRefs, @@ -84,21 +84,12 @@ def from_kerchunk_refs(decoded_arr_refs_zarray) -> "ArrayV3Metadata": numcodec_configs = [ num_codec_config_to_configurable(config) for config in codec_configs ] - return ArrayV3Metadata( - chunk_grid={ - "name": "regular", - "configuration": {"chunk_shape": tuple(decoded_arr_refs_zarray["chunks"])}, - }, - codecs=convert_to_codec_pipeline( - dtype=dtype, - codecs=numcodec_configs, - ), + return create_v3_array_metadata( + chunk_shape=tuple(decoded_arr_refs_zarray["chunks"]), data_type=dtype, + codecs=numcodec_configs, fill_value=fill_value, shape=tuple(decoded_arr_refs_zarray["shape"]), - chunk_key_encoding={"name": "default"}, - attributes={}, - dimension_names=None, ) From c245b0a5c2d1c6c4eba22557b22c497473284853 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Fri, 14 Feb 2025 10:12:54 -0800 Subject: [PATCH 64/78] Add links to zarr-python 3.0 issues for big endian, datetime and timedelta data types --- virtualizarr/tests/test_backend.py | 8 ++++++-- virtualizarr/tests/test_integration.py | 2 +- virtualizarr/tests/test_readers/test_fits.py | 4 +++- virtualizarr/tests/test_readers/test_netcdf3.py | 4 +++- 4 files changed, 13 insertions(+), 5 deletions(-) diff --git a/virtualizarr/tests/test_backend.py b/virtualizarr/tests/test_backend.py index 23af780f..e50223c5 100644 --- a/virtualizarr/tests/test_backend.py +++ b/virtualizarr/tests/test_backend.py @@ -232,7 +232,9 @@ class TestReadFromURL: pytest.param( "netcdf3", "https://github.com/pydata/xarray-data/raw/master/air_temperature.nc", - marks=pytest.mark.xfail(reason="Not supported by zarr-python 3.0"), + marks=pytest.mark.xfail( + reason="Big endian not yet supported by zarr-python 3.0" + ), # https://github.com/zarr-developers/zarr-python/issues/2324 ), ( "netcdf4", @@ -258,7 +260,9 @@ class TestReadFromURL: pytest.param( "fits", "https://fits.gsfc.nasa.gov/samples/WFPC2u5780205r_c0fx.fits", - marks=pytest.mark.xfail(reason="Not supported by zarr-python 3.0"), + marks=pytest.mark.xfail( + reason="Big endian not yet supported by zarr-python 3.0" + ), # https://github.com/zarr-developers/zarr-python/issues/2324 ), ( "jpg", diff --git a/virtualizarr/tests/test_integration.py b/virtualizarr/tests/test_integration.py index aef1560c..5821e8ce 100644 --- a/virtualizarr/tests/test_integration.py +++ b/virtualizarr/tests/test_integration.py @@ -240,7 +240,7 @@ def test_non_dimension_coordinates(self, tmpdir, roundtrip_func, hdf_backend): assert ds.coords[coord].attrs == roundtrip.coords[coord].attrs @pytest.mark.xfail( - reason="zarr-python 3.0 does not support datetime and timedelta data types" + reason="Datetime and timedelta data types not yet supported by zarr-python 3.0" # https://github.com/zarr-developers/zarr-python/issues/2616 ) def test_datetime64_dtype_fill_value( self, tmpdir, roundtrip_func, array_v3_metadata diff --git a/virtualizarr/tests/test_readers/test_fits.py b/virtualizarr/tests/test_readers/test_fits.py index 65cb7422..acaccec8 100644 --- a/virtualizarr/tests/test_readers/test_fits.py +++ b/virtualizarr/tests/test_readers/test_fits.py @@ -9,7 +9,9 @@ @requires_kerchunk @requires_network -@pytest.mark.xfail(reason="zarr-python 3.0 does not yet support big endian") +@pytest.mark.xfail( + reason="Big endian not yet supported by zarr-python 3.0" +) # https://github.com/zarr-developers/zarr-python/issues/2324 def test_open_hubble_data(): # data from https://registry.opendata.aws/hst/ vds = open_virtual_dataset( diff --git a/virtualizarr/tests/test_readers/test_netcdf3.py b/virtualizarr/tests/test_readers/test_netcdf3.py index 208d8d8f..e6baa3f2 100644 --- a/virtualizarr/tests/test_readers/test_netcdf3.py +++ b/virtualizarr/tests/test_readers/test_netcdf3.py @@ -8,7 +8,9 @@ @requires_scipy -@pytest.mark.xfail(reason="zarr-python 3.0 does not support big endian") +@pytest.mark.xfail( + reason="Big endian not yet supported by zarr-python 3.0" +) # https://github.com/zarr-developers/zarr-python/issues/2324 def test_read_netcdf3(netcdf3_file, array_v3_metadata): filepath = str(netcdf3_file) vds = open_virtual_dataset(filepath) From 23ac77617f84418f669ed372d8b12b556cd28f5f Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Fri, 14 Feb 2025 10:57:18 -0800 Subject: [PATCH 65/78] Reorganize conftest --- conftest.py | 280 ++++++++++-------- virtualizarr/tests/test_codecs.py | 8 +- .../tests/test_manifests/test_array.py | 26 +- .../tests/test_writers/test_icechunk.py | 56 ++-- 4 files changed, 192 insertions(+), 178 deletions(-) diff --git a/conftest.py b/conftest.py index b45a5ac8..a06d8cf1 100644 --- a/conftest.py +++ b/conftest.py @@ -1,20 +1,25 @@ +"""Pytest configuration and fixtures for virtualizarr tests.""" + +# Standard library imports import itertools -from itertools import product from pathlib import Path from typing import Any, Callable, Mapping, Optional +# Third-party imports import h5py # type: ignore[import] import numpy as np import pytest import xarray as xr from xarray.core.variable import Variable +# Local imports from virtualizarr.manifests import ChunkManifest, ManifestArray from virtualizarr.manifests.manifest import join from virtualizarr.manifests.utils import create_v3_array_metadata from virtualizarr.utils import ceildiv +# Pytest configuration def pytest_addoption(parser): """Add command-line flags for pytest.""" parser.addoption( @@ -25,65 +30,152 @@ def pytest_addoption(parser): def pytest_runtest_setup(item): - # based on https://stackoverflow.com/questions/47559524 + """Skip network tests unless explicitly enabled.""" if "network" in item.keywords and not item.config.getoption("--run-network-tests"): pytest.skip( "set --run-network-tests to run tests requiring an internet connection" ) +# Common codec configurations +DELTA_CODEC = {"name": "numcodecs.delta", "configuration": {"dtype": " dict[str, dict[str, Any]]: + """ + Generate chunk entries for a manifest based on shape and chunks. + + Parameters + ---------- + shape : tuple of int + The shape of the array + chunks : tuple of int + The chunk size for each dimension + entry_generator : callable + Function that takes chunk indices and returns an entry dict + + Returns + ------- + dict + Mapping of chunk keys to entry dictionaries + """ + chunk_grid_shape = tuple( + ceildiv(axis_length, chunk_length) + for axis_length, chunk_length in zip(shape, chunks) + ) + + if chunk_grid_shape == (): + return {"0": entry_generator((0,))} + + all_possible_combos = itertools.product( + *[range(length) for length in chunk_grid_shape] + ) + return {join(ind): entry_generator(ind) for ind in all_possible_combos} + + +def _offset_from_chunk_key(ind: tuple[int, ...]) -> int: + """Generate an offset value from chunk indices.""" + return sum(ind) * 10 + + +def _length_from_chunk_key(ind: tuple[int, ...]) -> int: + """Generate a length value from chunk indices.""" + return sum(ind) + 5 + + +def _entry_from_chunk_key(ind: tuple[int, ...]) -> dict[str, str | int]: + """Generate a (somewhat) unique manifest entry from a given chunk key.""" + entry = { + "path": f"/foo.{str(join(ind))}.nc", + "offset": _offset_from_chunk_key(ind), + "length": _length_from_chunk_key(ind), + } + return entry # type: ignore[return-value] + + +def _generate_chunk_manifest( + netcdf4_file: str, + shape: tuple[int, ...], + chunks: tuple[int, ...], + offset: int = 6144, + length: int = 48, +) -> ChunkManifest: + """Generate a chunk manifest with sequential offsets for each chunk.""" + current_offset = [offset] # Use list to allow mutation in closure + + def sequential_entry_generator(ind: tuple[int, ...]) -> dict[str, Any]: + entry = { + "path": netcdf4_file, + "offset": current_offset[0], + "length": length, + } + current_offset[0] += length + return entry + + entries = _generate_chunk_entries(shape, chunks, sequential_entry_generator) + return ChunkManifest(entries) + + +# NetCDF file fixtures @pytest.fixture def empty_netcdf4_file(tmp_path: Path) -> str: + """Create an empty NetCDF4 file.""" filepath = tmp_path / "empty.nc" - - # Set up example xarray dataset - with xr.Dataset() as ds: # Save it to disk as netCDF (in temporary directory) + with xr.Dataset() as ds: ds.to_netcdf(filepath, format="NETCDF4") - return str(filepath) @pytest.fixture def netcdf4_file(tmp_path: Path) -> str: + """Create a NetCDF4 file with air temperature data.""" filepath = tmp_path / "air.nc" - - # Set up example xarray dataset with xr.tutorial.open_dataset("air_temperature") as ds: - # Save it to disk as netCDF (in temporary directory) ds.to_netcdf(filepath, format="NETCDF4") - return str(filepath) @pytest.fixture def netcdf4_file_with_data_in_multiple_groups(tmp_path: Path) -> str: + """Create a NetCDF4 file with data in multiple groups.""" filepath = tmp_path / "test.nc" - ds1 = xr.DataArray([1, 2, 3], name="foo").to_dataset() ds1.to_netcdf(filepath) ds2 = xr.DataArray([4, 5], name="bar").to_dataset() ds2.to_netcdf(filepath, group="subgroup", mode="a") - return str(filepath) @pytest.fixture def netcdf4_files_factory(tmp_path: Path) -> Callable: + """Factory fixture to create multiple NetCDF4 files.""" + def create_netcdf4_files( encoding: Optional[Mapping[str, Mapping[str, Any]]] = None, ) -> tuple[str, str]: filepath1 = tmp_path / "air1.nc" filepath2 = tmp_path / "air2.nc" - with xr.tutorial.open_dataset("air_temperature") as ds: - # Split dataset into two parts ds1 = ds.isel(time=slice(None, 1460)) ds2 = ds.isel(time=slice(1460, None)) - - # Save datasets to disk as NetCDF in the temporary directory with the provided encoding ds1.to_netcdf(filepath1, encoding=encoding) ds2.to_netcdf(filepath2, encoding=encoding) - return str(filepath1), str(filepath2) return create_netcdf4_files @@ -91,16 +183,16 @@ def create_netcdf4_files( @pytest.fixture def netcdf4_file_with_2d_coords(tmp_path: Path) -> str: + """Create a NetCDF4 file with 2D coordinates.""" filepath = tmp_path / "ROMS_example.nc" - with xr.tutorial.open_dataset("ROMS_example") as ds: ds.to_netcdf(filepath, format="NETCDF4") - return str(filepath) @pytest.fixture def netcdf4_virtual_dataset(netcdf4_file): + """Create a virtual dataset from a NetCDF4 file.""" from virtualizarr import open_virtual_dataset return open_virtual_dataset(netcdf4_file, indexes={}) @@ -108,115 +200,58 @@ def netcdf4_virtual_dataset(netcdf4_file): @pytest.fixture def netcdf4_inlined_ref(netcdf4_file): + """Create an inlined reference from a NetCDF4 file.""" from kerchunk.hdf import SingleHdf5ToZarr return SingleHdf5ToZarr(netcdf4_file, inline_threshold=1000).translate() +# HDF5 file fixtures @pytest.fixture def hdf5_groups_file(tmp_path: Path) -> str: + """Create an HDF5 file with groups.""" filepath = tmp_path / "air.nc" - - # Set up example xarray dataset with xr.tutorial.open_dataset("air_temperature") as ds: - # Save it to disk as netCDF (in temporary directory) ds.to_netcdf(filepath, format="NETCDF4", group="test/group") - return str(filepath) @pytest.fixture def hdf5_empty(tmp_path: Path) -> str: + """Create an empty HDF5 file.""" filepath = tmp_path / "empty.nc" - with h5py.File(filepath, "w") as f: dataset = f.create_dataset("empty", shape=(), dtype="float32") dataset.attrs["empty"] = "true" - return str(filepath) @pytest.fixture def hdf5_scalar(tmp_path: Path) -> str: + """Create an HDF5 file with a scalar dataset.""" filepath = tmp_path / "scalar.nc" - with h5py.File(filepath, "w") as f: dataset = f.create_dataset("scalar", data=0.1, dtype="float32") dataset.attrs["scalar"] = "true" - return str(filepath) @pytest.fixture def simple_netcdf4(tmp_path: Path) -> str: + """Create a simple NetCDF4 file with a single variable.""" filepath = tmp_path / "simple.nc" - arr = np.arange(12, dtype=np.dtype("int32")).reshape(3, 4) var = Variable(data=arr, dims=["x", "y"]) ds = xr.Dataset({"foo": var}) ds.to_netcdf(filepath) - return str(filepath) -def offset_from_chunk_key(ind: tuple[int, ...]) -> int: - return sum(ind) * 10 - - -def length_from_chunk_key(ind: tuple[int, ...]) -> int: - return sum(ind) + 5 - - -def entry_from_chunk_key(ind: tuple[int, ...]) -> dict[str, str | int]: - """Generate a (somewhat) unique manifest entry from a given chunk key""" - entry = { - "path": f"/foo.{str(join(ind))}.nc", - "offset": offset_from_chunk_key(ind), - "length": length_from_chunk_key(ind), - } - return entry # type: ignore[return-value] - - -@pytest.fixture -def create_manifestarray(array_v3_metadata): - """ - Create an example ManifestArray with sensible defaults. - - The manifest is populated with a (somewhat) unique path, offset, and length for each key. - """ - - def _create_manifestarray( - shape: tuple = (5, 5), - chunks: tuple = (5, 5), - codecs: list[dict] | None = [ - ARRAYBYTES_CODEC, - ZLIB_CODEC, - ], - ): - metadata = array_v3_metadata(shape=shape, chunks=chunks, codecs=codecs) - chunk_grid_shape = tuple( - ceildiv(axis_length, chunk_length) - for axis_length, chunk_length in zip(shape, chunks) - ) - - if chunk_grid_shape == (): - d = {"0": entry_from_chunk_key((0,))} - else: - # create every possible combination of keys - all_possible_combos = itertools.product( - *[range(length) for length in chunk_grid_shape] - ) - d = {join(ind): entry_from_chunk_key(ind) for ind in all_possible_combos} - - chunkmanifest = ChunkManifest(entries=d) - - return ManifestArray(chunkmanifest=chunkmanifest, metadata=metadata) - - return _create_manifestarray - - +# Zarr ArrayV3Metadata, ManifestArray, virtual xr.Variable and virtual xr.Dataset fixtures @pytest.fixture def array_v3_metadata(): + """Create V3 array metadata with sensible defaults.""" + def _create_metadata( shape: tuple = (5, 5), chunks: tuple = (5, 5), @@ -236,33 +271,32 @@ def _create_metadata( return _create_metadata -def generate_chunk_manifest( - netcdf4_file: str, - shape: tuple[int, ...], - chunks: tuple[int, ...], - offset=6144, - length=48, -) -> ChunkManifest: - chunk_dict = {} - num_chunks = [shape[i] // chunks[i] for i in range(len(shape))] - offset = offset - - # Generate all possible chunk indices using Cartesian product - for chunk_indices in product(*[range(n) for n in num_chunks]): - chunk_index = ".".join(map(str, chunk_indices)) - chunk_dict[chunk_index] = { - "path": netcdf4_file, - "offset": offset, - "length": length, - } - offset += length # Increase offset for each chunk +@pytest.fixture +def manifest_array(array_v3_metadata): + """ + Create an example ManifestArray with sensible defaults. + + The manifest is populated with a (somewhat) unique path, offset, and length for each key. + """ + + def _manifest_array( + shape: tuple = (5, 5), + chunks: tuple = (5, 5), + codecs: list[dict] | None = [ARRAYBYTES_CODEC, ZLIB_CODEC], + ): + metadata = array_v3_metadata(shape=shape, chunks=chunks, codecs=codecs) + entries = _generate_chunk_entries(shape, chunks, _entry_from_chunk_key) + chunkmanifest = ChunkManifest(entries=entries) + return ManifestArray(chunkmanifest=chunkmanifest, metadata=metadata) - return ChunkManifest(chunk_dict) + return _manifest_array @pytest.fixture -def gen_virtual_variable(array_v3_metadata: Callable) -> Callable: - def _gen_virtual_variable( +def virtual_variable(array_v3_metadata: Callable) -> Callable: + """Generate a virtual variable with configurable parameters.""" + + def _virtual_variable( file_uri: str, shape: tuple[int, ...] = (3, 4), chunk_shape: tuple[int, ...] = (3, 4), @@ -275,7 +309,7 @@ def _gen_virtual_variable( dims: list[str] = [], attrs: dict[str, Any] = {}, ) -> xr.Variable: - manifest = generate_chunk_manifest( + manifest = _generate_chunk_manifest( file_uri, shape=shape, chunks=chunk_shape, @@ -297,12 +331,14 @@ def _gen_virtual_variable( attrs=attrs, ) - return _gen_virtual_variable + return _virtual_variable @pytest.fixture -def gen_virtual_dataset(gen_virtual_variable: Callable) -> Callable: - def _gen_virtual_dataset( +def virtual_dataset(virtual_variable: Callable) -> Callable: + """Generate a virtual dataset with configurable parameters.""" + + def _virtual_dataset( file_uri: str, shape: tuple[int, ...] = (3, 4), chunk_shape: tuple[int, ...] = (3, 4), @@ -317,7 +353,7 @@ def _gen_virtual_dataset( coords: Optional[xr.Coordinates] = None, ) -> xr.Dataset: with xr.open_dataset(file_uri) as ds: - var = gen_virtual_variable( + var = virtual_variable( file_uri=file_uri, shape=shape, chunk_shape=chunk_shape, @@ -330,26 +366,10 @@ def _gen_virtual_dataset( dims=dims or [str(name) for name in ds.dims], attrs=ds[variable_name].attrs, ) - return xr.Dataset( {variable_name: var}, coords=coords, attrs=ds.attrs, ) - return _gen_virtual_dataset - - -# Common codec configurations as global variables -DELTA_CODEC = {"name": "numcodecs.delta", "configuration": {"dtype": " Date: Fri, 14 Feb 2025 16:23:22 -0800 Subject: [PATCH 66/78] Remove obsolete comment --- virtualizarr/tests/test_manifests/test_array.py | 1 - 1 file changed, 1 deletion(-) diff --git a/virtualizarr/tests/test_manifests/test_array.py b/virtualizarr/tests/test_manifests/test_array.py index 87e154f9..6735fc50 100644 --- a/virtualizarr/tests/test_manifests/test_array.py +++ b/virtualizarr/tests/test_manifests/test_array.py @@ -227,7 +227,6 @@ def test_concat(self, array_v3_metadata): assert concatenated.shape == (10, 2, 20) assert concatenated.dtype == np.dtype("int32") - # FAILING: TypeError: no implementation found for 'numpy.concatenate' on types that implement __array_function__: [, ] def test_concat_empty(self, array_v3_metadata): chunks = (5, 1, 10) shape = (5, 1, 20) From 3e2e8ecbe7fc53e53486e52b9cf2a7a26bdb287d Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Fri, 14 Feb 2025 16:30:48 -0800 Subject: [PATCH 67/78] Rename function numcodec_config_to_configurable --- virtualizarr/codecs.py | 2 +- virtualizarr/readers/hdf/hdf.py | 4 ++-- virtualizarr/translators/kerchunk.py | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/virtualizarr/codecs.py b/virtualizarr/codecs.py index 5b5ab024..3545da64 100644 --- a/virtualizarr/codecs.py +++ b/virtualizarr/codecs.py @@ -19,7 +19,7 @@ ] -def num_codec_config_to_configurable(num_codec: dict) -> dict: +def numcodec_config_to_configurable(num_codec: dict) -> dict: """ Convert a numcodecs codec into a zarr v3 configurable. """ diff --git a/virtualizarr/readers/hdf/hdf.py b/virtualizarr/readers/hdf/hdf.py index d4d55b67..9fc30c0a 100644 --- a/virtualizarr/readers/hdf/hdf.py +++ b/virtualizarr/readers/hdf/hdf.py @@ -14,7 +14,7 @@ import numpy as np import xarray as xr -from virtualizarr.codecs import num_codec_config_to_configurable +from virtualizarr.codecs import numcodec_config_to_configurable from virtualizarr.manifests import ( ChunkEntry, ChunkManifest, @@ -304,7 +304,7 @@ def _dataset_to_variable( if isinstance(fill_value, np.generic): fill_value = fill_value.item() codec_configs = [ - num_codec_config_to_configurable(codec.get_config()) for codec in codecs + numcodec_config_to_configurable(codec.get_config()) for codec in codecs ] metadata = create_v3_array_metadata( shape=dataset.shape, diff --git a/virtualizarr/translators/kerchunk.py b/virtualizarr/translators/kerchunk.py index ecd46914..91d6bb0c 100644 --- a/virtualizarr/translators/kerchunk.py +++ b/virtualizarr/translators/kerchunk.py @@ -9,7 +9,7 @@ from zarr.core.metadata.v2 import ArrayV2Metadata from virtualizarr.codecs import ( - num_codec_config_to_configurable, + numcodec_config_to_configurable, ) from virtualizarr.manifests import ChunkManifest, ManifestArray from virtualizarr.manifests.manifest import ChunkEntry, ChunkKey @@ -82,7 +82,7 @@ def from_kerchunk_refs(decoded_arr_refs_zarray) -> "ArrayV3Metadata": # Ensure compressor is a list before unpacking codec_configs = [*filters, *(compressor if compressor is not None else [])] numcodec_configs = [ - num_codec_config_to_configurable(config) for config in codec_configs + numcodec_config_to_configurable(config) for config in codec_configs ] return create_v3_array_metadata( chunk_shape=tuple(decoded_arr_refs_zarray["chunks"]), From 92293a6272fd8f33b6c2414b8745b8737376b37d Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Fri, 14 Feb 2025 16:35:10 -0800 Subject: [PATCH 68/78] Fix parameters in docstring for convert_to_codec_pipeline --- virtualizarr/codecs.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/virtualizarr/codecs.py b/virtualizarr/codecs.py index 3545da64..5c9ebde0 100644 --- a/virtualizarr/codecs.py +++ b/virtualizarr/codecs.py @@ -57,9 +57,8 @@ def convert_to_codec_pipeline( Parameters ---------- - dtype : Any - The data type. - codecs: list[ZarrCodec] | None + dtype : np.dtype + codecs: list[dict] | None Returns ------- From eccc215ad32bf6c452e0204cce08ac7c5499f3e7 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Fri, 14 Feb 2025 16:40:55 -0800 Subject: [PATCH 69/78] Revert change to pytest mark skipif for astropy --- virtualizarr/tests/test_backend.py | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/virtualizarr/tests/test_backend.py b/virtualizarr/tests/test_backend.py index e50223c5..7308caea 100644 --- a/virtualizarr/tests/test_backend.py +++ b/virtualizarr/tests/test_backend.py @@ -14,6 +14,7 @@ from virtualizarr.readers import HDF5VirtualBackend from virtualizarr.readers.hdf import HDFVirtualBackend from virtualizarr.tests import ( + has_astropy, parametrize_over_hdf_backends, requires_hdf5plugin, requires_imagecodecs, @@ -260,9 +261,14 @@ class TestReadFromURL: pytest.param( "fits", "https://fits.gsfc.nasa.gov/samples/WFPC2u5780205r_c0fx.fits", - marks=pytest.mark.xfail( - reason="Big endian not yet supported by zarr-python 3.0" - ), # https://github.com/zarr-developers/zarr-python/issues/2324 + marks=[ + pytest.mark.skipif( + not has_astropy, reason="package astropy is not available" + ), + pytest.mark.xfail( + reason="Big endian not yet supported by zarr-python 3.0" + ), # https://github.com/zarr-developers/zarr-python/issues/2324 + ], ), ( "jpg", From 8fa72b230be0432ee027b0fd025b2c5dc8dca0d8 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Fri, 14 Feb 2025 16:41:47 -0800 Subject: [PATCH 70/78] Remove commented arguments --- virtualizarr/tests/test_codecs.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/virtualizarr/tests/test_codecs.py b/virtualizarr/tests/test_codecs.py index 4fc575b7..d05ea863 100644 --- a/virtualizarr/tests/test_codecs.py +++ b/virtualizarr/tests/test_codecs.py @@ -25,8 +25,6 @@ def create_zarr_array(self, codecs=None, zarr_format=3): dtype="int32", store=None, zarr_format=zarr_format, - # compressor=compressor, - # filters=filters, codecs=codecs, ) From f747681f798ea0d6cca0ac380d5eb1d13a69e48f Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Fri, 14 Feb 2025 16:51:33 -0800 Subject: [PATCH 71/78] Add classes to test_codecs and make zarr_array a fixture --- conftest.py | 24 ++++++ virtualizarr/tests/test_codecs.py | 129 ++++++++++++++---------------- 2 files changed, 82 insertions(+), 71 deletions(-) diff --git a/conftest.py b/conftest.py index a06d8cf1..05af253c 100644 --- a/conftest.py +++ b/conftest.py @@ -373,3 +373,27 @@ def _virtual_dataset( ) return _virtual_dataset + + +# Zarr fixtures +@pytest.fixture +def zarr_array(): + def create_zarr_array(codecs=None, zarr_format=3): + """Create a test Zarr array with the specified codecs.""" + import zarr + + # Create a Zarr array in memory with the codecs + zarr_array = zarr.create( + shape=(1000, 1000), + chunks=(100, 100), + dtype="int32", + store=None, + zarr_format=zarr_format, + codecs=codecs, + ) + + # Populate the Zarr array with data + zarr_array[:] = np.arange(1000 * 1000).reshape(1000, 1000) + return zarr_array + + return create_zarr_array diff --git a/virtualizarr/tests/test_codecs.py b/virtualizarr/tests/test_codecs.py index d05ea863..529cf607 100644 --- a/virtualizarr/tests/test_codecs.py +++ b/virtualizarr/tests/test_codecs.py @@ -13,33 +13,17 @@ from virtualizarr.codecs import convert_to_codec_pipeline, get_codecs -class TestCodecs: - def create_zarr_array(self, codecs=None, zarr_format=3): - """Create a test Zarr array with the specified codecs.""" - import zarr - - # Create a Zarr array in memory with the codecs - zarr_array = zarr.create( - shape=(1000, 1000), - chunks=(100, 100), - dtype="int32", - store=None, - zarr_format=zarr_format, - codecs=codecs, - ) - - # Populate the Zarr array with data - zarr_array[:] = np.arange(1000 * 1000).reshape(1000, 1000) - return zarr_array +class TestGetCodecs: + """Test the get_codecs function.""" def test_manifest_array_zarr_v3_default(self, manifest_array): """Test get_codecs with ManifestArray using default v3 codec.""" - manifest_array = manifest_array(codecs=None) - actual_codecs = get_codecs(manifest_array) + test_manifest_array = manifest_array(codecs=None) + actual_codecs = get_codecs(test_manifest_array) expected_codecs = tuple([BytesCodec(endian="little")]) assert actual_codecs == expected_codecs - def test_manifest_array_zarr_v3_with_codecs(self, manifest_array): + def test_manifest_array_with_codecs(self, manifest_array): """Test get_codecs with ManifestArray using multiple v3 codecs.""" test_codecs = [DELTA_CODEC, ARRAYBYTES_CODEC, BLOSC_CODEC] manifest_array = manifest_array(codecs=test_codecs) @@ -51,16 +35,16 @@ def test_manifest_array_zarr_v3_with_codecs(self, manifest_array): ] ) - def test_zarr_v3_default(self): + def test_zarr_v3_default_codecs(self, zarr_array): """Test get_codecs with Zarr array using default v3 codec.""" - zarr_array = self.create_zarr_array() + zarr_array = zarr_array() actual_codecs = get_codecs(zarr_array) assert isinstance(actual_codecs[0], BytesCodec) - def test_zarr_v3_with_codecs(self): + def test_zarr_v3_with_codecs(self, zarr_array): """Test get_codecs with Zarr array using multiple v3 codecs.""" test_codecs = [DELTA_CODEC, ARRAYBYTES_CODEC, BLOSC_CODEC] - zarr_array = self.create_zarr_array(codecs=test_codecs) + zarr_array = zarr_array(codecs=test_codecs) actual_codecs = get_codecs(zarr_array) assert actual_codecs == tuple( [ @@ -69,9 +53,9 @@ def test_zarr_v3_with_codecs(self): ] ) - def test_zarr_v2_error(self): + def test_zarr_v2_error(self, zarr_array): """Test that using v2 format raises an error.""" - zarr_array = self.create_zarr_array(zarr_format=2) + zarr_array = zarr_array(zarr_format=2) with pytest.raises( ValueError, match="Only zarr v3 format arrays are supported. Please convert your array to v3 format.", @@ -79,53 +63,56 @@ def test_zarr_v2_error(self): get_codecs(zarr_array) -@pytest.mark.parametrize( - "input_codecs,expected_pipeline", - [ - # Case 1: No codecs - should result in just BytesCodec - ( - None, - BatchedCodecPipeline( - array_array_codecs=(), - array_bytes_codec=BytesCodec(endian="little"), - bytes_bytes_codecs=(), - batch_size=1, - ), - ), - # Case 2: Delta codec - should result in DeltaCodec + BytesCodec - ( - [DELTA_CODEC], - BatchedCodecPipeline( - array_array_codecs=( - get_codec_class("numcodecs.delta").from_dict(DELTA_CODEC), # type: ignore[arg-type] +class TestConvertToCodecPipeline: + """Test the convert_to_codec_pipeline function.""" + + @pytest.mark.parametrize( + "input_codecs,expected_pipeline", + [ + # Case 1: No codecs - should result in just BytesCodec + ( + None, + BatchedCodecPipeline( + array_array_codecs=(), + array_bytes_codec=BytesCodec(endian="little"), + bytes_bytes_codecs=(), + batch_size=1, ), - array_bytes_codec=BytesCodec(endian="little"), - bytes_bytes_codecs=(), - batch_size=1, ), - ), - # Case 3: Delta + Blosc + Zlib - should result in all codecs + BytesCodec - ( - [DELTA_CODEC, BLOSC_CODEC, ZLIB_CODEC], - BatchedCodecPipeline( - array_array_codecs=( - get_codec_class("numcodecs.delta").from_dict(DELTA_CODEC), # type: ignore[arg-type] + # Case 2: Delta codec - should result in DeltaCodec + BytesCodec + ( + [DELTA_CODEC], + BatchedCodecPipeline( + array_array_codecs=( + get_codec_class("numcodecs.delta").from_dict(DELTA_CODEC), # type: ignore[arg-type] + ), + array_bytes_codec=BytesCodec(endian="little"), + bytes_bytes_codecs=(), + batch_size=1, ), - array_bytes_codec=BytesCodec(endian="little"), - bytes_bytes_codecs=( - get_codec_class(key="blosc").from_dict(BLOSC_CODEC), # type: ignore[arg-type] - get_codec_class("numcodecs.zlib").from_dict(ZLIB_CODEC), # type: ignore[arg-type] + ), + # Case 3: Delta + Blosc + Zlib - should result in all codecs + BytesCodec + ( + [DELTA_CODEC, BLOSC_CODEC, ZLIB_CODEC], + BatchedCodecPipeline( + array_array_codecs=( + get_codec_class("numcodecs.delta").from_dict(DELTA_CODEC), # type: ignore[arg-type] + ), + array_bytes_codec=BytesCodec(endian="little"), + bytes_bytes_codecs=( + get_codec_class(key="blosc").from_dict(BLOSC_CODEC), # type: ignore[arg-type] + get_codec_class("numcodecs.zlib").from_dict(ZLIB_CODEC), # type: ignore[arg-type] + ), + batch_size=1, ), - batch_size=1, ), - ), - ], -) -def test_convert_to_codec_pipeline_scenarios(input_codecs, expected_pipeline): - """Test different scenarios for convert_to_codec_pipeline function.""" - dtype = np.dtype(" Date: Sun, 16 Feb 2025 08:01:06 -0800 Subject: [PATCH 72/78] Add tests for extract_codecs --- virtualizarr/tests/test_codecs.py | 71 ++++++++++++++++++- .../tests/test_writers/test_kerchunk.py | 10 ++- 2 files changed, 74 insertions(+), 7 deletions(-) diff --git a/virtualizarr/tests/test_codecs.py b/virtualizarr/tests/test_codecs.py index 529cf607..f4ba4c5b 100644 --- a/virtualizarr/tests/test_codecs.py +++ b/virtualizarr/tests/test_codecs.py @@ -10,7 +10,7 @@ DELTA_CODEC, ZLIB_CODEC, ) -from virtualizarr.codecs import convert_to_codec_pipeline, get_codecs +from virtualizarr.codecs import convert_to_codec_pipeline, extract_codecs, get_codecs class TestGetCodecs: @@ -116,3 +116,72 @@ def test_convert_to_codec_pipeline_scenarios(self, input_codecs, expected_pipeli result = convert_to_codec_pipeline(dtype=dtype, codecs=input_codecs) assert result == expected_pipeline + + +class TestExtractCodecs: + """Test the extract_codecs function.""" + + def test_extract_codecs_with_all_types(self): + """Test extract_codecs with all types of codecs.""" + arrayarray_codec = get_codec_class("numcodecs.delta").from_dict(DELTA_CODEC) + arraybytes_codec = BytesCodec(endian="little") + bytesbytes_codec = get_codec_class("numcodecs.zlib").from_dict(ZLIB_CODEC) + + codecs = (arrayarray_codec, arraybytes_codec, bytesbytes_codec) + result = extract_codecs(codecs) + + assert result == ( + (arrayarray_codec,), + arraybytes_codec, + (bytesbytes_codec,), + ) + + def test_extract_codecs_with_only_arrayarray(self): + """Test extract_codecs with only ArrayArrayCodec.""" + arrayarray_codec = get_codec_class("numcodecs.delta").from_dict(DELTA_CODEC) + + codecs = (arrayarray_codec,) + result = extract_codecs(codecs) + + assert result == ( + (arrayarray_codec,), + None, + (), + ) + + def test_extract_codecs_with_only_arraybytes(self): + """Test extract_codecs with only ArrayBytesCodec.""" + arraybytes_codec = BytesCodec(endian="little") + + codecs = (arraybytes_codec,) + result = extract_codecs(codecs) + + assert result == ( + (), + arraybytes_codec, + (), + ) + + def test_extract_codecs_with_only_bytesbytes(self): + """Test extract_codecs with only BytesBytesCodec.""" + bytesbytes_codec = get_codec_class("numcodecs.zlib").from_dict(ZLIB_CODEC) + + codecs = (bytesbytes_codec,) + result = extract_codecs(codecs) + + assert result == ( + (), + None, + (bytesbytes_codec,), + ) + + def test_extract_codecs_with_empty_list(self): + """Test extract_codecs with an empty list.""" + codecs = () + result = extract_codecs(codecs) + + assert result == ( + (), + None, + (), + ) diff --git a/virtualizarr/tests/test_writers/test_kerchunk.py b/virtualizarr/tests/test_writers/test_kerchunk.py index 20620a5d..8af1a76f 100644 --- a/virtualizarr/tests/test_writers/test_kerchunk.py +++ b/virtualizarr/tests/test_writers/test_kerchunk.py @@ -3,6 +3,7 @@ from xarray import Dataset from zarr.core.metadata.v2 import ArrayV2Metadata +from conftest import ARRAYBYTES_CODEC, BLOSC_CODEC, DELTA_CODEC from virtualizarr.manifests import ChunkManifest, ManifestArray from virtualizarr.tests import requires_fastparquet, requires_kerchunk from virtualizarr.writers.kerchunk import convert_v3_to_v2_metadata @@ -151,12 +152,9 @@ def testconvert_v3_to_v2_metadata(array_v3_metadata): shape = (5, 20) chunks = (5, 10) codecs = [ - {"configuration": {"endian": "little"}, "name": "bytes"}, - {"name": "numcodecs.delta", "configuration": {"dtype": " Date: Sun, 16 Feb 2025 09:06:30 -0800 Subject: [PATCH 73/78] Add test for get_codec_config --- virtualizarr/codecs.py | 7 +-- virtualizarr/tests/test_codecs.py | 46 ++++++++++++++++++- .../tests/test_writers/test_kerchunk.py | 9 ++-- 3 files changed, 53 insertions(+), 9 deletions(-) diff --git a/virtualizarr/codecs.py b/virtualizarr/codecs.py index 5c9ebde0..13c29a7b 100644 --- a/virtualizarr/codecs.py +++ b/virtualizarr/codecs.py @@ -98,13 +98,10 @@ def get_codec_config(codec: ZarrCodec) -> dict[str, Any]: return codec.codec_config elif hasattr(codec, "get_config"): return codec.get_config() - elif hasattr(codec, "codec_name"): + elif hasattr(codec, "to_dict"): # If we can't get config, try to get the name and configuration directly # This assumes the codec follows the v3 spec format - return { - "id": codec.codec_name.replace("numcodecs.", ""), - **getattr(codec, "configuration", {}), - } + return codec.to_dict() else: raise ValueError(f"Unable to parse codec configuration: {codec}") diff --git a/virtualizarr/tests/test_codecs.py b/virtualizarr/tests/test_codecs.py index f4ba4c5b..ac29282e 100644 --- a/virtualizarr/tests/test_codecs.py +++ b/virtualizarr/tests/test_codecs.py @@ -10,7 +10,12 @@ DELTA_CODEC, ZLIB_CODEC, ) -from virtualizarr.codecs import convert_to_codec_pipeline, extract_codecs, get_codecs +from virtualizarr.codecs import ( + convert_to_codec_pipeline, + extract_codecs, + get_codec_config, + get_codecs, +) class TestGetCodecs: @@ -185,3 +190,42 @@ def test_extract_codecs_with_empty_list(self): None, (), ) + + +class TestGetCodecConfig: + """Test the get_codec_config function.""" + + def test_codec_with_codec_config(self): + """Test get_codec_config with a codec having codec_config attribute.""" + codec = get_codec_class("numcodecs.delta").from_dict(DELTA_CODEC) + expected_config = codec.codec_config + actual_config = get_codec_config(codec) + assert actual_config == expected_config + + def test_codec_with_to_dict(self): + """Test get_codec_config with a codec having get_config method.""" + from zarr.codecs import BloscCodec + + codec = BloscCodec(typesize=4, clevel=5, shuffle="shuffle", cname="lz4") + expected_config = codec.to_dict() + actual_config = get_codec_config(codec) + assert actual_config == expected_config + + def test_codec_with_get_config(self): + """Test get_codec_config with a codec having to_dict method.""" + from numcodecs import FixedScaleOffset + + codec = FixedScaleOffset(offset=0, scale=1, dtype=" Date: Sun, 16 Feb 2025 16:05:31 -0800 Subject: [PATCH 74/78] Remove obsolete comment --- virtualizarr/codecs.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/virtualizarr/codecs.py b/virtualizarr/codecs.py index 13c29a7b..fa8d01d9 100644 --- a/virtualizarr/codecs.py +++ b/virtualizarr/codecs.py @@ -99,8 +99,6 @@ def get_codec_config(codec: ZarrCodec) -> dict[str, Any]: elif hasattr(codec, "get_config"): return codec.get_config() elif hasattr(codec, "to_dict"): - # If we can't get config, try to get the name and configuration directly - # This assumes the codec follows the v3 spec format return codec.to_dict() else: raise ValueError(f"Unable to parse codec configuration: {codec}") From 8af37a030da22bf18fe45b8bdecd2c2d7fbda599 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Sun, 16 Feb 2025 16:13:55 -0800 Subject: [PATCH 75/78] Add test for copy_and_replace_metadata --- virtualizarr/tests/test_utils.py | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/virtualizarr/tests/test_utils.py b/virtualizarr/tests/test_utils.py index 55eaf0fc..7580c098 100644 --- a/virtualizarr/tests/test_utils.py +++ b/virtualizarr/tests/test_utils.py @@ -4,9 +4,11 @@ import fsspec import fsspec.implementations.local import fsspec.implementations.memory +import numpy as np import pytest import xarray as xr +from virtualizarr.manifests.utils import copy_and_replace_metadata from virtualizarr.tests import requires_scipy from virtualizarr.utils import _FsspecFSFromFilepath @@ -37,3 +39,25 @@ def test_fsspec_openfile_memory(dataset: xr.Dataset): result = _FsspecFSFromFilepath(filepath="memory://dataset.nc").open_file() with result: assert isinstance(result, fsspec.implementations.memory.MemoryFile) + + +def test_copy_and_replace_metadata(array_v3_metadata): + old_metadata = array_v3_metadata( + shape=(10, 10), + data_type=np.dtype("float32"), + chunks=(5, 5), + fill_value=0, + ) + + new_shape = (20, 20) + new_chunks = (10, 10) + + # Test updating both shape and chunk shape + updated_metadata = copy_and_replace_metadata( + old_metadata, new_shape=new_shape, new_chunks=new_chunks + ) + assert updated_metadata.shape == tuple(new_shape) + assert updated_metadata.chunks == tuple(new_chunks) + # Test other values are still the same + assert updated_metadata.data_type == old_metadata.data_type + assert updated_metadata.fill_value == old_metadata.fill_value From d4946ab271c00c1b6464fac3522ebf41d21852c1 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Sun, 16 Feb 2025 17:17:58 -0800 Subject: [PATCH 76/78] Add release notes --- docs/releases.rst | 13 +++++++++++++ virtualizarr/writers/kerchunk.py | 6 +++--- 2 files changed, 16 insertions(+), 3 deletions(-) diff --git a/docs/releases.rst b/docs/releases.rst index 48657642..1b5ab162 100644 --- a/docs/releases.rst +++ b/docs/releases.rst @@ -27,6 +27,19 @@ Documentation Internal Changes ~~~~~~~~~~~~~~~~ +(:pull:`429`) By `Aimee Barciauskas `_. +- `ManifestArrays` now internally use `zarr.core.metadata.v3.ArrayV3Metadata`. This replaces the `ZArray` class that was previously used to store metadata about manifest arrays. Notable internal changes: + - Make zarr-python a required dependency with a minimum version `>=3.0.2`. + - Specify a minimum numcodecs version of `>=0.15.1`. + - When creating a `ManifestArray`, the `metadata` property should be an [`zarr.core.metadata.v3.ArrayV3Metadata`](https://github.com/zarr-developers/zarr-python/blob/v3.0.2/src/zarr/core/metadata/v3.py) object. There is a helper function `create_v3_array_metadata` which should be used, as it has some useful defaults and includes `convert_to_codec_pipeline` (see next bullet). + - The function `convert_to_codec_pipeline` ensures the codec pipeline passed to `ArrayV3Metadata` has valid codecs in the expected order (`ArrayArrayCodec`s, `ArrayBytesCodec`, `BytesBytesCodec`s) and includes the required `ArrayBytesCodec` using the default for the data type. + - Note: `convert_to_codec_pipeline` uses the zarr-python function `get_codec_class` to convert codec configurations (i.e. `dict`s with a name and configuration key, see `parse_named_configuration`[LINK]) to valid Zarr V3 codec classes. + - Reader changes are minimal. + - Writer changes: + - Kerchunk uses Zarr version format 2 so we convert `ArrayV3Metadata` to `ArrayV2Metadata` using the `convert_v3_to_v2_metadata` function. This means the `to_kerchunk_json` function is now a bit more complex because we're converting `ArrayV2Metadata` filters and compressor to serializable objects. + - zarr-python 3.0 does not yet support the big endian data type. This means that FITS and NetCDF-3 are not currently supported ([zarr-python #2324](https://github.com/zarr-developers/zarr-python/issues/2324)). + - zarr-python 3.0 does not yet support datetime and timedelta data types ([zarr-python #2616](https://github.com/zarr-developers/zarr-python/issues/2616)). + .. _v1.3.0: v1.3.0 (3rd Feb 2025) diff --git a/virtualizarr/writers/kerchunk.py b/virtualizarr/writers/kerchunk.py index 520c7ccb..726ef0b8 100644 --- a/virtualizarr/writers/kerchunk.py +++ b/virtualizarr/writers/kerchunk.py @@ -163,6 +163,8 @@ def variable_to_kerchunk_arr_refs(var: Variable, var_name: str) -> KerchunkArrRe } array_v2_metadata = convert_v3_to_v2_metadata(marr.metadata) else: + from zarr.core.metadata.v2 import ArrayV2Metadata + try: np_arr = var.to_numpy() except AttributeError as e: @@ -191,14 +193,12 @@ def variable_to_kerchunk_arr_refs(var: Variable, var_name: str) -> KerchunkArrRe # TODO will this fail for a scalar? arr_refs = {join(0 for _ in np_arr.shape): inlined_data} - from zarr.core.metadata.v2 import ArrayV2Metadata - array_v2_metadata = ArrayV2Metadata( chunks=np_arr.shape, shape=np_arr.shape, dtype=np_arr.dtype, order="C", - fill_value=var.encoding.get("fill_value", None), + fill_value=None, ) zarray_dict = to_kerchunk_json(array_v2_metadata) From 2c43c615359b0964484c4b48d0ea83aa33f85e82 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Sun, 16 Feb 2025 17:39:15 -0800 Subject: [PATCH 77/78] Attempt to fix rst links --- docs/releases.rst | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/docs/releases.rst b/docs/releases.rst index 1b5ab162..d9e31251 100644 --- a/docs/releases.rst +++ b/docs/releases.rst @@ -27,18 +27,17 @@ Documentation Internal Changes ~~~~~~~~~~~~~~~~ -(:pull:`429`) By `Aimee Barciauskas `_. -- `ManifestArrays` now internally use `zarr.core.metadata.v3.ArrayV3Metadata`. This replaces the `ZArray` class that was previously used to store metadata about manifest arrays. Notable internal changes: +- `ManifestArrays` now internally use `zarr.core.metadata.v3.ArrayV3Metadata _`. This replaces the `ZArray` class that was previously used to store metadata about manifest arrays. (:pull:`429`) By `Aimee Barciauskas `_. Notable internal changes: - Make zarr-python a required dependency with a minimum version `>=3.0.2`. - Specify a minimum numcodecs version of `>=0.15.1`. - - When creating a `ManifestArray`, the `metadata` property should be an [`zarr.core.metadata.v3.ArrayV3Metadata`](https://github.com/zarr-developers/zarr-python/blob/v3.0.2/src/zarr/core/metadata/v3.py) object. There is a helper function `create_v3_array_metadata` which should be used, as it has some useful defaults and includes `convert_to_codec_pipeline` (see next bullet). + - When creating a `ManifestArray`, the `metadata` property should be an `zarr.core.metadata.v3.ArrayV3Metadata` object. There is a helper function `create_v3_array_metadata` which should be used, as it has some useful defaults and includes `convert_to_codec_pipeline` (see next bullet). - The function `convert_to_codec_pipeline` ensures the codec pipeline passed to `ArrayV3Metadata` has valid codecs in the expected order (`ArrayArrayCodec`s, `ArrayBytesCodec`, `BytesBytesCodec`s) and includes the required `ArrayBytesCodec` using the default for the data type. - - Note: `convert_to_codec_pipeline` uses the zarr-python function `get_codec_class` to convert codec configurations (i.e. `dict`s with a name and configuration key, see `parse_named_configuration`[LINK]) to valid Zarr V3 codec classes. + - Note: `convert_to_codec_pipeline` uses the zarr-python function `get_codec_class` to convert codec configurations (i.e. `dict`s with a name and configuration key, see `parse_named_configuration `_) to valid Zarr V3 codec classes. - Reader changes are minimal. - Writer changes: - Kerchunk uses Zarr version format 2 so we convert `ArrayV3Metadata` to `ArrayV2Metadata` using the `convert_v3_to_v2_metadata` function. This means the `to_kerchunk_json` function is now a bit more complex because we're converting `ArrayV2Metadata` filters and compressor to serializable objects. - - zarr-python 3.0 does not yet support the big endian data type. This means that FITS and NetCDF-3 are not currently supported ([zarr-python #2324](https://github.com/zarr-developers/zarr-python/issues/2324)). - - zarr-python 3.0 does not yet support datetime and timedelta data types ([zarr-python #2616](https://github.com/zarr-developers/zarr-python/issues/2616)). + - zarr-python 3.0 does not yet support the big endian data type. This means that FITS and NetCDF-3 are not currently supported (`zarr-python issue #2324 `_). + - zarr-python 3.0 does not yet support datetime and timedelta data types (`zarr-python issue #2616 `_). .. _v1.3.0: From 05cd57a5ead778d28ee661a0cf6c96ed6617d567 Mon Sep 17 00:00:00 2001 From: Aimee Barciauskas Date: Tue, 18 Feb 2025 18:55:39 -0500 Subject: [PATCH 78/78] Move convert_v3_to_v2_metadata to utils --- .../tests/test_writers/test_kerchunk.py | 2 +- virtualizarr/utils.py | 58 ++++++++++++++++++- virtualizarr/writers/kerchunk.py | 57 +----------------- 3 files changed, 60 insertions(+), 57 deletions(-) diff --git a/virtualizarr/tests/test_writers/test_kerchunk.py b/virtualizarr/tests/test_writers/test_kerchunk.py index 25153762..fbcb921e 100644 --- a/virtualizarr/tests/test_writers/test_kerchunk.py +++ b/virtualizarr/tests/test_writers/test_kerchunk.py @@ -6,7 +6,7 @@ from conftest import ARRAYBYTES_CODEC from virtualizarr.manifests import ChunkManifest, ManifestArray from virtualizarr.tests import requires_fastparquet, requires_kerchunk -from virtualizarr.writers.kerchunk import convert_v3_to_v2_metadata +from virtualizarr.utils import convert_v3_to_v2_metadata @requires_kerchunk diff --git a/virtualizarr/utils.py b/virtualizarr/utils.py index 4f6c4773..d36f0826 100644 --- a/virtualizarr/utils.py +++ b/virtualizarr/utils.py @@ -2,7 +2,12 @@ import importlib import io -from typing import TYPE_CHECKING, Iterable, Optional, Union +from typing import TYPE_CHECKING, Any, Iterable, Optional, Union + +from zarr.abc.codec import ArrayArrayCodec, BytesBytesCodec +from zarr.core.metadata import ArrayV2Metadata, ArrayV3Metadata + +from virtualizarr.codecs import extract_codecs, get_codec_config if TYPE_CHECKING: import fsspec.core @@ -116,3 +121,54 @@ def determine_chunk_grid_shape( ) -> tuple[int, ...]: """Calculate the shape of the chunk grid based on array shape and chunk size.""" return tuple(ceildiv(length, chunksize) for length, chunksize in zip(shape, chunks)) + + +def convert_v3_to_v2_metadata( + v3_metadata: ArrayV3Metadata, fill_value: Any = None +) -> ArrayV2Metadata: + """ + Convert ArrayV3Metadata to ArrayV2Metadata. + + Parameters + ---------- + v3_metadata : ArrayV3Metadata + The metadata object in v3 format. + fill_value : Any, optional + Override the fill value from v3 metadata. + + Returns + ------- + ArrayV2Metadata + The metadata object in v2 format. + """ + import warnings + + array_filters: tuple[ArrayArrayCodec, ...] + bytes_compressors: tuple[BytesBytesCodec, ...] + array_filters, _, bytes_compressors = extract_codecs(v3_metadata.codecs) + + # Handle compressor configuration + compressor_config: dict[str, Any] | None = None + if bytes_compressors: + if len(bytes_compressors) > 1: + warnings.warn( + "Multiple compressors found in v3 metadata. Using the first compressor, " + "others will be ignored. This may affect data compatibility.", + UserWarning, + ) + compressor_config = get_codec_config(bytes_compressors[0]) + + # Handle filter configurations + filter_configs = [get_codec_config(filter_) for filter_ in array_filters] + v2_metadata = ArrayV2Metadata( + shape=v3_metadata.shape, + dtype=v3_metadata.data_type.to_numpy(), + chunks=v3_metadata.chunks, + fill_value=fill_value or v3_metadata.fill_value, + compressor=compressor_config, + filters=filter_configs, + order="C", + attributes=v3_metadata.attributes, + dimension_separator=".", # Assuming '.' as default dimension separator + ) + return v2_metadata diff --git a/virtualizarr/writers/kerchunk.py b/virtualizarr/writers/kerchunk.py index 726ef0b8..c47bb75a 100644 --- a/virtualizarr/writers/kerchunk.py +++ b/virtualizarr/writers/kerchunk.py @@ -1,17 +1,15 @@ import base64 import json -from typing import Any, cast +from typing import cast import numpy as np from xarray import Dataset from xarray.coding.times import CFDatetimeCoder from xarray.core.variable import Variable -from zarr.abc.codec import ArrayArrayCodec, BytesBytesCodec -from zarr.core.metadata import ArrayV2Metadata, ArrayV3Metadata -from virtualizarr.codecs import extract_codecs, get_codec_config from virtualizarr.manifests.manifest import join from virtualizarr.types.kerchunk import KerchunkArrRefs, KerchunkStoreRefs +from virtualizarr.utils import convert_v3_to_v2_metadata class NumpyEncoder(json.JSONEncoder): @@ -90,57 +88,6 @@ def remove_file_uri_prefix(path: str): return path -def convert_v3_to_v2_metadata( - v3_metadata: ArrayV3Metadata, fill_value: Any = None -) -> ArrayV2Metadata: - """ - Convert ArrayV3Metadata to ArrayV2Metadata. - - Parameters - ---------- - v3_metadata : ArrayV3Metadata - The metadata object in v3 format. - fill_value : Any, optional - Override the fill value from v3 metadata. - - Returns - ------- - ArrayV2Metadata - The metadata object in v2 format. - """ - import warnings - - array_filters: tuple[ArrayArrayCodec, ...] - bytes_compressors: tuple[BytesBytesCodec, ...] - array_filters, _, bytes_compressors = extract_codecs(v3_metadata.codecs) - - # Handle compressor configuration - compressor_config: dict[str, Any] | None = None - if bytes_compressors: - if len(bytes_compressors) > 1: - warnings.warn( - "Multiple compressors found in v3 metadata. Using the first compressor, " - "others will be ignored. This may affect data compatibility.", - UserWarning, - ) - compressor_config = get_codec_config(bytes_compressors[0]) - - # Handle filter configurations - filter_configs = [get_codec_config(filter_) for filter_ in array_filters] - v2_metadata = ArrayV2Metadata( - shape=v3_metadata.shape, - dtype=v3_metadata.data_type.to_numpy(), - chunks=v3_metadata.chunks, - fill_value=fill_value or v3_metadata.fill_value, - compressor=compressor_config, - filters=filter_configs, - order="C", - attributes=v3_metadata.attributes, - dimension_separator=".", # Assuming '.' as default dimension separator - ) - return v2_metadata - - def variable_to_kerchunk_arr_refs(var: Variable, var_name: str) -> KerchunkArrRefs: """ Create a dictionary containing kerchunk-style array references from a single xarray.Variable (which wraps either a ManifestArray or a numpy array).