def test_rechunk_not_all_dimensions(self): data = np.random.RandomState(0).randint(2**30, size=(10, 20, 30)) ds = xarray.Dataset({'foo': (('time', 'x', 'y'), data)}) key = xbeam.Key({'x': 0, 'y': 0}) y_split_with_time_key = ([(key.with_offsets(time=0), ds)] | xbeam.SplitChunks({'y': 3})) x_split = [(key, ds)] | xbeam.SplitChunks({'x': 2}) actual = x_split | xbeam.Rechunk( dim_sizes=ds.sizes, source_chunks={ 'x': 2, 'y': -1 }, target_chunks={ 'x': -1, 'y': 3 }, itemsize=8, max_mem=10_000, ) self.assertIdenticalChunks(actual, y_split_with_time_key) with self.assertRaisesRegex( ValueError, 'source_chunks and target_chunks have different keys', ): xbeam.Rechunk( dim_sizes=ds.sizes, source_chunks={'x': 2}, target_chunks={'y': 3}, itemsize=8, max_mem=10_000, )
def test_rechunk_end_to_end(self): data = np.random.RandomState(0).randint(2**30, size=(10, 20, 30)) ds = xarray.Dataset({'foo': (('time', 'x', 'y'), data)}) key = xbeam.Key({'time': 0, 'x': 0, 'y': 0}) time_split = [(key, ds)] | xbeam.SplitChunks({'time': 1}) space_split = [(key, ds)] | xbeam.SplitChunks({'x': 5, 'y': 5}) with self.subTest('time-to-space'): actual = time_split | xbeam.Rechunk( dim_sizes=ds.sizes, source_chunks={ 'time': 1, 'x': 20, 'y': 30 }, target_chunks={ 'time': 10, 'x': 5, 'y': 5 }, itemsize=8, max_mem=10_000, ) self.assertIdenticalChunks(actual, space_split) with self.subTest('space-to-time'): actual = space_split | xbeam.Rechunk( dim_sizes=ds.sizes, source_chunks={ 'time': 10, 'x': 5, 'y': 5 }, target_chunks={ 'time': 1, 'x': 20, 'y': 30 }, itemsize=8, max_mem=10_000, ) self.assertIdenticalChunks(actual, time_split)
def test_rechunk_1d(self, size, max_mem, source_chunks, target_chunks): data = np.random.RandomState(0).randint(2**30, size=(size, )) ds = xarray.Dataset({'foo': ('x', data)}) key = xbeam.Key({'x': 0}) inputs = [(key, ds)] | xbeam.SplitChunks({'x': source_chunks}) expected = [(key, ds)] | xbeam.SplitChunks({'x': target_chunks}) actual = inputs | xbeam.Rechunk( dim_sizes=ds.sizes, source_chunks={'x': source_chunks}, target_chunks={'x': target_chunks}, itemsize=1, max_mem=max_mem, ) self.assertIdenticalChunks(actual, expected)
def test_rechunk_zarr_to_zarr(self, template_method, split_vars): src_dir = self.create_tempdir('source').full_path dest_dir = self.create_tempdir('destination').full_path source_chunks = {'t': 1, 'x': 100, 'y': 120} target_chunks = {'t': -1, 'x': 20, 'y': 20} rs = np.random.RandomState(0) raw_data = rs.randint(2**30, size=(60, 100, 120)) # 5.76 MB dataset = xarray.Dataset({ 'foo': (('t', 'x', 'y'), raw_data), 'bar': (('t', 'x', 'y'), raw_data - 1), }) dataset.chunk(source_chunks).to_zarr(src_dir, consolidated=True) on_disk = xarray.open_zarr(src_dir, consolidated=True) on_disk_chunked = on_disk.chunk(target_chunks) with beam.Pipeline('DirectRunner') as pipeline: # make template if template_method == 'eager': target_template = on_disk_chunked elif template_method == 'lazy': target_template = beam.pvalue.AsSingleton( pipeline | beam.Create([on_disk_chunked])) elif template_method == 'infer': target_template = None # run pipeline (pipeline | xbeam.DatasetToChunks(on_disk, split_vars=split_vars) | xbeam.Rechunk( on_disk.sizes, source_chunks, target_chunks, itemsize=8, max_mem=10_000_000, # require two stages ) | xbeam.ChunksToZarr(dest_dir, target_template)) roundtripped = xarray.open_zarr(dest_dir, consolidated=True, chunks=False) xarray.testing.assert_identical(roundtripped, dataset)
def test_rechunk_uneven_2d(self): data = np.random.RandomState(0).randint(2**30, size=(100, 100)) ds = xarray.Dataset({'foo': (('x', 'y'), data)}) key = xbeam.Key({'x': 0, 'y': 0}) inputs = [(key, ds)] | xbeam.SplitChunks({'x': 12}) expected = [(key, ds)] | xbeam.SplitChunks({'y': 15}) actual = inputs | xbeam.Rechunk( dim_sizes=ds.sizes, source_chunks={ 'x': 12, 'y': -1 }, target_chunks={ 'x': -1, 'y': 15 }, itemsize=1, max_mem=100 * 100 // 2, # half the full size ) self.assertIdenticalChunks(actual, expected)
def main(argv): source_dataset = xarray.open_zarr( INPUT_PATH.value, chunks=None, consolidated=True ) template = xarray.zeros_like(source_dataset.chunk()) source_chunks = {'latitude': -1, 'longitude': -1, 'time': 31} target_chunks = {'latitude': 5, 'longitude': 5, 'time': -1} with beam.Pipeline(runner=RUNNER.value, argv=argv) as root: ( root # Note: splitting across the 19 variables in this dataset is a critical # optimization step here, because it allows rechunking to make use of # much larger intermediate chunks. | xbeam.DatasetToChunks(source_dataset, source_chunks, split_vars=True) | xbeam.Rechunk( source_dataset.sizes, source_chunks, target_chunks, itemsize=4, ) | xbeam.ChunksToZarr(OUTPUT_PATH.value, template, target_chunks) )