diff --git a/src/zarr/api/asynchronous.py b/src/zarr/api/asynchronous.py index 285d777258..e63f9f113f 100644 --- a/src/zarr/api/asynchronous.py +++ b/src/zarr/api/asynchronous.py @@ -3,13 +3,13 @@ import asyncio import dataclasses import warnings -from typing import TYPE_CHECKING, Any, Literal, cast +from typing import TYPE_CHECKING, Any, Literal import numpy as np import numpy.typing as npt from typing_extensions import deprecated -from zarr.core.array import Array, AsyncArray, create_array, from_array, get_array_metadata +from zarr.core.array import Array, AsyncArray, create_array, from_array from zarr.core.array_spec import ArrayConfig, ArrayConfigLike, ArrayConfigParams from zarr.core.buffer import NDArrayLike from zarr.core.common import ( @@ -25,13 +25,19 @@ ) from zarr.core.group import ( AsyncGroup, - ConsolidatedMetadata, - GroupMetadata, create_hierarchy, + get_node, ) -from zarr.core.metadata import ArrayMetadataDict, ArrayV2Metadata, ArrayV3Metadata +from zarr.core.metadata import ArrayV2Metadata, ArrayV3Metadata +from zarr.core.metadata.group import ConsolidatedMetadata, GroupMetadata from zarr.core.metadata.v2 import _default_compressor, _default_filters -from zarr.errors import NodeTypeValidationError +from zarr.errors import ( + ArrayNotFoundError, + ContainsArrayError, + ContainsGroupError, + GroupNotFoundError, + NodeNotFoundError, +) from zarr.storage._common import make_store_path if TYPE_CHECKING: @@ -316,27 +322,48 @@ async def open( store_path = await make_store_path(store, mode=mode, path=path, storage_options=storage_options) - # TODO: the mode check below seems wrong! - if "shape" not in kwargs and mode in {"a", "r", "r+", "w"}: + extant_node: AsyncGroup | AsyncArray[Any] = None + # All of these modes will defer to an existing mode + if mode in {"a", "r", "r+", "w-"}: try: - metadata_dict = await get_array_metadata(store_path, zarr_format=zarr_format) - # TODO: remove this cast when we fix typing for array metadata dicts - _metadata_dict = cast(ArrayMetadataDict, metadata_dict) - # for v2, the above would already have raised an exception if not an array - zarr_format = _metadata_dict["zarr_format"] - is_v3_array = zarr_format == 3 and _metadata_dict.get("node_type") == "array" - if is_v3_array or zarr_format == 2: - return AsyncArray(store_path=store_path, metadata=_metadata_dict) - except (AssertionError, FileNotFoundError, NodeTypeValidationError): + extant_node = await get_node( + store=store_path.store, path=store_path.path, zarr_format=zarr_format + ) + except NodeNotFoundError: pass - return await open_group(store=store_path, zarr_format=zarr_format, mode=mode, **kwargs) - - try: + # we successfully found an existing node + if extant_node is not None: + # an existing node is an error if mode == w- + if mode == "w-": + if isinstance(extant_node, AsyncArray): + node_type = "array" + exc = ContainsArrayError + else: + node_type = "group" + exc = ContainsGroupError + msg = ( + f"A Zarr V{extant_node.zarr_format} {node_type} exists in store " + f"{store_path.store!r} at path {store_path.path!r}. " + f"Attempting to open a pre-existing {node_type} with access mode {mode} is an error. " + f"Remove the {node_type} from storage, or use an access mode that is compatible with " + "a pre-existing array, such as one of ('r','r+','a','w')." + ) + raise exc(msg) + else: + # otherwise, return the existing node + return extant_node + else: + if mode in ("r", "r+"): + msg = ( + f"Neither array nor group metadata were found in store {store_path.store!r} at " + f"path {store_path.path!r}. Attempting to open an non-existent node with access mode " + f"{mode} is an error. Create an array or group first, or use an access mode that " + "create an array or group, such as one of ('a', 'w', 'w-')." + ) + raise NodeNotFoundError(msg) + if "shape" in kwargs: return await open_array(store=store_path, zarr_format=zarr_format, mode=mode, **kwargs) - except (KeyError, NodeTypeValidationError): - # KeyError for a missing key - # NodeTypeValidationError for failing to parse node metadata as an array when it's - # actually a group + else: return await open_group(store=store_path, zarr_format=zarr_format, mode=mode, **kwargs) @@ -660,7 +687,7 @@ async def group( try: return await AsyncGroup.open(store=store_path, zarr_format=zarr_format) - except (KeyError, FileNotFoundError): + except (KeyError, GroupNotFoundError): _zarr_format = zarr_format or _default_zarr_format() return await AsyncGroup.from_store( store=store_path, @@ -818,7 +845,7 @@ async def open_group( return await AsyncGroup.open( store_path, zarr_format=zarr_format, use_consolidated=use_consolidated ) - except (KeyError, FileNotFoundError): + except GroupNotFoundError: pass if mode in _CREATE_MODES: overwrite = _infer_overwrite(mode) @@ -829,7 +856,8 @@ async def open_group( overwrite=overwrite, attributes=attributes, ) - raise FileNotFoundError(f"Unable to find group: {store_path}") + msg = f"Group metadata was not found in store {store_path.store!r} at path {store_path.path!r}." + raise GroupNotFoundError(msg) async def create( @@ -1259,7 +1287,7 @@ async def open_array( try: return await AsyncArray.open(store_path, zarr_format=zarr_format) - except FileNotFoundError: + except ArrayNotFoundError: if not store_path.read_only and mode in _CREATE_MODES: overwrite = _infer_overwrite(mode) _zarr_format = zarr_format or _default_zarr_format() diff --git a/src/zarr/core/array.py b/src/zarr/core/array.py index 62efe44e4c..329518470a 100644 --- a/src/zarr/core/array.py +++ b/src/zarr/core/array.py @@ -100,6 +100,8 @@ ArrayV3MetadataDict, T_ArrayMetadata, ) +from zarr.core.metadata._io import _read_array_metadata +from zarr.core.metadata.group import GroupMetadata from zarr.core.metadata.v2 import ( _default_compressor, _default_filters, @@ -108,7 +110,7 @@ ) from zarr.core.metadata.v3 import DataType, parse_node_type_array from zarr.core.sync import sync -from zarr.errors import MetadataValidationError +from zarr.errors import ArrayNotFoundError from zarr.registry import ( _parse_array_array_codec, _parse_array_bytes_codec, @@ -161,7 +163,7 @@ def create_codec_pipeline(metadata: ArrayMetadata) -> CodecPipeline: raise TypeError -async def get_array_metadata( +async def xget_array_metadata( store_path: StorePath, zarr_format: ZarrFormat | None = 3 ) -> dict[str, JSON]: if zarr_format == 2: @@ -170,11 +172,19 @@ async def get_array_metadata( (store_path / ZATTRS_JSON).get(prototype=cpu_buffer_prototype), ) if zarray_bytes is None: - raise FileNotFoundError(store_path) + msg = ( + "A Zarr V2 array metadata document was not found in store " + f"{store_path.store!r} at path {store_path.path!r}." + ) + raise ArrayNotFoundError(msg) elif zarr_format == 3: zarr_json_bytes = await (store_path / ZARR_JSON).get(prototype=cpu_buffer_prototype) if zarr_json_bytes is None: - raise FileNotFoundError(store_path) + msg = ( + "A Zarr V3 array metadata document was not found in store " + f"{store_path.store!r} at path {store_path.path!r}." + ) + raise ArrayNotFoundError(msg) elif zarr_format is None: zarr_json_bytes, zarray_bytes, zattrs_bytes = await gather( (store_path / ZARR_JSON).get(prototype=cpu_buffer_prototype), @@ -183,17 +193,27 @@ async def get_array_metadata( ) if zarr_json_bytes is not None and zarray_bytes is not None: # warn and favor v3 - msg = f"Both zarr.json (Zarr format 3) and .zarray (Zarr format 2) metadata objects exist at {store_path}. Zarr v3 will be used." + msg = ( + "Both Zarr V3 Zarr V2 metadata documents " + f"were found in store {store_path.store!r} at path {store_path.path!r}. " + "The Zarr V3 metadata will be used." + "To open Zarr V2 arrays, set zarr_format=2." + ) warnings.warn(msg, stacklevel=1) if zarr_json_bytes is None and zarray_bytes is None: - raise FileNotFoundError(store_path) + msg = ( + f"Neither Zarr V3 nor Zarr V2 array metadata documents " + f"were found in store {store_path.store!r} at path {store_path.path!r}." + ) + raise ArrayNotFoundError(msg) # set zarr_format based on which keys were found if zarr_json_bytes is not None: zarr_format = 3 else: zarr_format = 2 else: - raise MetadataValidationError("zarr_format", "2, 3, or None", zarr_format) + msg = f"Invalid value for zarr_format. Expected one of 2, 3, or None. Got {zarr_format}." # type: ignore[unreachable] + raise ValueError(msg) metadata_dict: dict[str, JSON] if zarr_format == 2: @@ -895,10 +915,10 @@ async def open( """ store_path = await make_store_path(store) - metadata_dict = await get_array_metadata(store_path, zarr_format=zarr_format) - # TODO: remove this cast when we have better type hints - _metadata_dict = cast(ArrayV3MetadataDict, metadata_dict) - return cls(store_path=store_path, metadata=_metadata_dict) + metadata_dict = await _read_array_metadata( + store_path.store, store_path.path, zarr_format=zarr_format + ) + return cls(store_path=store_path, metadata=metadata_dict) @property def store(self) -> Store: @@ -3738,7 +3758,7 @@ async def chunks_initialized( def _build_parents( node: AsyncArray[ArrayV2Metadata] | AsyncArray[ArrayV3Metadata] | AsyncGroup, ) -> list[AsyncGroup]: - from zarr.core.group import AsyncGroup, GroupMetadata + from zarr.core.group import AsyncGroup store = node.store_path.store path = node.store_path.path diff --git a/src/zarr/core/group.py b/src/zarr/core/group.py index 3f8dad1740..d153476b3a 100644 --- a/src/zarr/core/group.py +++ b/src/zarr/core/group.py @@ -1,15 +1,14 @@ from __future__ import annotations import asyncio -import itertools import json import logging import warnings from collections import defaultdict from collections.abc import Iterator, Mapping -from dataclasses import asdict, dataclass, field, fields, replace +from dataclasses import dataclass, replace from itertools import accumulate -from typing import TYPE_CHECKING, Literal, TypeVar, assert_never, cast, overload +from typing import TYPE_CHECKING, Literal, TypeVar, cast, overload import numpy as np import numpy.typing as npt @@ -17,7 +16,6 @@ import zarr.api.asynchronous as async_api from zarr._compat import _deprecate_positional_args -from zarr.abc.metadata import Metadata from zarr.abc.store import Store, set_or_delete from zarr.core._info import GroupInfo from zarr.core.array import ( @@ -42,16 +40,26 @@ ZGROUP_JSON, ZMETADATA_V2_JSON, ChunkCoords, - NodeType, ShapeLike, ZarrFormat, parse_shapelike, ) from zarr.core.config import config from zarr.core.metadata import ArrayV2Metadata, ArrayV3Metadata -from zarr.core.metadata.v3 import V3JsonEncoder, _replace_special_floats +from zarr.core.metadata._io import ( + _persist_metadata, + _read_group_metadata, + _read_metadata_v2, + _read_metadata_v3, +) +from zarr.core.metadata.group import GroupMetadata, ImplicitGroupMarker from zarr.core.sync import SyncMixin, sync -from zarr.errors import ContainsArrayError, ContainsGroupError, MetadataValidationError +from zarr.errors import ( + ContainsArrayError, + ContainsGroupError, + GroupNotFoundError, + NodeNotFoundError, +) from zarr.storage import StoreLike, StorePath from zarr.storage._common import ensure_no_existing_node, make_store_path from zarr.storage._utils import _join_paths, _normalize_path_keys, normalize_path @@ -67,7 +75,7 @@ from typing import Any from zarr.core.array_spec import ArrayConfig, ArrayConfigLike - from zarr.core.buffer import Buffer, BufferPrototype + from zarr.core.buffer import Buffer from zarr.core.chunk_key_encodings import ChunkKeyEncodingLike from zarr.core.common import MemoryOrder @@ -76,21 +84,6 @@ DefaultT = TypeVar("DefaultT") -def parse_zarr_format(data: Any) -> ZarrFormat: - """Parse the zarr_format field from metadata.""" - if data in (2, 3): - return cast(ZarrFormat, data) - msg = f"Invalid zarr_format. Expected one of 2 or 3. Got {data}." - raise ValueError(msg) - - -def parse_node_type(data: Any) -> NodeType: - """Parse the node_type field from metadata.""" - if data in ("array", "group"): - return cast(Literal["array", "group"], data) - raise MetadataValidationError("node_type", "array or group", data) - - # todo: convert None to empty dict def parse_attributes(data: Any) -> dict[str, Any]: """Parse the attributes field from metadata.""" @@ -122,309 +115,6 @@ def _parse_async_node( raise TypeError(f"Unknown node type, got {type(node)}") -@dataclass(frozen=True) -class ConsolidatedMetadata: - """ - Consolidated Metadata for this Group. - - This stores the metadata of child nodes below this group. Any child groups - will have their consolidated metadata set appropriately. - """ - - metadata: dict[str, ArrayV2Metadata | ArrayV3Metadata | GroupMetadata] - kind: Literal["inline"] = "inline" - must_understand: Literal[False] = False - - def to_dict(self) -> dict[str, JSON]: - return { - "kind": self.kind, - "must_understand": self.must_understand, - "metadata": {k: v.to_dict() for k, v in self.flattened_metadata.items()}, - } - - @classmethod - def from_dict(cls, data: dict[str, JSON]) -> ConsolidatedMetadata: - data = dict(data) - - kind = data.get("kind") - if kind != "inline": - raise ValueError(f"Consolidated metadata kind='{kind}' is not supported.") - - raw_metadata = data.get("metadata") - if not isinstance(raw_metadata, dict): - raise TypeError(f"Unexpected type for 'metadata': {type(raw_metadata)}") - - metadata: dict[str, ArrayV2Metadata | ArrayV3Metadata | GroupMetadata] = {} - if raw_metadata: - for k, v in raw_metadata.items(): - if not isinstance(v, dict): - raise TypeError( - f"Invalid value for metadata items. key='{k}', type='{type(v).__name__}'" - ) - - # zarr_format is present in v2 and v3. - zarr_format = parse_zarr_format(v["zarr_format"]) - - if zarr_format == 3: - node_type = parse_node_type(v.get("node_type", None)) - if node_type == "group": - metadata[k] = GroupMetadata.from_dict(v) - elif node_type == "array": - metadata[k] = ArrayV3Metadata.from_dict(v) - else: - assert_never(node_type) - elif zarr_format == 2: - if "shape" in v: - metadata[k] = ArrayV2Metadata.from_dict(v) - else: - metadata[k] = GroupMetadata.from_dict(v) - else: - assert_never(zarr_format) - - cls._flat_to_nested(metadata) - - return cls(metadata=metadata) - - @staticmethod - def _flat_to_nested( - metadata: dict[str, ArrayV2Metadata | ArrayV3Metadata | GroupMetadata], - ) -> None: - """ - Convert a flat metadata representation to a nested one. - - Notes - ----- - Flat metadata is used when persisting the consolidated metadata. The keys - include the full path, not just the node name. The key prefixes can be - used to determine which nodes are children of which other nodes. - - Nested metadata is used in-memory. The outermost level will only have the - *immediate* children of the Group. All nested child groups will be stored - under the consolidated metadata of their immediate parent. - """ - # We have a flat mapping from {k: v} where the keys include the *full* - # path segment: - # { - # "/a/b": { group_metadata }, - # "/a/b/array-0": { array_metadata }, - # "/a/b/array-1": { array_metadata }, - # } - # - # We want to reorganize the metadata such that each Group contains the - # array metadata of its immediate children. - # In the example, the group at `/a/b` will have consolidated metadata - # for its children `array-0` and `array-1`. - # - # metadata = dict(metadata) - - keys = sorted(metadata, key=lambda k: k.count("/")) - grouped = { - k: list(v) for k, v in itertools.groupby(keys, key=lambda k: k.rsplit("/", 1)[0]) - } - - # we go top down and directly manipulate metadata. - for key, children_keys in grouped.items(): - # key is a key like "a", "a/b", "a/b/c" - # The basic idea is to find the immediate parent (so "", "a", or "a/b") - # and update that node's consolidated metadata to include the metadata - # in children_keys - *prefixes, name = key.split("/") - parent = metadata - - while prefixes: - # e.g. a/b/c has a parent "a/b". Walk through to get - # metadata["a"]["b"] - part = prefixes.pop(0) - # we can assume that parent[part] here is a group - # otherwise we wouldn't have a node with this `part` prefix. - # We can also assume that the parent node will have consolidated metadata, - # because we're walking top to bottom. - parent = parent[part].consolidated_metadata.metadata # type: ignore[union-attr] - - node = parent[name] - children_keys = list(children_keys) - - if isinstance(node, ArrayV2Metadata | ArrayV3Metadata): - # These are already present, either thanks to being an array in the - # root, or by being collected as a child in the else clause - continue - children_keys = list(children_keys) - # We pop from metadata, since we're *moving* this under group - children = { - child_key.split("/")[-1]: metadata.pop(child_key) - for child_key in children_keys - if child_key != key - } - parent[name] = replace( - node, consolidated_metadata=ConsolidatedMetadata(metadata=children) - ) - - @property - def flattened_metadata(self) -> dict[str, ArrayV2Metadata | ArrayV3Metadata | GroupMetadata]: - """ - Return the flattened representation of Consolidated Metadata. - - The returned dictionary will have a key for each child node in the hierarchy - under this group. Under the default (nested) representation available through - ``self.metadata``, the dictionary only contains keys for immediate children. - - The keys of the dictionary will include the full path to a child node from - the current group, where segments are joined by ``/``. - - Examples - -------- - >>> cm = ConsolidatedMetadata( - ... metadata={ - ... "group-0": GroupMetadata( - ... consolidated_metadata=ConsolidatedMetadata( - ... { - ... "group-0-0": GroupMetadata(), - ... } - ... ) - ... ), - ... "group-1": GroupMetadata(), - ... } - ... ) - {'group-0': GroupMetadata(attributes={}, zarr_format=3, consolidated_metadata=None, node_type='group'), - 'group-0/group-0-0': GroupMetadata(attributes={}, zarr_format=3, consolidated_metadata=None, node_type='group'), - 'group-1': GroupMetadata(attributes={}, zarr_format=3, consolidated_metadata=None, node_type='group')} - """ - metadata = {} - - def flatten( - key: str, group: GroupMetadata | ArrayV2Metadata | ArrayV3Metadata - ) -> dict[str, ArrayV2Metadata | ArrayV3Metadata | GroupMetadata]: - children: dict[str, ArrayV2Metadata | ArrayV3Metadata | GroupMetadata] = {} - if isinstance(group, ArrayV2Metadata | ArrayV3Metadata): - children[key] = group - else: - if group.consolidated_metadata and group.consolidated_metadata.metadata is not None: - children[key] = replace( - group, consolidated_metadata=ConsolidatedMetadata(metadata={}) - ) - for name, val in group.consolidated_metadata.metadata.items(): - full_key = f"{key}/{name}" - if isinstance(val, GroupMetadata): - children.update(flatten(full_key, val)) - else: - children[full_key] = val - else: - children[key] = replace(group, consolidated_metadata=None) - return children - - for k, v in self.metadata.items(): - metadata.update(flatten(k, v)) - - return metadata - - -@dataclass(frozen=True) -class GroupMetadata(Metadata): - """ - Metadata for a Group. - """ - - attributes: dict[str, Any] = field(default_factory=dict) - zarr_format: ZarrFormat = 3 - consolidated_metadata: ConsolidatedMetadata | None = None - node_type: Literal["group"] = field(default="group", init=False) - - def to_buffer_dict(self, prototype: BufferPrototype) -> dict[str, Buffer]: - json_indent = config.get("json_indent") - if self.zarr_format == 3: - return { - ZARR_JSON: prototype.buffer.from_bytes( - json.dumps(_replace_special_floats(self.to_dict()), cls=V3JsonEncoder).encode() - ) - } - else: - items = { - ZGROUP_JSON: prototype.buffer.from_bytes( - json.dumps({"zarr_format": self.zarr_format}, indent=json_indent).encode() - ), - ZATTRS_JSON: prototype.buffer.from_bytes( - json.dumps(self.attributes, indent=json_indent).encode() - ), - } - if self.consolidated_metadata: - d = { - ZGROUP_JSON: {"zarr_format": self.zarr_format}, - ZATTRS_JSON: self.attributes, - } - consolidated_metadata = self.consolidated_metadata.to_dict()["metadata"] - assert isinstance(consolidated_metadata, dict) - for k, v in consolidated_metadata.items(): - attrs = v.pop("attributes", None) - d[f"{k}/{ZATTRS_JSON}"] = _replace_special_floats(attrs) - if "shape" in v: - # it's an array - d[f"{k}/{ZARRAY_JSON}"] = _replace_special_floats(v) - else: - d[f"{k}/{ZGROUP_JSON}"] = { - "zarr_format": self.zarr_format, - "consolidated_metadata": { - "metadata": {}, - "must_understand": False, - "kind": "inline", - }, - } - - items[ZMETADATA_V2_JSON] = prototype.buffer.from_bytes( - json.dumps( - {"metadata": d, "zarr_consolidated_format": 1}, - cls=V3JsonEncoder, - ).encode() - ) - - return items - - def __init__( - self, - attributes: dict[str, Any] | None = None, - zarr_format: ZarrFormat = 3, - consolidated_metadata: ConsolidatedMetadata | None = None, - ) -> None: - attributes_parsed = parse_attributes(attributes) - zarr_format_parsed = parse_zarr_format(zarr_format) - - object.__setattr__(self, "attributes", attributes_parsed) - object.__setattr__(self, "zarr_format", zarr_format_parsed) - object.__setattr__(self, "consolidated_metadata", consolidated_metadata) - - @classmethod - def from_dict(cls, data: dict[str, Any]) -> GroupMetadata: - data = dict(data) - assert data.pop("node_type", None) in ("group", None) - consolidated_metadata = data.pop("consolidated_metadata", None) - if consolidated_metadata: - data["consolidated_metadata"] = ConsolidatedMetadata.from_dict(consolidated_metadata) - - zarr_format = data.get("zarr_format") - if zarr_format == 2 or zarr_format is None: - # zarr v2 allowed arbitrary keys here. - # We don't want the GroupMetadata constructor to fail just because someone put an - # extra key in the metadata. - expected = {x.name for x in fields(cls)} - data = {k: v for k, v in data.items() if k in expected} - - return cls(**data) - - def to_dict(self) -> dict[str, Any]: - result = asdict(replace(self, consolidated_metadata=None)) - if self.consolidated_metadata: - result["consolidated_metadata"] = self.consolidated_metadata.to_dict() - return result - - -@dataclass(frozen=True) -class ImplicitGroupMarker(GroupMetadata): - """ - Marker for an implicit group. Instances of this class are only used in the context of group - creation as a placeholder to represent groups that should only be created if they do not - already exist in storage - """ - - @dataclass(frozen=True) class AsyncGroup: """ @@ -509,7 +199,11 @@ async def open( *[path.get() for path in paths] ) if zgroup_bytes is None: - raise FileNotFoundError(store_path) + msg = ( + "Zarr V2 group metadata was not found in " + f"store {store_path.store!r} at path {store_path.path!r}." + ) + raise GroupNotFoundError(msg) if use_consolidated or use_consolidated is None: maybe_consolidated_metadata_bytes = rest[0] @@ -520,7 +214,11 @@ async def open( elif zarr_format == 3: zarr_json_bytes = await (store_path / ZARR_JSON).get() if zarr_json_bytes is None: - raise FileNotFoundError(store_path) + msg = ( + "Zarr V3 group metadata was not found in store " + f"{store_path.store!r} at path {store_path.path!r}." + ) + raise GroupNotFoundError(msg) elif zarr_format is None: ( zarr_json_bytes, @@ -535,19 +233,29 @@ async def open( ) if zarr_json_bytes is not None and zgroup_bytes is not None: # warn and favor v3 - msg = f"Both zarr.json (Zarr format 3) and .zgroup (Zarr format 2) metadata objects exist at {store_path}. Zarr format 3 will be used." + msg = ( + "Both zarr.json (Zarr format 3) and .zarray (Zarr format 2) metadata objects " + f"exist in store {store_path.store!r} at path {store_path.path!r}. " + "The Zarr V3 metadata will be used." + "To open Zarr V2 groups, set zarr_format=2." + ) warnings.warn(msg, stacklevel=1) if zarr_json_bytes is None and zgroup_bytes is None: - raise FileNotFoundError( - f"could not find zarr.json or .zgroup objects in {store_path}" + msg = ( + f"Neither Zarr V2 nor Zarr V3 Group metadata was not found in store " + f"{store_path.store!r} at path {store_path.path!r}." ) + raise GroupNotFoundError(msg) # set zarr_format based on which keys were found if zarr_json_bytes is not None: zarr_format = 3 else: zarr_format = 2 else: - raise MetadataValidationError("zarr_format", "2, 3, or None", zarr_format) + msg = ( # type: ignore[unreachable] + f"Invalid value for zarr_format. Expected one of 2, 3, or None. Got {zarr_format}." + ) + raise ValueError(msg) if zarr_format == 2: # this is checked above, asserting here for mypy @@ -569,7 +277,11 @@ async def open( # V3 groups are comprised of a zarr.json object assert zarr_json_bytes is not None if not isinstance(use_consolidated, bool | None): - raise TypeError("use_consolidated must be a bool or None for Zarr format 3.") + msg = ( + "use_consolidated must be a bool or None for Zarr format 3. " + f"Got {use_consolidated}" + ) + raise TypeError(msg) return cls._from_bytes_v3( store_path, @@ -628,7 +340,7 @@ def _from_bytes_v3( ) -> AsyncGroup: group_metadata = json.loads(zarr_json_bytes.to_bytes()) if use_consolidated and group_metadata.get("consolidated_metadata") is None: - msg = f"Consolidated metadata requested with 'use_consolidated=True' but not found in '{store_path.path}'." + msg = f"Consolidated metadata requested with 'use_consolidated=True' but not found in '{store_path.path!r}'." raise ValueError(msg) elif use_consolidated is False: @@ -691,7 +403,7 @@ async def getitem( return await get_node( store=store_path.store, path=store_path.path, zarr_format=self.metadata.zarr_format ) - except FileNotFoundError as e: + except NodeNotFoundError as e: raise KeyError(key) from e def _getitem_consolidated( @@ -2981,7 +2693,7 @@ async def create_hierarchy( for key, value in zip(implicit_group_keys, maybe_extant_groups, strict=True): if isinstance(value, BaseException): - if isinstance(value, FileNotFoundError): + if isinstance(value, GroupNotFoundError): # this is fine -- there was no group there, so we will create one pass else: @@ -3025,8 +2737,9 @@ async def create_hierarchy( for key, extant_node in extant_node_query.items(): proposed_node = nodes_parsed[key] if isinstance(extant_node, BaseException): - if isinstance(extant_node, FileNotFoundError): - # ignore FileNotFoundError, because they represent nodes we can safely create + if isinstance(extant_node, NodeNotFoundError): + # ignore NodeNotFoundErrpr. + # it represents a node we can safely create. pass else: # Any other exception is a real error @@ -3043,10 +2756,12 @@ async def create_hierarchy( else: # we have proposed an explicit group, which is an error, given that a # group already exists. - raise ContainsGroupError(store, key) + msg = f"A group exists in store {store!r} at path {key!r}." + raise ContainsGroupError(msg) elif isinstance(extant_node, ArrayV2Metadata | ArrayV3Metadata): # we are trying to overwrite an existing array. this is an error. - raise ContainsArrayError(store, key) + msg = f"An array exists in store {store!r} at path {key!r}." + raise ContainsArrayError(msg) nodes_explicit: dict[str, GroupMetadata | ArrayV2Metadata | ArrayV3Metadata] = {} @@ -3209,7 +2924,8 @@ def _parse_hierarchy_dict( else: if not isinstance(out[subpath], GroupMetadata | ImplicitGroupMarker): msg = ( - f"The node at {subpath} contains other nodes, but it is not a Zarr group. " + f"The node at path {subpath!r} contains other nodes, " + "but it is not a Zarr group. " "This is invalid. Only Zarr groups can contain other nodes." ) raise ValueError(msg) @@ -3364,114 +3080,6 @@ async def _iter_members_deep( yield key, node -async def _read_metadata_v3(store: Store, path: str) -> ArrayV3Metadata | GroupMetadata: - """ - Given a store_path, return ArrayV3Metadata or GroupMetadata defined by the metadata - document stored at store_path.path / zarr.json. If no such document is found, raise a - FileNotFoundError. - """ - zarr_json_bytes = await store.get( - _join_paths([path, ZARR_JSON]), prototype=default_buffer_prototype() - ) - if zarr_json_bytes is None: - raise FileNotFoundError(path) - else: - zarr_json = json.loads(zarr_json_bytes.to_bytes()) - return _build_metadata_v3(zarr_json) - - -async def _read_metadata_v2(store: Store, path: str) -> ArrayV2Metadata | GroupMetadata: - """ - Given a store_path, return ArrayV2Metadata or GroupMetadata defined by the metadata - document stored at store_path.path / (.zgroup | .zarray). If no such document is found, - raise a FileNotFoundError. - """ - # TODO: consider first fetching array metadata, and only fetching group metadata when we don't - # find an array - zarray_bytes, zgroup_bytes, zattrs_bytes = await asyncio.gather( - store.get(_join_paths([path, ZARRAY_JSON]), prototype=default_buffer_prototype()), - store.get(_join_paths([path, ZGROUP_JSON]), prototype=default_buffer_prototype()), - store.get(_join_paths([path, ZATTRS_JSON]), prototype=default_buffer_prototype()), - ) - - if zattrs_bytes is None: - zattrs = {} - else: - zattrs = json.loads(zattrs_bytes.to_bytes()) - - # TODO: decide how to handle finding both array and group metadata. The spec does not seem to - # consider this situation. A practical approach would be to ignore that combination, and only - # return the array metadata. - if zarray_bytes is not None: - zmeta = json.loads(zarray_bytes.to_bytes()) - else: - if zgroup_bytes is None: - # neither .zarray or .zgroup were found results in KeyError - raise FileNotFoundError(path) - else: - zmeta = json.loads(zgroup_bytes.to_bytes()) - - return _build_metadata_v2(zmeta, zattrs) - - -async def _read_group_metadata_v2(store: Store, path: str) -> GroupMetadata: - """ - Read group metadata or error - """ - meta = await _read_metadata_v2(store=store, path=path) - if not isinstance(meta, GroupMetadata): - raise FileNotFoundError(f"Group metadata was not found in {store} at {path}") - return meta - - -async def _read_group_metadata_v3(store: Store, path: str) -> GroupMetadata: - """ - Read group metadata or error - """ - meta = await _read_metadata_v3(store=store, path=path) - if not isinstance(meta, GroupMetadata): - raise FileNotFoundError(f"Group metadata was not found in {store} at {path}") - return meta - - -async def _read_group_metadata( - store: Store, path: str, *, zarr_format: ZarrFormat -) -> GroupMetadata: - if zarr_format == 2: - return await _read_group_metadata_v2(store=store, path=path) - return await _read_group_metadata_v3(store=store, path=path) - - -def _build_metadata_v3(zarr_json: dict[str, JSON]) -> ArrayV3Metadata | GroupMetadata: - """ - Convert a dict representation of Zarr V3 metadata into the corresponding metadata class. - """ - if "node_type" not in zarr_json: - raise MetadataValidationError("node_type", "array or group", "nothing (the key is missing)") - match zarr_json: - case {"node_type": "array"}: - return ArrayV3Metadata.from_dict(zarr_json) - case {"node_type": "group"}: - return GroupMetadata.from_dict(zarr_json) - case _: # pragma: no cover - raise ValueError( - "invalid value for `node_type` key in metadata document" - ) # pragma: no cover - - -def _build_metadata_v2( - zarr_json: dict[str, JSON], attrs_json: dict[str, JSON] -) -> ArrayV2Metadata | GroupMetadata: - """ - Convert a dict representation of Zarr V2 metadata into the corresponding metadata class. - """ - match zarr_json: - case {"shape": _}: - return ArrayV2Metadata.from_dict(zarr_json | {"attributes": attrs_json}) - case _: # pragma: no cover - return GroupMetadata.from_dict(zarr_json | {"attributes": attrs_json}) - - @overload def _build_node( *, store: Store, path: str, metadata: ArrayV2Metadata @@ -3543,7 +3151,7 @@ async def _get_node_v3(store: Store, path: str) -> AsyncArray[ArrayV3Metadata] | async def get_node( - store: Store, path: str, zarr_format: ZarrFormat + store: Store, path: str, zarr_format: ZarrFormat | None ) -> AsyncArray[ArrayV2Metadata] | AsyncArray[ArrayV3Metadata] | AsyncGroup: """ Get an AsyncArray or AsyncGroup from a path in a Store. @@ -3554,8 +3162,8 @@ async def get_node( The store-like object to read from. path : str The path to the node to read. - zarr_format : {2, 3} - The zarr format of the node to read. + zarr_format : {2, 3} | None + The zarr format of the node to read. If ``None``, then V3 will be tried first, then V2. Returns ------- @@ -3567,56 +3175,22 @@ async def get_node( return await _get_node_v2(store=store, path=path) case 3: return await _get_node_v3(store=store, path=path) + case None: + try: + return await _get_node_v3(store=store, path=path) + except NodeNotFoundError: + try: + return await _get_node_v2(store=store, path=path) + except NodeNotFoundError as e: + msg = ( + "No Zarr V2 or V3 metadata documents were found " + "in store {store!r} at path {path!r}" + ) + raise NodeNotFoundError(msg) from e case _: # pragma: no cover raise ValueError(f"Unexpected zarr format: {zarr_format}") # pragma: no cover -async def _set_return_key( - *, store: Store, key: str, value: Buffer, semaphore: asyncio.Semaphore | None = None -) -> str: - """ - Write a value to storage at the given key. The key is returned. - Useful when saving values via routines that return results in execution order, - like asyncio.as_completed, because in this case we need to know which key was saved in order - to yield the right object to the caller. - - Parameters - ---------- - store : Store - The store to save the value to. - key : str - The key to save the value to. - value : Buffer - The value to save. - semaphore : asyncio.Semaphore | None - An optional semaphore to use to limit the number of concurrent writes. - """ - - if semaphore is not None: - async with semaphore: - await store.set(key, value) - else: - await store.set(key, value) - return key - - -def _persist_metadata( - store: Store, - path: str, - metadata: ArrayV2Metadata | ArrayV3Metadata | GroupMetadata, - semaphore: asyncio.Semaphore | None = None, -) -> tuple[Coroutine[None, None, str], ...]: - """ - Prepare to save a metadata document to storage, returning a tuple of coroutines that must be awaited. - """ - - to_save = metadata.to_buffer_dict(default_buffer_prototype()) - return tuple( - _set_return_key(store=store, key=_join_paths([path, key]), value=value, semaphore=semaphore) - for key, value in to_save.items() - ) - - async def create_rooted_hierarchy( *, store: Store, diff --git a/src/zarr/core/metadata/_io/__init__.py b/src/zarr/core/metadata/_io/__init__.py new file mode 100644 index 0000000000..d8be39f326 --- /dev/null +++ b/src/zarr/core/metadata/_io/__init__.py @@ -0,0 +1,272 @@ +from __future__ import annotations + +import asyncio +import json +from typing import TYPE_CHECKING, Literal, overload + +from zarr.core.buffer import default_buffer_prototype +from zarr.core.common import JSON, ZARR_JSON, ZARRAY_JSON, ZATTRS_JSON, ZGROUP_JSON, ZarrFormat +from zarr.core.metadata import ArrayV2Metadata, ArrayV3Metadata +from zarr.core.metadata.group import GroupMetadata +from zarr.errors import ( + ArrayNotFoundError, + GroupNotFoundError, + MetadataValidationError, + NodeNotFoundError, +) +from zarr.storage._utils import _join_paths, _set_return_key + +if TYPE_CHECKING: + from collections.abc import Coroutine + + from zarr.abc.store import Store + + +@overload +async def _read_array_metadata( + store: Store, path: str, zarr_format: Literal[2] +) -> ArrayV2Metadata: ... + + +@overload +async def _read_array_metadata( + store: Store, path: str, zarr_format: Literal[3] +) -> ArrayV3Metadata: ... + + +@overload +async def _read_array_metadata( + store: Store, path: str, zarr_format: None +) -> ArrayV2Metadata | ArrayV3Metadata: ... + + +async def _read_array_metadata( + store: Store, path: str, zarr_format: ZarrFormat | None +) -> ArrayV2Metadata | ArrayV3Metadata: + """ + Read array metadata from storage for Zarr formats 2 or 3. If zarr_format is ``None``, then + Zarr V3 will be tried first, followed by Zarr V2. + """ + if zarr_format == 2: + return await _read_array_metadata_v2(store=store, path=path) + elif zarr_format == 3: + return await _read_array_metadata_v3(store=store, path=path) + elif zarr_format is None: + try: + return await _read_array_metadata_v3(store=store, path=path) + except ArrayNotFoundError: + try: + return await _read_array_metadata_v2(store=store, path=path) + except ArrayNotFoundError as e: + msg = ( + "Neither Zarr V2 nor Zarr V3 array metadata documents were found in store " + f"{store!r} at path {path!r}." + ) + raise ArrayNotFoundError(msg) from e + else: + msg = f"Invalid value for zarr_format. Expected one of 2, 3, or None. Got {zarr_format}." + raise ValueError(msg) + + +def _build_metadata_v2( + zarr_json: dict[str, JSON], attrs_json: dict[str, JSON] +) -> ArrayV2Metadata | GroupMetadata: + """ + Convert a dict representation of Zarr V2 metadata into the corresponding metadata class. + """ + match zarr_json: + case {"shape": _}: + return ArrayV2Metadata.from_dict(zarr_json | {"attributes": attrs_json}) + case _: # pragma: no cover + return GroupMetadata.from_dict(zarr_json | {"attributes": attrs_json}) + + +async def _read_metadata_v2(store: Store, path: str) -> ArrayV2Metadata | GroupMetadata: + """ + Given a store_path, return ArrayV2Metadata or GroupMetadata defined by the metadata + document stored at store_path.path / (.zgroup | .zarray). If no metadata document is found, + this routine raises a ``NodeNotFoundError``. + """ + # TODO: consider first fetching array metadata, and only fetching group metadata when we don't + # find an array + zarray_bytes, zgroup_bytes, zattrs_bytes = await asyncio.gather( + store.get(_join_paths([path, ZARRAY_JSON]), prototype=default_buffer_prototype()), + store.get(_join_paths([path, ZGROUP_JSON]), prototype=default_buffer_prototype()), + store.get(_join_paths([path, ZATTRS_JSON]), prototype=default_buffer_prototype()), + ) + + if zattrs_bytes is None: + zattrs = {} + else: + zattrs = json.loads(zattrs_bytes.to_bytes()) + + # TODO: decide how to handle finding both array and group metadata. The spec does not seem to + # consider this situation. A practical approach would be to ignore that combination, and only + # return the array metadata. + if zarray_bytes is not None: + zmeta = json.loads(zarray_bytes.to_bytes()) + else: + if zgroup_bytes is not None: + zmeta = json.loads(zgroup_bytes.to_bytes()) + else: + # neither .zarray or .zgroup were found results in NodeNotFoundError + msg = ( + f"Neither array nor group metadata were found in store {store!r} at path {path!r}." + ) + raise NodeNotFoundError(msg) + + return _build_metadata_v2(zmeta, zattrs) + + +async def _read_group_metadata_v2(store: Store, path: str) -> GroupMetadata: + """ + Read Zarr V2 group metadata. + """ + try: + meta = await _read_metadata_v2(store=store, path=path) + except NodeNotFoundError as e: + # NodeNotFoundError is raised when neither array nor group metadata were found, + # but since this function is concerned with group metadata, + # it returns a more specific exception here. + msg = f"A group metadata document was not found in store {store!r} at path {path!r}." + raise GroupNotFoundError(msg) from e + if not isinstance(meta, GroupMetadata): + # TODO: test this exception + msg = ( + f"Group metadata was not found in store {store!r} at path {path!r}. " + "An array metadata document was found there instead." + ) + raise GroupNotFoundError(msg) + return meta + + +async def _read_group_metadata_v3(store: Store, path: str) -> GroupMetadata: + """ + Read Zarr V3 group metadata. + """ + try: + meta = await _read_metadata_v3(store=store, path=path) + except NodeNotFoundError as e: + # NodeNotFoundError is raised when neither array nor group metadata were found, + # but since this function is concerned with group metadata, + # it returns a more specific exception here. + msg = f"A group metadata document was not found in store {store!r} at path {path!r}." + raise GroupNotFoundError(msg) from e + if not isinstance(meta, GroupMetadata): + # TODO: test this exception + msg = ( + f"Group metadata was not found in store {store!r} at path {path!r}. " + "An array metadata document was found there instead." + ) + raise GroupNotFoundError(msg) + return meta + + +async def _read_group_metadata( + store: Store, path: str, *, zarr_format: ZarrFormat +) -> GroupMetadata: + if zarr_format == 2: + return await _read_group_metadata_v2(store=store, path=path) + return await _read_group_metadata_v3(store=store, path=path) + + +def _build_metadata_v3(zarr_json: dict[str, JSON]) -> ArrayV3Metadata | GroupMetadata: + """ + Convert a dict representation of Zarr V3 metadata into the corresponding metadata class. + """ + if "node_type" not in zarr_json: + msg = ( + "Invalid value for node_type. " + "Expected 'array' or 'group'. Got nothing (the key is missing)." + ) + raise MetadataValidationError(msg) + match zarr_json: + case {"node_type": "array"}: + return ArrayV3Metadata.from_dict(zarr_json) + case {"node_type": "group"}: + return GroupMetadata.from_dict(zarr_json) + case _: # pragma: no cover + raise ValueError( + "invalid value for `node_type` key in metadata document" + ) # pragma: no cover + + +async def _read_metadata_v3(store: Store, path: str) -> ArrayV3Metadata | GroupMetadata: + """ + Given a store_path, return ArrayV3Metadata or GroupMetadata defined by the metadata + document stored at store_path.path / zarr.json. If no such document is found, raise a + FileNotFoundError. + """ + zarr_json_bytes = await store.get( + _join_paths([path, ZARR_JSON]), prototype=default_buffer_prototype() + ) + if zarr_json_bytes is None: + msg = f"Neither array nor group metadata were found in store {store!r} at path {path!r}." + raise NodeNotFoundError(msg) + else: + zarr_json = json.loads(zarr_json_bytes.to_bytes()) + return _build_metadata_v3(zarr_json) + + +async def _read_array_metadata_v3(store: Store, path: str) -> ArrayV3Metadata: + """ + Read Zarr V3 array metadata from a store at a path. Raises ``ArrayNotFoundError`` if Zarr V3 + metadata was not found, or if Zarr V3 metadata was found, but it described a group instead of + an array. + """ + try: + maybe_array_meta = await _read_metadata_v3(store, path) + except NodeNotFoundError as e: + msg = f"Zarr V3 array metadata was not found in store {store!r} at path {path!r}." + raise ArrayNotFoundError(msg) from e + if not isinstance(maybe_array_meta, ArrayV3Metadata): + msg = ( + f"Zarr V3 array metadata was not found in store {store!r} at path {path!r}." + "Zarr V3 group metadata was found there instead." + ) + raise ArrayNotFoundError(msg) + return maybe_array_meta + + +async def _read_array_metadata_v2(store: Store, path: str) -> ArrayV2Metadata: + """ + Read Zarr V2 array metadata. Raises ``ArrayNotFoundError`` if .zarray is not found. + + Performance note: Only use this function to read from a storage location where an array is + expected. If a storage location could contain an array, or a group, then the function + ``_read_metadata_v2`` makes more efficient use of storage operations and should be used instead. + """ + zarray_bytes, zattrs_bytes = await asyncio.gather( + store.get(_join_paths([path, ZARRAY_JSON]), prototype=default_buffer_prototype()), + store.get(_join_paths([path, ZATTRS_JSON]), prototype=default_buffer_prototype()), + ) + + if zattrs_bytes is None: + attrs_json = {} + else: + attrs_json = json.loads(zattrs_bytes.to_bytes()) + + if zarray_bytes is not None: + zmeta = json.loads(zarray_bytes.to_bytes()) + return ArrayV2Metadata.from_dict(zmeta | {"attributes": attrs_json}) + else: + raise ArrayNotFoundError( + f"Zarr V2 array metadata was not found in store {store!r} at path {path!r}." + ) + + +def _persist_metadata( + store: Store, + path: str, + metadata: ArrayV2Metadata | ArrayV3Metadata | GroupMetadata, + semaphore: asyncio.Semaphore | None = None, +) -> tuple[Coroutine[None, None, str], ...]: + """ + Prepare to save a metadata document to storage, returning a tuple of coroutines that must be awaited. + """ + + to_save = metadata.to_buffer_dict(default_buffer_prototype()) + return tuple( + _set_return_key(store=store, key=_join_paths([path, key]), value=value, semaphore=semaphore) + for key, value in to_save.items() + ) diff --git a/src/zarr/core/metadata/group.py b/src/zarr/core/metadata/group.py new file mode 100644 index 0000000000..888d6538ab --- /dev/null +++ b/src/zarr/core/metadata/group.py @@ -0,0 +1,344 @@ +from __future__ import annotations + +import itertools +import json +from dataclasses import asdict, dataclass, field, fields, replace +from typing import TYPE_CHECKING, assert_never, cast, get_args + +from zarr.abc.metadata import Metadata +from zarr.core.common import ( + JSON, + ZARR_JSON, + ZARRAY_JSON, + ZATTRS_JSON, + ZGROUP_JSON, + ZMETADATA_V2_JSON, + NodeType, + ZarrFormat, +) +from zarr.core.config import config +from zarr.core.metadata import ArrayV2Metadata, ArrayV3Metadata +from zarr.core.metadata.common import parse_attributes +from zarr.core.metadata.v3 import Any, Literal, V3JsonEncoder, _replace_special_floats + +if TYPE_CHECKING: + from zarr.core.buffer import Buffer, BufferPrototype + + +@dataclass(frozen=True) +class GroupMetadata(Metadata): + """ + Metadata for a Group. + """ + + attributes: dict[str, Any] = field(default_factory=dict) + zarr_format: ZarrFormat = 3 + consolidated_metadata: ConsolidatedMetadata | None = None + node_type: Literal[group] = field(default="group", init=False) + + def to_buffer_dict(self, prototype: BufferPrototype) -> dict[str, Buffer]: + json_indent = config.get("json_indent") + if self.zarr_format == 3: + return { + ZARR_JSON: prototype.buffer.from_bytes( + json.dumps(_replace_special_floats(self.to_dict()), cls=V3JsonEncoder).encode() + ) + } + else: + items = { + ZGROUP_JSON: prototype.buffer.from_bytes( + json.dumps({"zarr_format": self.zarr_format}, indent=json_indent).encode() + ), + ZATTRS_JSON: prototype.buffer.from_bytes( + json.dumps(self.attributes, indent=json_indent).encode() + ), + } + if self.consolidated_metadata: + d = { + ZGROUP_JSON: {"zarr_format": self.zarr_format}, + ZATTRS_JSON: self.attributes, + } + consolidated_metadata = self.consolidated_metadata.to_dict()["metadata"] + assert isinstance(consolidated_metadata, dict) + for k, v in consolidated_metadata.items(): + attrs = v.pop("attributes", None) + d[f"{k}/{ZATTRS_JSON}"] = _replace_special_floats(attrs) + if "shape" in v: + # it's an array + d[f"{k}/{ZARRAY_JSON}"] = _replace_special_floats(v) + else: + d[f"{k}/{ZGROUP_JSON}"] = { + "zarr_format": self.zarr_format, + "consolidated_metadata": { + "metadata": {}, + "must_understand": False, + "kind": "inline", + }, + } + + items[ZMETADATA_V2_JSON] = prototype.buffer.from_bytes( + json.dumps( + {"metadata": d, "zarr_consolidated_format": 1}, + cls=V3JsonEncoder, + ).encode() + ) + + return items + + def __init__( + self, + attributes: dict[str, Any] | None = None, + zarr_format: ZarrFormat = 3, + consolidated_metadata: ConsolidatedMetadata | None = None, + ) -> None: + attributes_parsed = parse_attributes(attributes) + zarr_format_parsed = parse_zarr_format(zarr_format) + + object.__setattr__(self, "attributes", attributes_parsed) + object.__setattr__(self, "zarr_format", zarr_format_parsed) + object.__setattr__(self, "consolidated_metadata", consolidated_metadata) + + @classmethod + def from_dict(cls, data: dict[str, Any]) -> GroupMetadata: + data = dict(data) + assert data.pop("node_type", None) in ("group", None) + consolidated_metadata = data.pop("consolidated_metadata", None) + if consolidated_metadata: + data["consolidated_metadata"] = ConsolidatedMetadata.from_dict(consolidated_metadata) + + zarr_format = data.get("zarr_format") + if zarr_format == 2 or zarr_format is None: + # zarr v2 allowed arbitrary keys here. + # We don't want the GroupMetadata constructor to fail just because someone put an + # extra key in the metadata. + expected = {x.name for x in fields(cls)} + data = {k: v for k, v in data.items() if k in expected} + + return cls(**data) + + def to_dict(self) -> dict[str, Any]: + result = asdict(replace(self, consolidated_metadata=None)) + if self.consolidated_metadata: + result["consolidated_metadata"] = self.consolidated_metadata.to_dict() + return result + + +@dataclass(frozen=True) +class ConsolidatedMetadata: + """ + Consolidated Metadata for this Group. + + This stores the metadata of child nodes below this group. Any child groups + will have their consolidated metadata set appropriately. + """ + + metadata: dict[str, ArrayV2Metadata | ArrayV3Metadata | GroupMetadata] + kind: Literal[inline] = "inline" + must_understand: Literal[False] = False + + def to_dict(self) -> dict[str, JSON]: + return { + "kind": self.kind, + "must_understand": self.must_understand, + "metadata": {k: v.to_dict() for k, v in self.flattened_metadata.items()}, + } + + @classmethod + def from_dict(cls, data: dict[str, JSON]) -> ConsolidatedMetadata: + data = dict(data) + + kind = data.get("kind") + if kind != "inline": + raise ValueError(f"Consolidated metadata kind='{kind}' is not supported.") + + raw_metadata = data.get("metadata") + if not isinstance(raw_metadata, dict): + raise TypeError(f"Unexpected type for 'metadata': {type(raw_metadata)}") + + metadata: dict[str, ArrayV2Metadata | ArrayV3Metadata | GroupMetadata] = {} + if raw_metadata: + for k, v in raw_metadata.items(): + if not isinstance(v, dict): + raise TypeError( + f"Invalid value for metadata items. key='{k}', type='{type(v).__name__}'" + ) + + # zarr_format is present in v2 and v3. + zarr_format = parse_zarr_format(v["zarr_format"]) + + if zarr_format == 3: + node_type = parse_node_type(v.get("node_type", None)) + if node_type == "group": + metadata[k] = GroupMetadata.from_dict(v) + elif node_type == "array": + metadata[k] = ArrayV3Metadata.from_dict(v) + else: + assert_never(node_type) + elif zarr_format == 2: + if "shape" in v: + metadata[k] = ArrayV2Metadata.from_dict(v) + else: + metadata[k] = GroupMetadata.from_dict(v) + else: + assert_never(zarr_format) + + cls._flat_to_nested(metadata) + + return cls(metadata=metadata) + + @staticmethod + def _flat_to_nested( + metadata: dict[str, ArrayV2Metadata | ArrayV3Metadata | GroupMetadata], + ) -> None: + """ + Convert a flat metadata representation to a nested one. + + Notes + ----- + Flat metadata is used when persisting the consolidated metadata. The keys + include the full path, not just the node name. The key prefixes can be + used to determine which nodes are children of which other nodes. + + Nested metadata is used in-memory. The outermost level will only have the + *immediate* children of the Group. All nested child groups will be stored + under the consolidated metadata of their immediate parent. + """ + # We have a flat mapping from {k: v} where the keys include the *full* + # path segment: + # { + # "/a/b": { group_metadata }, + # "/a/b/array-0": { array_metadata }, + # "/a/b/array-1": { array_metadata }, + # } + # + # We want to reorganize the metadata such that each Group contains the + # array metadata of its immediate children. + # In the example, the group at `/a/b` will have consolidated metadata + # for its children `array-0` and `array-1`. + # + # metadata = dict(metadata) + + keys = sorted(metadata, key=lambda k: k.count("/")) + grouped = { + k: list(v) for k, v in itertools.groupby(keys, key=lambda k: k.rsplit("/", 1)[0]) + } + + # we go top down and directly manipulate metadata. + for key, children_keys in grouped.items(): + # key is a key like "a", "a/b", "a/b/c" + # The basic idea is to find the immediate parent (so "", "a", or "a/b") + # and update that node's consolidated metadata to include the metadata + # in children_keys + *prefixes, name = key.split("/") + parent = metadata + + while prefixes: + # e.g. a/b/c has a parent "a/b". Walk through to get + # metadata["a"]["b"] + part = prefixes.pop(0) + # we can assume that parent[part] here is a group + # otherwise we wouldn't have a node with this `part` prefix. + # We can also assume that the parent node will have consolidated metadata, + # because we're walking top to bottom. + parent = parent[part].consolidated_metadata.metadata # type: ignore[union-attr] + + node = parent[name] + children_keys = list(children_keys) + + if isinstance(node, ArrayV2Metadata | ArrayV3Metadata): + # These are already present, either thanks to being an array in the + # root, or by being collected as a child in the else clause + continue + children_keys = list(children_keys) + # We pop from metadata, since we're *moving* this under group + children = { + child_key.split("/")[-1]: metadata.pop(child_key) + for child_key in children_keys + if child_key != key + } + parent[name] = replace( + node, consolidated_metadata=ConsolidatedMetadata(metadata=children) + ) + + @property + def flattened_metadata(self) -> dict[str, ArrayV2Metadata | ArrayV3Metadata | GroupMetadata]: + """ + Return the flattened representation of Consolidated Metadata. + + The returned dictionary will have a key for each child node in the hierarchy + under this group. Under the default (nested) representation available through + ``self.metadata``, the dictionary only contains keys for immediate children. + + The keys of the dictionary will include the full path to a child node from + the current group, where segments are joined by ``/``. + + Examples + -------- + >>> cm = ConsolidatedMetadata( + ... metadata={ + ... "group-0": GroupMetadata( + ... consolidated_metadata=ConsolidatedMetadata( + ... { + ... "group-0-0": GroupMetadata(), + ... } + ... ) + ... ), + ... "group-1": GroupMetadata(), + ... } + ... ) + {'group-0': GroupMetadata(attributes={}, zarr_format=3, consolidated_metadata=None, node_type='group'), + 'group-0/group-0-0': GroupMetadata(attributes={}, zarr_format=3, consolidated_metadata=None, node_type='group'), + 'group-1': GroupMetadata(attributes={}, zarr_format=3, consolidated_metadata=None, node_type='group')} + """ + metadata = {} + + def flatten( + key: str, group: GroupMetadata | ArrayV2Metadata | ArrayV3Metadata + ) -> dict[str, ArrayV2Metadata | ArrayV3Metadata | GroupMetadata]: + children: dict[str, ArrayV2Metadata | ArrayV3Metadata | GroupMetadata] = {} + if isinstance(group, ArrayV2Metadata | ArrayV3Metadata): + children[key] = group + else: + if group.consolidated_metadata and group.consolidated_metadata.metadata is not None: + children[key] = replace( + group, consolidated_metadata=ConsolidatedMetadata(metadata={}) + ) + for name, val in group.consolidated_metadata.metadata.items(): + full_key = f"{key}/{name}" + if isinstance(val, GroupMetadata): + children.update(flatten(full_key, val)) + else: + children[full_key] = val + else: + children[key] = replace(group, consolidated_metadata=None) + return children + + for k, v in self.metadata.items(): + metadata.update(flatten(k, v)) + + return metadata + + +@dataclass(frozen=True) +class ImplicitGroupMarker(GroupMetadata): + """ + Marker for an implicit group. Instances of this class are only used in the context of group + creation as a placeholder to represent groups that should only be created if they do not + already exist in storage + """ + + +def parse_zarr_format(data: object) -> ZarrFormat: + """Parse the zarr_format field from metadata.""" + if data in get_args(ZarrFormat): + return cast(ZarrFormat, data) + msg = f"Invalid zarr_format. Expected one of 2 or 3. Got {data!r}." + raise ValueError(msg) + + +def parse_node_type(data: object) -> NodeType: + """Parse the node_type field from metadata.""" + if data in get_args(NodeType): + return cast(NodeType, data) + msg = f"Invalid node_type. Expected 'array' or 'group'. Got {data!r}." + raise ValueError(msg) diff --git a/src/zarr/core/metadata/v3.py b/src/zarr/core/metadata/v3.py index 9154762648..4253b85bac 100644 --- a/src/zarr/core/metadata/v3.py +++ b/src/zarr/core/metadata/v3.py @@ -1,7 +1,7 @@ from __future__ import annotations import warnings -from typing import TYPE_CHECKING, TypedDict, overload +from typing import TYPE_CHECKING, Any, Literal, TypedDict, cast, overload from zarr.abc.metadata import Metadata from zarr.core.buffer.core import default_buffer_prototype @@ -18,7 +18,6 @@ from collections.abc import Iterable, Sequence from dataclasses import dataclass, field, replace from enum import Enum -from typing import Any, Literal, cast import numcodecs.abc import numpy as np @@ -29,9 +28,7 @@ from zarr.core.chunk_grids import ChunkGrid, RegularChunkGrid from zarr.core.chunk_key_encodings import ChunkKeyEncoding, ChunkKeyEncodingLike from zarr.core.common import ( - JSON, ZARR_JSON, - ChunkCoords, parse_named_configuration, parse_shapelike, ) @@ -39,7 +36,6 @@ from zarr.core.metadata.common import parse_attributes from zarr.core.strings import _NUMPY_SUPPORTS_VLEN_STRING from zarr.core.strings import _STRING_DTYPE as STRING_NP_DTYPE -from zarr.errors import MetadataValidationError, NodeTypeValidationError from zarr.registry import get_codec_class DEFAULT_DTYPE = "float64" @@ -55,13 +51,15 @@ def parse_zarr_format(data: object) -> Literal[3]: if data == 3: return 3 - raise MetadataValidationError("zarr_format", 3, data) + msg = f"Invalid zarr_format. Expected 3. Got {data!r}." + raise ValueError(msg) def parse_node_type_array(data: object) -> Literal["array"]: if data == "array": return "array" - raise NodeTypeValidationError("node_type", "array", data) + msg = f"Invalid node_type. Expected 'array'. Got {data!r}." + raise ValueError(msg) def parse_codecs(data: object) -> tuple[Codec, ...]: diff --git a/src/zarr/core/sync_group.py b/src/zarr/core/sync_group.py index 39d8a17992..08031c0bea 100644 --- a/src/zarr/core/sync_group.py +++ b/src/zarr/core/sync_group.py @@ -2,11 +2,12 @@ from typing import TYPE_CHECKING -from zarr.core.group import Group, GroupMetadata, _parse_async_node +from zarr.core.group import Group, _parse_async_node from zarr.core.group import create_hierarchy as create_hierarchy_async from zarr.core.group import create_nodes as create_nodes_async from zarr.core.group import create_rooted_hierarchy as create_rooted_hierarchy_async from zarr.core.group import get_node as get_node_async +from zarr.core.metadata.group import GroupMetadata from zarr.core.sync import _collect_aiterator, sync if TYPE_CHECKING: @@ -140,7 +141,7 @@ def create_rooted_hierarchy( return _parse_async_node(async_node) -def get_node(store: Store, path: str, zarr_format: ZarrFormat) -> Array | Group: +def get_node(store: Store, path: str, zarr_format: ZarrFormat | None) -> Array | Group: """ Get an Array or Group from a path in a Store. @@ -150,8 +151,8 @@ def get_node(store: Store, path: str, zarr_format: ZarrFormat) -> Array | Group: The store-like object to read from. path : str The path to the node to read. - zarr_format : {2, 3} - The zarr format of the node to read. + zarr_format : {2, 3} | None + The zarr format of the node to read. If ``None``, then V3 will be tried first, then V2. Returns ------- diff --git a/src/zarr/errors.py b/src/zarr/errors.py index 441cdab9a3..81faf002a5 100644 --- a/src/zarr/errors.py +++ b/src/zarr/errors.py @@ -1,4 +1,4 @@ -from typing import Any +from typing_extensions import deprecated __all__ = [ "BaseZarrError", @@ -12,42 +12,42 @@ class BaseZarrError(ValueError): """ - Base error which all zarr errors are sub-classed from. + Base class for Zarr errors. """ - _msg = "" - - def __init__(self, *args: Any) -> None: - super().__init__(self._msg.format(*args)) - class ContainsGroupError(BaseZarrError): """Raised when a group already exists at a certain path.""" - _msg = "A group exists in store {!r} at path {!r}." - class ContainsArrayError(BaseZarrError): """Raised when an array already exists at a certain path.""" - _msg = "An array exists in store {!r} at path {!r}." +class ArrayNotFoundError(BaseZarrError): + """Raised when an array does not exist at a certain path.""" -class ContainsArrayAndGroupError(BaseZarrError): - """Raised when both array and group metadata are found at the same path.""" - _msg = ( - "Array and group metadata documents (.zarray and .zgroup) were both found in store " - "{!r} at path {!r}. " - "Only one of these files may be present in a given directory / prefix. " - "Remove the .zarray file, or the .zgroup file, or both." - ) +class GroupNotFoundError(BaseZarrError): + """Raised when a group does not exist at a certain path.""" -class MetadataValidationError(BaseZarrError): - """Raised when the Zarr metadata is invalid in some way""" +@deprecated("Use NodeNotFoundError instead.", category=None) +class PathNotFoundError(BaseZarrError): + # Backwards compatibility with v2. Superseded by NodeNotFoundError. + ... + + +class NodeNotFoundError(PathNotFoundError): + """Raised when an array or group does not exist at a certain path.""" + - _msg = "Invalid value for '{}'. Expected '{}'. Got '{}'." +class ContainsArrayAndGroupError(BaseZarrError): + """Raised when both array and group metadata are found at the same path. Zarr V2 only.""" + + +class MetadataValidationError(BaseZarrError): + """Raised when a Zarr metadata document is invalid""" class NodeTypeValidationError(MetadataValidationError): @@ -57,3 +57,9 @@ class NodeTypeValidationError(MetadataValidationError): This can be raised when the value is invalid or unexpected given the context, for example an 'array' node when we expected a 'group'. """ + + +class ReadOnlyError(PermissionError, BaseZarrError): + """ + Exception for when a mutation is attempted on an immutable resource. + """ diff --git a/src/zarr/storage/_utils.py b/src/zarr/storage/_utils.py index eda4342f47..b5d0620eeb 100644 --- a/src/zarr/storage/_utils.py +++ b/src/zarr/storage/_utils.py @@ -1,10 +1,12 @@ from __future__ import annotations +import asyncio import re from pathlib import Path from typing import TYPE_CHECKING, TypeVar -from zarr.abc.store import OffsetByteRequest, RangeByteRequest, SuffixByteRequest +from zarr.abc.store import OffsetByteRequest, RangeByteRequest, Store, SuffixByteRequest +from zarr.core.buffer import Buffer if TYPE_CHECKING: from collections.abc import Iterable, Mapping @@ -110,3 +112,32 @@ def _normalize_path_keys(data: Mapping[str, T]) -> dict[str, T]: """ parsed_keys = _normalize_paths(data.keys()) return dict(zip(parsed_keys, data.values(), strict=True)) + + +async def _set_return_key( + *, store: Store, key: str, value: Buffer, semaphore: asyncio.Semaphore | None = None +) -> str: + """ + Write a value to storage at the given key. The key is returned. + Useful when saving values via routines that return results in execution order, + like asyncio.as_completed, because in this case we need to know which key was saved in order + to yield the right object to the caller. + + Parameters + ---------- + store : Store + The store to save the value to. + key : str + The key to save the value to. + value : Buffer + The value to save. + semaphore : asyncio.Semaphore | None + An optional semaphore to use to limit the number of concurrent writes. + """ + + if semaphore is not None: + async with semaphore: + await store.set(key, value) + else: + await store.set(key, value) + return key diff --git a/tests/test_api.py b/tests/test_api.py index f03fd53f7a..12af9dd501 100644 --- a/tests/test_api.py +++ b/tests/test_api.py @@ -1,6 +1,9 @@ from __future__ import annotations -from typing import TYPE_CHECKING +import re +from typing import TYPE_CHECKING, get_args + +from zarr.core.common import AccessModeLiteral if TYPE_CHECKING: import pathlib @@ -33,7 +36,10 @@ save_group, ) from zarr.core.buffer import NDArrayLike -from zarr.errors import MetadataValidationError +from zarr.errors import ( + ArrayNotFoundError, + PathNotFoundError, +) from zarr.storage import MemoryStore from zarr.storage._utils import normalize_path from zarr.testing.utils import gpu_test @@ -137,7 +143,8 @@ async def test_open_array(memory_store: MemoryStore) -> None: assert z.read_only # path not found - with pytest.raises(FileNotFoundError): + msg = "Neither array nor group metadata were found in " + with pytest.raises(PathNotFoundError, match=msg): open(store="doesnotexist", mode="r") @@ -236,74 +243,53 @@ def test_save_errors() -> None: zarr.save("data/example.zarr", a, mode="w") -def test_open_with_mode_r(tmp_path: pathlib.Path) -> None: - # 'r' means read only (must exist) - with pytest.raises(FileNotFoundError): - zarr.open(store=tmp_path, mode="r") - z1 = zarr.ones(store=tmp_path, shape=(3, 3)) - assert z1.fill_value == 1 - z2 = zarr.open(store=tmp_path, mode="r") - assert isinstance(z2, Array) - assert z2.fill_value == 1 - result = z2[:] - assert isinstance(result, NDArrayLike) - assert (result == 1).all() - with pytest.raises(ValueError): - z2[:] = 3 - - -def test_open_with_mode_r_plus(tmp_path: pathlib.Path) -> None: - # 'r+' means read/write (must exist) - with pytest.raises(FileNotFoundError): - zarr.open(store=tmp_path, mode="r+") - zarr.ones(store=tmp_path, shape=(3, 3)) - z2 = zarr.open(store=tmp_path, mode="r+") - assert isinstance(z2, Array) - result = z2[:] - assert isinstance(result, NDArrayLike) - assert (result == 1).all() - z2[:] = 3 - - -async def test_open_with_mode_a(tmp_path: pathlib.Path) -> None: - # Open without shape argument should default to group - g = zarr.open(store=tmp_path, mode="a") - assert isinstance(g, Group) - await g.store_path.delete() - - # 'a' means read/write (create if doesn't exist) - arr = zarr.open(store=tmp_path, mode="a", shape=(3, 3)) - assert isinstance(arr, Array) - arr[...] = 1 - z2 = zarr.open(store=tmp_path, mode="a") - assert isinstance(z2, Array) - result = z2[:] - assert isinstance(result, NDArrayLike) - assert (result == 1).all() - z2[:] = 3 - - -def test_open_with_mode_w(tmp_path: pathlib.Path) -> None: - # 'w' means create (overwrite if exists); - arr = zarr.open(store=tmp_path, mode="w", shape=(3, 3)) - assert isinstance(arr, Array) - - arr[...] = 3 - z2 = zarr.open(store=tmp_path, mode="w", shape=(3, 3)) - assert isinstance(z2, Array) - result = z2[:] - assert isinstance(result, NDArrayLike) - assert not (result == 3).all() - z2[:] = 3 +@pytest.mark.parametrize("mode_str", get_args(AccessModeLiteral)) +@pytest.mark.parametrize("node_type", ["group", "array"]) +@pytest.mark.parametrize("zarr_format", [None, 2, 3]) +def test_open( + mode_str: AccessModeLiteral, + tmp_path: pathlib.Path, + node_type: Literal["group", "array"], + zarr_format: None | ZarrFormat, +) -> None: + attrs = {"foo": 10} + shape = (3, 3) + cls_expect = zarr.Group if node_type == "group" else zarr.Array + open_kwargs = ( + {"shape": shape, "attributes": attrs} if node_type == "array" else {"attributes": attrs} + ) + if mode_str in ("r", "r+"): + # Opening a path with no node is an error + msg = "Neither array nor group metadata were found in " + with pytest.raises(PathNotFoundError, match=msg): + zarr.open(store=tmp_path, mode=mode_str, zarr_format=zarr_format) + else: + # Opening a path with no node creates one + assert isinstance( + zarr.open( + store=tmp_path / "create", mode=mode_str, **open_kwargs, zarr_format=zarr_format + ), + cls_expect, + ) + extant = zarr.open(store=tmp_path, zarr_format=zarr_format, mode="w", **open_kwargs) + if mode_str in ("r", "r+", "a"): + # Opening a path with an existing node returns the node at that path + observed = zarr.open(store=tmp_path, mode=mode_str, zarr_format=zarr_format, **open_kwargs) + assert observed.metadata == extant.metadata -def test_open_with_mode_w_minus(tmp_path: pathlib.Path) -> None: - # 'w-' means create (fail if exists) - arr = zarr.open(store=tmp_path, mode="w-", shape=(3, 3)) - assert isinstance(arr, Array) - arr[...] = 1 - with pytest.raises(FileExistsError): - zarr.open(store=tmp_path, mode="w-") + elif mode_str == "w-": + # Opening an existing node is an error + with pytest.raises(FileExistsError): + zarr.open(store=tmp_path, mode=mode_str, zarr_format=zarr_format, **open_kwargs) + else: + # mode_str is 'w' + # opening a path with an existing node overwrites it + new_open_kwargs = open_kwargs | {"attributes": {"bar": 20}} + observed = zarr.open( + store=tmp_path, mode=mode_str, zarr_format=zarr_format, **new_open_kwargs + ) + assert observed.metadata != extant.metadata @pytest.mark.parametrize("zarr_format", [2, 3]) @@ -1104,29 +1090,17 @@ async def test_open_falls_back_to_open_group_async() -> None: assert group.attrs == {"key": "value"} -@pytest.mark.parametrize("mode", ["r", "r+", "w", "a"]) -def test_open_modes_creates_group(tmp_path: pathlib.Path, mode: str) -> None: - # https://github.com/zarr-developers/zarr-python/issues/2490 - zarr_dir = tmp_path / f"mode-{mode}-test.zarr" - if mode in ["r", "r+"]: - # Expect FileNotFoundError to be raised if 'r' or 'r+' mode - with pytest.raises(FileNotFoundError): - zarr.open(store=zarr_dir, mode=mode) - else: - group = zarr.open(store=zarr_dir, mode=mode) - assert isinstance(group, Group) - - async def test_metadata_validation_error() -> None: + msg = "Invalid value for zarr_format. Expected one of 2, 3, or None. Got 3.0." with pytest.raises( - MetadataValidationError, - match="Invalid value for 'zarr_format'. Expected '2, 3, or None'. Got '3.0'.", + ValueError, + match=msg, ): await zarr.api.asynchronous.open_group(zarr_format="3.0") # type: ignore [arg-type] with pytest.raises( - MetadataValidationError, - match="Invalid value for 'zarr_format'. Expected '2, 3, or None'. Got '3.0'.", + ValueError, + match=msg, ): await zarr.api.asynchronous.open_array(shape=(1,), zarr_format="3.0") # type: ignore [arg-type] @@ -1138,7 +1112,8 @@ async def test_metadata_validation_error() -> None: ) def test_open_array_with_mode_r_plus(store: Store) -> None: # 'r+' means read/write (must exist) - with pytest.raises(FileNotFoundError): + msg = "Neither Zarr V2 nor Zarr V3 array metadata documents were found" + with pytest.raises(ArrayNotFoundError, match=re.escape(msg)): zarr.open_array(store=store, mode="r+") zarr.ones(store=store, shape=(3, 3)) z2 = zarr.open_array(store=store, mode="r+") diff --git a/tests/test_group.py b/tests/test_group.py index 1e4f31b5d6..2ffeee5bc6 100644 --- a/tests/test_group.py +++ b/tests/test_group.py @@ -24,10 +24,6 @@ from zarr.core.buffer import default_buffer_prototype from zarr.core.config import config as zarr_config from zarr.core.group import ( - ConsolidatedMetadata, - GroupMetadata, - ImplicitGroupMarker, - _build_metadata_v3, _get_roots, _parse_hierarchy_dict, create_hierarchy, @@ -35,9 +31,17 @@ create_rooted_hierarchy, get_node, ) +from zarr.core.metadata._io import _build_metadata_v3 +from zarr.core.metadata.group import ConsolidatedMetadata, GroupMetadata, ImplicitGroupMarker from zarr.core.metadata.v3 import ArrayV3Metadata from zarr.core.sync import _collect_aiterator, sync -from zarr.errors import ContainsArrayError, ContainsGroupError, MetadataValidationError +from zarr.errors import ( + ContainsArrayError, + ContainsGroupError, + GroupNotFoundError, + MetadataValidationError, + NodeNotFoundError, +) from zarr.storage import LocalStore, MemoryStore, StorePath, ZipStore from zarr.storage._common import make_store_path from zarr.storage._utils import _join_paths, normalize_path @@ -296,7 +300,8 @@ def test_group_open(store: Store, zarr_format: ZarrFormat, overwrite: bool) -> N """ spath = StorePath(store) # attempt to open a group that does not exist - with pytest.raises(FileNotFoundError): + msg = f"Zarr V3 group metadata was not found in store {spath.store!r} at path {spath.path!r}." + with pytest.raises(GroupNotFoundError, match=re.escape(msg)): Group.open(store) # create the group @@ -868,8 +873,8 @@ async def test_asyncgroup_open_wrong_format( zarr_format_wrong = 3 else: raise AssertionError - - with pytest.raises(FileNotFoundError): + msg = f"Zarr V{zarr_format_wrong} group metadata was not found in store" + with pytest.raises(GroupNotFoundError, match=msg): await AsyncGroup.open(store=store, zarr_format=zarr_format_wrong) @@ -1597,7 +1602,8 @@ async def test_create_hierarchy( extra_group = sync_group.get_node(store=store, path="group/extra", zarr_format=zarr_format) assert extra_group.metadata.attributes == {"path": "group/extra"} else: - with pytest.raises(FileNotFoundError): + msg = "Neither array nor group metadata were found in store" + with pytest.raises(NodeNotFoundError, match=msg): await get_node(store=store, path="group/extra", zarr_format=zarr_format) assert expected_meta == {k: v.metadata for k, v in created.items()} @@ -2052,7 +2058,7 @@ def test_build_metadata_v3(option: Literal["array", "group", "invalid"]) -> None metadata_dict = GroupMetadata(zarr_format=3).to_dict() metadata_dict.pop("node_type") # TODO: fix the error message - msg = "Invalid value for 'node_type'. Expected 'array or group'. Got 'nothing (the key is missing)'." + msg = "Invalid value for node_type. Expected 'array' or 'group'. Got nothing (the key is missing)." with pytest.raises(MetadataValidationError, match=re.escape(msg)): _build_metadata_v3(metadata_dict) diff --git a/tests/test_metadata/test_consolidated.py b/tests/test_metadata/test_consolidated.py index a179982e94..c085702138 100644 --- a/tests/test_metadata/test_consolidated.py +++ b/tests/test_metadata/test_consolidated.py @@ -18,8 +18,8 @@ open_consolidated, ) from zarr.core.buffer import cpu, default_buffer_prototype -from zarr.core.group import ConsolidatedMetadata, GroupMetadata from zarr.core.metadata import ArrayV3Metadata +from zarr.core.metadata.group import ConsolidatedMetadata, GroupMetadata from zarr.core.metadata.v2 import ArrayV2Metadata from zarr.storage import StorePath diff --git a/tests/test_metadata/test_v2.py b/tests/test_metadata/test_v2.py index 4600a977d4..4f3aa6bc98 100644 --- a/tests/test_metadata/test_v2.py +++ b/tests/test_metadata/test_v2.py @@ -10,8 +10,8 @@ import zarr.storage from zarr.core.buffer import cpu from zarr.core.buffer.core import default_buffer_prototype -from zarr.core.group import ConsolidatedMetadata, GroupMetadata from zarr.core.metadata import ArrayV2Metadata +from zarr.core.metadata.group import ConsolidatedMetadata, GroupMetadata from zarr.core.metadata.v2 import parse_zarr_format if TYPE_CHECKING: diff --git a/tests/test_metadata/test_v3.py b/tests/test_metadata/test_v3.py index a47cbf43bb..8499844222 100644 --- a/tests/test_metadata/test_v3.py +++ b/tests/test_metadata/test_v3.py @@ -11,7 +11,7 @@ from zarr.core.buffer import default_buffer_prototype from zarr.core.chunk_key_encodings import DefaultChunkKeyEncoding, V2ChunkKeyEncoding from zarr.core.config import config -from zarr.core.group import GroupMetadata, parse_node_type +from zarr.core.metadata.group import GroupMetadata, parse_node_type from zarr.core.metadata.v3 import ( ArrayV3Metadata, DataType, @@ -20,7 +20,6 @@ parse_fill_value, parse_zarr_format, ) -from zarr.errors import MetadataValidationError if TYPE_CHECKING: from collections.abc import Sequence @@ -61,9 +60,7 @@ @pytest.mark.parametrize("data", [None, 1, 2, 4, 5, "3"]) def test_parse_zarr_format_invalid(data: Any) -> None: - with pytest.raises( - ValueError, match=f"Invalid value for 'zarr_format'. Expected '3'. Got '{data}'." - ): + with pytest.raises(ValueError, match=f"Invalid zarr_format. Expected 3. Got {data!r}."): parse_zarr_format(data) @@ -79,17 +76,15 @@ def test_parse_node_type_valid() -> None: @pytest.mark.parametrize("node_type", [None, 2, "other"]) def test_parse_node_type_invalid(node_type: Any) -> None: with pytest.raises( - MetadataValidationError, - match=f"Invalid value for 'node_type'. Expected 'array or group'. Got '{node_type}'.", + ValueError, + match=f"Invalid node_type. Expected 'array' or 'group'. Got {node_type!r}.", ): parse_node_type(node_type) @pytest.mark.parametrize("data", [None, "group"]) def test_parse_node_type_array_invalid(data: Any) -> None: - with pytest.raises( - ValueError, match=f"Invalid value for 'node_type'. Expected 'array'. Got '{data}'." - ): + with pytest.raises(ValueError, match=f"Invalid node_type. Expected 'array'. Got {data!r}."): parse_node_type_array(data)