From 7d5d8fb3012b3d5abe35f7906d68f987a117214c Mon Sep 17 00:00:00 2001 From: Davis Vann Bennett Date: Tue, 22 Oct 2024 11:12:47 +0200 Subject: [PATCH 01/16] add write_empty_chunks to config.array namespace --- src/zarr/core/config.py | 2 +- tests/test_config.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/zarr/core/config.py b/src/zarr/core/config.py index ec2c8c47a3..fabba70385 100644 --- a/src/zarr/core/config.py +++ b/src/zarr/core/config.py @@ -42,7 +42,7 @@ def reset(self) -> None: defaults=[ { "default_zarr_version": 3, - "array": {"order": "C"}, + "array": {"order": "C", "write_empty_chunks": False}, "async": {"concurrency": 10, "timeout": None}, "threading": {"max_workers": None}, "json_indent": 2, diff --git a/tests/test_config.py b/tests/test_config.py index c4cf794c5f..b20b433272 100644 --- a/tests/test_config.py +++ b/tests/test_config.py @@ -40,7 +40,7 @@ def test_config_defaults_set() -> None: assert config.defaults == [ { "default_zarr_version": 3, - "array": {"order": "C"}, + "array": {"order": "C", "write_empty_chunks": False}, "async": {"concurrency": 10, "timeout": None}, "threading": {"max_workers": None}, "json_indent": 2, From 25fc9caf55159d7b60046c126ef57e3d84e2111d Mon Sep 17 00:00:00 2001 From: Davis Vann Bennett Date: Tue, 22 Oct 2024 11:16:38 +0200 Subject: [PATCH 02/16] use write_empty_chunks from config in write_batch --- src/zarr/codecs/pipeline.py | 20 +++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/src/zarr/codecs/pipeline.py b/src/zarr/codecs/pipeline.py index 038a2eeac2..439558edeb 100644 --- a/src/zarr/codecs/pipeline.py +++ b/src/zarr/codecs/pipeline.py @@ -331,6 +331,7 @@ async def write_batch( value: NDBuffer, drop_axes: tuple[int, ...] = (), ) -> None: + write_empty_chunks = config.get("array.write_empty_chunks") == True # noqa: E712 if self.supports_partial_encode: await self.encode_partial_batch( [ @@ -377,15 +378,16 @@ async def _read_key( chunk_array_batch, batch_info, strict=False ) ] - - chunk_array_batch = [ - None - if chunk_array is None or chunk_array.all_equal(chunk_spec.fill_value) - else chunk_array - for chunk_array, (_, chunk_spec, _, _) in zip( - chunk_array_batch, batch_info, strict=False - ) - ] + for chunk_array, (_, chunk_spec, _, _) in zip( + chunk_array_batch, batch_info, strict=False + ): + if chunk_array is None: + chunk_array_batch.append(None) + else: + if not write_empty_chunks and chunk_array.all_equal(chunk_spec.fill_value): + chunk_array_batch.append(None) + else: + chunk_array_batch.append(chunk_array) chunk_bytes_batch = await self.encode_batch( [ From 3db2e224588238eab91ca8372b534d80b0b9ad0a Mon Sep 17 00:00:00 2001 From: Davis Vann Bennett Date: Tue, 22 Oct 2024 11:39:46 +0200 Subject: [PATCH 03/16] implement config-sensitive write_empty_chunks in write_batch, and add a test --- src/zarr/codecs/pipeline.py | 11 ++++++----- tests/test_array.py | 20 ++++++++++++++++++++ 2 files changed, 26 insertions(+), 5 deletions(-) diff --git a/src/zarr/codecs/pipeline.py b/src/zarr/codecs/pipeline.py index 439558edeb..485fb0a595 100644 --- a/src/zarr/codecs/pipeline.py +++ b/src/zarr/codecs/pipeline.py @@ -361,7 +361,7 @@ async def _read_key( _read_key, config.get("async.concurrency"), ) - chunk_array_batch = await self.decode_batch( + chunk_array_decoded = await self.decode_batch( [ (chunk_bytes, chunk_spec) for chunk_bytes, (_, chunk_spec, _, _) in zip( @@ -370,19 +370,20 @@ async def _read_key( ], ) - chunk_array_batch = [ + chunk_array_merged = [ self._merge_chunk_array( chunk_array, value, out_selection, chunk_spec, chunk_selection, drop_axes ) for chunk_array, (_, chunk_spec, chunk_selection, out_selection) in zip( - chunk_array_batch, batch_info, strict=False + chunk_array_decoded, batch_info, strict=False ) ] + chunk_array_batch: list[NDBuffer | None] = [] for chunk_array, (_, chunk_spec, _, _) in zip( - chunk_array_batch, batch_info, strict=False + chunk_array_merged, batch_info, strict=False ): if chunk_array is None: - chunk_array_batch.append(None) + chunk_array_batch.append(None) # type: ignore[unreachable] else: if not write_empty_chunks and chunk_array.all_equal(chunk_spec.fill_value): chunk_array_batch.append(None) diff --git a/tests/test_array.py b/tests/test_array.py index f182cb1a14..b4cad721cd 100644 --- a/tests/test_array.py +++ b/tests/test_array.py @@ -11,6 +11,7 @@ from zarr.core.array import chunks_initialized from zarr.core.buffer.cpu import NDBuffer from zarr.core.common import JSON, MemoryOrder, ZarrFormat +from zarr.core.config import config from zarr.core.group import AsyncGroup from zarr.core.indexing import ceildiv from zarr.core.sync import sync @@ -436,3 +437,22 @@ def test_array_create_order( assert vals.flags.f_contiguous else: raise AssertionError + + +@pytest.mark.parametrize("store", ["memory"], indirect=True) +@pytest.mark.parametrize("write_empty_chunks", [True, False]) +@pytest.mark.parametrize("fill_value", [0, 5]) +def test_write_empty_chunks( + zarr_format: ZarrFormat, store: MemoryStore, write_empty_chunks: bool, fill_value: int +) -> None: + arr = Array.create( + store=store, shape=(1,), zarr_format=zarr_format, dtype="i4", fill_value=fill_value + ) + + with config.set({"array.write_empty_chunks": write_empty_chunks}): + arr[:] = fill_value + + if not write_empty_chunks: + assert arr.nchunks_initialized == 0 + else: + assert arr.nchunks_initialized == arr.nchunks From 3cfface191541678ddc8ed3a798e6514f6b7653f Mon Sep 17 00:00:00 2001 From: Davis Vann Bennett Date: Tue, 22 Oct 2024 12:06:28 +0200 Subject: [PATCH 04/16] add literacy to test --- tests/test_array.py | 21 +++++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/tests/test_array.py b/tests/test_array.py index b4cad721cd..c740c2d4e2 100644 --- a/tests/test_array.py +++ b/tests/test_array.py @@ -445,13 +445,30 @@ def test_array_create_order( def test_write_empty_chunks( zarr_format: ZarrFormat, store: MemoryStore, write_empty_chunks: bool, fill_value: int ) -> None: + """ + Check that the write_empty_chunks value of the config is applied correctly. We expect that + when write_empty_chunks is True, writing chunks equal to the fill value will result in + those chunks appearing in the store. + + When write_empty_chunks is False, writing chunks that are equal to the fill value will result in + those chunks not being present in the store. In particular, they should be deleted if they were + already present. + """ arr = Array.create( - store=store, shape=(1,), zarr_format=zarr_format, dtype="i4", fill_value=fill_value + store=store, + shape=(2,), + zarr_format=zarr_format, + dtype="i4", + fill_value=fill_value, + chunk_shape=(1,), ) + # initialize the store with some non-fill value chunks + arr[:] = fill_value + 1 + assert arr.nchunks_initialized == arr.nchunks + with config.set({"array.write_empty_chunks": write_empty_chunks}): arr[:] = fill_value - if not write_empty_chunks: assert arr.nchunks_initialized == 0 else: From d2c7838ad33bbc221b1e031ea63e36bcfbc8f5cb Mon Sep 17 00:00:00 2001 From: Davis Vann Bennett Date: Tue, 22 Oct 2024 18:36:54 +0200 Subject: [PATCH 05/16] add warnings when write_empty_chunks is used as a kwarg --- src/zarr/api/asynchronous.py | 26 ++++++++++++-------------- src/zarr/core/common.py | 10 ++++++++++ tests/test_api.py | 15 +++++++++++++++ 3 files changed, 37 insertions(+), 14 deletions(-) diff --git a/src/zarr/api/asynchronous.py b/src/zarr/api/asynchronous.py index 680433565e..f80e505f6e 100644 --- a/src/zarr/api/asynchronous.py +++ b/src/zarr/api/asynchronous.py @@ -16,6 +16,7 @@ ChunkCoords, MemoryOrder, ZarrFormat, + _warn_write_empty_chunks_kwarg, ) from zarr.core.config import config from zarr.core.group import AsyncGroup, ConsolidatedMetadata, GroupMetadata @@ -724,7 +725,6 @@ async def create( read_only: bool | None = None, object_codec: Codec | None = None, # TODO: type has changed dimension_separator: Literal[".", "/"] | None = None, - write_empty_chunks: bool = False, # TODO: default has changed zarr_version: ZarrFormat | None = None, # deprecated zarr_format: ZarrFormat | None = None, meta_array: Any | None = None, # TODO: need type @@ -794,17 +794,6 @@ async def create( .. versionadded:: 2.8 - write_empty_chunks : bool, optional - If True (default), all chunks will be stored regardless of their - contents. If False, each chunk is compared to the array's fill value - prior to storing. If a chunk is uniformly equal to the fill value, then - that chunk is not be stored, and the store entry for that chunk's key - is deleted. This setting enables sparser storage, as only chunks with - non-fill-value data are stored, at the expense of overhead associated - with checking the data of each chunk. - - .. versionadded:: 2.11 - zarr_format : {2, 3, None}, optional The zarr format to use when saving. meta_array : array-like, optional @@ -856,8 +845,12 @@ async def create( RuntimeWarning, stacklevel=2, ) - if write_empty_chunks: - warnings.warn("write_empty_chunks is not yet implemented", RuntimeWarning, stacklevel=2) + + if "write_empty_chunks" in kwargs: + # warn users if the write_empty_chunks kwarg was used + write_empty_chunks = kwargs.pop("write_empty_chunks") + _warn_write_empty_chunks_kwarg(write_empty_chunks) + if meta_array is not None: warnings.warn("meta_array is not yet implemented", RuntimeWarning, stacklevel=2) @@ -1058,6 +1051,11 @@ async def open_array( zarr_format = _handle_zarr_version_or_format(zarr_version=zarr_version, zarr_format=zarr_format) + if "write_empty_chunks" in kwargs: + # warn users if the write_empty_chunks kwarg was used + write_empty_chunks = kwargs.pop("write_empty_chunks") + _warn_write_empty_chunks_kwarg(write_empty_chunks) + try: return await AsyncArray.open(store_path, zarr_format=zarr_format) except FileNotFoundError: diff --git a/src/zarr/core/common.py b/src/zarr/core/common.py index f3f49b0d5d..54401e5d22 100644 --- a/src/zarr/core/common.py +++ b/src/zarr/core/common.py @@ -3,6 +3,7 @@ import asyncio import functools import operator +import warnings from collections.abc import Iterable, Mapping from enum import Enum from itertools import starmap @@ -166,3 +167,12 @@ def parse_dtype(dtype: Any, zarr_format: ZarrFormat) -> np.dtype[Any]: else: return _STRING_DTYPE return np.dtype(dtype) + + +def _warn_write_empty_chunks_kwarg(write_empty_chunks: bool) -> None: + msg = ( + f"The `write_empty_chunks` keyword argument was provided to this function with a value of {write_empty_chunks}." + "This keyword argument has no effect. To control whether empty chunks are written to" + " storage, change the 'array.write_empty_chunks' configuration variable." + ) + warnings.warn(msg, RuntimeWarning, stacklevel=2) diff --git a/tests/test_api.py b/tests/test_api.py index 5b62e3a2fa..8525c982d0 100644 --- a/tests/test_api.py +++ b/tests/test_api.py @@ -48,6 +48,21 @@ def test_create_array(memory_store: Store) -> None: assert z.chunks == (40,) +@pytest.mark.parametrize("write_empty_chunks", [True, False]) +def test_write_empty_chunks_warns(write_empty_chunks: bool) -> None: + """ + Test that using the `write_empty_chunks` kwarg on array access will raise a warning. + """ + match = f"The `write_empty_chunks` keyword argument was provided to this function with a value of {write_empty_chunks}." + with pytest.warns(RuntimeWarning, match=match): + _ = zarr.array( + data=np.arange(10), shape=(10,), dtype="uint8", write_empty_chunks=write_empty_chunks + ) + + with pytest.warns(RuntimeWarning, match=match): + _ = zarr.create(shape=(10,), dtype="uint8", write_empty_chunks=write_empty_chunks) + + @pytest.mark.parametrize("path", ["foo", "/", "/foo", "///foo/bar"]) @pytest.mark.parametrize("node_type", ["array", "group"]) def test_open_normalized_path( From b7d117a37e82b03c16cc6d95e7bded55d3473794 Mon Sep 17 00:00:00 2001 From: Davis Vann Bennett Date: Tue, 29 Oct 2024 20:17:20 +0100 Subject: [PATCH 06/16] init --- src/zarr/core/array.py | 24 +++++++++++++++++++----- 1 file changed, 19 insertions(+), 5 deletions(-) diff --git a/src/zarr/core/array.py b/src/zarr/core/array.py index bdafa33f67..f540a410da 100644 --- a/src/zarr/core/array.py +++ b/src/zarr/core/array.py @@ -5,7 +5,7 @@ from dataclasses import dataclass, field, replace from itertools import starmap from logging import getLogger -from typing import TYPE_CHECKING, Any, Generic, Literal, cast, overload +from typing import TYPE_CHECKING, Any, Generic, Literal, TypedDict, cast, overload import numpy as np import numpy.typing as npt @@ -41,7 +41,8 @@ parse_shapelike, product, ) -from zarr.core.config import config, parse_indexing_order +from zarr.core.config import config as base_config +from zarr.core.config import parse_indexing_order from zarr.core.indexing import ( BasicIndexer, BasicSelection, @@ -96,6 +97,15 @@ logger = getLogger(__name__) +@dataclass(frozen=True, slots=True, kw_only=True) +class ArrayConfig: + write_empty_chunks: bool + + +class ArrayConfigDict(TypedDict): + write_empty_chunks: bool + + def parse_array_metadata(data: Any) -> ArrayMetadata: if isinstance(data, ArrayMetadata): return data @@ -205,6 +215,7 @@ class AsyncArray(Generic[T_ArrayMetadata]): store_path: StorePath codec_pipeline: CodecPipeline = field(init=False) order: MemoryOrder + config: ArrayConfig @overload def __init__( @@ -212,6 +223,7 @@ def __init__( metadata: ArrayV2Metadata | ArrayV2MetadataDict, store_path: StorePath, order: MemoryOrder | None = None, + config: ArrayConfig | ArrayConfigDict | None = None, ) -> None: ... @overload @@ -220,6 +232,7 @@ def __init__( metadata: ArrayV3Metadata | ArrayV3MetadataDict, store_path: StorePath, order: MemoryOrder | None = None, + config: ArrayConfig | ArrayConfigDict | None = None, ) -> None: ... def __init__( @@ -227,6 +240,7 @@ def __init__( metadata: ArrayMetadata | ArrayMetadataDict, store_path: StorePath, order: MemoryOrder | None = None, + config: ArrayConfig | ArrayConfigDict | None = None, ) -> None: if isinstance(metadata, dict): zarr_format = metadata["zarr_format"] @@ -240,7 +254,7 @@ def __init__( raise ValueError(f"Invalid zarr_format: {zarr_format}. Expected 2 or 3") metadata_parsed = parse_array_metadata(metadata) - order_parsed = parse_indexing_order(order or config.get("array.order")) + order_parsed = parse_indexing_order(order or base_config.get("array.order")) object.__setattr__(self, "metadata", metadata_parsed) object.__setattr__(self, "store_path", store_path) @@ -610,7 +624,7 @@ async def _create_v2( await ensure_no_existing_node(store_path, zarr_format=2) if order is None: - order = parse_indexing_order(config.get("array.order")) + order = parse_indexing_order(base_config.get("array.order")) if dimension_separator is None: dimension_separator = "." @@ -1123,7 +1137,7 @@ async def _delete_key(key: str) -> None: for chunk_coords in old_chunk_coords.difference(new_chunk_coords) ], _delete_key, - config.get("async.concurrency"), + base_config.get("async.concurrency"), ) # Write new metadata From 723e716c40582db800146b007627d4857f85de36 Mon Sep 17 00:00:00 2001 From: Norman Rzepka Date: Fri, 13 Dec 2024 21:44:30 +0100 Subject: [PATCH 07/16] add ArrayConfig --- src/zarr/api/asynchronous.py | 37 +++++++++++------- src/zarr/codecs/pipeline.py | 5 ++- src/zarr/codecs/sharding.py | 8 ++-- src/zarr/codecs/transpose.py | 2 +- src/zarr/core/array.py | 75 ++++++++++++++++++++++++------------ src/zarr/core/array_spec.py | 38 +++++++++++++++--- src/zarr/core/common.py | 21 ++++++++-- src/zarr/core/metadata/v2.py | 6 +-- src/zarr/core/metadata/v3.py | 9 ++--- tests/test_api.py | 24 ++++++++++-- tests/test_array.py | 29 +++++++++----- 11 files changed, 179 insertions(+), 75 deletions(-) diff --git a/src/zarr/api/asynchronous.py b/src/zarr/api/asynchronous.py index f80e505f6e..261696fba0 100644 --- a/src/zarr/api/asynchronous.py +++ b/src/zarr/api/asynchronous.py @@ -16,6 +16,7 @@ ChunkCoords, MemoryOrder, ZarrFormat, + _warn_order_kwarg, _warn_write_empty_chunks_kwarg, ) from zarr.core.config import config @@ -725,6 +726,7 @@ async def create( read_only: bool | None = None, object_codec: Codec | None = None, # TODO: type has changed dimension_separator: Literal[".", "/"] | None = None, + write_empty_chunks: bool | None = None, zarr_version: ZarrFormat | None = None, # deprecated zarr_format: ZarrFormat | None = None, meta_array: Any | None = None, # TODO: need type @@ -760,6 +762,7 @@ async def create( fill_value : object Default value to use for uninitialized portions of the array. order : {'C', 'F'}, optional + Deprecated in favor of the `array.order` configuration variable. Memory layout to be used within each chunk. Default is set in Zarr's config (`array.order`). store : Store or str @@ -794,6 +797,19 @@ async def create( .. versionadded:: 2.8 + write_empty_chunks : bool, optional + Deprecated in favor of the `array.write_empty_chunks` configuration variable. + + If True (default), all chunks will be stored regardless of their + contents. If False, each chunk is compared to the array's fill value + prior to storing. If a chunk is uniformly equal to the fill value, then + that chunk is not be stored, and the store entry for that chunk's key + is deleted. This setting enables sparser storage, as only chunks with + non-fill-value data are stored, at the expense of overhead associated + with checking the data of each chunk. + + .. versionadded:: 2.11 + zarr_format : {2, 3, None}, optional The zarr format to use when saving. meta_array : array-like, optional @@ -839,17 +855,11 @@ async def create( raise ValueError( "dimension_separator is not supported for zarr format 3, use chunk_key_encoding instead" ) - else: - warnings.warn( - "dimension_separator is not yet implemented", - RuntimeWarning, - stacklevel=2, - ) - if "write_empty_chunks" in kwargs: - # warn users if the write_empty_chunks kwarg was used - write_empty_chunks = kwargs.pop("write_empty_chunks") - _warn_write_empty_chunks_kwarg(write_empty_chunks) + if order is not None: + _warn_order_kwarg() + if write_empty_chunks is not None: + _warn_write_empty_chunks_kwarg() if meta_array is not None: warnings.warn("meta_array is not yet implemented", RuntimeWarning, stacklevel=2) @@ -878,6 +888,7 @@ async def create( dimension_names=dimension_names, attributes=attributes, order=order, + write_empty_chunks=write_empty_chunks, **kwargs, ) @@ -1051,10 +1062,10 @@ async def open_array( zarr_format = _handle_zarr_version_or_format(zarr_version=zarr_version, zarr_format=zarr_format) + if "order" in kwargs: + _warn_order_kwarg() if "write_empty_chunks" in kwargs: - # warn users if the write_empty_chunks kwarg was used - write_empty_chunks = kwargs.pop("write_empty_chunks") - _warn_write_empty_chunks_kwarg(write_empty_chunks) + _warn_write_empty_chunks_kwarg() try: return await AsyncArray.open(store_path, zarr_format=zarr_format) diff --git a/src/zarr/codecs/pipeline.py b/src/zarr/codecs/pipeline.py index 485fb0a595..5a1f069823 100644 --- a/src/zarr/codecs/pipeline.py +++ b/src/zarr/codecs/pipeline.py @@ -331,7 +331,6 @@ async def write_batch( value: NDBuffer, drop_axes: tuple[int, ...] = (), ) -> None: - write_empty_chunks = config.get("array.write_empty_chunks") == True # noqa: E712 if self.supports_partial_encode: await self.encode_partial_batch( [ @@ -385,7 +384,9 @@ async def _read_key( if chunk_array is None: chunk_array_batch.append(None) # type: ignore[unreachable] else: - if not write_empty_chunks and chunk_array.all_equal(chunk_spec.fill_value): + if not chunk_spec.config.write_empty_chunks and chunk_array.all_equal( + chunk_spec.fill_value + ): chunk_array_batch.append(None) else: chunk_array_batch.append(chunk_array) diff --git a/src/zarr/codecs/sharding.py b/src/zarr/codecs/sharding.py index e7fd14ecb3..edb6de0402 100644 --- a/src/zarr/codecs/sharding.py +++ b/src/zarr/codecs/sharding.py @@ -20,7 +20,7 @@ from zarr.abc.store import ByteGetter, ByteRangeRequest, ByteSetter from zarr.codecs.bytes import BytesCodec from zarr.codecs.crc32c_ import Crc32cCodec -from zarr.core.array_spec import ArraySpec +from zarr.core.array_spec import ArrayConfig, ArraySpec from zarr.core.buffer import ( Buffer, BufferPrototype, @@ -665,7 +665,9 @@ def _get_index_chunk_spec(self, chunks_per_shard: ChunkCoords) -> ArraySpec: shape=chunks_per_shard + (2,), dtype=np.dtype(" ArraySpec: shape=self.chunk_shape, dtype=shard_spec.dtype, fill_value=shard_spec.fill_value, - order=shard_spec.order, + config=shard_spec.config, prototype=shard_spec.prototype, ) diff --git a/src/zarr/codecs/transpose.py b/src/zarr/codecs/transpose.py index 3a471beaf5..1aa1eb40e2 100644 --- a/src/zarr/codecs/transpose.py +++ b/src/zarr/codecs/transpose.py @@ -84,7 +84,7 @@ def resolve_metadata(self, chunk_spec: ArraySpec) -> ArraySpec: shape=tuple(chunk_spec.shape[self.order[i]] for i in range(chunk_spec.ndim)), dtype=chunk_spec.dtype, fill_value=chunk_spec.fill_value, - order=chunk_spec.order, + config=chunk_spec.config, prototype=chunk_spec.prototype, ) diff --git a/src/zarr/core/array.py b/src/zarr/core/array.py index bdafa33f67..462ca2ed32 100644 --- a/src/zarr/core/array.py +++ b/src/zarr/core/array.py @@ -14,6 +14,7 @@ from zarr.abc.store import Store, set_or_delete from zarr.codecs import _get_default_array_bytes_codec from zarr.codecs._v2 import V2Compressor, V2Filters +from zarr.core.array_spec import ArrayConfig from zarr.core.attributes import Attributes from zarr.core.buffer import ( BufferPrototype, @@ -41,7 +42,7 @@ parse_shapelike, product, ) -from zarr.core.config import config, parse_indexing_order +from zarr.core.config import config as zarr_config from zarr.core.indexing import ( BasicIndexer, BasicSelection, @@ -186,8 +187,8 @@ class AsyncArray(Generic[T_ArrayMetadata]): The metadata of the array. store_path : StorePath The path to the Zarr store. - order : {'C', 'F'}, optional - The order of the array data in memory, by default None. + config : ArrayConfig, optional + The runtime configuration of the array, by default None. Attributes ---------- @@ -197,21 +198,21 @@ class AsyncArray(Generic[T_ArrayMetadata]): The path to the Zarr store. codec_pipeline : CodecPipeline The codec pipeline used for encoding and decoding chunks. - order : {'C', 'F'} - The order of the array data in memory. + config : ArrayConfig + The runtime configuration of the array. """ metadata: T_ArrayMetadata store_path: StorePath codec_pipeline: CodecPipeline = field(init=False) - order: MemoryOrder + config: ArrayConfig @overload def __init__( self: AsyncArray[ArrayV2Metadata], metadata: ArrayV2Metadata | ArrayV2MetadataDict, store_path: StorePath, - order: MemoryOrder | None = None, + config: ArrayConfig | None = None, ) -> None: ... @overload @@ -219,14 +220,14 @@ def __init__( self: AsyncArray[ArrayV3Metadata], metadata: ArrayV3Metadata | ArrayV3MetadataDict, store_path: StorePath, - order: MemoryOrder | None = None, + config: ArrayConfig | None = None, ) -> None: ... def __init__( self, metadata: ArrayMetadata | ArrayMetadataDict, store_path: StorePath, - order: MemoryOrder | None = None, + config: ArrayConfig | None = None, ) -> None: if isinstance(metadata, dict): zarr_format = metadata["zarr_format"] @@ -240,11 +241,12 @@ def __init__( raise ValueError(f"Invalid zarr_format: {zarr_format}. Expected 2 or 3") metadata_parsed = parse_array_metadata(metadata) - order_parsed = parse_indexing_order(order or config.get("array.order")) + + config = ArrayConfig() if config is None else config object.__setattr__(self, "metadata", metadata_parsed) object.__setattr__(self, "store_path", store_path) - object.__setattr__(self, "order", order_parsed) + object.__setattr__(self, "config", config) object.__setattr__(self, "codec_pipeline", create_codec_pipeline(metadata=metadata_parsed)) # this overload defines the function signature when zarr_format is 2 @@ -352,6 +354,7 @@ async def create( chunks: ShapeLike | None = None, dimension_separator: Literal[".", "/"] | None = None, order: MemoryOrder | None = None, + write_empty_chunks: bool | None = None, filters: list[dict[str, JSON]] | None = None, compressor: dict[str, JSON] | None = None, # runtime @@ -384,6 +387,7 @@ async def create( chunks: ShapeLike | None = None, dimension_separator: Literal[".", "/"] | None = None, order: MemoryOrder | None = None, + write_empty_chunks: bool | None = None, filters: list[dict[str, JSON]] | None = None, compressor: dict[str, JSON] | None = None, # runtime @@ -422,7 +426,11 @@ async def create( The dimension separator (default is None). V2 only. V3 arrays cannot have a dimension separator. order : Literal["C", "F"], optional + Deprecated in favor of the `array.order` configuration variable. The order of the array (default is None). + write_empty_chunks: bool, optional + Deprecated in favor of the `array.write_empty_chunks` configuration variable. + If true, empty chunks will be written to the store. filters : list[dict[str, JSON]], optional The filters used to compress the data (default is None). V2 only. V3 arrays should not have 'filters' parameter. @@ -465,6 +473,8 @@ async def create( else: _chunks = normalize_chunks(chunk_shape, shape, dtype_parsed.itemsize) + config_parsed = ArrayConfig(order=order, write_empty_chunks=write_empty_chunks) + result: AsyncArray[ArrayV3Metadata] | AsyncArray[ArrayV2Metadata] if zarr_format == 3: if dimension_separator is not None: @@ -490,7 +500,7 @@ async def create( dimension_names=dimension_names, attributes=attributes, exists_ok=exists_ok, - order=order, + config=config_parsed, ) elif zarr_format == 2: if dtype is str or dtype == "str": @@ -518,7 +528,7 @@ async def create( chunks=_chunks, dimension_separator=dimension_separator, fill_value=fill_value, - order=order, + config=config_parsed, filters=filters, compressor=compressor, attributes=attributes, @@ -541,8 +551,8 @@ async def _create_v3( shape: ShapeLike, dtype: npt.DTypeLike, chunk_shape: ChunkCoords, + config: ArrayConfig, fill_value: Any | None = None, - order: MemoryOrder | None = None, chunk_key_encoding: ( ChunkKeyEncoding | tuple[Literal["default"], Literal[".", "/"]] @@ -586,7 +596,7 @@ async def _create_v3( attributes=attributes or {}, ) - array = cls(metadata=metadata, store_path=store_path, order=order) + array = cls(metadata=metadata, store_path=store_path, config=config) await array._save_metadata(metadata, ensure_parents=True) return array @@ -598,9 +608,9 @@ async def _create_v2( shape: ChunkCoords, dtype: npt.DTypeLike, chunks: ChunkCoords, + config: ArrayConfig, dimension_separator: Literal[".", "/"] | None = None, fill_value: None | float = None, - order: MemoryOrder | None = None, filters: list[dict[str, JSON]] | None = None, compressor: dict[str, JSON] | None = None, attributes: dict[str, JSON] | None = None, @@ -609,9 +619,6 @@ async def _create_v2( if not exists_ok: await ensure_no_existing_node(store_path, zarr_format=2) - if order is None: - order = parse_indexing_order(config.get("array.order")) - if dimension_separator is None: dimension_separator = "." @@ -619,14 +626,14 @@ async def _create_v2( shape=shape, dtype=np.dtype(dtype), chunks=chunks, - order=order, + order=config.order, dimension_separator=dimension_separator, fill_value=fill_value, compressor=compressor, filters=filters, attributes=attributes, ) - array = cls(metadata=metadata, store_path=store_path, order=order) + array = cls(metadata=metadata, store_path=store_path, config=config) await array._save_metadata(metadata, ensure_parents=True) return array @@ -742,6 +749,17 @@ def dtype(self) -> np.dtype[Any]: """ return self.metadata.dtype + @property + def order(self) -> MemoryOrder: + """Returns the memory order of the array. + + Returns + ------- + bool + Memory order of the array + """ + return self.config.order + @property def attrs(self) -> dict[str, JSON]: """Returns the attributes of the array. @@ -951,7 +969,7 @@ async def _get_selection( out_buffer = prototype.nd_buffer.create( shape=indexer.shape, dtype=out_dtype, - order=self.order, + order=self.config.order, fill_value=self.metadata.fill_value, ) if product(indexer.shape) > 0: @@ -960,7 +978,9 @@ async def _get_selection( [ ( self.store_path / self.metadata.encode_chunk_key(chunk_coords), - self.metadata.get_chunk_spec(chunk_coords, self.order, prototype=prototype), + self.metadata.get_chunk_spec( + chunk_coords, self.config, prototype=prototype + ), chunk_selection, out_selection, ) @@ -1079,7 +1099,7 @@ async def _set_selection( [ ( self.store_path / self.metadata.encode_chunk_key(chunk_coords), - self.metadata.get_chunk_spec(chunk_coords, self.order, prototype), + self.metadata.get_chunk_spec(chunk_coords, self.config, prototype), chunk_selection, out_selection, ) @@ -1123,7 +1143,7 @@ async def _delete_key(key: str) -> None: for chunk_coords in old_chunk_coords.difference(new_chunk_coords) ], _delete_key, - config.get("async.concurrency"), + zarr_config.get("async.concurrency"), ) # Write new metadata @@ -1179,6 +1199,7 @@ def create( chunks: ChunkCoords | None = None, dimension_separator: Literal[".", "/"] | None = None, order: MemoryOrder | None = None, + write_empty_chunks: bool | None = None, filters: list[dict[str, JSON]] | None = None, compressor: dict[str, JSON] | None = None, # runtime @@ -1208,6 +1229,9 @@ def create( The dimension separator (default is None). order : Literal["C", "F"], optional The order of the array (default is None). + write_empty_chunks: bool, optional + Deprecated in favor of the `array.write_empty_chunks` configuration variable. + If true, empty chunks will be written to the store. filters : list[dict[str, JSON]], optional The filters used to compress the data (default is None). compressor : dict[str, JSON], optional @@ -1235,6 +1259,7 @@ def create( chunks=chunks, dimension_separator=dimension_separator, order=order, + write_empty_chunks=write_empty_chunks, filters=filters, compressor=compressor, exists_ok=exists_ok, diff --git a/src/zarr/core/array_spec.py b/src/zarr/core/array_spec.py index c4d9c363fa..6402c5dc82 100644 --- a/src/zarr/core/array_spec.py +++ b/src/zarr/core/array_spec.py @@ -5,19 +5,44 @@ import numpy as np -from zarr.core.common import MemoryOrder, parse_fill_value, parse_order, parse_shapelike +from zarr.core.common import ( + MemoryOrder, + parse_fill_value, + parse_order, + parse_shapelike, + parse_write_empty_chunks, +) +from zarr.core.config import config as zarr_config if TYPE_CHECKING: from zarr.core.buffer import BufferPrototype from zarr.core.common import ChunkCoords +@dataclass(frozen=True) +class ArrayConfig: + order: MemoryOrder + write_empty_chunks: bool + + def __init__( + self, *, order: MemoryOrder | None = None, write_empty_chunks: bool | None = None + ) -> None: + order_parsed = parse_order(order) if order is not None else zarr_config.get("array.order") + write_empty_chunks_parsed = ( + parse_write_empty_chunks(write_empty_chunks) + if write_empty_chunks is not None + else zarr_config.get("array.write_empty_chunks") + ) + object.__setattr__(self, "order", order_parsed) + object.__setattr__(self, "write_empty_chunks", write_empty_chunks_parsed) + + @dataclass(frozen=True) class ArraySpec: shape: ChunkCoords dtype: np.dtype[Any] fill_value: Any - order: MemoryOrder + config: ArrayConfig prototype: BufferPrototype def __init__( @@ -25,20 +50,23 @@ def __init__( shape: ChunkCoords, dtype: np.dtype[Any], fill_value: Any, - order: MemoryOrder, + config: ArrayConfig, prototype: BufferPrototype, ) -> None: shape_parsed = parse_shapelike(shape) dtype_parsed = np.dtype(dtype) fill_value_parsed = parse_fill_value(fill_value) - order_parsed = parse_order(order) object.__setattr__(self, "shape", shape_parsed) object.__setattr__(self, "dtype", dtype_parsed) object.__setattr__(self, "fill_value", fill_value_parsed) - object.__setattr__(self, "order", order_parsed) + object.__setattr__(self, "config", config) object.__setattr__(self, "prototype", prototype) @property def ndim(self) -> int: return len(self.shape) + + @property + def order(self) -> MemoryOrder: + return self.config.order diff --git a/src/zarr/core/common.py b/src/zarr/core/common.py index 54401e5d22..fd0166109d 100644 --- a/src/zarr/core/common.py +++ b/src/zarr/core/common.py @@ -159,6 +159,12 @@ def parse_order(data: Any) -> Literal["C", "F"]: raise ValueError(f"Expected one of ('C', 'F'), got {data} instead.") +def parse_write_empty_chunks(data: Any) -> bool: + if isinstance(data, bool): + return data + raise ValueError(f"Expected bool, got {data} instead.") + + def parse_dtype(dtype: Any, zarr_format: ZarrFormat) -> np.dtype[Any]: if dtype is str or dtype == "str": if zarr_format == 2: @@ -169,10 +175,17 @@ def parse_dtype(dtype: Any, zarr_format: ZarrFormat) -> np.dtype[Any]: return np.dtype(dtype) -def _warn_write_empty_chunks_kwarg(write_empty_chunks: bool) -> None: +def _warn_write_empty_chunks_kwarg() -> None: + msg = ( + "The `write_empty_chunks` keyword argument is deprecated and will be removed in future versions. " + "To control whether empty chunks are written to storage, change the 'array.write_empty_chunks' configuration variable." + ) + warnings.warn(msg, RuntimeWarning, stacklevel=2) + + +def _warn_order_kwarg() -> None: msg = ( - f"The `write_empty_chunks` keyword argument was provided to this function with a value of {write_empty_chunks}." - "This keyword argument has no effect. To control whether empty chunks are written to" - " storage, change the 'array.write_empty_chunks' configuration variable." + "The `order` keyword argument is deprecated and will be removed in future versions. " + "To control whether empty chunks are written to storage, change the 'array.order' configuration variable." ) warnings.warn(msg, RuntimeWarning, stacklevel=2) diff --git a/src/zarr/core/metadata/v2.py b/src/zarr/core/metadata/v2.py index f18f2e4e8d..6a794b95c2 100644 --- a/src/zarr/core/metadata/v2.py +++ b/src/zarr/core/metadata/v2.py @@ -22,7 +22,7 @@ import numcodecs import numpy as np -from zarr.core.array_spec import ArraySpec +from zarr.core.array_spec import ArrayConfig, ArraySpec from zarr.core.chunk_grids import RegularChunkGrid from zarr.core.chunk_key_encodings import parse_separator from zarr.core.common import ZARRAY_JSON, ZATTRS_JSON, MemoryOrder, parse_shapelike @@ -185,13 +185,13 @@ def to_dict(self) -> dict[str, JSON]: return zarray_dict def get_chunk_spec( - self, _chunk_coords: ChunkCoords, order: MemoryOrder, prototype: BufferPrototype + self, _chunk_coords: ChunkCoords, array_config: ArrayConfig, prototype: BufferPrototype ) -> ArraySpec: return ArraySpec( shape=self.chunks, dtype=self.dtype, fill_value=self.fill_value, - order=order, + config=array_config, prototype=prototype, ) diff --git a/src/zarr/core/metadata/v3.py b/src/zarr/core/metadata/v3.py index 6b6f28dd96..52f742ff45 100644 --- a/src/zarr/core/metadata/v3.py +++ b/src/zarr/core/metadata/v3.py @@ -24,14 +24,13 @@ import numpy.typing as npt from zarr.abc.codec import ArrayArrayCodec, ArrayBytesCodec, BytesBytesCodec, Codec -from zarr.core.array_spec import ArraySpec +from zarr.core.array_spec import ArrayConfig, ArraySpec from zarr.core.chunk_grids import ChunkGrid, RegularChunkGrid from zarr.core.chunk_key_encodings import ChunkKeyEncoding from zarr.core.common import ( JSON, ZARR_JSON, ChunkCoords, - MemoryOrder, parse_named_configuration, parse_shapelike, ) @@ -244,7 +243,7 @@ def __init__( shape=shape_parsed, dtype=data_type_parsed.to_numpy(), fill_value=fill_value_parsed, - order="C", # TODO: order is not needed here. + config=ArrayConfig(), # TODO: config is not needed here. prototype=default_buffer_prototype(), # TODO: prototype is not needed here. ) codecs_parsed = [c.evolve_from_array_spec(array_spec) for c in codecs_parsed_partial] @@ -290,7 +289,7 @@ def ndim(self) -> int: return len(self.shape) def get_chunk_spec( - self, _chunk_coords: ChunkCoords, order: MemoryOrder, prototype: BufferPrototype + self, _chunk_coords: ChunkCoords, array_config: ArrayConfig, prototype: BufferPrototype ) -> ArraySpec: assert isinstance( self.chunk_grid, RegularChunkGrid @@ -299,7 +298,7 @@ def get_chunk_spec( shape=self.chunk_grid.chunk_shape, dtype=self.dtype, fill_value=self.fill_value, - order=order, + config=array_config, prototype=prototype, ) diff --git a/tests/test_api.py b/tests/test_api.py index 8525c982d0..84270b0390 100644 --- a/tests/test_api.py +++ b/tests/test_api.py @@ -53,7 +53,7 @@ def test_write_empty_chunks_warns(write_empty_chunks: bool) -> None: """ Test that using the `write_empty_chunks` kwarg on array access will raise a warning. """ - match = f"The `write_empty_chunks` keyword argument was provided to this function with a value of {write_empty_chunks}." + match = "The `write_empty_chunks` keyword argument .*" with pytest.warns(RuntimeWarning, match=match): _ = zarr.array( data=np.arange(10), shape=(10,), dtype="uint8", write_empty_chunks=write_empty_chunks @@ -221,10 +221,26 @@ def test_open_with_mode_w_minus(tmp_path: pathlib.Path) -> None: zarr.open(store=tmp_path, mode="w-") -@pytest.mark.parametrize("order", ["C", "F", None]) @pytest.mark.parametrize("zarr_format", [2, 3]) -def test_array_order(order: MemoryOrder | None, zarr_format: ZarrFormat) -> None: - arr = zarr.ones(shape=(2, 2), order=order, zarr_format=zarr_format) +def test_array_order(zarr_format: ZarrFormat) -> None: + arr = zarr.ones(shape=(2, 2), order=None, zarr_format=zarr_format) + expected = zarr.config.get("array.order") + assert arr.order == expected + + vals = np.asarray(arr) + if expected == "C": + assert vals.flags.c_contiguous + elif expected == "F": + assert vals.flags.f_contiguous + else: + raise AssertionError + + +@pytest.mark.parametrize("order", ["C", "F"]) +@pytest.mark.parametrize("zarr_format", [2, 3]) +def test_array_order_warns(order: MemoryOrder | None, zarr_format: ZarrFormat) -> None: + with pytest.warns(RuntimeWarning, match="The `order` keyword argument .*"): + arr = zarr.ones(shape=(2, 2), order=order, zarr_format=zarr_format) expected = order or zarr.config.get("array.order") assert arr.order == expected diff --git a/tests/test_array.py b/tests/test_array.py index c740c2d4e2..1f941dedd4 100644 --- a/tests/test_array.py +++ b/tests/test_array.py @@ -454,22 +454,31 @@ def test_write_empty_chunks( those chunks not being present in the store. In particular, they should be deleted if they were already present. """ - arr = Array.create( - store=store, - shape=(2,), - zarr_format=zarr_format, - dtype="i4", - fill_value=fill_value, - chunk_shape=(1,), - ) + with config.set({"array.write_empty_chunks": write_empty_chunks}): + arr = Array.create( + store=store, + shape=(2,), + zarr_format=zarr_format, + dtype="i4", + fill_value=fill_value, + chunk_shape=(1,), + ) # initialize the store with some non-fill value chunks arr[:] = fill_value + 1 assert arr.nchunks_initialized == arr.nchunks - with config.set({"array.write_empty_chunks": write_empty_chunks}): - arr[:] = fill_value + arr[:] = fill_value + if not write_empty_chunks: assert arr.nchunks_initialized == 0 else: assert arr.nchunks_initialized == arr.nchunks + + +@pytest.mark.parametrize("store", ["memory"], indirect=True) +def test_write_empty_chunks_deprecated(store: MemoryStore) -> None: + with pytest.warns( + RuntimeWarning, match="The `write_empty_chunks` keyword argument is deprecated.*" + ): + zarr.open(store=store, mode="a", shape=(3, 3), write_empty_chunks=True) From 49941bfd73bf2529cf971ef9a08fe8e69df84953 Mon Sep 17 00:00:00 2001 From: Norman Rzepka Date: Fri, 13 Dec 2024 21:54:57 +0100 Subject: [PATCH 08/16] docstring --- src/zarr/core/array.py | 1 + 1 file changed, 1 insertion(+) diff --git a/src/zarr/core/array.py b/src/zarr/core/array.py index 462ca2ed32..bec4348110 100644 --- a/src/zarr/core/array.py +++ b/src/zarr/core/array.py @@ -1228,6 +1228,7 @@ def create( dimension_separator : Literal[".", "/"], optional The dimension separator (default is None). order : Literal["C", "F"], optional + Deprecated in favor of the `array.order` configuration variable. The order of the array (default is None). write_empty_chunks: bool, optional Deprecated in favor of the `array.write_empty_chunks` configuration variable. From 6272cfd4e41463dbeb5ca85fce191dd3e3265398 Mon Sep 17 00:00:00 2001 From: Norman Rzepka Date: Fri, 13 Dec 2024 22:10:44 +0100 Subject: [PATCH 09/16] ignore warning --- tests/test_v2.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/test_v2.py b/tests/test_v2.py index 890d4039a3..231698709a 100644 --- a/tests/test_v2.py +++ b/tests/test_v2.py @@ -125,6 +125,7 @@ async def test_create_dtype_str(dtype: Any) -> None: np.testing.assert_array_equal(result, np.array(["a", "bb", "ccc"], dtype="object")) +@pytest.mark.filterwarnings("ignore:The `order` keyword argument.*") @pytest.mark.parametrize("filters", [[], [numcodecs.Delta(dtype=" None: From 3493c7012653273f6c25c6b0b4bb1a9b8527d58f Mon Sep 17 00:00:00 2001 From: Norman Rzepka Date: Fri, 13 Dec 2024 22:12:34 +0100 Subject: [PATCH 10/16] fix v2 test --- tests/test_v2.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/test_v2.py b/tests/test_v2.py index 231698709a..b495b8e9ee 100644 --- a/tests/test_v2.py +++ b/tests/test_v2.py @@ -12,6 +12,7 @@ import zarr.core.buffer import zarr.storage from zarr import Array +from zarr.core.config import config from zarr.storage import MemoryStore, StorePath @@ -125,12 +126,12 @@ async def test_create_dtype_str(dtype: Any) -> None: np.testing.assert_array_equal(result, np.array(["a", "bb", "ccc"], dtype="object")) -@pytest.mark.filterwarnings("ignore:The `order` keyword argument.*") @pytest.mark.parametrize("filters", [[], [numcodecs.Delta(dtype=" None: array_fixture = [42] - arr = zarr.create(shape=1, dtype=" Date: Thu, 19 Dec 2024 14:57:57 +0100 Subject: [PATCH 11/16] add test to ensure that write_empty_chunks can be set via the global config --- tests/test_array.py | 35 ++++++++++++++++++++++++----------- 1 file changed, 24 insertions(+), 11 deletions(-) diff --git a/tests/test_array.py b/tests/test_array.py index ba43e493c3..1755ee8e90 100644 --- a/tests/test_array.py +++ b/tests/test_array.py @@ -16,7 +16,6 @@ from zarr.core.buffer import default_buffer_prototype from zarr.core.buffer.cpu import NDBuffer from zarr.core.common import JSON, MemoryOrder, ZarrFormat -from zarr.core.config import config from zarr.core.group import AsyncGroup from zarr.core.indexing import ceildiv from zarr.core.metadata.v3 import DataType @@ -759,10 +758,21 @@ def test_array_create_order( raise AssertionError +@pytest.mark.parametrize("write_empty_chunks", [True, False]) +def test_write_empty_chunks_config(write_empty_chunks: bool) -> None: + """ + Test that the value of write_empty_chunks is sensitive to the global config when not set + explicitly + """ + with zarr.config.set({"array.write_empty_chunks": write_empty_chunks}): + arr = Array.create({}, shape=(2, 2), dtype="i4") + assert arr._async_array.config.write_empty_chunks == write_empty_chunks + + @pytest.mark.parametrize("store", ["memory"], indirect=True) @pytest.mark.parametrize("write_empty_chunks", [True, False]) @pytest.mark.parametrize("fill_value", [0, 5]) -def test_write_empty_chunks( +def test_write_empty_chunks_behavior( zarr_format: ZarrFormat, store: MemoryStore, write_empty_chunks: bool, fill_value: int ) -> None: """ @@ -774,15 +784,18 @@ def test_write_empty_chunks( those chunks not being present in the store. In particular, they should be deleted if they were already present. """ - with config.set({"array.write_empty_chunks": write_empty_chunks}): - arr = Array.create( - store=store, - shape=(2,), - zarr_format=zarr_format, - dtype="i4", - fill_value=fill_value, - chunk_shape=(1,), - ) + + arr = Array.create( + store=store, + shape=(2,), + zarr_format=zarr_format, + dtype="i4", + fill_value=fill_value, + chunk_shape=(1,), + write_empty_chunks=write_empty_chunks, + ) + + assert arr._async_array.config.write_empty_chunks == write_empty_chunks # initialize the store with some non-fill value chunks arr[:] = fill_value + 1 From 08a2d529a0f0e12e905206e1d5d95d517253a1fb Mon Sep 17 00:00:00 2001 From: Norman Rzepka Date: Thu, 19 Dec 2024 15:56:32 +0100 Subject: [PATCH 12/16] fix tests --- src/zarr/api/synchronous.py | 2 +- src/zarr/core/array.py | 14 +++++++------- tests/test_array.py | 4 ++-- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/zarr/api/synchronous.py b/src/zarr/api/synchronous.py index 6ae062865c..ac621049af 100644 --- a/src/zarr/api/synchronous.py +++ b/src/zarr/api/synchronous.py @@ -542,7 +542,7 @@ def create( read_only: bool | None = None, object_codec: Codec | None = None, # TODO: type has changed dimension_separator: Literal[".", "/"] | None = None, - write_empty_chunks: bool = False, # TODO: default has changed + write_empty_chunks: bool | None = None, # TODO: default has changed zarr_version: ZarrFormat | None = None, # deprecated zarr_format: ZarrFormat | None = None, meta_array: Any | None = None, # TODO: need type diff --git a/src/zarr/core/array.py b/src/zarr/core/array.py index f6f1cc4a5b..c20aa0dbc3 100644 --- a/src/zarr/core/array.py +++ b/src/zarr/core/array.py @@ -199,14 +199,14 @@ class AsyncArray(Generic[T_ArrayMetadata]): The path to the Zarr store. codec_pipeline : CodecPipeline The codec pipeline used for encoding and decoding chunks. - config : ArrayConfig + _config : ArrayConfig The runtime configuration of the array. """ metadata: T_ArrayMetadata store_path: StorePath codec_pipeline: CodecPipeline = field(init=False) - config: ArrayConfig + _config: ArrayConfig @overload def __init__( @@ -247,7 +247,7 @@ def __init__( object.__setattr__(self, "metadata", metadata_parsed) object.__setattr__(self, "store_path", store_path) - object.__setattr__(self, "config", config) + object.__setattr__(self, "_config", config) object.__setattr__(self, "codec_pipeline", create_codec_pipeline(metadata=metadata_parsed)) # this overload defines the function signature when zarr_format is 2 @@ -800,7 +800,7 @@ def order(self) -> MemoryOrder: bool Memory order of the array """ - return self.config.order + return self._config.order @property def attrs(self) -> dict[str, JSON]: @@ -1024,7 +1024,7 @@ async def _get_selection( out_buffer = prototype.nd_buffer.create( shape=indexer.shape, dtype=out_dtype, - order=self.config.order, + order=self._config.order, fill_value=self.metadata.fill_value, ) if product(indexer.shape) > 0: @@ -1034,7 +1034,7 @@ async def _get_selection( ( self.store_path / self.metadata.encode_chunk_key(chunk_coords), self.metadata.get_chunk_spec( - chunk_coords, self.config, prototype=prototype + chunk_coords, self._config, prototype=prototype ), chunk_selection, out_selection, @@ -1157,7 +1157,7 @@ async def _set_selection( [ ( self.store_path / self.metadata.encode_chunk_key(chunk_coords), - self.metadata.get_chunk_spec(chunk_coords, self.config, prototype), + self.metadata.get_chunk_spec(chunk_coords, self._config, prototype), chunk_selection, out_selection, ) diff --git a/tests/test_array.py b/tests/test_array.py index 1755ee8e90..78d6d7fc9c 100644 --- a/tests/test_array.py +++ b/tests/test_array.py @@ -766,7 +766,7 @@ def test_write_empty_chunks_config(write_empty_chunks: bool) -> None: """ with zarr.config.set({"array.write_empty_chunks": write_empty_chunks}): arr = Array.create({}, shape=(2, 2), dtype="i4") - assert arr._async_array.config.write_empty_chunks == write_empty_chunks + assert arr._async_array._config.write_empty_chunks == write_empty_chunks @pytest.mark.parametrize("store", ["memory"], indirect=True) @@ -795,7 +795,7 @@ def test_write_empty_chunks_behavior( write_empty_chunks=write_empty_chunks, ) - assert arr._async_array.config.write_empty_chunks == write_empty_chunks + assert arr._async_array._config.write_empty_chunks == write_empty_chunks # initialize the store with some non-fill value chunks arr[:] = fill_value + 1 From 0b6e7d826a70cd29523335c7f8da4482c02f1114 Mon Sep 17 00:00:00 2001 From: Davis Vann Bennett Date: Fri, 20 Dec 2024 13:25:04 +0100 Subject: [PATCH 13/16] remove write_empty_chunks from Array.create; separate metadata order from config order --- src/zarr/api/asynchronous.py | 54 ++++++++++++++++-------- src/zarr/api/synchronous.py | 26 ++++++------ src/zarr/core/array.py | 81 +++++++++++++++--------------------- src/zarr/core/array_spec.py | 71 +++++++++++++++++++++++++------ src/zarr/core/common.py | 14 +++++-- src/zarr/core/metadata/v3.py | 2 +- tests/test_array.py | 38 ++++++++++++++--- 7 files changed, 187 insertions(+), 99 deletions(-) diff --git a/src/zarr/api/asynchronous.py b/src/zarr/api/asynchronous.py index e28a44fd48..14078944d7 100644 --- a/src/zarr/api/asynchronous.py +++ b/src/zarr/api/asynchronous.py @@ -10,6 +10,7 @@ from typing_extensions import deprecated from zarr.core.array import Array, AsyncArray, get_array_metadata +from zarr.core.array_spec import ArrayConfig, ArrayConfigParams from zarr.core.buffer import NDArrayLike from zarr.core.common import ( JSON, @@ -812,6 +813,7 @@ async def create( codecs: Iterable[Codec | dict[str, JSON]] | None = None, dimension_names: Iterable[str] | None = None, storage_options: dict[str, Any] | None = None, + config: ArrayConfig | ArrayConfigParams | None = None, **kwargs: Any, ) -> AsyncArray[ArrayV2Metadata] | AsyncArray[ArrayV3Metadata]: """Create an array. @@ -858,9 +860,10 @@ async def create( These defaults can be changed by modifying the value of ``array.v2_default_compressor`` in :mod:`zarr.core.config`. fill_value : object Default value to use for uninitialized portions of the array. order : {'C', 'F'}, optional - Deprecated in favor of the `array.order` configuration variable. + Deprecated in favor of the ``config`` keyword argument. + Pass ``{'order': }`` to ``create`` instead of using this parameter. Memory layout to be used within each chunk. - If not specified, default is taken from the Zarr config ```array.order```. + If not specified, the ``array.order`` parameter in the global config will be used. store : Store or str Store or path to directory in file system or name of zip file. synchronizer : object, optional @@ -894,32 +897,26 @@ async def create( Separator placed between the dimensions of a chunk. V2 only. V3 arrays should use ``chunk_key_encoding`` instead. Default is ".". - .. versionadded:: 2.8 - write_empty_chunks : bool, optional - Deprecated in favor of the `array.write_empty_chunks` configuration variable. - - If True (default), all chunks will be stored regardless of their + Deprecated in favor of the ``config`` keyword argument. + Pass ``{'write_empty_chunks': }`` to ``create`` instead of using this parameter. + If True, all chunks will be stored regardless of their contents. If False, each chunk is compared to the array's fill value prior to storing. If a chunk is uniformly equal to the fill value, then that chunk is not be stored, and the store entry for that chunk's key - is deleted. This setting enables sparser storage, as only chunks with - non-fill-value data are stored, at the expense of overhead associated - with checking the data of each chunk. - - .. versionadded:: 2.11 - + is deleted. zarr_format : {2, 3, None}, optional The zarr format to use when saving. Default is 3. meta_array : array-like, optional An array instance to use for determining arrays to create and return to users. Use `numpy.empty(())` by default. - - .. versionadded:: 2.13 storage_options : dict If using an fsspec URL to create the store, these will be passed to the backend implementation. Ignored otherwise. + config : ArrayConfig or ArrayConfigParams, optional + Runtime configuration of the array. If provided, will override the + default values from `zarr.config.array`. Returns ------- @@ -973,6 +970,30 @@ async def create( if mode is None: mode = "a" store_path = await make_store_path(store, path=path, mode=mode, storage_options=storage_options) + + config_dict: ArrayConfigParams = {} + + if write_empty_chunks is not None: + if config is not None: + msg = ( + "Both write_empty_chunks and config keyword arguments are set. " + "This is redundant. When both are set, write_empty_chunks will be ignored and " + "config will be used." + ) + warnings.warn(UserWarning(msg), stacklevel=1) + config_dict["write_empty_chunks"] = write_empty_chunks + if order is not None: + if config is not None: + msg = ( + "Both order and config keyword arguments are set. " + "This is redundant. When both are set, order will be ignored and " + "config will be used." + ) + warnings.warn(UserWarning(msg), stacklevel=1) + config_dict["order"] = order + + config_parsed = ArrayConfig.from_dict(config_dict) + return await AsyncArray.create( store_path, shape=shape, @@ -989,8 +1010,7 @@ async def create( codecs=codecs, dimension_names=dimension_names, attributes=attributes, - order=order, - write_empty_chunks=write_empty_chunks, + config=config_parsed, **kwargs, ) diff --git a/src/zarr/api/synchronous.py b/src/zarr/api/synchronous.py index ac621049af..cd1ef8b38d 100644 --- a/src/zarr/api/synchronous.py +++ b/src/zarr/api/synchronous.py @@ -17,6 +17,7 @@ from zarr.abc.codec import Codec from zarr.api.asynchronous import ArrayLike, PathLike + from zarr.core.array_spec import ArrayConfig, ArrayConfigParams from zarr.core.buffer import NDArrayLike from zarr.core.chunk_key_encodings import ChunkKeyEncoding from zarr.core.common import JSON, AccessModeLiteral, ChunkCoords, MemoryOrder, ZarrFormat @@ -558,6 +559,7 @@ def create( codecs: Iterable[Codec | dict[str, JSON]] | None = None, dimension_names: Iterable[str] | None = None, storage_options: dict[str, Any] | None = None, + config: ArrayConfig | ArrayConfigParams | None = None, **kwargs: Any, ) -> Array: """Create an array. @@ -578,8 +580,10 @@ def create( fill_value : object Default value to use for uninitialized portions of the array. order : {'C', 'F'}, optional + Deprecated in favor of the ``config`` keyword argument. + Pass ``{'order': }`` to ``create`` instead of using this parameter. Memory layout to be used within each chunk. - Default is set in Zarr's config (`array.order`). + If not specified, the ``array.order`` parameter in the global config will be used. store : Store or str Store or path to directory in file system or name of zip file. synchronizer : object, optional @@ -609,30 +613,25 @@ def create( A codec to encode object arrays, only needed if dtype=object. dimension_separator : {'.', '/'}, optional Separator placed between the dimensions of a chunk. - - .. versionadded:: 2.8 - write_empty_chunks : bool, optional - If True (default), all chunks will be stored regardless of their + Deprecated in favor of the ``config`` keyword argument. + Pass ``{'write_empty_chunks': }`` to ``create`` instead of using this parameter. + If True, all chunks will be stored regardless of their contents. If False, each chunk is compared to the array's fill value prior to storing. If a chunk is uniformly equal to the fill value, then that chunk is not be stored, and the store entry for that chunk's key - is deleted. This setting enables sparser storage, as only chunks with - non-fill-value data are stored, at the expense of overhead associated - with checking the data of each chunk. - - .. versionadded:: 2.11 - + is deleted. zarr_format : {2, 3, None}, optional The zarr format to use when saving. meta_array : array-like, optional An array instance to use for determining arrays to create and return to users. Use `numpy.empty(())` by default. - - .. versionadded:: 2.13 storage_options : dict If using an fsspec URL to create the store, these will be passed to the backend implementation. Ignored otherwise. + config : ArrayConfig or ArrayConfigParams, optional + Runtime configuration of the array. If provided, will override the + default values from `zarr.config.array`. Returns ------- @@ -669,6 +668,7 @@ def create( codecs=codecs, dimension_names=dimension_names, storage_options=storage_options, + config=config, **kwargs, ) ) diff --git a/src/zarr/core/array.py b/src/zarr/core/array.py index 22acbc5fd5..49611a7d2d 100644 --- a/src/zarr/core/array.py +++ b/src/zarr/core/array.py @@ -15,7 +15,7 @@ from zarr.abc.store import Store, set_or_delete from zarr.codecs._v2 import V2Codec from zarr.core._info import ArrayInfo -from zarr.core.array_spec import ArrayConfig +from zarr.core.array_spec import ArrayConfig, ArrayConfigParams, normalize_array_config from zarr.core.attributes import Attributes from zarr.core.buffer import ( BufferPrototype, @@ -38,8 +38,10 @@ MemoryOrder, ShapeLike, ZarrFormat, + _warn_order_kwarg, concurrent_map, parse_dtype, + parse_order, parse_shapelike, product, ) @@ -243,7 +245,7 @@ def __init__( metadata_parsed = parse_array_metadata(metadata) - config = ArrayConfig() if config is None else config + config = ArrayConfig.from_dict({}) if config is None else config object.__setattr__(self, "metadata", metadata_parsed) object.__setattr__(self, "store_path", store_path) @@ -271,6 +273,7 @@ async def create( # runtime overwrite: bool = False, data: npt.ArrayLike | None = None, + config: ArrayConfig | ArrayConfigParams | None = None, ) -> AsyncArray[ArrayV2Metadata]: ... # this overload defines the function signature when zarr_format is 3 @@ -299,34 +302,7 @@ async def create( # runtime overwrite: bool = False, data: npt.ArrayLike | None = None, - ) -> AsyncArray[ArrayV3Metadata]: ... - - # this overload is necessary to handle the case where the `zarr_format` kwarg is unspecified - @overload - @classmethod - async def create( - cls, - store: StoreLike, - *, - # v2 and v3 - shape: ShapeLike, - dtype: npt.DTypeLike, - zarr_format: Literal[3] = 3, - fill_value: Any | None = None, - attributes: dict[str, JSON] | None = None, - # v3 only - chunk_shape: ShapeLike | None = None, - chunk_key_encoding: ( - ChunkKeyEncoding - | tuple[Literal["default"], Literal[".", "/"]] - | tuple[Literal["v2"], Literal[".", "/"]] - | None - ) = None, - codecs: Iterable[Codec | dict[str, JSON]] | None = None, - dimension_names: Iterable[str] | None = None, - # runtime - overwrite: bool = False, - data: npt.ArrayLike | None = None, + config: ArrayConfig | ArrayConfigParams | None = None, ) -> AsyncArray[ArrayV3Metadata]: ... @overload @@ -355,12 +331,12 @@ async def create( chunks: ShapeLike | None = None, dimension_separator: Literal[".", "/"] | None = None, order: MemoryOrder | None = None, - write_empty_chunks: bool | None = None, filters: list[dict[str, JSON]] | None = None, compressor: dict[str, JSON] | None = None, # runtime overwrite: bool = False, data: npt.ArrayLike | None = None, + config: ArrayConfig | ArrayConfigParams | None = None, ) -> AsyncArray[ArrayV3Metadata] | AsyncArray[ArrayV2Metadata]: ... @classmethod @@ -388,12 +364,12 @@ async def create( chunks: ShapeLike | None = None, dimension_separator: Literal[".", "/"] | None = None, order: MemoryOrder | None = None, - write_empty_chunks: bool | None = None, filters: list[dict[str, JSON]] | None = None, compressor: dict[str, JSON] | None = None, # runtime overwrite: bool = False, data: npt.ArrayLike | None = None, + config: ArrayConfig | ArrayConfigParams | None = None, ) -> AsyncArray[ArrayV2Metadata] | AsyncArray[ArrayV3Metadata]: """ Method to create a new asynchronous array instance. @@ -443,11 +419,11 @@ async def create( The dimension separator (default is "."). V2 only. V3 arrays should use ``chunk_key_encoding`` instead. order : Literal["C", "F"], optional - Deprecated in favor of the ``array.order`` in :mod:`zarr.core.config`. - The order of the array (default is "C"). - write_empty_chunks : bool, optional - Deprecated in favor of the ``array.write_empty_chunks`` in :mod:`zarr.core.config`. - If true, empty chunks will be written to the store. + The memory of the array (default is "C"). + If ``zarr_format`` is 2, this parameter sets the memory order of the array. + If `zarr_format`` is 3, then this parameter is deprecated, because memory order + is a runtime parameter for zarr 3 arrays. The recommended way to specify the memory + order for zarr v3 arrays is via the ``config`` parameter, e.g. ``{'config': 'C'}``. filters : list[dict[str, JSON]], optional Sequence of filters to use to encode chunk data prior to compression. V2 only. V3 arrays should use ``codecs`` instead. If neither ``compressor`` @@ -499,8 +475,7 @@ async def create( _chunks = normalize_chunks(chunks, shape, dtype_parsed.itemsize) else: _chunks = normalize_chunks(chunk_shape, shape, dtype_parsed.itemsize) - - config_parsed = ArrayConfig(order=order, write_empty_chunks=write_empty_chunks) + config_parsed = normalize_array_config(config) result: AsyncArray[ArrayV3Metadata] | AsyncArray[ArrayV2Metadata] if zarr_format == 3: @@ -516,6 +491,10 @@ async def create( raise ValueError( "compressor cannot be used for arrays with version 3. Use bytes-to-bytes codecs instead." ) + + if order is not None: + _warn_order_kwarg() + result = await cls._create_v3( store_path, shape=shape, @@ -540,6 +519,12 @@ async def create( ) if dimension_names is not None: raise ValueError("dimension_names cannot be used for arrays with version 2.") + + if order is None: + order_parsed = parse_order(zarr_config.get("array.order")) + else: + order_parsed = order + result = await cls._create_v2( store_path, shape=shape, @@ -547,6 +532,7 @@ async def create( chunks=_chunks, dimension_separator=dimension_separator, fill_value=fill_value, + order=order_parsed, config=config_parsed, filters=filters, compressor=compressor, @@ -636,6 +622,7 @@ async def _create_v2( shape: ChunkCoords, dtype: np.dtype[Any], chunks: ChunkCoords, + order: MemoryOrder, config: ArrayConfig, dimension_separator: Literal[".", "/"] | None = None, fill_value: float | None = None, @@ -667,7 +654,7 @@ async def _create_v2( shape=shape, dtype=np.dtype(dtype), chunks=chunks, - order=config.order, + order=order, dimension_separator=dimension_separator, fill_value=fill_value, compressor=compressor, @@ -1519,11 +1506,11 @@ def create( chunks: ChunkCoords | None = None, dimension_separator: Literal[".", "/"] | None = None, order: MemoryOrder | None = None, - write_empty_chunks: bool | None = None, filters: list[dict[str, JSON]] | None = None, compressor: dict[str, JSON] | None = None, # runtime overwrite: bool = False, + config: ArrayConfig | ArrayConfigParams | None = None, ) -> Array: """Creates a new Array instance from an initialized store. @@ -1566,11 +1553,11 @@ def create( The dimension separator (default is "."). V2 only. V3 arrays should use ``chunk_key_encoding`` instead. order : Literal["C", "F"], optional - Deprecated in favor of the ``array.order`` in :mod:`zarr.core.config`. - The order of the array (default is "C"). - write_empty_chunks : bool, optional - Deprecated in favor of the ``array.write_empty_chunks`` in :mod:`zarr.core.config`. - If true, empty chunks will be written to the store. + The memory of the array (default is "C"). + If ``zarr_format`` is 2, this parameter sets the memory order of the array. + If `zarr_format`` is 3, then this parameter is deprecated, because memory order + is a runtime parameter for zarr 3 arrays. The recommended way to specify the memory + order for zarr v3 arrays is via the ``config`` parameter, e.g. ``{'order': 'C'}``. filters : list[dict[str, JSON]], optional Sequence of filters to use to encode chunk data prior to compression. V2 only. V3 arrays should use ``codecs`` instead. If neither ``compressor`` @@ -1610,10 +1597,10 @@ def create( chunks=chunks, dimension_separator=dimension_separator, order=order, - write_empty_chunks=write_empty_chunks, filters=filters, compressor=compressor, overwrite=overwrite, + config=config, ), ) return cls(async_array) diff --git a/src/zarr/core/array_spec.py b/src/zarr/core/array_spec.py index 6402c5dc82..ee6934d05f 100644 --- a/src/zarr/core/array_spec.py +++ b/src/zarr/core/array_spec.py @@ -1,41 +1,88 @@ from __future__ import annotations -from dataclasses import dataclass -from typing import TYPE_CHECKING, Any +from dataclasses import dataclass, fields +from typing import TYPE_CHECKING, Any, Literal, Self, TypedDict, cast import numpy as np from zarr.core.common import ( MemoryOrder, + parse_bool, parse_fill_value, parse_order, parse_shapelike, - parse_write_empty_chunks, ) from zarr.core.config import config as zarr_config if TYPE_CHECKING: + from typing import NotRequired + from zarr.core.buffer import BufferPrototype from zarr.core.common import ChunkCoords +class ArrayConfigParams(TypedDict): + """ + A TypedDict model of the attributes of an ArrayConfig class, but with no required fields. + This allows for partial construction of an ArrayConfig, with the assumption that the unset + keys will be taken from a global configuration. + """ + + order: NotRequired[MemoryOrder] + write_empty_chunks: NotRequired[bool] + + @dataclass(frozen=True) class ArrayConfig: + """ + A model of the runtime configuration of an array. + + Parameters + ---------- + order : MemoryOrder + The memory layout of the arrays returned when reading data from the store. + write_empty_chunks : bool + If True, empty chunks will be written to the store. + """ + order: MemoryOrder write_empty_chunks: bool - def __init__( - self, *, order: MemoryOrder | None = None, write_empty_chunks: bool | None = None - ) -> None: - order_parsed = parse_order(order) if order is not None else zarr_config.get("array.order") - write_empty_chunks_parsed = ( - parse_write_empty_chunks(write_empty_chunks) - if write_empty_chunks is not None - else zarr_config.get("array.write_empty_chunks") - ) + def __init__(self, order: MemoryOrder, write_empty_chunks: bool) -> None: + order_parsed = parse_order(order) + write_empty_chunks_parsed = parse_bool(write_empty_chunks) + object.__setattr__(self, "order", order_parsed) object.__setattr__(self, "write_empty_chunks", write_empty_chunks_parsed) + @classmethod + def from_dict(cls, data: ArrayConfigParams) -> Self: + """ + Create an ArrayConfig from a dict. The keys of that dict are a subset of the + attributes of the ArrayConfig class. Any keys missing from that dict will be set to the + the values in the ``array`` namespace of ``zarr.config``. + """ + kwargs_out: ArrayConfigParams = {} + for f in fields(ArrayConfig): + field_name = cast(Literal["order", "write_empty_chunks"], f.name) + if field_name not in data: + kwargs_out[field_name] = zarr_config.get(f"array.{field_name}") + else: + kwargs_out[field_name] = data[field_name] + return cls(**kwargs_out) + + +def normalize_array_config(data: ArrayConfig | ArrayConfigParams | None) -> ArrayConfig: + """ + Convert various types of data to an ArrayConfig. + """ + if data is None: + return ArrayConfig.from_dict({}) + elif isinstance(data, ArrayConfig): + return data + else: + return ArrayConfig.from_dict(data) + @dataclass(frozen=True) class ArraySpec: diff --git a/src/zarr/core/common.py b/src/zarr/core/common.py index 75a23d83ee..3db00b1a06 100644 --- a/src/zarr/core/common.py +++ b/src/zarr/core/common.py @@ -161,7 +161,7 @@ def parse_order(data: Any) -> Literal["C", "F"]: raise ValueError(f"Expected one of ('C', 'F'), got {data} instead.") -def parse_write_empty_chunks(data: Any) -> bool: +def parse_bool(data: Any) -> bool: if isinstance(data, bool): return data raise ValueError(f"Expected bool, got {data} instead.") @@ -178,16 +178,22 @@ def parse_dtype(dtype: Any, zarr_format: ZarrFormat) -> np.dtype[Any]: def _warn_write_empty_chunks_kwarg() -> None: + # TODO: link to docs page on array configuration in this message msg = ( "The `write_empty_chunks` keyword argument is deprecated and will be removed in future versions. " - "To control whether empty chunks are written to storage, change the 'array.write_empty_chunks' configuration variable." + "To control whether empty chunks are written to storage, either use the `config` keyword " + "argument, as in `config={'write_empty_chunks: True}`," + "or change the global 'array.write_empty_chunks' configuration variable." ) warnings.warn(msg, RuntimeWarning, stacklevel=2) def _warn_order_kwarg() -> None: + # TODO: link to docs page on array configuration in this message msg = ( - "The `order` keyword argument is deprecated and will be removed in future versions. " - "To control whether empty chunks are written to storage, change the 'array.order' configuration variable." + "The `order` keyword argument has no effect for zarr v3 arrays. " + "To control the memory layout of the array, either use the `config` keyword " + "argument, as in `config={'order: 'C'}`," + "or change the global 'array.order' configuration variable." ) warnings.warn(msg, RuntimeWarning, stacklevel=2) diff --git a/src/zarr/core/metadata/v3.py b/src/zarr/core/metadata/v3.py index bcbad174d2..4cf5860ffd 100644 --- a/src/zarr/core/metadata/v3.py +++ b/src/zarr/core/metadata/v3.py @@ -251,7 +251,7 @@ def __init__( shape=shape_parsed, dtype=data_type_parsed.to_numpy(), fill_value=fill_value_parsed, - config=ArrayConfig(), # TODO: config is not needed here. + config=ArrayConfig.from_dict({}), # TODO: config is not needed here. prototype=default_buffer_prototype(), # TODO: prototype is not needed here. ) codecs_parsed = [c.evolve_from_array_spec(array_spec) for c in codecs_parsed_partial] diff --git a/tests/test_array.py b/tests/test_array.py index 4ef29ad999..1899e384dc 100644 --- a/tests/test_array.py +++ b/tests/test_array.py @@ -748,14 +748,42 @@ def test_append_bad_shape(store: MemoryStore, zarr_format: ZarrFormat) -> None: @pytest.mark.parametrize("order", ["C", "F", None]) -@pytest.mark.parametrize("zarr_format", [2, 3]) @pytest.mark.parametrize("store", ["memory"], indirect=True) -def test_array_create_order( +def test_array_create_metadata_order_v2( order: MemoryOrder | None, zarr_format: int, store: MemoryStore ) -> None: - arr = Array.create(store=store, shape=(2, 2), order=order, zarr_format=zarr_format, dtype="i4") + """ + Test that the ``order`` attribute in zarr v2 array metadata is set correctly via the ``order`` + keyword argument to ``Array.create``. When ``order`` is ``None``, the value of the + ``array.order`` config is used. + """ + arr = Array.create(store=store, shape=(2, 2), order=order, zarr_format=2, dtype="i4") + expected = order or zarr.config.get("array.order") - assert arr.order == expected + assert arr.metadata.order == expected # type: ignore[union-attr] + + +@pytest.mark.parametrize("order_config", ["C", "F", None]) +@pytest.mark.parametrize("store", ["memory"], indirect=True) +def test_array_create_order( + order_config: MemoryOrder | None, + zarr_format: int, + store: MemoryStore, +) -> None: + """ + Test that the arrays generated by array indexing have a memory order defined by the config order + value + """ + if order_config is None: + config = {} + expected = zarr.config.get("array.order") + else: + config = {"order": order_config} + expected = order_config + + arr = Array.create( + store=store, shape=(2, 2), zarr_format=zarr_format, dtype="i4", config=config + ) vals = np.asarray(arr) if expected == "C": @@ -800,7 +828,7 @@ def test_write_empty_chunks_behavior( dtype="i4", fill_value=fill_value, chunk_shape=(1,), - write_empty_chunks=write_empty_chunks, + config={"write_empty_chunks": write_empty_chunks}, ) assert arr._async_array._config.write_empty_chunks == write_empty_chunks From df8f63103d884d71b83c71c6234beff9251dc759 Mon Sep 17 00:00:00 2001 From: Davis Vann Bennett Date: Fri, 20 Dec 2024 13:36:41 +0100 Subject: [PATCH 14/16] remove missing overload --- src/zarr/core/array.py | 27 +++++++++++++++++++++++++++ 1 file changed, 27 insertions(+) diff --git a/src/zarr/core/array.py b/src/zarr/core/array.py index 49611a7d2d..3e9d10652a 100644 --- a/src/zarr/core/array.py +++ b/src/zarr/core/array.py @@ -305,6 +305,33 @@ async def create( config: ArrayConfig | ArrayConfigParams | None = None, ) -> AsyncArray[ArrayV3Metadata]: ... + @overload + @classmethod + async def create( + cls, + store: StoreLike, + *, + # v2 and v3 + shape: ShapeLike, + dtype: npt.DTypeLike, + zarr_format: Literal[3] = 3, + fill_value: Any | None = None, + attributes: dict[str, JSON] | None = None, + # v3 only + chunk_shape: ShapeLike | None = None, + chunk_key_encoding: ( + ChunkKeyEncoding + | tuple[Literal["default"], Literal[".", "/"]] + | tuple[Literal["v2"], Literal[".", "/"]] + | None + ) = None, + codecs: Iterable[Codec | dict[str, JSON]] | None = None, + dimension_names: Iterable[str] | None = None, + # runtime + overwrite: bool = False, + data: npt.ArrayLike | None = None, + config: ArrayConfig | ArrayConfigParams | None = None, + ) -> AsyncArray[ArrayV3Metadata]: ... @overload @classmethod async def create( From 6a066a890c7d8c6bfa26d37e87cc175fcd53343a Mon Sep 17 00:00:00 2001 From: Davis Bennett Date: Fri, 20 Dec 2024 13:46:14 +0100 Subject: [PATCH 15/16] Update src/zarr/core/array.py Co-authored-by: Norman Rzepka --- src/zarr/core/array.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/zarr/core/array.py b/src/zarr/core/array.py index 3e9d10652a..0cdc019fb4 100644 --- a/src/zarr/core/array.py +++ b/src/zarr/core/array.py @@ -449,8 +449,8 @@ async def create( The memory of the array (default is "C"). If ``zarr_format`` is 2, this parameter sets the memory order of the array. If `zarr_format`` is 3, then this parameter is deprecated, because memory order - is a runtime parameter for zarr 3 arrays. The recommended way to specify the memory - order for zarr v3 arrays is via the ``config`` parameter, e.g. ``{'config': 'C'}``. + is a runtime parameter for Zarr 3 arrays. The recommended way to specify the memory + order for Zarr 3 arrays is via the ``config`` parameter, e.g. ``{'config': 'C'}``. filters : list[dict[str, JSON]], optional Sequence of filters to use to encode chunk data prior to compression. V2 only. V3 arrays should use ``codecs`` instead. If neither ``compressor`` From 8a05f7fa9e17dba83efd932aabf534d7b882cada Mon Sep 17 00:00:00 2001 From: Davis Bennett Date: Fri, 20 Dec 2024 13:46:19 +0100 Subject: [PATCH 16/16] Update src/zarr/core/array.py Co-authored-by: Norman Rzepka --- src/zarr/core/array.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/zarr/core/array.py b/src/zarr/core/array.py index 0cdc019fb4..717eff36dc 100644 --- a/src/zarr/core/array.py +++ b/src/zarr/core/array.py @@ -1583,8 +1583,8 @@ def create( The memory of the array (default is "C"). If ``zarr_format`` is 2, this parameter sets the memory order of the array. If `zarr_format`` is 3, then this parameter is deprecated, because memory order - is a runtime parameter for zarr 3 arrays. The recommended way to specify the memory - order for zarr v3 arrays is via the ``config`` parameter, e.g. ``{'order': 'C'}``. + is a runtime parameter for Zarr 3 arrays. The recommended way to specify the memory + order for Zarr 3 arrays is via the ``config`` parameter, e.g. ``{'order': 'C'}``. filters : list[dict[str, JSON]], optional Sequence of filters to use to encode chunk data prior to compression. V2 only. V3 arrays should use ``codecs`` instead. If neither ``compressor``