|
19 | 19 | import glob |
20 | 20 | import logging |
21 | 21 | import os |
| 22 | +import shutil |
22 | 23 | import tempfile |
23 | 24 | import unittest |
24 | 25 |
|
@@ -911,6 +912,60 @@ def test_must_handle_error_output(self): |
911 | 912 | ''', |
912 | 913 | providers=TEST_PROVIDERS) |
913 | 914 |
|
| 915 | + def test_error_handling_log_combined_errors(self): |
| 916 | + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( |
| 917 | + pickle_library='cloudpickle')) as p: |
| 918 | + result = p | YamlTransform( |
| 919 | + ''' |
| 920 | + type: composite |
| 921 | + transforms: |
| 922 | + - type: Create |
| 923 | + name: Input1 |
| 924 | + config: |
| 925 | + elements: [1, 2, 0] |
| 926 | + - type: Create |
| 927 | + name: Input2 |
| 928 | + config: |
| 929 | + elements: [3, 'a', 5] |
| 930 | + - type: MapToFields |
| 931 | + name: Inverse |
| 932 | + input: Input1 |
| 933 | + config: |
| 934 | + language: python |
| 935 | + fields: |
| 936 | + inverse: "1 / element" |
| 937 | + error_handling: |
| 938 | + output: errors |
| 939 | + - type: MapToFields |
| 940 | + name: Square |
| 941 | + input: Input2 |
| 942 | + config: |
| 943 | + language: python |
| 944 | + fields: |
| 945 | + square: "element * element" |
| 946 | + error_handling: |
| 947 | + output: errors |
| 948 | + - type: LogForTesting |
| 949 | + input: |
| 950 | + - Inverse.errors |
| 951 | + - Square.errors |
| 952 | + - type: Flatten |
| 953 | + name: GoodData |
| 954 | + input: |
| 955 | + - Inverse |
| 956 | + - Square |
| 957 | + output: GoodData |
| 958 | + ''', |
| 959 | + providers=TEST_PROVIDERS) |
| 960 | + assert_that( |
| 961 | + result, |
| 962 | + equal_to([ |
| 963 | + beam.Row(inverse=1.0, square=None), |
| 964 | + beam.Row(inverse=0.5, square=None), |
| 965 | + beam.Row(square=9, inverse=None), |
| 966 | + beam.Row(square=25, inverse=None) |
| 967 | + ])) |
| 968 | + |
914 | 969 | def test_mapping_errors(self): |
915 | 970 | with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( |
916 | 971 | pickle_library='cloudpickle')) as p: |
@@ -1297,6 +1352,106 @@ def test_prefers_same_provider_class(self): |
1297 | 1352 | label='StartWith3') |
1298 | 1353 |
|
1299 | 1354 |
|
| 1355 | +class TestExternalYamlProvider(unittest.TestCase): |
| 1356 | + def setUp(self): |
| 1357 | + self.temp_dir = tempfile.mkdtemp() |
| 1358 | + self.provider_path = os.path.join(self.temp_dir, 'power_provider.yaml') |
| 1359 | + with open(self.provider_path, 'w') as f: |
| 1360 | + f.write( |
| 1361 | + """ |
| 1362 | +- type: yaml |
| 1363 | + transforms: |
| 1364 | + RaiseElementToPower: |
| 1365 | + config_schema: |
| 1366 | + properties: |
| 1367 | + n: {type: integer} |
| 1368 | + body: |
| 1369 | + type: MapToFields |
| 1370 | + config: |
| 1371 | + language: python |
| 1372 | + append: true |
| 1373 | + fields: |
| 1374 | + power: "element ** {{n}}" |
| 1375 | + error_handling: |
| 1376 | + output: my_error |
| 1377 | +""") |
| 1378 | + |
| 1379 | + def tearDown(self): |
| 1380 | + shutil.rmtree(self.temp_dir) |
| 1381 | + |
| 1382 | + def test_provider_with_error_handling(self): |
| 1383 | + loaded_providers = yaml_provider.load_providers(self.provider_path) |
| 1384 | + test_providers = yaml_provider.InlineProvider(TEST_PROVIDERS) |
| 1385 | + merged_providers = yaml_provider.merge_providers( |
| 1386 | + loaded_providers, [test_providers]) |
| 1387 | + |
| 1388 | + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( |
| 1389 | + pickle_library='cloudpickle')) as p: |
| 1390 | + results = p | YamlTransform( |
| 1391 | + ''' |
| 1392 | + type: composite |
| 1393 | + transforms: |
| 1394 | + - type: Create |
| 1395 | + config: |
| 1396 | + elements: [2, 'bad', 3] |
| 1397 | + - type: RaiseElementToPower |
| 1398 | + input: Create |
| 1399 | + config: |
| 1400 | + n: 2 |
| 1401 | + - type: PyMap |
| 1402 | + name: TrimErrors |
| 1403 | + input: RaiseElementToPower.my_error |
| 1404 | + config: |
| 1405 | + fn: "lambda x: x.msg" |
| 1406 | + output: |
| 1407 | + good: RaiseElementToPower.good |
| 1408 | + bad: TrimErrors |
| 1409 | + ''', |
| 1410 | + providers=merged_providers) |
| 1411 | + |
| 1412 | + assert_that( |
| 1413 | + results['good'], |
| 1414 | + equal_to([beam.Row(element=2, power=4), beam.Row(element=3, |
| 1415 | + power=9)]), |
| 1416 | + label="CheckGood") |
| 1417 | + assert_that( |
| 1418 | + results['bad'], |
| 1419 | + equal_to([ |
| 1420 | + 'TypeError("unsupported operand type(s) for ** or pow(): ' + |
| 1421 | + '\'str\' and \'int\'")' |
| 1422 | + ]), |
| 1423 | + label="CheckBad") |
| 1424 | + |
| 1425 | + def test_must_consume_error_output(self): |
| 1426 | + # By adding a dummy error_handling block here, we signal to the static |
| 1427 | + # checker that this transform has an error output that must be consumed. |
| 1428 | + # The framework is able to handle the "nesting" where the provider for |
| 1429 | + # RaiseElementToPower also defines error handling internally. |
| 1430 | + loaded_providers = yaml_provider.load_providers(self.provider_path) |
| 1431 | + test_providers = yaml_provider.InlineProvider(TEST_PROVIDERS) |
| 1432 | + merged_providers = yaml_provider.merge_providers( |
| 1433 | + loaded_providers, [test_providers]) |
| 1434 | + |
| 1435 | + with self.assertRaisesRegex(Exception, 'Unconsumed error output.*'): |
| 1436 | + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( |
| 1437 | + pickle_library='cloudpickle')) as p: |
| 1438 | + _ = p | YamlTransform( |
| 1439 | + ''' |
| 1440 | + type: composite |
| 1441 | + transforms: |
| 1442 | + - type: Create |
| 1443 | + config: |
| 1444 | + elements: [2, 'bad', 3] |
| 1445 | + - type: RaiseElementToPower |
| 1446 | + input: Create |
| 1447 | + config: |
| 1448 | + n: 2 |
| 1449 | + error_handling: |
| 1450 | + output: my_error |
| 1451 | + ''', |
| 1452 | + providers=merged_providers) |
| 1453 | + |
| 1454 | + |
1300 | 1455 | @beam.transforms.ptransform.annotate_yaml |
1301 | 1456 | class LinearTransform(beam.PTransform): |
1302 | 1457 | """A transform used for testing annotate_yaml.""" |
|
0 commit comments