-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Expand file tree
/
Copy pathyaml_testing.py
More file actions
560 lines (471 loc) · 18.2 KB
/
yaml_testing.py
File metadata and controls
560 lines (471 loc) · 18.2 KB
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
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
import collections
import functools
import json
import random
import unittest
import uuid
from typing import Dict
from typing import List
from typing import Mapping
from typing import Optional
from typing import Tuple
from typing import TypeVar
from typing import Union
import yaml
import apache_beam as beam
from apache_beam.testing.util import assert_that
from apache_beam.testing.util import equal_to
from apache_beam.yaml import yaml_provider
from apache_beam.yaml import yaml_transform
from apache_beam.yaml import yaml_utils
class YamlTestCase(unittest.TestCase):
def __init__(self, pipeline_spec, test_spec, options, fix_tests):
super().__init__()
self._pipeline_spec = pipeline_spec
self._test_spec = test_spec
self._options = options
self._fix_tests = fix_tests
self._fixes = None
def runTest(self):
self._fixes = run_test(
self._pipeline_spec, self._test_spec, self._options, self._fix_tests)
def fixed_test(self):
fixed_test_spec = yaml_transform.SafeLineLoader.strip_metadata(
self._test_spec)
if self._fixes:
expectation_by_id = {(loc, expectation['name']): expectation
for loc in ('expected_inputs', 'expected_outputs')
for expectation in fixed_test_spec.get(loc, [])}
for name_loc, values in self._fixes.items():
expectation_by_id[name_loc]['elements'] = sorted(values, key=json.dumps)
return fixed_test_spec
def id(self):
return (
self._test_spec.get('name', 'unknown') +
f' (line {yaml_transform.SafeLineLoader.get_line(self._test_spec)})')
def __str__(self):
return self.id()
def run_test(pipeline_spec, test_spec, options=None, fix_failures=False):
if isinstance(pipeline_spec, str):
pipeline_spec_dict = yaml.load(
pipeline_spec, Loader=yaml_utils.SafeLineLoader)
else:
pipeline_spec_dict = pipeline_spec
processed_pipeline_spec = _preprocess_for_testing(pipeline_spec_dict)
transform_spec, recording_ids = inject_test_tranforms(
processed_pipeline_spec,
test_spec,
fix_failures)
allowed_sources = set(test_spec.get('allowed_sources', []) + ['Create'])
for transform in transform_spec['transforms']:
name_or_type = transform.get('name', transform['type'])
if (not yaml_transform.empty_if_explicitly_empty(transform.get('input', []))
and not transform.get('name') in allowed_sources and
not transform['type'] in allowed_sources):
raise ValueError(
f'Non-mocked source {name_or_type} '
f'at line {yaml_transform.SafeLineLoader.get_line(transform)}')
if options is None:
options = beam.options.pipeline_options.PipelineOptions(
pickle_library='cloudpickle',
**yaml_transform.SafeLineLoader.strip_metadata(
pipeline_spec_dict.get('options', {})))
providers = yaml_provider.merge_providers(
yaml_provider.parse_providers(
'', pipeline_spec_dict.get('providers', [])),
{
'AssertEqualAndRecord': yaml_provider.as_provider_list(
'AssertEqualAndRecord', AssertEqualAndRecord)
})
with beam.Pipeline(options=options) as p:
_ = p | yaml_transform.YamlTransform(transform_spec, providers=providers)
if fix_failures:
fixes = {}
for recording_id in recording_ids:
if AssertEqualAndRecord.has_recorded_result(recording_id):
fixes[recording_id[1:]] = [
_try_row_as_dict(row)
for row in AssertEqualAndRecord.get_recorded_result(recording_id)
]
AssertEqualAndRecord.remove_recorded_result(recording_id)
return fixes
def _preprocess_for_testing(pipeline_spec):
spec = yaml_transform.pipeline_as_composite(pipeline_spec['pipeline'])
# These are idempotent, so it's OK to do them preemptively.
for phase in [
yaml_transform.ensure_transforms_have_types,
yaml_transform.preprocess_source_sink,
yaml_transform.preprocess_chain,
yaml_transform.tag_explicit_inputs,
yaml_transform.normalize_inputs_outputs,
]:
spec = yaml_transform.apply_phase(phase, spec)
return spec
def validate_test_spec(test_spec):
if not isinstance(test_spec, dict):
raise TypeError(
f'Test specification must be an object, got {type(test_spec)}')
identifier = (
test_spec.get('name', 'unknown') +
f' at line {yaml_transform.SafeLineLoader.get_line(test_spec)}')
if not isinstance(test_spec.get('allowed_sources', []), list):
raise TypeError(
f'allowed_sources of test specification {identifier} '
f'must be a list, got {type(test_spec["allowed_sources"])}')
if (not test_spec.get('expected_outputs', []) and
not test_spec.get('expected_inputs', [])):
raise ValueError(
f'test specification {identifier} '
f'must have at least one expected_outputs or expected_inputs')
unknown_attrs = set(
yaml_transform.SafeLineLoader.strip_metadata(test_spec).keys()) - set([
'name',
'mock_inputs',
'mock_outputs',
'expected_outputs',
'expected_inputs',
'allowed_sources',
])
if unknown_attrs:
raise ValueError(
f'test specification {identifier} '
f'has unknown attributes {list(unknown_attrs)}')
for attr_type in ('mock_inputs',
'mock_outputs',
'expected_outputs',
'expected_inputs'):
attr = test_spec.get(attr_type, [])
if not isinstance(attr, list):
raise TypeError(
f'{attr_type} of test specification {identifier} '
f'must be a list, got {type(attr_type)}')
for ix, attr_item in enumerate(attr):
if not isinstance(attr_item, dict):
raise TypeError(
f'{attr_type} {ix} of test specification {identifier} '
f'must be an object, got {type(attr_item)}')
if 'name' not in attr_item:
raise TypeError(
f'{attr_type} {ix} of test specification {identifier} '
f'missing a name')
if 'elements' not in attr_item:
raise TypeError(
f'{attr_type} {ix} of test specification {identifier} '
f'missing a elements')
if not isinstance(attr_item['elements'], list):
raise TypeError(
f'{attr_type} {ix} of test specification {identifier} '
f'must be a list, got {type(attr_item["elements"])}')
def inject_test_tranforms(spec, test_spec, fix_failures):
validate_test_spec(test_spec)
scope = yaml_transform.LightweightScope(spec['transforms'])
mocked_inputs_by_id = {
scope.get_transform_id(mock_input['name']): mock_input
for mock_input in test_spec.get('mock_inputs', [])
}
mocked_outputs_by_id = _composite_key_to_nested({
scope.get_transform_id_and_output_name(mock_output['name']): mock_output
for mock_output in test_spec.get('mock_outputs', [])
})
recording_id_prefix = str(uuid.uuid4())
recording_ids = []
transforms = []
@functools.cache
def create_inputs(transform_id: str) -> InputsType:
def require_output_or_outputs(name_or_names):
if isinstance(name_or_names, str):
return require_output(name_or_names)
else:
return [require_output(name) for name in name_or_names]
if transform_id in mocked_inputs_by_id:
return create_mocked_input(transform_id)
else:
input_spec = scope.get_transform_spec(transform_id)['input']
return {
tag: require_output_or_outputs(input_ref)
for tag, input_ref in yaml_transform.empty_if_explicitly_empty(
input_spec).items()
}
def require_output(name: str) -> str:
# The same output may be referenced under different names.
# Normalize before we cache.
transform_id, tag = scope.get_transform_id_and_output_name(name)
return _require_output(transform_id, tag) or name
@functools.cache
def _require_output(transform_id: str, tag: str) -> Optional[str]:
if transform_id in mocked_outputs_by_id:
if tag not in mocked_outputs_by_id[transform_id]:
name = next(iter(
mocked_outputs_by_id[transform_id].values()))['name'].split('.')[0]
raise ValueError(
f'Unmocked output {tag} of {name}.'
'If any used output is mocked all used outputs must be mocked.')
return create_mocked_output(transform_id, tag)
else:
_use_transform(transform_id)
return None # Use original name.
@functools.cache
def _use_transform(transform_id: str) -> None:
transform_spec = dict(scope.get_transform_spec(transform_id))
transform_spec['input'] = create_inputs(transform_id)
transforms.append(transform_spec)
@functools.cache
def create_mocked_input(transform_id: str) -> str:
transform = create_create(
f'MockInput[{mocked_inputs_by_id[transform_id]["name"]}]',
mocked_inputs_by_id[transform_id]['elements'],
mocked_inputs_by_id[transform_id]['name'])
transforms.append(transform)
return transform['__uuid__']
@functools.cache
def create_mocked_output(transform_id: str, tag: str) -> str:
transform = create_create(
f'MockOutput[{mocked_outputs_by_id[transform_id][tag]["name"]}]',
mocked_outputs_by_id[transform_id][tag]['elements'],
mocked_outputs_by_id[transform_id][tag]['name'])
transforms.append(transform)
return transform['__uuid__']
def create_create(name, elements, line_source):
return {
'__uuid__': yaml_utils.SafeLineLoader.create_uuid(),
'__line__': yaml_utils.SafeLineLoader.get_line(line_source),
'name': name,
'type': 'Create',
'config': {
'elements': elements,
},
}
def create_assertion(name, inputs, elements, recording_id, line_source):
return {
'__uuid__': yaml_utils.SafeLineLoader.create_uuid(),
'__line__': yaml_utils.SafeLineLoader.get_line(line_source),
'name': name,
'input': inputs,
'type': 'AssertEqualAndRecord',
'config': {
'elements': elements,
'recording_id': recording_id,
},
}
for expected_output in test_spec.get('expected_outputs', []):
if fix_failures:
recording_id = (
recording_id_prefix, 'expected_outputs', expected_output['name'])
recording_ids.append(recording_id)
else:
recording_id = None
require_output(expected_output['name'])
transforms.append(
create_assertion(
f'CheckExpectedOutput[{expected_output["name"]}]',
expected_output['name'],
expected_output['elements'],
recording_id,
expected_output['name']))
for expected_input in test_spec.get('expected_inputs', []):
if fix_failures:
recording_id = (
recording_id_prefix, 'expected_inputs', expected_input['name'])
recording_ids.append(recording_id)
else:
recording_id = None
transform_id = scope.get_transform_id(expected_input['name'])
transforms.append(
create_assertion(
f'CheckExpectedInput[{expected_input["name"]}]',
create_inputs(transform_id),
expected_input['elements'],
recording_id,
expected_input['name']))
return {
'__uuid__': yaml_utils.SafeLineLoader.create_uuid(),
'__line__': 0,
'type': 'composite',
'transforms': transforms,
}, recording_ids
class AssertEqualAndRecord(beam.PTransform):
_recorded_results = {}
@classmethod
def store_recorded_result(cls, recording_id, value):
assert recording_id not in cls._recorded_results
cls._recorded_results[recording_id] = value
@classmethod
def has_recorded_result(cls, recording_id):
return recording_id in cls._recorded_results
@classmethod
def get_recorded_result(cls, recording_id):
return cls._recorded_results[recording_id]
@classmethod
def remove_recorded_result(cls, recording_id):
del cls._recorded_results[recording_id]
def __init__(self, elements, recording_id):
self._elements = elements
self._recording_id = recording_id
def expand(self, pcoll):
# Convert elements to rows outside the matcher to avoid capturing
# any grpc channels that might be created during the conversion
expected_rows = yaml_provider.dicts_to_rows(self._elements)
recording_id = self._recording_id
# Create a serializable matcher function that doesn't capture
# any external references that might contain grpc channels
class SerializableMatcher:
def __init__(self, expected_rows, recording_id):
self.expected_rows = expected_rows
self.recording_id = recording_id
self.equal_to_matcher = equal_to(expected_rows)
def __call__(self, actual):
try:
self.equal_to_matcher(actual)
except Exception:
if self.recording_id:
AssertEqualAndRecord.store_recorded_result(
tuple(self.recording_id), actual)
else:
raise
matcher = SerializableMatcher(expected_rows, recording_id)
return assert_that(
pcoll | beam.Map(lambda row: beam.Row(**row._asdict())), matcher)
def create_test(
pipeline_spec, options=None, max_num_inputs=40, min_num_outputs=3):
if isinstance(pipeline_spec, str):
pipeline_spec = yaml.load(pipeline_spec, Loader=yaml_utils.SafeLineLoader)
transform_spec = _preprocess_for_testing(pipeline_spec)
if options is None:
options = beam.options.pipeline_options.PipelineOptions(
pickle_library='cloudpickle',
**yaml_transform.SafeLineLoader.strip_metadata(
pipeline_spec.get('options', {})))
providers = yaml_provider.merge_providers(
yaml_provider.parse_providers('', pipeline_spec.get('providers', [])),
{
'AssertEqualAndRecord': yaml_provider.as_provider_list(
'AssertEqualAndRecord', AssertEqualAndRecord)
})
def get_name(transform):
if 'name' in transform:
return str(transform['name'])
else:
if sum(1 for t in transform_spec['transforms']
if t['type'] == transform['type']) > 1:
raise ValueError('Ambiguous unnamed transform {transform["type"]}')
return str(transform['type'])
input_transforms = [
t for t in transform_spec['transforms'] if t['type'] != 'Create' and
not yaml_transform.empty_if_explicitly_empty(t.get('input', []))
]
mock_outputs = [{
'name': get_name(t),
'elements': [
_try_row_as_dict(row)
for row in _first_n(t, options, max_num_inputs, providers)
],
} for t in input_transforms]
output_transforms = [
t for t in transform_spec['transforms'] if t['type'] == 'LogForTesting' or
yaml_transform.empty_if_explicitly_empty(t.get('output', [])) or
t['type'].startswith('Write')
]
expected_inputs = [{
'name': get_name(t),
'elements': [],
} for t in output_transforms]
if not expected_inputs:
# TODO: Optionally take this as a parameter.
raise ValueError('No output transforms detected.')
num_inputs = min_num_outputs
while True:
test_spec = {
'mock_outputs': [{
'name': t['name'],
'elements': random.sample(
t['elements'], min(len(t['elements']), num_inputs)),
} for t in mock_outputs],
'expected_inputs': expected_inputs,
}
fixes = run_test(pipeline_spec, test_spec, options, fix_failures=True)
if len(fixes) < len(output_transforms):
actual_output_size = 0
else:
actual_output_size = min(len(e) for e in fixes.values())
if actual_output_size >= min_num_outputs:
break
elif num_inputs == max_num_inputs:
break
else:
num_inputs = min(2 * num_inputs, max_num_inputs)
for expected_input in test_spec['expected_inputs']:
if ('expected_inputs', expected_input['name']) in fixes:
expected_input['elements'] = fixes['expected_inputs',
expected_input['name']]
return test_spec
class _DoneException(Exception):
pass
class RecordElements(beam.PTransform):
_recorded_results = collections.defaultdict(list)
def __init__(self, n):
self._n = n
self._id = str(uuid.uuid4())
def get_and_remove(self):
listing = RecordElements._recorded_results[self._id]
del RecordElements._recorded_results[self._id]
return listing
def expand(self, pcoll):
def record(element):
listing = RecordElements._recorded_results[self._id]
if len(listing) < self._n:
listing.append(element)
else:
raise _DoneException()
return pcoll | beam.Map(record)
def _first_n(transform_spec, options, n, providers=None):
recorder = RecordElements(n)
if providers is None:
providers = {
'AssertEqualAndRecord': yaml_provider.as_provider_list(
'AssertEqualAndRecord', AssertEqualAndRecord)
}
try:
with beam.Pipeline(options=options) as p:
_ = (
p
| yaml_transform.YamlTransform(transform_spec, providers=providers)
| recorder)
except _DoneException:
pass
except Exception as exn:
# Runners don't always raise a faithful exception type.
if not '_DoneException' in str(exn):
raise
return recorder.get_and_remove()
K1 = TypeVar('K1')
K2 = TypeVar('K2')
V = TypeVar('V')
InputsType = Dict[str, Union[str, List[str]]]
def _composite_key_to_nested(
d: Mapping[Tuple[K1, K2], V]) -> Mapping[K1, Mapping[K2, V]]:
nested = collections.defaultdict(dict)
for (k1, k2), v in d.items():
nested[k1][k2] = v
return nested
def _try_row_as_dict(row):
try:
return row._asdict()
except AttributeError:
return row
# Linter: No need for unittest.main here.