diff --git a/.github/trigger_files/beam_PostCommit_Python_Dependency.json b/.github/trigger_files/beam_PostCommit_Python_Dependency.json index 907b485d4d30..a7fc54b3e4bb 100644 --- a/.github/trigger_files/beam_PostCommit_Python_Dependency.json +++ b/.github/trigger_files/beam_PostCommit_Python_Dependency.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 0 + "modification": 1 } \ No newline at end of file diff --git a/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json b/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json index f1ba03a243ee..455144f02a35 100644 --- a/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json +++ b/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 5 + "modification": 6 } diff --git a/.github/workflows/beam_PostCommit_Python_Dependency.yml b/.github/workflows/beam_PostCommit_Python_Dependency.yml index c1eef980c82a..d11a47219305 100644 --- a/.github/workflows/beam_PostCommit_Python_Dependency.yml +++ b/.github/workflows/beam_PostCommit_Python_Dependency.yml @@ -59,7 +59,7 @@ jobs: matrix: job_name: [beam_PostCommit_Python_Dependency] job_phrase: [Run Python PostCommit Dependency] - timeout-minutes: 120 + timeout-minutes: 180 if: | github.event_name == 'workflow_dispatch' || github.event_name == 'pull_request_target' || diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml index d6582cea858a..0ad20571f92c 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml @@ -57,7 +57,7 @@ jobs: (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Python_Xlang_Gcp_Direct PostCommit' runs-on: [self-hosted, ubuntu-20.04, highmem] - timeout-minutes: 100 + timeout-minutes: 160 name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) strategy: matrix: @@ -98,4 +98,4 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/pytest*.xml' - large_files: true \ No newline at end of file + large_files: true diff --git a/CHANGES.md b/CHANGES.md index dda05276be48..e4bbc21a0232 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -65,6 +65,7 @@ * New highly anticipated feature X added to Python SDK ([#X](https://github.com/apache/beam/issues/X)). * New highly anticipated feature Y added to Java SDK ([#Y](https://github.com/apache/beam/issues/Y)). +* [Python] Prism runner now enabled by default for most Python pipelines using the direct runner ([#34612](https://github.com/apache/beam/pull/34612)). This may break some tests, see https://github.com/apache/beam/pull/34612 for details on how to handle issues. ## I/Os @@ -81,6 +82,7 @@ ## Breaking Changes +* [Python] Prism runner now enabled by default for most Python pipelines using the direct runner ([#34612](https://github.com/apache/beam/pull/34612)). This may break some tests, see https://github.com/apache/beam/pull/34612 for details on how to handle issues. * X behavior was changed ([#X](https://github.com/apache/beam/issues/X)). ## Deprecations diff --git a/sdks/python/apache_beam/coders/coders_test.py b/sdks/python/apache_beam/coders/coders_test.py index 2cde92a76def..baa6d969421c 100644 --- a/sdks/python/apache_beam/coders/coders_test.py +++ b/sdks/python/apache_beam/coders/coders_test.py @@ -267,7 +267,7 @@ def test_numpy_int(self): # this type is not supported as the key import numpy as np - with self.assertRaises(TypeError): + with self.assertRaises(Exception): with TestPipeline() as p: indata = p | "Create" >> beam.Create([(a, int(a)) for a in np.arange(3)]) diff --git a/sdks/python/apache_beam/examples/snippets/snippets_test.py b/sdks/python/apache_beam/examples/snippets/snippets_test.py index 3714c0574e05..27996fbede58 100644 --- a/sdks/python/apache_beam/examples/snippets/snippets_test.py +++ b/sdks/python/apache_beam/examples/snippets/snippets_test.py @@ -307,8 +307,8 @@ def test_bad_types(self): # When running this pipeline, you'd get a runtime error, # possibly on a remote machine, possibly very late. - with self.assertRaises(TypeError): - p.run() + with self.assertRaises(Exception): + p.run().wait_until_finish() # To catch this early, we can assert what types we expect. with self.assertRaises(typehints.TypeCheckError): @@ -372,8 +372,8 @@ def process(self, element): # When running this pipeline, you'd get a runtime error, # possibly on a remote machine, possibly very late. - with self.assertRaises(TypeError): - p.run() + with self.assertRaises(Exception): + p.run().wait_until_finish() # To catch this early, we can annotate process() with the expected types. # Beam will then use these as type hints and perform type checking before @@ -439,12 +439,13 @@ def test_runtime_checks_off(self): def test_runtime_checks_on(self): # pylint: disable=expression-not-assigned - with self.assertRaises(typehints.TypeCheckError): + with self.assertRaises(Exception): # [START type_hints_runtime_on] p = TestPipeline(options=PipelineOptions(runtime_type_check=True)) p | beam.Create(['a']) | beam.Map(lambda x: 3).with_output_types(str) - p.run() + result = p.run() # [END type_hints_runtime_on] + result.wait_until_finish() def test_deterministic_key(self): with TestPipeline() as p: diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py index 8a7cdfbe9263..afa2bca7ec68 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py @@ -60,28 +60,52 @@ def validate_enrichment_with_vertex_ai_legacy(): return expected +def std_out_to_dict(stdout_lines, row_key): + output_dict = {} + for stdout_line in stdout_lines: + # parse the stdout in a dictionary format so that it can be + # evaluated/compared as one. This allows us to compare without + # considering the order of the stdout or the order that the fields of the + # row are arranged in. + fmtd = '{\"' + stdout_line[4:-1].replace('=', '\": ').replace( + ', ', ', \"').replace('\"\'', '\'') + "}" + stdout_dict = eval(fmtd) # pylint: disable=eval-used + output_dict[stdout_dict[row_key]] = stdout_dict + return output_dict + + @mock.patch('sys.stdout', new_callable=StringIO) class EnrichmentTest(unittest.TestCase): def test_enrichment_with_bigtable(self, mock_stdout): enrichment_with_bigtable() output = mock_stdout.getvalue().splitlines() expected = validate_enrichment_with_bigtable() - self.assertEqual(output, expected) + + self.assertEqual(len(output), len(expected)) + self.assertEqual( + std_out_to_dict(output, 'sale_id'), + std_out_to_dict(expected, 'sale_id')) def test_enrichment_with_vertex_ai(self, mock_stdout): enrichment_with_vertex_ai() output = mock_stdout.getvalue().splitlines() expected = validate_enrichment_with_vertex_ai() - for i in range(len(expected)): - self.assertEqual(set(output[i].split(',')), set(expected[i].split(','))) + self.assertEqual(len(output), len(expected)) + self.assertEqual( + std_out_to_dict(output, 'user_id'), + std_out_to_dict(expected, 'user_id')) def test_enrichment_with_vertex_ai_legacy(self, mock_stdout): enrichment_with_vertex_ai_legacy() output = mock_stdout.getvalue().splitlines() expected = validate_enrichment_with_vertex_ai_legacy() self.maxDiff = None - self.assertEqual(output, expected) + + self.assertEqual(len(output), len(expected)) + self.assertEqual( + std_out_to_dict(output, 'entity_id'), + std_out_to_dict(expected, 'entity_id')) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo_dofn_methods.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo_dofn_methods.py index 8cc43fadb10c..46d4f5955b0c 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo_dofn_methods.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo_dofn_methods.py @@ -37,8 +37,6 @@ def pardo_dofn_methods(test=None): # Portable runners do not guarantee that teardown will be executed, so we # use FnApiRunner instead of prism. runner = 'FnApiRunner' - # TODO(damccorm) - remove after next release - runner = 'DirectRunner' # [START pardo_dofn_methods] import apache_beam as beam diff --git a/sdks/python/apache_beam/io/fileio_test.py b/sdks/python/apache_beam/io/fileio_test.py index 6733df8c70bf..adb2680b5237 100644 --- a/sdks/python/apache_beam/io/fileio_test.py +++ b/sdks/python/apache_beam/io/fileio_test.py @@ -106,7 +106,7 @@ def test_match_files_one_directory_failure1(self): files.append(self._create_temp_file(dir=directories[0])) files.append(self._create_temp_file(dir=directories[0])) - with self.assertRaises(beam.io.filesystem.BeamIOError): + with self.assertRaises(Exception): with TestPipeline() as p: files_pc = ( p @@ -259,7 +259,7 @@ def test_fail_on_directories(self): files.append(self._create_temp_file(dir=tempdir, content=content)) files.append(self._create_temp_file(dir=tempdir, content=content)) - with self.assertRaises(beam.io.filesystem.BeamIOError): + with self.assertRaises(Exception): with TestPipeline() as p: _ = ( p @@ -501,10 +501,14 @@ def test_write_to_dynamic_destination(self): fileio.TextSink() # pass a FileSink object ] + # Test assumes that all records will be handled by same worker process, + # pin to FnApiRunner to guarantee hthis + runner = 'FnApiRunner' + for sink in sink_params: dir = self._new_tempdir() - with TestPipeline() as p: + with TestPipeline(runner) as p: _ = ( p | "Create" >> beam.Create(range(100)) @@ -515,7 +519,7 @@ def test_write_to_dynamic_destination(self): sink=sink, file_naming=fileio.destination_prefix_naming("test"))) - with TestPipeline() as p: + with TestPipeline(runner) as p: result = ( p | fileio.MatchFiles(FileSystems.join(dir, '*')) diff --git a/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py b/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py index 6400365918d2..4b41900b80c7 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py @@ -447,8 +447,8 @@ def test_records_traverse_transform_with_mocks(self): validate=False, temp_file_format=bigquery_tools.FileFormat.JSON) - # Need to test this with the DirectRunner to avoid serializing mocks - with TestPipeline('DirectRunner') as p: + # Need to test this with the FnApiRunner to avoid serializing mocks + with TestPipeline('FnApiRunner') as p: outputs = p | beam.Create(_ELEMENTS) | transform dest_files = outputs[bqfl.BigQueryBatchFileLoads.DESTINATION_FILE_PAIRS] diff --git a/sdks/python/apache_beam/io/gcp/experimental/spannerio_test.py b/sdks/python/apache_beam/io/gcp/experimental/spannerio_test.py index 4e391900eaa7..8761c15b78e6 100644 --- a/sdks/python/apache_beam/io/gcp/experimental/spannerio_test.py +++ b/sdks/python/apache_beam/io/gcp/experimental/spannerio_test.py @@ -387,7 +387,7 @@ def test_read_with_transaction( def test_invalid_transaction( self, mock_batch_snapshot_class, mock_client_class): # test exception raises at pipeline execution time - with self.assertRaises(ValueError), TestPipeline() as p: + with self.assertRaises(Exception), TestPipeline() as p: transaction = ( p | beam.Create([{ "invalid": "transaction" diff --git a/sdks/python/apache_beam/io/parquetio_test.py b/sdks/python/apache_beam/io/parquetio_test.py index fd19ec9520a9..c602f4cc801b 100644 --- a/sdks/python/apache_beam/io/parquetio_test.py +++ b/sdks/python/apache_beam/io/parquetio_test.py @@ -52,7 +52,6 @@ try: import pyarrow as pa - import pyarrow.lib as pl import pyarrow.parquet as pq except ImportError: pa = None @@ -338,17 +337,16 @@ def test_write_batched_display_data(self): ARROW_MAJOR_VERSION >= 13, 'pyarrow 13.x and above does not throw ArrowInvalid error') def test_sink_transform_int96(self): - with tempfile.NamedTemporaryFile() as dst: + with self.assertRaisesRegex(Exception, 'would lose data'): + # Should throw an error "ArrowInvalid: Casting from timestamp[ns] to + # timestamp[us] would lose data" + dst = tempfile.NamedTemporaryFile() path = dst.name - # pylint: disable=c-extension-no-member - with self.assertRaises(pl.ArrowInvalid): - # Should throw an error "ArrowInvalid: Casting from timestamp[ns] to - # timestamp[us] would lose data" - with TestPipeline() as p: - _ = p \ - | Create(self.RECORDS) \ - | WriteToParquet( - path, self.SCHEMA96, num_shards=1, shard_name_template='') + with TestPipeline() as p: + _ = p \ + | Create(self.RECORDS) \ + | WriteToParquet( + path, self.SCHEMA96, num_shards=1, shard_name_template='') def test_sink_transform(self): with TemporaryDirectory() as tmp_dirname: @@ -571,7 +569,8 @@ def test_selective_columns(self): def test_sink_transform_multiple_row_group(self): with TemporaryDirectory() as tmp_dirname: path = os.path.join(tmp_dirname + "tmp_filename") - with TestPipeline() as p: + # Pin to FnApiRunner since test assumes fixed bundle size + with TestPipeline('FnApiRunner') as p: # writing 623200 bytes of data _ = p \ | Create(self.RECORDS * 4000) \ diff --git a/sdks/python/apache_beam/io/requestresponse_test.py b/sdks/python/apache_beam/io/requestresponse_test.py index 3bc85a5e103a..7fb2e9a9355f 100644 --- a/sdks/python/apache_beam/io/requestresponse_test.py +++ b/sdks/python/apache_beam/io/requestresponse_test.py @@ -31,8 +31,6 @@ from apache_beam.io.requestresponse import Caller from apache_beam.io.requestresponse import DefaultThrottler from apache_beam.io.requestresponse import RequestResponseIO - from apache_beam.io.requestresponse import UserCodeExecutionException - from apache_beam.io.requestresponse import UserCodeTimeoutException from apache_beam.io.requestresponse import retry_on_exception except ImportError: raise unittest.SkipTest('RequestResponseIO dependencies are not installed.') @@ -98,7 +96,7 @@ def test_valid_call(self): def test_call_timeout(self): caller = CallerWithTimeout() - with self.assertRaises(UserCodeTimeoutException): + with self.assertRaises(Exception): with TestPipeline() as test_pipeline: _ = ( test_pipeline @@ -107,7 +105,7 @@ def test_call_timeout(self): def test_call_runtime_error(self): caller = CallerWithRuntimeError() - with self.assertRaises(UserCodeExecutionException): + with self.assertRaises(Exception): with TestPipeline() as test_pipeline: _ = ( test_pipeline @@ -120,23 +118,23 @@ def test_retry_on_exception(self): def test_caller_backoff_retry_strategy(self): caller = CallerThatRetries() - with self.assertRaises(TooManyRequests) as cm: + with self.assertRaises(Exception) as cm: with TestPipeline() as test_pipeline: _ = ( test_pipeline | beam.Create(["sample_request"]) | RequestResponseIO(caller=caller)) - self.assertRegex(cm.exception.message, 'retries = 2') + self.assertRegex(str(cm.exception), 'retries = 2') def test_caller_no_retry_strategy(self): caller = CallerThatRetries() - with self.assertRaises(TooManyRequests) as cm: + with self.assertRaises(Exception) as cm: with TestPipeline() as test_pipeline: _ = ( test_pipeline | beam.Create(["sample_request"]) | RequestResponseIO(caller=caller, repeater=None)) - self.assertRegex(cm.exception.message, 'retries = 0') + self.assertRegex(str(cm.exception), 'retries = 0') @retry( retry=retry_if_exception_type(IndexError), @@ -148,7 +146,11 @@ def test_default_throttler(self): window_ms=10000, bucket_ms=5000, overload_ratio=1) # manually override the number of received requests for testing. throttler.throttler._all_requests.add(time.time() * 1000, 100) - test_pipeline = TestPipeline() + # TODO(https://github.com/apache/beam/issues/34549): This test relies on + # metrics filtering which doesn't work on Prism yet because Prism renames + # steps (e.g. "Do" becomes "ref_AppliedPTransform_Do_7"). + # https://github.com/apache/beam/blob/5f9cd73b7c9a2f37f83971ace3a399d633201dd1/sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner.py#L1590 + test_pipeline = TestPipeline('FnApiRunner') _ = ( test_pipeline | beam.Create(['sample_request']) diff --git a/sdks/python/apache_beam/metrics/metric_test.py b/sdks/python/apache_beam/metrics/metric_test.py index 2e2e51b267a7..bdba0512dfa2 100644 --- a/sdks/python/apache_beam/metrics/metric_test.py +++ b/sdks/python/apache_beam/metrics/metric_test.py @@ -201,7 +201,7 @@ def process(self, element): # Verify user distribution counter. metric_results = res.metrics().query() matcher = MetricResultMatcher( - step='ApplyPardo', + step=hc.contains_string('ApplyPardo'), namespace=hc.contains_string('SomeDoFn'), name='element_dist', committed=DistributionMatcher( @@ -209,8 +209,7 @@ def process(self, element): count_value=hc.greater_than_or_equal_to(0), min_value=hc.greater_than_or_equal_to(0), max_value=hc.greater_than_or_equal_to(0))) - hc.assert_that( - metric_results['distributions'], hc.contains_inanyorder(matcher)) + hc.assert_that(metric_results['distributions'], hc.has_item(matcher)) def test_create_counter_distribution(self): sampler = statesampler.StateSampler('', counters.CounterFactory()) diff --git a/sdks/python/apache_beam/ml/anomaly/detectors/pyod_adapter_test.py b/sdks/python/apache_beam/ml/anomaly/detectors/pyod_adapter_test.py index bb83e1aeca1c..3df357a79e79 100644 --- a/sdks/python/apache_beam/ml/anomaly/detectors/pyod_adapter_test.py +++ b/sdks/python/apache_beam/ml/anomaly/detectors/pyod_adapter_test.py @@ -142,17 +142,17 @@ def test_scoring_with_unmatched_features(self): # (see the `test_scoring_with_matched_features`) detector = PyODFactory.create_detector(self.pickled_model_uri) options = PipelineOptions([]) - p = beam.Pipeline(options=options) - _ = ( - p | beam.Create(self.get_test_data_with_target()) - | beam.Map( - lambda x: beam.Row(**dict(zip(["a", "b", "target"], map(int, x))))) - | beam.WithKeys(0) - | AnomalyDetection(detector=detector)) - # This should raise a ValueError with message # "X has 3 features, but IsolationForest is expecting 2 features as input." - self.assertRaises(ValueError, p.run) + with self.assertRaises(Exception): + with beam.Pipeline(options=options) as p: + _ = ( + p | beam.Create(self.get_test_data_with_target()) + | beam.Map( + lambda x: beam.Row( + **dict(zip(["a", "b", "target"], map(int, x))))) + | beam.WithKeys(0) + | AnomalyDetection(detector=detector)) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/ml/anomaly/transforms_test.py b/sdks/python/apache_beam/ml/anomaly/transforms_test.py index ed5252c6a485..423e51abf635 100644 --- a/sdks/python/apache_beam/ml/anomaly/transforms_test.py +++ b/sdks/python/apache_beam/ml/anomaly/transforms_test.py @@ -64,8 +64,18 @@ def _prediction_iterable_is_equal_to( if len(a_list) != len(b_list): return False - return all( - map(lambda x: _prediction_is_equal_to(x[0], x[1]), zip(a_list, b_list))) + a_dict = {} + b_dict = {} + for i in a_list: + a_dict[i.model_id] = i + for i in b_list: + b_dict[i.model_id] = i + + for k, a_val in a_dict.items(): + if k not in b_dict or not _prediction_is_equal_to(a_val, b_dict[k]): + return False + + return True def _prediction_is_equal_to(a: AnomalyPrediction, b: AnomalyPrediction): diff --git a/sdks/python/apache_beam/ml/inference/base_test.py b/sdks/python/apache_beam/ml/inference/base_test.py index 29ac0ad4247d..6422804efe2d 100644 --- a/sdks/python/apache_beam/ml/inference/base_test.py +++ b/sdks/python/apache_beam/ml/inference/base_test.py @@ -1044,20 +1044,20 @@ def test_forwards_batch_args(self): def test_run_inference_unkeyed_examples_with_keyed_model_handler(self): pipeline = TestPipeline() - with self.assertRaises(TypeError): + with self.assertRaises(Exception): examples = [1, 3, 5] model_handler = base.KeyedModelHandler(FakeModelHandler()) _ = ( pipeline | 'Unkeyed' >> beam.Create(examples) | 'RunUnkeyed' >> base.RunInference(model_handler)) - pipeline.run() + pipeline.run().wait_until_finish() def test_run_inference_keyed_examples_with_unkeyed_model_handler(self): pipeline = TestPipeline() examples = [1, 3, 5] keyed_examples = [(i, example) for i, example in enumerate(examples)] model_handler = FakeModelHandler() - with self.assertRaises(TypeError): + with self.assertRaises(Exception): _ = ( pipeline | 'keyed' >> beam.Create(keyed_examples) | 'RunKeyed' >> base.RunInference(model_handler)) diff --git a/sdks/python/apache_beam/ml/inference/onnx_inference_test.py b/sdks/python/apache_beam/ml/inference/onnx_inference_test.py index e9e017661d41..115e38b06b8f 100644 --- a/sdks/python/apache_beam/ml/inference/onnx_inference_test.py +++ b/sdks/python/apache_beam/ml/inference/onnx_inference_test.py @@ -40,7 +40,6 @@ try: import onnxruntime as ort import torch - from onnxruntime.capi.onnxruntime_pybind11_state import InvalidArgument import tensorflow as tf import tf2onnx from tensorflow.keras import layers @@ -406,8 +405,7 @@ def test_pipeline_gcs_model(self): equal_to(expected_predictions, equals_fn=_compare_prediction_result)) def test_invalid_input_type(self): - with self.assertRaisesRegex(InvalidArgument, - "Got invalid dimensions for input"): + with self.assertRaisesRegex(Exception, "Got invalid dimensions for input"): with TestPipeline() as pipeline: examples = [np.array([1], dtype="float32")] path = os.path.join(self.tmpdir, 'my_onnx_pytorch_path') @@ -461,8 +459,7 @@ def test_pipeline_gcs_model(self): equal_to(expected_predictions, equals_fn=_compare_prediction_result)) def test_invalid_input_type(self): - with self.assertRaisesRegex(InvalidArgument, - "Got invalid dimensions for input"): + with self.assertRaisesRegex(Exception, "Got invalid dimensions for input"): with TestPipeline() as pipeline: examples = [np.array([1], dtype="float32")] path = os.path.join(self.tmpdir, 'my_onnx_tensorflow_path') @@ -517,7 +514,7 @@ def test_pipeline_gcs_model(self): equal_to(expected_predictions, equals_fn=_compare_prediction_result)) def test_invalid_input_type(self): - with self.assertRaises(InvalidArgument): + with self.assertRaises(Exception): with TestPipeline() as pipeline: examples = [np.array([1], dtype="float32")] path = os.path.join(self.tmpdir, 'my_onnx_sklearn_path') diff --git a/sdks/python/apache_beam/ml/inference/pytorch_inference_test.py b/sdks/python/apache_beam/ml/inference/pytorch_inference_test.py index 91556f05801f..fcc374c06d78 100644 --- a/sdks/python/apache_beam/ml/inference/pytorch_inference_test.py +++ b/sdks/python/apache_beam/ml/inference/pytorch_inference_test.py @@ -715,7 +715,7 @@ def batch_validator_tensor_inference_fn( equal_to(expected_predictions, equals_fn=_compare_prediction_result)) def test_invalid_input_type(self): - with self.assertRaisesRegex(TypeError, "expected Tensor as element"): + with self.assertRaisesRegex(Exception, "expected Tensor as element"): with TestPipeline() as pipeline: examples = np.array([1, 5, 3, 10], dtype="float32").reshape(-1, 1) diff --git a/sdks/python/apache_beam/ml/inference/tensorflow_inference_test.py b/sdks/python/apache_beam/ml/inference/tensorflow_inference_test.py index 75f15c87f5ce..7286274e180c 100644 --- a/sdks/python/apache_beam/ml/inference/tensorflow_inference_test.py +++ b/sdks/python/apache_beam/ml/inference/tensorflow_inference_test.py @@ -128,7 +128,8 @@ def test_predict_tensor_with_batch_size(self): model = _create_mult2_model() model_path = os.path.join(self.tmpdir, f'mult2_{uuid.uuid4()}.keras') tf.keras.models.save_model(model, model_path) - with TestPipeline() as pipeline: + # FnApiRunner guarantees large batches, which this pipeline assumes + with TestPipeline('FnApiRunner') as pipeline: def fake_batching_inference_fn( model: tf.Module, diff --git a/sdks/python/apache_beam/ml/rag/chunking/langchain_test.py b/sdks/python/apache_beam/ml/rag/chunking/langchain_test.py index 638f971c5550..c46d9161490d 100644 --- a/sdks/python/apache_beam/ml/rag/chunking/langchain_test.py +++ b/sdks/python/apache_beam/ml/rag/chunking/langchain_test.py @@ -186,7 +186,7 @@ def test_invalid_document_field(self): metadata_fields={}, text_splitter=splitter) - with self.assertRaises(KeyError): + with self.assertRaises(Exception): with TestPipeline() as p: _ = ( p diff --git a/sdks/python/apache_beam/ml/rag/enrichment/bigquery_vector_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/bigquery_vector_search_it_test.py index 03334b0331bf..e0f1247e40ca 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/bigquery_vector_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/bigquery_vector_search_it_test.py @@ -32,7 +32,6 @@ # pylint: disable=ungrouped-imports try: - from google.api_core.exceptions import BadRequest from apache_beam.transforms.enrichment import Enrichment from apache_beam.ml.rag.enrichment.bigquery_vector_search import \ BigQueryVectorSearchEnrichmentHandler @@ -859,7 +858,7 @@ def test_invalid_query(self): handler = BigQueryVectorSearchEnrichmentHandler( vector_search_parameters=params) - with self.assertRaises(BadRequest): + with self.assertRaises(Exception): with TestPipeline() as p: _ = (p | beam.Create(test_chunks) | Enrichment(handler)) @@ -898,7 +897,7 @@ def test_missing_embedding(self): handler = BigQueryVectorSearchEnrichmentHandler( vector_search_parameters=params) - with self.assertRaises(ValueError) as context: + with self.assertRaises(Exception) as context: with TestPipeline() as p: _ = (p | beam.Create(test_chunks) | Enrichment(handler)) self.assertIn("missing embedding", str(context.exception)) diff --git a/sdks/python/apache_beam/ml/transforms/base_test.py b/sdks/python/apache_beam/ml/transforms/base_test.py index 39aff233aecd..460e4641da65 100644 --- a/sdks/python/apache_beam/ml/transforms/base_test.py +++ b/sdks/python/apache_beam/ml/transforms/base_test.py @@ -498,7 +498,7 @@ def test_handler_with_list_data(self): }, { 'x': ['Apache Beam', 'Hello world'], }] - with self.assertRaises(TypeError): + with self.assertRaises(Exception): with beam.Pipeline() as p: _ = ( p diff --git a/sdks/python/apache_beam/ml/transforms/embeddings/huggingface_test.py b/sdks/python/apache_beam/ml/transforms/embeddings/huggingface_test.py index 924cca679bd7..bf03cea2990a 100644 --- a/sdks/python/apache_beam/ml/transforms/embeddings/huggingface_test.py +++ b/sdks/python/apache_beam/ml/transforms/embeddings/huggingface_test.py @@ -181,7 +181,7 @@ def test_sentence_transformer_with_int_data_types(self): model_name = DEFAULT_MODEL_NAME embedding_config = SentenceTransformerEmbeddings( model_name=model_name, columns=[test_query_column]) - with self.assertRaises(TypeError): + with self.assertRaises(Exception): with beam.Pipeline() as pipeline: _ = ( pipeline @@ -316,7 +316,7 @@ def test_sentence_transformer_images_with_str_data_types(self): model_name=IMAGE_MODEL_NAME, columns=[test_query_column], image_model=True) - with self.assertRaises(TypeError): + with self.assertRaises(Exception): with beam.Pipeline() as pipeline: _ = ( pipeline diff --git a/sdks/python/apache_beam/ml/transforms/embeddings/open_ai_it_test.py b/sdks/python/apache_beam/ml/transforms/embeddings/open_ai_it_test.py index 0922b8b94ce4..118c656c33c3 100644 --- a/sdks/python/apache_beam/ml/transforms/embeddings/open_ai_it_test.py +++ b/sdks/python/apache_beam/ml/transforms/embeddings/open_ai_it_test.py @@ -175,7 +175,7 @@ def test_with_int_data_types(self): model_name=model_name, columns=[test_query_column], api_key=self.api_key) - with self.assertRaises(TypeError): + with self.assertRaises(Exception): with beam.Pipeline() as pipeline: _ = ( pipeline diff --git a/sdks/python/apache_beam/ml/transforms/embeddings/tensorflow_hub_test.py b/sdks/python/apache_beam/ml/transforms/embeddings/tensorflow_hub_test.py index 24bca5155fa7..64dc1e95d641 100644 --- a/sdks/python/apache_beam/ml/transforms/embeddings/tensorflow_hub_test.py +++ b/sdks/python/apache_beam/ml/transforms/embeddings/tensorflow_hub_test.py @@ -161,7 +161,7 @@ def assert_element(element): def test_with_int_data_types(self): embedding_config = TensorflowHubTextEmbeddings( hub_url=hub_url, columns=[test_query_column]) - with self.assertRaises(TypeError): + with self.assertRaises(Exception): with beam.Pipeline() as pipeline: _ = ( pipeline diff --git a/sdks/python/apache_beam/ml/transforms/embeddings/vertex_ai_test.py b/sdks/python/apache_beam/ml/transforms/embeddings/vertex_ai_test.py index bf2298ac77d5..74ffae6865d9 100644 --- a/sdks/python/apache_beam/ml/transforms/embeddings/vertex_ai_test.py +++ b/sdks/python/apache_beam/ml/transforms/embeddings/vertex_ai_test.py @@ -153,7 +153,7 @@ def assert_element(element): def test_with_int_data_types(self): embedding_config = VertexAITextEmbeddings( model_name=model_name, columns=[test_query_column]) - with self.assertRaises(TypeError): + with self.assertRaises(Exception): with beam.Pipeline() as pipeline: _ = ( pipeline diff --git a/sdks/python/apache_beam/pipeline_test.py b/sdks/python/apache_beam/pipeline_test.py index f1f6ee8f7d46..dc0d9a7cc58f 100644 --- a/sdks/python/apache_beam/pipeline_test.py +++ b/sdks/python/apache_beam/pipeline_test.py @@ -431,7 +431,7 @@ def test_pipeline_as_context(self): def raise_exception(exn): raise exn - with self.assertRaises(ValueError): + with self.assertRaises(Exception): with Pipeline() as p: # pylint: disable=expression-not-assigned p | Create([ValueError('msg')]) | Map(raise_exception) @@ -921,7 +921,7 @@ def test_map( return (x, context_a, context_b, context_c) self.assertEqual(_TestContext.live_contexts, 0) - with TestPipeline() as p: + with TestPipeline('FnApiRunner') as p: pcoll = p | Create([1, 2]) | beam.Map(test_map) assert_that(pcoll, equal_to([(1, 'a', 'b', 'c'), (2, 'a', 'b', 'c')])) self.assertEqual(_TestContext.live_contexts, 0) diff --git a/sdks/python/apache_beam/runners/direct/direct_runner.py b/sdks/python/apache_beam/runners/direct/direct_runner.py index 1df2e88f6140..564a6c7df204 100644 --- a/sdks/python/apache_beam/runners/direct/direct_runner.py +++ b/sdks/python/apache_beam/runners/direct/direct_runner.py @@ -52,6 +52,7 @@ from apache_beam.transforms.ptransform import PTransform from apache_beam.transforms.timeutil import TimeDomain from apache_beam.typehints import trivial_inference +from apache_beam.utils.interactive_utils import is_in_ipython __all__ = ['BundleBasedDirectRunner', 'DirectRunner', 'SwitchingDirectRunner'] @@ -66,9 +67,14 @@ class SwitchingDirectRunner(PipelineRunner): which supports streaming execution and certain primitives not yet implemented in the FnApiRunner. """ + _is_interactive = False + def is_fnapi_compatible(self): return BundleBasedDirectRunner.is_fnapi_compatible() + def is_interactive(self): + self._is_interactive = True + def run_pipeline(self, pipeline, options): from apache_beam.pipeline import PipelineVisitor @@ -112,9 +118,25 @@ def visit_transform(self, applied_ptransform): class _PrismRunnerSupportVisitor(PipelineVisitor): """Visitor determining if a Pipeline can be run on the PrismRunner.""" - def accept(self, pipeline): + def accept(self, pipeline, is_interactive): + all_options = options.get_all_options() self.supported_by_prism_runner = True - pipeline.visit(self) + # TODO(https://github.com/apache/beam/issues/33623): Prism currently + # double fires on AfterCount trigger, once appropriately, and once + # incorrectly at the end of the window. This if condition could be + # more targeted, but for now we'll just ignore all unsafe triggers. + if pipeline.allow_unsafe_triggers: + self.supported_by_prism_runner = False + # TODO(https://github.com/apache/beam/issues/33623): Prism currently + # does not support interactive mode + elif is_in_ipython() or is_interactive: + self.supported_by_prism_runner = False + # TODO(https://github.com/apache/beam/issues/33623): Prism currently + # does not support the update compat flag + elif all_options['update_compatibility_version']: + self.supported_by_prism_runner = False + else: + pipeline.visit(self) return self.supported_by_prism_runner def visit_transform(self, applied_ptransform): @@ -123,8 +145,26 @@ def visit_transform(self, applied_ptransform): # being used. if isinstance(transform, TestStream): self.supported_by_prism_runner = False + if isinstance(transform, beam.ExternalTransform): + # TODO(https://github.com/apache/beam/issues/33623): Prism currently + # seems to not be able to consistently bring up external transforms. + # It does sometimes, but at volume suites start to fail. We will try + # to enable this in a future release. + self.supported_by_prism_runner = False if isinstance(transform, beam.ParDo): dofn = transform.dofn + # TODO(https://github.com/apache/beam/issues/33623): Prism currently + # does not seem to handle DoFns using exception handling very well. + # This may be limited just to subprocess DoFns, but more + # investigation is needed before making it default + if isinstance(dofn, + beam.transforms.core._ExceptionHandlingWrapperDoFn): + self.supported_by_prism_runner = False + # https://github.com/apache/beam/issues/34549 + # Remote once we can support local materialization + if (hasattr(dofn, 'is_materialize_values_do_fn') and + dofn.is_materialize_values_do_fn): + self.supported_by_prism_runner = False # It's uncertain if the Prism Runner supports execution of CombineFns # with deferred side inputs. if isinstance(dofn, CombineValuesDoFn): @@ -136,33 +176,32 @@ def visit_transform(self, applied_ptransform): if userstate.is_stateful_dofn(dofn): # https://github.com/apache/beam/issues/32786 - # Remove once Real time clock is used. - _, timer_specs = userstate.get_dofn_specs(dofn) + state_specs, timer_specs = userstate.get_dofn_specs(dofn) for timer in timer_specs: if timer.time_domain == TimeDomain.REAL_TIME: self.supported_by_prism_runner = False - tryingPrism = False + for state in state_specs: + if isinstance(state, userstate.CombiningValueStateSpec): + self.supported_by_prism_runner = False + # TODO(https://github.com/apache/beam/issues/33623): Prism seems to + # not handle session windows correctly. Examples are: + # util_test.py::ReshuffleTest::test_reshuffle_window_fn_preserved + # and util_test.py::ReshuffleTest::test_reshuffle_windows_unchanged + if isinstance(transform, beam.WindowInto) and isinstance( + transform.get_windowing('').windowfn, beam.window.Sessions): + self.supported_by_prism_runner = False + + # Use BundleBasedDirectRunner if other runners are missing needed features. + runner = BundleBasedDirectRunner() + # Check whether all transforms used in the pipeline are supported by the - # FnApiRunner, and the pipeline was not meant to be run as streaming. - if _FnApiRunnerSupportVisitor().accept(pipeline): - from apache_beam.portability.api import beam_provision_api_pb2 - from apache_beam.runners.portability.fn_api_runner import fn_runner - from apache_beam.runners.portability.portable_runner import JobServiceHandle - all_options = options.get_all_options() - encoded_options = JobServiceHandle.encode_pipeline_options(all_options) - provision_info = fn_runner.ExtendedProvisionInfo( - beam_provision_api_pb2.ProvisionInfo( - pipeline_options=encoded_options)) - runner = fn_runner.FnApiRunner(provision_info=provision_info) - elif _PrismRunnerSupportVisitor().accept(pipeline): + # PrismRunner + if _PrismRunnerSupportVisitor().accept(pipeline, self._is_interactive): _LOGGER.info('Running pipeline with PrismRunner.') from apache_beam.runners.portability import prism_runner runner = prism_runner.PrismRunner() - tryingPrism = True - else: - runner = BundleBasedDirectRunner() - if tryingPrism: try: pr = runner.run_pipeline(pipeline, options) # This is non-blocking, so if the state is *already* finished, something @@ -170,7 +209,7 @@ def visit_transform(self, applied_ptransform): if (PipelineState.is_terminal(pr.state) and pr.state != PipelineState.DONE): _LOGGER.info( - 'Pipeline failed on PrismRunner, falling back toDirectRunner.') + 'Pipeline failed on PrismRunner, falling back to DirectRunner.') runner = BundleBasedDirectRunner() else: return pr @@ -182,6 +221,19 @@ def visit_transform(self, applied_ptransform): _LOGGER.info('Falling back to DirectRunner') runner = BundleBasedDirectRunner() + # Check whether all transforms used in the pipeline are supported by the + # FnApiRunner, and the pipeline was not meant to be run as streaming. + if _FnApiRunnerSupportVisitor().accept(pipeline): + from apache_beam.portability.api import beam_provision_api_pb2 + from apache_beam.runners.portability.fn_api_runner import fn_runner + from apache_beam.runners.portability.portable_runner import JobServiceHandle + all_options = options.get_all_options() + encoded_options = JobServiceHandle.encode_pipeline_options(all_options) + provision_info = fn_runner.ExtendedProvisionInfo( + beam_provision_api_pb2.ProvisionInfo( + pipeline_options=encoded_options)) + runner = fn_runner.FnApiRunner(provision_info=provision_info) + return runner.run_pipeline(pipeline, options) diff --git a/sdks/python/apache_beam/runners/interactive/interactive_runner.py b/sdks/python/apache_beam/runners/interactive/interactive_runner.py index 17619fbb6ddc..c8b0be0941d0 100644 --- a/sdks/python/apache_beam/runners/interactive/interactive_runner.py +++ b/sdks/python/apache_beam/runners/interactive/interactive_runner.py @@ -80,6 +80,8 @@ def __init__( """ self._underlying_runner = ( underlying_runner or direct_runner.DirectRunner()) + if hasattr(self._underlying_runner, 'is_interactive'): + self._underlying_runner.is_interactive() self._render_option = render_option self._in_session = False self._skip_display = skip_display diff --git a/sdks/python/apache_beam/runners/interactive/recording_manager.py b/sdks/python/apache_beam/runners/interactive/recording_manager.py index ce6fbd6d8ae8..f72ec2fe8e17 100644 --- a/sdks/python/apache_beam/runners/interactive/recording_manager.py +++ b/sdks/python/apache_beam/runners/interactive/recording_manager.py @@ -364,6 +364,8 @@ def record_pipeline(self) -> bool: runner = self.user_pipeline.runner if isinstance(runner, ir.InteractiveRunner): runner = runner._underlying_runner + if hasattr(runner, 'is_interactive'): + runner.is_interactive() # Make sure that sources without a user reference are still cached. ie.current_env().add_user_pipeline(self.user_pipeline) diff --git a/sdks/python/apache_beam/runners/pipeline_utils_test.py b/sdks/python/apache_beam/runners/pipeline_utils_test.py index 4359f943cfb8..ba144f5e6cc2 100644 --- a/sdks/python/apache_beam/runners/pipeline_utils_test.py +++ b/sdks/python/apache_beam/runners/pipeline_utils_test.py @@ -189,7 +189,9 @@ def test_external_merged(self): # All our external environments are equal and consolidated. # We also have a placeholder "default" environment that has not been # resolved do anything concrete yet. - self.assertEqual(len(pipeline_proto.components.environments), 2) + envs = pipeline_proto.components.environments + self.assertEqual( + len(envs), 2, f'should be 2 environments, instead got: {envs}') if __name__ == '__main__': diff --git a/sdks/python/apache_beam/testing/util_test.py b/sdks/python/apache_beam/testing/util_test.py index dbb5d0fd37a5..3c39e2ec0af7 100644 --- a/sdks/python/apache_beam/testing/util_test.py +++ b/sdks/python/apache_beam/testing/util_test.py @@ -26,7 +26,6 @@ from apache_beam import Create from apache_beam.options.pipeline_options import StandardOptions from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import BeamAssertException from apache_beam.testing.util import TestWindowedValue from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to @@ -69,22 +68,22 @@ def test_assert_that_fails(self): assert_that(p | Create([1, 10, 100]), equal_to([1, 2, 3])) def test_assert_missing(self): - with self.assertRaisesRegex(BeamAssertException, - r"missing elements \['c'\]"): + with self.assertRaisesRegex(Exception, r".*missing elements \['c'\]"): with TestPipeline() as p: assert_that(p | Create(['a', 'b']), equal_to(['a', 'b', 'c'])) def test_assert_unexpected(self): - with self.assertRaisesRegex(BeamAssertException, - r"unexpected elements \['c', 'd'\]|" + with self.assertRaisesRegex(Exception, + r".*unexpected elements \['c', 'd'\]|" r"unexpected elements \['d', 'c'\]"): with TestPipeline() as p: assert_that(p | Create(['a', 'b', 'c', 'd']), equal_to(['a', 'b'])) def test_assert_missing_and_unexpected(self): - with self.assertRaisesRegex( - BeamAssertException, - r"unexpected elements \['c'\].*missing elements \['d'\]"): + with self.assertRaisesRegex(Exception, + r".*unexpected elements \[" + r"'c'\].*missing elements" + r" \['d'\]"): with TestPipeline() as p: assert_that(p | Create(['a', 'b', 'c']), equal_to(['a', 'b', 'd'])) @@ -144,7 +143,7 @@ def test_assert_that_passes_is_not_empty(self): assert_that(p | Create([1, 2, 3]), is_not_empty()) def test_assert_that_fails_on_is_not_empty_expected(self): - with self.assertRaises(BeamAssertException): + with self.assertRaises(Exception): with TestPipeline() as p: assert_that(p | Create([]), is_not_empty()) @@ -168,7 +167,7 @@ def test_equal_to_per_window_passes(self): reify_windows=True) def test_equal_to_per_window_fail_unmatched_window(self): - with self.assertRaises(BeamAssertException): + with self.assertRaises(Exception): expected = { window.IntervalWindow(50, 100): [('k', [1])], } @@ -199,7 +198,7 @@ def test_multiple_assert_that_labels(self): assert_that(outputs, equal_to([2, 3, 4])) def test_equal_to_per_window_fail_unmatched_element(self): - with self.assertRaises(BeamAssertException): + with self.assertRaises(Exception): start = int(MIN_TIMESTAMP.micros // 1e6) - 5 end = start + 20 expected = { @@ -237,7 +236,7 @@ def test_equal_to_per_window_succeeds_no_reify_windows(self): equal_to_per_window(expected)) def test_equal_to_per_window_fail_unexpected_element(self): - with self.assertRaises(BeamAssertException): + with self.assertRaises(Exception): start = int(MIN_TIMESTAMP.micros // 1e6) - 5 end = start + 20 expected = { @@ -289,7 +288,7 @@ class RowTuple(NamedTuple): self.assertFalse( row_namedtuple_equals_fn(beam.Row(a='123'), RowTuple(a='123', b=4567))) self.assertFalse(row_namedtuple_equals_fn(beam.Row(a='123'), '123')) - self.assertFalse(row_namedtuple_equals_fn('123', RowTuple(a='123', b=456))) + self.assertFalse(row_namedtuple_equals_fn('123', RowTuple(a='123', b=4567))) class NestedNamedTuple(NamedTuple): a: str diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigquery_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigquery_it_test.py index dd99e386555e..5db3ff418900 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigquery_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigquery_it_test.py @@ -33,7 +33,6 @@ # pylint: disable=ungrouped-imports try: - from google.api_core.exceptions import BadRequest from testcontainers.redis import RedisContainer from apache_beam.transforms.enrichment import Enrichment from apache_beam.transforms.enrichment_handlers.bigquery import \ @@ -286,7 +285,7 @@ def test_bigquery_enrichment_bad_request(self): column_names=['wrong_column'], condition_value_fn=condition_value_fn, ) - with self.assertRaises(BadRequest): + with self.assertRaises(Exception): test_pipeline = beam.Pipeline() _ = ( test_pipeline diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py index 0dfee0c1191a..d345f08b4423 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py @@ -30,7 +30,6 @@ # pylint: disable=ungrouped-imports try: - from google.api_core.exceptions import NotFound from google.cloud.bigtable import Client from google.cloud.bigtable.row_filters import ColumnRangeFilter from testcontainers.redis import RedisContainer @@ -272,7 +271,7 @@ def test_enrichment_with_bigtable_bad_row_filter(self): table_id=self.table_id, row_key=self.row_key, row_filter=column_filter) - with self.assertRaises(NotFound): + with self.assertRaises(Exception): test_pipeline = beam.Pipeline() _ = ( test_pipeline @@ -289,7 +288,7 @@ def test_enrichment_with_bigtable_raises_key_error(self): instance_id=self.instance_id, table_id=self.table_id, row_key='car_name') - with self.assertRaises(KeyError): + with self.assertRaises(Exception): test_pipeline = beam.Pipeline() _ = ( test_pipeline @@ -306,7 +305,7 @@ def test_enrichment_with_bigtable_raises_not_found(self): instance_id=self.instance_id, table_id='invalid_table', row_key=self.row_key) - with self.assertRaises(NotFound): + with self.assertRaises(Exception): test_pipeline = beam.Pipeline() _ = ( test_pipeline @@ -325,7 +324,7 @@ def test_enrichment_with_bigtable_exception_level(self): row_key=self.row_key, exception_level=ExceptionLevel.RAISE) req = [beam.Row(sale_id=1, customer_id=1, product_id=11, quantity=1)] - with self.assertRaises(ValueError): + with self.assertRaises(Exception): test_pipeline = beam.Pipeline() _ = ( test_pipeline diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/vertex_ai_feature_store_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/vertex_ai_feature_store_it_test.py index c5482309a251..139194777361 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/vertex_ai_feature_store_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/vertex_ai_feature_store_it_test.py @@ -27,7 +27,6 @@ # pylint: disable=ungrouped-imports try: - from google.api_core.exceptions import NotFound from testcontainers.redis import RedisContainer from apache_beam.transforms.enrichment import Enrichment from apache_beam.transforms.enrichment_handlers.utils import ExceptionLevel @@ -131,7 +130,7 @@ def test_vertex_ai_feature_store_wrong_name(self): beam.Row(entity_id="16050", name='stripe t-shirt'), ] - with self.assertRaises(NotFound): + with self.assertRaises(Exception): handler = VertexAIFeatureStoreEnrichmentHandler( project=self.project, location=self.location, @@ -158,7 +157,7 @@ def test_vertex_ai_feature_store_bigtable_serving_enrichment_bad(self): row_key=self.entity_type_name, exception_level=ExceptionLevel.RAISE, ) - with self.assertRaises(ValueError): + with self.assertRaises(Exception): test_pipeline = beam.Pipeline() _ = ( test_pipeline @@ -209,7 +208,7 @@ def test_vertex_ai_legacy_feature_store_enrichment_bad(self): exception_level=ExceptionLevel.RAISE, ) - with self.assertRaises(ValueError): + with self.assertRaises(Exception): test_pipeline = beam.Pipeline() _ = ( test_pipeline @@ -225,7 +224,7 @@ def test_vertex_ai_legacy_feature_store_invalid_featurestore(self): feature_store_id = "invalid_name" entity_type_id = "movies" - with self.assertRaises(NotFound): + with self.assertRaises(Exception): handler = VertexAIFeatureStoreLegacyEnrichmentHandler( project=self.project, location=self.location, diff --git a/sdks/python/apache_beam/transforms/ptransform.py b/sdks/python/apache_beam/transforms/ptransform.py index 7f48c24a2dbd..c4f0e3455d4a 100644 --- a/sdks/python/apache_beam/transforms/ptransform.py +++ b/sdks/python/apache_beam/transforms/ptransform.py @@ -228,6 +228,9 @@ def _materialize_transform(self, pipeline): from apache_beam import ParDo class _MaterializeValuesDoFn(DoFn): + def __init__(self): + self.is_materialize_values_do_fn = True + def process(self, element): result.elements.append(element) diff --git a/sdks/python/apache_beam/transforms/ptransform_test.py b/sdks/python/apache_beam/transforms/ptransform_test.py index de9838beb4d9..78d5c3ef38b9 100644 --- a/sdks/python/apache_beam/transforms/ptransform_test.py +++ b/sdks/python/apache_beam/transforms/ptransform_test.py @@ -155,7 +155,9 @@ def test_do_with_side_input_as_keyword_arg(self): assert_that(result, equal_to([11, 12, 13])) def test_do_with_do_fn_returning_string_raises_warning(self): - with self.assertRaises(typehints.TypeCheckError) as cm: + ex_details = r'.*Returning a str from a ParDo or FlatMap is discouraged.' + + with self.assertRaisesRegex(Exception, ex_details): with TestPipeline() as pipeline: pipeline._options.view_as(TypeOptions).runtime_type_check = True pcoll = pipeline | 'Start' >> beam.Create(['2', '9', '3']) @@ -164,13 +166,10 @@ def test_do_with_do_fn_returning_string_raises_warning(self): # Since the DoFn directly returns a string we should get an # error warning us when the pipeliene runs. - expected_error_prefix = ( - 'Returning a str from a ParDo or FlatMap ' - 'is discouraged.') - self.assertStartswith(cm.exception.args[0], expected_error_prefix) - def test_do_with_do_fn_returning_dict_raises_warning(self): - with self.assertRaises(typehints.TypeCheckError) as cm: + ex_details = r'.*Returning a dict from a ParDo or FlatMap is discouraged.' + + with self.assertRaisesRegex(Exception, ex_details): with TestPipeline() as pipeline: pipeline._options.view_as(TypeOptions).runtime_type_check = True pcoll = pipeline | 'Start' >> beam.Create(['2', '9', '3']) @@ -179,11 +178,6 @@ def test_do_with_do_fn_returning_dict_raises_warning(self): # Since the DoFn directly returns a dict we should get an error warning # us when the pipeliene runs. - expected_error_prefix = ( - 'Returning a dict from a ParDo or FlatMap ' - 'is discouraged.') - self.assertStartswith(cm.exception.args[0], expected_error_prefix) - def test_do_with_multiple_outputs_maintains_unique_name(self): with TestPipeline() as pipeline: pcoll = pipeline | 'Start' >> beam.Create([1, 2, 3]) @@ -222,10 +216,11 @@ def process(self, element): metric_results = res.metrics().query( MetricsFilter().with_name('recordsRead')) outputs_counter = metric_results['counters'][0] - self.assertStartswith(outputs_counter.key.step, 'Read') + msg = outputs_counter.key.step + cont = 'SDFBoundedSourceReader' + self.assertTrue(cont in msg, '"%s" does not contain "%s"' % (msg, cont)) self.assertEqual(outputs_counter.key.metric.name, 'recordsRead') self.assertEqual(outputs_counter.committed, 100) - self.assertEqual(outputs_counter.attempted, 100) @pytest.mark.it_validatesrunner def test_par_do_with_multiple_outputs_and_using_yield(self): @@ -292,7 +287,9 @@ def test_do_requires_do_fn_returning_iterable(self): def incorrect_par_do_fn(x): return x + 5 - with self.assertRaises(typehints.TypeCheckError) as cm: + ex_details = r'.*FlatMap and ParDo must return an iterable.' + + with self.assertRaisesRegex(Exception, ex_details): with TestPipeline() as pipeline: pipeline._options.view_as(TypeOptions).runtime_type_check = True pcoll = pipeline | 'Start' >> beam.Create([2, 9, 3]) @@ -300,9 +297,6 @@ def incorrect_par_do_fn(x): # It's a requirement that all user-defined functions to a ParDo return # an iterable. - expected_error_prefix = 'FlatMap and ParDo must return an iterable.' - self.assertStartswith(cm.exception.args[0], expected_error_prefix) - def test_do_fn_with_finish(self): class MyDoFn(beam.DoFn): def process(self, element): @@ -661,7 +655,7 @@ def partition_for(self, element, num_partitions, offset): # Check that a bad partition label will yield an error. For the # DirectRunner, this error manifests as an exception. - with self.assertRaises(ValueError): + with self.assertRaises(Exception): with TestPipeline() as pipeline: pcoll = pipeline | 'Start' >> beam.Create([0, 1, 2, 3, 4, 5, 6, 7, 8]) partitions = pcoll | beam.Partition(SomePartitionFn(), 4, 10000) @@ -1644,15 +1638,11 @@ def int_to_string(x): self.p | 'T' >> beam.Create(['some_string']) | 'ToStr' >> beam.Map(int_to_string)) - with self.assertRaises(typehints.TypeCheckError) as e: - self.p.run() + error_regex = "Type-hint for argument: 'x' violated. Expected an instance " + "of {}, instead found some_string, an instance of {}.".format(int, str) - self.assertStartswith( - e.exception.args[0], - "Runtime type violation detected within ParDo(ToStr): " - "Type-hint for argument: 'x' violated. " - "Expected an instance of {}, " - "instead found some_string, an instance of {}.".format(int, str)) + with self.assertRaisesRegex(Exception, error_regex): + self.p.run() def test_run_time_type_checking_enabled_types_satisfied(self): self.p._options.view_as(TypeOptions).pipeline_type_check = False @@ -1698,16 +1688,10 @@ def is_even_as_key(a): # Although all the types appear to be correct when checked at pipeline # construction. Runtime type-checking should detect the 'is_even_as_key' is # returning Tuple[int, int], instead of Tuple[bool, int]. - with self.assertRaises(typehints.TypeCheckError) as e: - self.p.run() + error_regex = "Runtime type violation detected" - self.assertStartswith( - e.exception.args[0], - "Runtime type violation detected within ParDo(IsEven): " - "Tuple[, ] hint type-constraint violated. " - "The type of element #0 in the passed tuple is incorrect. " - "Expected an instance of type , " - "instead received an instance of type int.") + with self.assertRaisesRegex(Exception, error_regex): + self.p.run() def test_pipeline_checking_satisfied_run_time_checking_satisfied(self): self.p._options.view_as(TypeOptions).pipeline_type_check = False @@ -1736,7 +1720,9 @@ def test_pipeline_runtime_checking_violation_simple_type_input(self): # The type-hinted applied via the 'with_input_types()' method indicates the # ParDo should receive an instance of type 'str', however an 'int' will be # passed instead. - with self.assertRaises(typehints.TypeCheckError) as e: + error_regex = "Runtime type violation detected" + + with self.assertRaisesRegex(Exception, error_regex): ( self.p | beam.Create([1, 1, 1]) @@ -1745,18 +1731,13 @@ def test_pipeline_runtime_checking_violation_simple_type_input(self): str).with_output_types(int))) self.p.run() - self.assertStartswith( - e.exception.args[0], - "Runtime type violation detected within ParDo(ToInt): " - "Type-hint for argument: 'x' violated. " - "Expected an instance of {}, " - "instead found 1, an instance of {}.".format(str, int)) - def test_pipeline_runtime_checking_violation_composite_type_input(self): self.p._options.view_as(TypeOptions).runtime_type_check = True self.p._options.view_as(TypeOptions).pipeline_type_check = False - with self.assertRaises(typehints.TypeCheckError) as e: + error_regex = "Runtime type violation detected" + + with self.assertRaisesRegex(Exception, error_regex): ( self.p | beam.Create([(1, 3.0), (2, 4.9), (3, 9.5)]) @@ -1766,15 +1747,6 @@ def test_pipeline_runtime_checking_violation_composite_type_input(self): typing.Tuple[int, int]).with_output_types(int))) self.p.run() - self.assertStartswith( - e.exception.args[0], - "Runtime type violation detected within ParDo(Add): " - "Type-hint for argument: 'x_y' violated: " - "Tuple[, ] hint type-constraint violated. " - "The type of element #1 in the passed tuple is incorrect. " - "Expected an instance of type , instead received an " - "instance of type float.") - def test_pipeline_runtime_checking_violation_simple_type_output(self): self.p._options.view_as(TypeOptions).runtime_type_check = True self.p._options.view_as(TypeOptions).pipeline_type_check = False @@ -1787,31 +1759,29 @@ def test_pipeline_runtime_checking_violation_simple_type_output(self): ( 'ToInt' >> beam.FlatMap(lambda x: [float(x)]).with_input_types( int).with_output_types(int)).get_type_hints()) - with self.assertRaises(typehints.TypeCheckError) as e: + error_regex = "" + + if self.p._options.view_as(TypeOptions).runtime_type_check: + error_regex = ( + "Runtime type violation detected within ParDo\\(ToInt\\):" + + " According to type-hint expected output should be of type . Instead, received '1.0', an instance of type " + ) + + if self.p._options.view_as(TypeOptions).performance_runtime_type_check: + error_regex = ( + "Runtime type violation detected within ToInt: Type-hint " + + "for argument: 'x' violated. Expected an instance of , " + + "instead found 1.0, an instance of ") + + with self.assertRaisesRegex(Exception, error_regex): ( self.p | beam.Create([1, 1, 1]) | ( 'ToInt' >> beam.FlatMap(lambda x: [float(x)]).with_input_types( int).with_output_types(int))) - self.p.run() - - if self.p._options.view_as(TypeOptions).runtime_type_check: - self.assertStartswith( - e.exception.args[0], - "Runtime type violation detected within " - "ParDo(ToInt): " - "According to type-hint expected output should be " - "of type {}. Instead, received '1.0', " - "an instance of type {}.".format(int, float)) - - if self.p._options.view_as(TypeOptions).performance_runtime_type_check: - self.assertStartswith( - e.exception.args[0], - "Runtime type violation detected within ToInt: " - "Type-hint for argument: 'x' violated. " - "Expected an instance of {}, " - "instead found 1.0, an instance of {}".format(int, float)) + self.p.run().wait_until_finish() def test_pipeline_runtime_checking_violation_composite_type_output(self): self.p._options.view_as(TypeOptions).runtime_type_check = True @@ -1820,7 +1790,13 @@ def test_pipeline_runtime_checking_violation_composite_type_output(self): # The type-hinted applied via the 'returns()' method indicates the ParDo # should return an instance of type: Tuple[float, int]. However, an instance # of 'int' will be generated instead. - with self.assertRaises(typehints.TypeCheckError) as e: + error_regex = ( + "Runtime type violation detected within " + + "ParDo\\(Swap\\): Tuple type constraint violated. " + + "Valid object instance must be of type 'tuple'. Instead, " + + "an instance of 'float' was received.") + + with self.assertRaisesRegex(Exception, error_regex): ( self.p | beam.Create([(1, 3.0), (2, 4.9), (3, 9.5)]) @@ -1831,22 +1807,6 @@ def test_pipeline_runtime_checking_violation_composite_type_output(self): typing.Tuple[float, int]))) self.p.run() - if self.p._options.view_as(TypeOptions).runtime_type_check: - self.assertStartswith( - e.exception.args[0], - "Runtime type violation detected within " - "ParDo(Swap): Tuple type constraint violated. " - "Valid object instance must be of type 'tuple'. Instead, " - "an instance of 'float' was received.") - - if self.p._options.view_as(TypeOptions).performance_runtime_type_check: - self.assertStartswith( - e.exception.args[0], - "Runtime type violation detected within " - "Swap: Type-hint for argument: 'x_y1' violated: " - "Tuple type constraint violated. " - "Valid object instance must be of type 'tuple'. ") - def test_pipeline_runtime_checking_violation_with_side_inputs_decorator(self): self.p._options.view_as(TypeOptions).pipeline_type_check = False self.p._options.view_as(TypeOptions).runtime_type_check = True @@ -1856,22 +1816,18 @@ def test_pipeline_runtime_checking_violation_with_side_inputs_decorator(self): def add(a, b): return a + b - with self.assertRaises(typehints.TypeCheckError) as e: + error_regex = "Runtime type violation detected" + + with self.assertRaisesRegex(Exception, error_regex): (self.p | beam.Create([1, 2, 3, 4]) | 'Add 1' >> beam.Map(add, 1.0)) self.p.run() - self.assertStartswith( - e.exception.args[0], - "Runtime type violation detected within ParDo(Add 1): " - "Type-hint for argument: 'b' violated. " - "Expected an instance of {}, " - "instead found 1.0, an instance of {}.".format(int, float)) - def test_pipeline_runtime_checking_violation_with_side_inputs_via_method(self): # pylint: disable=line-too-long self.p._options.view_as(TypeOptions).runtime_type_check = True self.p._options.view_as(TypeOptions).pipeline_type_check = False - with self.assertRaises(typehints.TypeCheckError) as e: + error_regex = "Runtime type violation detected" + with self.assertRaisesRegex(Exception, error_regex): ( self.p | beam.Create([1, 2, 3, 4]) @@ -1880,13 +1836,6 @@ def test_pipeline_runtime_checking_violation_with_side_inputs_via_method(self): int, int).with_output_types(float))) self.p.run() - self.assertStartswith( - e.exception.args[0], - "Runtime type violation detected within ParDo(Add 1): " - "Type-hint for argument: 'one' violated. " - "Expected an instance of {}, " - "instead found 1.0, an instance of {}.".format(int, float)) - def test_combine_properly_pipeline_type_checks_using_decorator(self): @with_output_types(int) @with_input_types(ints=typing.Iterable[int]) @@ -1980,20 +1929,18 @@ def test_combine_runtime_type_check_violation_using_decorators(self): def iter_mul(ints): return str(reduce(operator.mul, ints, 1)) - with self.assertRaises(typehints.TypeCheckError) as e: + error_regex = ( + "Runtime type violation detected within " + + "Mul/CombinePerKey: Type-hint for return type violated. " + + "Expected an instance of {}, instead found".format(int)) + + with self.assertRaisesRegex(Exception, error_regex): ( self.p | 'K' >> beam.Create([5, 5, 5, 5]).with_output_types(int) | 'Mul' >> beam.CombineGlobally(iter_mul)) self.p.run() - self.assertStartswith( - e.exception.args[0], - "Runtime type violation detected within " - "Mul/CombinePerKey: " - "Type-hint for return type violated. " - "Expected an instance of {}, instead found".format(int)) - def test_combine_pipeline_type_check_using_methods(self): d = ( self.p @@ -2043,7 +1990,13 @@ def test_combine_runtime_type_check_violation_using_methods(self): self.p._options.view_as(TypeOptions).pipeline_type_check = False self.p._options.view_as(TypeOptions).runtime_type_check = True - with self.assertRaises(typehints.TypeCheckError) as e: + error_regex = ( + "Runtime type violation detected within " + + "ParDo\\(SortJoin/KeyWithVoid\\): " + + "Type-hint for argument: 'v' violated. Expected an instance of " + + ", instead found 0, an instance of .") + + with self.assertRaisesRegex(Exception, error_regex): ( self.p | beam.Create([0]).with_output_types(int) @@ -2052,14 +2005,6 @@ def test_combine_runtime_type_check_violation_using_methods(self): with_input_types(str).with_output_types(str))) self.p.run() - self.assertStartswith( - e.exception.args[0], - "Runtime type violation detected within " - "ParDo(SortJoin/KeyWithVoid): " - "Type-hint for argument: 'v' violated. " - "Expected an instance of {}, " - "instead found 0, an instance of {}.".format(str, int)) - def test_combine_insufficient_type_hint_information(self): self.p._options.view_as(TypeOptions).type_check_strictness = 'ALL_REQUIRED' @@ -2114,23 +2059,14 @@ def test_mean_globally_runtime_checking_violated(self): self.p._options.view_as(TypeOptions).pipeline_type_check = False self.p._options.view_as(TypeOptions).runtime_type_check = True - with self.assertRaises(typehints.TypeCheckError) as e: + error_regex = "Runtime type violation detected" + + with self.assertRaisesRegex(Exception, error_regex): ( self.p | 'C' >> beam.Create(['t', 'e', 's', 't']).with_output_types(str) | 'Mean' >> combine.Mean.Globally()) self.p.run() - self.assertEqual( - "Runtime type violation detected for transform input " - "when executing ParDoFlatMap(Combine): Tuple[Any, " - "Iterable[Union[int, float]]] hint type-constraint " - "violated. The type of element #1 in the passed tuple " - "is incorrect. Iterable[Union[int, float]] hint " - "type-constraint violated. The type of element #0 in " - "the passed Iterable is incorrect: Union[int, float] " - "type-constraint violated. Expected an instance of one " - "of: ('int', 'float'), received str instead.", - e.exception.args[0]) def test_mean_per_key_pipeline_checking_satisfied(self): d = ( @@ -2183,7 +2119,9 @@ def test_mean_per_key_runtime_checking_violated(self): self.p._options.view_as(TypeOptions).pipeline_type_check = False self.p._options.view_as(TypeOptions).runtime_type_check = True - with self.assertRaises(typehints.TypeCheckError) as e: + error_regex = "Runtime type violation detected" + + with self.assertRaisesRegex(Exception, error_regex): ( self.p | beam.Create(range(5)).with_output_types(int) @@ -2194,18 +2132,6 @@ def test_mean_per_key_runtime_checking_violated(self): | 'OddMean' >> combine.Mean.PerKey()) self.p.run() - expected_msg = \ - "Runtime type violation detected within " \ - "OddMean/CombinePerKey(MeanCombineFn): " \ - "Type-hint for argument: 'element' violated: " \ - "Union[, , , ] type-constraint violated. " \ - "Expected an instance of one of: (\"\", \"\", \"\", \"\"), " \ - "received str instead" - - self.assertStartswith(e.exception.args[0], expected_msg) - def test_count_globally_pipeline_type_checking_satisfied(self): d = ( self.p @@ -2522,21 +2448,15 @@ def test_to_dict_runtime_check_satisfied(self): def test_runtime_type_check_python_type_error(self): self.p._options.view_as(TypeOptions).runtime_type_check = True - with self.assertRaises(TypeError) as e: + error_regex = "object of type 'int' has no len()" + + with self.assertRaisesRegex(Exception, error_regex): ( self.p | beam.Create([1, 2, 3]).with_output_types(int) | 'Len' >> beam.Map(lambda x: len(x)).with_output_types(int)) self.p.run() - # Our special type-checking related TypeError shouldn't have been raised. - # Instead the above pipeline should have triggered a regular Python runtime - # TypeError. - self.assertEqual( - "object of type 'int' has no len() [while running 'Len']", - e.exception.args[0]) - self.assertFalse(isinstance(e, typehints.TypeCheckError)) - def test_pardo_type_inference(self): self.assertEqual(int, beam.Filter(lambda x: False).infer_output_type(int)) self.assertEqual( diff --git a/sdks/python/apache_beam/transforms/timestamped_value_type_test.py b/sdks/python/apache_beam/transforms/timestamped_value_type_test.py index 3852b9a85bf1..256e9d6f0c00 100644 --- a/sdks/python/apache_beam/transforms/timestamped_value_type_test.py +++ b/sdks/python/apache_beam/transforms/timestamped_value_type_test.py @@ -21,7 +21,6 @@ import apache_beam as beam from apache_beam.transforms.window import TimestampedValue -from apache_beam.typehints.decorators import TypeCheckError T = TypeVar("T") @@ -98,7 +97,7 @@ def test_opts_with_check_list_str(self): | beam.Map(print)) def test_opts_with_check_wrong_data(self): - with self.assertRaises(TypeCheckError): + with self.assertRaises(Exception): with beam.Pipeline(options=self.opts) as p: _ = ( p @@ -107,7 +106,7 @@ def test_opts_with_check_wrong_data(self): | beam.Map(print)) def test_opts_with_check_wrong_data_list_str(self): - with self.assertRaises(TypeCheckError): + with self.assertRaises(Exception): with beam.Pipeline(options=self.opts) as p: _ = ( p @@ -115,7 +114,7 @@ def test_opts_with_check_wrong_data_list_str(self): | "With timestamps" >> beam.Map(ConvertToTimestampedValue_2) | beam.Map(print)) - with self.assertRaises(TypeCheckError): + with self.assertRaises(Exception): with beam.Pipeline(options=self.opts) as p: _ = ( p @@ -124,7 +123,7 @@ def test_opts_with_check_wrong_data_list_str(self): | beam.Map(print)) def test_opts_with_check_typevar(self): - with self.assertRaises(RuntimeError): + with self.assertRaises(Exception): with beam.Pipeline(options=self.opts) as p: _ = ( p diff --git a/sdks/python/apache_beam/transforms/trigger_test.py b/sdks/python/apache_beam/transforms/trigger_test.py index 79fd3151c083..b9a8cdc594b5 100644 --- a/sdks/python/apache_beam/transforms/trigger_test.py +++ b/sdks/python/apache_beam/transforms/trigger_test.py @@ -1352,7 +1352,12 @@ def merge(_, to_be_merged, merge_result): if IntervalWindow(*pane['window']) not in merged_away ] - with TestPipeline() as p: + # TODO(https://github.com/apache/beam/issues/34549): This test relies on + # the correct error message being reported, but because of how it is + # structured Prism often fails on splitting at the same time as it fails + # on an element, leading to good logs but potentially inconsistent error + # messages. Its not a bug, but it does mess with regex matching. + with TestPipeline('FnApiRunner') as p: input_pc = ( p | beam.Create(inputs) diff --git a/sdks/python/apache_beam/transforms/userstate_test.py b/sdks/python/apache_beam/transforms/userstate_test.py index 8f2cb34f982e..6a8efd1a536f 100644 --- a/sdks/python/apache_beam/transforms/userstate_test.py +++ b/sdks/python/apache_beam/transforms/userstate_test.py @@ -744,7 +744,9 @@ def process( def emit_values(self, set_state=beam.DoFn.StateParam(SET_STATE)): yield sorted(set_state.read()) - with TestPipeline() as p: + # Pin to FnApiRunner since this assumes a large bundle size to contain + # all elements + with TestPipeline('FnApiRunner') as p: values = p | beam.Create([('key', 1), ('key', 2), ('key', 3), ('key', 4), ('key', 5)]) actual_values = ( @@ -992,7 +994,7 @@ def emit_callback( sorted(StatefulDoFnOnDirectRunnerTest.all_records)) @pytest.mark.no_xdist - @pytest.mark.timeout(10) + @pytest.mark.timeout(60) def test_dynamic_timer_clear_then_set_timer(self): class EmitTwoEvents(DoFn): EMIT_CLEAR_SET_TIMER = TimerSpec('emitclear', TimeDomain.WATERMARK) diff --git a/sdks/python/apache_beam/transforms/util_test.py b/sdks/python/apache_beam/transforms/util_test.py index a9bec0df973d..d08c7a860210 100644 --- a/sdks/python/apache_beam/transforms/util_test.py +++ b/sdks/python/apache_beam/transforms/util_test.py @@ -254,8 +254,8 @@ def test_constant_batch_no_metrics(self): self.assertEqual(len(results["distributions"]), 0) def test_grows_to_max_batch(self): - # Assumes a single bundle... - with TestPipeline() as p: + # Assumes a single bundle, so we pin to the FnApiRunner + with TestPipeline('FnApiRunner') as p: res = ( p | beam.Create(range(164)) @@ -265,8 +265,8 @@ def test_grows_to_max_batch(self): assert_that(res, equal_to([1, 1, 2, 4, 8, 16, 32, 50, 50])) def test_windowed_batches(self): - # Assumes a single bundle, in order... - with TestPipeline() as p: + # Assumes a single bundle in order, so we pin to the FnApiRunner + with TestPipeline('FnApiRunner') as p: res = ( p | beam.Create(range(47), reshuffle=False) @@ -287,8 +287,8 @@ def test_windowed_batches(self): ])) def test_global_batch_timestamps(self): - # Assumes a single bundle - with TestPipeline() as p: + # Assumes a single bundle, so we pin to the FnApiRunner + with TestPipeline('FnApiRunner') as p: res = ( p | beam.Create(range(3), reshuffle=False) @@ -327,8 +327,8 @@ def test_sized_batches(self): assert_that(res, equal_to([2, 10, 10, 10])) def test_sized_windowed_batches(self): - # Assumes a single bundle, in order... - with TestPipeline() as p: + # Assumes a single bundle, in order so we pin to the FnApiRunner + with TestPipeline('FnApiRunner') as p: res = ( p | beam.Create(range(1, 8), reshuffle=False) @@ -527,8 +527,8 @@ def test_numpy_regression(self): util._BatchSizeEstimator.linear_regression_numpy, True) def test_stateful_constant_batch(self): - # Assumes a single bundle... - p = TestPipeline() + # Assumes a single bundle, so we pin to the FnApiRunner + p = TestPipeline('FnApiRunner') output = ( p | beam.Create(range(35)) @@ -649,8 +649,8 @@ def test_stateful_buffering_timer_in_global_window_streaming(self): assert_that(num_elements_per_batch, equal_to([9, 1])) def test_stateful_grows_to_max_batch(self): - # Assumes a single bundle... - with TestPipeline() as p: + # Assumes a single bundle, so we pin to the FnApiRunner + with TestPipeline('FnApiRunner') as p: res = ( p | beam.Create(range(164)) @@ -707,7 +707,7 @@ class AddTimestampDoFn(beam.DoFn): def process(self, element): yield window.TimestampedValue(element, expected_timestamp) - with self.assertRaisesRegex(ValueError, r'window.*None.*add_timestamps2'): + with self.assertRaisesRegex(Exception, r'.*window.*None.*add_timestamps2'): with TestPipeline() as pipeline: data = [(1, 1), (2, 1), (3, 1), (1, 2), (2, 2), (1, 4)] expected_windows = [ @@ -723,7 +723,7 @@ def process(self, element): equal_to(expected_windows), label='before_identity', reify_windows=True) - after_identity = ( + _ = ( before_identity | 'window' >> beam.WindowInto( beam.transforms.util._IdentityWindowFn( @@ -733,11 +733,6 @@ def process(self, element): # contain a window of None. IdentityWindowFn should # raise an exception. | 'add_timestamps2' >> beam.ParDo(AddTimestampDoFn())) - assert_that( - after_identity, - equal_to(expected_windows), - label='after_identity', - reify_windows=True) class ReshuffleTest(unittest.TestCase): @@ -1089,14 +1084,19 @@ def test_reshuffle_default_window_preserves_metadata(self, compat_version): index=1, nonspeculative_index=1) + # Portable runners may not have the same level of precision on timestamps - + # this gets the largest supported timestamp with the extra non-supported + # bits truncated + gt = GlobalWindow().max_timestamp() + truncated_gt = gt - (gt % 0.001) + expected_preserved = [ TestWindowedValue('a', MIN_TIMESTAMP, [GlobalWindow()], no_firing), TestWindowedValue( 'b', timestamp.Timestamp(0), [GlobalWindow()], on_time_only), TestWindowedValue( 'c', timestamp.Timestamp(33), [GlobalWindow()], late_firing), - TestWindowedValue( - 'd', GlobalWindow().max_timestamp(), [GlobalWindow()], no_firing) + TestWindowedValue('d', truncated_gt, [GlobalWindow()], no_firing) ] expected_not_preserved = [ @@ -1107,9 +1107,7 @@ def test_reshuffle_default_window_preserves_metadata(self, compat_version): TestWindowedValue( 'c', timestamp.Timestamp(33), [GlobalWindow()], PANE_INFO_UNKNOWN), TestWindowedValue( - 'd', - GlobalWindow().max_timestamp(), [GlobalWindow()], - PANE_INFO_UNKNOWN) + 'd', truncated_gt, [GlobalWindow()], PANE_INFO_UNKNOWN) ] expected = ( @@ -1125,8 +1123,7 @@ def test_reshuffle_default_window_preserves_metadata(self, compat_version): 'b', timestamp.Timestamp(0), [GlobalWindow()], on_time_only), WindowedValue( 'c', timestamp.Timestamp(33), [GlobalWindow()], late_firing), - WindowedValue( - 'd', GlobalWindow().max_timestamp(), [GlobalWindow()], no_firing) + WindowedValue('d', truncated_gt, [GlobalWindow()], no_firing) ] after_reshuffle = ( diff --git a/sdks/python/apache_beam/typehints/typecheck_test.py b/sdks/python/apache_beam/typehints/typecheck_test.py index bafb21c3dc17..15e822b6f2db 100644 --- a/sdks/python/apache_beam/typehints/typecheck_test.py +++ b/sdks/python/apache_beam/typehints/typecheck_test.py @@ -36,7 +36,6 @@ from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to -from apache_beam.typehints import TypeCheckError from apache_beam.typehints import decorators from apache_beam.typehints import with_input_types from apache_beam.typehints import with_output_types @@ -95,7 +94,7 @@ def test_setup(self): def fn(e: int) -> int: return str(e) # type: ignore - with self.assertRaisesRegex(TypeCheckError, + with self.assertRaisesRegex(Exception, r'output should be.*int.*received.*str'): _ = self.p | beam.Create([1, 2, 3]) | beam.Map(fn) self.p.run() @@ -146,34 +145,27 @@ def assertStartswith(self, msg, prefix): msg.startswith(prefix), '"%s" does not start with "%s"' % (msg, prefix)) def test_simple_input_error(self): - with self.assertRaises(TypeCheckError) as e: + with self.assertRaisesRegex(Exception, + "Type-hint for argument: 'x' violated. " + "Expected an instance of {}, " + "instead found 1, an instance of {}".format( + str, int)): ( self.p | beam.Create([1, 1]) | beam.FlatMap(lambda x: [int(x)]).with_input_types( str).with_output_types(int)) - self.p.run() - - self.assertIn( - "Type-hint for argument: 'x' violated. " - "Expected an instance of {}, " - "instead found 1, an instance of {}".format(str, int), - e.exception.args[0]) + self.p.run().wait_until_finish() def test_simple_output_error(self): - with self.assertRaises(TypeCheckError) as e: + with self.assertRaisesRegex(Exception, + "Type-hint for argument: 'x' violated. "): ( self.p | beam.Create(['1', '1']) | beam.FlatMap(lambda x: [int(x)]).with_input_types( int).with_output_types(int)) - self.p.run() - - self.assertIn( - "Type-hint for argument: 'x' violated. " - "Expected an instance of {}, " - "instead found 1, an instance of {}.".format(int, str), - e.exception.args[0]) + self.p.run().wait_until_finish() def test_simple_input_error_with_kwarg_typehints(self): @with_input_types(element=int) @@ -182,28 +174,19 @@ class ToInt(beam.DoFn): def process(self, element, *args, **kwargs): yield int(element) - with self.assertRaises(TypeCheckError) as e: + with self.assertRaisesRegex(Exception, + "Type-hint for argument: 'element' violated"): (self.p | beam.Create(['1', '1']) | beam.ParDo(ToInt())) - self.p.run() - - self.assertStartswith( - e.exception.args[0], - "Runtime type violation detected within " - "ParDo(ToInt): Type-hint for argument: " - "'element' violated. Expected an instance of " - "{}, instead found 1, " - "an instance of {}.".format(int, str)) + self.p.run().wait_until_finish() def test_do_fn_returning_non_iterable_throws_error(self): # This function is incorrect because it returns a non-iterable object def incorrect_par_do_fn(x): return x + 5 - with self.assertRaises(TypeError) as cm: + with self.assertRaisesRegex(Exception, "'int' object is not iterable "): (self.p | beam.Create([1, 1]) | beam.FlatMap(incorrect_par_do_fn)) - self.p.run() - - self.assertStartswith(cm.exception.args[0], "'int' object is not iterable ") + self.p.run().wait_until_finish() def test_simple_type_satisfied(self): @with_input_types(int, int) @@ -232,15 +215,9 @@ def int_to_string(x): self.p | 'Create' >> beam.Create(['some_string']) | 'ToStr' >> beam.Map(int_to_string)) - with self.assertRaises(TypeCheckError) as e: - self.p.run() - - self.assertStartswith( - e.exception.args[0], - "Runtime type violation detected within ParDo(ToStr): " - "Type-hint for argument: 'x' violated. " - "Expected an instance of {}, " - "instead found some_string, an instance of {}.".format(int, str)) + with self.assertRaisesRegex(Exception, + "Type-hint for argument: 'x' violated. "): + self.p.run().wait_until_finish() def test_pipeline_checking_satisfied_but_run_time_types_violate(self): self.p._options.view_as(TypeOptions).pipeline_type_check = False @@ -258,17 +235,13 @@ def is_even_as_key(a): | 'IsEven' >> beam.Map(is_even_as_key) | 'Parity' >> beam.GroupByKey()) - with self.assertRaises(TypeCheckError) as e: - self.p.run() - - self.assertStartswith( - e.exception.args[0], - "Runtime type violation detected within ParDo(IsEven): " - "Type-hint for return type violated: " - "Tuple[, ] hint type-constraint " - "violated. The type of element #0 in the passed tuple is incorrect. " - "Expected an instance of type , " - "instead received an instance of type int. ") + with self.assertRaisesRegex( + Exception, + ("Type-hint for return type violated: Tuple\\[, \\] hint type-constraint violated. The type of element #0 in " + + "the passed tuple is incorrect. Expected an instance of type , instead received an instance of type int.")): + self.p.run().wait_until_finish() def test_pipeline_runtime_checking_violation_composite_type_output(self): self.p._options.view_as(TypeOptions).pipeline_type_check = False @@ -276,7 +249,10 @@ def test_pipeline_runtime_checking_violation_composite_type_output(self): # The type-hinted applied via the 'returns()' method indicates the ParDo # should return an instance of type: Tuple[float, int]. However, an instance # of 'int' will be generated instead. - with self.assertRaises(TypeCheckError) as e: + with self.assertRaisesRegex( + Exception, + ("Type-hint for return type violated. Expected an instance of {}, " + + "instead found 4.0, an instance of {}.").format(int, float)): ( self.p | beam.Create([(1, 3.0)]) @@ -284,14 +260,7 @@ def test_pipeline_runtime_checking_violation_composite_type_output(self): 'Swap' >> beam.FlatMap(lambda x_y1: [x_y1[0] + x_y1[1]]).with_input_types( Tuple[int, float]).with_output_types(int))) - self.p.run() - - self.assertStartswith( - e.exception.args[0], - "Runtime type violation detected within ParDo(Swap): " - "Type-hint for return type violated. " - "Expected an instance of {}, " - "instead found 4.0, an instance of {}.".format(int, float)) + self.p.run().wait_until_finish() def test_downstream_input_type_hint_error_has_descriptive_error_msg(self): @with_input_types(int) @@ -309,21 +278,16 @@ def process(self, element, *args, **kwargs): # This will raise a type check error in IntToInt even though the actual # type check error won't happen until StrToInt. The user will be told that # StrToInt's input type hints were not satisfied while running IntToInt. - with self.assertRaises(TypeCheckError) as e: + with self.assertRaisesRegex( + Exception, + ("Type-hint for argument: 'element' violated. Expected an instance of " + + "{}, instead found 9, an instance of {}.").format(str, int)): ( self.p | beam.Create([9]) | beam.ParDo(IntToInt()) | beam.ParDo(StrToInt())) - self.p.run() - - self.assertStartswith( - e.exception.args[0], - "Runtime type violation detected within ParDo(StrToInt): " - "Type-hint for argument: 'element' violated. " - "Expected an instance of {}, " - "instead found 9, an instance of {}. " - "[while running 'ParDo(IntToInt)']".format(str, int)) + self.p.run().wait_until_finish() if __name__ == '__main__': diff --git a/sdks/python/apache_beam/yaml/yaml_mapping_test.py b/sdks/python/apache_beam/yaml/yaml_mapping_test.py index d5179d385caf..cc2fe4639abc 100644 --- a/sdks/python/apache_beam/yaml/yaml_mapping_test.py +++ b/sdks/python/apache_beam/yaml/yaml_mapping_test.py @@ -175,7 +175,7 @@ def test_validate(self): label='Errors') def test_validate_explicit_types(self): - with self.assertRaisesRegex(TypeError, r'.*violates schema.*'): + with self.assertRaisesRegex(Exception, r'.*violates schema.*'): with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( pickle_library='cloudpickle')) as p: elements = p | beam.Create([ @@ -284,7 +284,7 @@ def test_partition_with_unknown(self): label='Other') def test_partition_without_unknown(self): - with self.assertRaisesRegex(ValueError, r'.*Unknown output name.*"o".*'): + with self.assertRaisesRegex(Exception, r'.*Unknown output name.*"o".*'): with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( pickle_library='cloudpickle')) as p: elements = p | beam.Create([ @@ -416,8 +416,8 @@ def test_partition_bad_static_type(self): ''') def test_partition_bad_runtime_type(self): - with self.assertRaisesRegex(ValueError, - r'.*Returned output name.*must be a string.*'): + with self.assertRaisesRegex(Exception, + r'Returned output name.*must be a string.*'): with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( pickle_library='cloudpickle')) as p: elements = p | beam.Create([