-
Notifications
You must be signed in to change notification settings - Fork 54
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
ConsolidatedDimensionCoords Transform -- cleaned up version of PR #556 #671
Changes from all commits
06e475c
d613934
3d67039
6c39134
b645d9a
8f5bfe4
4ec50f1
4d58722
910920b
792d40a
4f26242
cde3cfe
fbe8859
f1d43ab
22e9f42
4d986a4
5b19225
36b4faa
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -21,7 +21,7 @@ | |
from .combiners import CombineMultiZarrToZarr, CombineXarraySchemas | ||
from .openers import open_url, open_with_kerchunk, open_with_xarray | ||
from .patterns import CombineOp, Dimension, FileType, Index, augment_index_with_start_stop | ||
from .rechunking import combine_fragments, split_fragment | ||
from .rechunking import combine_fragments, consolidate_dimension_coordinates, split_fragment | ||
from .storage import CacheFSSpecTarget, FSSpecTarget | ||
from .writers import ( | ||
ZarrWriterMixin, | ||
|
@@ -364,9 +364,14 @@ class PrepareZarrTarget(beam.PTransform): | |
If chunking is present in the schema for a given dimension, the length of | ||
the first fragment will be used. Otherwise, the dimension will not be chunked. | ||
:param attrs: Extra group-level attributes to inject into the dataset. | ||
:param consolidated_metadata: Bool controlling if xarray.to_zarr() | ||
writes consolidated metadata. Default's to True. | ||
:param encoding: Dictionary describing encoding for xarray.to_zarr() | ||
:param consolidated_metadata: Bool controlling if xarray.to_zarr() | ||
writes consolidated metadata. Default's to False. In StoreToZarr, | ||
always default to unconsolidated. This leaves it up to the | ||
user whether or not they want to consolidate with ConsolidateMetadata(). Also, | ||
it prevents a broken/inconsistent state that could arise from metadata being | ||
consolidated here, and then falling out of sync with coordinates if | ||
ConsolidateDimensionCoordinates() is applied to the output of StoreToZarr(). | ||
""" | ||
|
||
target: str | FSSpecTarget | ||
|
@@ -386,8 +391,8 @@ def expand(self, pcoll: beam.PCollection) -> beam.PCollection: | |
target_store=store, | ||
target_chunks=self.target_chunks, | ||
attrs=self.attrs, | ||
consolidated_metadata=self.consolidated_metadata, | ||
encoding=self.encoding, | ||
consolidated_metadata=False, | ||
) | ||
return initialized_target | ||
|
||
|
@@ -430,6 +435,13 @@ def expand(self, pcoll: beam.PCollection) -> beam.PCollection: | |
return new_fragments | ||
|
||
|
||
class ConsolidateDimensionCoordinates(beam.PTransform): | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can we add a docstring here? |
||
def expand( | ||
self, pcoll: beam.PCollection[zarr.storage.FSStore] | ||
) -> beam.PCollection[zarr.storage.FSStore]: | ||
return pcoll | beam.Map(consolidate_dimension_coordinates) | ||
|
||
|
||
@dataclass | ||
class CombineReferences(beam.PTransform): | ||
"""Combines Kerchunk references into a single reference dataset. | ||
|
@@ -573,6 +585,9 @@ class StoreToZarr(beam.PTransform, ZarrWriterMixin): | |
`store_name` will be appended to this prefix to create a full path. | ||
:param target_chunks: Dictionary mapping dimension names to chunks sizes. | ||
If a dimension is a not named, the chunks will be inferred from the data. | ||
:param consolidate_dimension_coordinates: Whether to rewrite coordinate variables as a | ||
single chunk. We recommend consolidating coordinate variables to avoid | ||
many small read requests to get the coordinates in xarray. Defaults to ``True``. | ||
Comment on lines
+588
to
+590
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Point of clarification for my own understanding: do we know for sure that this make a meaningful difference for kerchunk datasets? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Good question! I don't actually know. Maybe @abarciauskas-bgse has some insight into this? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I don't know if there is a significant difference if we consolidate dimension coordinates in kerchunk datasets. I experienced slow performance when working with zarr stores and saw a relationship between performance (specifically for creating image tiles using rio_tiler's XarrayReader) and the number of coordinate chunks. The notebook which demonstrates this is unfortunately not a brief one (https://github.com/developmentseed/tile-benchmarking/blob/9758c732597c154d1cf7cd796b21c858c3130046/profiling/profile.ipynb) but I could streamline it if we need to do more investigation. For kerchunk, I would guess there would be a performance impact if a coordinate's chunks are all read all at once or in multiple reads. If it's in multiple reads, it still might impact performance. Do we know which it is? Or should I investigate/run some more tests? |
||
:param dynamic_chunking_fn: Optionally provide a function that takes an ``xarray.Dataset`` | ||
template dataset as its first argument and returns a dynamically generated chunking dict. | ||
If provided, ``target_chunks`` cannot also be passed. You can use this to determine chunking | ||
|
@@ -581,8 +596,7 @@ class StoreToZarr(beam.PTransform, ZarrWriterMixin): | |
out https://github.com/jbusecke/dynamic_chunks | ||
:param dynamic_chunking_fn_kwargs: Optional keyword arguments for ``dynamic_chunking_fn``. | ||
:param attrs: Extra group-level attributes to inject into the dataset. | ||
:param consolidated_metadata: Bool controlling if xarray.to_zarr() | ||
writes consolidated metadata. Default's to True. | ||
|
||
:param encoding: Dictionary encoding for xarray.to_zarr(). | ||
""" | ||
|
||
|
@@ -597,7 +611,6 @@ class StoreToZarr(beam.PTransform, ZarrWriterMixin): | |
dynamic_chunking_fn: Optional[Callable[[xr.Dataset], dict]] = None | ||
dynamic_chunking_fn_kwargs: Optional[dict] = field(default_factory=dict) | ||
attrs: Dict[str, str] = field(default_factory=dict) | ||
consolidated_metadata: Optional[bool] = True | ||
encoding: Optional[dict] = field(default_factory=dict) | ||
|
||
def __post_init__(self): | ||
|
@@ -624,7 +637,6 @@ def expand( | |
target=self.get_full_target(), | ||
target_chunks=target_chunks, | ||
attrs=self.attrs, | ||
consolidated_metadata=self.consolidated_metadata, | ||
encoding=self.encoding, | ||
) | ||
n_target_stores = rechunked_datasets | StoreDatasetFragments(target_store=target_store) | ||
|
@@ -633,7 +645,5 @@ def expand( | |
| beam.combiners.Sample.FixedSizeGlobally(1) | ||
| beam.FlatMap(lambda x: x) # https://stackoverflow.com/a/47146582 | ||
) | ||
# TODO: optionally use `singleton_target_store` to | ||
# consolidate metadata and/or coordinate dims here | ||
|
||
return singleton_target_store |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I imagine since this is a
zarr.Group
this iteration isn't very expensive b/c it's on summary data and never really that large?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ehh, I think you're right? Since it's just scanning across vars in groups and looking at the .attrs.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't fully understand this function but the fact that it's the same as we had in the pre-Beam release checks out to me!
pangeo-forge-recipes/pangeo_forge_recipes/recipes/xarray_zarr.py
Line 655 in 3b3c13c