-
Notifications
You must be signed in to change notification settings - Fork 8
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
Better documentation of behaviour with irregular source chunks #33
Comments
This should totally work! This happens internally in rechunking all the time, if the source and destination chunks are not a multiple of the other. If you can provide an reproducible example, that would be very helpful for narrowing this down.
You can do this via Xarray-Beam uses a smarter algorithm than splitting into the GCD of all source chunk sizes. Instead, it should only make just the right irregular chunks (in |
To give a concrete example, here's what it would look like to rechunk "per year" daily data into a chunk-size of exactly 365 days: import apache_beam as beam
import pandas as pd
import numpy as np
import xarray
import xarray_beam as xbeam
def make_chunks():
offset = 0
for year in range(2010, 2020):
time = pd.date_range(f'{year}-01-01', f'{year}-12-31', freq='D')
key = xbeam.Key({'time': offset})
chunk = xarray.Dataset({'foo': ('time', np.zeros(time.size)), 'time': time})
yield key, chunk
offset += time.size
print(
make_chunks()
| beam.MapTuple(lambda k, v: v.sizes['time'])
)
# [365, 365, 366, 365, 365, 365, 366, 365, 365, 365]
print(
make_chunks()
| xbeam.SplitChunks({'time': 365})
| beam.MapTuple(lambda k, v: v.sizes['time'])
)
# [365, 365, 365, 1, 364, 1, 364, 1, 364, 1, 364, 2, 363, 2, 363, 2, 363, 2]
print(
make_chunks()
| xbeam.SplitChunks({'time': 365})
| xbeam.ConsolidateChunks({'time': 365})
| beam.MapTuple(lambda k, v: v.sizes['time'])
)
# [365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 2] |
Ah that's great, thanks! This solves my problem, but just to give a bit more detail why I was confused about this: With With inputs = [
(xb.Key({"time": 0}, {"foo"}), xa.Dataset({"foo": (("time",), np.zeros(2))})),
(xb.Key({"time": 2}, {"foo"}), xa.Dataset({"foo": (("time",), np.ones(7))})),
(xb.Key({"time": 9}, {"foo"}), xa.Dataset({"foo": (("time",), np.ones(3))})),
]
with beam.Pipeline() as p:
p | beam.Create(inputs) | xb.ConsolidateChunks(target_chunks={"time": 7})
=> AssertionError: (Key(offsets={'time': 7}, vars={'foo'}), Key(offsets={'time': 9}, vars={'foo'})) My mistake although a better error would help here. I also wasn't aware that It would help if these functions ( Your example above would be great to include in the docs too, to demonstrate how to work with irregularly-chunked source data :) Thanks again! |
Indeed, I agree that this would all be well worth documenting! |
Thinking about this case a little more, I think we could actually probably safely remove this assertion error (after adding suitable documentation). The behavior of |
Hiya!
In some source datasets the original chunks have irregular sizes. For example one netcdf file per year, where some years are leap years hence shorter than others.
Although the data model would seem to support this in principle, I couldn't get Rechunk or ConsolidateChunks to work without regular source chunk sizes.
Is this a fundamental limitation or something that could be in scope to address? And do you have any recommendations for getting data like this into regular chunks? In the leap year case, first splitting the source chunks into one chunk per day using SplitChunks could be an option. Although in general this would require splitting into chunks of
gcd(*all_possible_source_chunk_sizes)
which could be too small.The text was updated successfully, but these errors were encountered: