-
Notifications
You must be signed in to change notification settings - Fork 54
/
test_rechunking.py
260 lines (214 loc) · 10.3 KB
/
test_rechunking.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
import itertools
import random
from collections import namedtuple
import pytest
import xarray as xr
from pangeo_forge_recipes.rechunking import GroupKey, combine_fragments, split_fragment
from pangeo_forge_recipes.types import CombineOp, Dimension, Index, IndexedPosition, Position
from .conftest import split_up_files_by_variable_and_day
from .data_generation import make_ds
@pytest.mark.parametrize(
"nt_dayparam",
[(5, "1D"), (10, "2D")],
)
@pytest.mark.parametrize("time_chunks", [1, 2, 5])
def test_split_and_combine_fragments_with_merge_dim(nt_dayparam, time_chunks):
"""Test if sub-fragments split from datasets with merge dims can be combined with each other."""
target_chunks = {"time": time_chunks}
nt, dayparam = nt_dayparam
ds = make_ds(nt=nt)
dsets, _, _ = split_up_files_by_variable_and_day(ds, dayparam)
# replicates indexes created by IndexItems transform.
time_positions = {t: i for i, t in enumerate(ds.time.values)}
merge_dim = Dimension("variable", CombineOp.MERGE)
concat_dim = Dimension("time", CombineOp.CONCAT)
indexes = [
Index(
{
merge_dim: Position((0 if "bar" in ds.data_vars else 1)),
concat_dim: IndexedPosition(time_positions[ds.time[0].values], dimsize=nt),
}
)
for ds in dsets
]
# split the (mock indexed) datasets into sub-fragments.
# the splits list are nested tuples which are a bit confusing for humans to think about.
# create a namedtuple to help remember the structure of these tuples and cast the
# elements of splits list to this more descriptive type.
splits = [
list(split_fragment((index, ds), target_chunks=target_chunks))
for index, ds in zip(indexes, dsets)
]
Subfragment = namedtuple("Subfragment", "groupkey, content")
subfragments = list(itertools.chain(*[[Subfragment(*s) for s in split] for split in splits]))
# combine subfragments, starting by grouping subfragments by groupkey.
# replicates behavior of `... | beam.GroupByKey() | beam.MapTuple(combine_fragments)`
# in the `Rechunk` transform.
groupkeys = set([sf.groupkey for sf in subfragments])
grouped_subfragments: dict[GroupKey, list[Subfragment]] = {g: [] for g in groupkeys}
for sf in subfragments:
grouped_subfragments[sf.groupkey].append(sf)
for g in sorted(groupkeys):
# just confirms that grouping logic within this test is correct
assert all([sf.groupkey == g for sf in grouped_subfragments[g]])
# for the merge dimension of each subfragment in the current group, assert that there
# is only one positional value present. this verifies that `split_fragments` has not
# grouped distinct merge dimension positional values together under the same groupkey.
merge_position_vals = [sf.content[0][merge_dim].value for sf in grouped_subfragments[g]]
assert all([v == merge_position_vals[0] for v in merge_position_vals])
# now actually try to combine the fragments
_, ds_combined = combine_fragments(
g,
[sf.content for sf in grouped_subfragments[g]],
)
# ensure vars are *not* combined (we only want to concat, not merge)
assert len([k for k in ds_combined.data_vars.keys()]) == 1
# check that time chunking is correct
if nt % time_chunks == 0:
assert len(ds_combined.time) == time_chunks
else:
# if `nt` is not evenly divisible by `time_chunks`, all chunks will be of
# `len(time_chunks)` except the last one, which will be the lenth of the remainder
assert len(ds_combined.time) in [time_chunks, nt % time_chunks]
@pytest.mark.parametrize("offset", [0, 5]) # hypothetical offset of this fragment
@pytest.mark.parametrize("time_chunks", [1, 3, 5, 10, 11])
def test_split_fragment(time_chunks, offset):
"""A thorough test of 1D splitting logic that should cover all major edge cases."""
nt_total = 20 # the total size of the hypothetical dataset
target_chunks = {"time": time_chunks}
nt = 10
ds = make_ds(nt=nt) # this represents a single dataset fragment
dimension = Dimension("time", CombineOp.CONCAT)
extra_indexes = [
(Dimension("foo", CombineOp.CONCAT), Position(0)),
(Dimension("bar", CombineOp.MERGE), Position(1)),
]
index = Index([(dimension, IndexedPosition(offset, dimsize=nt_total))] + extra_indexes)
all_splits = list(split_fragment((index, ds), target_chunks=target_chunks))
group_keys = [item[0] for item in all_splits]
new_indexes = [item[1][0] for item in all_splits]
new_datasets = [item[1][1] for item in all_splits]
for n in range(len(all_splits)):
chunk_number = offset // time_chunks + n
assert group_keys[n] == (("time", chunk_number), ("bar", 1))
chunk_start = time_chunks * chunk_number
chunk_stop = min(time_chunks * (chunk_number + 1), nt_total)
fragment_start = max(chunk_start, offset)
fragment_stop = min(chunk_stop, fragment_start + time_chunks, offset + nt)
# other dimensions in the index should be passed through unchanged
assert new_indexes[n] == Index(
[(dimension, IndexedPosition(fragment_start, dimsize=nt_total))] + extra_indexes
)
start, stop = fragment_start - offset, fragment_stop - offset
xr.testing.assert_equal(new_datasets[n], ds.isel(time=slice(start, stop)))
# make sure we got the whole dataset back
ds_concat = xr.concat(new_datasets, "time")
xr.testing.assert_equal(ds, ds_concat)
def test_split_multidim():
"""A simple test that checks whether splitting logic is applied correctly
for multiple dimensions."""
nt = 2
ds = make_ds(nt=nt)
nlat = ds.dims["lat"]
dimension = Dimension("time", CombineOp.CONCAT)
index = Index({dimension: IndexedPosition(0, dimsize=nt)})
time_chunks = 1
lat_chunks = nlat // 2
target_chunks = {"time": time_chunks, "lat": lat_chunks}
all_splits = list(split_fragment((index, ds), target_chunks=target_chunks))
group_keys = [item[0] for item in all_splits]
assert group_keys == [
(("lat", 0), ("time", 0)),
(("lat", 1), ("time", 0)),
(("lat", 0), ("time", 1)),
(("lat", 1), ("time", 1)),
]
for group_key, (fragment_index, fragment_ds) in all_splits:
n_lat_chunk = group_key[0][1]
n_time_chunk = group_key[1][1]
time_start, time_stop = n_time_chunk * time_chunks, (n_time_chunk + 1) * time_chunks
lat_start, lat_stop = n_lat_chunk * lat_chunks, (n_lat_chunk + 1) * lat_chunks
expected_index = Index(
{
Dimension("time", CombineOp.CONCAT): IndexedPosition(time_start, dimsize=nt),
Dimension("lat", CombineOp.CONCAT): IndexedPosition(lat_start, dimsize=nlat),
}
)
assert fragment_index == expected_index
xr.testing.assert_equal(
fragment_ds, ds.isel(time=slice(time_start, time_stop), lat=slice(lat_start, lat_stop))
)
@pytest.mark.parametrize("time_chunk", [1, 2, 3, 5, 10])
def test_combine_fragments(time_chunk):
"""The function applied after GroupBy to combine fragments into a single chunk.
All concat dims that appear more than once are combined.
"""
nt = 10
ds = make_ds(nt=nt)
fragments = []
time_dim = Dimension("time", CombineOp.CONCAT)
for nfrag, start in enumerate(range(0, nt, time_chunk)):
stop = min(start + time_chunk, nt)
index_frag = Index({time_dim: IndexedPosition(start)})
ds_frag = ds.isel(time=slice(start, stop))
fragments.append((index_frag, ds_frag))
group = (("time", 0),) # not actually used
index, ds_comb = combine_fragments(group, fragments)
assert index == Index({time_dim: IndexedPosition(0)})
xr.testing.assert_equal(ds, ds_comb)
@pytest.mark.parametrize("time_chunk", [1, 2, 3, 5, 10])
@pytest.mark.parametrize("lat_chunk", [8, 9, 17, 18])
def test_combine_fragments_multidim(time_chunk, lat_chunk):
"""The function applied after GroupBy to combine fragments into a single chunk.
All concat dims that appear more than once are combined.
"""
nt = 10
ds = make_ds(nt=nt)
ny = ds.dims["lat"]
fragments = []
time_dim = Dimension("time", CombineOp.CONCAT)
lat_dim = Dimension("lat", CombineOp.CONCAT)
for start_t in range(0, nt, time_chunk):
stop_t = min(start_t + time_chunk, nt)
for start_y in range(0, ny, lat_chunk):
stop_y = min(start_y + lat_chunk, ny)
ds_frag = ds.isel(time=slice(start_t, stop_t), lat=slice(start_y, stop_y))
index_frag = Index(
{time_dim: IndexedPosition(start_t), lat_dim: IndexedPosition(start_y)}
)
fragments.append((index_frag, ds_frag))
# fragments will arrive in a random order
random.shuffle(fragments)
group = (("time", 0), ("lat", 0)) # not actually used
index, ds_comb = combine_fragments(group, fragments)
assert index == Index({time_dim: IndexedPosition(0), lat_dim: IndexedPosition(0)})
xr.testing.assert_equal(ds, ds_comb)
def test_combine_fragments_errors():
ds = make_ds(nt=1)
group = (("time", 0),) # not actually used
# check for inconsistent indexes
index0 = Index({Dimension("time", CombineOp.CONCAT): IndexedPosition(0)})
bad_indexes = [
Index(
{
Dimension("timestep", CombineOp.CONCAT): IndexedPosition(1),
}
),
Index(
{
Dimension("time", CombineOp.CONCAT): IndexedPosition(1),
Dimension("variable", CombineOp.MERGE): Position(0),
}
),
]
for index1 in bad_indexes:
with pytest.raises(ValueError, match="different combine dims"):
_ = combine_fragments(group, [(index0, ds), (index1, ds)])
# check for missing start stop
index1 = Index({Dimension("time", CombineOp.CONCAT): Position(1)})
with pytest.raises(ValueError, match="positions must be indexed"):
_ = combine_fragments(group, [(index0, ds), (index1, ds)])
# check for non-contiguous indexes
index1 = Index({Dimension("time", CombineOp.CONCAT): IndexedPosition(2)})
with pytest.raises(ValueError, match="are not consistent"):
_ = combine_fragments(group, [(index0, ds), (index1, ds)])