diff --git a/xarray/core/datatree.py b/xarray/core/datatree.py index c923ca2eb87..38f8f8cd495 100644 --- a/xarray/core/datatree.py +++ b/xarray/core/datatree.py @@ -1,7 +1,8 @@ from __future__ import annotations -import copy import itertools +import textwrap +from collections import ChainMap from collections.abc import Hashable, Iterable, Iterator, Mapping, MutableMapping from html import escape from typing import ( @@ -16,6 +17,7 @@ ) from xarray.core import utils +from xarray.core.alignment import align from xarray.core.common import TreeAttrAccessMixin from xarray.core.coordinates import DatasetCoordinates from xarray.core.dataarray import DataArray @@ -31,7 +33,7 @@ MappedDataWithCoords, ) from xarray.core.datatree_render import RenderDataTree -from xarray.core.formatting import datatree_repr +from xarray.core.formatting import datatree_repr, dims_and_coords_repr from xarray.core.formatting_html import ( datatree_repr as datatree_repr_html, ) @@ -79,11 +81,24 @@ T_Path = Union[str, NodePath] +def _collect_data_and_coord_variables( + data: Dataset, +) -> tuple[dict[Hashable, Variable], dict[Hashable, Variable]]: + data_variables = {} + coord_variables = {} + for k, v in data.variables.items(): + if k in data._coord_names: + coord_variables[k] = v + else: + data_variables[k] = v + return data_variables, coord_variables + + def _coerce_to_dataset(data: Dataset | DataArray | None) -> Dataset: if isinstance(data, DataArray): ds = data.to_dataset() elif isinstance(data, Dataset): - ds = data + ds = data.copy(deep=False) elif data is None: ds = Dataset() else: @@ -93,14 +108,57 @@ def _coerce_to_dataset(data: Dataset | DataArray | None) -> Dataset: return ds -def _check_for_name_collisions( - children: Iterable[str], variables: Iterable[Hashable] +def _join_path(root: str, name: str) -> str: + return str(NodePath(root) / name) + + +def _inherited_dataset(ds: Dataset, parent: Dataset) -> Dataset: + return Dataset._construct_direct( + variables=parent._variables | ds._variables, + coord_names=parent._coord_names | ds._coord_names, + dims=parent._dims | ds._dims, + attrs=ds._attrs, + indexes=parent._indexes | ds._indexes, + encoding=ds._encoding, + close=ds._close, + ) + + +def _without_header(text: str) -> str: + return "\n".join(text.split("\n")[1:]) + + +def _indented(text: str) -> str: + return textwrap.indent(text, prefix=" ") + + +def _check_alignment( + path: str, + node_ds: Dataset, + parent_ds: Dataset | None, + children: Mapping[str, DataTree], ) -> None: - colliding_names = set(children).intersection(set(variables)) - if colliding_names: - raise KeyError( - f"Some names would collide between variables and children: {list(colliding_names)}" - ) + if parent_ds is not None: + try: + align(node_ds, parent_ds, join="exact") + except ValueError as e: + node_repr = _indented(_without_header(repr(node_ds))) + parent_repr = _indented(dims_and_coords_repr(parent_ds)) + raise ValueError( + f"group {path!r} is not aligned with its parents:\n" + f"Group:\n{node_repr}\nFrom parents:\n{parent_repr}" + ) from e + + if children: + if parent_ds is not None: + base_ds = _inherited_dataset(node_ds, parent_ds) + else: + base_ds = node_ds + + for child_name, child in children.items(): + child_path = str(NodePath(path) / child_name) + child_ds = child.to_dataset(inherited=False) + _check_alignment(child_path, child_ds, base_ds, child.children) class DatasetView(Dataset): @@ -118,7 +176,7 @@ class DatasetView(Dataset): __slots__ = ( "_attrs", - "_cache", + "_cache", # used by _CachedAccessor "_coord_names", "_dims", "_encoding", @@ -136,21 +194,27 @@ def __init__( raise AttributeError("DatasetView objects are not to be initialized directly") @classmethod - def _from_node( + def _constructor( cls, - wrapping_node: DataTree, + variables: dict[Any, Variable], + coord_names: set[Hashable], + dims: dict[Any, int], + attrs: dict | None, + indexes: dict[Any, Index], + encoding: dict | None, + close: Callable[[], None] | None, ) -> DatasetView: - """Constructor, using dataset attributes from wrapping node""" - + """Private constructor, from Dataset attributes.""" + # We override Dataset._construct_direct below, so we need a new + # constructor for creating DatasetView objects. obj: DatasetView = object.__new__(cls) - obj._variables = wrapping_node._variables - obj._coord_names = wrapping_node._coord_names - obj._dims = wrapping_node._dims - obj._indexes = wrapping_node._indexes - obj._attrs = wrapping_node._attrs - obj._close = wrapping_node._close - obj._encoding = wrapping_node._encoding - + obj._variables = variables + obj._coord_names = coord_names + obj._dims = dims + obj._indexes = indexes + obj._attrs = attrs + obj._close = close + obj._encoding = encoding return obj def __setitem__(self, key, val) -> None: @@ -337,27 +401,27 @@ class DataTree( _name: str | None _parent: DataTree | None _children: dict[str, DataTree] + _cache: dict[str, Any] # used by _CachedAccessor + _data_variables: dict[Hashable, Variable] + _node_coord_variables: dict[Hashable, Variable] + _node_dims: dict[Hashable, int] + _node_indexes: dict[Hashable, Index] _attrs: dict[Hashable, Any] | None - _cache: dict[str, Any] - _coord_names: set[Hashable] - _dims: dict[Hashable, int] _encoding: dict[Hashable, Any] | None _close: Callable[[], None] | None - _indexes: dict[Hashable, Index] - _variables: dict[Hashable, Variable] __slots__ = ( "_name", "_parent", "_children", + "_cache", # used by _CachedAccessor + "_data_variables", + "_node_coord_variables", + "_node_dims", + "_node_indexes", "_attrs", - "_cache", - "_coord_names", - "_dims", "_encoding", "_close", - "_indexes", - "_variables", ) def __init__( @@ -370,14 +434,15 @@ def __init__( """ Create a single node of a DataTree. - The node may optionally contain data in the form of data and coordinate variables, stored in the same way as - data is stored in an xarray.Dataset. + The node may optionally contain data in the form of data and coordinate + variables, stored in the same way as data is stored in an + xarray.Dataset. Parameters ---------- data : Dataset, DataArray, or None, optional - Data to store under the .ds attribute of this node. DataArrays will be promoted to Datasets. - Default is None. + Data to store under the .ds attribute of this node. DataArrays will + be promoted to Datasets. Default is None. parent : DataTree, optional Parent node to this node. Default is None. children : Mapping[str, DataTree], optional @@ -393,30 +458,48 @@ def __init__( -------- DataTree.from_dict """ - - # validate input if children is None: children = {} - ds = _coerce_to_dataset(data) - _check_for_name_collisions(children, ds.variables) super().__init__(name=name) + self._set_node_data(_coerce_to_dataset(data)) + self.parent = parent + self.children = children - # set data attributes - self._replace( - inplace=True, - variables=ds._variables, - coord_names=ds._coord_names, - dims=ds._dims, - indexes=ds._indexes, - attrs=ds._attrs, - encoding=ds._encoding, - ) + def _set_node_data(self, ds: Dataset): + data_vars, coord_vars = _collect_data_and_coord_variables(ds) + self._data_variables = data_vars + self._node_coord_variables = coord_vars + self._node_dims = ds._dims + self._node_indexes = ds._indexes + self._encoding = ds._encoding + self._attrs = ds._attrs self._close = ds._close - # set tree attributes (must happen after variables set to avoid initialization errors) - self.children = children - self.parent = parent + def _pre_attach(self: DataTree, parent: DataTree, name: str) -> None: + super()._pre_attach(parent, name) + if name in parent.ds.variables: + raise KeyError( + f"parent {parent.name} already contains a variable named {name}" + ) + path = str(NodePath(parent.path) / name) + node_ds = self.to_dataset(inherited=False) + parent_ds = parent._to_dataset_view(rebuild_dims=False) + _check_alignment(path, node_ds, parent_ds, self.children) + + @property + def _coord_variables(self) -> ChainMap[Hashable, Variable]: + return ChainMap( + self._node_coord_variables, *(p._node_coord_variables for p in self.parents) + ) + + @property + def _dims(self) -> ChainMap[Hashable, int]: + return ChainMap(self._node_dims, *(p._node_dims for p in self.parents)) + + @property + def _indexes(self) -> ChainMap[Hashable, Index]: + return ChainMap(self._node_indexes, *(p._node_indexes for p in self.parents)) @property def parent(self: DataTree) -> DataTree | None: @@ -429,71 +512,87 @@ def parent(self: DataTree, new_parent: DataTree) -> None: raise ValueError("Cannot set an unnamed node as a child of another node") self._set_parent(new_parent, self.name) + def _to_dataset_view(self, rebuild_dims: bool) -> DatasetView: + variables = dict(self._data_variables) + variables |= self._coord_variables + if rebuild_dims: + dims = calculate_dimensions(variables) + else: + # Note: rebuild_dims=False can create technically invalid Dataset + # objects because it may not contain all dimensions on its direct + # member variables, e.g., consider: + # tree = DataTree.from_dict( + # { + # "/": xr.Dataset({"a": (("x",), [1, 2])}), # x has size 2 + # "/b/c": xr.Dataset({"d": (("x",), [3])}), # x has size1 + # } + # ) + # However, they are fine for internal use cases, for align() or + # building a repr(). + dims = dict(self._dims) + return DatasetView._constructor( + variables=variables, + coord_names=set(self._coord_variables), + dims=dims, + attrs=self._attrs, + indexes=dict(self._indexes), + encoding=self._encoding, + close=None, + ) + @property def ds(self) -> DatasetView: """ An immutable Dataset-like view onto the data in this node. - For a mutable Dataset containing the same data as in this node, use `.to_dataset()` instead. + Includes inherited coordinates and indexes from parent nodes. + + For a mutable Dataset containing the same data as in this node, use + `.to_dataset()` instead. See Also -------- DataTree.to_dataset """ - return DatasetView._from_node(self) + return self._to_dataset_view(rebuild_dims=True) @ds.setter def ds(self, data: Dataset | DataArray | None = None) -> None: - # Known mypy issue for setters with different type to property: - # https://github.com/python/mypy/issues/3004 ds = _coerce_to_dataset(data) + self._replace_node(ds) - _check_for_name_collisions(self.children, ds.variables) - - self._replace( - inplace=True, - variables=ds._variables, - coord_names=ds._coord_names, - dims=ds._dims, - indexes=ds._indexes, - attrs=ds._attrs, - encoding=ds._encoding, - ) - self._close = ds._close - - def _pre_attach(self: DataTree, parent: DataTree) -> None: - """ - Method which superclass calls before setting parent, here used to prevent having two - children with duplicate names (or a data variable with the same name as a child). - """ - super()._pre_attach(parent) - if self.name in list(parent.ds.variables): - raise KeyError( - f"parent {parent.name} already contains a data variable named {self.name}" - ) - - def to_dataset(self) -> Dataset: + def to_dataset(self, inherited: bool = True) -> Dataset: """ Return the data in this node as a new xarray.Dataset object. + Parameters + ---------- + inherited : bool, optional + If False, only include coordinates and indexes defined at the level + of this DataTree node, excluding inherited coordinates. + See Also -------- DataTree.ds """ + coord_vars = self._coord_variables if inherited else self._node_coord_variables + variables = dict(self._data_variables) + variables |= coord_vars + dims = calculate_dimensions(variables) if inherited else dict(self._node_dims) return Dataset._construct_direct( - self._variables, - self._coord_names, - self._dims, - self._attrs, - self._indexes, - self._encoding, + variables, + set(coord_vars), + dims, + None if self._attrs is None else dict(self._attrs), + dict(self._indexes if inherited else self._node_indexes), + None if self._encoding is None else dict(self._encoding), self._close, ) @property - def has_data(self): - """Whether or not there are any data variables in this node.""" - return len(self._variables) > 0 + def has_data(self) -> bool: + """Whether or not there are any variables in this node.""" + return bool(self._data_variables or self._node_coord_variables) @property def has_attrs(self) -> bool: @@ -518,7 +617,7 @@ def variables(self) -> Mapping[Hashable, Variable]: Dataset invariants. It contains all variable objects constituting this DataTree node, including both data variables and coordinates. """ - return Frozen(self._variables) + return Frozen(self._data_variables | self._coord_variables) @property def attrs(self) -> dict[Hashable, Any]: @@ -579,7 +678,7 @@ def _attr_sources(self) -> Iterable[Mapping[Hashable, Any]]: def _item_sources(self) -> Iterable[Mapping[Any, Any]]: """Places to look-up items for key-completion""" yield self.data_vars - yield HybridMappingProxy(keys=self._coord_names, mapping=self.coords) + yield HybridMappingProxy(keys=self._coord_variables, mapping=self.coords) # virtual coordinates yield HybridMappingProxy(keys=self.dims, mapping=self) @@ -621,10 +720,10 @@ def __contains__(self, key: object) -> bool: return key in self.variables or key in self.children def __bool__(self) -> bool: - return bool(self.ds.data_vars) or bool(self.children) + return bool(self._data_variables) or bool(self._children) def __iter__(self) -> Iterator[Hashable]: - return itertools.chain(self.ds.data_vars, self.children) + return itertools.chain(self._data_variables, self._children) def __array__(self, dtype=None, copy=None): raise TypeError( @@ -646,122 +745,32 @@ def _repr_html_(self): return f"
{escape(repr(self))}
" return datatree_repr_html(self) - @classmethod - def _construct_direct( - cls, - variables: dict[Any, Variable], - coord_names: set[Hashable], - dims: dict[Any, int] | None = None, - attrs: dict | None = None, - indexes: dict[Any, Index] | None = None, - encoding: dict | None = None, - name: str | None = None, - parent: DataTree | None = None, - children: dict[str, DataTree] | None = None, - close: Callable[[], None] | None = None, - ) -> DataTree: - """Shortcut around __init__ for internal use when we want to skip costly validation.""" + def _replace_node( + self: DataTree, + data: Dataset | Default = _default, + children: dict[str, DataTree] | Default = _default, + ) -> None: - # data attributes - if dims is None: - dims = calculate_dimensions(variables) - if indexes is None: - indexes = {} - if children is None: - children = dict() + ds = self.to_dataset(inherited=False) if data is _default else data - obj: DataTree = object.__new__(cls) - obj._variables = variables - obj._coord_names = coord_names - obj._dims = dims - obj._indexes = indexes - obj._attrs = attrs - obj._close = close - obj._encoding = encoding - - # tree attributes - obj._name = name - obj._children = children - obj._parent = parent + if children is _default: + children = self._children - return obj + for child_name in children: + if child_name in ds.variables: + raise ValueError(f"node already contains a variable named {child_name}") - def _replace( - self: DataTree, - variables: dict[Hashable, Variable] | None = None, - coord_names: set[Hashable] | None = None, - dims: dict[Any, int] | None = None, - attrs: dict[Hashable, Any] | None | Default = _default, - indexes: dict[Hashable, Index] | None = None, - encoding: dict | None | Default = _default, - name: str | None | Default = _default, - parent: DataTree | None | Default = _default, - children: dict[str, DataTree] | None = None, - inplace: bool = False, - ) -> DataTree: - """ - Fastpath constructor for internal use. + parent_ds = ( + self.parent._to_dataset_view(rebuild_dims=False) + if self.parent is not None + else None + ) + _check_alignment(self.path, ds, parent_ds, children) - Returns an object with optionally replaced attributes. + if data is not _default: + self._set_node_data(ds) - Explicitly passed arguments are *not* copied when placed on the new - datatree. It is up to the caller to ensure that they have the right type - and are not used elsewhere. - """ - # TODO Adding new children inplace using this method will cause bugs. - # You will end up with an inconsistency between the name of the child node and the key the child is stored under. - # Use ._set() instead for now - if inplace: - if variables is not None: - self._variables = variables - if coord_names is not None: - self._coord_names = coord_names - if dims is not None: - self._dims = dims - if attrs is not _default: - self._attrs = attrs - if indexes is not None: - self._indexes = indexes - if encoding is not _default: - self._encoding = encoding - if name is not _default: - self._name = name - if parent is not _default: - self._parent = parent - if children is not None: - self._children = children - obj = self - else: - if variables is None: - variables = self._variables.copy() - if coord_names is None: - coord_names = self._coord_names.copy() - if dims is None: - dims = self._dims.copy() - if attrs is _default: - attrs = copy.copy(self._attrs) - if indexes is None: - indexes = self._indexes.copy() - if encoding is _default: - encoding = copy.copy(self._encoding) - if name is _default: - name = self._name # no need to copy str objects or None - if parent is _default: - parent = copy.copy(self._parent) - if children is _default: - children = copy.copy(self._children) - obj = self._construct_direct( - variables, - coord_names, - dims, - attrs, - indexes, - encoding, - name, - parent, - children, - ) - return obj + self._children = children def copy( self: DataTree, @@ -813,9 +822,8 @@ def _copy_node( deep: bool = False, ) -> DataTree: """Copy just one node of a tree""" - new_node: DataTree = DataTree() - new_node.name = self.name - new_node.ds = self.to_dataset().copy(deep=deep) # type: ignore[assignment] + data = self.ds.copy(deep=deep) + new_node: DataTree = DataTree(data, name=self.name) return new_node def __copy__(self: DataTree) -> DataTree: @@ -963,11 +971,12 @@ def update( raise TypeError(f"Type {type(v)} cannot be assigned to a DataTree") vars_merge_result = dataset_update_method(self.to_dataset(), new_variables) + data = Dataset._construct_direct(**vars_merge_result._asdict()) + # TODO are there any subtleties with preserving order of children like this? merged_children = {**self.children, **new_children} - self._replace( - inplace=True, children=merged_children, **vars_merge_result._asdict() - ) + + self._replace_node(data, children=merged_children) def assign( self, items: Mapping[Any, Any] | None = None, **items_kwargs: Any @@ -1042,10 +1051,12 @@ def drop_nodes( if extra: raise KeyError(f"Cannot drop all nodes - nodes {extra} not present") + result = self.copy() children_to_keep = { - name: child for name, child in self.children.items() if name not in names + name: child for name, child in result.children.items() if name not in names } - return self._replace(children=children_to_keep) + result._replace_node(children=children_to_keep) + return result @classmethod def from_dict( @@ -1137,7 +1148,9 @@ def indexes(self) -> Indexes[pd.Index]: @property def xindexes(self) -> Indexes[Index]: """Mapping of xarray Index objects used for label based indexing.""" - return Indexes(self._indexes, {k: self._variables[k] for k in self._indexes}) + return Indexes( + self._indexes, {k: self._coord_variables[k] for k in self._indexes} + ) @property def coords(self) -> DatasetCoordinates: diff --git a/xarray/core/datatree_io.py b/xarray/core/datatree_io.py index 1473e624d9e..36665a0d153 100644 --- a/xarray/core/datatree_io.py +++ b/xarray/core/datatree_io.py @@ -85,7 +85,7 @@ def _datatree_to_netcdf( unlimited_dims = {} for node in dt.subtree: - ds = node.ds + ds = node.to_dataset(inherited=False) group_path = node.path if ds is None: _create_empty_netcdf_group(filepath, group_path, mode, engine) @@ -151,7 +151,7 @@ def _datatree_to_zarr( ) for node in dt.subtree: - ds = node.ds + ds = node.to_dataset(inherited=False) group_path = node.path if ds is None: _create_empty_zarr_group(store, group_path, mode) diff --git a/xarray/core/formatting.py b/xarray/core/formatting.py index 5c4a3015843..6dca4eba8e8 100644 --- a/xarray/core/formatting.py +++ b/xarray/core/formatting.py @@ -748,6 +748,27 @@ def dataset_repr(ds): return "\n".join(summary) +def dims_and_coords_repr(ds) -> str: + """Partial Dataset repr for use inside DataTree inheritance errors.""" + summary = [] + + col_width = _calculate_col_width(ds.coords) + max_rows = OPTIONS["display_max_rows"] + + dims_start = pretty_print("Dimensions:", col_width) + dims_values = dim_summary_limited(ds, col_width=col_width + 1, max_rows=max_rows) + summary.append(f"{dims_start}({dims_values})") + + if ds.coords: + summary.append(coords_repr(ds.coords, col_width=col_width, max_rows=max_rows)) + + unindexed_dims_str = unindexed_dims_repr(ds.dims, ds.coords, max_rows=max_rows) + if unindexed_dims_str: + summary.append(unindexed_dims_str) + + return "\n".join(summary) + + def diff_dim_summary(a, b): if a.sizes != b.sizes: return f"Differing dimensions:\n ({dim_summary(a)}) != ({dim_summary(b)})" @@ -1030,7 +1051,7 @@ def diff_datatree_repr(a: DataTree, b: DataTree, compat): def _single_node_repr(node: DataTree) -> str: """Information about this node, not including its relationships to other nodes.""" if node.has_data or node.has_attrs: - ds_info = "\n" + repr(node.ds) + ds_info = "\n" + repr(node._to_dataset_view(rebuild_dims=False)) else: ds_info = "" return f"Group: {node.path}{ds_info}" diff --git a/xarray/core/formatting_html.py b/xarray/core/formatting_html.py index 9bf5befbe3f..24b290031eb 100644 --- a/xarray/core/formatting_html.py +++ b/xarray/core/formatting_html.py @@ -386,7 +386,7 @@ def summarize_datatree_children(children: Mapping[str, DataTree]) -> str: def datatree_node_repr(group_title: str, dt: DataTree) -> str: header_components = [f"
{escape(group_title)}
"] - ds = dt.ds + ds = dt._to_dataset_view(rebuild_dims=False) sections = [ children_section(dt.children), diff --git a/xarray/core/treenode.py b/xarray/core/treenode.py index 6f51e1ffa38..77e7ed23a51 100644 --- a/xarray/core/treenode.py +++ b/xarray/core/treenode.py @@ -138,14 +138,14 @@ def _attach(self, parent: Tree | None, child_name: str | None = None) -> None: "To directly set parent, child needs a name, but child is unnamed" ) - self._pre_attach(parent) + self._pre_attach(parent, child_name) parentchildren = parent._children assert not any( child is self for child in parentchildren ), "Tree is corrupt." parentchildren[child_name] = self self._parent = parent - self._post_attach(parent) + self._post_attach(parent, child_name) else: self._parent = None @@ -415,11 +415,11 @@ def _post_detach(self: Tree, parent: Tree) -> None: """Method call after detaching from `parent`.""" pass - def _pre_attach(self: Tree, parent: Tree) -> None: + def _pre_attach(self: Tree, parent: Tree, name: str) -> None: """Method call before attaching to `parent`.""" pass - def _post_attach(self: Tree, parent: Tree) -> None: + def _post_attach(self: Tree, parent: Tree, name: str) -> None: """Method call after attaching to `parent`.""" pass @@ -567,6 +567,9 @@ def same_tree(self, other: Tree) -> bool: return self.root is other.root +AnyNamedNode = TypeVar("AnyNamedNode", bound="NamedNode") + + class NamedNode(TreeNode, Generic[Tree]): """ A TreeNode which knows its own name. @@ -606,10 +609,9 @@ def __repr__(self, level=0): def __str__(self) -> str: return f"NamedNode('{self.name}')" if self.name else "NamedNode()" - def _post_attach(self: NamedNode, parent: NamedNode) -> None: + def _post_attach(self: AnyNamedNode, parent: AnyNamedNode, name: str) -> None: """Ensures child has name attribute corresponding to key under which it has been stored.""" - key = next(k for k, v in parent.children.items() if v is self) - self.name = key + self.name = name @property def path(self) -> str: diff --git a/xarray/tests/test_backends_datatree.py b/xarray/tests/test_backends_datatree.py index 4e819eec0b5..b4c4f481359 100644 --- a/xarray/tests/test_backends_datatree.py +++ b/xarray/tests/test_backends_datatree.py @@ -1,10 +1,12 @@ from __future__ import annotations -from typing import TYPE_CHECKING +from typing import TYPE_CHECKING, cast import pytest +import xarray as xr from xarray.backends.api import open_datatree +from xarray.core.datatree import DataTree from xarray.testing import assert_equal from xarray.tests import ( requires_h5netcdf, @@ -13,11 +15,11 @@ ) if TYPE_CHECKING: - from xarray.backends.api import T_NetcdfEngine + from xarray.core.datatree_io import T_DataTreeNetcdfEngine class DatatreeIOBase: - engine: T_NetcdfEngine | None = None + engine: T_DataTreeNetcdfEngine | None = None def test_to_netcdf(self, tmpdir, simple_datatree): filepath = tmpdir / "test.nc" @@ -27,6 +29,21 @@ def test_to_netcdf(self, tmpdir, simple_datatree): roundtrip_dt = open_datatree(filepath, engine=self.engine) assert_equal(original_dt, roundtrip_dt) + def test_to_netcdf_inherited_coords(self, tmpdir): + filepath = tmpdir / "test.nc" + original_dt = DataTree.from_dict( + { + "/": xr.Dataset({"a": (("x",), [1, 2])}, coords={"x": [3, 4]}), + "/sub": xr.Dataset({"b": (("x",), [5, 6])}), + } + ) + original_dt.to_netcdf(filepath, engine=self.engine) + + roundtrip_dt = open_datatree(filepath, engine=self.engine) + assert_equal(original_dt, roundtrip_dt) + subtree = cast(DataTree, roundtrip_dt["/sub"]) + assert "x" not in subtree.to_dataset(inherited=False).coords + def test_netcdf_encoding(self, tmpdir, simple_datatree): filepath = tmpdir / "test.nc" original_dt = simple_datatree @@ -48,12 +65,12 @@ def test_netcdf_encoding(self, tmpdir, simple_datatree): @requires_netCDF4 class TestNetCDF4DatatreeIO(DatatreeIOBase): - engine: T_NetcdfEngine | None = "netcdf4" + engine: T_DataTreeNetcdfEngine | None = "netcdf4" @requires_h5netcdf class TestH5NetCDFDatatreeIO(DatatreeIOBase): - engine: T_NetcdfEngine | None = "h5netcdf" + engine: T_DataTreeNetcdfEngine | None = "h5netcdf" @requires_zarr @@ -119,3 +136,18 @@ def test_to_zarr_default_write_mode(self, tmpdir, simple_datatree): # with default settings, to_zarr should not overwrite an existing dir with pytest.raises(zarr.errors.ContainsGroupError): simple_datatree.to_zarr(tmpdir) + + def test_to_zarr_inherited_coords(self, tmpdir): + original_dt = DataTree.from_dict( + { + "/": xr.Dataset({"a": (("x",), [1, 2])}, coords={"x": [3, 4]}), + "/sub": xr.Dataset({"b": (("x",), [5, 6])}), + } + ) + filepath = tmpdir / "test.zarr" + original_dt.to_zarr(filepath) + + roundtrip_dt = open_datatree(filepath, engine="zarr") + assert_equal(original_dt, roundtrip_dt) + subtree = cast(DataTree, roundtrip_dt["/sub"]) + assert "x" not in subtree.to_dataset(inherited=False).coords diff --git a/xarray/tests/test_datatree.py b/xarray/tests/test_datatree.py index b0dc2accd3e..f2b58fa2489 100644 --- a/xarray/tests/test_datatree.py +++ b/xarray/tests/test_datatree.py @@ -1,3 +1,5 @@ +import re +import typing from copy import copy, deepcopy from textwrap import dedent @@ -149,22 +151,37 @@ def test_is_hollow(self): assert not eve.is_hollow +class TestToDataset: + def test_to_dataset(self): + base = xr.Dataset(coords={"a": 1}) + sub = xr.Dataset(coords={"b": 2}) + tree = DataTree.from_dict({"/": base, "/sub": sub}) + subtree = typing.cast(DataTree, tree["sub"]) + + assert_identical(tree.to_dataset(inherited=False), base) + assert_identical(subtree.to_dataset(inherited=False), sub) + + sub_and_base = xr.Dataset(coords={"a": 1, "b": 2}) + assert_identical(tree.to_dataset(inherited=True), base) + assert_identical(subtree.to_dataset(inherited=True), sub_and_base) + + class TestVariablesChildrenNameCollisions: def test_parent_already_has_variable_with_childs_name(self): dt: DataTree = DataTree(data=xr.Dataset({"a": [0], "b": 1})) - with pytest.raises(KeyError, match="already contains a data variable named a"): + with pytest.raises(KeyError, match="already contains a variable named a"): DataTree(name="a", data=None, parent=dt) def test_assign_when_already_child_with_variables_name(self): dt: DataTree = DataTree(data=None) DataTree(name="a", data=None, parent=dt) - with pytest.raises(KeyError, match="names would collide"): + with pytest.raises(ValueError, match="node already contains a variable"): dt.ds = xr.Dataset({"a": 0}) # type: ignore[assignment] dt.ds = xr.Dataset() # type: ignore[assignment] new_ds = dt.to_dataset().assign(a=xr.DataArray(0)) - with pytest.raises(KeyError, match="names would collide"): + with pytest.raises(ValueError, match="node already contains a variable"): dt.ds = new_ds # type: ignore[assignment] @@ -561,7 +578,9 @@ def test_methods(self): def test_arithmetic(self, create_test_datatree): dt = create_test_datatree() - expected = create_test_datatree(modify=lambda ds: 10.0 * ds)["set1"] + expected = create_test_datatree(modify=lambda ds: 10.0 * ds)[ + "set1" + ].to_dataset() result = 10.0 * dt["set1"].ds assert result.identical(expected) @@ -648,13 +667,18 @@ def test_repr(self): │ Data variables: │ e (x) float64 16B 1.0 2.0 └── Group: /b - │ Dimensions: (y: 1) + │ Dimensions: (x: 2, y: 1) + │ Coordinates: + │ * x (x) float64 16B 2.0 3.0 │ Dimensions without coordinates: y │ Data variables: │ f (y) float64 8B 3.0 ├── Group: /b/c └── Group: /b/d - Dimensions: () + Dimensions: (x: 2, y: 1) + Coordinates: + * x (x) float64 16B 2.0 3.0 + Dimensions without coordinates: y Data variables: g float64 8B 4.0 """ @@ -666,13 +690,18 @@ def test_repr(self): """ Group: /b - │ Dimensions: (y: 1) + │ Dimensions: (x: 2, y: 1) + │ Coordinates: + │ * x (x) float64 16B 2.0 3.0 │ Dimensions without coordinates: y │ Data variables: │ f (y) float64 8B 3.0 ├── Group: /b/c └── Group: /b/d - Dimensions: () + Dimensions: (x: 2, y: 1) + Coordinates: + * x (x) float64 16B 2.0 3.0 + Dimensions without coordinates: y Data variables: g float64 8B 4.0 """ @@ -680,6 +709,192 @@ def test_repr(self): assert result == expected +def _exact_match(message: str) -> str: + return re.escape(dedent(message).strip()) + return "^" + re.escape(dedent(message.rstrip())) + "$" + + +class TestInheritance: + def test_inherited_dims(self): + dt = DataTree.from_dict( + { + "/": xr.Dataset({"d": (("x",), [1, 2])}), + "/b": xr.Dataset({"e": (("y",), [3])}), + "/c": xr.Dataset({"f": (("y",), [3, 4, 5])}), + } + ) + assert dt.sizes == {"x": 2} + # nodes should include inherited dimensions + assert dt.b.sizes == {"x": 2, "y": 1} + assert dt.c.sizes == {"x": 2, "y": 3} + # dataset objects created from nodes should not + assert dt.b.ds.sizes == {"y": 1} + assert dt.b.to_dataset(inherited=True).sizes == {"y": 1} + assert dt.b.to_dataset(inherited=False).sizes == {"y": 1} + + def test_inherited_coords_index(self): + dt = DataTree.from_dict( + { + "/": xr.Dataset({"d": (("x",), [1, 2])}, coords={"x": [2, 3]}), + "/b": xr.Dataset({"e": (("y",), [3])}), + } + ) + assert "x" in dt["/b"].indexes + assert "x" in dt["/b"].coords + xr.testing.assert_identical(dt["/x"], dt["/b/x"]) + + def test_inherited_coords_override(self): + dt = DataTree.from_dict( + { + "/": xr.Dataset(coords={"x": 1, "y": 2}), + "/b": xr.Dataset(coords={"x": 4, "z": 3}), + } + ) + assert dt.coords.keys() == {"x", "y"} + root_coords = {"x": 1, "y": 2} + sub_coords = {"x": 4, "y": 2, "z": 3} + xr.testing.assert_equal(dt["/x"], xr.DataArray(1, coords=root_coords)) + xr.testing.assert_equal(dt["/y"], xr.DataArray(2, coords=root_coords)) + assert dt["/b"].coords.keys() == {"x", "y", "z"} + xr.testing.assert_equal(dt["/b/x"], xr.DataArray(4, coords=sub_coords)) + xr.testing.assert_equal(dt["/b/y"], xr.DataArray(2, coords=sub_coords)) + xr.testing.assert_equal(dt["/b/z"], xr.DataArray(3, coords=sub_coords)) + + def test_inconsistent_dims(self): + expected_msg = _exact_match( + """ + group '/b' is not aligned with its parents: + Group: + Dimensions: (x: 1) + Dimensions without coordinates: x + Data variables: + c (x) float64 8B 3.0 + From parents: + Dimensions: (x: 2) + Dimensions without coordinates: x + """ + ) + + with pytest.raises(ValueError, match=expected_msg): + DataTree.from_dict( + { + "/": xr.Dataset({"a": (("x",), [1.0, 2.0])}), + "/b": xr.Dataset({"c": (("x",), [3.0])}), + } + ) + + dt: DataTree = DataTree() + dt["/a"] = xr.DataArray([1.0, 2.0], dims=["x"]) + with pytest.raises(ValueError, match=expected_msg): + dt["/b/c"] = xr.DataArray([3.0], dims=["x"]) + + b: DataTree = DataTree(data=xr.Dataset({"c": (("x",), [3.0])})) + with pytest.raises(ValueError, match=expected_msg): + DataTree( + data=xr.Dataset({"a": (("x",), [1.0, 2.0])}), + children={"b": b}, + ) + + def test_inconsistent_child_indexes(self): + expected_msg = _exact_match( + """ + group '/b' is not aligned with its parents: + Group: + Dimensions: (x: 1) + Coordinates: + * x (x) float64 8B 2.0 + Data variables: + *empty* + From parents: + Dimensions: (x: 1) + Coordinates: + * x (x) float64 8B 1.0 + """ + ) + + with pytest.raises(ValueError, match=expected_msg): + DataTree.from_dict( + { + "/": xr.Dataset(coords={"x": [1.0]}), + "/b": xr.Dataset(coords={"x": [2.0]}), + } + ) + + dt: DataTree = DataTree() + dt.ds = xr.Dataset(coords={"x": [1.0]}) # type: ignore + dt["/b"] = DataTree() + with pytest.raises(ValueError, match=expected_msg): + dt["/b"].ds = xr.Dataset(coords={"x": [2.0]}) + + b: DataTree = DataTree(xr.Dataset(coords={"x": [2.0]})) + with pytest.raises(ValueError, match=expected_msg): + DataTree(data=xr.Dataset(coords={"x": [1.0]}), children={"b": b}) + + def test_inconsistent_grandchild_indexes(self): + expected_msg = _exact_match( + """ + group '/b/c' is not aligned with its parents: + Group: + Dimensions: (x: 1) + Coordinates: + * x (x) float64 8B 2.0 + Data variables: + *empty* + From parents: + Dimensions: (x: 1) + Coordinates: + * x (x) float64 8B 1.0 + """ + ) + + with pytest.raises(ValueError, match=expected_msg): + DataTree.from_dict( + { + "/": xr.Dataset(coords={"x": [1.0]}), + "/b/c": xr.Dataset(coords={"x": [2.0]}), + } + ) + + dt: DataTree = DataTree() + dt.ds = xr.Dataset(coords={"x": [1.0]}) # type: ignore + dt["/b/c"] = DataTree() + with pytest.raises(ValueError, match=expected_msg): + dt["/b/c"].ds = xr.Dataset(coords={"x": [2.0]}) + + c: DataTree = DataTree(xr.Dataset(coords={"x": [2.0]})) + b: DataTree = DataTree(children={"c": c}) + with pytest.raises(ValueError, match=expected_msg): + DataTree(data=xr.Dataset(coords={"x": [1.0]}), children={"b": b}) + + def test_inconsistent_grandchild_dims(self): + expected_msg = _exact_match( + """ + group '/b/c' is not aligned with its parents: + Group: + Dimensions: (x: 1) + Dimensions without coordinates: x + Data variables: + d (x) float64 8B 3.0 + From parents: + Dimensions: (x: 2) + Dimensions without coordinates: x + """ + ) + + with pytest.raises(ValueError, match=expected_msg): + DataTree.from_dict( + { + "/": xr.Dataset({"a": (("x",), [1.0, 2.0])}), + "/b/c": xr.Dataset({"d": (("x",), [3.0])}), + } + ) + + dt: DataTree = DataTree() + dt["/a"] = xr.DataArray([1.0, 2.0], dims=["x"]) + with pytest.raises(ValueError, match=expected_msg): + dt["/b/c/d"] = xr.DataArray([3.0], dims=["x"]) + + class TestRestructuring: def test_drop_nodes(self): sue = DataTree.from_dict({"Mary": None, "Kate": None, "Ashley": None}) diff --git a/xarray/tests/test_utils.py b/xarray/tests/test_utils.py index 50061c774a8..ecec3ca507b 100644 --- a/xarray/tests/test_utils.py +++ b/xarray/tests/test_utils.py @@ -7,7 +7,11 @@ import pytest from xarray.core import duck_array_ops, utils -from xarray.core.utils import either_dict_or_kwargs, infix_dims, iterate_nested +from xarray.core.utils import ( + either_dict_or_kwargs, + infix_dims, + iterate_nested, +) from xarray.tests import assert_array_equal, requires_dask