From 54576c3fbf570ecca986e34b81306f3a492fb2a7 Mon Sep 17 00:00:00 2001 From: Reeba Qureshi <64488642+reeba212@users.noreply.github.com> Date: Thu, 22 Aug 2024 23:01:38 +0530 Subject: [PATCH 01/78] Create yaml_enrichment.py --- .../apache_beam/yaml/yaml_enrichment.py | 55 +++++++++++++++++++ 1 file changed, 55 insertions(+) create mode 100644 sdks/python/apache_beam/yaml/yaml_enrichment.py diff --git a/sdks/python/apache_beam/yaml/yaml_enrichment.py b/sdks/python/apache_beam/yaml/yaml_enrichment.py new file mode 100644 index 000000000000..1d73e8c1794e --- /dev/null +++ b/sdks/python/apache_beam/yaml/yaml_enrichment.py @@ -0,0 +1,55 @@ +from typing import Any, Dict +import apache_beam as beam +from apache_beam.transforms.enrichment_handlers.bigquery import BigQueryEnrichmentHandler +from apache_beam.transforms.enrichment_handlers.bigtable import BigTableEnrichmentHandler +from apache_beam.transforms.enrichment_handlers.feast_feature_store import FeastFeatureStoreEnrichmentHandler +from apache_beam.transforms.enrichment_handlers.vertex_ai_feature_store import VertexAIFeatureStoreEnrichmentHandler +from apache_beam.transforms.enrichment import Enrichment +from typing import Optional + +@beam.ptransform.ptransform_fn +def enrichment_transform(pcoll, enrichment_handler: str, handler_config: Dict[str, Any], timeout: Optional[float] = 30): + """ + The Enrichment transform allows you to dynamically enhance elements in a pipeline + by performing key-value lookups against external services like APIs or databases. + + Args: + enrichment_handler: Specifies the source from where data needs to be extracted + into the pipeline for enriching data. It can be a string value in ["BigQuery", + "BigTable", "FeastFeatureStore", "VertexAIFeatureStore"]. + handler_config: Specifies the parameters for the respective enrichment_handler in a dictionary format. + BigQuery: project, table_name, row_restriction_template, fields, column_names, condition_value_fn, query_fn, min_batch_size, max_batch_size + BigTable: project_id, instance_id, table_id row_key, row_filter, app_profile_id, encoding, ow_key_fn, exception_level, include_timestamp + FeastFeatureStore: feature_store_yaml_path, feature_names, feature_service_name, full_feature_names, entity_row_fn, exception_level + VertexAIFeatureStore: project, location, api_endpoint, feature_store_name:, feature_view_name, row_key, exception_level + + Example Usage: + + - type: Enrichment + config: + enrichment_handler: 'BigTable' + handler_config: + project_id: 'apache-beam-testing' + instance_id: 'beam-test' + table_id: 'bigtable-enrichment-test' + row_key: 'product_id' + timeout: 30 + + """ + if enrichment_handler is None: + raise ValueError("Missing 'source' in enrichment spec.") + if handler_config is None: + raise ValueError("Missing 'handler_config' in enrichment spec.") + + handler_map = { + 'BigQuery': BigQueryEnrichmentHandler, + 'BigTable': BigTableEnrichmentHandler, + 'FeastFeatureStore': FeastFeatureStoreEnrichmentHandler, + 'VertexAIFeatureStore': VertexAIFeatureStoreEnrichmentHandler + } + + if enrichment_handler not in handler_map: + raise ValueError(f"Unknown enrichment source: {enrichment_handler}") + + handler = handler_map[enrichment_handler](**handler_config) + return pcoll | Enrichment(source_handler = handler, timeout = timeout) From 02a22bc2901ebec8c82455ed53e224b5a674c4df Mon Sep 17 00:00:00 2001 From: Reeba Qureshi <64488642+reeba212@users.noreply.github.com> Date: Thu, 22 Aug 2024 23:02:26 +0530 Subject: [PATCH 02/78] Create yaml_enrichment_test.py --- .../apache_beam/yaml/yaml_enrichment_test.py | 62 +++++++++++++++++++ 1 file changed, 62 insertions(+) create mode 100644 sdks/python/apache_beam/yaml/yaml_enrichment_test.py diff --git a/sdks/python/apache_beam/yaml/yaml_enrichment_test.py b/sdks/python/apache_beam/yaml/yaml_enrichment_test.py new file mode 100644 index 000000000000..4042e33c3520 --- /dev/null +++ b/sdks/python/apache_beam/yaml/yaml_enrichment_test.py @@ -0,0 +1,62 @@ +import unittest +import logging +import mock +import apache_beam as beam +from apache_beam.testing.util import assert_that, equal_to +from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.transforms import Map +from apache_beam.yaml.yaml_enrichment import enrichment_transform +from apache_beam import Row +from unittest.mock import patch +from apache_beam.yaml.yaml_transform import YamlTransform + +class FakeEnrichmentTransform: + def __init__(self, enrichment_handler, handler_config, timeout = 30): + self._enrichment_handler = enrichment_handler + self._handler_config = handler_config + self._timeout = timeout + + def __call__(self, enrichment_handler, *, handler_config, timeout = 30): + assert enrichment_handler == self._enrichment_handler + assert handler_config == self._handler_config + assert timeout == self._timeout + return beam.Map(lambda x: beam.Row(**x._asdict())) + + +class EnrichmentTransformTest(unittest.TestCase): + + @patch('apache_beam.yaml.yaml_enrichment.enrichment_transform', FakeEnrichmentTransform) + def test_enrichment_with_bigquery(self): + input_data = [ + Row(label = "item1", rank = 0), + Row(label = "item2", rank = 1), + ] + + handler = 'BigQuery' + config = { + "project": "apache-beam-testing", + "table_name": "project.database.table", + "row_restriction_template": "label='item1' or label='item2'", + "fields": ["label"] + } + + with beam.Pipeline() as p: + with mock.patch('apache_beam.yaml.yaml_enrichment.enrichment_transform', + FakeEnrichmentTransform( + enrichment_handler = handler, + handler_config = config)): + input_pcoll = p | 'CreateInput' >> beam.Create(input_data) + result = input_pcoll | YamlTransform( + f''' + type: Enrichment + config: + enrichment_handler: {handler} + handler_config: {config} + ''') + assert_that( + result, + equal_to(input_data)) + +if __name__ == '__main__': + logging.getLogger().setLevel(logging.INFO) + unittest.main() From f9fc865fbe4a06df44c461cabb8c841d279668b7 Mon Sep 17 00:00:00 2001 From: Reeba Qureshi <64488642+reeba212@users.noreply.github.com> Date: Thu, 22 Aug 2024 23:03:41 +0530 Subject: [PATCH 03/78] Create enrichment integration test --- .../apache_beam/yaml/tests/enrichment.yaml | 66 +++++++++++++++++++ 1 file changed, 66 insertions(+) create mode 100644 sdks/python/apache_beam/yaml/tests/enrichment.yaml diff --git a/sdks/python/apache_beam/yaml/tests/enrichment.yaml b/sdks/python/apache_beam/yaml/tests/enrichment.yaml new file mode 100644 index 000000000000..7c107b2e71c1 --- /dev/null +++ b/sdks/python/apache_beam/yaml/tests/enrichment.yaml @@ -0,0 +1,66 @@ +fixtures: + - name: BQ_TABLE + type: "apache_beam.yaml.integration_tests.temp_bigquery_table" + config: + project: "apache-beam-testing" + - name: TEMP_DIR + type: "apache_beam.yaml.integration_tests.gcs_temp_dir" + config: + bucket: "gs://temp-storage-for-end-to-end-tests/temp-it" + +pipelines: + - pipeline: + type: chain + transforms: + - type: Create + name: Rows + config: + elements: + - {label: '11a', rank: 0} + - {label: '37a', rank: 1} + - {label: '389a', rank: 2} + + - type: WriteToBigQuery + config: + table: "{BQ_TABLE}" + + - pipeline: + type: chain + transforms: + - type: Create + name: Data + config: + elements: + - {label: '11a', name: 'S1'} + - {label: '37a', name: 'S2'} + - {label: '389a', name: 'S3'} + - type: Enrichment + name: Enriched + config: + enrichment_handler: 'BigQuery' + handler_config: + project: apache-beam-testing + table_name: "{BQ_TABLE}" + fields: ['label'] + row_restriction_template: "label = '37a'" + timeout: 30 + + - type: MapToFields + config: + language: python + fields: + label: + callable: 'lambda x: x.label' + output_type: string + rank: + callable: 'lambda x: x.rank' + output_type: integer + name: + callable: 'lambda x: x.name' + output_type: string + + - type: AssertEqual + config: + elements: + - {label: '37a', rank: 1, name: 'S2'} + From ba2b10e6b32a339dc2a29b19cb0f79ececde9121 Mon Sep 17 00:00:00 2001 From: Reeba Qureshi <64488642+reeba212@users.noreply.github.com> Date: Thu, 22 Aug 2024 23:07:06 +0530 Subject: [PATCH 04/78] Register enrichment transform in standard_providers.yaml --- sdks/python/apache_beam/yaml/standard_providers.yaml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/sdks/python/apache_beam/yaml/standard_providers.yaml b/sdks/python/apache_beam/yaml/standard_providers.yaml index 574179805959..242faaa9a77b 100644 --- a/sdks/python/apache_beam/yaml/standard_providers.yaml +++ b/sdks/python/apache_beam/yaml/standard_providers.yaml @@ -101,3 +101,8 @@ Explode: "beam:schematransform:org.apache.beam:yaml:explode:v1" config: gradle_target: 'sdks:java:extensions:sql:expansion-service:shadowJar' + +- type: 'python' + config: {} + transforms: + Enrichment: 'apache_beam.yaml.yaml_enrichment.enrichment_transform' From 788e4a20e95efd2ab9ce487f2844b0f800ac3694 Mon Sep 17 00:00:00 2001 From: Reeba Qureshi Date: Mon, 26 Aug 2024 16:45:57 +0530 Subject: [PATCH 05/78] minor changes 1. Added links for different handlers and removed code for unreachable conditions 2. Removed patch decorator in test --- sdks/python/apache_beam/yaml/yaml_enrichment.py | 13 ++++--------- .../python/apache_beam/yaml/yaml_enrichment_test.py | 1 - 2 files changed, 4 insertions(+), 10 deletions(-) diff --git a/sdks/python/apache_beam/yaml/yaml_enrichment.py b/sdks/python/apache_beam/yaml/yaml_enrichment.py index 1d73e8c1794e..a3d5a4f8c942 100644 --- a/sdks/python/apache_beam/yaml/yaml_enrichment.py +++ b/sdks/python/apache_beam/yaml/yaml_enrichment.py @@ -18,10 +18,10 @@ def enrichment_transform(pcoll, enrichment_handler: str, handler_config: Dict[st into the pipeline for enriching data. It can be a string value in ["BigQuery", "BigTable", "FeastFeatureStore", "VertexAIFeatureStore"]. handler_config: Specifies the parameters for the respective enrichment_handler in a dictionary format. - BigQuery: project, table_name, row_restriction_template, fields, column_names, condition_value_fn, query_fn, min_batch_size, max_batch_size - BigTable: project_id, instance_id, table_id row_key, row_filter, app_profile_id, encoding, ow_key_fn, exception_level, include_timestamp - FeastFeatureStore: feature_store_yaml_path, feature_names, feature_service_name, full_feature_names, entity_row_fn, exception_level - VertexAIFeatureStore: project, location, api_endpoint, feature_store_name:, feature_view_name, row_key, exception_level + BigQuery : project, table_name, row_restriction_template, fields, column_names, condition_value_fn, query_fn, min_batch_size, max_batch_size + BigTable : project_id, instance_id, table_id row_key, row_filter, app_profile_id, encoding, ow_key_fn, exception_level, include_timestamp + FeastFeatureStore : feature_store_yaml_path, feature_names, feature_service_name, full_feature_names, entity_row_fn, exception_level + VertexAIFeatureStore : project, location, api_endpoint, feature_store_name:, feature_view_name, row_key, exception_level Example Usage: @@ -36,11 +36,6 @@ def enrichment_transform(pcoll, enrichment_handler: str, handler_config: Dict[st timeout: 30 """ - if enrichment_handler is None: - raise ValueError("Missing 'source' in enrichment spec.") - if handler_config is None: - raise ValueError("Missing 'handler_config' in enrichment spec.") - handler_map = { 'BigQuery': BigQueryEnrichmentHandler, 'BigTable': BigTableEnrichmentHandler, diff --git a/sdks/python/apache_beam/yaml/yaml_enrichment_test.py b/sdks/python/apache_beam/yaml/yaml_enrichment_test.py index 4042e33c3520..35e333fc2f41 100644 --- a/sdks/python/apache_beam/yaml/yaml_enrichment_test.py +++ b/sdks/python/apache_beam/yaml/yaml_enrichment_test.py @@ -25,7 +25,6 @@ def __call__(self, enrichment_handler, *, handler_config, timeout = 30): class EnrichmentTransformTest(unittest.TestCase): - @patch('apache_beam.yaml.yaml_enrichment.enrichment_transform', FakeEnrichmentTransform) def test_enrichment_with_bigquery(self): input_data = [ Row(label = "item1", rank = 0), From e20166d87c0369ee7cb243bea59d5bc84daa3f22 Mon Sep 17 00:00:00 2001 From: Reeba Qureshi Date: Thu, 19 Sep 2024 21:48:32 +0530 Subject: [PATCH 06/78] minor updates --- .../apache_beam/yaml/integration_tests.py | 2 +- .../apache_beam/yaml/tests/enrichment.yaml | 17 ++++ .../apache_beam/yaml/yaml_enrichment.py | 97 ++++++++++++++----- .../apache_beam/yaml/yaml_enrichment_test.py | 94 ++++++++++-------- 4 files changed, 144 insertions(+), 66 deletions(-) diff --git a/sdks/python/apache_beam/yaml/integration_tests.py b/sdks/python/apache_beam/yaml/integration_tests.py index af1be7b1e8e5..72b3918195da 100644 --- a/sdks/python/apache_beam/yaml/integration_tests.py +++ b/sdks/python/apache_beam/yaml/integration_tests.py @@ -69,7 +69,7 @@ def temp_bigquery_table(project, prefix='yaml_bq_it_'): dataset_id = '%s_%s' % (prefix, uuid.uuid4().hex) bigquery_client.get_or_create_dataset(project, dataset_id) logging.info("Created dataset %s in project %s", dataset_id, project) - yield f'{project}:{dataset_id}.tmp_table' + yield f'{project}.{dataset_id}.tmp_table' request = bigquery.BigqueryDatasetsDeleteRequest( projectId=project, datasetId=dataset_id, deleteContents=True) logging.info("Deleting dataset %s in project %s", dataset_id, project) diff --git a/sdks/python/apache_beam/yaml/tests/enrichment.yaml b/sdks/python/apache_beam/yaml/tests/enrichment.yaml index 7c107b2e71c1..216a18add83f 100644 --- a/sdks/python/apache_beam/yaml/tests/enrichment.yaml +++ b/sdks/python/apache_beam/yaml/tests/enrichment.yaml @@ -1,3 +1,20 @@ +# +# 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. +# + fixtures: - name: BQ_TABLE type: "apache_beam.yaml.integration_tests.temp_bigquery_table" diff --git a/sdks/python/apache_beam/yaml/yaml_enrichment.py b/sdks/python/apache_beam/yaml/yaml_enrichment.py index a3d5a4f8c942..77428bbd59f5 100644 --- a/sdks/python/apache_beam/yaml/yaml_enrichment.py +++ b/sdks/python/apache_beam/yaml/yaml_enrichment.py @@ -1,3 +1,20 @@ +# +# 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. +# + from typing import Any, Dict import apache_beam as beam from apache_beam.transforms.enrichment_handlers.bigquery import BigQueryEnrichmentHandler @@ -7,21 +24,53 @@ from apache_beam.transforms.enrichment import Enrichment from typing import Optional + @beam.ptransform.ptransform_fn -def enrichment_transform(pcoll, enrichment_handler: str, handler_config: Dict[str, Any], timeout: Optional[float] = 30): - """ - The Enrichment transform allows you to dynamically enhance elements in a pipeline - by performing key-value lookups against external services like APIs or databases. +def enrichment_transform( + pcoll, + enrichment_handler: str, + handler_config: Dict[str, Any], + timeout: Optional[float] = 30): + """ + The Enrichment transform allows you to dynamically + enhance elements in a pipeline by performing key-value + lookups against external services like APIs or databases. Args: - enrichment_handler: Specifies the source from where data needs to be extracted - into the pipeline for enriching data. It can be a string value in ["BigQuery", - "BigTable", "FeastFeatureStore", "VertexAIFeatureStore"]. - handler_config: Specifies the parameters for the respective enrichment_handler in a dictionary format. - BigQuery : project, table_name, row_restriction_template, fields, column_names, condition_value_fn, query_fn, min_batch_size, max_batch_size - BigTable : project_id, instance_id, table_id row_key, row_filter, app_profile_id, encoding, ow_key_fn, exception_level, include_timestamp - FeastFeatureStore : feature_store_yaml_path, feature_names, feature_service_name, full_feature_names, entity_row_fn, exception_level - VertexAIFeatureStore : project, location, api_endpoint, feature_store_name:, feature_view_name, row_key, exception_level + enrichment_handler: Specifies the source from + where data needs to be extracted + into the pipeline for enriching data. + It can be a string value in ["BigQuery", + "BigTable", "FeastFeatureStore", + "VertexAIFeatureStore"]. + handler_config: Specifies the parameters for + the respective enrichment_handler in a dictionary format. + BigQuery = ( + "BigQuery: " + "project, table_name, row_restriction_template, " + "fields, column_names, "condition_value_fn, " + "query_fn, min_batch_size, max_batch_size" + ) + + BigTable = ( + "BigTable: " + "project_id, instance_id, table_id, " + "row_key, row_filter, app_profile_id, " + "encoding, ow_key_fn, exception_level, include_timestamp" + ) + + FeastFeatureStore = ( + "FeastFeatureStore: " + "feature_store_yaml_path, feature_names, " + "feature_service_name, full_feature_names, " + "entity_row_fn, exception_level" + ) + + VertexAIFeatureStore = ( + "VertexAIFeatureStore: " + "project, location, api_endpoint, feature_store_name, " + "feature_view_name, row_key, exception_level" + ) Example Usage: @@ -36,15 +85,15 @@ def enrichment_transform(pcoll, enrichment_handler: str, handler_config: Dict[st timeout: 30 """ - handler_map = { - 'BigQuery': BigQueryEnrichmentHandler, - 'BigTable': BigTableEnrichmentHandler, - 'FeastFeatureStore': FeastFeatureStoreEnrichmentHandler, - 'VertexAIFeatureStore': VertexAIFeatureStoreEnrichmentHandler - } - - if enrichment_handler not in handler_map: - raise ValueError(f"Unknown enrichment source: {enrichment_handler}") - - handler = handler_map[enrichment_handler](**handler_config) - return pcoll | Enrichment(source_handler = handler, timeout = timeout) + handler_map = { + 'BigQuery': BigQueryEnrichmentHandler, + 'BigTable': BigTableEnrichmentHandler, + 'FeastFeatureStore': FeastFeatureStoreEnrichmentHandler, + 'VertexAIFeatureStore': VertexAIFeatureStoreEnrichmentHandler + } + + if enrichment_handler not in handler_map: + raise ValueError(f"Unknown enrichment source: {enrichment_handler}") + + handler = handler_map[enrichment_handler](**handler_config) + return pcoll | Enrichment(source_handler=handler, timeout=timeout) diff --git a/sdks/python/apache_beam/yaml/yaml_enrichment_test.py b/sdks/python/apache_beam/yaml/yaml_enrichment_test.py index 35e333fc2f41..9cd28995dfe4 100644 --- a/sdks/python/apache_beam/yaml/yaml_enrichment_test.py +++ b/sdks/python/apache_beam/yaml/yaml_enrichment_test.py @@ -1,61 +1,73 @@ +# +# 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 unittest import logging import mock import apache_beam as beam -from apache_beam.testing.util import assert_that, equal_to -from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.transforms import Map -from apache_beam.yaml.yaml_enrichment import enrichment_transform +from apache_beam.testing.util import assert_that +from apache_beam.testing.util import equal_to from apache_beam import Row -from unittest.mock import patch from apache_beam.yaml.yaml_transform import YamlTransform + class FakeEnrichmentTransform: - def __init__(self, enrichment_handler, handler_config, timeout = 30): - self._enrichment_handler = enrichment_handler - self._handler_config = handler_config - self._timeout = timeout + def __init__(self, enrichment_handler, handler_config, timeout=30): + self._enrichment_handler = enrichment_handler + self._handler_config = handler_config + self._timeout = timeout - def __call__(self, enrichment_handler, *, handler_config, timeout = 30): - assert enrichment_handler == self._enrichment_handler - assert handler_config == self._handler_config - assert timeout == self._timeout - return beam.Map(lambda x: beam.Row(**x._asdict())) + def __call__(self, enrichment_handler, *, handler_config, timeout=30): + assert enrichment_handler == self._enrichment_handler + assert handler_config == self._handler_config + assert timeout == self._timeout + return beam.Map(lambda x: beam.Row(**x._asdict())) class EnrichmentTransformTest(unittest.TestCase): + def test_enrichment_with_bigquery(self): + input_data = [ + Row(label="item1", rank=0), + Row(label="item2", rank=1), + ] - def test_enrichment_with_bigquery(self): - input_data = [ - Row(label = "item1", rank = 0), - Row(label = "item2", rank = 1), - ] - - handler = 'BigQuery' - config = { - "project": "apache-beam-testing", - "table_name": "project.database.table", - "row_restriction_template": "label='item1' or label='item2'", - "fields": ["label"] - } - - with beam.Pipeline() as p: - with mock.patch('apache_beam.yaml.yaml_enrichment.enrichment_transform', - FakeEnrichmentTransform( - enrichment_handler = handler, - handler_config = config)): - input_pcoll = p | 'CreateInput' >> beam.Create(input_data) - result = input_pcoll | YamlTransform( - f''' + handler = 'BigQuery' + config = { + "project": "apache-beam-testing", + "table_name": "project.database.table", + "row_restriction_template": "label='item1' or label='item2'", + "fields": ["label"] + } + + with beam.Pipeline() as p: + with mock.patch('apache_beam.yaml.yaml_enrichment.enrichment_transform', + FakeEnrichmentTransform(enrichment_handler=handler, + handler_config=config)): + input_pcoll = p | 'CreateInput' >> beam.Create(input_data) + result = input_pcoll | YamlTransform( + f''' type: Enrichment config: enrichment_handler: {handler} handler_config: {config} ''') - assert_that( - result, - equal_to(input_data)) + assert_that(result, equal_to(input_data)) + if __name__ == '__main__': - logging.getLogger().setLevel(logging.INFO) - unittest.main() + logging.getLogger().setLevel(logging.INFO) + unittest.main() From ee6258ef5abfbf110067497975c509033e547a97 Mon Sep 17 00:00:00 2001 From: Reeba Qureshi Date: Fri, 20 Sep 2024 00:36:57 +0530 Subject: [PATCH 07/78] fixing lint failures --- sdks/python/apache_beam/yaml/yaml_enrichment.py | 8 +++++--- sdks/python/apache_beam/yaml/yaml_enrichment_test.py | 6 ++++-- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/sdks/python/apache_beam/yaml/yaml_enrichment.py b/sdks/python/apache_beam/yaml/yaml_enrichment.py index 77428bbd59f5..4ec8a5a786d3 100644 --- a/sdks/python/apache_beam/yaml/yaml_enrichment.py +++ b/sdks/python/apache_beam/yaml/yaml_enrichment.py @@ -15,14 +15,16 @@ # limitations under the License. # -from typing import Any, Dict +from typing import Any +from typing import Dict +from typing import Optional + import apache_beam as beam +from apache_beam.transforms.enrichment import Enrichment from apache_beam.transforms.enrichment_handlers.bigquery import BigQueryEnrichmentHandler from apache_beam.transforms.enrichment_handlers.bigtable import BigTableEnrichmentHandler from apache_beam.transforms.enrichment_handlers.feast_feature_store import FeastFeatureStoreEnrichmentHandler from apache_beam.transforms.enrichment_handlers.vertex_ai_feature_store import VertexAIFeatureStoreEnrichmentHandler -from apache_beam.transforms.enrichment import Enrichment -from typing import Optional @beam.ptransform.ptransform_fn diff --git a/sdks/python/apache_beam/yaml/yaml_enrichment_test.py b/sdks/python/apache_beam/yaml/yaml_enrichment_test.py index 9cd28995dfe4..e26d6140af23 100644 --- a/sdks/python/apache_beam/yaml/yaml_enrichment_test.py +++ b/sdks/python/apache_beam/yaml/yaml_enrichment_test.py @@ -15,13 +15,15 @@ # limitations under the License. # -import unittest import logging +import unittest + import mock + import apache_beam as beam +from apache_beam import Row from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to -from apache_beam import Row from apache_beam.yaml.yaml_transform import YamlTransform From c647e47c573e884196c4919027ad59e4575af429 Mon Sep 17 00:00:00 2001 From: Reeba Qureshi Date: Fri, 20 Sep 2024 07:39:31 +0530 Subject: [PATCH 08/78] disable feast if not installed --- sdks/python/apache_beam/yaml/yaml_enrichment.py | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/yaml/yaml_enrichment.py b/sdks/python/apache_beam/yaml/yaml_enrichment.py index 4ec8a5a786d3..0fbe57321395 100644 --- a/sdks/python/apache_beam/yaml/yaml_enrichment.py +++ b/sdks/python/apache_beam/yaml/yaml_enrichment.py @@ -20,12 +20,17 @@ from typing import Optional import apache_beam as beam +from apache_beam.yaml import options from apache_beam.transforms.enrichment import Enrichment from apache_beam.transforms.enrichment_handlers.bigquery import BigQueryEnrichmentHandler from apache_beam.transforms.enrichment_handlers.bigtable import BigTableEnrichmentHandler -from apache_beam.transforms.enrichment_handlers.feast_feature_store import FeastFeatureStoreEnrichmentHandler from apache_beam.transforms.enrichment_handlers.vertex_ai_feature_store import VertexAIFeatureStoreEnrichmentHandler +try: + from apache_beam.transforms.enrichment_handlers.feast_feature_store import FeastFeatureStoreEnrichmentHandler +except ImportError: + FeastFeatureStoreEnrichmentHandler = None + @beam.ptransform.ptransform_fn def enrichment_transform( @@ -87,6 +92,12 @@ def enrichment_transform( timeout: 30 """ + options.YamlOptions.check_enabled(pcoll.pipeline, 'Enrichment') + if (enrichment_handler == 'FeastFeatureStore' and + not FeastFeatureStoreEnrichmentHandler): + raise ValueError( + "FeastFeatureStore handler requires 'feast' package to be installed. " + + "Please install using 'pip install feast[gcp]' and try again.") handler_map = { 'BigQuery': BigQueryEnrichmentHandler, 'BigTable': BigTableEnrichmentHandler, From e917933c09005d6d37d4ad9c0116a6688322258b Mon Sep 17 00:00:00 2001 From: Reeba Qureshi Date: Thu, 3 Oct 2024 23:36:34 +0530 Subject: [PATCH 09/78] fix failures --- .../apache_beam/yaml/standard_providers.yaml | 3 ++- .../apache_beam/yaml/tests/enrichment.yaml | 3 ++- .../apache_beam/yaml/yaml_enrichment.py | 21 ++++++++++++++----- sdks/python/apache_beam/yaml/yaml_provider.py | 16 ++++++++++---- 4 files changed, 32 insertions(+), 11 deletions(-) diff --git a/sdks/python/apache_beam/yaml/standard_providers.yaml b/sdks/python/apache_beam/yaml/standard_providers.yaml index 242faaa9a77b..15d5fdc24914 100644 --- a/sdks/python/apache_beam/yaml/standard_providers.yaml +++ b/sdks/python/apache_beam/yaml/standard_providers.yaml @@ -103,6 +103,7 @@ gradle_target: 'sdks:java:extensions:sql:expansion-service:shadowJar' - type: 'python' - config: {} + config: + requires_gcp: true transforms: Enrichment: 'apache_beam.yaml.yaml_enrichment.enrichment_transform' diff --git a/sdks/python/apache_beam/yaml/tests/enrichment.yaml b/sdks/python/apache_beam/yaml/tests/enrichment.yaml index 216a18add83f..6469c094b8b4 100644 --- a/sdks/python/apache_beam/yaml/tests/enrichment.yaml +++ b/sdks/python/apache_beam/yaml/tests/enrichment.yaml @@ -80,4 +80,5 @@ pipelines: config: elements: - {label: '37a', rank: 1, name: 'S2'} - + options: + yaml_experimental_features: [ 'Enrichment' ] \ No newline at end of file diff --git a/sdks/python/apache_beam/yaml/yaml_enrichment.py b/sdks/python/apache_beam/yaml/yaml_enrichment.py index 0fbe57321395..e2dc72f3dac8 100644 --- a/sdks/python/apache_beam/yaml/yaml_enrichment.py +++ b/sdks/python/apache_beam/yaml/yaml_enrichment.py @@ -21,15 +21,19 @@ import apache_beam as beam from apache_beam.yaml import options -from apache_beam.transforms.enrichment import Enrichment -from apache_beam.transforms.enrichment_handlers.bigquery import BigQueryEnrichmentHandler -from apache_beam.transforms.enrichment_handlers.bigtable import BigTableEnrichmentHandler -from apache_beam.transforms.enrichment_handlers.vertex_ai_feature_store import VertexAIFeatureStoreEnrichmentHandler try: + from apache_beam.transforms.enrichment import Enrichment + from apache_beam.transforms.enrichment_handlers.bigquery import BigQueryEnrichmentHandler + from apache_beam.transforms.enrichment_handlers.bigtable import BigTableEnrichmentHandler + from apache_beam.transforms.enrichment_handlers.vertex_ai_feature_store import VertexAIFeatureStoreEnrichmentHandler from apache_beam.transforms.enrichment_handlers.feast_feature_store import FeastFeatureStoreEnrichmentHandler except ImportError: - FeastFeatureStoreEnrichmentHandler = None + Enrichment = None # type: ignore + BigQueryEnrichmentHandler = None # type: ignore + BigTableEnrichmentHandler = None # type: ignore + VertexAIFeatureStoreEnrichmentHandler = None # type: ignore + FeastFeatureStoreEnrichmentHandler = None # type: ignore @beam.ptransform.ptransform_fn @@ -93,11 +97,18 @@ def enrichment_transform( """ options.YamlOptions.check_enabled(pcoll.pipeline, 'Enrichment') + + if not Enrichment: + raise ValueError( + f"gcp dependencies not installed. Cannot use {enrichment_handler} " + f"handler. Please install using 'pip install apache-beam[gcp]'.") + if (enrichment_handler == 'FeastFeatureStore' and not FeastFeatureStoreEnrichmentHandler): raise ValueError( "FeastFeatureStore handler requires 'feast' package to be installed. " + "Please install using 'pip install feast[gcp]' and try again.") + handler_map = { 'BigQuery': BigQueryEnrichmentHandler, 'BigTable': BigTableEnrichmentHandler, diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index c2cba936abce..2d6ed2e5b956 100755 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -231,9 +231,17 @@ def provider_from_spec(cls, spec): result.to_json = lambda: spec return result except Exception as exn: - raise ValueError( - f'Unable to instantiate provider of type {type} ' - f'at line {SafeLineLoader.get_line(spec)}: {exn}') from exn + if isinstance(exn, ModuleNotFoundError) and config.get('requires_gcp', + False): + print( + f"gcp dependencies not installed. Cannot use transforms: " + f"{', '.join(urns.keys())}. Please install using " + f"'pip install apache-beam[gcp]'.") + return InlineProvider({}) + else: + raise ValueError( + f'Unable to instantiate provider of type {type} ' + f'at line {SafeLineLoader.get_line(spec)}: {exn}') from exn else: raise NotImplementedError( f'Unknown provider type: {type} ' @@ -335,7 +343,7 @@ def cache_artifacts(self): @ExternalProvider.register_provider_type('python') -def python(urns, packages=()): +def python(urns, packages=(), requires_gcp=False): if packages: return ExternalPythonProvider(urns, packages) else: From e24ea06263f20d3425673e96e245d0527ec83743 Mon Sep 17 00:00:00 2001 From: Reeba Qureshi Date: Fri, 4 Oct 2024 00:31:41 +0530 Subject: [PATCH 10/78] separate block for feast import error --- sdks/python/apache_beam/yaml/yaml_enrichment.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/yaml/yaml_enrichment.py b/sdks/python/apache_beam/yaml/yaml_enrichment.py index e2dc72f3dac8..00f2a5c1b1d1 100644 --- a/sdks/python/apache_beam/yaml/yaml_enrichment.py +++ b/sdks/python/apache_beam/yaml/yaml_enrichment.py @@ -27,12 +27,15 @@ from apache_beam.transforms.enrichment_handlers.bigquery import BigQueryEnrichmentHandler from apache_beam.transforms.enrichment_handlers.bigtable import BigTableEnrichmentHandler from apache_beam.transforms.enrichment_handlers.vertex_ai_feature_store import VertexAIFeatureStoreEnrichmentHandler - from apache_beam.transforms.enrichment_handlers.feast_feature_store import FeastFeatureStoreEnrichmentHandler except ImportError: Enrichment = None # type: ignore BigQueryEnrichmentHandler = None # type: ignore BigTableEnrichmentHandler = None # type: ignore VertexAIFeatureStoreEnrichmentHandler = None # type: ignore + +try: + from apache_beam.transforms.enrichment_handlers.feast_feature_store import FeastFeatureStoreEnrichmentHandler +except ImportError: FeastFeatureStoreEnrichmentHandler = None # type: ignore From 19044ec97729f1328577ac4930d79a517e713741 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 3 Oct 2024 15:46:21 -0400 Subject: [PATCH 11/78] Touch Flink trigger files for testing addition of Flink 1.19 support --- .../setup-default-test-properties/test-properties.json | 2 +- .github/trigger_files/beam_PostCommit_Go_VR_Flink.json | 3 ++- .../beam_PostCommit_Java_Examples_Flink.json | 3 +++ .../beam_PostCommit_Java_Jpms_Flink_Java11.json | 3 +++ .../beam_PostCommit_Java_ValidatesRunner_Flink.json | 3 ++- ...beam_PostCommit_Java_ValidatesRunner_Flink_Java11.json | 3 ++- .../beam_PostCommit_Java_ValidatesRunner_Flink_Java8.json | 4 ++++ .../beam_PostCommit_Python_ValidatesRunner_Flink.json | 3 +++ .github/trigger_files/beam_PostCommit_XVR_Flink.json | 3 +++ sdks/go/examples/wasm/README.md | 2 +- sdks/python/apache_beam/options/pipeline_options.py | 2 +- sdks/typescript/src/apache_beam/runners/flink.ts | 2 +- .../www/site/content/en/documentation/runners/flink.md | 8 +++++++- 13 files changed, 33 insertions(+), 8 deletions(-) create mode 100644 .github/trigger_files/beam_PostCommit_Java_Examples_Flink.json create mode 100644 .github/trigger_files/beam_PostCommit_Java_Jpms_Flink_Java11.json create mode 100644 .github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Flink_Java8.json create mode 100644 .github/trigger_files/beam_PostCommit_XVR_Flink.json diff --git a/.github/actions/setup-default-test-properties/test-properties.json b/.github/actions/setup-default-test-properties/test-properties.json index 098e4ca1935c..efe66de8ee1e 100644 --- a/.github/actions/setup-default-test-properties/test-properties.json +++ b/.github/actions/setup-default-test-properties/test-properties.json @@ -14,7 +14,7 @@ }, "JavaTestProperties": { "SUPPORTED_VERSIONS": ["8", "11", "17", "21"], - "FLINK_VERSIONS": ["1.15", "1.16", "1.17", "1.18"], + "FLINK_VERSIONS": ["1.15", "1.16", "1.17", "1.18", "1.19"], "SPARK_VERSIONS": ["2", "3"] }, "GoTestProperties": { diff --git a/.github/trigger_files/beam_PostCommit_Go_VR_Flink.json b/.github/trigger_files/beam_PostCommit_Go_VR_Flink.json index e3d6056a5de9..b98aece75634 100644 --- a/.github/trigger_files/beam_PostCommit_Go_VR_Flink.json +++ b/.github/trigger_files/beam_PostCommit_Go_VR_Flink.json @@ -1,4 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 1 + "modification": 1, + "https://github.com/apache/beam/pull/32648": "testing addition of Flink 1.19 support" } diff --git a/.github/trigger_files/beam_PostCommit_Java_Examples_Flink.json b/.github/trigger_files/beam_PostCommit_Java_Examples_Flink.json new file mode 100644 index 000000000000..dd9afb90e638 --- /dev/null +++ b/.github/trigger_files/beam_PostCommit_Java_Examples_Flink.json @@ -0,0 +1,3 @@ +{ + "https://github.com/apache/beam/pull/32648": "testing flink 1.19 support" +} diff --git a/.github/trigger_files/beam_PostCommit_Java_Jpms_Flink_Java11.json b/.github/trigger_files/beam_PostCommit_Java_Jpms_Flink_Java11.json new file mode 100644 index 000000000000..dd9afb90e638 --- /dev/null +++ b/.github/trigger_files/beam_PostCommit_Java_Jpms_Flink_Java11.json @@ -0,0 +1,3 @@ +{ + "https://github.com/apache/beam/pull/32648": "testing flink 1.19 support" +} diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Flink.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Flink.json index b970762c8397..9200c368abbe 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Flink.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Flink.json @@ -1,4 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test" + "https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test", + "https://github.com/apache/beam/pull/32648": "testing addition of Flink 1.19 support" } diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.json index b970762c8397..9200c368abbe 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.json @@ -1,4 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test" + "https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test", + "https://github.com/apache/beam/pull/32648": "testing addition of Flink 1.19 support" } diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Flink_Java8.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Flink_Java8.json new file mode 100644 index 000000000000..b07a3c47e196 --- /dev/null +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Flink_Java8.json @@ -0,0 +1,4 @@ +{ + + "https://github.com/apache/beam/pull/32648": "testing addition of Flink 1.19 support" +} diff --git a/.github/trigger_files/beam_PostCommit_Python_ValidatesRunner_Flink.json b/.github/trigger_files/beam_PostCommit_Python_ValidatesRunner_Flink.json index e69de29bb2d1..0b34d452d42c 100644 --- a/.github/trigger_files/beam_PostCommit_Python_ValidatesRunner_Flink.json +++ b/.github/trigger_files/beam_PostCommit_Python_ValidatesRunner_Flink.json @@ -0,0 +1,3 @@ +{ + "https://github.com/apache/beam/pull/32648": "testing addition of Flink 1.19 support" +} diff --git a/.github/trigger_files/beam_PostCommit_XVR_Flink.json b/.github/trigger_files/beam_PostCommit_XVR_Flink.json new file mode 100644 index 000000000000..0b34d452d42c --- /dev/null +++ b/.github/trigger_files/beam_PostCommit_XVR_Flink.json @@ -0,0 +1,3 @@ +{ + "https://github.com/apache/beam/pull/32648": "testing addition of Flink 1.19 support" +} diff --git a/sdks/go/examples/wasm/README.md b/sdks/go/examples/wasm/README.md index 84d30a3c6a63..a78649134305 100644 --- a/sdks/go/examples/wasm/README.md +++ b/sdks/go/examples/wasm/README.md @@ -68,7 +68,7 @@ cd $BEAM_HOME Expected output should include the following, from which you acquire the latest flink runner version. ```shell -'flink_versions: 1.15,1.16,1.17,1.18' +'flink_versions: 1.15,1.16,1.17,1.18,1.19' ``` #### 2. Set to the latest flink runner version i.e. 1.16 diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index 4497ab0993a4..837dc0f5439f 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -1679,7 +1679,7 @@ def _add_argparse_args(cls, parser): class FlinkRunnerOptions(PipelineOptions): # These should stay in sync with gradle.properties. - PUBLISHED_FLINK_VERSIONS = ['1.15', '1.16', '1.17', '1.18'] + PUBLISHED_FLINK_VERSIONS = ['1.15', '1.16', '1.17', '1.18', '1.19'] @classmethod def _add_argparse_args(cls, parser): diff --git a/sdks/typescript/src/apache_beam/runners/flink.ts b/sdks/typescript/src/apache_beam/runners/flink.ts index ad4339b431f5..e21876c0d517 100644 --- a/sdks/typescript/src/apache_beam/runners/flink.ts +++ b/sdks/typescript/src/apache_beam/runners/flink.ts @@ -28,7 +28,7 @@ import { JavaJarService } from "../utils/service"; const MAGIC_HOST_NAMES = ["[local]", "[auto]"]; // These should stay in sync with gradle.properties. -const PUBLISHED_FLINK_VERSIONS = ["1.15", "1.16", "1.17", "1.18"]; +const PUBLISHED_FLINK_VERSIONS = ["1.15", "1.16", "1.17", "1.18", "1.19"]; const defaultOptions = { flinkMaster: "[local]", diff --git a/website/www/site/content/en/documentation/runners/flink.md b/website/www/site/content/en/documentation/runners/flink.md index 7325c480955c..2c28aa7062ec 100644 --- a/website/www/site/content/en/documentation/runners/flink.md +++ b/website/www/site/content/en/documentation/runners/flink.md @@ -93,7 +93,7 @@ from the [compatibility table](#flink-version-compatibility) below. For example: {{< highlight java >}} org.apache.beam - beam-runners-flink-1.17 + beam-runners-flink-1.19 {{< param release_latest >}} {{< /highlight >}} @@ -200,6 +200,7 @@ Starting with Beam 2.18.0, pre-built Flink Job Service Docker images are availab [Flink 1.16](https://hub.docker.com/r/apache/beam_flink1.16_job_server). [Flink 1.17](https://hub.docker.com/r/apache/beam_flink1.17_job_server). [Flink 1.18](https://hub.docker.com/r/apache/beam_flink1.18_job_server). +[Flink 1.19](https://hub.docker.com/r/apache/beam_flink1.19_job_server). {{< /paragraph >}} @@ -326,6 +327,11 @@ To find out which version of Flink is compatible with Beam please see the table Artifact Id Supported Beam Versions + + 1.19.x + beam-runners-flink-1.19 + ≥ 2.61.0 + 1.18.x beam-runners-flink-1.18 From c288f1ad38410b266a8bf6ed5b54dc2d55935de3 Mon Sep 17 00:00:00 2001 From: Reeba Qureshi Date: Fri, 4 Oct 2024 20:01:56 +0530 Subject: [PATCH 12/78] minor changes --- .../apache_beam/yaml/standard_providers.yaml | 3 +-- sdks/python/apache_beam/yaml/yaml_provider.py | 16 ++++------------ 2 files changed, 5 insertions(+), 14 deletions(-) diff --git a/sdks/python/apache_beam/yaml/standard_providers.yaml b/sdks/python/apache_beam/yaml/standard_providers.yaml index 15d5fdc24914..242faaa9a77b 100644 --- a/sdks/python/apache_beam/yaml/standard_providers.yaml +++ b/sdks/python/apache_beam/yaml/standard_providers.yaml @@ -103,7 +103,6 @@ gradle_target: 'sdks:java:extensions:sql:expansion-service:shadowJar' - type: 'python' - config: - requires_gcp: true + config: {} transforms: Enrichment: 'apache_beam.yaml.yaml_enrichment.enrichment_transform' diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index 2d6ed2e5b956..c2cba936abce 100755 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -231,17 +231,9 @@ def provider_from_spec(cls, spec): result.to_json = lambda: spec return result except Exception as exn: - if isinstance(exn, ModuleNotFoundError) and config.get('requires_gcp', - False): - print( - f"gcp dependencies not installed. Cannot use transforms: " - f"{', '.join(urns.keys())}. Please install using " - f"'pip install apache-beam[gcp]'.") - return InlineProvider({}) - else: - raise ValueError( - f'Unable to instantiate provider of type {type} ' - f'at line {SafeLineLoader.get_line(spec)}: {exn}') from exn + raise ValueError( + f'Unable to instantiate provider of type {type} ' + f'at line {SafeLineLoader.get_line(spec)}: {exn}') from exn else: raise NotImplementedError( f'Unknown provider type: {type} ' @@ -343,7 +335,7 @@ def cache_artifacts(self): @ExternalProvider.register_provider_type('python') -def python(urns, packages=(), requires_gcp=False): +def python(urns, packages=()): if packages: return ExternalPythonProvider(urns, packages) else: From 42a0d20ed299ab4b6beb2db2fec5629bfd4062f2 Mon Sep 17 00:00:00 2001 From: Jack McCluskey Date: Thu, 10 Oct 2024 10:44:19 -0400 Subject: [PATCH 13/78] Update Pydoc Dependencies --- sdks/python/setup.py | 4 ++-- sdks/python/tox.ini | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 6eb74e9099c1..375354ccc117 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -400,11 +400,11 @@ def get_portability_package_data(): extras_require={ 'docs': [ 'jinja2>=3.0,<3.2', - 'Sphinx>=1.5.2,<2.0', + 'Sphinx>=7.0.0,<8.0', 'docstring-parser>=0.15,<1.0', # Pinning docutils as a workaround for Sphinx issue: # https://github.com/sphinx-doc/sphinx/issues/9727 - 'docutils==0.17.1', + 'docutils>=0.18.1', 'pandas<2.2.0', 'openai' ], diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index 2dfe0670ed0f..0bba4f574f5b 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -163,10 +163,10 @@ commands = [testenv:docs] extras = test,gcp,docs,interactive,dataframe,dask deps = - Sphinx==1.8.5 + Sphinx==7.4.7 sphinx_rtd_theme==0.4.3 - docutils<0.18 - Jinja2==3.0.3 # TODO(https://github.com/apache/beam/issues/21587): Sphinx version is too old. + docutils>=0.18.1 + Jinja2==3.1.0 commands = time {toxinidir}/scripts/generate_pydoc.sh From f646944ae7000a52b7c2886ed01f968f26b31c7d Mon Sep 17 00:00:00 2001 From: Jack McCluskey Date: Thu, 10 Oct 2024 11:05:59 -0400 Subject: [PATCH 14/78] Bump rtd theme --- sdks/python/setup.py | 2 -- sdks/python/tox.ini | 2 +- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 375354ccc117..15671eeb145b 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -402,8 +402,6 @@ def get_portability_package_data(): 'jinja2>=3.0,<3.2', 'Sphinx>=7.0.0,<8.0', 'docstring-parser>=0.15,<1.0', - # Pinning docutils as a workaround for Sphinx issue: - # https://github.com/sphinx-doc/sphinx/issues/9727 'docutils>=0.18.1', 'pandas<2.2.0', 'openai' diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index 0bba4f574f5b..495d515e6d6f 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -164,7 +164,7 @@ commands = extras = test,gcp,docs,interactive,dataframe,dask deps = Sphinx==7.4.7 - sphinx_rtd_theme==0.4.3 + sphinx_rtd_theme==3.0.1 docutils>=0.18.1 Jinja2==3.1.0 commands = From 38ecfa9bb94bf38d33e36601bfc451b5d8929f88 Mon Sep 17 00:00:00 2001 From: liferoad Date: Thu, 10 Oct 2024 11:09:39 -0400 Subject: [PATCH 15/78] Fix the FlinkRunner for Dataproc --- .../interactive/dataproc/dataproc_cluster_manager.py | 4 +++- .../apache_beam/runners/interactive/interactive_beam.py | 7 ++++--- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/runners/interactive/dataproc/dataproc_cluster_manager.py b/sdks/python/apache_beam/runners/interactive/dataproc/dataproc_cluster_manager.py index 2b39279f43e9..4d260d4a6a56 100644 --- a/sdks/python/apache_beam/runners/interactive/dataproc/dataproc_cluster_manager.py +++ b/sdks/python/apache_beam/runners/interactive/dataproc/dataproc_cluster_manager.py @@ -169,6 +169,7 @@ def create_cluster(self, cluster: dict) -> None: def create_flink_cluster(self) -> None: """Calls _create_cluster with a configuration that enables FlinkRunner.""" init_action_path = self.stage_init_action() + # https://cloud.google.com/php/docs/reference/cloud-dataproc/latest/V1.Cluster cluster = { 'project_id': self.cluster_metadata.project_id, 'cluster_name': self.cluster_metadata.cluster_name, @@ -194,7 +195,8 @@ def create_flink_cluster(self) -> None: }, 'service_account_scopes': [ 'https://www.googleapis.com/auth/cloud-platform' - ] + ], + 'internal_ip_only': False }, 'master_config': { # There must be 1 and only 1 instance of master. diff --git a/sdks/python/apache_beam/runners/interactive/interactive_beam.py b/sdks/python/apache_beam/runners/interactive/interactive_beam.py index 9554abf3a47a..60453b5066c3 100644 --- a/sdks/python/apache_beam/runners/interactive/interactive_beam.py +++ b/sdks/python/apache_beam/runners/interactive/interactive_beam.py @@ -408,10 +408,11 @@ class Clusters: To configure a pipeline to run on a local FlinkRunner, explicitly set the default cluster metadata to None: ib.clusters.set_default_cluster(None). """ - # Explicitly set the Flink version here to ensure compatibility with 2.1 + # Explicitly set the Flink version here to ensure compatibility with 2.2 # Dataproc images: - # https://cloud.google.com/dataproc/docs/concepts/versioning/dataproc-release-2.1 - DATAPROC_FLINK_VERSION = '1.15' + # https://cloud.google.com/dataproc/docs/concepts/versioning/dataproc-release-2.2 + # you can manually override this by importing Clusters + DATAPROC_FLINK_VERSION = '1.17' # The minimum worker number to create a Dataproc cluster. DATAPROC_MINIMUM_WORKER_NUM = 2 From 8c944ca4ec706c4d203c26c399dab6be68a34e80 Mon Sep 17 00:00:00 2001 From: Jack McCluskey Date: Thu, 10 Oct 2024 11:15:14 -0400 Subject: [PATCH 16/78] Remove warning fail --- sdks/python/scripts/generate_pydoc.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/python/scripts/generate_pydoc.sh b/sdks/python/scripts/generate_pydoc.sh index 490406d579e5..d8c14c122225 100755 --- a/sdks/python/scripts/generate_pydoc.sh +++ b/sdks/python/scripts/generate_pydoc.sh @@ -275,7 +275,7 @@ python $(type -p sphinx-build) -v -a -E -q target/docs/source \ # Fail if there are errors or warnings in docs ! grep -q "ERROR:" target/docs/sphinx-build.log || exit 1 -! grep -q "WARNING:" target/docs/sphinx-build.log || exit 1 +# ! grep -q "WARNING:" target/docs/sphinx-build.log || exit 1 # Run tests for code samples, these can be: # - Code blocks using '.. testsetup::', '.. testcode::' and '.. testoutput::' @@ -288,7 +288,7 @@ python -msphinx -M doctest target/docs/source \ # Fail if there are errors or warnings in docs ! grep -q "ERROR:" target/docs/sphinx-doctest.log || exit 1 -! grep -q "WARNING:" target/docs/sphinx-doctest.log || exit 1 +# ! grep -q "WARNING:" target/docs/sphinx-doctest.log || exit 1 # Message is useful only when this script is run locally. In a remote # test environment, this path will be removed when the test completes. From 95bf98338a291b87996ecc2b7d59111b9be35dce Mon Sep 17 00:00:00 2001 From: twosom <72733442+twosom@users.noreply.github.com> Date: Fri, 11 Oct 2024 00:28:27 +0900 Subject: [PATCH 17/78] Add support for Read with Meatadata in `MqttIO` (#32668) * add support for read with metadata in MqttIO * Update CHANGES.md * update javadoc * update javadoc * refactor : change to use SchemaCoder in MqttIO - remove MqttRecordCoder - refactor MqttRecord to use AutoValueSchema - change related test --- CHANGES.md | 1 + .../org/apache/beam/sdk/io/mqtt/MqttIO.java | 163 ++++++++++++++---- .../apache/beam/sdk/io/mqtt/MqttRecord.java | 49 ++++++ .../apache/beam/sdk/io/mqtt/MqttIOTest.java | 78 ++++++++- 4 files changed, 257 insertions(+), 34 deletions(-) create mode 100644 sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttRecord.java diff --git a/CHANGES.md b/CHANGES.md index 6a70a49b2ab1..39bad44dc52c 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -68,6 +68,7 @@ ## New Features / Improvements +* Added support for read with metadata in MqttIO (Java) ([#32195](https://github.com/apache/beam/issues/32195)) * X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). * Added support for processing events which use a global sequence to "ordered" extension (Java) [#32540](https://github.com/apache/beam/pull/32540) diff --git a/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java b/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java index e1868e2c8461..efc51362d06a 100644 --- a/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java +++ b/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.io.mqtt; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; import com.google.auto.value.AutoValue; import java.io.IOException; @@ -36,6 +37,7 @@ import org.apache.beam.sdk.coders.SerializableCoder; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.schemas.NoSuchSchemaException; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; @@ -80,6 +82,48 @@ * * } * + *

Reading with Metadata from a MQTT broker

+ * + *

The {@code readWithMetadata} method extends the functionality of the basic {@code read} method + * by returning a {@link PCollection} of metadata that includes both the topic name and the payload. + * The metadata is encapsulated in a container class {@link MqttRecord} that includes the topic name + * and payload. This allows you to implement business logic that can differ depending on the topic + * from which the message was received. + * + *

{@code
+ * PCollection records = pipeline.apply(
+ *   MqttIO.readWithMetadata()
+ *    .withConnectionConfiguration(MqttIO.ConnectionConfiguration.create(
+ *      "tcp://host:11883",
+ *      "my_topic_pattern"))
+ *
+ * }
+ * + *

By using the topic information, you can apply different processing logic depending on the + * source topic, enhancing the flexibility of message processing. + * + *

Example

+ * + *
{@code
+ * pipeline
+ *   .apply(MqttIO.readWithMetadata()
+ *     .withConnectionConfiguration(MqttIO.ConnectionConfiguration.create(
+ *       "tcp://host:1883", "my_topic_pattern")))
+ *   .apply(ParDo.of(new DoFn() {
+ *     @ProcessElement
+ *     public void processElement(ProcessContext c) {
+ *       MqttRecord record = c.element();
+ *       String topic = record.getTopic();
+ *       byte[] payload = record.getPayload();
+ *       // Apply business logic based on the topic
+ *       if (topic.equals("important_topic")) {
+ *         // Special processing for important_topic
+ *       }
+ *     }
+ *   }));
+ *
+ * }
+ * *

Writing to a MQTT broker

* *

MqttIO sink supports writing {@code byte[]} to a topic on a MQTT broker. @@ -130,9 +174,18 @@ public class MqttIO { private static final Logger LOG = LoggerFactory.getLogger(MqttIO.class); private static final int MQTT_3_1_MAX_CLIENT_ID_LENGTH = 23; - public static Read read() { - return new AutoValue_MqttIO_Read.Builder() + public static Read read() { + return new AutoValue_MqttIO_Read.Builder() .setMaxReadTime(null) + .setWithMetadata(false) + .setMaxNumRecords(Long.MAX_VALUE) + .build(); + } + + public static Read readWithMetadata() { + return new AutoValue_MqttIO_Read.Builder() + .setMaxReadTime(null) + .setWithMetadata(true) .setMaxNumRecords(Long.MAX_VALUE) .build(); } @@ -267,7 +320,7 @@ private MQTT createClient() throws Exception { /** A {@link PTransform} to read from a MQTT broker. */ @AutoValue - public abstract static class Read extends PTransform> { + public abstract static class Read extends PTransform> { abstract @Nullable ConnectionConfiguration connectionConfiguration(); @@ -275,21 +328,29 @@ public abstract static class Read extends PTransform abstract @Nullable Duration maxReadTime(); - abstract Builder builder(); + abstract Builder builder(); + + abstract boolean withMetadata(); + + abstract @Nullable Coder coder(); @AutoValue.Builder - abstract static class Builder { - abstract Builder setConnectionConfiguration(ConnectionConfiguration config); + abstract static class Builder { + abstract Builder setConnectionConfiguration(ConnectionConfiguration config); + + abstract Builder setMaxNumRecords(long maxNumRecords); - abstract Builder setMaxNumRecords(long maxNumRecords); + abstract Builder setMaxReadTime(Duration maxReadTime); - abstract Builder setMaxReadTime(Duration maxReadTime); + abstract Builder setWithMetadata(boolean withMetadata); - abstract Read build(); + abstract Builder setCoder(Coder coder); + + abstract Read build(); } /** Define the MQTT connection configuration used to connect to the MQTT broker. */ - public Read withConnectionConfiguration(ConnectionConfiguration configuration) { + public Read withConnectionConfiguration(ConnectionConfiguration configuration) { checkArgument(configuration != null, "configuration can not be null"); return builder().setConnectionConfiguration(configuration).build(); } @@ -299,7 +360,7 @@ public Read withConnectionConfiguration(ConnectionConfiguration configuration) { * records is lower than {@code Long.MAX_VALUE}, the {@link Read} will provide a bounded {@link * PCollection}. */ - public Read withMaxNumRecords(long maxNumRecords) { + public Read withMaxNumRecords(long maxNumRecords) { return builder().setMaxNumRecords(maxNumRecords).build(); } @@ -307,19 +368,33 @@ public Read withMaxNumRecords(long maxNumRecords) { * Define the max read time (duration) while the {@link Read} will receive messages. When this * max read time is not null, the {@link Read} will provide a bounded {@link PCollection}. */ - public Read withMaxReadTime(Duration maxReadTime) { + public Read withMaxReadTime(Duration maxReadTime) { return builder().setMaxReadTime(maxReadTime).build(); } @Override - public PCollection expand(PBegin input) { + @SuppressWarnings("unchecked") + public PCollection expand(PBegin input) { checkArgument(connectionConfiguration() != null, "connectionConfiguration can not be null"); checkArgument(connectionConfiguration().getTopic() != null, "topic can not be null"); - org.apache.beam.sdk.io.Read.Unbounded unbounded = - org.apache.beam.sdk.io.Read.from(new UnboundedMqttSource(this)); + Coder coder; + if (withMetadata()) { + try { + coder = + (Coder) input.getPipeline().getSchemaRegistry().getSchemaCoder(MqttRecord.class); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e.getMessage()); + } + } else { + coder = (Coder) ByteArrayCoder.of(); + } + + org.apache.beam.sdk.io.Read.Unbounded unbounded = + org.apache.beam.sdk.io.Read.from( + new UnboundedMqttSource<>(this.builder().setCoder(coder).build())); - PTransform> transform = unbounded; + PTransform> transform = unbounded; if (maxNumRecords() < Long.MAX_VALUE || maxReadTime() != null) { transform = unbounded.withMaxReadTime(maxReadTime()).withMaxNumRecords(maxNumRecords()); @@ -403,27 +478,39 @@ public int hashCode() { } @VisibleForTesting - static class UnboundedMqttSource extends UnboundedSource { + static class UnboundedMqttSource extends UnboundedSource { - private final Read spec; + private final Read spec; - public UnboundedMqttSource(Read spec) { + public UnboundedMqttSource(Read spec) { this.spec = spec; } @Override - public UnboundedReader createReader( + @SuppressWarnings("unchecked") + public UnboundedReader createReader( PipelineOptions options, MqttCheckpointMark checkpointMark) { - return new UnboundedMqttReader(this, checkpointMark); + final UnboundedMqttReader unboundedMqttReader; + if (spec.withMetadata()) { + unboundedMqttReader = + new UnboundedMqttReader<>( + this, + checkpointMark, + message -> (T) MqttRecord.of(message.getTopic(), message.getPayload())); + } else { + unboundedMqttReader = new UnboundedMqttReader<>(this, checkpointMark); + } + + return unboundedMqttReader; } @Override - public List split(int desiredNumSplits, PipelineOptions options) { + public List> split(int desiredNumSplits, PipelineOptions options) { // MQTT is based on a pub/sub pattern // so, if we create several subscribers on the same topic, they all will receive the same // message, resulting to duplicate messages in the PCollection. // So, for MQTT, we limit to number of split ot 1 (unique source). - return Collections.singletonList(new UnboundedMqttSource(spec)); + return Collections.singletonList(new UnboundedMqttSource<>(spec)); } @Override @@ -437,23 +524,24 @@ public Coder getCheckpointMarkCoder() { } @Override - public Coder getOutputCoder() { - return ByteArrayCoder.of(); + public Coder getOutputCoder() { + return checkNotNull(this.spec.coder(), "coder can not be null"); } } @VisibleForTesting - static class UnboundedMqttReader extends UnboundedSource.UnboundedReader { + static class UnboundedMqttReader extends UnboundedSource.UnboundedReader { - private final UnboundedMqttSource source; + private final UnboundedMqttSource source; private MQTT client; private BlockingConnection connection; - private byte[] current; + private T current; private Instant currentTimestamp; private MqttCheckpointMark checkpointMark; + private SerializableFunction extractFn; - public UnboundedMqttReader(UnboundedMqttSource source, MqttCheckpointMark checkpointMark) { + public UnboundedMqttReader(UnboundedMqttSource source, MqttCheckpointMark checkpointMark) { this.source = source; this.current = null; if (checkpointMark != null) { @@ -461,12 +549,21 @@ public UnboundedMqttReader(UnboundedMqttSource source, MqttCheckpointMark checkp } else { this.checkpointMark = new MqttCheckpointMark(); } + this.extractFn = message -> (T) message.getPayload(); + } + + public UnboundedMqttReader( + UnboundedMqttSource source, + MqttCheckpointMark checkpointMark, + SerializableFunction extractFn) { + this(source, checkpointMark); + this.extractFn = extractFn; } @Override public boolean start() throws IOException { LOG.debug("Starting MQTT reader ..."); - Read spec = source.spec; + Read spec = source.spec; try { client = spec.connectionConfiguration().createClient(); LOG.debug("Reader client ID is {}", client.getClientId()); @@ -488,7 +585,7 @@ public boolean advance() throws IOException { if (message == null) { return false; } - current = message.getPayload(); + current = this.extractFn.apply(message); currentTimestamp = Instant.now(); checkpointMark.add(message, currentTimestamp); } catch (Exception e) { @@ -520,7 +617,7 @@ public UnboundedSource.CheckpointMark getCheckpointMark() { } @Override - public byte[] getCurrent() { + public T getCurrent() { if (current == null) { throw new NoSuchElementException(); } @@ -536,7 +633,7 @@ public Instant getCurrentTimestamp() { } @Override - public UnboundedMqttSource getCurrentSource() { + public UnboundedMqttSource getCurrentSource() { return source; } } diff --git a/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttRecord.java b/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttRecord.java new file mode 100644 index 000000000000..bbf27f5c73e7 --- /dev/null +++ b/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttRecord.java @@ -0,0 +1,49 @@ +/* + * 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. + */ +package org.apache.beam.sdk.io.mqtt; + +import com.google.auto.value.AutoValue; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; + +/** A container class for MQTT message metadata, including the topic name and payload. */ +@DefaultSchema(AutoValueSchema.class) +@AutoValue +public abstract class MqttRecord { + public abstract String getTopic(); + + @SuppressWarnings("mutable") + public abstract byte[] getPayload(); + + static Builder builder() { + return new AutoValue_MqttRecord.Builder(); + } + + static MqttRecord of(String topic, byte[] payload) { + return builder().setTopic(topic).setPayload(payload).build(); + } + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setTopic(String topic); + + abstract Builder setPayload(byte[] payload); + + abstract MqttRecord build(); + } +} diff --git a/sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java b/sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java index 8dfa7838d66a..64b0728c879a 100644 --- a/sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java +++ b/sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java @@ -44,6 +44,7 @@ import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; @@ -93,7 +94,7 @@ public void startBroker() throws Exception { @Ignore("https://github.com/apache/beam/issues/18723 Test timeout failure.") public void testReadNoClientId() throws Exception { final String topicName = "READ_TOPIC_NO_CLIENT_ID"; - Read mqttReader = + Read mqttReader = MqttIO.read() .withConnectionConfiguration( MqttIO.ConnectionConfiguration.create("tcp://localhost:" + port, topicName)) @@ -214,6 +215,81 @@ public void testRead() throws Exception { publishConnection.disconnect(); } + @Test(timeout = 60 * 1000) + public void testReadWithMetadata() throws Exception { + final String wildcardTopic = "topic/#"; + final String topic1 = "topic/1"; + final String topic2 = "topic/2"; + + final PTransform> mqttReaderWithMetadata = + MqttIO.readWithMetadata() + .withConnectionConfiguration( + MqttIO.ConnectionConfiguration.create("tcp://localhost:" + port, wildcardTopic)) + .withMaxNumRecords(10); + + final PCollection output = pipeline.apply(mqttReaderWithMetadata); + PAssert.that(output) + .containsInAnyOrder( + MqttRecord.of(topic1, "This is test 0".getBytes(StandardCharsets.UTF_8)), + MqttRecord.of(topic1, "This is test 1".getBytes(StandardCharsets.UTF_8)), + MqttRecord.of(topic1, "This is test 2".getBytes(StandardCharsets.UTF_8)), + MqttRecord.of(topic1, "This is test 3".getBytes(StandardCharsets.UTF_8)), + MqttRecord.of(topic1, "This is test 4".getBytes(StandardCharsets.UTF_8)), + MqttRecord.of(topic2, "This is test 5".getBytes(StandardCharsets.UTF_8)), + MqttRecord.of(topic2, "This is test 6".getBytes(StandardCharsets.UTF_8)), + MqttRecord.of(topic2, "This is test 7".getBytes(StandardCharsets.UTF_8)), + MqttRecord.of(topic2, "This is test 8".getBytes(StandardCharsets.UTF_8)), + MqttRecord.of(topic2, "This is test 9".getBytes(StandardCharsets.UTF_8))); + + // produce messages on the brokerService in another thread + // This thread prevents to block the pipeline waiting for new messages + MQTT client = new MQTT(); + client.setHost("tcp://localhost:" + port); + final BlockingConnection publishConnection = client.blockingConnection(); + publishConnection.connect(); + Thread publisherThread = + new Thread( + () -> { + try { + LOG.info( + "Waiting pipeline connected to the MQTT broker before sending " + + "messages ..."); + boolean pipelineConnected = false; + while (!pipelineConnected) { + Thread.sleep(1000); + for (Connection connection : brokerService.getBroker().getClients()) { + if (!connection.getConnectionId().isEmpty()) { + pipelineConnected = true; + } + } + } + for (int i = 0; i < 5; i++) { + publishConnection.publish( + topic1, + ("This is test " + i).getBytes(StandardCharsets.UTF_8), + QoS.EXACTLY_ONCE, + false); + } + for (int i = 5; i < 10; i++) { + publishConnection.publish( + topic2, + ("This is test " + i).getBytes(StandardCharsets.UTF_8), + QoS.EXACTLY_ONCE, + false); + } + + } catch (Exception e) { + // nothing to do + } + }); + + publisherThread.start(); + pipeline.run(); + + publishConnection.disconnect(); + publisherThread.join(); + } + /** Test for BEAM-3282: this test should not timeout. */ @Test(timeout = 30 * 1000) public void testReceiveWithTimeoutAndNoData() throws Exception { From ec0f2f6b933e218a699fe0c8c55bfb646fbe0a33 Mon Sep 17 00:00:00 2001 From: Shunping Huang Date: Thu, 10 Oct 2024 11:42:25 -0400 Subject: [PATCH 18/78] Attempt to fix the permred "PostCommit Java IO Performance Tests" (#32736) * Attempt to fix permred test. * Also remove token format, making it consistent with java_tests and python_tests. * Remove the triggering file since the test is only triggered with the yaml file on master. --- .../workflows/beam_PostCommit_Java_IO_Performance_Tests.yml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml index b8c79e2677ca..2cc1523e51b5 100644 --- a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml +++ b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml @@ -89,11 +89,10 @@ jobs: with: java-version: default - name: Authenticate on GCP - uses: google-github-actions/auth@v2 + uses: google-github-actions/auth@v1 with: credentials_json: ${{ secrets.GCP_SA_KEY }} project_id: ${{ secrets.GCP_PROJECT_ID }} - token_format: 'access_token' - name: Setup gcloud uses: google-github-actions/setup-gcloud@v2 with: From 75d804c1acbcc0bf7951c11dd16e2446b90ae830 Mon Sep 17 00:00:00 2001 From: fozzie15 <134544020+fozzie15@users.noreply.github.com> Date: Thu, 10 Oct 2024 15:49:28 +0000 Subject: [PATCH 19/78] Add Application Default Authentication as an Option for Kafka Clients in KafkaIO When Using Managed Service for Apache Kafka (#32722) * Update setup.py to latest version of facets to fix outdated alias issue * Add application default support in KafkaIO for users who want to authenticate their Kafka Clients with Google's server. * Add application default support in KafkaIO for users who want to authenticate their Kafka Clients with Google's server. --------- Co-authored-by: Ozzie Fernandez --- sdks/java/io/kafka/build.gradle | 1 + .../org/apache/beam/sdk/io/kafka/KafkaIO.java | 37 ++++++++++++ .../apache/beam/sdk/io/kafka/KafkaIOIT.java | 57 +++++++++++++++++++ 3 files changed, 95 insertions(+) diff --git a/sdks/java/io/kafka/build.gradle b/sdks/java/io/kafka/build.gradle index e30099906391..e6df08d41199 100644 --- a/sdks/java/io/kafka/build.gradle +++ b/sdks/java/io/kafka/build.gradle @@ -72,6 +72,7 @@ dependencies { implementation library.java.jackson_annotations implementation library.java.jackson_databind implementation "org.springframework:spring-expression:5.3.27" + implementation group: 'com.google.cloud.hosted.kafka', name: 'managed-kafka-auth-login-handler', version: '1.0.2' implementation ("io.confluent:kafka-avro-serializer:${confluentVersion}") { // zookeeper depends on "spotbugs-annotations:3.1.9" which clashes with current // "spotbugs-annotations:3.1.12" used in Beam. Not required. diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java index 0f28edf19dd8..cb7b3020c66a 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java @@ -109,6 +109,7 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Comparators; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -118,6 +119,7 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.config.SaslConfigs; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; @@ -1453,6 +1455,24 @@ public Read withConsumerPollingTimeout(long duration) { return toBuilder().setConsumerPollingTimeout(duration).build(); } + /** + * Creates and sets the Application Default Credentials for a Kafka consumer. This allows the + * consumer to be authenticated with a Google Kafka Server using OAuth. + */ + public Read withGCPApplicationDefaultCredentials() { + + return withConsumerConfigUpdates( + ImmutableMap.of( + CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, + "SASL_SSL", + SaslConfigs.SASL_MECHANISM, + "OAUTHBEARER", + SaslConfigs.SASL_LOGIN_CALLBACK_HANDLER_CLASS, + "com.google.cloud.hosted.kafka.auth.GcpLoginCallbackHandler", + SaslConfigs.SASL_JAAS_CONFIG, + "org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule required;")); + } + /** Returns a {@link PTransform} for PCollection of {@link KV}, dropping Kafka metatdata. */ public PTransform>> withoutMetadata() { return new TypedWithoutMetadata<>(this); @@ -3362,6 +3382,23 @@ public Write withBadRecordErrorHandler(ErrorHandler badRecor getWriteRecordsTransform().withBadRecordErrorHandler(badRecordErrorHandler)); } + /** + * Creates and sets the Application Default Credentials for a Kafka producer. This allows the + * consumer to be authenticated with a Google Kafka Server using OAuth. + */ + public Write withGCPApplicationDefaultCredentials() { + return withProducerConfigUpdates( + ImmutableMap.of( + CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, + "SASL_SSL", + SaslConfigs.SASL_MECHANISM, + "OAUTHBEARER", + SaslConfigs.SASL_LOGIN_CALLBACK_HANDLER_CLASS, + "com.google.cloud.hosted.kafka.auth.GcpLoginCallbackHandler", + SaslConfigs.SASL_JAAS_CONFIG, + "org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule required;")); + } + @Override public PDone expand(PCollection> input) { final String topic = Preconditions.checkStateNotNull(getTopic(), "withTopic() is required"); diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java index fba81c51130d..c6edaf7761b5 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java @@ -103,6 +103,7 @@ import org.joda.time.Duration; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; @@ -815,6 +816,62 @@ public void testWatermarkUpdateWithSparseMessages() throws IOException, Interrup } } + @Ignore( + "Test is ignored until GMK is utilized as part of this test suite (https://github.com/apache/beam/issues/32721).") + @Test + public void testReadAndWriteFromKafkaIOWithGCPApplicationDefaultCredentials() throws IOException { + AdminClient client = + AdminClient.create( + ImmutableMap.of("bootstrap.servers", options.getKafkaBootstrapServerAddresses())); + + String topicName = "TestApplicationDefaultCreds-" + UUID.randomUUID(); + Map records = new HashMap<>(); + for (int i = 0; i < 5; i++) { + records.put(i, String.valueOf(i)); + } + + try { + client.createTopics(ImmutableSet.of(new NewTopic(topicName, 1, (short) 1))); + + writePipeline + .apply("Generate Write Elements", Create.of(records)) + .apply( + "Write to Kafka", + KafkaIO.write() + .withBootstrapServers(options.getKafkaBootstrapServerAddresses()) + .withTopic(topicName) + .withKeySerializer(IntegerSerializer.class) + .withValueSerializer(StringSerializer.class) + .withGCPApplicationDefaultCredentials()); + + writePipeline.run().waitUntilFinish(Duration.standardSeconds(15)); + + client.createPartitions(ImmutableMap.of(topicName, NewPartitions.increaseTo(3))); + + sdfReadPipeline.apply( + "Read from Kafka", + KafkaIO.read() + .withBootstrapServers(options.getKafkaBootstrapServerAddresses()) + .withConsumerConfigUpdates(ImmutableMap.of("auto.offset.reset", "earliest")) + .withTopic(topicName) + .withKeyDeserializer(IntegerDeserializer.class) + .withValueDeserializer(StringDeserializer.class) + .withGCPApplicationDefaultCredentials() + .withoutMetadata()); + + PipelineResult readResult = sdfReadPipeline.run(); + + // Only waiting 5 seconds here because we don't expect any processing at this point + PipelineResult.State readState = readResult.waitUntilFinish(Duration.standardSeconds(5)); + + cancelIfTimeouted(readResult, readState); + // Fail the test if pipeline failed. + assertNotEquals(readState, PipelineResult.State.FAILED); + } finally { + client.deleteTopics(ImmutableSet.of(topicName)); + } + } + private static class KeyByPartition extends DoFn, KV>> { From 538db45b3bed63160bce6454a0fd7e85b7b83fe8 Mon Sep 17 00:00:00 2001 From: Shunping Huang Date: Thu, 10 Oct 2024 12:43:36 -0400 Subject: [PATCH 20/78] Remove setup-gcloud as they are not in java_tests and python_tests (#32741) --- .../workflows/beam_PostCommit_Java_IO_Performance_Tests.yml | 5 ----- 1 file changed, 5 deletions(-) diff --git a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml index 2cc1523e51b5..a6a2749c8d82 100644 --- a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml +++ b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml @@ -93,11 +93,6 @@ jobs: with: credentials_json: ${{ secrets.GCP_SA_KEY }} project_id: ${{ secrets.GCP_PROJECT_ID }} - - name: Setup gcloud - uses: google-github-actions/setup-gcloud@v2 - with: - project_id: ${{ secrets.GCP_PROJECT_ID }} - skip_install: true - name: run scheduled javaPostcommitIOPerformanceTests script if: github.event_name == 'schedule' #This ensures only scheduled runs publish metrics publicly by changing which exportTable is configured uses: ./.github/actions/gradle-command-self-hosted-action From 4cf99ea31aa2f3b4e8fa0da479939bbe92bd46dd Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Thu, 10 Oct 2024 13:56:49 -0400 Subject: [PATCH 21/78] Add check to avro and kafka compatibility tests (#32698) * Also fix Kafka integration test version resolution --- .../gradle/kafka/KafkaTestUtilities.groovy | 4 +- sdks/java/extensions/avro/build.gradle | 1 + .../avro/AvroVersionVerificationTest.java | 39 +++++++++++++++++++ sdks/java/io/kafka/build.gradle | 2 + .../io/kafka/kafka-integration-test.gradle | 6 +-- .../beam/sdk/io/kafka/ReadFromKafkaDoFn.java | 14 ++++--- .../apache/beam/sdk/io/kafka/KafkaIOIT.java | 14 +++++++ .../apache/beam/sdk/io/kafka/KafkaIOTest.java | 17 ++++++++ .../sdk/io/kafka/ReadFromKafkaDoFnTest.java | 5 +++ 9 files changed, 91 insertions(+), 11 deletions(-) create mode 100644 sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/AvroVersionVerificationTest.java diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/kafka/KafkaTestUtilities.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/kafka/KafkaTestUtilities.groovy index bb08e79edd3c..a3ae6833d579 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/kafka/KafkaTestUtilities.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/kafka/KafkaTestUtilities.groovy @@ -28,11 +28,13 @@ class KafkaTestUtilities { @Inject KafkaBatchIT(String delimited, String undelimited, Boolean sdfCompatible, ConfigurationContainer configurations, Project runningProject){ + def kafkaioProject = runningProject.findProject(":sdks:java:io:kafka") group = "Verification" description = "Runs KafkaIO IT tests with Kafka clients API $delimited" outputs.upToDateWhen { false } testClassesDirs = runningProject.findProject(":sdks:java:io:kafka").sourceSets.test.output.classesDirs - classpath = configurations."kafkaVersion$undelimited" + runningProject.sourceSets.test.runtimeClasspath + runningProject.findProject(":sdks:java:io:kafka").sourceSets.test.runtimeClasspath + classpath = runningProject.sourceSets.test.runtimeClasspath + kafkaioProject.configurations."kafkaVersion$undelimited" + kafkaioProject.sourceSets.test.runtimeClasspath + systemProperty "beam.target.kafka.version", delimited def pipelineOptions = [ '--sourceOptions={' + diff --git a/sdks/java/extensions/avro/build.gradle b/sdks/java/extensions/avro/build.gradle index 8ff0612a0eab..6631779e609c 100644 --- a/sdks/java/extensions/avro/build.gradle +++ b/sdks/java/extensions/avro/build.gradle @@ -128,6 +128,7 @@ avroVersions.each { k, v -> description = "Runs Avro extension tests with Avro version $v" outputs.upToDateWhen { false } classpath = sourceSets."avro$k".runtimeClasspath + systemProperty "beam.target.avro.version", v include '**/*.class' exclude '**/AvroIOTest$NeedsRunnerTests$*.class' diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/AvroVersionVerificationTest.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/AvroVersionVerificationTest.java new file mode 100644 index 000000000000..f9e9a54b0531 --- /dev/null +++ b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/AvroVersionVerificationTest.java @@ -0,0 +1,39 @@ +/* + * 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. + */ +package org.apache.beam.sdk.extensions.avro; + +import static org.junit.Assert.assertEquals; + +import org.apache.avro.Schema; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.junit.Assume; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class AvroVersionVerificationTest { + @Test + public void testAvroVersion() { + @Nullable String targetVer = System.getProperty("beam.target.avro.version"); + Assume.assumeTrue(!Strings.isNullOrEmpty(targetVer)); + String actualVer = Schema.class.getPackage().getImplementationVersion(); + assertEquals(targetVer, actualVer); + } +} diff --git a/sdks/java/io/kafka/build.gradle b/sdks/java/io/kafka/build.gradle index e6df08d41199..0ba6fa642a02 100644 --- a/sdks/java/io/kafka/build.gradle +++ b/sdks/java/io/kafka/build.gradle @@ -123,6 +123,8 @@ kafkaVersions.each {kv -> outputs.upToDateWhen { false } testClassesDirs = sourceSets.test.output.classesDirs classpath = configurations."kafkaVersion${kv.key}" + sourceSets.test.runtimeClasspath + systemProperty "beam.target.kafka.version", kv.value + include '**/KafkaIOTest.class' } } diff --git a/sdks/java/io/kafka/kafka-integration-test.gradle b/sdks/java/io/kafka/kafka-integration-test.gradle index 1aeb0c97f93b..3bbab72ff77c 100644 --- a/sdks/java/io/kafka/kafka-integration-test.gradle +++ b/sdks/java/io/kafka/kafka-integration-test.gradle @@ -29,10 +29,8 @@ provideIntegrationTestingDependencies() enableJavaPerformanceTesting() dependencies { - implementation "org.apache.kafka:kafka-clients:$delimited" - permitUnusedDeclared "org.apache.kafka:kafka-clients:$delimited" - implementation project(":sdks:java:io:kafka") - permitUnusedDeclared project(":sdks:java:io:kafka") + // Do not set kafka-client dependency here otherwise the version will be overwritten by BeamModulePlugin + // instead, rely on io/kafka/build.gradle's custom configurations with forced kafka-client resolutionStrategy testImplementation 'org.junit.jupiter:junit-jupiter-api:5.8.1' testRuntimeOnly 'org.junit.jupiter:junit-jupiter-engine:5.8.1' } diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFn.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFn.java index 952e29f75104..4bda8cf28d4e 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFn.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFn.java @@ -632,13 +632,15 @@ public AverageRecordSize load(TopicPartition topicPartition) throws Exception { @Teardown public void teardown() throws Exception { - final Deserializer keyDeserializerInstance = - Preconditions.checkStateNotNull(this.keyDeserializerInstance); - final Deserializer valueDeserializerInstance = - Preconditions.checkStateNotNull(this.valueDeserializerInstance); try { - Closeables.close(keyDeserializerInstance, true); - Closeables.close(valueDeserializerInstance, true); + if (valueDeserializerInstance != null) { + Closeables.close(valueDeserializerInstance, true); + valueDeserializerInstance = null; + } + if (keyDeserializerInstance != null) { + Closeables.close(keyDeserializerInstance, true); + keyDeserializerInstance = null; + } } catch (Exception anyException) { LOG.warn("Fail to close resource during finishing bundle.", anyException); } diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java index c6edaf7761b5..cef3bc80d613 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNull; +import static org.junit.Assume.assumeFalse; import java.io.IOException; import java.time.Instant; @@ -86,6 +87,7 @@ import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; @@ -99,6 +101,7 @@ import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.utils.AppInfoParser; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; import org.junit.AfterClass; @@ -169,6 +172,13 @@ public class KafkaIOIT { @BeforeClass public static void setup() throws IOException { + // check kafka version first + @Nullable String targetVer = System.getProperty("beam.target.kafka.version"); + if (!Strings.isNullOrEmpty(targetVer)) { + String actualVer = AppInfoParser.getVersion(); + assertEquals(targetVer, actualVer); + } + options = IOITHelper.readIOTestPipelineOptions(Options.class); sourceOptions = fromJsonString(options.getSourceOptions(), SyntheticSourceOptions.class); if (options.isWithTestcontainers()) { @@ -360,6 +370,10 @@ public void processElement(@Element String element, OutputReceiver outpu // This test verifies that bad data from Kafka is properly sent to the error handler @Test public void testKafkaIOSDFReadWithErrorHandler() throws IOException { + // TODO(https://github.com/apache/beam/issues/32704) re-enable when fixed, or remove the support + // for these old kafka-client versions + String actualVer = AppInfoParser.getVersion(); + assumeFalse(actualVer.compareTo("2.0.0") >= 0 && actualVer.compareTo("2.3.0") < 0); writePipeline .apply(Create.of(KV.of("key", "val"))) .apply( diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java index fb8b29fe7280..764e406f71cb 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java @@ -115,6 +115,7 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; @@ -146,12 +147,14 @@ import org.apache.kafka.common.serialization.LongDeserializer; import org.apache.kafka.common.serialization.LongSerializer; import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.common.utils.Utils; import org.checkerframework.checker.nullness.qual.Nullable; import org.hamcrest.collection.IsIterableContainingInAnyOrder; import org.hamcrest.collection.IsIterableWithSize; import org.joda.time.Duration; import org.joda.time.Instant; +import org.junit.Assume; import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; @@ -515,6 +518,15 @@ public void testReadAvroGenericRecordsWithConfluentSchemaRegistry() { p.run(); } + @Test + public void testKafkaVersion() { + // KafkaIO compatibility tests run unit tests in KafkaIOTest + @Nullable String targetVer = System.getProperty("beam.target.kafka.version"); + Assume.assumeTrue(!Strings.isNullOrEmpty(targetVer)); + String actualVer = AppInfoParser.getVersion(); + assertEquals(targetVer, actualVer); + } + @Test public void testReadAvroSpecificRecordsWithConfluentSchemaRegistry() { int numElements = 100; @@ -1582,6 +1594,11 @@ public byte[] serialize(String topic, Long data) { public void configure(Map configs, boolean isKey) { // intentionally left blank for compatibility with older kafka versions } + + @Override + public void close() { + // intentionally left blank for compatibility with kafka-client v2.2 or older + } } @Test diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFnTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFnTest.java index a9e4a4eddb61..3189bbb140f0 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFnTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFnTest.java @@ -150,6 +150,11 @@ public static class FailingDeserializer implements Deserializer { public FailingDeserializer() {} + @Override + public void configure(Map configs, boolean isKey) { + // intentionally left blank for compatibility with older kafka versions + } + @Override public String deserialize(String topic, byte[] data) { throw new SerializationException("Intentional serialization exception"); From 02241acca4e664f4eafcc593070a2168342713ae Mon Sep 17 00:00:00 2001 From: liferoad Date: Thu, 10 Oct 2024 15:39:16 -0400 Subject: [PATCH 22/78] fixed the tox and pyproject settings for numpy 2 --- sdks/python/pyproject.toml | 2 +- sdks/python/tox.ini | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/python/pyproject.toml b/sdks/python/pyproject.toml index 037e5a8aed6b..4eb827297019 100644 --- a/sdks/python/pyproject.toml +++ b/sdks/python/pyproject.toml @@ -26,7 +26,7 @@ requires = [ # Avoid https://github.com/pypa/virtualenv/issues/2006 "distlib==0.3.7", # Numpy headers - "numpy>=1.14.3,<1.27", # Update setup.py as well. + "numpy>=1.14.3,<2.2.0", # Update setup.py as well. # having cython here will create wheels that are platform dependent. "cython>=3.0,<4", ## deps for generating external transform wrappers: diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index 2dfe0670ed0f..b6ea59e83584 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -318,6 +318,7 @@ commands = [testenv:py{39,310}-pandas-{14,15,20}] deps = 14: pandas>=1.4.3,<1.5.0 + 14: numpy>=1.14.3,<1.27.0 # Exclude 1.5.0 and 1.5.1 because of https://github.com/pandas-dev/pandas/issues/45725 15: pandas>=1.5.2,<1.6.0 20: pandas>=2.0.0,<2.1.0 From fefd18475efe0ba22449264becee0348c4eb6349 Mon Sep 17 00:00:00 2001 From: Jack McCluskey Date: Thu, 10 Oct 2024 15:46:39 -0400 Subject: [PATCH 23/78] fix broken imports --- sdks/python/scripts/generate_pydoc.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sdks/python/scripts/generate_pydoc.sh b/sdks/python/scripts/generate_pydoc.sh index d8c14c122225..3462429190c8 100755 --- a/sdks/python/scripts/generate_pydoc.sh +++ b/sdks/python/scripts/generate_pydoc.sh @@ -143,6 +143,8 @@ napoleon_custom_sections = ['Differences from pandas'] doctest_global_setup = ''' import apache_beam as beam +import pandas as pd +import numpy as np ''' intersphinx_mapping = { @@ -283,6 +285,7 @@ python $(type -p sphinx-build) -v -a -E -q target/docs/source \ python -msphinx -M doctest target/docs/source \ target/docs/_build -c target/docs/source \ 2>&1 | grep -E -v 'apache_beam\.dataframe.*WARNING:' \ + 2>&1 | grep -E -v 'apache_beam\.dataframe.*ERROR:' \ 2>&1 | grep -E -v 'apache_beam\.io\.textio\.(ReadFrom|WriteTo)(Csv|Json).*WARNING:' \ 2>&1 | tee "target/docs/sphinx-doctest.log" From 255e6cc42a9346fd9a7b72a17e1a06a0d3259592 Mon Sep 17 00:00:00 2001 From: Jack McCluskey Date: Thu, 10 Oct 2024 16:19:17 -0400 Subject: [PATCH 24/78] fix edge case --- sdks/python/apache_beam/dataframe/frame_base.py | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/sdks/python/apache_beam/dataframe/frame_base.py b/sdks/python/apache_beam/dataframe/frame_base.py index 90f34d45dd98..3b9755232e80 100644 --- a/sdks/python/apache_beam/dataframe/frame_base.py +++ b/sdks/python/apache_beam/dataframe/frame_base.py @@ -606,6 +606,21 @@ def wrap(func): " :skipif: True"), re.sub(r"^", " ", content, flags=re.MULTILINE), ]) + elif "Examples" in content and ">>>" in content: + # some new examples don't have the correct heading + # this catches those examples + split_content = content.split("Examples") + content = '\n\n'.join([ + split_content[0], + "Examples\n", + # Indent the code snippet under a doctest heading, + # add skipif option. This makes sure our doctest + # framework doesn't run these pandas tests. + (".. doctest::\n" + " :skipif: True"), + re.sub(r"^", " ", content, flags=re.MULTILINE), + split_content[1] + ]) else: content = content.replace('DataFrame', 'DeferredDataFrame').replace( 'Series', 'DeferredSeries') From 84aa5ac4a7795986a1f0a6031ad890c74d0b5960 Mon Sep 17 00:00:00 2001 From: Shunping Huang Date: Thu, 10 Oct 2024 17:06:58 -0400 Subject: [PATCH 25/78] Fix the snapshot publishing script (#32744) * Fix the snapshot publishing script * Support distribution from setuptools prior to 69.3.0 as well. --- sdks/python/scripts/run_snapshot_publish.sh | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sdks/python/scripts/run_snapshot_publish.sh b/sdks/python/scripts/run_snapshot_publish.sh index bc379077349d..0d7c7764748d 100755 --- a/sdks/python/scripts/run_snapshot_publish.sh +++ b/sdks/python/scripts/run_snapshot_publish.sh @@ -31,7 +31,9 @@ DEP_SNAPSHOT_FILE_NAME="beam-py-requirements-$time.txt" cd $WORKSPACE/sdks/python/build # Rename the file to be apache-beam-{VERSION}-{datetime}.tar.gz -for file in "apache-beam-$VERSION*.tar.gz"; do +# Notice that the distribution name of beam can be "apache-beam" with +# setuptools<69.3.0 or "apache_beam" with setuptools>=69.3.0. +for file in "apache[-_]beam-$VERSION*.tar.gz"; do mv $file $SNAPSHOT done From 7bb3aa8669e0c50c9f539694f71355b383793215 Mon Sep 17 00:00:00 2001 From: liferoad Date: Thu, 10 Oct 2024 18:08:27 -0400 Subject: [PATCH 26/78] retry the subprocess to start the server --- sdks/python/apache_beam/utils/subprocess_server.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sdks/python/apache_beam/utils/subprocess_server.py b/sdks/python/apache_beam/utils/subprocess_server.py index 944c12625d7c..e91a608731ba 100644 --- a/sdks/python/apache_beam/utils/subprocess_server.py +++ b/sdks/python/apache_beam/utils/subprocess_server.py @@ -39,6 +39,7 @@ import grpc from apache_beam.io.filesystems import FileSystems +from apache_beam.utils import retry from apache_beam.version import __version__ as beam_version _LOGGER = logging.getLogger(__name__) @@ -160,6 +161,7 @@ def __enter__(self): def __exit__(self, *unused_args): self.stop() + @retry.with_exponential_backoff(num_retries=4, initial_delay_secs=2) def start(self): try: process, endpoint = self.start_process() From d629485d21d036e3855a5ed9ec357f875a8be514 Mon Sep 17 00:00:00 2001 From: liferoad Date: Thu, 10 Oct 2024 18:49:15 -0400 Subject: [PATCH 27/78] revert the retry --- sdks/python/apache_beam/utils/subprocess_server.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/sdks/python/apache_beam/utils/subprocess_server.py b/sdks/python/apache_beam/utils/subprocess_server.py index e91a608731ba..944c12625d7c 100644 --- a/sdks/python/apache_beam/utils/subprocess_server.py +++ b/sdks/python/apache_beam/utils/subprocess_server.py @@ -39,7 +39,6 @@ import grpc from apache_beam.io.filesystems import FileSystems -from apache_beam.utils import retry from apache_beam.version import __version__ as beam_version _LOGGER = logging.getLogger(__name__) @@ -161,7 +160,6 @@ def __enter__(self): def __exit__(self, *unused_args): self.stop() - @retry.with_exponential_backoff(num_retries=4, initial_delay_secs=2) def start(self): try: process, endpoint = self.start_process() From 07322cc86d35fd2af5c32228796e7936f58416d6 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Thu, 10 Oct 2024 23:33:55 -0400 Subject: [PATCH 28/78] Fix build_wheels and build_release_candidate (#32725) * Fix build_wheels and build_release_candidate --- .github/workflows/build_release_candidate.yml | 5 +-- .github/workflows/build_wheels.yml | 34 ++++++------------- 2 files changed, 13 insertions(+), 26 deletions(-) diff --git a/.github/workflows/build_release_candidate.yml b/.github/workflows/build_release_candidate.yml index fdbae21336e5..fbb0ca22f333 100644 --- a/.github/workflows/build_release_candidate.yml +++ b/.github/workflows/build_release_candidate.yml @@ -310,11 +310,12 @@ jobs: uses: actions/setup-node@v4 with: node-version: '16' - - name: Install Java 11 + # TODO(https://github.com/apache/beam/issues/32726) switch to Java11 + - name: Install Java 8 uses: actions/setup-java@v4 with: distribution: 'temurin' - java-version: '11' + java-version: '8' - name: Remove default github maven configuration # This step is a workaround to avoid a decryption issue of Beam's # net.linguica.gradle.maven.settings plugin and github's provided maven diff --git a/.github/workflows/build_wheels.yml b/.github/workflows/build_wheels.yml index d1e99f2bd579..0a15ba9d150c 100644 --- a/.github/workflows/build_wheels.yml +++ b/.github/workflows/build_wheels.yml @@ -202,7 +202,6 @@ jobs: if: needs.check_env_variables.outputs.gcp-variables-set == 'true' steps: - name: Download compressed sources from artifacts - # Pinned to v3 because of https://github.com/actions/download-artifact/issues/249 uses: actions/download-artifact@v4.1.8 with: name: source_zip @@ -229,18 +228,16 @@ jobs: {"os": "windows-latest", "runner": "windows-latest", "python": "${{ needs.check_env_variables.outputs.py-versions-test }}", arch: "auto" }, {"os": "ubuntu-20.04", "runner": [self-hosted, ubuntu-20.04, main], "python": "${{ needs.check_env_variables.outputs.py-versions-test }}", arch: "aarch64" } ] - # Keep in sync with PY_VERSIONS_FULL env var abvove - if changed, change that as well. - py_version: ["cp38-*", "cp39-*", "cp310-*", "cp311-*", "cp312-*"] + # Keep in sync (remove asterisks) with PY_VERSIONS_FULL env var above - if changed, change that as well. + py_version: ["cp38-", "cp39-", "cp310-", "cp311-", "cp312-"] steps: - name: Download python source distribution from artifacts - # Pinned to v3 because of https://github.com/actions/download-artifact/issues/249 uses: actions/download-artifact@v4.1.8 with: name: source path: apache-beam-source - name: Download Python SDK RC source distribution from artifacts if: ${{ needs.build_source.outputs.is_rc == 1 }} - # Pinned to v3 because of https://github.com/actions/download-artifact/issues/249 uses: actions/download-artifact@v4.1.8 with: name: source_rc${{ needs.build_source.outputs.rc_num }} @@ -260,7 +257,7 @@ jobs: if: ${{ contains(matrix.os_python.python, matrix.py_version) }} working-directory: apache-beam-source env: - CIBW_BUILD: ${{ matrix.py_version }} + CIBW_BUILD: ${{ matrix.py_version }}* # TODO: https://github.com/apache/beam/issues/23048 CIBW_SKIP: "*-musllinux_*" CIBW_BEFORE_BUILD: pip install cython==0.29.36 numpy --config-settings=setup-args="-Dallow-noblas=true" && pip install --upgrade setuptools @@ -279,17 +276,16 @@ jobs: shell: bash - name: Upload wheels as artifacts if: ${{ contains(matrix.os_python.python, matrix.py_version) }} - # Pinned to v3 because of https://github.com/actions/upload-artifact?tab=readme-ov-file#breaking-changes - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 with: - name: wheelhouse-${{ matrix.os_python.os }}${{ (matrix.os_python.arch == 'aarch64' && '-aarch64') || '' }} + name: wheelhouse-${{ matrix.py_version }}${{ matrix.os_python.os }}${{ (matrix.os_python.arch == 'aarch64' && '-aarch64') || '' }} path: apache-beam-source/wheelhouse/ - name: Build RC wheels # Only build wheel if it is one of the target versions for this platform, otherwise no-op if: ${{ needs.build_source.outputs.is_rc == 1 && contains(matrix.os_python.python, matrix.py_version) }} working-directory: apache-beam-source-rc env: - CIBW_BUILD: ${{ matrix.py_version }} + CIBW_BUILD: ${{ matrix.py_version }}* # TODO: https://github.com/apache/beam/issues/23048 CIBW_SKIP: "*-musllinux_*" CIBW_BEFORE_BUILD: pip install cython==0.29.36 numpy --config-settings=setup-args="-Dallow-noblas=true" && pip install --upgrade setuptools @@ -305,10 +301,9 @@ jobs: shell: bash - name: Upload RC wheels as artifacts if: ${{ needs.build_source.outputs.is_rc == 1 }} - # Pinned to v3 because of https://github.com/actions/download-artifact/issues/249 uses: actions/upload-artifact@v4 with: - name: wheelhouse-rc${{ needs.build_source.outputs.rc_num }}-${{ matrix.os_python.os }}${{ (matrix.arch == 'aarch64' && '-aarch64') || '' }} + name: wheelhouse-rc${{ needs.build_source.outputs.rc_num }}-${{ matrix.py_version }}${{ matrix.os_python.os }}${{ (matrix.os_python.arch == 'aarch64' && '-aarch64') || '' }} path: apache-beam-source-rc/wheelhouse/ upload_wheels_to_gcs: @@ -318,21 +313,12 @@ jobs: - check_env_variables runs-on: [self-hosted, ubuntu-20.04, main] if: needs.check_env_variables.outputs.gcp-variables-set == 'true' && github.event_name != 'pull_request' - strategy: - matrix: - # Temporarily pin to macos-13 because macos-latest breaks this build - # TODO(https://github.com/apache/beam/issues/31114) - os : [ubuntu-20.04, macos-13, windows-latest] - arch: [auto] - include: - - os: ubuntu-20.04 - arch: aarch64 steps: - name: Download wheels from artifacts - # Pinned to v3 because of https://github.com/actions/upload-artifact?tab=readme-ov-file#breaking-changes - uses: actions/download-artifact@v3 + uses: actions/download-artifact@v4 with: - name: wheelhouse-${{ matrix.os }}${{ (matrix.arch == 'aarch64' && '-aarch64') || '' }} + pattern: wheelhouse-* + merge-multiple: true path: wheelhouse/ - name: Copy wheels to GCS bucket run: gsutil cp -r -a public-read wheelhouse/* ${{ env.GCP_PATH }} From c92e39f05676fdf0feb800c3257acb5197544f90 Mon Sep 17 00:00:00 2001 From: liferoad Date: Fri, 11 Oct 2024 09:08:43 -0400 Subject: [PATCH 29/78] Fix more numpy 2 test issues --- sdks/python/tox.ini | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index 3de8dae046bd..8cc125d946a8 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -166,7 +166,7 @@ deps = Sphinx==7.4.7 sphinx_rtd_theme==3.0.1 docutils>=0.18.1 - Jinja2==3.1.0 + Jinja2==3.1.0 commands = time {toxinidir}/scripts/generate_pydoc.sh @@ -297,6 +297,7 @@ deps = # Since Pandas 2 requires pyarrow>=7, downgrade pandas for this test. 3: pyarrow>=3,<4 3: pandas<2 + 3: numpy>=1.14.3,<1.27.0 # Test against versions of pyarrow released in last ~2 years. 9: pyarrow>=9,<10 10: pyarrow>=10,<11 @@ -321,8 +322,10 @@ deps = 14: numpy>=1.14.3,<1.27.0 # Exclude 1.5.0 and 1.5.1 because of https://github.com/pandas-dev/pandas/issues/45725 15: pandas>=1.5.2,<1.6.0 + 15: numpy>=1.14.3,<1.27.0 20: pandas>=2.0.0,<2.1.0 20: pyarrow>=7 + 20: numpy>=1.14.3,<1.27.0 commands = # Log pandas and numpy version for debugging /bin/sh -c "pip freeze | grep -E '(pandas|numpy)'" From e7ec432db7bf4d7c0b8c77a1dc5f54acab903462 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Fri, 11 Oct 2024 10:16:11 -0400 Subject: [PATCH 30/78] Fix docker search (#32750) --- .github/workflows/finalize_release.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/finalize_release.yml b/.github/workflows/finalize_release.yml index 17ef17ed7841..126e1024908d 100644 --- a/.github/workflows/finalize_release.yml +++ b/.github/workflows/finalize_release.yml @@ -55,7 +55,7 @@ jobs: echo "Publish SDK docker images to Docker Hub." echo "================Pull RC Containers from DockerHub===========" - IMAGES=$(docker search apache/beam_ --format "{{.Name}}" --limit 100) + IMAGES=$(docker search apache/beam --format "{{.Name}}" --limit 100) KNOWN_IMAGES=() echo "We are using ${RC_VERSION} to push docker images for ${RELEASE}." while read IMAGE; do From b6e7fd92e5a633187c3062f32d3e6f44a2301ff1 Mon Sep 17 00:00:00 2001 From: liferoad Date: Fri, 11 Oct 2024 14:16:07 -0400 Subject: [PATCH 31/78] Do not fail the job when submission_environment_dependencies.txt cannot be loaded (#32752) --- sdks/python/container/boot.go | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/sdks/python/container/boot.go b/sdks/python/container/boot.go index 696604c64886..b7cbc07dca68 100644 --- a/sdks/python/container/boot.go +++ b/sdks/python/container/boot.go @@ -189,7 +189,12 @@ func launchSDKProcess() error { fmtErr := fmt.Errorf("failed to retrieve staged files: %v", err) // Send error message to logging service before returning up the call stack logger.Errorf(ctx, fmtErr.Error()) - return fmtErr + // No need to fail the job if submission_environment_dependencies.txt cannot be loaded + if strings.Contains(fmtErr.Error(), "submission_environment_dependencies.txt") { + logger.Printf(ctx, "Ignore the error when loading submission_environment_dependencies.txt.") + } else { + return fmtErr + } } // TODO(herohde): the packages to install should be specified explicitly. It From 7398a52fc9c2cfc9c27f065d76bfe782865c084f Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Fri, 11 Oct 2024 14:28:43 -0400 Subject: [PATCH 32/78] Add 2.51->52 breaking incompatibility to CHANGES.md (#32753) * Add 2.51->52 breaking incompatibility to CHANGES.md * Add blog --- CHANGES.md | 2 ++ website/www/site/content/en/blog/beam-2.52.0.md | 7 +++++++ 2 files changed, 9 insertions(+) diff --git a/CHANGES.md b/CHANGES.md index 39bad44dc52c..cc1268635046 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -485,6 +485,7 @@ classes finally moved to `extensions/avro`. In case if it's still required to us as a workaround, a copy of "old" `CountingSource` class should be placed into a project code and used directly ([#25252](https://github.com/apache/beam/issues/25252)). * Renamed `host` to `firestoreHost` in `FirestoreOptions` to avoid potential conflict of command line arguments (Java) ([#29201](https://github.com/apache/beam/pull/29201)). +* Transforms which use `SnappyCoder` are update incompatible with previous versions of the same transform (Java) on some runners. This includes PubSubIO's read ([#28655](https://github.com/apache/beam/pull/28655#issuecomment-2407839769)). ## Bugfixes @@ -502,6 +503,7 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a * MLTransform drops the identical elements in the output PCollection. For any duplicate elements, a single element will be emitted downstream. ([#29600](https://github.com/apache/beam/issues/29600)). * Some Python pipelines that run with 2.52.0-2.54.0 SDKs and use large materialized side inputs might be affected by a performance regression. To restore the prior behavior on these SDK versions, supply the `--max_cache_memory_usage_mb=0` pipeline option. (Python) ([#30360](https://github.com/apache/beam/issues/30360)). * Users who lauch Python pipelines in an environment without internet access and use the `--setup_file` pipeline option might experience an increase in pipeline submission time. This has been fixed in 2.56.0 ([#31070](https://github.com/apache/beam/pull/31070)). +* Transforms which use `SnappyCoder` are update incompatible with previous versions of the same transform (Java) on some runners. This includes PubSubIO's read ([#28655](https://github.com/apache/beam/pull/28655#issuecomment-2407839769)). # [2.51.0] - 2023-10-03 diff --git a/website/www/site/content/en/blog/beam-2.52.0.md b/website/www/site/content/en/blog/beam-2.52.0.md index 2e604c8fabf8..f468eb4811e3 100644 --- a/website/www/site/content/en/blog/beam-2.52.0.md +++ b/website/www/site/content/en/blog/beam-2.52.0.md @@ -52,6 +52,7 @@ classes finally moved to `extensions/avro`. In case if it's still required to us as a workaround, a copy of "old" `CountingSource` class should be placed into a project code and used directly ([#25252](https://github.com/apache/beam/issues/25252)). * Renamed `host` to `firestoreHost` in `FirestoreOptions` to avoid potential conflict of command line arguments (Java) ([#29201](https://github.com/apache/beam/pull/29201)). +* Transforms which use `SnappyCoder` are update incompatible with previous versions of the same transform (Java) on some runners. This includes PubSubIO's read ([#28655](https://github.com/apache/beam/pull/28655#issuecomment-2407839769)). ## Bugfixes @@ -64,6 +65,12 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a * Fixed [CVE-2023-39325](https://www.cve.org/CVERecord?id=CVE-2023-39325) (Java/Python/Go) ([#29118](https://github.com/apache/beam/issues/29118)). * Mitigated [CVE-2023-47248](https://nvd.nist.gov/vuln/detail/CVE-2023-47248) (Python) [#29392](https://github.com/apache/beam/issues/29392). +## Known issues +* MLTransform drops the identical elements in the output PCollection. For any duplicate elements, a single element will be emitted downstream. ([#29600](https://github.com/apache/beam/issues/29600)). +* Some Python pipelines that run with 2.52.0-2.54.0 SDKs and use large materialized side inputs might be affected by a performance regression. To restore the prior behavior on these SDK versions, supply the `--max_cache_memory_usage_mb=0` pipeline option. (Python) ([#30360](https://github.com/apache/beam/issues/30360)). +* Users who lauch Python pipelines in an environment without internet access and use the `--setup_file` pipeline option might experience an increase in pipeline submission time. This has been fixed in 2.56.0 ([#31070](https://github.com/apache/beam/pull/31070)). +* Transforms which use `SnappyCoder` are update incompatible with previous versions of the same transform (Java) on some runners. This includes PubSubIO's read ([#28655](https://github.com/apache/beam/pull/28655#issuecomment-2407839769)). + ## List of Contributors According to git shortlog, the following people contributed to the 2.52.0 release. Thank you to all contributors! From 578d6d7f5710a1c8b7358d9656bea423917f9cd3 Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Fri, 11 Oct 2024 15:46:08 -0400 Subject: [PATCH 33/78] fix: skip close on bundles (#32661) --- .../org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 389d2e43c74e..932099e01763 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -2021,10 +2021,9 @@ public void close() throws IOException { reader.close(); reader = null; } - if (serviceEntry != null) { - serviceEntry.close(); - serviceEntry = null; - } + // Skipping closing the service entry on each bundle. + // In the future we'll close the Bigtable client in + // teardown. } @Override From 13f4ae0c6e00f6775951cc72505ff681e4aab8d4 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Fri, 11 Oct 2024 15:48:15 -0400 Subject: [PATCH 34/78] disable expansion service cache (#32751) * Disable cache for io-expansion-service shadowJar temporarily * disable cache on sql expansion service --- sdks/java/expansion-service/build.gradle | 4 ++++ .../apache/beam/sdk/expansion/service/ExpansionService.java | 2 +- sdks/java/extensions/sql/expansion-service/build.gradle | 4 ++++ sdks/java/io/expansion-service/build.gradle | 1 + sdks/python/apache_beam/io/filesystems.py | 4 ++-- 5 files changed, 12 insertions(+), 3 deletions(-) diff --git a/sdks/java/expansion-service/build.gradle b/sdks/java/expansion-service/build.gradle index 4dd8c8968ed9..a25583870acf 100644 --- a/sdks/java/expansion-service/build.gradle +++ b/sdks/java/expansion-service/build.gradle @@ -57,3 +57,7 @@ task runExpansionService (type: JavaExec) { classpath = sourceSets.main.runtimeClasspath args = [project.findProperty("constructionService.port") ?: "8097"] } + +compileJava { + outputs.upToDateWhen { false } +} \ No newline at end of file diff --git a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java index 770da14fa1cf..150fe9729573 100644 --- a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java +++ b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java @@ -143,7 +143,7 @@ public static class ExternalTransformRegistrarLoader public Map knownTransforms() { Map providers = new HashMap<>(); - // First check and register ExternalTransformBuilder in serviceloader style, converting + // First check and register ExternalTransformBuilder in ServiceLoader style, converting // to TransformProvider after validation. Map registeredBuilders = loadTransformBuilders(); for (Map.Entry registeredBuilder : diff --git a/sdks/java/extensions/sql/expansion-service/build.gradle b/sdks/java/extensions/sql/expansion-service/build.gradle index b6963cf7547b..b8d78e4e1bb9 100644 --- a/sdks/java/extensions/sql/expansion-service/build.gradle +++ b/sdks/java/extensions/sql/expansion-service/build.gradle @@ -46,3 +46,7 @@ task runExpansionService (type: JavaExec) { classpath = sourceSets.main.runtimeClasspath args = [project.findProperty("constructionService.port") ?: "8097"] } + +shadowJar { + outputs.upToDateWhen { false } +} \ No newline at end of file diff --git a/sdks/java/io/expansion-service/build.gradle b/sdks/java/io/expansion-service/build.gradle index 26a001b6ea24..b09a92ca315c 100644 --- a/sdks/java/io/expansion-service/build.gradle +++ b/sdks/java/io/expansion-service/build.gradle @@ -35,6 +35,7 @@ configurations.runtimeClasspath { shadowJar { mergeServiceFiles() + outputs.upToDateWhen { false } } description = "Apache Beam :: SDKs :: Java :: IO :: Expansion Service" diff --git a/sdks/python/apache_beam/io/filesystems.py b/sdks/python/apache_beam/io/filesystems.py index a32b85332b60..87f45f3308ee 100644 --- a/sdks/python/apache_beam/io/filesystems.py +++ b/sdks/python/apache_beam/io/filesystems.py @@ -393,12 +393,12 @@ def get_chunk_size(path): @staticmethod def report_source_lineage(path, level=None): """ - Report source :class:`~apache_beam.metrics.metric.Lineage`. + Report source :class:`~apache_beam.metrics.metric.LineageLevel`. Args: path: string path to be reported. level: the level of file path. default to - :class:`~apache_beam.io.filesystem.FileSystem.Lineage`.FILE. + :class:`~apache_beam.io.filesystem.FileSystem.LineageLevel`.FILE. """ filesystem = FileSystems.get_filesystem(path) filesystem.report_lineage(path, Lineage.sources(), level=level) From 69bd697c7d1404eab57eb780eba603c44008d4e2 Mon Sep 17 00:00:00 2001 From: twosom <72733442+twosom@users.noreply.github.com> Date: Mon, 14 Oct 2024 11:58:24 +0900 Subject: [PATCH 35/78] Fix flaky mqtt test (#32765) * fix mqtt flaky test * extract common method for connect to Mqtt --- .../apache/beam/sdk/io/mqtt/MqttIOTest.java | 78 ++++++++++--------- 1 file changed, 41 insertions(+), 37 deletions(-) diff --git a/sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java b/sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java index 64b0728c879a..3ee6ed577a07 100644 --- a/sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java +++ b/sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java @@ -69,6 +69,18 @@ @RunWith(JUnit4.class) public class MqttIOTest { + /** Functional interface used to verify the connection status of an MQTT client. */ + @FunctionalInterface + interface ConnectionCondition { + /** + * Evaluates whether the given {@link Connection} satisfies the condition. + * + * @param connection the MQTT connection to check + * @return {@code true} if the condition is met, {@code false} otherwise + */ + boolean check(Connection connection); + } + private static final Logger LOG = LoggerFactory.getLogger(MqttIOTest.class); private BrokerService brokerService; @@ -123,18 +135,7 @@ public void testReadNoClientId() throws Exception { new Thread( () -> { try { - LOG.info( - "Waiting pipeline connected to the MQTT broker before sending " - + "messages ..."); - boolean pipelineConnected = false; - while (!pipelineConnected) { - Thread.sleep(1000); - for (Connection connection : brokerService.getBroker().getClients()) { - if (!connection.getConnectionId().isEmpty()) { - pipelineConnected = true; - } - } - } + doConnect(connection -> !connection.getConnectionId().isEmpty()); for (int i = 0; i < 10; i++) { publishConnection.publish( topicName, @@ -185,18 +186,7 @@ public void testRead() throws Exception { new Thread( () -> { try { - LOG.info( - "Waiting pipeline connected to the MQTT broker before sending " - + "messages ..."); - boolean pipelineConnected = false; - while (!pipelineConnected) { - for (Connection connection : brokerService.getBroker().getClients()) { - if (connection.getConnectionId().startsWith("READ_PIPELINE")) { - pipelineConnected = true; - } - } - Thread.sleep(1000); - } + doConnect(connection -> connection.getConnectionId().startsWith("READ_PIPELINE")); for (int i = 0; i < 10; i++) { publishConnection.publish( "READ_TOPIC", @@ -225,7 +215,8 @@ public void testReadWithMetadata() throws Exception { MqttIO.readWithMetadata() .withConnectionConfiguration( MqttIO.ConnectionConfiguration.create("tcp://localhost:" + port, wildcardTopic)) - .withMaxNumRecords(10); + .withMaxNumRecords(10) + .withMaxReadTime(Duration.standardSeconds(5)); final PCollection output = pipeline.apply(mqttReaderWithMetadata); PAssert.that(output) @@ -251,18 +242,7 @@ public void testReadWithMetadata() throws Exception { new Thread( () -> { try { - LOG.info( - "Waiting pipeline connected to the MQTT broker before sending " - + "messages ..."); - boolean pipelineConnected = false; - while (!pipelineConnected) { - Thread.sleep(1000); - for (Connection connection : brokerService.getBroker().getClients()) { - if (!connection.getConnectionId().isEmpty()) { - pipelineConnected = true; - } - } - } + doConnect(connection -> !connection.getConnectionId().isEmpty()); for (int i = 0; i < 5; i++) { publishConnection.publish( topic1, @@ -581,6 +561,30 @@ public void testReadObject() throws Exception { assertEquals(cp1.oldestMessageTimestamp, cp2.oldestMessageTimestamp); } + /** + * Attempts to establish a connection to the MQTT broker by checking each available client + * connection until the specified condition is met. + * + *

This method repeatedly checks the connection status of each MQTT client using the provided + * {@link ConnectionCondition}. It blocks execution within a loop, sleeping for 1 second between + * each check, until the condition is satisfied. + * + * @param condition the condition used to verify the connection status + * @throws Exception if any error occurs during the connection process + */ + private void doConnect(ConnectionCondition condition) throws Exception { + LOG.info("Waiting pipeline connected to the MQTT broker before sending messages ..."); + boolean pipelineConnected = false; + while (!pipelineConnected) { + for (Connection connection : brokerService.getBroker().getClients()) { + if (condition.check(connection)) { + pipelineConnected = true; + } + } + Thread.sleep(1000); + } + } + @After public void stopBroker() throws Exception { if (brokerService != null) { From b94f8a74b5a9c2a5216320342f038b03879dd289 Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Mon, 14 Oct 2024 11:03:51 -0400 Subject: [PATCH 36/78] Pin protobuf version for TF dependency tests (#32719) * Pin protobuf version for TF dependency tests * pin specfic protobuf version * increment tested TF version * pin at higher version of protobuf * fix incorrect configuration * try bumping tf version * further specify dependency versions * try adding other ml testing deps for compat * fix transformers tests * tweak deps for transformers * whitespace --- sdks/python/tox.ini | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index 8cc125d946a8..8cdc4a98bbfe 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -394,10 +394,12 @@ commands = [testenv:py39-tensorflow-212] deps = - 212: tensorflow>=2.12rc1,<2.13 - # Help pip resolve conflict with typing-extensions for old version of TF https://github.com/apache/beam/issues/30852 - 212: pydantic<2.7 -extras = test,gcp + 212: + tensorflow>=2.12rc1,<2.13 + # Help pip resolve conflict with typing-extensions for old version of TF https://github.com/apache/beam/issues/30852 + pydantic<2.7 + protobuf==4.25.5 +extras = test,gcp,ml_test commands = # Log tensorflow version for debugging /bin/sh -c "pip freeze | grep -E tensorflow" @@ -428,7 +430,8 @@ deps = 430: transformers>=4.30.0,<4.31.0 torch>=1.9.0,<1.14.0 tensorflow==2.12.0 -extras = test,gcp + protobuf==4.25.5 +extras = test,gcp,ml_test commands = # Log transformers and its dependencies version for debugging /bin/sh -c "pip freeze | grep -E transformers" From 1bc848b4b1ebfe4f3576b219d2d667f9a5cf8cbe Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Mon, 14 Oct 2024 11:35:18 -0400 Subject: [PATCH 37/78] Usse ubuntu-22.04 for release candidate build for now (#32767) --- .github/workflows/build_release_candidate.yml | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/.github/workflows/build_release_candidate.yml b/.github/workflows/build_release_candidate.yml index fbb0ca22f333..a70091726c1b 100644 --- a/.github/workflows/build_release_candidate.yml +++ b/.github/workflows/build_release_candidate.yml @@ -97,7 +97,7 @@ jobs: stage_java_source: if: ${{ fromJson(github.event.inputs.STAGE).java_source == 'yes'}} - runs-on: ubuntu-latest + runs-on: ubuntu-22.04 steps: - name: Mask Apache Password run: | @@ -161,7 +161,7 @@ jobs: stage_python_artifacts: if: ${{ fromJson(github.event.inputs.STAGE).python_artifacts == 'yes'}} - runs-on: ubuntu-latest + runs-on: ubuntu-22.04 steps: - name: Checkout uses: actions/checkout@v4 @@ -246,7 +246,7 @@ jobs: stage_docker: if: ${{ fromJson(github.event.inputs.STAGE).docker_artifacts == 'yes'}} # Note: if this ever changes to self-hosted, remove the "Remove default github maven configuration" step - runs-on: ubuntu-latest + runs-on: ubuntu-22.04 steps: - name: Checkout uses: actions/checkout@v4 @@ -282,7 +282,7 @@ jobs: beam_site_pr: if: ${{ fromJson(github.event.inputs.STAGE).beam_site_pr == 'yes'}} # Note: if this ever changes to self-hosted, remove the "Remove default github maven configuration" step - runs-on: ubuntu-latest + runs-on: ubuntu-22.04 env: RC_TAG: "v${{ github.event.inputs.RELEASE }}-RC${{ github.event.inputs.RC }}" BRANCH_NAME: updates_release_${{ github.event.inputs.RELEASE }} @@ -402,7 +402,7 @@ jobs: build_and_stage_prism: if: ${{ fromJson(github.event.inputs.STAGE).prism == 'yes'}} - runs-on: ubuntu-latest + runs-on: ubuntu-22.04 steps: - name: Checkout uses: actions/checkout@v4 From 1f575d4d816032e136b44b15ea940a19c67a9466 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Mon, 14 Oct 2024 11:38:19 -0400 Subject: [PATCH 38/78] Avoid repeated run of setDefaultPipelineOptionsOnce in TestPipelineOptions.create (#32723) --- .../org/apache/beam/sdk/io/FileSystems.java | 23 +++++++++++++++++-- .../apache/beam/sdk/testing/TestPipeline.java | 4 ++-- 2 files changed, 23 insertions(+), 4 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java index fb25cac6262f..5ca22749b163 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java @@ -565,11 +565,13 @@ static FileSystem getFileSystemInternal(String scheme) { * *

It will be used in {@link FileSystemRegistrar FileSystemRegistrars} for all schemes. * - *

This is expected only to be used by runners after {@code Pipeline.run}, or in tests. + *

Outside of workers where Beam FileSystem API is used (e.g. test methods, user code executed + * during pipeline submission), consider use {@link #registerFileSystemsOnce} if initialize + * FIleSystem of supported schema is the main goal. */ @Internal public static void setDefaultPipelineOptions(PipelineOptions options) { - checkNotNull(options, "options"); + checkNotNull(options, "options cannot be null"); long id = options.getOptionsId(); int nextRevision = options.revision(); @@ -593,6 +595,23 @@ public static void setDefaultPipelineOptions(PipelineOptions options) { } } + /** + * Register file systems once if never done before. + * + *

This method executes {@link #setDefaultPipelineOptions} only if it has never been run, + * otherwise it returns immediately. + * + *

It is internally used by test setup to avoid repeated filesystem registrations (involves + * expensive ServiceLoader calls) when there are multiple pipeline and PipelineOptions object + * initialized, which is commonly seen in test execution. + */ + @Internal + public static synchronized void registerFileSystemsOnce(PipelineOptions options) { + if (FILESYSTEM_REVISION.get() == null) { + setDefaultPipelineOptions(options); + } + } + @VisibleForTesting static Map verifySchemesAreUnique( PipelineOptions options, Set registrars) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java index ed61f7f3d6f2..328bf19c466c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java @@ -373,7 +373,7 @@ public PipelineResult runWithAdditionalOptionArgs(List additionalArgs) { } newOptions.setStableUniqueNames(CheckEnabled.ERROR); - FileSystems.setDefaultPipelineOptions(options); + FileSystems.registerFileSystemsOnce(options); return run(newOptions); } catch (IOException e) { throw new RuntimeException( @@ -515,7 +515,7 @@ public static PipelineOptions testingPipelineOptions() { } options.setStableUniqueNames(CheckEnabled.ERROR); - FileSystems.setDefaultPipelineOptions(options); + FileSystems.registerFileSystemsOnce(options); return options; } catch (IOException e) { throw new RuntimeException( From c04e91d89d5ad84b20043e3f3fbfe8d5edfac6e9 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Mon, 14 Oct 2024 15:43:19 -0400 Subject: [PATCH 39/78] Fix download artifact truncate page (#32772) --- .../src/main/scripts/download_github_actions_artifacts.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/release/src/main/scripts/download_github_actions_artifacts.py b/release/src/main/scripts/download_github_actions_artifacts.py index 99526f1ac7d1..5c553efc6500 100644 --- a/release/src/main/scripts/download_github_actions_artifacts.py +++ b/release/src/main/scripts/download_github_actions_artifacts.py @@ -279,8 +279,11 @@ def fetch_github_artifacts(run_id, repo_url, artifacts_dir, github_token, rc_num print("Starting downloading artifacts ... (it may take a while)") run_data = get_single_workflow_run_data(run_id, repo_url, github_token) artifacts_url = safe_get(run_data, "artifacts_url") - data_artifacts = request_url(artifacts_url, github_token) + data_artifacts = request_url(artifacts_url + '?per_page=100', github_token) artifacts = safe_get(data_artifacts, "artifacts", artifacts_url) + total_count = safe_get(data_artifacts, "total_count", artifacts_url) + if int(total_count) != len(artifacts): + raise RuntimeError(f"Expected total count {total_count} different than returned list length {len(data_artifacts)}") print('Filtering ', len(artifacts), ' artifacts') filtered_artifacts = filter_artifacts(artifacts, rc_number) print('Preparing to download ', len(filtered_artifacts), ' artifacts') From f3708e06e130dc40baafc96f5e8786dc697ddb3c Mon Sep 17 00:00:00 2001 From: liferoad Date: Mon, 14 Oct 2024 17:06:03 -0400 Subject: [PATCH 40/78] Update pyproject.toml --- sdks/python/pyproject.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/pyproject.toml b/sdks/python/pyproject.toml index 4eb827297019..d34afcf70a5e 100644 --- a/sdks/python/pyproject.toml +++ b/sdks/python/pyproject.toml @@ -21,7 +21,7 @@ requires = [ "setuptools", "wheel>=0.36.0", - "grpcio-tools==1.62.1", + "grpcio-tools>=1.62.1", "mypy-protobuf==3.5.0", # Avoid https://github.com/pypa/virtualenv/issues/2006 "distlib==0.3.7", From e47c261502d722140b6bcb0e275bed4cb7b5624f Mon Sep 17 00:00:00 2001 From: liferoad Date: Mon, 14 Oct 2024 19:23:26 -0400 Subject: [PATCH 41/78] use 1.65.5 --- sdks/python/pyproject.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/pyproject.toml b/sdks/python/pyproject.toml index d34afcf70a5e..a99599a2ce2b 100644 --- a/sdks/python/pyproject.toml +++ b/sdks/python/pyproject.toml @@ -21,7 +21,7 @@ requires = [ "setuptools", "wheel>=0.36.0", - "grpcio-tools>=1.62.1", + "grpcio-tools==1.65.5", "mypy-protobuf==3.5.0", # Avoid https://github.com/pypa/virtualenv/issues/2006 "distlib==0.3.7", From 9a2d456f72b07ac449cd24c6195179a3b1b8be65 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> Date: Tue, 15 Oct 2024 11:10:31 +0300 Subject: [PATCH 42/78] [Python] Managed Transforms API (#31495) * managed module * clean up * lint * try with real example * cleanup * add documentation * fix doc * add pyyaml dependency * cleanup * return deps * return deps * fix doc * address some comments * doc updates * define managed transform URNs in proto * fix URN * remove managed dependency * add managed iceberg integration test * lint * lint * dependency fix * lint * dependency fix * dependency fix * lint * lint * dependency fix * rename test file --- ...m_PostCommit_Python_Xlang_IO_Dataflow.json | 2 +- ...eam_PostCommit_Python_Xlang_IO_Direct.json | 4 + ...beam_PostCommit_Python_Xlang_IO_Direct.yml | 96 +++++++++ CHANGES.md | 2 + .../pipeline/v1/external_transforms.proto | 14 ++ sdks/java/io/expansion-service/build.gradle | 2 + sdks/java/io/iceberg/build.gradle | 3 +- .../apache/beam/sdk/io/iceberg/IcebergIO.java | 5 +- .../IcebergReadSchemaTransformProvider.java | 6 +- .../IcebergWriteSchemaTransformProvider.java | 5 +- sdks/java/io/kafka/build.gradle | 1 + .../KafkaReadSchemaTransformProvider.java | 4 +- .../KafkaWriteSchemaTransformProvider.java | 5 +- sdks/java/managed/build.gradle | 1 + .../org/apache/beam/sdk/managed/Managed.java | 11 +- .../managed/ManagedTransformConstants.java | 13 +- .../apache_beam/portability/common_urns.py | 1 + .../python/apache_beam/transforms/__init__.py | 1 + sdks/python/apache_beam/transforms/managed.py | 182 ++++++++++++++++++ .../transforms/managed_iceberg_it_test.py | 70 +++++++ sdks/python/setup.py | 3 +- sdks/standard_expansion_services.yaml | 2 +- 22 files changed, 407 insertions(+), 26 deletions(-) create mode 100644 .github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json create mode 100644 .github/workflows/beam_PostCommit_Python_Xlang_IO_Direct.yml create mode 100644 sdks/python/apache_beam/transforms/managed.py create mode 100644 sdks/python/apache_beam/transforms/managed_iceberg_it_test.py diff --git a/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Dataflow.json b/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Dataflow.json index e3d6056a5de9..b26833333238 100644 --- a/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Dataflow.json +++ b/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Dataflow.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 1 + "modification": 2 } diff --git a/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json b/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json new file mode 100644 index 000000000000..e3d6056a5de9 --- /dev/null +++ b/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json @@ -0,0 +1,4 @@ +{ + "comment": "Modify this file in a trivial way to cause this test suite to run", + "modification": 1 +} diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_IO_Direct.yml b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Direct.yml new file mode 100644 index 000000000000..5092a1981154 --- /dev/null +++ b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Direct.yml @@ -0,0 +1,96 @@ +# 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. + +name: PostCommit Python Xlang IO Direct + +on: + schedule: + - cron: '30 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json'] + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: write + checks: write + contents: read + deployments: read + id-token: none + issues: write + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' + cancel-in-progress: true + +env: + DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_PostCommit_Python_Xlang_IO_Direct: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || + github.event.comment.body == 'Run Python_Xlang_IO_Direct PostCommit' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PostCommit_Python_Xlang_IO_Direct"] + job_phrase: ["Run Python_Xlang_IO_Direct PostCommit"] + steps: + - uses: actions/checkout@v4 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: | + 3.9 + 3.12 + - name: run PostCommit Python Xlang IO Direct script + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:test-suites:direct:ioCrossLanguagePostCommit + arguments: -PuseWheelDistribution + - name: Archive Python Test Results + uses: actions/upload-artifact@v4 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 + if: always() + with: + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' \ No newline at end of file diff --git a/CHANGES.md b/CHANGES.md index cc1268635046..4e21e400e60d 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -59,11 +59,13 @@ * 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] Introduce Managed Transforms API ([#31495](https://github.com/apache/beam/pull/31495)) ## I/Os * Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). * [Managed Iceberg] Support creating tables if needed ([#32686](https://github.com/apache/beam/pull/32686)) +* [Managed Iceberg] Now available in Python SDK ([#31495](https://github.com/apache/beam/pull/31495)) * [Managed Iceberg] Add support for TIMESTAMP, TIME, and DATE types ([#32688](https://github.com/apache/beam/pull/32688)) ## New Features / Improvements diff --git a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/external_transforms.proto b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/external_transforms.proto index 429371e11055..b03350966d6c 100644 --- a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/external_transforms.proto +++ b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/external_transforms.proto @@ -59,6 +59,20 @@ message ExpansionMethods { } } +// Defines the URNs for managed transforms. +message ManagedTransforms { + enum Urns { + ICEBERG_READ = 0 [(org.apache.beam.model.pipeline.v1.beam_urn) = + "beam:schematransform:org.apache.beam:iceberg_read:v1"]; + ICEBERG_WRITE = 1 [(org.apache.beam.model.pipeline.v1.beam_urn) = + "beam:schematransform:org.apache.beam:iceberg_write:v1"]; + KAFKA_READ = 2 [(org.apache.beam.model.pipeline.v1.beam_urn) = + "beam:schematransform:org.apache.beam:kafka_read:v1"]; + KAFKA_WRITE = 3 [(org.apache.beam.model.pipeline.v1.beam_urn) = + "beam:schematransform:org.apache.beam:kafka_write:v1"]; + } +} + // A configuration payload for an external transform. // Used to define a Java transform that can be directly instantiated by a Java // expansion service. diff --git a/sdks/java/io/expansion-service/build.gradle b/sdks/java/io/expansion-service/build.gradle index b09a92ca315c..cc8eccf98997 100644 --- a/sdks/java/io/expansion-service/build.gradle +++ b/sdks/java/io/expansion-service/build.gradle @@ -44,6 +44,8 @@ ext.summary = "Expansion service serving several Java IOs" dependencies { implementation project(":sdks:java:expansion-service") permitUnusedDeclared project(":sdks:java:expansion-service") // BEAM-11761 + implementation project(":sdks:java:managed") + permitUnusedDeclared project(":sdks:java:managed") // BEAM-11761 implementation project(":sdks:java:io:iceberg") permitUnusedDeclared project(":sdks:java:io:iceberg") // BEAM-11761 implementation project(":sdks:java:io:kafka") diff --git a/sdks/java/io/iceberg/build.gradle b/sdks/java/io/iceberg/build.gradle index 3d653d6b276e..e10c6f38e20f 100644 --- a/sdks/java/io/iceberg/build.gradle +++ b/sdks/java/io/iceberg/build.gradle @@ -44,7 +44,7 @@ def orc_version = "1.9.2" dependencies { implementation library.java.vendored_guava_32_1_2_jre implementation project(path: ":sdks:java:core", configuration: "shadow") - implementation project(":sdks:java:managed") + implementation project(path: ":model:pipeline", configuration: "shadow") implementation library.java.slf4j_api implementation library.java.joda_time implementation "org.apache.parquet:parquet-column:$parquet_version" @@ -55,6 +55,7 @@ dependencies { implementation "org.apache.iceberg:iceberg-orc:$iceberg_version" implementation library.java.hadoop_common + testImplementation project(":sdks:java:managed") testImplementation library.java.hadoop_client testImplementation library.java.bigdataoss_gcsio testImplementation library.java.bigdataoss_gcs_connector diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java index fa4ff9714c7f..1d4b36585237 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java @@ -24,7 +24,6 @@ import java.util.List; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.io.Read; -import org.apache.beam.sdk.managed.Managed; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PBegin; @@ -45,8 +44,8 @@ * A connector that reads and writes to Apache Iceberg * tables. * - *

{@link IcebergIO} is offered as a {@link Managed} transform. This class is subject to change - * and should not be used directly. Instead, use it via {@link Managed#ICEBERG} like so: + *

{@link IcebergIO} is offered as a Managed transform. This class is subject to change and + * should not be used directly. Instead, use it like so: * *

{@code
  * Map config = Map.of(
diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java
index df7bda4560dd..d44149fda08e 100644
--- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java
+++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java
@@ -17,10 +17,12 @@
  */
 package org.apache.beam.sdk.io.iceberg;
 
+import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn;
+
 import com.google.auto.service.AutoService;
 import java.util.Collections;
 import java.util.List;
-import org.apache.beam.sdk.managed.ManagedTransformConstants;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms;
 import org.apache.beam.sdk.schemas.NoSuchSchemaException;
 import org.apache.beam.sdk.schemas.SchemaRegistry;
 import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
@@ -53,7 +55,7 @@ public List outputCollectionNames() {
 
   @Override
   public String identifier() {
-    return ManagedTransformConstants.ICEBERG_READ;
+    return getUrn(ExternalTransforms.ManagedTransforms.Urns.ICEBERG_READ);
   }
 
   static class IcebergReadSchemaTransform extends SchemaTransform {
diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java
index ea46e8560815..6aa830e7fbc6 100644
--- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java
+++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java
@@ -18,13 +18,14 @@
 package org.apache.beam.sdk.io.iceberg;
 
 import static org.apache.beam.sdk.io.iceberg.IcebergWriteSchemaTransformProvider.Configuration;
+import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn;
 
 import com.google.auto.service.AutoService;
 import com.google.auto.value.AutoValue;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-import org.apache.beam.sdk.managed.ManagedTransformConstants;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms;
 import org.apache.beam.sdk.schemas.AutoValueSchema;
 import org.apache.beam.sdk.schemas.NoSuchSchemaException;
 import org.apache.beam.sdk.schemas.Schema;
@@ -151,7 +152,7 @@ public List outputCollectionNames() {
 
   @Override
   public String identifier() {
-    return ManagedTransformConstants.ICEBERG_WRITE;
+    return getUrn(ExternalTransforms.ManagedTransforms.Urns.ICEBERG_WRITE);
   }
 
   static class IcebergWriteSchemaTransform extends SchemaTransform {
diff --git a/sdks/java/io/kafka/build.gradle b/sdks/java/io/kafka/build.gradle
index 0ba6fa642a02..ec4654bd88df 100644
--- a/sdks/java/io/kafka/build.gradle
+++ b/sdks/java/io/kafka/build.gradle
@@ -54,6 +54,7 @@ dependencies {
   provided library.java.jackson_dataformat_csv
   permitUnusedDeclared library.java.jackson_dataformat_csv
   implementation project(path: ":sdks:java:core", configuration: "shadow")
+  implementation project(path: ":model:pipeline", configuration: "shadow")
   implementation project(":sdks:java:extensions:avro")
   implementation project(":sdks:java:extensions:protobuf")
   implementation project(":sdks:java:expansion-service")
diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProvider.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProvider.java
index e87669ab2b0a..a3fd1d8c3fd7 100644
--- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProvider.java
+++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProvider.java
@@ -18,6 +18,7 @@
 package org.apache.beam.sdk.io.kafka;
 
 import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull;
+import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn;
 
 import com.google.auto.service.AutoService;
 import java.io.FileOutputStream;
@@ -34,6 +35,7 @@
 import java.util.Map;
 import java.util.stream.Collectors;
 import org.apache.avro.generic.GenericRecord;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms;
 import org.apache.beam.sdk.extensions.avro.coders.AvroCoder;
 import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils;
 import org.apache.beam.sdk.extensions.protobuf.ProtoByteUtils;
@@ -103,7 +105,7 @@ public Row apply(byte[] input) {
 
   @Override
   public String identifier() {
-    return "beam:schematransform:org.apache.beam:kafka_read:v1";
+    return getUrn(ExternalTransforms.ManagedTransforms.Urns.KAFKA_READ);
   }
 
   @Override
diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java
index 09b338492b47..d6f46b11cb7d 100644
--- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java
+++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.io.kafka;
 
+import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn;
+
 import com.google.auto.service.AutoService;
 import com.google.auto.value.AutoValue;
 import java.io.Serializable;
@@ -26,6 +28,7 @@
 import java.util.Map;
 import java.util.Set;
 import javax.annotation.Nullable;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms;
 import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils;
 import org.apache.beam.sdk.extensions.protobuf.ProtoByteUtils;
 import org.apache.beam.sdk.metrics.Counter;
@@ -249,7 +252,7 @@ public byte[] apply(Row input) {
 
   @Override
   public @UnknownKeyFor @NonNull @Initialized String identifier() {
-    return "beam:schematransform:org.apache.beam:kafka_write:v1";
+    return getUrn(ExternalTransforms.ManagedTransforms.Urns.KAFKA_WRITE);
   }
 
   @Override
diff --git a/sdks/java/managed/build.gradle b/sdks/java/managed/build.gradle
index add0d7f3cc0d..c6e868872246 100644
--- a/sdks/java/managed/build.gradle
+++ b/sdks/java/managed/build.gradle
@@ -28,6 +28,7 @@ ext.summary = """Library that provides managed IOs."""
 
 dependencies {
     implementation project(path: ":sdks:java:core", configuration: "shadow")
+    implementation project(path: ":model:pipeline", configuration: "shadow")
     implementation library.java.vendored_guava_32_1_2_jre
     implementation library.java.slf4j_api
 
diff --git a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/Managed.java b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/Managed.java
index 911e25cdda14..8477726686ee 100644
--- a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/Managed.java
+++ b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/Managed.java
@@ -17,11 +17,14 @@
  */
 package org.apache.beam.sdk.managed;
 
+import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn;
+
 import com.google.auto.value.AutoValue;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import javax.annotation.Nullable;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms;
 import org.apache.beam.sdk.coders.RowCoder;
 import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
 import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider;
@@ -87,13 +90,13 @@ public class Managed {
   // Supported SchemaTransforms
   public static final Map READ_TRANSFORMS =
       ImmutableMap.builder()
-          .put(ICEBERG, ManagedTransformConstants.ICEBERG_READ)
-          .put(KAFKA, ManagedTransformConstants.KAFKA_READ)
+          .put(ICEBERG, getUrn(ExternalTransforms.ManagedTransforms.Urns.ICEBERG_READ))
+          .put(KAFKA, getUrn(ExternalTransforms.ManagedTransforms.Urns.KAFKA_READ))
           .build();
   public static final Map WRITE_TRANSFORMS =
       ImmutableMap.builder()
-          .put(ICEBERG, ManagedTransformConstants.ICEBERG_WRITE)
-          .put(KAFKA, ManagedTransformConstants.KAFKA_WRITE)
+          .put(ICEBERG, getUrn(ExternalTransforms.ManagedTransforms.Urns.ICEBERG_WRITE))
+          .put(KAFKA, getUrn(ExternalTransforms.ManagedTransforms.Urns.KAFKA_WRITE))
           .build();
 
   /**
diff --git a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java
index 51d0b67b4b89..4cf752747be5 100644
--- a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java
+++ b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java
@@ -17,7 +17,10 @@
  */
 package org.apache.beam.sdk.managed;
 
+import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn;
+
 import java.util.Map;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms;
 import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
 
 /**
@@ -41,12 +44,6 @@ public class ManagedTransformConstants {
   // Standard input PCollection tag
   public static final String INPUT = "input";
 
-  public static final String ICEBERG_READ = "beam:schematransform:org.apache.beam:iceberg_read:v1";
-  public static final String ICEBERG_WRITE =
-      "beam:schematransform:org.apache.beam:iceberg_write:v1";
-  public static final String KAFKA_READ = "beam:schematransform:org.apache.beam:kafka_read:v1";
-  public static final String KAFKA_WRITE = "beam:schematransform:org.apache.beam:kafka_write:v1";
-
   private static final Map KAFKA_READ_MAPPINGS =
       ImmutableMap.builder().put("data_format", "format").build();
 
@@ -55,7 +52,7 @@ public class ManagedTransformConstants {
 
   public static final Map> MAPPINGS =
       ImmutableMap.>builder()
-          .put(KAFKA_READ, KAFKA_READ_MAPPINGS)
-          .put(KAFKA_WRITE, KAFKA_WRITE_MAPPINGS)
+          .put(getUrn(ExternalTransforms.ManagedTransforms.Urns.KAFKA_READ), KAFKA_READ_MAPPINGS)
+          .put(getUrn(ExternalTransforms.ManagedTransforms.Urns.KAFKA_WRITE), KAFKA_WRITE_MAPPINGS)
           .build();
 }
diff --git a/sdks/python/apache_beam/portability/common_urns.py b/sdks/python/apache_beam/portability/common_urns.py
index 4effc91c3d40..74d9a39bb052 100644
--- a/sdks/python/apache_beam/portability/common_urns.py
+++ b/sdks/python/apache_beam/portability/common_urns.py
@@ -38,6 +38,7 @@
 StandardSideInputTypes = beam_runner_api_pb2_urns.StandardSideInputTypes
 StandardUserStateTypes = beam_runner_api_pb2_urns.StandardUserStateTypes
 ExpansionMethods = external_transforms_pb2_urns.ExpansionMethods
+ManagedTransforms = external_transforms_pb2_urns.ManagedTransforms
 MonitoringInfo = metrics_pb2_urns.MonitoringInfo
 MonitoringInfoSpecs = metrics_pb2_urns.MonitoringInfoSpecs
 MonitoringInfoTypeUrns = metrics_pb2_urns.MonitoringInfoTypeUrns
diff --git a/sdks/python/apache_beam/transforms/__init__.py b/sdks/python/apache_beam/transforms/__init__.py
index 4e66a290842c..b8b6839019e8 100644
--- a/sdks/python/apache_beam/transforms/__init__.py
+++ b/sdks/python/apache_beam/transforms/__init__.py
@@ -22,6 +22,7 @@
 from apache_beam.transforms import combiners
 from apache_beam.transforms.core import *
 from apache_beam.transforms.external import *
+from apache_beam.transforms.managed import *
 from apache_beam.transforms.ptransform import *
 from apache_beam.transforms.stats import *
 from apache_beam.transforms.timeutil import TimeDomain
diff --git a/sdks/python/apache_beam/transforms/managed.py b/sdks/python/apache_beam/transforms/managed.py
new file mode 100644
index 000000000000..22ee15b1de1c
--- /dev/null
+++ b/sdks/python/apache_beam/transforms/managed.py
@@ -0,0 +1,182 @@
+#
+# 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.
+#
+
+"""Managed Transforms.
+
+This module builds and instantiates turnkey transforms that can be managed by
+the underlying runner. This means the runner can upgrade the transform to a
+more optimal/updated version without requiring the user to do anything. It may
+also replace the transform with something entirely different if it chooses to.
+By default, however, the specified transform will remain unchanged.
+
+Using Managed Transforms
+========================
+Managed turnkey transforms have a defined configuration and can be built using
+an inline :class:`dict` like so::
+
+  results = p | beam.managed.Read(
+                    beam.managed.ICEBERG,
+                    config={"table": "foo",
+                            "catalog_name": "bar",
+                            "catalog_properties": {
+                                "warehouse": "path/to/warehouse",
+                                "catalog-impl": "org.apache.my.CatalogImpl"}})
+
+A YAML configuration file can also be used to build a Managed transform. Say we
+have the following `config.yaml` file::
+
+  topic: "foo"
+  bootstrap_servers: "localhost:1234"
+  format: "AVRO"
+
+Simply provide the location to the file like so::
+
+  input_rows = p | beam.Create(...)
+  input_rows | beam.managed.Write(
+                    beam.managed.KAFKA,
+                    config_url="path/to/config.yaml")
+
+Available transforms
+====================
+Available transforms are:
+
+- **Kafka Read and Write**
+- **Iceberg Read and Write**
+
+**Note:** inputs and outputs need to be PCollection(s) of Beam
+:py:class:`apache_beam.pvalue.Row` elements.
+
+**Note:** Today, all managed transforms are essentially cross-language
+transforms, and Java's ManagedSchemaTransform is used under the hood.
+"""
+
+from typing import Any
+from typing import Dict
+from typing import Optional
+
+import yaml
+
+from apache_beam.portability.common_urns import ManagedTransforms
+from apache_beam.transforms.external import BeamJarExpansionService
+from apache_beam.transforms.external import SchemaAwareExternalTransform
+from apache_beam.transforms.ptransform import PTransform
+
+ICEBERG = "iceberg"
+KAFKA = "kafka"
+_MANAGED_IDENTIFIER = "beam:transform:managed:v1"
+_EXPANSION_SERVICE_JAR_TARGETS = {
+    "sdks:java:io:expansion-service:shadowJar": [KAFKA, ICEBERG],
+}
+
+__all__ = ["ICEBERG", "KAFKA", "Read", "Write"]
+
+
+class Read(PTransform):
+  """Read using Managed Transforms"""
+  _READ_TRANSFORMS = {
+      ICEBERG: ManagedTransforms.Urns.ICEBERG_READ.urn,
+      KAFKA: ManagedTransforms.Urns.KAFKA_READ.urn,
+  }
+
+  def __init__(
+      self,
+      source: str,
+      config: Optional[Dict[str, Any]] = None,
+      config_url: Optional[str] = None,
+      expansion_service=None):
+    super().__init__()
+    self._source = source
+    identifier = self._READ_TRANSFORMS.get(source.lower())
+    if not identifier:
+      raise ValueError(
+          f"An unsupported source was specified: '{source}'. Please specify "
+          f"one of the following sources: {list(self._READ_TRANSFORMS.keys())}")
+
+    self._expansion_service = _resolve_expansion_service(
+        source, identifier, expansion_service)
+    self._underlying_identifier = identifier
+    self._yaml_config = yaml.dump(config)
+    self._config_url = config_url
+
+  def expand(self, input):
+    return input | SchemaAwareExternalTransform(
+        identifier=_MANAGED_IDENTIFIER,
+        expansion_service=self._expansion_service,
+        rearrange_based_on_discovery=True,
+        transform_identifier=self._underlying_identifier,
+        config=self._yaml_config,
+        config_url=self._config_url)
+
+  def default_label(self) -> str:
+    return "Managed Read(%s)" % self._source.upper()
+
+
+class Write(PTransform):
+  """Write using Managed Transforms"""
+  _WRITE_TRANSFORMS = {
+      ICEBERG: ManagedTransforms.Urns.ICEBERG_WRITE.urn,
+      KAFKA: ManagedTransforms.Urns.KAFKA_WRITE.urn,
+  }
+
+  def __init__(
+      self,
+      sink: str,
+      config: Optional[Dict[str, Any]] = None,
+      config_url: Optional[str] = None,
+      expansion_service=None):
+    super().__init__()
+    self._sink = sink
+    identifier = self._WRITE_TRANSFORMS.get(sink.lower())
+    if not identifier:
+      raise ValueError(
+          f"An unsupported sink was specified: '{sink}'. Please specify "
+          f"one of the following sinks: {list(self._WRITE_TRANSFORMS.keys())}")
+
+    self._expansion_service = _resolve_expansion_service(
+        sink, identifier, expansion_service)
+    self._underlying_identifier = identifier
+    self._yaml_config = yaml.dump(config)
+    self._config_url = config_url
+
+  def expand(self, input):
+    return input | SchemaAwareExternalTransform(
+        identifier=_MANAGED_IDENTIFIER,
+        expansion_service=self._expansion_service,
+        rearrange_based_on_discovery=True,
+        transform_identifier=self._underlying_identifier,
+        config=self._yaml_config,
+        config_url=self._config_url)
+
+  def default_label(self) -> str:
+    return "Managed Write(%s)" % self._sink.upper()
+
+
+def _resolve_expansion_service(
+    transform_name: str, identifier: str, expansion_service):
+  if expansion_service:
+    return expansion_service
+
+  default_target = None
+  for gradle_target, transforms in _EXPANSION_SERVICE_JAR_TARGETS.items():
+    if transform_name.lower() in transforms:
+      default_target = gradle_target
+      break
+  if not default_target:
+    raise ValueError(
+        "No expansion service was specified and could not find a "
+        f"default expansion service for {transform_name}: '{identifier}'.")
+  return BeamJarExpansionService(default_target)
diff --git a/sdks/python/apache_beam/transforms/managed_iceberg_it_test.py b/sdks/python/apache_beam/transforms/managed_iceberg_it_test.py
new file mode 100644
index 000000000000..2d7262bac031
--- /dev/null
+++ b/sdks/python/apache_beam/transforms/managed_iceberg_it_test.py
@@ -0,0 +1,70 @@
+import os
+import secrets
+import shutil
+import tempfile
+import time
+import unittest
+
+import pytest
+
+import apache_beam as beam
+from apache_beam.testing.util import assert_that
+from apache_beam.testing.util import equal_to
+
+
+@pytest.mark.uses_io_java_expansion_service
+@unittest.skipUnless(
+    os.environ.get('EXPANSION_JARS'),
+    "EXPANSION_JARS environment var is not provided, "
+    "indicating that jars have not been built")
+class ManagedIcebergIT(unittest.TestCase):
+  def setUp(self):
+    self._tempdir = tempfile.mkdtemp()
+    if not os.path.exists(self._tempdir):
+      os.mkdir(self._tempdir)
+    test_warehouse_name = 'test_warehouse_%d_%s' % (
+        int(time.time()), secrets.token_hex(3))
+    self.warehouse_path = os.path.join(self._tempdir, test_warehouse_name)
+    os.mkdir(self.warehouse_path)
+
+  def tearDown(self):
+    shutil.rmtree(self._tempdir, ignore_errors=False)
+
+  def _create_row(self, num: int):
+    return beam.Row(
+        int_=num,
+        str_=str(num),
+        bytes_=bytes(num),
+        bool_=(num % 2 == 0),
+        float_=(num + float(num) / 100))
+
+  def test_write_read_pipeline(self):
+    iceberg_config = {
+        "table": "test.write_read",
+        "catalog_name": "default",
+        "catalog_properties": {
+            "type": "hadoop",
+            "warehouse": f"file://{self.warehouse_path}",
+        }
+    }
+
+    rows = [self._create_row(i) for i in range(100)]
+    expected_dicts = [row.as_dict() for row in rows]
+
+    with beam.Pipeline() as write_pipeline:
+      _ = (
+          write_pipeline
+          | beam.Create(rows)
+          | beam.managed.Write(beam.managed.ICEBERG, config=iceberg_config))
+
+    with beam.Pipeline() as read_pipeline:
+      output_dicts = (
+          read_pipeline
+          | beam.managed.Read(beam.managed.ICEBERG, config=iceberg_config)
+          | beam.Map(lambda row: row._asdict()))
+
+      assert_that(output_dicts, equal_to(expected_dicts))
+
+
+if __name__ == '__main__':
+  unittest.main()
diff --git a/sdks/python/setup.py b/sdks/python/setup.py
index 15671eeb145b..b4175ad98e92 100644
--- a/sdks/python/setup.py
+++ b/sdks/python/setup.py
@@ -391,6 +391,7 @@ def get_portability_package_data():
           'sortedcontainers>=2.4.0',
           'typing-extensions>=3.7.0',
           'zstandard>=0.18.0,<1',
+          'pyyaml>=3.12,<7.0.0',
           # Dynamic dependencies must be specified in a separate list, otherwise
           # Dependabot won't be able to parse the main list. Any dynamic
           # dependencies will not receive updates from Dependabot.
@@ -415,7 +416,6 @@ def get_portability_package_data():
               'pandas<2.2.0',
               'parameterized>=0.7.1,<0.10.0',
               'pyhamcrest>=1.9,!=1.10.0,<3.0.0',
-              'pyyaml>=3.12,<7.0.0',
               'requests_mock>=1.7,<2.0',
               'tenacity>=8.0.0,<9',
               'pytest>=7.1.2,<8.0',
@@ -523,7 +523,6 @@ def get_portability_package_data():
           'yaml': [
               'docstring-parser>=0.15,<1.0',
               'jinja2>=3.0,<3.2',
-              'pyyaml>=3.12,<7.0.0',
               'virtualenv-clone>=0.5,<1.0',
               # https://github.com/PiotrDabkowski/Js2Py/issues/317
               'js2py>=0.74,<1; python_version<"3.12"',
diff --git a/sdks/standard_expansion_services.yaml b/sdks/standard_expansion_services.yaml
index ad965c8a1ee3..623e33fe2e7c 100644
--- a/sdks/standard_expansion_services.yaml
+++ b/sdks/standard_expansion_services.yaml
@@ -48,7 +48,7 @@
     # Handwritten Kafka wrappers already exist in apache_beam/io/kafka.py
     - 'beam:schematransform:org.apache.beam:kafka_write:v1'
     - 'beam:schematransform:org.apache.beam:kafka_read:v1'
-    # Not ready to generate
+    # Available through apache_beam.transforms.managed.[Read/Write]
     - 'beam:schematransform:org.apache.beam:iceberg_write:v1'
     - 'beam:schematransform:org.apache.beam:iceberg_read:v1'
 

From 80cba5644c338599144c3b227bc1db6b10a039f0 Mon Sep 17 00:00:00 2001
From: martin trieu 
Date: Tue, 15 Oct 2024 04:44:20 -0700
Subject: [PATCH 43/78] plumb backend worker token to work items (#32777)

---
 .../worker/streaming/ActiveWorkState.java     | 24 ++++++++++++----
 .../dataflow/worker/streaming/Work.java       | 28 +++++++++++++++----
 .../client/grpc/GrpcDirectGetWorkStream.java  |  6 +++-
 3 files changed, 47 insertions(+), 11 deletions(-)

diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java
index 4607096dd66a..aec52cd7d9a6 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java
@@ -240,18 +240,20 @@ synchronized Optional completeWorkAndGetNextWorkForKey(
     @Nullable Queue workQueue = activeWork.get(shardedKey);
     if (workQueue == null) {
       // Work may have been completed due to clearing of stuck commits.
-      LOG.warn("Unable to complete inactive work for key {} and token {}.", shardedKey, workId);
+      LOG.warn(
+          "Unable to complete inactive work for key={} and token={}.  Work queue for key does not exist.",
+          shardedKey,
+          workId);
       return Optional.empty();
     }
+
     removeCompletedWorkFromQueue(workQueue, shardedKey, workId);
     return getNextWork(workQueue, shardedKey);
   }
 
   private synchronized void removeCompletedWorkFromQueue(
       Queue workQueue, ShardedKey shardedKey, WorkId workId) {
-    // avoid Preconditions.checkState here to prevent eagerly evaluating the
-    // format string parameters for the error message.
-    ExecutableWork completedWork = workQueue.peek();
+    @Nullable ExecutableWork completedWork = workQueue.peek();
     if (completedWork == null) {
       // Work may have been completed due to clearing of stuck commits.
       LOG.warn("Active key {} without work, expected token {}", shardedKey, workId);
@@ -337,8 +339,18 @@ synchronized void printActiveWork(PrintWriter writer, Instant now) {
     writer.println(
         "");
+    // Columns.
     writer.println(
-        "");
+        ""
+            + ""
+            + ""
+            + ""
+            + ""
+            + ""
+            + ""
+            + ""
+            + ""
+            + "");
     // Use StringBuilder because we are appending in loop.
     StringBuilder activeWorkStatus = new StringBuilder();
     int commitsPendingCount = 0;
@@ -366,6 +378,8 @@ synchronized void printActiveWork(PrintWriter writer, Instant now) {
       activeWorkStatus.append(elapsedString(activeWork.getStateStartTime(), now));
       activeWorkStatus.append("\n");
     }
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java
index 03d1e1ae469a..6f97cbca9a80 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java
@@ -56,7 +56,7 @@
 /**
  * Represents the state of an attempt to process a {@link WorkItem} by executing user code.
  *
- * @implNote Not thread safe, should not be executed or accessed by more than 1 thread at a time.
+ * @implNote Not thread safe, should not be modified by more than 1 thread at a time.
  */
 @NotThreadSafe
 @Internal
@@ -70,7 +70,7 @@ public final class Work implements RefreshableWork {
   private final Map totalDurationPerState;
   private final WorkId id;
   private final String latencyTrackingId;
-  private TimedState currentState;
+  private volatile TimedState currentState;
   private volatile boolean isFailed;
   private volatile String processingThreadName = "";
 
@@ -111,7 +111,18 @@ public static ProcessingContext createProcessingContext(
       GetDataClient getDataClient,
       Consumer workCommitter,
       HeartbeatSender heartbeatSender) {
-    return ProcessingContext.create(computationId, getDataClient, workCommitter, heartbeatSender);
+    return ProcessingContext.create(
+        computationId, getDataClient, workCommitter, heartbeatSender, /* backendWorkerToken= */ "");
+  }
+
+  public static ProcessingContext createProcessingContext(
+      String computationId,
+      GetDataClient getDataClient,
+      Consumer workCommitter,
+      HeartbeatSender heartbeatSender,
+      String backendWorkerToken) {
+    return ProcessingContext.create(
+        computationId, getDataClient, workCommitter, heartbeatSender, backendWorkerToken);
   }
 
   private static LatencyAttribution.Builder createLatencyAttributionWithActiveLatencyBreakdown(
@@ -168,6 +179,10 @@ public GlobalData fetchSideInput(GlobalDataRequest request) {
     return processingContext.getDataClient().getSideInputData(request);
   }
 
+  public String backendWorkerToken() {
+    return processingContext.backendWorkerToken();
+  }
+
   public Watermarks watermarks() {
     return watermarks;
   }
@@ -351,9 +366,10 @@ private static ProcessingContext create(
         String computationId,
         GetDataClient getDataClient,
         Consumer workCommitter,
-        HeartbeatSender heartbeatSender) {
+        HeartbeatSender heartbeatSender,
+        String backendWorkerToken) {
       return new AutoValue_Work_ProcessingContext(
-          computationId, getDataClient, heartbeatSender, workCommitter);
+          computationId, getDataClient, heartbeatSender, workCommitter, backendWorkerToken);
     }
 
     /** Computation that the {@link Work} belongs to. */
@@ -370,6 +386,8 @@ private static ProcessingContext create(
      */
     public abstract Consumer workCommitter();
 
+    public abstract String backendWorkerToken();
+
     private Optional fetchKeyedState(KeyedGetDataRequest request) {
       return Optional.ofNullable(getDataClient().getStateData(computationId(), request));
     }
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java
index 45d010d7cfac..19de998b1da8 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java
@@ -254,7 +254,11 @@ private void consumeAssembledWorkItem(AssembledWorkItem assembledWorkItem) {
 
   private Work.ProcessingContext createProcessingContext(String computationId) {
     return Work.createProcessingContext(
-        computationId, getDataClient.get(), workCommitter.get()::commit, heartbeatSender.get());
+        computationId,
+        getDataClient.get(),
+        workCommitter.get()::commit,
+        heartbeatSender.get(),
+        backendWorkerToken());
   }
 
   @Override

From 45490aca9ace73e03d6c42e5d2c8668267daade4 Mon Sep 17 00:00:00 2001
From: Hai Joey Tran 
Date: Tue, 15 Oct 2024 10:27:16 -0400
Subject: [PATCH 44/78] Polish .with_exception_handling docstring (#32739)

* replace 'record' with 'input' and fix example

* more tweaking
---
 sdks/python/apache_beam/transforms/core.py | 13 +++++++------
 1 file changed, 7 insertions(+), 6 deletions(-)

diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py
index 8a5bb00eeb98..be3cec6304f4 100644
--- a/sdks/python/apache_beam/transforms/core.py
+++ b/sdks/python/apache_beam/transforms/core.py
@@ -1595,7 +1595,7 @@ def with_exception_handling(
       error_handler=None,
       on_failure_callback: typing.Optional[typing.Callable[
           [Exception, typing.Any], None]] = None):
-    """Automatically provides a dead letter output for skipping bad records.
+    """Automatically provides a dead letter output for saving bad inputs.
     This can allow a pipeline to continue successfully rather than fail or
     continuously throw errors on retry when bad elements are encountered.
 
@@ -1606,17 +1606,18 @@ def with_exception_handling(
 
     For example, one would write::
 
-        good, bad = Map(maybe_error_raising_function).with_exception_handling()
+        good, bad = inputs | Map(maybe_erroring_fn).with_exception_handling()
 
     and `good` will be a PCollection of mapped records and `bad` will contain
-    those that raised exceptions.
+    tuples of the form `(input, error_string`) for each input that raised an
+    exception.
 
 
     Args:
       main_tag: tag to be used for the main (good) output of the DoFn,
           useful to avoid possible conflicts if this DoFn already produces
           multiple outputs.  Optional, defaults to 'good'.
-      dead_letter_tag: tag to be used for the bad records, useful to avoid
+      dead_letter_tag: tag to be used for the bad inputs, useful to avoid
           possible conflicts if this DoFn already produces multiple outputs.
           Optional, defaults to 'bad'.
       exc_class: An exception class, or tuple of exception classes, to catch.
@@ -1635,9 +1636,9 @@ def with_exception_handling(
           than a new process per element, so the overhead should be minimal
           (and can be amortized if there's any per-process or per-bundle
           initialization that needs to be done). Optional, defaults to False.
-      threshold: An upper bound on the ratio of records that can be bad before
+      threshold: An upper bound on the ratio of inputs that can be bad before
           aborting the entire pipeline. Optional, defaults to 1.0 (meaning
-          up to 100% of records can be bad and the pipeline will still succeed).
+          up to 100% of inputs can be bad and the pipeline will still succeed).
       threshold_windowing: Event-time windowing to use for threshold. Optional,
           defaults to the windowing of the input.
       timeout: If the element has not finished processing in timeout seconds,

From 8fa19f24b5e1efa9d3b788432e1556c661fab64d Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 15 Oct 2024 10:27:53 -0400
Subject: [PATCH 45/78] Bump github.com/aws/aws-sdk-go-v2 from 1.32.1 to 1.32.2
 in /sdks (#32711)

Bumps [github.com/aws/aws-sdk-go-v2](https://github.com/aws/aws-sdk-go-v2) from 1.32.1 to 1.32.2.
- [Release notes](https://github.com/aws/aws-sdk-go-v2/releases)
- [Commits](https://github.com/aws/aws-sdk-go-v2/compare/v1.32.1...v1.32.2)

---
updated-dependencies:
- dependency-name: github.com/aws/aws-sdk-go-v2
  dependency-type: direct:production
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] 
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
---
 sdks/go.mod | 2 +-
 sdks/go.sum | 4 ++--
 2 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/sdks/go.mod b/sdks/go.mod
index 223ee2062b81..04362a5c43a3 100644
--- a/sdks/go.mod
+++ b/sdks/go.mod
@@ -30,7 +30,7 @@ require (
 	cloud.google.com/go/pubsub v1.43.0
 	cloud.google.com/go/spanner v1.67.0
 	cloud.google.com/go/storage v1.44.0
-	github.com/aws/aws-sdk-go-v2 v1.32.1
+	github.com/aws/aws-sdk-go-v2 v1.32.2
 	github.com/aws/aws-sdk-go-v2/config v1.27.42
 	github.com/aws/aws-sdk-go-v2/credentials v1.17.40
 	github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.28
diff --git a/sdks/go.sum b/sdks/go.sum
index 515c0c07a39a..ebeba7862d24 100644
--- a/sdks/go.sum
+++ b/sdks/go.sum
@@ -689,8 +689,8 @@ github.com/aws/aws-sdk-go v1.30.19/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZve
 github.com/aws/aws-sdk-go v1.34.0 h1:brux2dRrlwCF5JhTL7MUT3WUwo9zfDHZZp3+g3Mvlmo=
 github.com/aws/aws-sdk-go v1.34.0/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0=
 github.com/aws/aws-sdk-go-v2 v1.7.1/go.mod h1:L5LuPC1ZgDr2xQS7AmIec/Jlc7O/Y1u2KxJyNVab250=
-github.com/aws/aws-sdk-go-v2 v1.32.1 h1:8WuZ43ytA+TV6QEPT/R23mr7pWyI7bSSiEHdt9BS2Pw=
-github.com/aws/aws-sdk-go-v2 v1.32.1/go.mod h1:2SK5n0a2karNTv5tbP1SjsX0uhttou00v/HpXKM1ZUo=
+github.com/aws/aws-sdk-go-v2 v1.32.2 h1:AkNLZEyYMLnx/Q/mSKkcMqwNFXMAvFto9bNsHqcTduI=
+github.com/aws/aws-sdk-go-v2 v1.32.2/go.mod h1:2SK5n0a2karNTv5tbP1SjsX0uhttou00v/HpXKM1ZUo=
 github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.6 h1:pT3hpW0cOHRJx8Y0DfJUEQuqPild8jRGmSFmBgvydr0=
 github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.6/go.mod h1:j/I2++U0xX+cr44QjHay4Cvxj6FUbnxrgmqN3H1jTZA=
 github.com/aws/aws-sdk-go-v2/config v1.5.0/go.mod h1:RWlPOAW3E3tbtNAqTwvSW54Of/yP3oiZXMI0xfUdjyA=

From ade80d5cb3a311ce75261238a10af76469cb07b1 Mon Sep 17 00:00:00 2001
From: Shunping Huang 
Date: Tue, 15 Oct 2024 10:28:49 -0400
Subject: [PATCH 46/78] enable ordered list state (#32755)

---
 sdks/python/apache_beam/transforms/environments.py | 1 +
 1 file changed, 1 insertion(+)

diff --git a/sdks/python/apache_beam/transforms/environments.py b/sdks/python/apache_beam/transforms/environments.py
index dbb227802925..77704e0522b2 100644
--- a/sdks/python/apache_beam/transforms/environments.py
+++ b/sdks/python/apache_beam/transforms/environments.py
@@ -895,6 +895,7 @@ def _python_sdk_capabilities_iter():
   yield common_urns.primitives.TO_STRING.urn
   yield common_urns.protocols.DATA_SAMPLING.urn
   yield common_urns.protocols.SDK_CONSUMING_RECEIVED_DATA.urn
+  yield common_urns.protocols.ORDERED_LIST_STATE.urn
 
 
 def python_sdk_dependencies(options, tmp_dir=None):

From 547cc3989f3dacbc7510b59e0fae91ad31b14d27 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 15 Oct 2024 09:21:11 -0700
Subject: [PATCH 47/78] Bump github.com/aws/aws-sdk-go-v2/feature/s3/manager in
 /sdks (#32768)

Bumps [github.com/aws/aws-sdk-go-v2/feature/s3/manager](https://github.com/aws/aws-sdk-go-v2) from 1.17.28 to 1.17.32.
- [Release notes](https://github.com/aws/aws-sdk-go-v2/releases)
- [Commits](https://github.com/aws/aws-sdk-go-v2/compare/credentials/v1.17.28...credentials/v1.17.32)

---
updated-dependencies:
- dependency-name: github.com/aws/aws-sdk-go-v2/feature/s3/manager
  dependency-type: direct:production
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] 
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
---
 sdks/go.mod | 28 +++++++++++++--------------
 sdks/go.sum | 56 ++++++++++++++++++++++++++---------------------------
 2 files changed, 42 insertions(+), 42 deletions(-)

diff --git a/sdks/go.mod b/sdks/go.mod
index 04362a5c43a3..0b5ac98df404 100644
--- a/sdks/go.mod
+++ b/sdks/go.mod
@@ -31,10 +31,10 @@ require (
 	cloud.google.com/go/spanner v1.67.0
 	cloud.google.com/go/storage v1.44.0
 	github.com/aws/aws-sdk-go-v2 v1.32.2
-	github.com/aws/aws-sdk-go-v2/config v1.27.42
-	github.com/aws/aws-sdk-go-v2/credentials v1.17.40
-	github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.28
-	github.com/aws/aws-sdk-go-v2/service/s3 v1.65.0
+	github.com/aws/aws-sdk-go-v2/config v1.27.43
+	github.com/aws/aws-sdk-go-v2/credentials v1.17.41
+	github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.32
+	github.com/aws/aws-sdk-go-v2/service/s3 v1.65.3
 	github.com/aws/smithy-go v1.22.0
 	github.com/docker/go-connections v0.5.0
 	github.com/dustin/go-humanize v1.0.1
@@ -131,18 +131,18 @@ require (
 	github.com/apache/thrift v0.17.0 // indirect
 	github.com/aws/aws-sdk-go v1.34.0 // indirect
 	github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.6 // indirect
-	github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.16 // indirect
-	github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.20 // indirect
-	github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.20 // indirect
+	github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.17 // indirect
+	github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.21 // indirect
+	github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.21 // indirect
 	github.com/aws/aws-sdk-go-v2/internal/ini v1.8.1 // indirect
-	github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.19 // indirect
+	github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.21 // indirect
 	github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.12.0 // indirect
-	github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.4.0 // indirect
-	github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.1 // indirect
-	github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.0 // indirect
-	github.com/aws/aws-sdk-go-v2/service/sso v1.24.1 // indirect
-	github.com/aws/aws-sdk-go-v2/service/ssooidc v1.28.1 // indirect
-	github.com/aws/aws-sdk-go-v2/service/sts v1.32.1 // indirect
+	github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.4.2 // indirect
+	github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.2 // indirect
+	github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.2 // indirect
+	github.com/aws/aws-sdk-go-v2/service/sso v1.24.2 // indirect
+	github.com/aws/aws-sdk-go-v2/service/ssooidc v1.28.2 // indirect
+	github.com/aws/aws-sdk-go-v2/service/sts v1.32.2 // indirect
 	github.com/cenkalti/backoff/v4 v4.2.1 // indirect
 	github.com/census-instrumentation/opencensus-proto v0.4.1 // indirect
 	github.com/cespare/xxhash/v2 v2.3.0 // indirect
diff --git a/sdks/go.sum b/sdks/go.sum
index ebeba7862d24..db6d71b061b5 100644
--- a/sdks/go.sum
+++ b/sdks/go.sum
@@ -694,48 +694,48 @@ github.com/aws/aws-sdk-go-v2 v1.32.2/go.mod h1:2SK5n0a2karNTv5tbP1SjsX0uhttou00v
 github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.6 h1:pT3hpW0cOHRJx8Y0DfJUEQuqPild8jRGmSFmBgvydr0=
 github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.6/go.mod h1:j/I2++U0xX+cr44QjHay4Cvxj6FUbnxrgmqN3H1jTZA=
 github.com/aws/aws-sdk-go-v2/config v1.5.0/go.mod h1:RWlPOAW3E3tbtNAqTwvSW54Of/yP3oiZXMI0xfUdjyA=
-github.com/aws/aws-sdk-go-v2/config v1.27.42 h1:Zsy9coUPuOsCWkjTvHpl2/DB9bptXtv7WeNPxvFr87s=
-github.com/aws/aws-sdk-go-v2/config v1.27.42/go.mod h1:FGASs+PuJM2EY+8rt8qyQKLPbbX/S5oY+6WzJ/KE7ko=
+github.com/aws/aws-sdk-go-v2/config v1.27.43 h1:p33fDDihFC390dhhuv8nOmX419wjOSDQRb+USt20RrU=
+github.com/aws/aws-sdk-go-v2/config v1.27.43/go.mod h1:pYhbtvg1siOOg8h5an77rXle9tVG8T+BWLWAo7cOukc=
 github.com/aws/aws-sdk-go-v2/credentials v1.3.1/go.mod h1:r0n73xwsIVagq8RsxmZbGSRQFj9As3je72C2WzUIToc=
-github.com/aws/aws-sdk-go-v2/credentials v1.17.40 h1:RjnlA7t0p/IamxAM7FUJ5uS13Vszh4sjVGvsx91tGro=
-github.com/aws/aws-sdk-go-v2/credentials v1.17.40/go.mod h1:dgpdnSs1Bp/atS6vLlW83h9xZPP+uSPB/27dFSgC1BM=
+github.com/aws/aws-sdk-go-v2/credentials v1.17.41 h1:7gXo+Axmp+R4Z+AK8YFQO0ZV3L0gizGINCOWxSLY9W8=
+github.com/aws/aws-sdk-go-v2/credentials v1.17.41/go.mod h1:u4Eb8d3394YLubphT4jLEwN1rLNq2wFOlT6OuxFwPzU=
 github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.3.0/go.mod h1:2LAuqPx1I6jNfaGDucWfA2zqQCYCOMCDHiCOciALyNw=
-github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.16 h1:fwrer1pJeaiia0CcOfWVbZxvj9Adc7rsuaMTwPR0DIA=
-github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.16/go.mod h1:XyEwwp8XI4zMar7MTnJ0Sk7qY/9aN8Hp929XhuX5SF8=
+github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.17 h1:TMH3f/SCAWdNtXXVPPu5D6wrr4G5hI1rAxbcocKfC7Q=
+github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.17/go.mod h1:1ZRXLdTpzdJb9fwTMXiLipENRxkGMTn1sfKexGllQCw=
 github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.3.2/go.mod h1:qaqQiHSrOUVOfKe6fhgQ6UzhxjwqVW8aHNegd6Ws4w4=
-github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.28 h1:yUPy1fwOKNZ9L52E9TCMomU+mKXNCgqi17dtYIdSolk=
-github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.28/go.mod h1:bJJP1cGMO0fPBgCjqHAWbc0WRbKrxrWU4hQfc/0ciAA=
-github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.20 h1:OErdlGnt+hg3tTwGYAlKvFkKVUo/TXkoHcxDxuhYYU8=
-github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.20/go.mod h1:HsPfuL5gs+407ByRXBMgpYoyrV1sgMrzd18yMXQHJpo=
-github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.20 h1:822cE1CYSwY/EZnErlF46pyynuxvf1p+VydHRQW+XNs=
-github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.20/go.mod h1:79/Tn7H7hYC5Gjz6fbnOV4OeBpkao7E8Tv95RO72pMM=
+github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.32 h1:C2hE+gJ40Cb4vzhFJ+tTzjvBpPloUq7XP6PD3A2Fk7g=
+github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.32/go.mod h1:0OmMtVNp+10JFBTfmA2AIeqBDm0YthDXmE+N7poaptk=
+github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.21 h1:UAsR3xA31QGf79WzpG/ixT9FZvQlh5HY1NRqSHBNOCk=
+github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.21/go.mod h1:JNr43NFf5L9YaG3eKTm7HQzls9J+A9YYcGI5Quh1r2Y=
+github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.21 h1:6jZVETqmYCadGFvrYEQfC5fAQmlo80CeL5psbno6r0s=
+github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.21/go.mod h1:1SR0GbLlnN3QUmYaflZNiH1ql+1qrSiB2vwcJ+4UM60=
 github.com/aws/aws-sdk-go-v2/internal/ini v1.1.1/go.mod h1:Zy8smImhTdOETZqfyn01iNOe0CNggVbPjCajyaz6Gvg=
 github.com/aws/aws-sdk-go-v2/internal/ini v1.8.1 h1:VaRN3TlFdd6KxX1x3ILT5ynH6HvKgqdiXoTxAF4HQcQ=
 github.com/aws/aws-sdk-go-v2/internal/ini v1.8.1/go.mod h1:FbtygfRFze9usAadmnGJNc8KsP346kEe+y2/oyhGAGc=
-github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.19 h1:FKdiFzTxlTRO71p0C7VrLbkkdW8qfMKF5+ej6bTmkT0=
-github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.19/go.mod h1:abO3pCj7WLQPTllnSeYImqFfkGrmJV0JovWo/gqT5N0=
+github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.21 h1:7edmS3VOBDhK00b/MwGtGglCm7hhwNYnjJs/PgFdMQE=
+github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.21/go.mod h1:Q9o5h4HoIWG8XfzxqiuK/CGUbepCJ8uTlaE3bAbxytQ=
 github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.2.1/go.mod h1:v33JQ57i2nekYTA70Mb+O18KeH4KqhdqxTJZNK1zdRE=
 github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.12.0 h1:TToQNkvGguu209puTojY/ozlqy2d/SFNcoLIqTFi42g=
 github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.12.0/go.mod h1:0jp+ltwkf+SwG2fm/PKo8t4y8pJSgOCO4D8Lz3k0aHQ=
-github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.4.0 h1:FQNWhRuSq8QwW74GtU0MrveNhZbqvHsA4dkA9w8fTDQ=
-github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.4.0/go.mod h1:j/zZ3zmWfGCK91K73YsfHP53BSTLSjL/y6YN39XbBLM=
+github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.4.2 h1:4FMHqLfk0efmTqhXVRL5xYRqlEBNBiRI7N6w4jsEdd4=
+github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.4.2/go.mod h1:LWoqeWlK9OZeJxsROW2RqrSPvQHKTpp69r/iDjwsSaw=
 github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.2.1/go.mod h1:zceowr5Z1Nh2WVP8bf/3ikB41IZW59E4yIYbg+pC6mw=
-github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.1 h1:5vBMBTakOvtd8aNaicswcrr9qqCYUlasuzyoU6/0g8I=
-github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.1/go.mod h1:WSUbDa5qdg05Q558KXx2Scb+EDvOPXT9gfET0fyrJSk=
+github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.2 h1:s7NA1SOw8q/5c0wr8477yOPp0z+uBaXBnLE0XYb0POA=
+github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.2/go.mod h1:fnjjWyAW/Pj5HYOxl9LJqWtEwS7W2qgcRLWP+uWbss0=
 github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.5.1/go.mod h1:6EQZIwNNvHpq/2/QSJnp4+ECvqIy55w95Ofs0ze+nGQ=
-github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.0 h1:1NKXS8XfhMM0bg5wVYa/eOH8AM2f6JijugbKEyQFTIg=
-github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.0/go.mod h1:ph931DUfVfgrhZR7py9olSvHCiRpvaGxNvlWBcXxFds=
+github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.2 h1:t7iUP9+4wdc5lt3E41huP+GvQZJD38WLsgVp4iOtAjg=
+github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.2/go.mod h1:/niFCtmuQNxqx9v8WAPq5qh7EH25U4BF6tjoyq9bObM=
 github.com/aws/aws-sdk-go-v2/service/s3 v1.11.1/go.mod h1:XLAGFrEjbvMCLvAtWLLP32yTv8GpBquCApZEycDLunI=
-github.com/aws/aws-sdk-go-v2/service/s3 v1.65.0 h1:2dSm7frMrw2tdJ0QvyccQNJyPGaP24dyDgZ6h1QJMGU=
-github.com/aws/aws-sdk-go-v2/service/s3 v1.65.0/go.mod h1:4XSVpw66upN8wND3JZA29eXl2NOZvfFVq7DIP6xvfuQ=
+github.com/aws/aws-sdk-go-v2/service/s3 v1.65.3 h1:xxHGZ+wUgZNACQmxtdvP5tgzfsxGS3vPpTP5Hy3iToE=
+github.com/aws/aws-sdk-go-v2/service/s3 v1.65.3/go.mod h1:cB6oAuus7YXRZhWCc1wIwPywwZ1XwweNp2TVAEGYeB8=
 github.com/aws/aws-sdk-go-v2/service/sso v1.3.1/go.mod h1:J3A3RGUvuCZjvSuZEcOpHDnzZP/sKbhDWV2T1EOzFIM=
-github.com/aws/aws-sdk-go-v2/service/sso v1.24.1 h1:aAIr0WhAgvKrxZtkBqne87Gjmd7/lJVTFkR2l2yuhL8=
-github.com/aws/aws-sdk-go-v2/service/sso v1.24.1/go.mod h1:8XhxGMWUfikJuginPQl5SGZ0LSJuNX3TCEQmFWZwHTM=
-github.com/aws/aws-sdk-go-v2/service/ssooidc v1.28.1 h1:J6kIsIkgFOaU6aKjigXJoue1XEHtKIIrpSh4vKdmRTs=
-github.com/aws/aws-sdk-go-v2/service/ssooidc v1.28.1/go.mod h1:2V2JLP7tXOmUbL3Hd1ojq+774t2KUAEQ35//shoNEL0=
+github.com/aws/aws-sdk-go-v2/service/sso v1.24.2 h1:bSYXVyUzoTHoKalBmwaZxs97HU9DWWI3ehHSAMa7xOk=
+github.com/aws/aws-sdk-go-v2/service/sso v1.24.2/go.mod h1:skMqY7JElusiOUjMJMOv1jJsP7YUg7DrhgqZZWuzu1U=
+github.com/aws/aws-sdk-go-v2/service/ssooidc v1.28.2 h1:AhmO1fHINP9vFYUE0LHzCWg/LfUWUF+zFPEcY9QXb7o=
+github.com/aws/aws-sdk-go-v2/service/ssooidc v1.28.2/go.mod h1:o8aQygT2+MVP0NaV6kbdE1YnnIM8RRVQzoeUH45GOdI=
 github.com/aws/aws-sdk-go-v2/service/sts v1.6.0/go.mod h1:q7o0j7d7HrJk/vr9uUt3BVRASvcU7gYZB9PUgPiByXg=
-github.com/aws/aws-sdk-go-v2/service/sts v1.32.1 h1:q76Ig4OaJzVJGNUSGO3wjSTBS94g+EhHIbpY9rPvkxs=
-github.com/aws/aws-sdk-go-v2/service/sts v1.32.1/go.mod h1:664dajZ7uS7JMUMUG0R5bWbtN97KECNCVdFDdQ6Ipu8=
+github.com/aws/aws-sdk-go-v2/service/sts v1.32.2 h1:CiS7i0+FUe+/YY1GvIBLLrR/XNGZ4CtM1Ll0XavNuVo=
+github.com/aws/aws-sdk-go-v2/service/sts v1.32.2/go.mod h1:HtaiBI8CjYoNVde8arShXb94UbQQi9L4EMr6D+xGBwo=
 github.com/aws/smithy-go v1.6.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E=
 github.com/aws/smithy-go v1.22.0 h1:uunKnWlcoL3zO7q+gG2Pk53joueEOsnNB28QdMsmiMM=
 github.com/aws/smithy-go v1.22.0/go.mod h1:irrKGvNn1InZwb2d7fkIRNucdfwR8R+Ts3wxYa/cJHg=

From 89dd0887a4341fa4a1f59fdf413d35372a1fdffe Mon Sep 17 00:00:00 2001
From: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com>
Date: Tue, 15 Oct 2024 20:42:00 +0300
Subject: [PATCH 48/78] Add license to fix RAT failure (#32785)

---
 .../transforms/managed_iceberg_it_test.py       | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)

diff --git a/sdks/python/apache_beam/transforms/managed_iceberg_it_test.py b/sdks/python/apache_beam/transforms/managed_iceberg_it_test.py
index 2d7262bac031..0dfa2aa19c51 100644
--- a/sdks/python/apache_beam/transforms/managed_iceberg_it_test.py
+++ b/sdks/python/apache_beam/transforms/managed_iceberg_it_test.py
@@ -1,3 +1,20 @@
+#
+# 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 os
 import secrets
 import shutil

From 06ecee96e7541d108613b30ecc09abfcaf9929bd Mon Sep 17 00:00:00 2001
From: Danny McCormick 
Date: Tue, 15 Oct 2024 14:19:04 -0400
Subject: [PATCH 49/78] vLLM model handler efficiency improvements (#32687)

* vLLM model handler efficiency improvements

* fmt

* Remove bad exceptions

* lint

* lint
---
 .../trigger_files/beam_PostCommit_Python.json |   2 +-
 .../ml/inference/vllm_inference.py            | 103 ++++++++++++------
 2 files changed, 71 insertions(+), 34 deletions(-)

diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json
index 1eb60f6e4959..9e1d1e1b80dd 100644
--- a/.github/trigger_files/beam_PostCommit_Python.json
+++ b/.github/trigger_files/beam_PostCommit_Python.json
@@ -1,5 +1,5 @@
 {
   "comment": "Modify this file in a trivial way to cause this test suite to run.",
-  "modification": 3
+  "modification": 4
 }
 
diff --git a/sdks/python/apache_beam/ml/inference/vllm_inference.py b/sdks/python/apache_beam/ml/inference/vllm_inference.py
index 28890083d93e..e1ba4f49b8fd 100644
--- a/sdks/python/apache_beam/ml/inference/vllm_inference.py
+++ b/sdks/python/apache_beam/ml/inference/vllm_inference.py
@@ -17,6 +17,7 @@
 
 # pytype: skip-file
 
+import asyncio
 import logging
 import os
 import subprocess
@@ -35,6 +36,7 @@
 from apache_beam.ml.inference.base import ModelHandler
 from apache_beam.ml.inference.base import PredictionResult
 from apache_beam.utils import subprocess_server
+from openai import AsyncOpenAI
 from openai import OpenAI
 
 try:
@@ -94,6 +96,15 @@ def getVLLMClient(port) -> OpenAI:
   )
 
 
+def getAsyncVLLMClient(port) -> AsyncOpenAI:
+  openai_api_key = "EMPTY"
+  openai_api_base = f"http://localhost:{port}/v1"
+  return AsyncOpenAI(
+      api_key=openai_api_key,
+      base_url=openai_api_base,
+  )
+
+
 class _VLLMModelServer():
   def __init__(self, model_name: str, vllm_server_kwargs: Dict[str, str]):
     self._model_name = model_name
@@ -184,6 +195,34 @@ def __init__(
   def load_model(self) -> _VLLMModelServer:
     return _VLLMModelServer(self._model_name, self._vllm_server_kwargs)
 
+  async def _async_run_inference(
+      self,
+      batch: Sequence[str],
+      model: _VLLMModelServer,
+      inference_args: Optional[Dict[str, Any]] = None
+  ) -> Iterable[PredictionResult]:
+    client = getAsyncVLLMClient(model.get_server_port())
+    inference_args = inference_args or {}
+    async_predictions = []
+    for prompt in batch:
+      try:
+        completion = client.completions.create(
+            model=self._model_name, prompt=prompt, **inference_args)
+        async_predictions.append(completion)
+      except Exception as e:
+        model.check_connectivity()
+        raise e
+
+    predictions = []
+    for p in async_predictions:
+      try:
+        predictions.append(await p)
+      except Exception as e:
+        model.check_connectivity()
+        raise e
+
+    return [PredictionResult(x, y) for x, y in zip(batch, predictions)]
+
   def run_inference(
       self,
       batch: Sequence[str],
@@ -200,22 +239,7 @@ def run_inference(
     Returns:
       An Iterable of type PredictionResult.
     """
-    client = getVLLMClient(model.get_server_port())
-    inference_args = inference_args or {}
-    predictions = []
-    # TODO(https://github.com/apache/beam/issues/32528): We should add support
-    # for taking in batches and doing a bunch of async calls. That will end up
-    # being more efficient when we can do in bundle batching.
-    for prompt in batch:
-      try:
-        completion = client.completions.create(
-            model=self._model_name, prompt=prompt, **inference_args)
-        predictions.append(completion)
-      except Exception as e:
-        model.check_connectivity()
-        raise e
-
-    return [PredictionResult(x, y) for x, y in zip(batch, predictions)]
+    return asyncio.run(self._async_run_inference(batch, model, inference_args))
 
   def share_model_across_processes(self) -> bool:
     return True
@@ -272,28 +296,15 @@ def load_model(self) -> _VLLMModelServer:
 
     return _VLLMModelServer(self._model_name, self._vllm_server_kwargs)
 
-  def run_inference(
+  async def _async_run_inference(
       self,
       batch: Sequence[Sequence[OpenAIChatMessage]],
       model: _VLLMModelServer,
       inference_args: Optional[Dict[str, Any]] = None
   ) -> Iterable[PredictionResult]:
-    """Runs inferences on a batch of text strings.
-
-    Args:
-      batch: A sequence of examples as OpenAI messages.
-      model: A _VLLMModelServer for connecting to the spun up server.
-      inference_args: Any additional arguments for an inference.
-
-    Returns:
-      An Iterable of type PredictionResult.
-    """
-    client = getVLLMClient(model.get_server_port())
+    client = getAsyncVLLMClient(model.get_server_port())
     inference_args = inference_args or {}
-    predictions = []
-    # TODO(https://github.com/apache/beam/issues/32528): We should add support
-    # for taking in batches and doing a bunch of async calls. That will end up
-    # being more efficient when we can do in bundle batching.
+    async_predictions = []
     for messages in batch:
       formatted = []
       for message in messages:
@@ -301,12 +312,38 @@ def run_inference(
       try:
         completion = client.chat.completions.create(
             model=self._model_name, messages=formatted, **inference_args)
-        predictions.append(completion)
+        async_predictions.append(completion)
+      except Exception as e:
+        model.check_connectivity()
+        raise e
+
+    predictions = []
+    for p in async_predictions:
+      try:
+        predictions.append(await p)
       except Exception as e:
         model.check_connectivity()
         raise e
 
     return [PredictionResult(x, y) for x, y in zip(batch, predictions)]
 
+  def run_inference(
+      self,
+      batch: Sequence[Sequence[OpenAIChatMessage]],
+      model: _VLLMModelServer,
+      inference_args: Optional[Dict[str, Any]] = None
+  ) -> Iterable[PredictionResult]:
+    """Runs inferences on a batch of text strings.
+
+    Args:
+      batch: A sequence of examples as OpenAI messages.
+      model: A _VLLMModelServer for connecting to the spun up server.
+      inference_args: Any additional arguments for an inference.
+
+    Returns:
+      An Iterable of type PredictionResult.
+    """
+    return asyncio.run(self._async_run_inference(batch, model, inference_args))
+
   def share_model_across_processes(self) -> bool:
     return True

From 1b744a7fbc2725ed98a130027a042c3adc10b96b Mon Sep 17 00:00:00 2001
From: claudevdm <33973061+claudevdm@users.noreply.github.com>
Date: Tue, 15 Oct 2024 14:22:40 -0400
Subject: [PATCH 50/78] Created using Colab (#32789)

---
 bigquery_enrichment_transform.ipynb | 781 ++++++++++++++++++++++++++++
 1 file changed, 781 insertions(+)
 create mode 100644 bigquery_enrichment_transform.ipynb

diff --git a/bigquery_enrichment_transform.ipynb b/bigquery_enrichment_transform.ipynb
new file mode 100644
index 000000000000..331ecb9ba93d
--- /dev/null
+++ b/bigquery_enrichment_transform.ipynb
@@ -0,0 +1,781 @@
+{
+  "nbformat": 4,
+  "nbformat_minor": 0,
+  "metadata": {
+    "colab": {
+      "provenance": [],
+      "include_colab_link": true
+    },
+    "kernelspec": {
+      "name": "python3",
+      "display_name": "Python 3"
+    },
+    "language_info": {
+      "name": "python"
+    }
+  },
+  "cells": [
+    {
+      "cell_type": "markdown",
+      "metadata": {
+        "id": "view-in-github",
+        "colab_type": "text"
+      },
+      "source": [
+        "\"Open"
+      ]
+    },
+    {
+      "cell_type": "code",
+      "source": [
+        "# @title ###### Licensed to the Apache Software Foundation (ASF), Version 2.0 (the \"License\")\n",
+        "\n",
+        "# Licensed to the Apache Software Foundation (ASF) under one\n",
+        "# or more contributor license agreements. See the NOTICE file\n",
+        "# distributed with this work for additional information\n",
+        "# regarding copyright ownership. The ASF licenses this file\n",
+        "# to you under the Apache License, Version 2.0 (the\n",
+        "# \"License\"); you may not use this file except in compliance\n",
+        "# with the License. You may obtain a copy of the License at\n",
+        "#\n",
+        "#   http://www.apache.org/licenses/LICENSE-2.0\n",
+        "#\n",
+        "# Unless required by applicable law or agreed to in writing,\n",
+        "# software distributed under the License is distributed on an\n",
+        "# \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY\n",
+        "# KIND, either express or implied. See the License for the\n",
+        "# specific language governing permissions and limitations\n",
+        "# under the License"
+      ],
+      "metadata": {
+        "id": "55h6JBJeJGqg",
+        "cellView": "form"
+      },
+      "execution_count": null,
+      "outputs": []
+    },
+    {
+      "cell_type": "markdown",
+      "source": [
+        "# Use Apache Beam and BigQuery to enrich data\n",
+        "\n",
+        "
KeyTokenQueuedActive ForStateState Active ForProcessing Thread
KeyTokenQueuedActive ForStateState Active ForProcessing ThreadBackend
"); activeWorkStatus.append(activeWork.getProcessingThreadName()); + activeWorkStatus.append(""); + activeWorkStatus.append(activeWork.backendWorkerToken()); activeWorkStatus.append("
\n", + " \n", + " \n", + "
\n", + " Run in Google Colab\n", + " \n", + " View source on GitHub\n", + "
\n" + ], + "metadata": { + "id": "YrOuxMeKJZxC" + } + }, + { + "cell_type": "markdown", + "source": [ + "This notebook shows how to enrich data by using the Apache Beam [enrichment transform](https://beam.apache.org/documentation/transforms/python/elementwise/enrichment/) with [BigQuery](https://cloud.google.com/bigquery/docs/overview). The enrichment transform is an Apache Beam turnkey transform that lets you enrich data by using a key-value lookup. This transform has the following features:\n", + "\n", + "- The transform has a built-in Apache Beam handler that interacts with BigQuery data during enrichment.\n", + "- The enrichment transform uses client-side throttling to rate limit the requests. The default retry strategy uses exponential backoff. You can configure rate limiting to suit your use case.\n", + "\n", + "This notebook demonstrates the following telecommunications company use case:\n", + "\n", + "A telecom company wants to predict which customers are likely to cancel their subscriptions so that the company can proactively offer these customers incentives to stay. The example uses customer demographic data and usage data stored in BigQuery to enrich a stream of customer IDs. The enriched data is then used to predict the likelihood of customer churn.\n", + "\n", + "## Before you begin\n", + "Set up your environment and download dependencies.\n", + "\n", + "### Install Apache Beam\n", + "To use the enrichment transform with the built-in BigQuery handler, install the Apache Beam SDK version 2.57.0 or later." + ], + "metadata": { + "id": "pf2bL-PmJScZ" + } + }, + { + "cell_type": "code", + "source": [ + "!pip install torch\n", + "!pip install apache_beam[interactive,gcp]==2.57.0 --quiet" + ], + "metadata": { + "id": "oVbWf73FJSzf" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "Import the following modules:\n", + "- Pub/Sub for streaming data\n", + "- BigQuery for enrichment\n", + "- Apache Beam for running the streaming pipeline\n", + "- PyTorch to predict customer churn" + ], + "metadata": { + "id": "siSUsfR5tKX9" + } + }, + { + "cell_type": "code", + "source": [ + "import datetime\n", + "import json\n", + "import math\n", + "\n", + "from typing import Any\n", + "from typing import Dict\n", + "\n", + "import torch\n", + "from google.cloud import pubsub_v1\n", + "from google.cloud import bigquery\n", + "from google.api_core.exceptions import Conflict\n", + "\n", + "import apache_beam as beam\n", + "import apache_beam.runners.interactive.interactive_beam as ib\n", + "from apache_beam.ml.inference.base import KeyedModelHandler\n", + "from apache_beam.ml.inference.base import RunInference\n", + "from apache_beam.ml.inference.pytorch_inference import PytorchModelHandlerTensor\n", + "from apache_beam.options import pipeline_options\n", + "from apache_beam.runners.interactive.interactive_runner import InteractiveRunner\n", + "from apache_beam.transforms.enrichment import Enrichment\n", + "from apache_beam.transforms.enrichment_handlers.bigquery import BigQueryEnrichmentHandler\n", + "\n", + "import pandas as pd\n", + "\n", + "from sklearn.preprocessing import LabelEncoder" + ], + "metadata": { + "id": "p6bruDqFJkXE" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "### Authenticate with Google Cloud\n", + "This notebook reads data from Pub/Sub and BigQuery. To use your Google Cloud account, authenticate this notebook.\n", + "To prepare for this step, replace `` with your Google Cloud project ID." + ], + "metadata": { + "id": "t0QfhuUlJozO" + } + }, + { + "cell_type": "code", + "source": [ + "PROJECT_ID = \"\"\n" + ], + "metadata": { + "id": "RwoBZjD1JwnD" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "code", + "source": [ + "from google.colab import auth\n", + "auth.authenticate_user(project_id=PROJECT_ID)" + ], + "metadata": { + "id": "rVAyQxoeKflB" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "### Set up the BigQuery tables\n", + "\n", + "Create sample BigQuery tables for this notebook.\n", + "\n", + "- Replace `` with the name of your BigQuery dataset. Only letters (uppercase or lowercase), numbers, and underscores are allowed.\n", + "- If the dataset does not exist, a new dataset with this ID is created." + ], + "metadata": { + "id": "1vDwknoHKoa-" + } + }, + { + "cell_type": "code", + "source": [ + "DATASET_ID = \"\"\n", + "\n", + "CUSTOMERS_TABLE_ID = f'{PROJECT_ID}.{DATASET_ID}.customers'\n", + "USAGE_TABLE_ID = f'{PROJECT_ID}.{DATASET_ID}.usage'" + ], + "metadata": { + "id": "UxeGFqSJu-G6" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "Create customer and usage tables, and insert fake data." + ], + "metadata": { + "id": "Gw4RfZavyfpo" + } + }, + { + "cell_type": "code", + "source": [ + "client = bigquery.Client(project=PROJECT_ID)\n", + "\n", + "# Create dataset if it does not exist.\n", + "client.create_dataset(bigquery.Dataset(f\"{PROJECT_ID}.{DATASET_ID}\"), exists_ok=True)\n", + "print(f\"Created dataset {DATASET_ID}\")\n", + "\n", + "# Prepare the fake customer data.\n", + "customer_data = {\n", + " 'customer_id': [1, 2, 3, 4, 5],\n", + " 'age': [35, 28, 45, 62, 22],\n", + " 'plan': ['Gold', 'Silver', 'Bronze', 'Gold', 'Silver'],\n", + " 'contract_length': [12, 24, 6, 36, 12]\n", + "}\n", + "\n", + "customers_df = pd.DataFrame(customer_data)\n", + "\n", + "# Insert customer data.\n", + "job_config = bigquery.LoadJobConfig(\n", + " schema=[\n", + " bigquery.SchemaField(\"customer_id\", \"INTEGER\"),\n", + " bigquery.SchemaField(\"age\", \"INTEGER\"),\n", + " bigquery.SchemaField(\"plan\", \"STRING\"),\n", + " bigquery.SchemaField(\"contract_length\", \"INTEGER\"),\n", + " ],\n", + " write_disposition=\"WRITE_TRUNCATE\",\n", + ")\n", + "\n", + "job = client.load_table_from_dataframe(\n", + " customers_df, CUSTOMERS_TABLE_ID, job_config=job_config\n", + ")\n", + "job.result() # Wait for the job to complete.\n", + "print(f\"Customers table created and populated: {CUSTOMERS_TABLE_ID}\")\n", + "\n", + "# Prepare the fake usage data.\n", + "usage_data = {\n", + " 'customer_id': [1, 1, 2, 2, 3, 3, 4, 4, 5, 5],\n", + " 'date': pd.to_datetime(['2024-09-01', '2024-10-01', '2024-09-01', '2024-10-01', '2024-09-01', '2024-10-01', '2024-09-01', '2024-10-01', '2024-09-01', '2024-10-01']),\n", + " 'calls_made': [50, 65, 20, 18, 100, 110, 30, 28, 60, 70],\n", + " 'data_usage_gb': [10, 12, 5, 4, 20, 22, 8, 7, 15, 18]\n", + "}\n", + "usage_df = pd.DataFrame(usage_data)\n", + "\n", + "# Insert usage data.\n", + "job_config = bigquery.LoadJobConfig(\n", + " schema=[\n", + " bigquery.SchemaField(\"customer_id\", \"INTEGER\"),\n", + " bigquery.SchemaField(\"date\", \"DATE\"),\n", + " bigquery.SchemaField(\"calls_made\", \"INTEGER\"),\n", + " bigquery.SchemaField(\"data_usage_gb\", \"FLOAT\"),\n", + " ],\n", + " write_disposition=\"WRITE_TRUNCATE\",\n", + ")\n", + "job = client.load_table_from_dataframe(\n", + " usage_df, USAGE_TABLE_ID, job_config=job_config\n", + ")\n", + "job.result() # Wait for the job to complete.\n", + "\n", + "print(f\"Usage table created and populated: {USAGE_TABLE_ID}\")" + ], + "metadata": { + "id": "-QRZC4v0KipK" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "### Train the model" + ], + "metadata": { + "id": "PZCjCzxaLOJt" + } + }, + { + "cell_type": "markdown", + "source": [ + "Create sample data and train a simple model for churn prediction." + ], + "metadata": { + "id": "R4dIHclDLfIj" + } + }, + { + "cell_type": "code", + "source": [ + "# Create fake training data\n", + "data = {\n", + " 'customer_id': [1, 2, 3, 4, 5, 6, 7, 8, 9, 10],\n", + " 'age': [35, 28, 45, 62, 22, 38, 55, 25, 40, 30],\n", + " 'plan': ['Gold', 'Silver', 'Bronze', 'Gold', 'Silver', 'Bronze', 'Gold', 'Silver', 'Bronze', 'Silver'],\n", + " 'contract_length': [12, 24, 6, 36, 12, 18, 30, 12, 24, 18],\n", + " 'avg_monthly_calls': [57.5, 19, 100, 30, 60, 45, 25, 70, 50, 35],\n", + " 'avg_monthly_data_usage_gb': [11, 4.5, 20, 8, 15, 10, 7, 18, 12, 8],\n", + " 'churned': [0, 0, 1, 0, 1, 0, 0, 1, 0, 1] # Target variable\n", + "}\n", + "plan_encoder = LabelEncoder()\n", + "plan_encoder.fit(data['plan'])\n", + "df = pd.DataFrame(data)\n", + "df['plan'] = plan_encoder.transform(data['plan'])\n", + "\n" + ], + "metadata": { + "id": "YoMjdqJ1KxOM" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "Preprocess the data:\n", + "\n", + "1. Convert the lists to tensors.\n", + "2. Separate the features from the expected prediction." + ], + "metadata": { + "id": "EgIFJx76MF3v" + } + }, + { + "cell_type": "code", + "source": [ + "features = ['age', 'plan', 'contract_length', 'avg_monthly_calls', 'avg_monthly_data_usage_gb']\n", + "target = 'churned'\n", + "\n", + "X = torch.tensor(df[features].values, dtype=torch.float)\n", + "Y = torch.tensor(df[target], dtype=torch.float)" + ], + "metadata": { + "id": "P-8lKzdzLnGo" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "Define a model that has five input features and predicts a single value." + ], + "metadata": { + "id": "4mcNOez1MQZP" + } + }, + { + "cell_type": "code", + "source": [ + "def build_model(n_inputs, n_outputs):\n", + " \"\"\"build_model builds and returns a model that takes\n", + " `n_inputs` features and predicts `n_outputs` value\"\"\"\n", + " return torch.nn.Sequential(\n", + " torch.nn.Linear(n_inputs, 8),\n", + " torch.nn.ReLU(),\n", + " torch.nn.Linear(8, 16),\n", + " torch.nn.ReLU(),\n", + " torch.nn.Linear(16, n_outputs),\n", + " torch.nn.Sigmoid())" + ], + "metadata": { + "id": "YvdPNlzoMTtl" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "Train the model." + ], + "metadata": { + "id": "GaLBmcvrMOWy" + } + }, + { + "cell_type": "code", + "source": [ + "model = build_model(n_inputs=5, n_outputs=1)\n", + "\n", + "loss_fn = torch.nn.BCELoss()\n", + "optimizer = torch.optim.Adam(model.parameters())\n", + "\n", + "for epoch in range(1000):\n", + " print(f'Epoch {epoch}: ---')\n", + " optimizer.zero_grad()\n", + " for i in range(len(X)):\n", + " pred = model(X[i])\n", + " loss = loss_fn(pred, Y[i].unsqueeze(0))\n", + " loss.backward()\n", + " optimizer.step()" + ], + "metadata": { + "id": "0XqctMiPMaim" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "Save the model to the `STATE_DICT_PATH` variable." + ], + "metadata": { + "id": "m7MD6RwGMdyU" + } + }, + { + "cell_type": "code", + "source": [ + "STATE_DICT_PATH = './model.pth'\n", + "torch.save(model.state_dict(), STATE_DICT_PATH)" + ], + "metadata": { + "id": "Q9WIjw53MgcR" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "### Publish messages to Pub/Sub\n", + "Create the Pub/Sub topic and subscription to use for data streaming." + ], + "metadata": { + "id": "CJVYA0N0MnZS" + } + }, + { + "cell_type": "code", + "source": [ + "# Replace with the name of your Pub/Sub topic.\n", + "TOPIC = \"\"\n", + "\n", + "# Replace with the subscription for your topic.\n", + "SUBSCRIPTION = \"\"" + ], + "metadata": { + "id": "0uwZz_ijyzL8" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "code", + "source": [ + "from google.api_core.exceptions import AlreadyExists\n", + "\n", + "publisher = pubsub_v1.PublisherClient()\n", + "topic_path = publisher.topic_path(PROJECT_ID, TOPIC)\n", + "try:\n", + " topic = publisher.create_topic(request={\"name\": topic_path})\n", + " print(f\"Created topic: {topic.name}\")\n", + "except AlreadyExists:\n", + " print(f\"Topic {topic_path} already exists.\")\n", + "\n", + "subscriber = pubsub_v1.SubscriberClient()\n", + "subscription_path = subscriber.subscription_path(PROJECT_ID, SUBSCRIPTION)\n", + "try:\n", + " subscription = subscriber.create_subscription(\n", + " request={\"name\": subscription_path, \"topic\": topic_path}\n", + " )\n", + " print(f\"Created subscription: {subscription.name}\")\n", + "except AlreadyExists:\n", + " print(f\"Subscription {subscription_path} already exists.\")" + ], + "metadata": { + "id": "hIgsCWIozdDu" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "\n", + "Use the Pub/Sub Python client to publish messages." + ], + "metadata": { + "id": "VqUaFm_yywjU" + } + }, + { + "cell_type": "code", + "source": [ + "messages = [\n", + " {'customer_id': i}\n", + " for i in range(1, 6)\n", + "]\n", + "\n", + "for message in messages:\n", + " data = json.dumps(message).encode('utf-8')\n", + " publish_future = publisher.publish(topic_path, data)" + ], + "metadata": { + "id": "fOq1uNXvMku-" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "## Use the BigQuery enrichment handler\n", + "\n", + "The [`BigQueryEnrichmentHandler`](https://beam.apache.org/releases/pydoc/current/apache_beam.transforms.enrichment_handlers.bigquery.html#apache_beam.transforms.enrichment_handlers.bigquery.BigQueryEnrichmentHandler) is a built-in handler included in the Apache Beam SDK versions 2.57.0 and later.\n", + "\n", + "Configure the `BigQueryEnrichmentHandler` handler with the following parameters.\n", + "\n", + "### Required parameters\n", + "\n", + "The following parameters are required.\n", + "\n", + "* `project` (str): The Google Cloud project ID for the BigQuery table\n", + "\n", + "You must also provide one of the following combinations:\n", + "* `table_name`, `row_restriction_template`, and `fields`\n", + "* `table_name`, `row_restriction_template`, and `condition_value_fn`\n", + "* `query_fn`\n", + "\n", + "### Optional parameters\n", + "\n", + "The following parameters are optional.\n", + "\n", + "* `table_name` (str): The fully qualified BigQuery table name in the format `project.dataset.table`\n", + "* `row_restriction_template` (str): A template string for the `WHERE` clause in the BigQuery query with placeholders (`{}`) to dynamically filter rows based on input data\n", + "* `fields` (Optional[List[str]]): A list of field names present in the input `beam.Row`. These fields names are used to construct the `WHERE` clause if `condition_value_fn` is not provided.\n", + "* `column_names` (Optional[List[str]]): The names of columns to select from the BigQuery table. If not provided, all columns (`*`) are selected.\n", + "* `condition_value_fn` (Optional[Callable[[beam.Row], List[Any]]]): A function that takes a `beam.Row` and returns a list of values to populate in the placeholder `{}` of the `WHERE` clause in the query\n", + "* `query_fn` (Optional[Callable[[beam.Row], str]]): A function that takes a `beam.Row` and returns a complete BigQuery SQL query string\n", + "* `min_batch_size` (int): The minimum number of rows to batch together when querying BigQuery. Defaults to `1` if `query_fn` is not specified.\n", + "* `max_batch_size` (int): The maximum number of rows to batch together. Defaults to `10,000` if `query_fn` is not specified.\n", + "\n", + "### Parameter requirements\n", + "\n", + "When you use parameters, consider the following requirements.\n", + "\n", + "* You can't define the `min_batch_size` and `max_batch_size` parameters if you provide the `query_fn` parameter.\n", + "* You must provide either the `fields` parameter or the `condition_value_fn` parameter for query construction if you don't provide the `query_fn` parameter.\n", + "* You must grant the appropriate permissions to access BigQuery.\n", + "\n", + "### Create handlers\n", + "\n", + "In this example, you create two handlers:\n", + "\n", + "* One for customer data that specifies `table_name` and `row_restriction_template`\n", + "* One for for usage data that uses a custom aggregation query by using the `query_fn` function\n", + "\n", + "These handlers are used in the Enrichment transforms in this pipeline to fetch and join data from BigQuery with the streaming data." + ], + "metadata": { + "id": "giXOGruKM8ZL" + } + }, + { + "cell_type": "code", + "source": [ + "user_data_handler = BigQueryEnrichmentHandler(\n", + " project=PROJECT_ID,\n", + " table_name=f\"`{CUSTOMERS_TABLE_ID}`\",\n", + " row_restriction_template='customer_id = {}',\n", + " fields=['customer_id']\n", + ")\n", + "\n", + "# Define the SQL query for usage data aggregation.\n", + "usage_data_query_template = f\"\"\"\n", + "WITH monthly_aggregates AS (\n", + " SELECT\n", + " customer_id,\n", + " DATE_TRUNC(date, MONTH) as month,\n", + " SUM(calls_made) as total_calls,\n", + " SUM(data_usage_gb) as total_data_usage_gb\n", + " FROM\n", + " `{USAGE_TABLE_ID}`\n", + " WHERE\n", + " customer_id = @customer_id\n", + " GROUP BY\n", + " customer_id, month\n", + ")\n", + "SELECT\n", + " customer_id,\n", + " AVG(total_calls) as avg_monthly_calls,\n", + " AVG(total_data_usage_gb) as avg_monthly_data_usage_gb\n", + "FROM\n", + " monthly_aggregates\n", + "GROUP BY\n", + " customer_id\n", + "\"\"\"\n", + "\n", + "def usage_data_query_fn(row: beam.Row) -> str:\n", + " return usage_data_query_template.replace('@customer_id', str(row.customer_id))\n", + "\n", + "usage_data_handler = BigQueryEnrichmentHandler(\n", + " project=PROJECT_ID,\n", + " query_fn=usage_data_query_fn\n", + ")" + ], + "metadata": { + "id": "C8XLmBDeMyrB" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "In this example:\n", + "1. The `user_data_handler` handler uses the `table_name`, `row_restriction_template`, and `fields` parameter combination to fetch customer data.\n", + "2. The `usage_data_handler` handler uses the `query_fn` parameter to execute a more complex query that aggregates usage data." + ], + "metadata": { + "id": "3oPYypvmPiyg" + } + }, + { + "cell_type": "markdown", + "source": [ + "## Use the `PytorchModelHandlerTensor` interface to run inference\n", + "\n", + "Define functions to convert enriched data to the tensor format for the model." + ], + "metadata": { + "id": "ksON9uOBQbZm" + } + }, + { + "cell_type": "code", + "source": [ + "def convert_row_to_tensor(customer_data):\n", + " import pandas as pd\n", + " customer_df = pd.DataFrame([customer_data[1].as_dict()])\n", + " customer_df['plan'] = plan_encoder.transform(customer_df['plan'])\n", + " return (customer_data[0], torch.tensor(customer_df[features].values, dtype=torch.float))\n", + "\n", + "keyed_model_handler = KeyedModelHandler(PytorchModelHandlerTensor(\n", + " state_dict_path=STATE_DICT_PATH,\n", + " model_class=build_model,\n", + " model_params={'n_inputs':5, 'n_outputs':1}\n", + ")).with_preprocess_fn(convert_row_to_tensor)" + ], + "metadata": { + "id": "XgPontIVP0Cv" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "Define a `DoFn` to format the output." + ], + "metadata": { + "id": "O9e7ddgGQxh2" + } + }, + { + "cell_type": "code", + "source": [ + "class PostProcessor(beam.DoFn):\n", + " def process(self, element, *args, **kwargs):\n", + " print('Customer %d churn risk: %s' % (element[0], \"High\" if element[1].inference[0].item() > 0.5 else \"Low\"))" + ], + "metadata": { + "id": "NMj0V5VyQukk" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "## Run the pipeline\n", + "\n", + "Configure the pipeline to run in streaming mode." + ], + "metadata": { + "id": "-N3a1s2FQ66z" + } + }, + { + "cell_type": "code", + "source": [ + "options = pipeline_options.PipelineOptions()\n", + "options.view_as(pipeline_options.StandardOptions).streaming = True # Streaming mode is set True" + ], + "metadata": { + "id": "rgJeV-jWQ4wo" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "Pub/Sub sends the data in bytes. Convert the data to `beam.Row` objects by using a `DoFn`." + ], + "metadata": { + "id": "NRljYVR5RCMi" + } + }, + { + "cell_type": "code", + "source": [ + "class DecodeBytes(beam.DoFn):\n", + " \"\"\"\n", + " The DecodeBytes `DoFn` converts the data read from Pub/Sub to `beam.Row`.\n", + " First, decode the encoded string. Convert the output to\n", + " a `dict` with `json.loads()`, which is used to create a `beam.Row`.\n", + " \"\"\"\n", + " def process(self, element, *args, **kwargs):\n", + " element_dict = json.loads(element.decode('utf-8'))\n", + " yield beam.Row(**element_dict)" + ], + "metadata": { + "id": "Bb-e3yjtQ2iU" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "Use the following code to run the pipeline.\n", + "\n", + "**Note:** Because this pipeline is a streaming pipeline, you need to manually stop the cell. If you don't stop the cell, the pipeline continues to run." + ], + "metadata": { + "id": "Q1HV8wH-RIbj" + } + }, + { + "cell_type": "code", + "source": [ + "with beam.Pipeline(options=options) as p:\n", + " _ = (p\n", + " | \"Read from Pub/Sub\" >> beam.io.ReadFromPubSub(subscription=f\"projects/{PROJECT_ID}/subscriptions/{SUBSCRIPTION}\")\n", + " | \"ConvertToRow\" >> beam.ParDo(DecodeBytes())\n", + " | \"Enrich with customer data\" >> Enrichment(user_data_handler)\n", + " | \"Enrich with usage data\" >> Enrichment(usage_data_handler)\n", + " | \"Key data\" >> beam.Map(lambda x: (x.customer_id, x))\n", + " | \"RunInference\" >> RunInference(keyed_model_handler)\n", + " | \"Format Output\" >> beam.ParDo(PostProcessor())\n", + " )" + ], + "metadata": { + "id": "y6HBH8yoRFp2" + }, + "execution_count": null, + "outputs": [] + } + ] +} \ No newline at end of file From 06692ca1ed32f3815379d561fc61e120f01eaa4e Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 15 Oct 2024 15:15:02 -0400 Subject: [PATCH 51/78] Revert disabled Gradle cache in #32751 (#32771) --- sdks/java/expansion-service/build.gradle | 4 ---- sdks/java/extensions/sql/expansion-service/build.gradle | 4 ---- sdks/java/io/expansion-service/build.gradle | 1 - sdks/python/apache_beam/yaml/yaml_provider.py | 2 +- 4 files changed, 1 insertion(+), 10 deletions(-) diff --git a/sdks/java/expansion-service/build.gradle b/sdks/java/expansion-service/build.gradle index a25583870acf..4dd8c8968ed9 100644 --- a/sdks/java/expansion-service/build.gradle +++ b/sdks/java/expansion-service/build.gradle @@ -57,7 +57,3 @@ task runExpansionService (type: JavaExec) { classpath = sourceSets.main.runtimeClasspath args = [project.findProperty("constructionService.port") ?: "8097"] } - -compileJava { - outputs.upToDateWhen { false } -} \ No newline at end of file diff --git a/sdks/java/extensions/sql/expansion-service/build.gradle b/sdks/java/extensions/sql/expansion-service/build.gradle index b8d78e4e1bb9..b6963cf7547b 100644 --- a/sdks/java/extensions/sql/expansion-service/build.gradle +++ b/sdks/java/extensions/sql/expansion-service/build.gradle @@ -46,7 +46,3 @@ task runExpansionService (type: JavaExec) { classpath = sourceSets.main.runtimeClasspath args = [project.findProperty("constructionService.port") ?: "8097"] } - -shadowJar { - outputs.upToDateWhen { false } -} \ No newline at end of file diff --git a/sdks/java/io/expansion-service/build.gradle b/sdks/java/io/expansion-service/build.gradle index cc8eccf98997..8b817163ae39 100644 --- a/sdks/java/io/expansion-service/build.gradle +++ b/sdks/java/io/expansion-service/build.gradle @@ -35,7 +35,6 @@ configurations.runtimeClasspath { shadowJar { mergeServiceFiles() - outputs.upToDateWhen { false } } description = "Apache Beam :: SDKs :: Java :: IO :: Expansion Service" diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index c2cba936abce..ef2316f51f0e 100755 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -117,7 +117,7 @@ def affinity(self, other: "Provider"): (e.g. to encourage fusion). """ # TODO(yaml): This is a very rough heuristic. Consider doing better. - # E.g. we could look at the the expected environments themselves. + # E.g. we could look at the expected environments themselves. # Possibly, we could provide multiple expansions and have the runner itself # choose the actual implementation based on fusion (and other) criteria. a = self.underlying_provider() From e52868c29f9d6cae3c91aedff9814f90de241b36 Mon Sep 17 00:00:00 2001 From: pablo rodriguez defino Date: Tue, 15 Oct 2024 12:28:22 -0700 Subject: [PATCH 52/78] Enable BigQuery CDC configuration for Python BigQuery sink (#32529) * include CDC configuration on the storage write transform provider * adding the primary key configuration for CDC and tests * fixing List.of references to use ImmutableList * fixing test, missing calling the cdc info row builder() method * fix test, add config validations * added the xlang params to storage write python wrapper * adding missing comma * shortening property name * changing xlang config property * set use cdc schema property as nullable, added safe retrieval method * fixes property name reference and argument type definition * python format fix * adding xlang IT with BQ * adding missing primary key column to test * python format fix * format xlang test * more format xlang test fixes * and more format xlang test fixes * adding missing import * missing self reference * enabled create if needed functionality for CDC python integration, implemented table constraint support on the bigquery fake dataset services * Update bigquery.py * triggering the xlang tests * fixing lint * addressing few comments * cdc info is added after row transformation now * remove not used param * removed typing information for callable * adding test for cdc using dicts as input and cdc write callable * simplifying the xlang configuration from python perspective, will add callable on a future PR * spotless apply * wrong property passed to xlang builder * missing self * fixing xlang it * fixes wrong property reference * change cdc xlang test to use beam.io.WriteToBigQuery * force another build * modifying comment to trigger build. * addressing PR comments, included new dicts based test for xlang python tests, included the CDC configurations into the existing RowDynamicDestinations object, improved error message for mutation information schema checks. --- ..._PostCommit_Python_Xlang_Gcp_Dataflow.json | 3 +- ...am_PostCommit_Python_Xlang_Gcp_Direct.json | 2 +- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 8 +- ...torageWriteApiSchemaTransformProvider.java | 122 +++++++++++++++++- .../sdk/io/gcp/testing/TableContainer.java | 32 ++++- ...geWriteApiSchemaTransformProviderTest.java | 115 ++++++++++++++++- .../io/external/xlang_bigqueryio_it_test.py | 122 ++++++++++++++++++ sdks/python/apache_beam/io/gcp/bigquery.py | 30 ++++- 8 files changed, 414 insertions(+), 20 deletions(-) diff --git a/.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Dataflow.json b/.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Dataflow.json index 6b3a9dc134ee..27c1f3ae26cd 100644 --- a/.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Dataflow.json +++ b/.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Dataflow.json @@ -1,4 +1,5 @@ { - "comment": "Modify this file in a trivial way to cause this test suite to run" + "comment": "Modify this file in a trivial way to cause this test suite to run", + "modification": 1 } diff --git a/.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Direct.json b/.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Direct.json index e3d6056a5de9..b26833333238 100644 --- a/.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Direct.json +++ b/.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Direct.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 1 + "modification": 2 } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index 88dfa2c26348..84bf90bd4121 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -2259,6 +2259,7 @@ public static Write applyRowMutations() { .withFormatFunction(RowMutation::getTableRow) .withRowMutationInformationFn(RowMutation::getMutationInformation); } + /** * A {@link PTransform} that writes a {@link PCollection} containing {@link GenericRecord * GenericRecords} to a BigQuery table. @@ -2367,8 +2368,10 @@ public enum Method { abstract WriteDisposition getWriteDisposition(); abstract Set getSchemaUpdateOptions(); + /** Table description. Default is empty. */ abstract @Nullable String getTableDescription(); + /** An option to indicate if table validation is desired. Default is true. */ abstract boolean getValidate(); @@ -3455,7 +3458,10 @@ && getStorageApiTriggeringFrequency(bqOptions) != null) { LOG.error("The Storage API sink does not support the WRITE_TRUNCATE write disposition."); } if (getRowMutationInformationFn() != null) { - checkArgument(getMethod() == Method.STORAGE_API_AT_LEAST_ONCE); + checkArgument( + getMethod() == Method.STORAGE_API_AT_LEAST_ONCE, + "When using row updates on BigQuery, StorageWrite API should execute using" + + " \"at least once\" mode."); checkArgument( getCreateDisposition() == CreateDisposition.CREATE_NEVER || getPrimaryKey() != null, "If specifying CREATE_IF_NEEDED along with row updates, a primary key needs to be specified"); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProvider.java index 980d783ec43c..c1c06fc592f4 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProvider.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProvider.java @@ -20,6 +20,7 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; +import com.google.api.services.bigquery.model.TableConstraints; import com.google.api.services.bigquery.model.TableSchema; import com.google.auto.service.AutoService; import com.google.auto.value.AutoValue; @@ -27,6 +28,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Optional; import javax.annotation.Nullable; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; @@ -37,6 +39,7 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryStorageApiInsertError; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils; import org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinations; +import org.apache.beam.sdk.io.gcp.bigquery.RowMutationInformation; import org.apache.beam.sdk.io.gcp.bigquery.TableDestination; import org.apache.beam.sdk.io.gcp.bigquery.WriteResult; import org.apache.beam.sdk.io.gcp.bigquery.providers.BigQueryStorageWriteApiSchemaTransformProvider.BigQueryStorageWriteApiSchemaTransformConfiguration; @@ -87,6 +90,14 @@ public class BigQueryStorageWriteApiSchemaTransformProvider private static final String FAILED_ROWS_WITH_ERRORS_TAG = "FailedRowsWithErrors"; // magic string that tells us to write to dynamic destinations protected static final String DYNAMIC_DESTINATIONS = "DYNAMIC_DESTINATIONS"; + protected static final String ROW_PROPERTY_MUTATION_INFO = "row_mutation_info"; + protected static final String ROW_PROPERTY_MUTATION_TYPE = "mutation_type"; + protected static final String ROW_PROPERTY_MUTATION_SQN = "change_sequence_number"; + protected static final Schema ROW_SCHEMA_MUTATION_INFO = + Schema.builder() + .addStringField("mutation_type") + .addStringField("change_sequence_number") + .build(); @Override protected SchemaTransform from( @@ -257,6 +268,20 @@ public static Builder builder() { @Nullable public abstract ErrorHandling getErrorHandling(); + @SchemaFieldDescription( + "This option enables the use of BigQuery CDC functionality. The expected PCollection" + + " should contain Beam Rows with a schema wrapping the record to be inserted and" + + " adding the CDC info similar to: {row_mutation_info: {mutation_type:\"...\", " + + "change_sequence_number:\"...\"}, record: {...}}") + @Nullable + public abstract Boolean getUseCdcWrites(); + + @SchemaFieldDescription( + "If CREATE_IF_NEEDED disposition is set, BigQuery table(s) will be created with this" + + " columns as primary key. Required when CDC writes are enabled with CREATE_IF_NEEDED.") + @Nullable + public abstract List getPrimaryKey(); + /** Builder for {@link BigQueryStorageWriteApiSchemaTransformConfiguration}. */ @AutoValue.Builder public abstract static class Builder { @@ -277,6 +302,10 @@ public abstract static class Builder { public abstract Builder setErrorHandling(ErrorHandling errorHandling); + public abstract Builder setUseCdcWrites(Boolean cdcWrites); + + public abstract Builder setPrimaryKey(List pkColumns); + /** Builds a {@link BigQueryStorageWriteApiSchemaTransformConfiguration} instance. */ public abstract BigQueryStorageWriteApiSchemaTransformProvider .BigQueryStorageWriteApiSchemaTransformConfiguration @@ -343,15 +372,27 @@ public void process(ProcessContext c) {} } private static class RowDynamicDestinations extends DynamicDestinations { - Schema schema; + final Schema schema; + final String fixedDestination; + final List primaryKey; RowDynamicDestinations(Schema schema) { this.schema = schema; + this.fixedDestination = null; + this.primaryKey = null; + } + + public RowDynamicDestinations( + Schema schema, String fixedDestination, List primaryKey) { + this.schema = schema; + this.fixedDestination = fixedDestination; + this.primaryKey = primaryKey; } @Override public String getDestination(ValueInSingleWindow element) { - return element.getValue().getString("destination"); + return Optional.ofNullable(fixedDestination) + .orElseGet(() -> element.getValue().getString("destination")); } @Override @@ -363,6 +404,17 @@ public TableDestination getTable(String destination) { public TableSchema getSchema(String destination) { return BigQueryUtils.toTableSchema(schema); } + + @Override + public TableConstraints getTableConstraints(String destination) { + return Optional.ofNullable(this.primaryKey) + .filter(pk -> !pk.isEmpty()) + .map( + pk -> + new TableConstraints() + .setPrimaryKey(new TableConstraints.PrimaryKey().setColumns(pk))) + .orElse(null); + } } @Override @@ -453,6 +505,13 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { } } + void validateDynamicDestinationsExpectedSchema(Schema schema) { + checkArgument( + schema.getFieldNames().containsAll(Arrays.asList("destination", "record")), + "When writing to dynamic destinations, we expect Row Schema with a " + + "\"destination\" string field and a \"record\" Row field."); + } + BigQueryIO.Write createStorageWriteApiTransform(Schema schema) { Method writeMethod = configuration.getUseAtLeastOnceSemantics() != null @@ -466,11 +525,11 @@ BigQueryIO.Write createStorageWriteApiTransform(Schema schema) { .withFormatFunction(BigQueryUtils.toTableRow()) .withWriteDisposition(WriteDisposition.WRITE_APPEND); - if (configuration.getTable().equals(DYNAMIC_DESTINATIONS)) { - checkArgument( - schema.getFieldNames().equals(Arrays.asList("destination", "record")), - "When writing to dynamic destinations, we expect Row Schema with a " - + "\"destination\" string field and a \"record\" Row field."); + // in case CDC writes are configured we validate and include them in the configuration + if (Optional.ofNullable(configuration.getUseCdcWrites()).orElse(false)) { + write = validateAndIncludeCDCInformation(write, schema); + } else if (configuration.getTable().equals(DYNAMIC_DESTINATIONS)) { + validateDynamicDestinationsExpectedSchema(schema); write = write .to(new RowDynamicDestinations(schema.getField("record").getType().getRowSchema())) @@ -485,6 +544,7 @@ BigQueryIO.Write createStorageWriteApiTransform(Schema schema) { configuration.getCreateDisposition().toUpperCase()); write = write.withCreateDisposition(createDisposition); } + if (!Strings.isNullOrEmpty(configuration.getWriteDisposition())) { WriteDisposition writeDisposition = BigQueryStorageWriteApiSchemaTransformConfiguration.WRITE_DISPOSITIONS.get( @@ -498,5 +558,53 @@ BigQueryIO.Write createStorageWriteApiTransform(Schema schema) { return write; } + + BigQueryIO.Write validateAndIncludeCDCInformation( + BigQueryIO.Write write, Schema schema) { + checkArgument( + schema.getFieldNames().containsAll(Arrays.asList(ROW_PROPERTY_MUTATION_INFO, "record")), + "When writing using CDC functionality, we expect Row Schema with a " + + "\"" + + ROW_PROPERTY_MUTATION_INFO + + "\" Row field and a \"record\" Row field."); + + Schema rowSchema = schema.getField(ROW_PROPERTY_MUTATION_INFO).getType().getRowSchema(); + + checkArgument( + rowSchema.equals(ROW_SCHEMA_MUTATION_INFO), + "When writing using CDC functionality, we expect a \"" + + ROW_PROPERTY_MUTATION_INFO + + "\" field of Row type with schema:\n" + + ROW_SCHEMA_MUTATION_INFO.toString() + + "\n" + + "Received \"" + + ROW_PROPERTY_MUTATION_INFO + + "\" field with schema:\n" + + rowSchema.toString()); + + String tableDestination = null; + + if (configuration.getTable().equals(DYNAMIC_DESTINATIONS)) { + validateDynamicDestinationsExpectedSchema(schema); + } else { + tableDestination = configuration.getTable(); + } + + return write + .to( + new RowDynamicDestinations( + schema.getField("record").getType().getRowSchema(), + tableDestination, + configuration.getPrimaryKey())) + .withFormatFunction(row -> BigQueryUtils.toTableRow(row.getRow("record"))) + .withPrimaryKey(configuration.getPrimaryKey()) + .withRowMutationInformationFn( + row -> + RowMutationInformation.of( + RowMutationInformation.MutationType.valueOf( + row.getRow(ROW_PROPERTY_MUTATION_INFO) + .getString(ROW_PROPERTY_MUTATION_TYPE)), + row.getRow(ROW_PROPERTY_MUTATION_INFO).getString(ROW_PROPERTY_MUTATION_SQN))); + } } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/TableContainer.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/TableContainer.java index b50aa4d32d76..b44b9596cc12 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/TableContainer.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/TableContainer.java @@ -18,11 +18,13 @@ package org.apache.beam.sdk.io.gcp.testing; import com.google.api.services.bigquery.model.Table; +import com.google.api.services.bigquery.model.TableConstraints; import com.google.api.services.bigquery.model.TableRow; import java.util.AbstractMap; import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.stream.Collectors; import java.util.stream.IntStream; import org.apache.beam.sdk.io.gcp.bigquery.TableRowJsonCoder; @@ -51,12 +53,24 @@ class TableContainer { this.keyedRows = Maps.newHashMap(); this.ids = new ArrayList<>(); this.sizeBytes = 0L; + // extract primary key information from Table if present + List pkColumns = primaryKeyColumns(table); + this.primaryKeyColumns = pkColumns; + this.primaryKeyColumnIndices = primaryColumnFieldIndices(pkColumns, table); } - // Only top-level columns supported. - void setPrimaryKeyColumns(List primaryKeyColumns) { - this.primaryKeyColumns = primaryKeyColumns; + static @Nullable List primaryKeyColumns(Table table) { + return Optional.ofNullable(table.getTableConstraints()) + .flatMap(constraints -> Optional.ofNullable(constraints.getPrimaryKey())) + .map(TableConstraints.PrimaryKey::getColumns) + .orElse(null); + } + static @Nullable List primaryColumnFieldIndices( + @Nullable List primaryKeyColumns, Table table) { + if (primaryKeyColumns == null) { + return null; + } Map indices = IntStream.range(0, table.getSchema().getFields().size()) .boxed() @@ -65,7 +79,13 @@ void setPrimaryKeyColumns(List primaryKeyColumns) { for (String columnName : primaryKeyColumns) { primaryKeyColumnIndices.add(Preconditions.checkStateNotNull(indices.get(columnName))); } - this.primaryKeyColumnIndices = primaryKeyColumnIndices; + return primaryKeyColumnIndices; + } + + // Only top-level columns supported. + void setPrimaryKeyColumns(List primaryKeyColumns) { + this.primaryKeyColumns = primaryKeyColumns; + this.primaryKeyColumnIndices = primaryColumnFieldIndices(primaryKeyColumns, table); } @Nullable @@ -80,7 +100,7 @@ List getPrimaryKey(TableRow tableRow) { .stream() .map(cell -> Preconditions.checkStateNotNull(cell.get("v"))) .collect(Collectors.toList()); - ; + return Preconditions.checkStateNotNull(primaryKeyColumnIndices).stream() .map(cellValues::get) .collect(Collectors.toList()); @@ -91,7 +111,7 @@ List getPrimaryKey(TableRow tableRow) { long addRow(TableRow row, String id) { List primaryKey = getPrimaryKey(row); - if (primaryKey != null) { + if (primaryKey != null && !primaryKey.isEmpty()) { if (keyedRows.putIfAbsent(primaryKey, row) != null) { throw new RuntimeException( "Primary key validation error! Multiple inserts with the same primary key."); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProviderTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProviderTest.java index 64ea0b11d1b9..87ba2961461a 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProviderTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProviderTest.java @@ -30,6 +30,8 @@ import java.util.List; import java.util.function.Function; import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers; import org.apache.beam.sdk.io.gcp.bigquery.providers.BigQueryStorageWriteApiSchemaTransformProvider.BigQueryStorageWriteApiSchemaTransform; @@ -54,6 +56,7 @@ import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -221,6 +224,117 @@ public void testWriteToDynamicDestinations() throws Exception { fakeDatasetService.getAllRows("project", "dataset", "dynamic_write_3").get(0))); } + List createCDCUpsertRows(List rows, boolean dynamicDestination, String tablePrefix) { + + Schema.Builder schemaBuilder = + Schema.builder() + .addRowField("record", SCHEMA) + .addRowField( + BigQueryStorageWriteApiSchemaTransformProvider.ROW_PROPERTY_MUTATION_INFO, + BigQueryStorageWriteApiSchemaTransformProvider.ROW_SCHEMA_MUTATION_INFO); + + if (dynamicDestination) { + schemaBuilder = schemaBuilder.addStringField("destination"); + } + + Schema schemaWithCDC = schemaBuilder.build(); + return IntStream.range(0, rows.size()) + .mapToObj( + idx -> { + Row row = rows.get(idx); + Row.FieldValueBuilder rowBuilder = + Row.withSchema(schemaWithCDC) + .withFieldValue( + BigQueryStorageWriteApiSchemaTransformProvider.ROW_PROPERTY_MUTATION_INFO, + Row.withSchema( + BigQueryStorageWriteApiSchemaTransformProvider + .ROW_SCHEMA_MUTATION_INFO) + .withFieldValue( + BigQueryStorageWriteApiSchemaTransformProvider + .ROW_PROPERTY_MUTATION_TYPE, + "UPSERT") + .withFieldValue( + BigQueryStorageWriteApiSchemaTransformProvider + .ROW_PROPERTY_MUTATION_SQN, + "AAA" + idx) + .build()) + .withFieldValue("record", row); + if (dynamicDestination) { + rowBuilder = + rowBuilder.withFieldValue("destination", tablePrefix + row.getInt64("number")); + } + return rowBuilder.build(); + }) + .collect(Collectors.toList()); + } + + @Test + public void testCDCWrites() throws Exception { + String tableSpec = "project:dataset.cdc_write"; + List primaryKeyColumns = ImmutableList.of("name"); + + BigQueryStorageWriteApiSchemaTransformConfiguration config = + BigQueryStorageWriteApiSchemaTransformConfiguration.builder() + .setUseAtLeastOnceSemantics(true) + .setTable(tableSpec) + .setUseCdcWrites(true) + .setPrimaryKey(primaryKeyColumns) + .build(); + + List rowsDuplicated = + Stream.concat(ROWS.stream(), ROWS.stream()).collect(Collectors.toList()); + + runWithConfig(config, createCDCUpsertRows(rowsDuplicated, false, "")); + p.run().waitUntilFinish(); + + assertTrue( + rowEquals( + rowsDuplicated.get(3), + fakeDatasetService.getAllRows("project", "dataset", "cdc_write").get(0))); + assertTrue( + rowEquals( + rowsDuplicated.get(4), + fakeDatasetService.getAllRows("project", "dataset", "cdc_write").get(1))); + assertTrue( + rowEquals( + rowsDuplicated.get(5), + fakeDatasetService.getAllRows("project", "dataset", "cdc_write").get(2))); + } + + @Test + public void testCDCWriteToDynamicDestinations() throws Exception { + List primaryKeyColumns = ImmutableList.of("name"); + String dynamic = BigQueryStorageWriteApiSchemaTransformProvider.DYNAMIC_DESTINATIONS; + BigQueryStorageWriteApiSchemaTransformConfiguration config = + BigQueryStorageWriteApiSchemaTransformConfiguration.builder() + .setUseAtLeastOnceSemantics(true) + .setTable(dynamic) + .setUseCdcWrites(true) + .setPrimaryKey(primaryKeyColumns) + .build(); + + String baseTableSpec = "project:dataset.cdc_dynamic_write_"; + + List rowsDuplicated = + Stream.concat(ROWS.stream(), ROWS.stream()).collect(Collectors.toList()); + + runWithConfig(config, createCDCUpsertRows(rowsDuplicated, true, baseTableSpec)); + p.run().waitUntilFinish(); + + assertTrue( + rowEquals( + rowsDuplicated.get(3), + fakeDatasetService.getAllRows("project", "dataset", "cdc_dynamic_write_1").get(0))); + assertTrue( + rowEquals( + rowsDuplicated.get(4), + fakeDatasetService.getAllRows("project", "dataset", "cdc_dynamic_write_2").get(0))); + assertTrue( + rowEquals( + rowsDuplicated.get(5), + fakeDatasetService.getAllRows("project", "dataset", "cdc_dynamic_write_3").get(0))); + } + @Test public void testInputElementCount() throws Exception { String tableSpec = "project:dataset.input_count"; @@ -292,7 +406,6 @@ public void testFailedRows() throws Exception { MapElements.into(TypeDescriptors.rows()) .via((rowAndError) -> rowAndError.getValue("failed_row"))) .setRowSchema(SCHEMA); - ; PAssert.that(failedRows).containsInAnyOrder(expectedFailedRows); p.run().waitUntilFinish(); diff --git a/sdks/python/apache_beam/io/external/xlang_bigqueryio_it_test.py b/sdks/python/apache_beam/io/external/xlang_bigqueryio_it_test.py index cfbb411b4e5f..7f3a16e02aa3 100644 --- a/sdks/python/apache_beam/io/external/xlang_bigqueryio_it_test.py +++ b/sdks/python/apache_beam/io/external/xlang_bigqueryio_it_test.py @@ -245,6 +245,128 @@ def test_write_with_beam_rows(self): | StorageWriteToBigQuery(table=table_id)) hamcrest_assert(p, bq_matcher) + def test_write_with_beam_rows_cdc(self): + table = 'write_with_beam_rows_cdc' + table_id = '{}:{}.{}'.format(self.project, self.dataset_id, table) + + expected_data_on_bq = [ + # (name, value) + { + "name": "cdc_test", + "value": 5, + } + ] + + rows_with_cdc = [ + beam.Row( + row_mutation_info=beam.Row( + mutation_type="UPSERT", change_sequence_number="AAA/2"), + record=beam.Row(name="cdc_test", value=5)), + beam.Row( + row_mutation_info=beam.Row( + mutation_type="UPSERT", change_sequence_number="AAA/1"), + record=beam.Row(name="cdc_test", value=3)) + ] + + bq_matcher = BigqueryFullResultMatcher( + project=self.project, + query="SELECT * FROM {}.{}".format(self.dataset_id, table), + data=self.parse_expected_data(expected_data_on_bq)) + + with beam.Pipeline(argv=self.args) as p: + _ = ( + p + | beam.Create(rows_with_cdc) + | beam.io.WriteToBigQuery( + table=table_id, + method=beam.io.WriteToBigQuery.Method.STORAGE_WRITE_API, + use_at_least_once=True, + use_cdc_writes=True, + primary_key=["name"])) + hamcrest_assert(p, bq_matcher) + + def test_write_with_dicts_cdc(self): + table = 'write_with_dicts_cdc' + table_id = '{}:{}.{}'.format(self.project, self.dataset_id, table) + + expected_data_on_bq = [ + # (name, value) + { + "name": "cdc_test", + "value": 5, + } + ] + + data_with_cdc = [ + # record: (name, value) + { + 'row_mutation_info': { + 'mutation_type': 'UPSERT', 'change_sequence_number': 'AAA/2' + }, + 'record': { + 'name': 'cdc_test', 'value': 5 + } + }, + { + 'row_mutation_info': { + 'mutation_type': 'UPSERT', 'change_sequence_number': 'AAA/1' + }, + 'record': { + 'name': 'cdc_test', 'value': 3 + } + } + ] + + schema = { + "fields": [ + # include both record and mutation info fields as part of the schema + { + "name": "row_mutation_info", + "type": "STRUCT", + "fields": [ + # setting both fields are required + { + "name": "mutation_type", + "type": "STRING", + "mode": "REQUIRED" + }, + { + "name": "change_sequence_number", + "type": "STRING", + "mode": "REQUIRED" + } + ] + }, + { + "name": "record", + "type": "STRUCT", + "fields": [{ + "name": "name", "type": "STRING" + }, { + "name": "value", "type": "INTEGER" + }] + } + ] + } + + bq_matcher = BigqueryFullResultMatcher( + project=self.project, + query="SELECT * FROM {}.{}".format(self.dataset_id, table), + data=self.parse_expected_data(expected_data_on_bq)) + + with beam.Pipeline(argv=self.args) as p: + _ = ( + p + | beam.Create(data_with_cdc) + | beam.io.WriteToBigQuery( + table=table_id, + method=beam.io.WriteToBigQuery.Method.STORAGE_WRITE_API, + use_at_least_once=True, + use_cdc_writes=True, + schema=schema, + primary_key=["name"])) + hamcrest_assert(p, bq_matcher) + def test_write_to_dynamic_destinations(self): base_table_spec = '{}.dynamic_dest_'.format(self.dataset_id) spec_with_project = '{}:{}'.format(self.project, base_table_spec) diff --git a/sdks/python/apache_beam/io/gcp/bigquery.py b/sdks/python/apache_beam/io/gcp/bigquery.py index 2cb64742f26c..11e0d098b2f3 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery.py +++ b/sdks/python/apache_beam/io/gcp/bigquery.py @@ -1930,6 +1930,8 @@ def __init__( load_job_project_id=None, max_insert_payload_size=MAX_INSERT_PAYLOAD_SIZE, num_streaming_keys=DEFAULT_SHARDS_PER_DESTINATION, + use_cdc_writes: bool = False, + primary_key: List[str] = None, expansion_service=None): """Initialize a WriteToBigQuery transform. @@ -2095,6 +2097,15 @@ def __init__( GCP expansion service. Used for STORAGE_WRITE_API method. max_insert_payload_size: The maximum byte size for a BigQuery legacy streaming insert payload. + use_cdc_writes: Configure the usage of CDC writes on BigQuery. + The argument can be used by passing True and the Beam Rows will be + sent as they are to the BigQuery sink which expects a 'record' + and 'row_mutation_info' properties. + Used for STORAGE_WRITE_API, working on 'at least once' mode. + primary_key: When using CDC write on BigQuery and + CREATE_IF_NEEDED mode for the underlying tables a list of column names + is required to be configured as the primary key. Used for + STORAGE_WRITE_API, working on 'at least once' mode. """ self._table = table self._dataset = dataset @@ -2136,6 +2147,8 @@ def __init__( self.load_job_project_id = load_job_project_id self._max_insert_payload_size = max_insert_payload_size self._num_streaming_keys = num_streaming_keys + self._use_cdc_writes = use_cdc_writes + self._primary_key = primary_key # Dict/schema methods were moved to bigquery_tools, but keep references # here for backward compatibility. @@ -2289,8 +2302,9 @@ def find_in_nested_dict(schema): use_at_least_once=self.use_at_least_once, with_auto_sharding=self.with_auto_sharding, num_storage_api_streams=self._num_storage_api_streams, + use_cdc_writes=self._use_cdc_writes, + primary_key=self._primary_key, expansion_service=self.expansion_service) - else: raise ValueError(f"Unsupported method {method_to_use}") @@ -2518,6 +2532,10 @@ class StorageWriteToBigQuery(PTransform): # fields for rows sent to Storage API with dynamic destinations DESTINATION = "destination" RECORD = "record" + # field names for rows sent to Storage API for CDC functionality + CDC_INFO = "row_mutation_info" + CDC_MUTATION_TYPE = "mutation_type" + CDC_SQN = "change_sequence_number" # magic string to tell Java that these rows are going to dynamic destinations DYNAMIC_DESTINATIONS = "DYNAMIC_DESTINATIONS" @@ -2532,6 +2550,8 @@ def __init__( use_at_least_once=False, with_auto_sharding=False, num_storage_api_streams=0, + use_cdc_writes: bool = False, + primary_key: List[str] = None, expansion_service=None): self._table = table self._table_side_inputs = table_side_inputs @@ -2542,6 +2562,8 @@ def __init__( self._use_at_least_once = use_at_least_once self._with_auto_sharding = with_auto_sharding self._num_storage_api_streams = num_storage_api_streams + self._use_cdc_writes = use_cdc_writes + self._primary_key = primary_key self._expansion_service = expansion_service or BeamJarExpansionService( 'sdks:java:io:google-cloud-platform:expansion-service:build') @@ -2552,11 +2574,11 @@ def expand(self, input): is_rows = True except TypeError as exn: raise ValueError( - "A schema is required in order to prepare rows" + "A schema is required in order to prepare rows " "for writing with STORAGE_WRITE_API.") from exn elif callable(self._schema): raise NotImplementedError( - "Writing with dynamic schemas is not" + "Writing with dynamic schemas is not " "supported for this write method.") elif isinstance(self._schema, vp.ValueProvider): schema = self._schema.get() @@ -2624,6 +2646,8 @@ def expand(self, input): auto_sharding=self._with_auto_sharding, num_streams=self._num_storage_api_streams, use_at_least_once_semantics=self._use_at_least_once, + use_cdc_writes=self._use_cdc_writes, + primary_key=self._primary_key, error_handling={ 'output': StorageWriteToBigQuery.FAILED_ROWS_WITH_ERRORS })) From e39e5d724c946901ba7065442781bd3457b4c4de Mon Sep 17 00:00:00 2001 From: pablo rodriguez defino Date: Tue, 15 Oct 2024 12:29:07 -0700 Subject: [PATCH 53/78] Update CHANGES.md (#32788) --- CHANGES.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGES.md b/CHANGES.md index 4e21e400e60d..f2b865cec236 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -67,6 +67,7 @@ * [Managed Iceberg] Support creating tables if needed ([#32686](https://github.com/apache/beam/pull/32686)) * [Managed Iceberg] Now available in Python SDK ([#31495](https://github.com/apache/beam/pull/31495)) * [Managed Iceberg] Add support for TIMESTAMP, TIME, and DATE types ([#32688](https://github.com/apache/beam/pull/32688)) +* BigQuery CDC writes are now available in Python SDK, only supported when using StorageWrite API at least once mode ([#32527](https://github.com/apache/beam/issues/32527)) ## New Features / Improvements From 0feaaa6c96b223fab520585786600104ab362a62 Mon Sep 17 00:00:00 2001 From: liferoad Date: Tue, 15 Oct 2024 16:43:09 -0400 Subject: [PATCH 54/78] Add the beam summit 2024 overview blog --- .../en/blog/beam-summit-2024-overview.md | 59 +++++++++++++++++++ 1 file changed, 59 insertions(+) create mode 100644 website/www/site/content/en/blog/beam-summit-2024-overview.md diff --git a/website/www/site/content/en/blog/beam-summit-2024-overview.md b/website/www/site/content/en/blog/beam-summit-2024-overview.md new file mode 100644 index 000000000000..8d5eb209bea2 --- /dev/null +++ b/website/www/site/content/en/blog/beam-summit-2024-overview.md @@ -0,0 +1,59 @@ +--- +title: "Apache Beam Summit 2024: Unlocking the power of ML for data processing" +date: 2024-10-16 00:00:01 -0800 +categories: + - blog +aliases: + - /blog/2024/10/16/beam-summit-2024-overview.html +authors: + - liferoad + - damccorm + - rezarokni +--- + + +At the recently concluded [Beam Summit 2024](https://beamsummit.org/), a two-day event held from September 4 to 5, numerous captivating presentations showcased the potential of Beam to address a wide range of challenges, with an emphasis on machine learning (ML). These challenges included feature engineering, data enrichment, and model inference for large-scale distributed data. In all, the summit included [47 talks](https://beamsummit.org/sessions/2024/), with 16 focused specifically on ML use cases or features and many more touching on these topics. + +The talks displayed the breadth and diversity of the Beam community. Among the speakers and attendees, [23 countries](https://docs.google.com/presentation/d/1IJ1sExHzrzIFF5QXKWlcAuPdp7lKOepRQKl9BnfHxJw/edit#slide=id.g3058d3e2f5f_0_10) were represented. Attendees included Beam users, committers in the Beam project, Beam Google Summer of Code contributors, and data processing/machine learning experts. + +## User-friendly turnkey transforms for ML + +With the features recently added to Beam, Beam now offers a set of rich turn-key transforms for ML users that handle a wide range of ML-Ops tasks. These transforms include: + +* [RunInference](https://beam.apache.org/documentation/ml/overview/#prediction-and-inference): deploy ML models on CPUs and GPUs +* [Enrichment](https://beam.apache.org/documentation/ml/overview/#data-processing): enrich data for ML feature enhancements +* [MLTransform](https://beam.apache.org/documentation/ml/overview/#data-processing): transform data into ML features + +The Summit talks covering both how to use these features and how people are already using them. Highlights included: + +* A talk about [scaling autonomous driving at Cruise](https://beamsummit.org/slides/2024/ScalingAutonomousDrivingwithApacheBeam.pdf) +* Multiple talks about deploying LLMs for batch and streaming inference +* Three different talks about streaming processing for [RAG](https://cloud.google.com/use-cases/retrieval-augmented-generation) (including [a talk](https://www.youtube.com/watch?v=X_VzKQOcpC4) from one of Beam's Google Summer of Code contributors\!) + +## Beam YAML: Simplifying ML data processing + +Beam pipeline creation can be challenging and often requires learning concepts, managing dependencies, debugging, and maintaining code for ML tasks. To simplify the entry point, [Beam YAML](https://beam.apache.org/blog/beam-yaml-release/) introduces a declarative approach that uses YAML configuration files to create data processing pipelines. No coding is required. + +Beam Summit was the first opportunity that the Beam community had to show off some of the use cases of Beam YAML. It featured several talks about how Beam YAML is already a core part of many users' workflows at companies like [MavenCode](https://beamsummit.org/slides/2024/ALowCodeStructuredApproachtoDeployingApacheBeamMLWorkloadsonKubernetesusingBeamStack.pdf) and [ChartBoost](https://youtu.be/avSXvbScbW0). With Beam YAML, these companies are able to build configuration-based data processing systems, significantly lowering the bar for entry at their companies. + +## Prism: Provide a unified ML pipeline development framework for local and remote runner environments + +Beam provides a variety of support for portable runners, but developing a local pipeline has traditionally been painful. Local runners are often incomplete and incompatible with remote runners, such as DataflowRunner and FlinkRunner. + +At Beam Summit, Beam contributors introduced [the Prism local runner](https://youtu.be/R4iNwLBa3VQ) to the community. Prism greatly improves the local developer experience and reduces the gap between local and remote execution. In particular, when handling complicated ML tasks, Prism guarantees consistent runner behavior across these runners, a task that had previously lacked consistent support. + +## Summary + +Beam Summit 2024 showcased the tremendous potential of Apache Beam for addressing a wide range of data processing and machine learning challenges. We look forward to seeing even more innovative use cases and contributions in the future. + +To stay updated on the latest Beam developments and events, visit [the Apache Beam website](https://beam.apache.org/get-started/) and follow us on [social media](https://www.linkedin.com/company/apache-beam/). We encourage you to join [the Beam community](https://beam.apache.org/community/contact-us/) and [contribute to the project](https://beam.apache.org/contribute/). Together, let's unlock the full potential of Beam and shape the future of data processing and machine learning. \ No newline at end of file From a50f91c386c00940b08ef8a5e4d0817422ea230f Mon Sep 17 00:00:00 2001 From: reuvenlax Date: Tue, 15 Oct 2024 14:04:30 -0700 Subject: [PATCH 55/78] Merge pull request #32757: Schema inference parameterized types --- .../beam/sdk/schemas/AutoValueSchema.java | 8 +- .../schemas/FieldValueTypeInformation.java | 89 ++++++----- .../beam/sdk/schemas/JavaBeanSchema.java | 12 +- .../beam/sdk/schemas/JavaFieldSchema.java | 10 +- .../beam/sdk/schemas/SchemaProvider.java | 3 +- .../beam/sdk/schemas/SchemaRegistry.java | 39 ++--- .../transforms/providers/JavaRowUdf.java | 3 +- .../sdk/schemas/utils/AutoValueUtils.java | 20 ++- .../sdk/schemas/utils/ByteBuddyUtils.java | 53 ++++--- .../sdk/schemas/utils/ConvertHelpers.java | 6 +- .../beam/sdk/schemas/utils/JavaBeanUtils.java | 10 +- .../beam/sdk/schemas/utils/POJOUtils.java | 20 ++- .../beam/sdk/schemas/utils/ReflectUtils.java | 83 ++++++++-- .../schemas/utils/StaticSchemaInference.java | 91 +++++------ .../beam/sdk/schemas/AutoValueSchemaTest.java | 149 ++++++++++++++++++ .../beam/sdk/schemas/JavaBeanSchemaTest.java | 124 +++++++++++++++ .../beam/sdk/schemas/JavaFieldSchemaTest.java | 120 ++++++++++++++ .../sdk/schemas/utils/JavaBeanUtilsTest.java | 33 +++- .../beam/sdk/schemas/utils/POJOUtilsTest.java | 36 +++-- .../beam/sdk/schemas/utils/TestJavaBeans.java | 91 +++++++++++ .../beam/sdk/schemas/utils/TestPOJOs.java | 121 +++++++++++++- .../schemas/utils/AvroByteBuddyUtils.java | 6 +- .../avro/schemas/utils/AvroUtils.java | 10 +- .../protobuf/ProtoByteBuddyUtils.java | 4 +- .../protobuf/ProtoMessageSchema.java | 8 +- .../python/PythonExternalTransform.java | 4 +- .../beam/sdk/io/thrift/ThriftSchema.java | 5 +- 27 files changed, 961 insertions(+), 197 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/AutoValueSchema.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/AutoValueSchema.java index 5ccfe39b92af..c369eefeb65c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/AutoValueSchema.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/AutoValueSchema.java @@ -19,8 +19,10 @@ import java.lang.reflect.Method; import java.lang.reflect.Modifier; +import java.lang.reflect.Type; import java.util.Comparator; import java.util.List; +import java.util.Map; import java.util.stream.Collectors; import org.apache.beam.sdk.schemas.annotations.SchemaIgnore; import org.apache.beam.sdk.schemas.utils.AutoValueUtils; @@ -61,8 +63,9 @@ public List get(TypeDescriptor typeDescriptor) { .filter(m -> !m.isAnnotationPresent(SchemaIgnore.class)) .collect(Collectors.toList()); List types = Lists.newArrayListWithCapacity(methods.size()); + Map boundTypes = ReflectUtils.getAllBoundTypes(typeDescriptor); for (int i = 0; i < methods.size(); ++i) { - types.add(FieldValueTypeInformation.forGetter(methods.get(i), i)); + types.add(FieldValueTypeInformation.forGetter(methods.get(i), i, boundTypes)); } types.sort(Comparator.comparing(FieldValueTypeInformation::getNumber)); validateFieldNumbers(types); @@ -143,7 +146,8 @@ public SchemaUserTypeCreator schemaTypeCreator( @Override public @Nullable Schema schemaFor(TypeDescriptor typeDescriptor) { + Map boundTypes = ReflectUtils.getAllBoundTypes(typeDescriptor); return JavaBeanUtils.schemaFromJavaBeanClass( - typeDescriptor, AbstractGetterTypeSupplier.INSTANCE); + typeDescriptor, AbstractGetterTypeSupplier.INSTANCE, boundTypes); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueTypeInformation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueTypeInformation.java index 750709192c08..64687e6d3381 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueTypeInformation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueTypeInformation.java @@ -24,10 +24,12 @@ import java.lang.reflect.Field; import java.lang.reflect.Member; import java.lang.reflect.Method; +import java.lang.reflect.Type; import java.util.Arrays; import java.util.Collections; import java.util.Map; import java.util.stream.Stream; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.schemas.annotations.SchemaCaseFormat; import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; import org.apache.beam.sdk.schemas.annotations.SchemaFieldName; @@ -44,6 +46,7 @@ "nullness", // TODO(https://github.com/apache/beam/issues/20497) "rawtypes" }) +@Internal public abstract class FieldValueTypeInformation implements Serializable { /** Optionally returns the field index. */ public abstract @Nullable Integer getNumber(); @@ -125,8 +128,10 @@ public static FieldValueTypeInformation forOneOf( .build(); } - public static FieldValueTypeInformation forField(Field field, int index) { - TypeDescriptor type = TypeDescriptor.of(field.getGenericType()); + public static FieldValueTypeInformation forField( + Field field, int index, Map boundTypes) { + TypeDescriptor type = + TypeDescriptor.of(ReflectUtils.resolveType(field.getGenericType(), boundTypes)); return new AutoValue_FieldValueTypeInformation.Builder() .setName(getNameOverride(field.getName(), field)) .setNumber(getNumberOverride(index, field)) @@ -134,9 +139,9 @@ public static FieldValueTypeInformation forField(Field field, int index) { .setType(type) .setRawType(type.getRawType()) .setField(field) - .setElementType(getIterableComponentType(field)) - .setMapKeyType(getMapKeyType(field)) - .setMapValueType(getMapValueType(field)) + .setElementType(getIterableComponentType(field, boundTypes)) + .setMapKeyType(getMapKeyType(field, boundTypes)) + .setMapValueType(getMapValueType(field, boundTypes)) .setOneOfTypes(Collections.emptyMap()) .setDescription(getFieldDescription(field)) .build(); @@ -184,7 +189,8 @@ public static String getNameOverride( return fieldDescription.value(); } - public static FieldValueTypeInformation forGetter(Method method, int index) { + public static FieldValueTypeInformation forGetter( + Method method, int index, Map boundTypes) { String name; if (method.getName().startsWith("get")) { name = ReflectUtils.stripPrefix(method.getName(), "get"); @@ -194,7 +200,8 @@ public static FieldValueTypeInformation forGetter(Method method, int index) { throw new RuntimeException("Getter has wrong prefix " + method.getName()); } - TypeDescriptor type = TypeDescriptor.of(method.getGenericReturnType()); + TypeDescriptor type = + TypeDescriptor.of(ReflectUtils.resolveType(method.getGenericReturnType(), boundTypes)); boolean nullable = hasNullableReturnType(method); return new AutoValue_FieldValueTypeInformation.Builder() .setName(getNameOverride(name, method)) @@ -203,9 +210,9 @@ public static FieldValueTypeInformation forGetter(Method method, int index) { .setType(type) .setRawType(type.getRawType()) .setMethod(method) - .setElementType(getIterableComponentType(type)) - .setMapKeyType(getMapKeyType(type)) - .setMapValueType(getMapValueType(type)) + .setElementType(getIterableComponentType(type, boundTypes)) + .setMapKeyType(getMapKeyType(type, boundTypes)) + .setMapValueType(getMapValueType(type, boundTypes)) .setOneOfTypes(Collections.emptyMap()) .setDescription(getFieldDescription(method)) .build(); @@ -252,11 +259,13 @@ private static boolean isNullableAnnotation(Annotation annotation) { return annotation.annotationType().getSimpleName().equals("Nullable"); } - public static FieldValueTypeInformation forSetter(Method method) { - return forSetter(method, "set"); + public static FieldValueTypeInformation forSetter( + Method method, Map boundParameters) { + return forSetter(method, "set", boundParameters); } - public static FieldValueTypeInformation forSetter(Method method, String setterPrefix) { + public static FieldValueTypeInformation forSetter( + Method method, String setterPrefix, Map boundTypes) { String name; if (method.getName().startsWith(setterPrefix)) { name = ReflectUtils.stripPrefix(method.getName(), setterPrefix); @@ -264,7 +273,9 @@ public static FieldValueTypeInformation forSetter(Method method, String setterPr throw new RuntimeException("Setter has wrong prefix " + method.getName()); } - TypeDescriptor type = TypeDescriptor.of(method.getGenericParameterTypes()[0]); + TypeDescriptor type = + TypeDescriptor.of( + ReflectUtils.resolveType(method.getGenericParameterTypes()[0], boundTypes)); boolean nullable = hasSingleNullableParameter(method); return new AutoValue_FieldValueTypeInformation.Builder() .setName(name) @@ -272,9 +283,9 @@ public static FieldValueTypeInformation forSetter(Method method, String setterPr .setType(type) .setRawType(type.getRawType()) .setMethod(method) - .setElementType(getIterableComponentType(type)) - .setMapKeyType(getMapKeyType(type)) - .setMapValueType(getMapValueType(type)) + .setElementType(getIterableComponentType(type, boundTypes)) + .setMapKeyType(getMapKeyType(type, boundTypes)) + .setMapValueType(getMapValueType(type, boundTypes)) .setOneOfTypes(Collections.emptyMap()) .build(); } @@ -283,13 +294,15 @@ public FieldValueTypeInformation withName(String name) { return toBuilder().setName(name).build(); } - private static FieldValueTypeInformation getIterableComponentType(Field field) { - return getIterableComponentType(TypeDescriptor.of(field.getGenericType())); + private static FieldValueTypeInformation getIterableComponentType( + Field field, Map boundTypes) { + return getIterableComponentType(TypeDescriptor.of(field.getGenericType()), boundTypes); } - static @Nullable FieldValueTypeInformation getIterableComponentType(TypeDescriptor valueType) { + static @Nullable FieldValueTypeInformation getIterableComponentType( + TypeDescriptor valueType, Map boundTypes) { // TODO: Figure out nullable elements. - TypeDescriptor componentType = ReflectUtils.getIterableComponentType(valueType); + TypeDescriptor componentType = ReflectUtils.getIterableComponentType(valueType, boundTypes); if (componentType == null) { return null; } @@ -299,41 +312,43 @@ private static FieldValueTypeInformation getIterableComponentType(Field field) { .setNullable(false) .setType(componentType) .setRawType(componentType.getRawType()) - .setElementType(getIterableComponentType(componentType)) - .setMapKeyType(getMapKeyType(componentType)) - .setMapValueType(getMapValueType(componentType)) + .setElementType(getIterableComponentType(componentType, boundTypes)) + .setMapKeyType(getMapKeyType(componentType, boundTypes)) + .setMapValueType(getMapValueType(componentType, boundTypes)) .setOneOfTypes(Collections.emptyMap()) .build(); } // If the Field is a map type, returns the key type, otherwise returns a null reference. - private static @Nullable FieldValueTypeInformation getMapKeyType(Field field) { - return getMapKeyType(TypeDescriptor.of(field.getGenericType())); + private static @Nullable FieldValueTypeInformation getMapKeyType( + Field field, Map boundTypes) { + return getMapKeyType(TypeDescriptor.of(field.getGenericType()), boundTypes); } private static @Nullable FieldValueTypeInformation getMapKeyType( - TypeDescriptor typeDescriptor) { - return getMapType(typeDescriptor, 0); + TypeDescriptor typeDescriptor, Map boundTypes) { + return getMapType(typeDescriptor, 0, boundTypes); } // If the Field is a map type, returns the value type, otherwise returns a null reference. - private static @Nullable FieldValueTypeInformation getMapValueType(Field field) { - return getMapType(TypeDescriptor.of(field.getGenericType()), 1); + private static @Nullable FieldValueTypeInformation getMapValueType( + Field field, Map boundTypes) { + return getMapType(TypeDescriptor.of(field.getGenericType()), 1, boundTypes); } private static @Nullable FieldValueTypeInformation getMapValueType( - TypeDescriptor typeDescriptor) { - return getMapType(typeDescriptor, 1); + TypeDescriptor typeDescriptor, Map boundTypes) { + return getMapType(typeDescriptor, 1, boundTypes); } // If the Field is a map type, returns the key or value type (0 is key type, 1 is value). // Otherwise returns a null reference. @SuppressWarnings("unchecked") private static @Nullable FieldValueTypeInformation getMapType( - TypeDescriptor valueType, int index) { - TypeDescriptor mapType = ReflectUtils.getMapType(valueType, index); + TypeDescriptor valueType, int index, Map boundTypes) { + TypeDescriptor mapType = ReflectUtils.getMapType(valueType, index, boundTypes); if (mapType == null) { return null; } @@ -342,9 +357,9 @@ private static FieldValueTypeInformation getIterableComponentType(Field field) { .setNullable(false) .setType(mapType) .setRawType(mapType.getRawType()) - .setElementType(getIterableComponentType(mapType)) - .setMapKeyType(getMapKeyType(mapType)) - .setMapValueType(getMapValueType(mapType)) + .setElementType(getIterableComponentType(mapType, boundTypes)) + .setMapKeyType(getMapKeyType(mapType, boundTypes)) + .setMapValueType(getMapValueType(mapType, boundTypes)) .setOneOfTypes(Collections.emptyMap()) .build(); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/JavaBeanSchema.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/JavaBeanSchema.java index a9cf01c52057..ad71576670bf 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/JavaBeanSchema.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/JavaBeanSchema.java @@ -19,8 +19,10 @@ import java.lang.reflect.Constructor; import java.lang.reflect.Method; +import java.lang.reflect.Type; import java.util.Comparator; import java.util.List; +import java.util.Map; import java.util.stream.Collectors; import org.apache.beam.sdk.schemas.annotations.SchemaCaseFormat; import org.apache.beam.sdk.schemas.annotations.SchemaFieldName; @@ -67,8 +69,9 @@ public List get(TypeDescriptor typeDescriptor) { .filter(m -> !m.isAnnotationPresent(SchemaIgnore.class)) .collect(Collectors.toList()); List types = Lists.newArrayListWithCapacity(methods.size()); + Map boundTypes = ReflectUtils.getAllBoundTypes(typeDescriptor); for (int i = 0; i < methods.size(); ++i) { - types.add(FieldValueTypeInformation.forGetter(methods.get(i), i)); + types.add(FieldValueTypeInformation.forGetter(methods.get(i), i, boundTypes)); } types.sort(Comparator.comparing(FieldValueTypeInformation::getNumber)); validateFieldNumbers(types); @@ -111,10 +114,11 @@ public static class SetterTypeSupplier implements FieldValueTypeSupplier { @Override public List get(TypeDescriptor typeDescriptor) { + Map boundTypes = ReflectUtils.getAllBoundTypes(typeDescriptor); return ReflectUtils.getMethods(typeDescriptor.getRawType()).stream() .filter(ReflectUtils::isSetter) .filter(m -> !m.isAnnotationPresent(SchemaIgnore.class)) - .map(FieldValueTypeInformation::forSetter) + .map(m -> FieldValueTypeInformation.forSetter(m, boundTypes)) .map( t -> { if (t.getMethod().getAnnotation(SchemaFieldNumber.class) != null) { @@ -156,8 +160,10 @@ public boolean equals(@Nullable Object obj) { @Override public Schema schemaFor(TypeDescriptor typeDescriptor) { + Map boundTypes = ReflectUtils.getAllBoundTypes(typeDescriptor); Schema schema = - JavaBeanUtils.schemaFromJavaBeanClass(typeDescriptor, GetterTypeSupplier.INSTANCE); + JavaBeanUtils.schemaFromJavaBeanClass( + typeDescriptor, GetterTypeSupplier.INSTANCE, boundTypes); // If there are no creator methods, then validate that we have setters for every field. // Otherwise, we will have no way of creating instances of the class. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/JavaFieldSchema.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/JavaFieldSchema.java index 21f07c47b47f..da0f59c8ee96 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/JavaFieldSchema.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/JavaFieldSchema.java @@ -21,8 +21,10 @@ import java.lang.reflect.Field; import java.lang.reflect.Method; import java.lang.reflect.Modifier; +import java.lang.reflect.Type; import java.util.Comparator; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; import javax.annotation.Nullable; @@ -62,9 +64,11 @@ public List get(TypeDescriptor typeDescriptor) { ReflectUtils.getFields(typeDescriptor.getRawType()).stream() .filter(m -> !m.isAnnotationPresent(SchemaIgnore.class)) .collect(Collectors.toList()); + List types = Lists.newArrayListWithCapacity(fields.size()); + Map boundTypes = ReflectUtils.getAllBoundTypes(typeDescriptor); for (int i = 0; i < fields.size(); ++i) { - types.add(FieldValueTypeInformation.forField(fields.get(i), i)); + types.add(FieldValueTypeInformation.forField(fields.get(i), i, boundTypes)); } types.sort(Comparator.comparing(FieldValueTypeInformation::getNumber)); validateFieldNumbers(types); @@ -111,7 +115,9 @@ private static void validateFieldNumbers(List types) @Override public Schema schemaFor(TypeDescriptor typeDescriptor) { - return POJOUtils.schemaFromPojoClass(typeDescriptor, JavaFieldTypeSupplier.INSTANCE); + Map boundTypes = ReflectUtils.getAllBoundTypes(typeDescriptor); + return POJOUtils.schemaFromPojoClass( + typeDescriptor, JavaFieldTypeSupplier.INSTANCE, boundTypes); } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaProvider.java index 37b4952e529c..b7e3cdf60c18 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaProvider.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaProvider.java @@ -38,8 +38,7 @@ public interface SchemaProvider extends Serializable { * Given a type, return a function that converts that type to a {@link Row} object If no schema * exists, returns null. */ - @Nullable - SerializableFunction toRowFunction(TypeDescriptor typeDescriptor); + @Nullable SerializableFunction toRowFunction(TypeDescriptor typeDescriptor); /** * Given a type, returns a function that converts from a {@link Row} object to that type. If no diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaRegistry.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaRegistry.java index 679a1fcf54fc..5d8b7aab6193 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaRegistry.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaRegistry.java @@ -76,13 +76,12 @@ void registerProvider(TypeDescriptor typeDescriptor, SchemaProvider schemaProvid providers.put(typeDescriptor, schemaProvider); } - @Override - public @Nullable Schema schemaFor(TypeDescriptor typeDescriptor) { + private @Nullable SchemaProvider schemaProviderFor(TypeDescriptor typeDescriptor) { TypeDescriptor type = typeDescriptor; do { SchemaProvider schemaProvider = providers.get(type); if (schemaProvider != null) { - return schemaProvider.schemaFor(type); + return schemaProvider; } Class superClass = type.getRawType().getSuperclass(); if (superClass == null || superClass.equals(Object.class)) { @@ -92,38 +91,24 @@ void registerProvider(TypeDescriptor typeDescriptor, SchemaProvider schemaProvid } while (true); } + @Override + public @Nullable Schema schemaFor(TypeDescriptor typeDescriptor) { + @Nullable SchemaProvider schemaProvider = schemaProviderFor(typeDescriptor); + return schemaProvider != null ? schemaProvider.schemaFor(typeDescriptor) : null; + } + @Override public @Nullable SerializableFunction toRowFunction( TypeDescriptor typeDescriptor) { - TypeDescriptor type = typeDescriptor; - do { - SchemaProvider schemaProvider = providers.get(type); - if (schemaProvider != null) { - return (SerializableFunction) schemaProvider.toRowFunction(type); - } - Class superClass = type.getRawType().getSuperclass(); - if (superClass == null || superClass.equals(Object.class)) { - return null; - } - type = TypeDescriptor.of(superClass); - } while (true); + @Nullable SchemaProvider schemaProvider = schemaProviderFor(typeDescriptor); + return schemaProvider != null ? schemaProvider.toRowFunction(typeDescriptor) : null; } @Override public @Nullable SerializableFunction fromRowFunction( TypeDescriptor typeDescriptor) { - TypeDescriptor type = typeDescriptor; - do { - SchemaProvider schemaProvider = providers.get(type); - if (schemaProvider != null) { - return (SerializableFunction) schemaProvider.fromRowFunction(type); - } - Class superClass = type.getRawType().getSuperclass(); - if (superClass == null || superClass.equals(Object.class)) { - return null; - } - type = TypeDescriptor.of(superClass); - } while (true); + @Nullable SchemaProvider schemaProvider = schemaProviderFor(typeDescriptor); + return schemaProvider != null ? schemaProvider.fromRowFunction(typeDescriptor) : null; } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaRowUdf.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaRowUdf.java index 54e2a595fa71..c3a71bbb454b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaRowUdf.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaRowUdf.java @@ -160,7 +160,8 @@ public FunctionAndType(Type outputType, Function function) { public FunctionAndType(TypeDescriptor outputType, Function function) { this( - StaticSchemaInference.fieldFromType(outputType, new EmptyFieldValueTypeSupplier()), + StaticSchemaInference.fieldFromType( + outputType, new EmptyFieldValueTypeSupplier(), Collections.emptyMap()), function); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AutoValueUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AutoValueUtils.java index d7fddd8abfed..74e97bad4f0f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AutoValueUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AutoValueUtils.java @@ -53,6 +53,7 @@ import net.bytebuddy.implementation.bytecode.member.MethodVariableAccess; import net.bytebuddy.jar.asm.ClassWriter; import net.bytebuddy.matcher.ElementMatchers; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.schemas.FieldValueTypeInformation; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.SchemaUserTypeCreator; @@ -63,6 +64,7 @@ import org.apache.beam.sdk.util.common.ReflectHelpers; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; import org.checkerframework.checker.nullness.qual.Nullable; /** Utilities for managing AutoValue schemas. */ @@ -70,6 +72,7 @@ "nullness", // TODO(https://github.com/apache/beam/issues/20497) "rawtypes" }) +@Internal public class AutoValueUtils { public static TypeDescriptor getBaseAutoValueClass(TypeDescriptor typeDescriptor) { // AutoValue extensions may be nested @@ -161,7 +164,7 @@ private static boolean matchConstructor( // Verify that constructor parameters match (name and type) the inferred schema. for (Parameter parameter : constructor.getParameters()) { FieldValueTypeInformation type = typeMap.get(parameter.getName()); - if (type == null || type.getRawType() != parameter.getType()) { + if (type == null || !type.getRawType().equals(parameter.getType())) { valid = false; break; } @@ -178,7 +181,7 @@ private static boolean matchConstructor( } name = name.substring(0, name.length() - 1); FieldValueTypeInformation type = typeMap.get(name); - if (type == null || type.getRawType() != parameter.getType()) { + if (type == null || !type.getRawType().equals(parameter.getType())) { return false; } } @@ -196,11 +199,12 @@ private static boolean matchConstructor( return null; } - Map setterTypes = - ReflectUtils.getMethods(builderClass).stream() - .filter(ReflectUtils::isSetter) - .map(FieldValueTypeInformation::forSetter) - .collect(Collectors.toMap(FieldValueTypeInformation::getName, Function.identity())); + Map boundTypes = ReflectUtils.getAllBoundTypes(TypeDescriptor.of(builderClass)); + Map setterTypes = Maps.newHashMap(); + ReflectUtils.getMethods(builderClass).stream() + .filter(ReflectUtils::isSetter) + .map(m -> FieldValueTypeInformation.forSetter(m, boundTypes)) + .forEach(fv -> setterTypes.putIfAbsent(fv.getName(), fv)); List setterMethods = Lists.newArrayList(); // The builder methods to call in order. @@ -321,7 +325,7 @@ public ByteCodeAppender appender(final Target implementationTarget) { Duplication.SINGLE, typeConversionsFactory .createSetterConversions(readParameter) - .convert(TypeDescriptor.of(parameter.getType())), + .convert(TypeDescriptor.of(parameter.getParameterizedType())), MethodInvocation.invoke(new ForLoadedMethod(setterMethod)), Removal.SINGLE); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ByteBuddyUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ByteBuddyUtils.java index c2b33c2d2315..65adc33a1bab 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ByteBuddyUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ByteBuddyUtils.java @@ -344,19 +344,22 @@ protected Type convertArray(TypeDescriptor type) { @Override protected Type convertCollection(TypeDescriptor type) { - TypeDescriptor ret = createCollectionType(ReflectUtils.getIterableComponentType(type)); + TypeDescriptor ret = + createCollectionType(ReflectUtils.getIterableComponentType(type, Collections.emptyMap())); return returnRawTypes ? ret.getRawType() : ret.getType(); } @Override protected Type convertList(TypeDescriptor type) { - TypeDescriptor ret = createCollectionType(ReflectUtils.getIterableComponentType(type)); + TypeDescriptor ret = + createCollectionType(ReflectUtils.getIterableComponentType(type, Collections.emptyMap())); return returnRawTypes ? ret.getRawType() : ret.getType(); } @Override protected Type convertIterable(TypeDescriptor type) { - TypeDescriptor ret = createIterableType(ReflectUtils.getIterableComponentType(type)); + TypeDescriptor ret = + createIterableType(ReflectUtils.getIterableComponentType(type, Collections.emptyMap())); return returnRawTypes ? ret.getRawType() : ret.getType(); } @@ -687,7 +690,8 @@ protected StackManipulation convertArray(TypeDescriptor type) { @Override protected StackManipulation convertIterable(TypeDescriptor type) { - TypeDescriptor componentType = ReflectUtils.getIterableComponentType(type); + TypeDescriptor componentType = + ReflectUtils.getIterableComponentType(type, Collections.emptyMap()); Type convertedComponentType = getFactory().createTypeConversion(true).convert(componentType); final TypeDescriptor finalComponentType = ReflectUtils.boxIfPrimitive(componentType); @@ -707,7 +711,8 @@ protected StackManipulation convertIterable(TypeDescriptor type) { @Override protected StackManipulation convertCollection(TypeDescriptor type) { - TypeDescriptor componentType = ReflectUtils.getIterableComponentType(type); + TypeDescriptor componentType = + ReflectUtils.getIterableComponentType(type, Collections.emptyMap()); Type convertedComponentType = getFactory().createTypeConversion(true).convert(componentType); final TypeDescriptor finalComponentType = ReflectUtils.boxIfPrimitive(componentType); if (!finalComponentType.hasUnresolvedParameters()) { @@ -726,7 +731,8 @@ protected StackManipulation convertCollection(TypeDescriptor type) { @Override protected StackManipulation convertList(TypeDescriptor type) { - TypeDescriptor componentType = ReflectUtils.getIterableComponentType(type); + TypeDescriptor componentType = + ReflectUtils.getIterableComponentType(type, Collections.emptyMap()); Type convertedComponentType = getFactory().createTypeConversion(true).convert(componentType); final TypeDescriptor finalComponentType = ReflectUtils.boxIfPrimitive(componentType); if (!finalComponentType.hasUnresolvedParameters()) { @@ -745,8 +751,8 @@ protected StackManipulation convertList(TypeDescriptor type) { @Override protected StackManipulation convertMap(TypeDescriptor type) { - final TypeDescriptor keyType = ReflectUtils.getMapType(type, 0); - final TypeDescriptor valueType = ReflectUtils.getMapType(type, 1); + final TypeDescriptor keyType = ReflectUtils.getMapType(type, 0, Collections.emptyMap()); + final TypeDescriptor valueType = ReflectUtils.getMapType(type, 1, Collections.emptyMap()); Type convertedKeyType = getFactory().createTypeConversion(true).convert(keyType); Type convertedValueType = getFactory().createTypeConversion(true).convert(valueType); @@ -1038,8 +1044,9 @@ protected StackManipulation convertIterable(TypeDescriptor type) { Type rowElementType = getFactory() .createTypeConversion(false) - .convert(ReflectUtils.getIterableComponentType(type)); - final TypeDescriptor iterableElementType = ReflectUtils.getIterableComponentType(type); + .convert(ReflectUtils.getIterableComponentType(type, Collections.emptyMap())); + final TypeDescriptor iterableElementType = + ReflectUtils.getIterableComponentType(type, Collections.emptyMap()); if (!iterableElementType.hasUnresolvedParameters()) { ForLoadedType conversionFunction = new ForLoadedType( @@ -1060,8 +1067,9 @@ protected StackManipulation convertCollection(TypeDescriptor type) { Type rowElementType = getFactory() .createTypeConversion(false) - .convert(ReflectUtils.getIterableComponentType(type)); - final TypeDescriptor collectionElementType = ReflectUtils.getIterableComponentType(type); + .convert(ReflectUtils.getIterableComponentType(type, Collections.emptyMap())); + final TypeDescriptor collectionElementType = + ReflectUtils.getIterableComponentType(type, Collections.emptyMap()); if (!collectionElementType.hasUnresolvedParameters()) { ForLoadedType conversionFunction = @@ -1083,8 +1091,9 @@ protected StackManipulation convertList(TypeDescriptor type) { Type rowElementType = getFactory() .createTypeConversion(false) - .convert(ReflectUtils.getIterableComponentType(type)); - final TypeDescriptor collectionElementType = ReflectUtils.getIterableComponentType(type); + .convert(ReflectUtils.getIterableComponentType(type, Collections.emptyMap())); + final TypeDescriptor collectionElementType = + ReflectUtils.getIterableComponentType(type, Collections.emptyMap()); StackManipulation readTrasformedValue = readValue; if (!collectionElementType.hasUnresolvedParameters()) { @@ -1113,11 +1122,17 @@ protected StackManipulation convertList(TypeDescriptor type) { @Override protected StackManipulation convertMap(TypeDescriptor type) { Type rowKeyType = - getFactory().createTypeConversion(false).convert(ReflectUtils.getMapType(type, 0)); - final TypeDescriptor keyElementType = ReflectUtils.getMapType(type, 0); + getFactory() + .createTypeConversion(false) + .convert(ReflectUtils.getMapType(type, 0, Collections.emptyMap())); + final TypeDescriptor keyElementType = + ReflectUtils.getMapType(type, 0, Collections.emptyMap()); Type rowValueType = - getFactory().createTypeConversion(false).convert(ReflectUtils.getMapType(type, 1)); - final TypeDescriptor valueElementType = ReflectUtils.getMapType(type, 1); + getFactory() + .createTypeConversion(false) + .convert(ReflectUtils.getMapType(type, 1, Collections.emptyMap())); + final TypeDescriptor valueElementType = + ReflectUtils.getMapType(type, 1, Collections.emptyMap()); StackManipulation readTrasformedValue = readValue; if (!keyElementType.hasUnresolvedParameters() @@ -1475,7 +1490,7 @@ public ByteCodeAppender appender(final Target implementationTarget) { Parameter parameter = parameters.get(i); ForLoadedType convertedType = new ForLoadedType( - (Class) convertType.convert(TypeDescriptor.of(parameter.getType()))); + (Class) convertType.convert(TypeDescriptor.of(parameter.getParameterizedType()))); // The instruction to read the parameter. Use the fieldMapping to reorder parameters as // necessary. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ConvertHelpers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ConvertHelpers.java index 7f2403035d97..e98a0b9495cf 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ConvertHelpers.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ConvertHelpers.java @@ -22,6 +22,7 @@ import java.io.Serializable; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Type; +import java.util.Collections; import java.util.ServiceLoader; import net.bytebuddy.ByteBuddy; import net.bytebuddy.asm.AsmVisitorWrapper; @@ -36,6 +37,7 @@ import net.bytebuddy.implementation.bytecode.member.MethodVariableAccess; import net.bytebuddy.jar.asm.ClassWriter; import net.bytebuddy.matcher.ElementMatchers; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.schemas.JavaFieldSchema.JavaFieldTypeSupplier; import org.apache.beam.sdk.schemas.NoSuchSchemaException; import org.apache.beam.sdk.schemas.Schema; @@ -56,6 +58,7 @@ "nullness", // TODO(https://github.com/apache/beam/issues/20497) "rawtypes" }) +@Internal public class ConvertHelpers { private static class SchemaInformationProviders { private static final ServiceLoader INSTANCE = @@ -148,7 +151,8 @@ public static SerializableFunction getConvertPrimitive( TypeDescriptor outputTypeDescriptor, TypeConversionsFactory typeConversionsFactory) { FieldType expectedFieldType = - StaticSchemaInference.fieldFromType(outputTypeDescriptor, JavaFieldTypeSupplier.INSTANCE); + StaticSchemaInference.fieldFromType( + outputTypeDescriptor, JavaFieldTypeSupplier.INSTANCE, Collections.emptyMap()); if (!expectedFieldType.equals(fieldType)) { throw new IllegalArgumentException( "Element argument type " diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtils.java index 911f79f6eeed..83f6b5c928d8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtils.java @@ -22,6 +22,7 @@ import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; +import java.lang.reflect.Type; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -42,6 +43,7 @@ import net.bytebuddy.implementation.bytecode.member.MethodVariableAccess; import net.bytebuddy.jar.asm.ClassWriter; import net.bytebuddy.matcher.ElementMatchers; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.schemas.FieldValueGetter; import org.apache.beam.sdk.schemas.FieldValueSetter; import org.apache.beam.sdk.schemas.FieldValueTypeInformation; @@ -61,11 +63,15 @@ "nullness", // TODO(https://github.com/apache/beam/issues/20497) "rawtypes" }) +@Internal public class JavaBeanUtils { /** Create a {@link Schema} for a Java Bean class. */ public static Schema schemaFromJavaBeanClass( - TypeDescriptor typeDescriptor, FieldValueTypeSupplier fieldValueTypeSupplier) { - return StaticSchemaInference.schemaFromClass(typeDescriptor, fieldValueTypeSupplier); + TypeDescriptor typeDescriptor, + FieldValueTypeSupplier fieldValueTypeSupplier, + Map boundTypes) { + return StaticSchemaInference.schemaFromClass( + typeDescriptor, fieldValueTypeSupplier, boundTypes); } private static final String CONSTRUCTOR_HELP_STRING = diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/POJOUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/POJOUtils.java index 571b9c690900..1e60c9312cb3 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/POJOUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/POJOUtils.java @@ -49,6 +49,7 @@ import net.bytebuddy.implementation.bytecode.member.MethodVariableAccess; import net.bytebuddy.jar.asm.ClassWriter; import net.bytebuddy.matcher.ElementMatchers; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.schemas.FieldValueGetter; import org.apache.beam.sdk.schemas.FieldValueSetter; import org.apache.beam.sdk.schemas.FieldValueTypeInformation; @@ -70,11 +71,15 @@ "nullness", // TODO(https://github.com/apache/beam/issues/20497) "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) }) +@Internal public class POJOUtils { public static Schema schemaFromPojoClass( - TypeDescriptor typeDescriptor, FieldValueTypeSupplier fieldValueTypeSupplier) { - return StaticSchemaInference.schemaFromClass(typeDescriptor, fieldValueTypeSupplier); + TypeDescriptor typeDescriptor, + FieldValueTypeSupplier fieldValueTypeSupplier, + Map boundTypes) { + return StaticSchemaInference.schemaFromClass( + typeDescriptor, fieldValueTypeSupplier, boundTypes); } // Static ByteBuddy instance used by all helpers. @@ -301,7 +306,7 @@ public static SchemaUserTypeCreator createStaticCreator( field.getDeclaringClass(), typeConversionsFactory .createTypeConversion(false) - .convert(TypeDescriptor.of(field.getType()))); + .convert(TypeDescriptor.of(field.getGenericType()))); builder = implementGetterMethods(builder, field, typeInformation.getName(), typeConversionsFactory); try { @@ -383,7 +388,7 @@ private static FieldValueSetter createSetter( field.getDeclaringClass(), typeConversionsFactory .createTypeConversion(false) - .convert(TypeDescriptor.of(field.getType()))); + .convert(TypeDescriptor.of(field.getGenericType()))); builder = implementSetterMethods(builder, field, typeConversionsFactory); try { return builder @@ -491,7 +496,7 @@ public ByteCodeAppender appender(final Target implementationTarget) { // Do any conversions necessary. typeConversionsFactory .createSetterConversions(readField) - .convert(TypeDescriptor.of(field.getType())), + .convert(TypeDescriptor.of(field.getGenericType())), // Now update the field and return void. FieldAccess.forField(new ForLoadedField(field)).write(), MethodReturn.VOID); @@ -546,7 +551,8 @@ public ByteCodeAppender appender(final Target implementationTarget) { Field field = fields.get(i); ForLoadedType convertedType = - new ForLoadedType((Class) convertType.convert(TypeDescriptor.of(field.getType()))); + new ForLoadedType( + (Class) convertType.convert(TypeDescriptor.of(field.getGenericType()))); // The instruction to read the parameter. StackManipulation readParameter = @@ -563,7 +569,7 @@ public ByteCodeAppender appender(final Target implementationTarget) { // Do any conversions necessary. typeConversionsFactory .createSetterConversions(readParameter) - .convert(TypeDescriptor.of(field.getType())), + .convert(TypeDescriptor.of(field.getGenericType())), // Now update the field. FieldAccess.forField(new ForLoadedField(field)).write()); stackManipulation = new StackManipulation.Compound(stackManipulation, updateField); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ReflectUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ReflectUtils.java index 4349a04c28ad..32cfa5689193 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ReflectUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ReflectUtils.java @@ -26,16 +26,17 @@ import java.lang.reflect.Modifier; import java.lang.reflect.ParameterizedType; import java.lang.reflect.Type; +import java.lang.reflect.TypeVariable; import java.security.InvalidParameterException; import java.util.Arrays; import java.util.Collection; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.annotations.SchemaCreate; +import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; @@ -88,14 +89,23 @@ public static List getMethods(Class clazz) { return DECLARED_METHODS.computeIfAbsent( clazz, c -> { - return Arrays.stream(c.getDeclaredMethods()) - .filter( - m -> !m.isBridge()) // Covariant overloads insert bridge functions, which we must - // ignore. - .filter(m -> !Modifier.isPrivate(m.getModifiers())) - .filter(m -> !Modifier.isProtected(m.getModifiers())) - .filter(m -> !Modifier.isStatic(m.getModifiers())) - .collect(Collectors.toList()); + List methods = Lists.newArrayList(); + do { + if (c.getPackage() != null && c.getPackage().getName().startsWith("java.")) { + break; // skip java built-in classes + } + Arrays.stream(c.getDeclaredMethods()) + .filter( + m -> + !m.isBridge()) // Covariant overloads insert bridge functions, which we must + // ignore. + .filter(m -> !Modifier.isPrivate(m.getModifiers())) + .filter(m -> !Modifier.isProtected(m.getModifiers())) + .filter(m -> !Modifier.isStatic(m.getModifiers())) + .forEach(methods::add); + c = c.getSuperclass(); + } while (c != null); + return methods; }); } @@ -201,7 +211,8 @@ public static String stripSetterPrefix(String method) { } /** For an array T[] or a subclass of Iterable, return a TypeDescriptor describing T. */ - public static @Nullable TypeDescriptor getIterableComponentType(TypeDescriptor valueType) { + public static @Nullable TypeDescriptor getIterableComponentType( + TypeDescriptor valueType, Map boundTypes) { TypeDescriptor componentType = null; if (valueType.isArray()) { Type component = valueType.getComponentType().getType(); @@ -215,7 +226,7 @@ public static String stripSetterPrefix(String method) { ParameterizedType ptype = (ParameterizedType) collection.getType(); java.lang.reflect.Type[] params = ptype.getActualTypeArguments(); checkArgument(params.length == 1); - componentType = TypeDescriptor.of(params[0]); + componentType = TypeDescriptor.of(resolveType(params[0], boundTypes)); } else { throw new RuntimeException("Collection parameter is not parameterized!"); } @@ -223,14 +234,15 @@ public static String stripSetterPrefix(String method) { return componentType; } - public static TypeDescriptor getMapType(TypeDescriptor valueType, int index) { + public static TypeDescriptor getMapType( + TypeDescriptor valueType, int index, Map boundTypes) { TypeDescriptor mapType = null; if (valueType.isSubtypeOf(TypeDescriptor.of(Map.class))) { TypeDescriptor> map = valueType.getSupertype(Map.class); if (map.getType() instanceof ParameterizedType) { ParameterizedType ptype = (ParameterizedType) map.getType(); java.lang.reflect.Type[] params = ptype.getActualTypeArguments(); - mapType = TypeDescriptor.of(params[index]); + mapType = TypeDescriptor.of(resolveType(params[index], boundTypes)); } else { throw new RuntimeException("Map type is not parameterized! " + map); } @@ -243,4 +255,49 @@ public static TypeDescriptor boxIfPrimitive(TypeDescriptor typeDescriptor) { ? TypeDescriptor.of(Primitives.wrap(typeDescriptor.getRawType())) : typeDescriptor; } + + /** + * If this (or a base class)is a paremeterized type, return a map of all TypeVariable->Type + * bindings. This allows us to resolve types in any contained fields or methods. + */ + public static Map getAllBoundTypes(TypeDescriptor typeDescriptor) { + Map boundParameters = Maps.newHashMap(); + TypeDescriptor currentType = typeDescriptor; + do { + if (currentType.getType() instanceof ParameterizedType) { + ParameterizedType parameterizedType = (ParameterizedType) currentType.getType(); + TypeVariable[] typeVariables = currentType.getRawType().getTypeParameters(); + Type[] typeArguments = parameterizedType.getActualTypeArguments(); + ; + if (typeArguments.length != typeVariables.length) { + throw new RuntimeException("Unmatching arguments lengths in type " + typeDescriptor); + } + for (int i = 0; i < typeVariables.length; ++i) { + boundParameters.put(typeVariables[i], typeArguments[i]); + } + } + Type superClass = currentType.getRawType().getGenericSuperclass(); + if (superClass == null || superClass.equals(Object.class)) { + break; + } + currentType = TypeDescriptor.of(superClass); + } while (true); + return boundParameters; + } + + public static Type resolveType(Type type, Map boundTypes) { + TypeDescriptor typeDescriptor = TypeDescriptor.of(type); + if (typeDescriptor.isSubtypeOf(TypeDescriptor.of(Iterable.class)) + || typeDescriptor.isSubtypeOf(TypeDescriptor.of(Map.class))) { + // Don't resolve these as we special case map and interable. + return type; + } + + if (type instanceof TypeVariable) { + TypeVariable typeVariable = (TypeVariable) type; + return Preconditions.checkArgumentNotNull(boundTypes.get(typeVariable)); + } else { + return type; + } + } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/StaticSchemaInference.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/StaticSchemaInference.java index 196ee6f86593..275bc41be53d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/StaticSchemaInference.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/StaticSchemaInference.java @@ -19,7 +19,7 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; -import java.lang.reflect.ParameterizedType; +import java.lang.reflect.Type; import java.math.BigDecimal; import java.nio.ByteBuffer; import java.util.Arrays; @@ -29,10 +29,12 @@ import java.util.Map; import java.util.function.Function; import java.util.stream.Collectors; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.schemas.FieldValueTypeInformation; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.FieldType; import org.apache.beam.sdk.schemas.logicaltypes.EnumerationType; +import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.joda.time.ReadableInstant; @@ -42,6 +44,7 @@ "nullness", // TODO(https://github.com/apache/beam/issues/20497) "rawtypes" }) +@Internal public class StaticSchemaInference { public static List sortBySchema( List types, Schema schema) { @@ -85,14 +88,17 @@ enum MethodType { * public getter methods, or special annotations on the class. */ public static Schema schemaFromClass( - TypeDescriptor typeDescriptor, FieldValueTypeSupplier fieldValueTypeSupplier) { - return schemaFromClass(typeDescriptor, fieldValueTypeSupplier, new HashMap<>()); + TypeDescriptor typeDescriptor, + FieldValueTypeSupplier fieldValueTypeSupplier, + Map boundTypes) { + return schemaFromClass(typeDescriptor, fieldValueTypeSupplier, new HashMap<>(), boundTypes); } private static Schema schemaFromClass( TypeDescriptor typeDescriptor, FieldValueTypeSupplier fieldValueTypeSupplier, - Map, Schema> alreadyVisitedSchemas) { + Map, Schema> alreadyVisitedSchemas, + Map boundTypes) { if (alreadyVisitedSchemas.containsKey(typeDescriptor)) { Schema existingSchema = alreadyVisitedSchemas.get(typeDescriptor); if (existingSchema == null) { @@ -106,7 +112,7 @@ private static Schema schemaFromClass( Schema.Builder builder = Schema.builder(); for (FieldValueTypeInformation type : fieldValueTypeSupplier.get(typeDescriptor)) { Schema.FieldType fieldType = - fieldFromType(type.getType(), fieldValueTypeSupplier, alreadyVisitedSchemas); + fieldFromType(type.getType(), fieldValueTypeSupplier, alreadyVisitedSchemas, boundTypes); Schema.Field f = type.isNullable() ? Schema.Field.nullable(type.getName(), fieldType) @@ -123,15 +129,18 @@ private static Schema schemaFromClass( /** Map a Java field type to a Beam Schema FieldType. */ public static Schema.FieldType fieldFromType( - TypeDescriptor type, FieldValueTypeSupplier fieldValueTypeSupplier) { - return fieldFromType(type, fieldValueTypeSupplier, new HashMap<>()); + TypeDescriptor type, + FieldValueTypeSupplier fieldValueTypeSupplier, + Map boundTypes) { + return fieldFromType(type, fieldValueTypeSupplier, new HashMap<>(), boundTypes); } // TODO(https://github.com/apache/beam/issues/21567): support type inference for logical types private static Schema.FieldType fieldFromType( TypeDescriptor type, FieldValueTypeSupplier fieldValueTypeSupplier, - Map, Schema> alreadyVisitedSchemas) { + Map, Schema> alreadyVisitedSchemas, + Map boundTypes) { FieldType primitiveType = PRIMITIVE_TYPES.get(type.getRawType()); if (primitiveType != null) { return primitiveType; @@ -152,27 +161,25 @@ private static Schema.FieldType fieldFromType( } else { // Otherwise this is an array type. return FieldType.array( - fieldFromType(component, fieldValueTypeSupplier, alreadyVisitedSchemas)); + fieldFromType(component, fieldValueTypeSupplier, alreadyVisitedSchemas, boundTypes)); } } else if (type.isSubtypeOf(TypeDescriptor.of(Map.class))) { - TypeDescriptor> map = type.getSupertype(Map.class); - if (map.getType() instanceof ParameterizedType) { - ParameterizedType ptype = (ParameterizedType) map.getType(); - java.lang.reflect.Type[] params = ptype.getActualTypeArguments(); - checkArgument(params.length == 2); - FieldType keyType = - fieldFromType( - TypeDescriptor.of(params[0]), fieldValueTypeSupplier, alreadyVisitedSchemas); - FieldType valueType = - fieldFromType( - TypeDescriptor.of(params[1]), fieldValueTypeSupplier, alreadyVisitedSchemas); - checkArgument( - keyType.getTypeName().isPrimitiveType(), - "Only primitive types can be map keys. type: " + keyType.getTypeName()); - return FieldType.map(keyType, valueType); - } else { - throw new RuntimeException("Cannot infer schema from unparameterized map."); - } + FieldType keyType = + fieldFromType( + ReflectUtils.getMapType(type, 0, boundTypes), + fieldValueTypeSupplier, + alreadyVisitedSchemas, + boundTypes); + FieldType valueType = + fieldFromType( + ReflectUtils.getMapType(type, 1, boundTypes), + fieldValueTypeSupplier, + alreadyVisitedSchemas, + boundTypes); + checkArgument( + keyType.getTypeName().isPrimitiveType(), + "Only primitive types can be map keys. type: " + keyType.getTypeName()); + return FieldType.map(keyType, valueType); } else if (type.isSubtypeOf(TypeDescriptor.of(CharSequence.class))) { return FieldType.STRING; } else if (type.isSubtypeOf(TypeDescriptor.of(ReadableInstant.class))) { @@ -180,26 +187,22 @@ private static Schema.FieldType fieldFromType( } else if (type.isSubtypeOf(TypeDescriptor.of(ByteBuffer.class))) { return FieldType.BYTES; } else if (type.isSubtypeOf(TypeDescriptor.of(Iterable.class))) { - TypeDescriptor> iterable = type.getSupertype(Iterable.class); - if (iterable.getType() instanceof ParameterizedType) { - ParameterizedType ptype = (ParameterizedType) iterable.getType(); - java.lang.reflect.Type[] params = ptype.getActualTypeArguments(); - checkArgument(params.length == 1); - // TODO: should this be AbstractCollection? - if (type.isSubtypeOf(TypeDescriptor.of(Collection.class))) { - return FieldType.array( - fieldFromType( - TypeDescriptor.of(params[0]), fieldValueTypeSupplier, alreadyVisitedSchemas)); - } else { - return FieldType.iterable( - fieldFromType( - TypeDescriptor.of(params[0]), fieldValueTypeSupplier, alreadyVisitedSchemas)); - } + FieldType elementType = + fieldFromType( + Preconditions.checkArgumentNotNull( + ReflectUtils.getIterableComponentType(type, boundTypes)), + fieldValueTypeSupplier, + alreadyVisitedSchemas, + boundTypes); + // TODO: should this be AbstractCollection? + if (type.isSubtypeOf(TypeDescriptor.of(Collection.class))) { + return FieldType.array(elementType); } else { - throw new RuntimeException("Cannot infer schema from unparameterized collection."); + return FieldType.iterable(elementType); } } else { - return FieldType.row(schemaFromClass(type, fieldValueTypeSupplier, alreadyVisitedSchemas)); + return FieldType.row( + schemaFromClass(type, fieldValueTypeSupplier, alreadyVisitedSchemas, boundTypes)); } } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AutoValueSchemaTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AutoValueSchemaTest.java index d0ee623dea7c..49fd2bfe2259 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AutoValueSchemaTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AutoValueSchemaTest.java @@ -28,6 +28,7 @@ import java.math.BigDecimal; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; +import java.util.Map; import org.apache.beam.sdk.schemas.Schema.Field; import org.apache.beam.sdk.schemas.Schema.FieldType; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; @@ -39,6 +40,7 @@ import org.apache.beam.sdk.schemas.utils.SchemaTestUtils; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.CaseFormat; import org.joda.time.DateTime; import org.joda.time.Instant; @@ -886,4 +888,151 @@ public void testSchema_SchemaFieldDescription() throws NoSuchSchemaException { assertEquals(FIELD_DESCRIPTION_SCHEMA.getField("lng"), schema.getField("lng")); assertEquals(FIELD_DESCRIPTION_SCHEMA.getField("str"), schema.getField("str")); } + + @AutoValue + @DefaultSchema(AutoValueSchema.class) + abstract static class ParameterizedAutoValue { + abstract W getValue1(); + + abstract T getValue2(); + + abstract V getValue3(); + + abstract X getValue4(); + } + + @Test + public void testAutoValueWithTypeParameter() throws NoSuchSchemaException { + SchemaRegistry registry = SchemaRegistry.createDefault(); + TypeDescriptor> typeDescriptor = + new TypeDescriptor>() {}; + Schema schema = registry.getSchema(typeDescriptor); + + final Schema expectedSchema = + Schema.builder() + .addBooleanField("value1") + .addStringField("value2") + .addInt64Field("value3") + .addRowField("value4", SIMPLE_SCHEMA) + .build(); + assertTrue(expectedSchema.equivalent(schema)); + } + + @DefaultSchema(AutoValueSchema.class) + abstract static class ParameterizedAutoValueSubclass + extends ParameterizedAutoValue { + abstract T getValue5(); + } + + @Test + public void testAutoValueWithInheritedTypeParameter() throws NoSuchSchemaException { + SchemaRegistry registry = SchemaRegistry.createDefault(); + TypeDescriptor> typeDescriptor = + new TypeDescriptor>() {}; + Schema schema = registry.getSchema(typeDescriptor); + + final Schema expectedSchema = + Schema.builder() + .addBooleanField("value1") + .addStringField("value2") + .addInt64Field("value3") + .addRowField("value4", SIMPLE_SCHEMA) + .addInt16Field("value5") + .build(); + assertTrue(expectedSchema.equivalent(schema)); + } + + @AutoValue + @DefaultSchema(AutoValueSchema.class) + abstract static class NestedParameterizedCollectionAutoValue { + abstract Iterable getNested(); + + abstract Map getMap(); + } + + @Test + public void testAutoValueWithNestedCollectionTypeParameter() throws NoSuchSchemaException { + SchemaRegistry registry = SchemaRegistry.createDefault(); + TypeDescriptor< + NestedParameterizedCollectionAutoValue< + ParameterizedAutoValue, String>> + typeDescriptor = + new TypeDescriptor< + NestedParameterizedCollectionAutoValue< + ParameterizedAutoValue, String>>() {}; + Schema schema = registry.getSchema(typeDescriptor); + + final Schema expectedInnerSchema = + Schema.builder() + .addBooleanField("value1") + .addStringField("value2") + .addInt64Field("value3") + .addRowField("value4", SIMPLE_SCHEMA) + .build(); + final Schema expectedSchema = + Schema.builder() + .addIterableField("nested", FieldType.row(expectedInnerSchema)) + .addMapField("map", FieldType.STRING, FieldType.row(expectedInnerSchema)) + .build(); + assertTrue(expectedSchema.equivalent(schema)); + } + + @Test + public void testAutoValueWithDoublyNestedCollectionTypeParameter() throws NoSuchSchemaException { + SchemaRegistry registry = SchemaRegistry.createDefault(); + TypeDescriptor< + NestedParameterizedCollectionAutoValue< + Iterable>, String>> + typeDescriptor = + new TypeDescriptor< + NestedParameterizedCollectionAutoValue< + Iterable>, + String>>() {}; + Schema schema = registry.getSchema(typeDescriptor); + + final Schema expectedInnerSchema = + Schema.builder() + .addBooleanField("value1") + .addStringField("value2") + .addInt64Field("value3") + .addRowField("value4", SIMPLE_SCHEMA) + .build(); + final Schema expectedSchema = + Schema.builder() + .addIterableField("nested", FieldType.iterable(FieldType.row(expectedInnerSchema))) + .addMapField( + "map", FieldType.STRING, FieldType.iterable(FieldType.row(expectedInnerSchema))) + .build(); + assertTrue(expectedSchema.equivalent(schema)); + } + + @AutoValue + @DefaultSchema(AutoValueSchema.class) + abstract static class NestedParameterizedAutoValue { + abstract T getNested(); + } + + @Test + public void testAutoValueWithNestedTypeParameter() throws NoSuchSchemaException { + SchemaRegistry registry = SchemaRegistry.createDefault(); + TypeDescriptor< + NestedParameterizedAutoValue< + ParameterizedAutoValue>> + typeDescriptor = + new TypeDescriptor< + NestedParameterizedAutoValue< + ParameterizedAutoValue>>() {}; + Schema schema = registry.getSchema(typeDescriptor); + + final Schema expectedInnerSchema = + Schema.builder() + .addBooleanField("value1") + .addStringField("value2") + .addInt64Field("value3") + .addRowField("value4", SIMPLE_SCHEMA) + .build(); + final Schema expectedSchema = + Schema.builder().addRowField("nested", expectedInnerSchema).build(); + assertTrue(expectedSchema.equivalent(schema)); + } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaBeanSchemaTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaBeanSchemaTest.java index 5313feb5c6c0..2252c3aef0db 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaBeanSchemaTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaBeanSchemaTest.java @@ -68,6 +68,7 @@ import org.apache.beam.sdk.schemas.utils.TestJavaBeans.SimpleBeanWithAnnotations; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; @@ -625,4 +626,127 @@ public void testSetterConstructionWithRenamedFields() throws NoSuchSchemaExcepti assertEquals( registry.getFromRowFunction(BeanWithCaseFormat.class).apply(row), beanWithCaseFormat); } + + @Test + public void testBeanWithTypeParameter() throws NoSuchSchemaException { + SchemaRegistry registry = SchemaRegistry.createDefault(); + TypeDescriptor> + typeDescriptor = + new TypeDescriptor< + TestJavaBeans.SimpleParameterizedBean>() {}; + Schema schema = registry.getSchema(typeDescriptor); + + final Schema expectedSchema = + Schema.builder() + .addBooleanField("value1") + .addStringField("value2") + .addInt64Field("value3") + .addRowField("value4", SIMPLE_BEAN_SCHEMA) + .build(); + assertTrue(expectedSchema.equivalent(schema)); + } + + @Test + public void testBeanWithInheritedTypeParameter() throws NoSuchSchemaException { + SchemaRegistry registry = SchemaRegistry.createDefault(); + TypeDescriptor> typeDescriptor = + new TypeDescriptor>() {}; + Schema schema = registry.getSchema(typeDescriptor); + + final Schema expectedSchema = + Schema.builder() + .addBooleanField("value1") + .addStringField("value2") + .addInt64Field("value3") + .addRowField("value4", SIMPLE_BEAN_SCHEMA) + .addInt16Field("value5") + .build(); + assertTrue(expectedSchema.equivalent(schema)); + } + + @Test + public void testBeanWithNestedCollectionTypeParameter() throws NoSuchSchemaException { + SchemaRegistry registry = SchemaRegistry.createDefault(); + TypeDescriptor< + TestJavaBeans.NestedParameterizedCollectionBean< + TestJavaBeans.SimpleParameterizedBean, String>> + typeDescriptor = + new TypeDescriptor< + TestJavaBeans.NestedParameterizedCollectionBean< + TestJavaBeans.SimpleParameterizedBean, + String>>() {}; + Schema schema = registry.getSchema(typeDescriptor); + + final Schema expectedInnerSchema = + Schema.builder() + .addBooleanField("value1") + .addStringField("value2") + .addInt64Field("value3") + .addRowField("value4", SIMPLE_BEAN_SCHEMA) + .build(); + final Schema expectedSchema = + Schema.builder() + .addIterableField("nested", Schema.FieldType.row(expectedInnerSchema)) + .addMapField("map", Schema.FieldType.STRING, Schema.FieldType.row(expectedInnerSchema)) + .build(); + assertTrue(expectedSchema.equivalent(schema)); + } + + @Test + public void testBeanWithDoublyNestedCollectionTypeParameter() throws NoSuchSchemaException { + SchemaRegistry registry = SchemaRegistry.createDefault(); + TypeDescriptor< + TestJavaBeans.NestedParameterizedCollectionBean< + Iterable>, + String>> + typeDescriptor = + new TypeDescriptor< + TestJavaBeans.NestedParameterizedCollectionBean< + Iterable< + TestJavaBeans.SimpleParameterizedBean>, + String>>() {}; + Schema schema = registry.getSchema(typeDescriptor); + + final Schema expectedInnerSchema = + Schema.builder() + .addBooleanField("value1") + .addStringField("value2") + .addInt64Field("value3") + .addRowField("value4", SIMPLE_BEAN_SCHEMA) + .build(); + final Schema expectedSchema = + Schema.builder() + .addIterableField( + "nested", Schema.FieldType.iterable(Schema.FieldType.row(expectedInnerSchema))) + .addMapField( + "map", + Schema.FieldType.STRING, + Schema.FieldType.iterable(Schema.FieldType.row(expectedInnerSchema))) + .build(); + assertTrue(expectedSchema.equivalent(schema)); + } + + @Test + public void testBeanWithNestedTypeParameter() throws NoSuchSchemaException { + SchemaRegistry registry = SchemaRegistry.createDefault(); + TypeDescriptor< + TestJavaBeans.NestedParameterizedBean< + TestJavaBeans.SimpleParameterizedBean>> + typeDescriptor = + new TypeDescriptor< + TestJavaBeans.NestedParameterizedBean< + TestJavaBeans.SimpleParameterizedBean>>() {}; + Schema schema = registry.getSchema(typeDescriptor); + + final Schema expectedInnerSchema = + Schema.builder() + .addBooleanField("value1") + .addStringField("value2") + .addInt64Field("value3") + .addRowField("value4", SIMPLE_BEAN_SCHEMA) + .build(); + final Schema expectedSchema = + Schema.builder().addRowField("nested", expectedInnerSchema).build(); + assertTrue(expectedSchema.equivalent(schema)); + } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaFieldSchemaTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaFieldSchemaTest.java index 11bef79b26f7..70bc3030924b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaFieldSchemaTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaFieldSchemaTest.java @@ -76,6 +76,7 @@ import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; @@ -781,4 +782,123 @@ public void testCircularNestedPOJOThrows() throws NoSuchSchemaException { thrown.getMessage(), containsString("TestPOJOs$FirstCircularNestedPOJO")); } + + @Test + public void testPojoWithTypeParameter() throws NoSuchSchemaException { + SchemaRegistry registry = SchemaRegistry.createDefault(); + TypeDescriptor> + typeDescriptor = + new TypeDescriptor< + TestPOJOs.SimpleParameterizedPOJO>() {}; + Schema schema = registry.getSchema(typeDescriptor); + + final Schema expectedSchema = + Schema.builder() + .addBooleanField("value1") + .addStringField("value2") + .addInt64Field("value3") + .addRowField("value4", SIMPLE_POJO_SCHEMA) + .build(); + assertTrue(expectedSchema.equivalent(schema)); + } + + @Test + public void testPojoWithInheritedTypeParameter() throws NoSuchSchemaException { + SchemaRegistry registry = SchemaRegistry.createDefault(); + TypeDescriptor> typeDescriptor = + new TypeDescriptor>() {}; + Schema schema = registry.getSchema(typeDescriptor); + + final Schema expectedSchema = + Schema.builder() + .addBooleanField("value1") + .addStringField("value2") + .addInt64Field("value3") + .addRowField("value4", SIMPLE_POJO_SCHEMA) + .addInt16Field("value5") + .build(); + assertTrue(expectedSchema.equivalent(schema)); + } + + @Test + public void testPojoWithNestedCollectionTypeParameter() throws NoSuchSchemaException { + SchemaRegistry registry = SchemaRegistry.createDefault(); + TypeDescriptor< + TestPOJOs.NestedParameterizedCollectionPOJO< + TestPOJOs.SimpleParameterizedPOJO, String>> + typeDescriptor = + new TypeDescriptor< + TestPOJOs.NestedParameterizedCollectionPOJO< + TestPOJOs.SimpleParameterizedPOJO, + String>>() {}; + Schema schema = registry.getSchema(typeDescriptor); + + final Schema expectedInnerSchema = + Schema.builder() + .addBooleanField("value1") + .addStringField("value2") + .addInt64Field("value3") + .addRowField("value4", SIMPLE_POJO_SCHEMA) + .build(); + final Schema expectedSchema = + Schema.builder() + .addIterableField("nested", FieldType.row(expectedInnerSchema)) + .addMapField("map", FieldType.STRING, FieldType.row(expectedInnerSchema)) + .build(); + assertTrue(expectedSchema.equivalent(schema)); + } + + @Test + public void testPojoWithDoublyNestedCollectionTypeParameter() throws NoSuchSchemaException { + SchemaRegistry registry = SchemaRegistry.createDefault(); + TypeDescriptor< + TestPOJOs.NestedParameterizedCollectionPOJO< + Iterable>, + String>> + typeDescriptor = + new TypeDescriptor< + TestPOJOs.NestedParameterizedCollectionPOJO< + Iterable>, + String>>() {}; + Schema schema = registry.getSchema(typeDescriptor); + + final Schema expectedInnerSchema = + Schema.builder() + .addBooleanField("value1") + .addStringField("value2") + .addInt64Field("value3") + .addRowField("value4", SIMPLE_POJO_SCHEMA) + .build(); + final Schema expectedSchema = + Schema.builder() + .addIterableField("nested", FieldType.iterable(FieldType.row(expectedInnerSchema))) + .addMapField( + "map", FieldType.STRING, FieldType.iterable(FieldType.row(expectedInnerSchema))) + .build(); + assertTrue(expectedSchema.equivalent(schema)); + } + + @Test + public void testPojoWithNestedTypeParameter() throws NoSuchSchemaException { + SchemaRegistry registry = SchemaRegistry.createDefault(); + TypeDescriptor< + TestPOJOs.NestedParameterizedPOJO< + TestPOJOs.SimpleParameterizedPOJO>> + typeDescriptor = + new TypeDescriptor< + TestPOJOs.NestedParameterizedPOJO< + TestPOJOs.SimpleParameterizedPOJO>>() {}; + Schema schema = registry.getSchema(typeDescriptor); + + final Schema expectedInnerSchema = + Schema.builder() + .addBooleanField("value1") + .addStringField("value2") + .addInt64Field("value3") + .addRowField("value4", SIMPLE_POJO_SCHEMA) + .build(); + final Schema expectedSchema = + Schema.builder().addRowField("nested", expectedInnerSchema).build(); + assertTrue(expectedSchema.equivalent(schema)); + } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtilsTest.java index 021e39b84849..e0a45c2c82fe 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtilsTest.java @@ -34,6 +34,7 @@ import java.math.BigDecimal; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; +import java.util.Collections; import java.util.List; import org.apache.beam.sdk.schemas.FieldValueGetter; import org.apache.beam.sdk.schemas.FieldValueSetter; @@ -65,7 +66,9 @@ public class JavaBeanUtilsTest { public void testNullable() { Schema schema = JavaBeanUtils.schemaFromJavaBeanClass( - new TypeDescriptor() {}, GetterTypeSupplier.INSTANCE); + new TypeDescriptor() {}, + GetterTypeSupplier.INSTANCE, + Collections.emptyMap()); assertTrue(schema.getField("str").getType().getNullable()); assertFalse(schema.getField("anInt").getType().getNullable()); } @@ -74,7 +77,9 @@ public void testNullable() { public void testSimpleBean() { Schema schema = JavaBeanUtils.schemaFromJavaBeanClass( - new TypeDescriptor() {}, GetterTypeSupplier.INSTANCE); + new TypeDescriptor() {}, + GetterTypeSupplier.INSTANCE, + Collections.emptyMap()); SchemaTestUtils.assertSchemaEquivalent(SIMPLE_BEAN_SCHEMA, schema); } @@ -82,7 +87,9 @@ public void testSimpleBean() { public void testNestedBean() { Schema schema = JavaBeanUtils.schemaFromJavaBeanClass( - new TypeDescriptor() {}, GetterTypeSupplier.INSTANCE); + new TypeDescriptor() {}, + GetterTypeSupplier.INSTANCE, + Collections.emptyMap()); SchemaTestUtils.assertSchemaEquivalent(NESTED_BEAN_SCHEMA, schema); } @@ -90,7 +97,9 @@ public void testNestedBean() { public void testPrimitiveArray() { Schema schema = JavaBeanUtils.schemaFromJavaBeanClass( - new TypeDescriptor() {}, GetterTypeSupplier.INSTANCE); + new TypeDescriptor() {}, + GetterTypeSupplier.INSTANCE, + Collections.emptyMap()); SchemaTestUtils.assertSchemaEquivalent(PRIMITIVE_ARRAY_BEAN_SCHEMA, schema); } @@ -98,7 +107,9 @@ public void testPrimitiveArray() { public void testNestedArray() { Schema schema = JavaBeanUtils.schemaFromJavaBeanClass( - new TypeDescriptor() {}, GetterTypeSupplier.INSTANCE); + new TypeDescriptor() {}, + GetterTypeSupplier.INSTANCE, + Collections.emptyMap()); SchemaTestUtils.assertSchemaEquivalent(NESTED_ARRAY_BEAN_SCHEMA, schema); } @@ -106,7 +117,9 @@ public void testNestedArray() { public void testNestedCollection() { Schema schema = JavaBeanUtils.schemaFromJavaBeanClass( - new TypeDescriptor() {}, GetterTypeSupplier.INSTANCE); + new TypeDescriptor() {}, + GetterTypeSupplier.INSTANCE, + Collections.emptyMap()); SchemaTestUtils.assertSchemaEquivalent(NESTED_COLLECTION_BEAN_SCHEMA, schema); } @@ -114,7 +127,9 @@ public void testNestedCollection() { public void testPrimitiveMap() { Schema schema = JavaBeanUtils.schemaFromJavaBeanClass( - new TypeDescriptor() {}, GetterTypeSupplier.INSTANCE); + new TypeDescriptor() {}, + GetterTypeSupplier.INSTANCE, + Collections.emptyMap()); SchemaTestUtils.assertSchemaEquivalent(PRIMITIVE_MAP_BEAN_SCHEMA, schema); } @@ -122,7 +137,9 @@ public void testPrimitiveMap() { public void testNestedMap() { Schema schema = JavaBeanUtils.schemaFromJavaBeanClass( - new TypeDescriptor() {}, GetterTypeSupplier.INSTANCE); + new TypeDescriptor() {}, + GetterTypeSupplier.INSTANCE, + Collections.emptyMap()); SchemaTestUtils.assertSchemaEquivalent(NESTED_MAP_BEAN_SCHEMA, schema); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/POJOUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/POJOUtilsTest.java index 723353ed8d15..46c098dddaeb 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/POJOUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/POJOUtilsTest.java @@ -35,6 +35,7 @@ import java.math.BigDecimal; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; +import java.util.Collections; import java.util.List; import org.apache.beam.sdk.schemas.FieldValueGetter; import org.apache.beam.sdk.schemas.FieldValueSetter; @@ -71,7 +72,9 @@ public class POJOUtilsTest { public void testNullables() { Schema schema = POJOUtils.schemaFromPojoClass( - new TypeDescriptor() {}, JavaFieldTypeSupplier.INSTANCE); + new TypeDescriptor() {}, + JavaFieldTypeSupplier.INSTANCE, + Collections.emptyMap()); assertTrue(schema.getField("str").getType().getNullable()); assertFalse(schema.getField("anInt").getType().getNullable()); } @@ -80,7 +83,9 @@ public void testNullables() { public void testSimplePOJO() { Schema schema = POJOUtils.schemaFromPojoClass( - new TypeDescriptor() {}, JavaFieldTypeSupplier.INSTANCE); + new TypeDescriptor() {}, + JavaFieldTypeSupplier.INSTANCE, + Collections.emptyMap()); assertEquals(SIMPLE_POJO_SCHEMA, schema); } @@ -88,7 +93,9 @@ public void testSimplePOJO() { public void testNestedPOJO() { Schema schema = POJOUtils.schemaFromPojoClass( - new TypeDescriptor() {}, JavaFieldTypeSupplier.INSTANCE); + new TypeDescriptor() {}, + JavaFieldTypeSupplier.INSTANCE, + Collections.emptyMap()); SchemaTestUtils.assertSchemaEquivalent(NESTED_POJO_SCHEMA, schema); } @@ -97,7 +104,8 @@ public void testNestedPOJOWithSimplePOJO() { Schema schema = POJOUtils.schemaFromPojoClass( new TypeDescriptor() {}, - JavaFieldTypeSupplier.INSTANCE); + JavaFieldTypeSupplier.INSTANCE, + Collections.emptyMap()); SchemaTestUtils.assertSchemaEquivalent(NESTED_POJO_WITH_SIMPLE_POJO_SCHEMA, schema); } @@ -105,7 +113,9 @@ public void testNestedPOJOWithSimplePOJO() { public void testPrimitiveArray() { Schema schema = POJOUtils.schemaFromPojoClass( - new TypeDescriptor() {}, JavaFieldTypeSupplier.INSTANCE); + new TypeDescriptor() {}, + JavaFieldTypeSupplier.INSTANCE, + Collections.emptyMap()); SchemaTestUtils.assertSchemaEquivalent(PRIMITIVE_ARRAY_POJO_SCHEMA, schema); } @@ -113,7 +123,9 @@ public void testPrimitiveArray() { public void testNestedArray() { Schema schema = POJOUtils.schemaFromPojoClass( - new TypeDescriptor() {}, JavaFieldTypeSupplier.INSTANCE); + new TypeDescriptor() {}, + JavaFieldTypeSupplier.INSTANCE, + Collections.emptyMap()); SchemaTestUtils.assertSchemaEquivalent(NESTED_ARRAY_POJO_SCHEMA, schema); } @@ -121,7 +133,9 @@ public void testNestedArray() { public void testNestedCollection() { Schema schema = POJOUtils.schemaFromPojoClass( - new TypeDescriptor() {}, JavaFieldTypeSupplier.INSTANCE); + new TypeDescriptor() {}, + JavaFieldTypeSupplier.INSTANCE, + Collections.emptyMap()); SchemaTestUtils.assertSchemaEquivalent(NESTED_COLLECTION_POJO_SCHEMA, schema); } @@ -129,7 +143,9 @@ public void testNestedCollection() { public void testPrimitiveMap() { Schema schema = POJOUtils.schemaFromPojoClass( - new TypeDescriptor() {}, JavaFieldTypeSupplier.INSTANCE); + new TypeDescriptor() {}, + JavaFieldTypeSupplier.INSTANCE, + Collections.emptyMap()); SchemaTestUtils.assertSchemaEquivalent(PRIMITIVE_MAP_POJO_SCHEMA, schema); } @@ -137,7 +153,9 @@ public void testPrimitiveMap() { public void testNestedMap() { Schema schema = POJOUtils.schemaFromPojoClass( - new TypeDescriptor() {}, JavaFieldTypeSupplier.INSTANCE); + new TypeDescriptor() {}, + JavaFieldTypeSupplier.INSTANCE, + Collections.emptyMap()); SchemaTestUtils.assertSchemaEquivalent(NESTED_MAP_POJO_SCHEMA, schema); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestJavaBeans.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestJavaBeans.java index b5ad6f989d9e..cbc976144971 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestJavaBeans.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestJavaBeans.java @@ -1397,4 +1397,95 @@ public void setValue(@Nullable Float value) { Schema.Field.nullable("value", FieldType.FLOAT) .withDescription("This value is the value stored in the object as a float.")) .build(); + + @DefaultSchema(JavaBeanSchema.class) + public static class SimpleParameterizedBean { + @Nullable private W value1; + @Nullable private T value2; + @Nullable private V value3; + @Nullable private X value4; + + public W getValue1() { + return value1; + } + + public void setValue1(W value1) { + this.value1 = value1; + } + + public T getValue2() { + return value2; + } + + public void setValue2(T value2) { + this.value2 = value2; + } + + public V getValue3() { + return value3; + } + + public void setValue3(V value3) { + this.value3 = value3; + } + + public X getValue4() { + return value4; + } + + public void setValue4(X value4) { + this.value4 = value4; + } + } + + @DefaultSchema(JavaBeanSchema.class) + public static class SimpleParameterizedBeanSubclass + extends SimpleParameterizedBean { + @Nullable private T value5; + + public SimpleParameterizedBeanSubclass() {} + + public T getValue5() { + return value5; + } + + public void setValue5(T value5) { + this.value5 = value5; + } + } + + @DefaultSchema(JavaBeanSchema.class) + public static class NestedParameterizedCollectionBean { + private Iterable nested; + private Map map; + + public Iterable getNested() { + return nested; + } + + public Map getMap() { + return map; + } + + public void setNested(Iterable nested) { + this.nested = nested; + } + + public void setMap(Map map) { + this.map = map; + } + } + + @DefaultSchema(JavaBeanSchema.class) + public static class NestedParameterizedBean { + private T nested; + + public T getNested() { + return nested; + } + + public void setNested(T nested) { + this.nested = nested; + } + } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestPOJOs.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestPOJOs.java index 789de02adee8..ce7409365d09 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestPOJOs.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestPOJOs.java @@ -495,6 +495,125 @@ public int hashCode() { .addStringField("stringBuilder") .build(); + @DefaultSchema(JavaFieldSchema.class) + public static class SimpleParameterizedPOJO { + public W value1; + public T value2; + public V value3; + public X value4; + + public SimpleParameterizedPOJO() {} + + public SimpleParameterizedPOJO(W value1, T value2, V value3, X value4) { + this.value1 = value1; + this.value2 = value2; + this.value3 = value3; + this.value4 = value4; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof SimpleParameterizedPOJO)) { + return false; + } + SimpleParameterizedPOJO that = (SimpleParameterizedPOJO) o; + return Objects.equals(value1, that.value1) + && Objects.equals(value2, that.value2) + && Objects.equals(value3, that.value3) + && Objects.equals(value4, that.value4); + } + + @Override + public int hashCode() { + return Objects.hash(value1, value2, value3, value4); + } + } + + @DefaultSchema(JavaFieldSchema.class) + public static class SimpleParameterizedPOJOSubclass + extends SimpleParameterizedPOJO { + public T value5; + + public SimpleParameterizedPOJOSubclass() {} + + public SimpleParameterizedPOJOSubclass(T value5) { + this.value5 = value5; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof SimpleParameterizedPOJOSubclass)) { + return false; + } + SimpleParameterizedPOJOSubclass that = (SimpleParameterizedPOJOSubclass) o; + return Objects.equals(value5, that.value5); + } + + @Override + public int hashCode() { + return Objects.hash(value4); + } + } + + @DefaultSchema(JavaFieldSchema.class) + public static class NestedParameterizedCollectionPOJO { + public Iterable nested; + public Map map; + + public NestedParameterizedCollectionPOJO(Iterable nested, Map map) { + this.nested = nested; + this.map = map; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof NestedParameterizedCollectionPOJO)) { + return false; + } + NestedParameterizedCollectionPOJO that = (NestedParameterizedCollectionPOJO) o; + return Objects.equals(nested, that.nested) && Objects.equals(map, that.map); + } + + @Override + public int hashCode() { + return Objects.hash(nested, map); + } + } + + @DefaultSchema(JavaFieldSchema.class) + public static class NestedParameterizedPOJO { + public T nested; + + public NestedParameterizedPOJO(T nested) { + this.nested = nested; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof NestedParameterizedPOJO)) { + return false; + } + NestedParameterizedPOJO that = (NestedParameterizedPOJO) o; + return Objects.equals(nested, that.nested); + } + + @Override + public int hashCode() { + return Objects.hash(nested); + } + } /** A POJO containing a nested class. * */ @DefaultSchema(JavaFieldSchema.class) public static class NestedPOJO { @@ -887,7 +1006,7 @@ public boolean equals(@Nullable Object o) { if (this == o) { return true; } - if (!(o instanceof PojoWithNestedArray)) { + if (!(o instanceof PojoWithIterable)) { return false; } PojoWithIterable that = (PojoWithIterable) o; diff --git a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroByteBuddyUtils.java b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroByteBuddyUtils.java index 0a82663c1771..1a530a3f6ca5 100644 --- a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroByteBuddyUtils.java +++ b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroByteBuddyUtils.java @@ -78,8 +78,8 @@ private static SchemaUserTypeCreator createCreator(Class clazz, Schema sc // Generate a method call to create and invoke the SpecificRecord's constructor. . MethodCall construct = MethodCall.construct(baseConstructor); - for (int i = 0; i < baseConstructor.getParameterTypes().length; ++i) { - Class baseType = baseConstructor.getParameterTypes()[i]; + for (int i = 0; i < baseConstructor.getGenericParameterTypes().length; ++i) { + Type baseType = baseConstructor.getGenericParameterTypes()[i]; construct = construct.with(readAndConvertParameter(baseType, i), baseType); } @@ -110,7 +110,7 @@ private static SchemaUserTypeCreator createCreator(Class clazz, Schema sc } private static StackManipulation readAndConvertParameter( - Class constructorParameterType, int index) { + Type constructorParameterType, int index) { TypeConversionsFactory typeConversionsFactory = new AvroUtils.AvroTypeConversionFactory(); // The types in the AVRO-generated constructor might be the types returned by Beam's Row class, diff --git a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtils.java b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtils.java index 1b1c45969307..1324d254e44e 100644 --- a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtils.java +++ b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtils.java @@ -814,6 +814,9 @@ public List get(TypeDescriptor typeDescriptor) { @Override public List get(TypeDescriptor typeDescriptor, Schema schema) { + Map boundTypes = + ReflectUtils.getAllBoundTypes(typeDescriptor); + Map mapping = getMapping(schema); List methods = ReflectUtils.getMethods(typeDescriptor.getRawType()); List types = Lists.newArrayList(); @@ -821,7 +824,7 @@ public List get(TypeDescriptor typeDescriptor, Sch Method method = methods.get(i); if (ReflectUtils.isGetter(method)) { FieldValueTypeInformation fieldValueTypeInformation = - FieldValueTypeInformation.forGetter(method, i); + FieldValueTypeInformation.forGetter(method, i, boundTypes); String name = mapping.get(fieldValueTypeInformation.getName()); if (name != null) { types.add(fieldValueTypeInformation.withName(name)); @@ -865,13 +868,16 @@ private Map getMapping(Schema schema) { private static final class AvroPojoFieldValueTypeSupplier implements FieldValueTypeSupplier { @Override public List get(TypeDescriptor typeDescriptor) { + Map boundTypes = + ReflectUtils.getAllBoundTypes(typeDescriptor); List classFields = ReflectUtils.getFields(typeDescriptor.getRawType()); Map types = Maps.newHashMap(); for (int i = 0; i < classFields.size(); ++i) { java.lang.reflect.Field f = classFields.get(i); if (!f.isAnnotationPresent(AvroIgnore.class)) { - FieldValueTypeInformation typeInformation = FieldValueTypeInformation.forField(f, i); + FieldValueTypeInformation typeInformation = + FieldValueTypeInformation.forField(f, i, boundTypes); AvroName avroname = f.getAnnotation(AvroName.class); if (avroname != null) { typeInformation = typeInformation.withName(avroname.value()); diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteBuddyUtils.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteBuddyUtils.java index d159e9de44a8..fcfc40403b43 100644 --- a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteBuddyUtils.java +++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteBuddyUtils.java @@ -39,6 +39,7 @@ import java.lang.reflect.Modifier; import java.lang.reflect.Type; import java.util.Arrays; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -1045,7 +1046,8 @@ FieldValueSetter getProtoFieldValueSetter( } else { Method method = getProtoSetter(methods, field.getName(), field.getType()); return JavaBeanUtils.createSetter( - FieldValueTypeInformation.forSetter(method, protoSetterPrefix(field.getType())), + FieldValueTypeInformation.forSetter( + method, protoSetterPrefix(field.getType()), Collections.emptyMap()), new ProtoTypeConversionsFactory()); } } diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoMessageSchema.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoMessageSchema.java index faf3ad407af5..4b8d51abdea6 100644 --- a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoMessageSchema.java +++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoMessageSchema.java @@ -23,6 +23,7 @@ import com.google.protobuf.DynamicMessage; import com.google.protobuf.Message; import java.lang.reflect.Method; +import java.util.Collections; import java.util.List; import java.util.Map; import org.apache.beam.sdk.extensions.protobuf.ProtoByteBuddyUtils.ProtoTypeConversionsFactory; @@ -72,7 +73,8 @@ public List get(TypeDescriptor typeDescriptor, Sch Method method = getProtoGetter(methods, oneOfField.getName(), oneOfField.getType()); oneOfTypes.put( oneOfField.getName(), - FieldValueTypeInformation.forGetter(method, i).withName(field.getName())); + FieldValueTypeInformation.forGetter(method, i, Collections.emptyMap()) + .withName(field.getName())); } // Add an entry that encapsulates information about all possible getters. types.add( @@ -82,7 +84,9 @@ public List get(TypeDescriptor typeDescriptor, Sch } else { // This is a simple field. Add the getter. Method method = getProtoGetter(methods, field.getName(), field.getType()); - types.add(FieldValueTypeInformation.forGetter(method, i).withName(field.getName())); + types.add( + FieldValueTypeInformation.forGetter(method, i, Collections.emptyMap()) + .withName(field.getName())); } } return types; diff --git a/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java b/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java index d5f1745a9a2c..64f600903d87 100644 --- a/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java +++ b/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java @@ -25,6 +25,7 @@ import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -389,7 +390,8 @@ private Schema generateSchemaDirectly( fieldName, StaticSchemaInference.fieldFromType( TypeDescriptor.of(field.getClass()), - JavaFieldSchema.JavaFieldTypeSupplier.INSTANCE)); + JavaFieldSchema.JavaFieldTypeSupplier.INSTANCE, + Collections.emptyMap())); } counter++; diff --git a/sdks/java/io/thrift/src/main/java/org/apache/beam/sdk/io/thrift/ThriftSchema.java b/sdks/java/io/thrift/src/main/java/org/apache/beam/sdk/io/thrift/ThriftSchema.java index 5f4e195f227f..73b3709da832 100644 --- a/sdks/java/io/thrift/src/main/java/org/apache/beam/sdk/io/thrift/ThriftSchema.java +++ b/sdks/java/io/thrift/src/main/java/org/apache/beam/sdk/io/thrift/ThriftSchema.java @@ -242,10 +242,11 @@ private FieldValueTypeInformation fieldValueTypeInfo(Class type, String field if (factoryMethods.size() > 1) { throw new IllegalStateException("Overloaded factory methods: " + factoryMethods); } - return FieldValueTypeInformation.forSetter(factoryMethods.get(0), ""); + return FieldValueTypeInformation.forSetter(factoryMethods.get(0), "", Collections.emptyMap()); } else { try { - return FieldValueTypeInformation.forField(type.getDeclaredField(fieldName), 0); + return FieldValueTypeInformation.forField( + type.getDeclaredField(fieldName), 0, Collections.emptyMap()); } catch (NoSuchFieldException e) { throw new IllegalArgumentException(e); } From 7d0bfd0d6ef453fea4672d3c7752cc02e22de351 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 15 Oct 2024 22:38:59 -0700 Subject: [PATCH 56/78] Bump google.golang.org/protobuf from 1.34.2 to 1.35.1 in /sdks (#32799) Bumps google.golang.org/protobuf from 1.34.2 to 1.35.1. --- updated-dependencies: - dependency-name: google.golang.org/protobuf dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 0b5ac98df404..9aa839d67e1a 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -61,7 +61,7 @@ require ( google.golang.org/api v0.199.0 google.golang.org/genproto v0.0.0-20240903143218-8af14fe29dc1 google.golang.org/grpc v1.67.1 - google.golang.org/protobuf v1.34.2 + google.golang.org/protobuf v1.35.1 gopkg.in/yaml.v2 v2.4.0 gopkg.in/yaml.v3 v3.0.1 ) diff --git a/sdks/go.sum b/sdks/go.sum index db6d71b061b5..51e4d58c237d 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -1913,8 +1913,8 @@ google.golang.org/protobuf v1.28.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqw google.golang.org/protobuf v1.28.1/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= google.golang.org/protobuf v1.29.1/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= google.golang.org/protobuf v1.30.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= -google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6hg= -google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= +google.golang.org/protobuf v1.35.1 h1:m3LfL6/Ca+fqnjnlqQXNpFPABW1UD7mjh8KO2mKFytA= +google.golang.org/protobuf v1.35.1/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= From 6b3a1b29f264b829745309b0b43639bd3625da76 Mon Sep 17 00:00:00 2001 From: claudevdm <33973061+claudevdm@users.noreply.github.com> Date: Wed, 16 Oct 2024 09:06:02 -0400 Subject: [PATCH 57/78] Bigquery fixes (#32780) * Bigquery fixes * Remove unnecessary comprehension loop --------- Co-authored-by: Claude --- .../enrichment_handlers/bigquery.py | 19 +- .../enrichment_handlers/bigquery_it_test.py | 300 +++++++++++------- 2 files changed, 199 insertions(+), 120 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigquery.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigquery.py index 382ae123a81d..ea98fb6b0bbd 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigquery.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigquery.py @@ -171,6 +171,14 @@ def _execute_query(self, query: str): except RuntimeError as e: raise RuntimeError(f"Could not complete the query request: {query}. {e}") + def create_row_key(self, row: beam.Row): + if self.condition_value_fn: + return tuple(self.condition_value_fn(row)) + if self.fields: + row_dict = row._asdict() + return (tuple(row_dict[field] for field in self.fields)) + raise ValueError("Either fields or condition_value_fn must be specified") + def __call__(self, request: Union[beam.Row, List[beam.Row]], *args, **kwargs): if isinstance(request, List): values = [] @@ -180,7 +188,7 @@ def __call__(self, request: Union[beam.Row, List[beam.Row]], *args, **kwargs): raw_query = self.query_template if batch_size > 1: batched_condition_template = ' or '.join( - [self.row_restriction_template] * batch_size) + [fr'({self.row_restriction_template})'] * batch_size) raw_query = self.query_template.replace( self.row_restriction_template, batched_condition_template) for req in request: @@ -194,14 +202,15 @@ def __call__(self, request: Union[beam.Row, List[beam.Row]], *args, **kwargs): "Make sure the values passed in `fields` are the " "keys in the input `beam.Row`." + str(e)) values.extend(current_values) - requests_map.update((val, req) for val in current_values) + requests_map[self.create_row_key(req)] = req query = raw_query.format(*values) responses_dict = self._execute_query(query) for response in responses_dict: - for value in response.values(): - if value in requests_map: - responses.append((requests_map[value], beam.Row(**response))) + response_row = beam.Row(**response) + response_key = self.create_row_key(response_row) + if response_key in requests_map: + responses.append((requests_map[response_key], response_row)) return responses else: request_dict = request._asdict() 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 0b8a384b934d..dd99e386555e 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 @@ -14,7 +14,10 @@ # See the License for the specific language governing permissions and # limitations under the License. # +import functools import logging +import secrets +import time import unittest from unittest.mock import MagicMock @@ -22,7 +25,11 @@ import apache_beam as beam from apache_beam.coders import coders +from apache_beam.io.gcp.bigquery_tools import BigQueryWrapper +from apache_beam.io.gcp.internal.clients import bigquery from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.testing.util import assert_that +from apache_beam.testing.util import equal_to # pylint: disable=ungrouped-imports try: @@ -31,8 +38,7 @@ from apache_beam.transforms.enrichment import Enrichment from apache_beam.transforms.enrichment_handlers.bigquery import \ BigQueryEnrichmentHandler - from apache_beam.transforms.enrichment_handlers.vertex_ai_feature_store_it_test import \ - ValidateResponse + from apitools.base.py.exceptions import HttpError except ImportError: raise unittest.SkipTest( 'Google Cloud BigQuery dependencies are not installed.') @@ -40,24 +46,101 @@ _LOGGER = logging.getLogger(__name__) -def query_fn(row: beam.Row): - query = ( - "SELECT * FROM " - "`apache-beam-testing.my_ecommerce.product_details`" - " WHERE id = '{}'".format(row.id)) # type: ignore[attr-defined] - return query - - def condition_value_fn(row: beam.Row): return [row.id] # type: ignore[attr-defined] +def query_fn(table, row: beam.Row): + return f"SELECT * FROM `{table}` WHERE id = {row.id}" # type: ignore[attr-defined] + + +@pytest.mark.uses_testcontainer +class BigQueryEnrichmentIT(unittest.TestCase): + bigquery_dataset_id = 'python_enrichment_transform_read_table_' + project = "apache-beam-testing" + + @classmethod + def setUpClass(cls): + cls.bigquery_client = BigQueryWrapper() + cls.dataset_id = '%s%d%s' % ( + cls.bigquery_dataset_id, int(time.time()), secrets.token_hex(3)) + cls.bigquery_client.get_or_create_dataset(cls.project, cls.dataset_id) + _LOGGER.info( + "Created dataset %s in project %s", cls.dataset_id, cls.project) + + @classmethod + def tearDownClass(cls): + request = bigquery.BigqueryDatasetsDeleteRequest( + projectId=cls.project, datasetId=cls.dataset_id, deleteContents=True) + try: + _LOGGER.debug( + "Deleting dataset %s in project %s", cls.dataset_id, cls.project) + cls.bigquery_client.client.datasets.Delete(request) + except HttpError: + _LOGGER.warning( + 'Failed to clean up dataset %s in project %s', + cls.dataset_id, + cls.project) + + @pytest.mark.uses_testcontainer -class TestBigQueryEnrichmentIT(unittest.TestCase): +class TestBigQueryEnrichmentIT(BigQueryEnrichmentIT): + table_data = [ + { + "id": 1, "name": "A", 'quantity': 2, 'distribution_center_id': 3 + }, + { + "id": 2, "name": "B", 'quantity': 3, 'distribution_center_id': 1 + }, + { + "id": 3, "name": "C", 'quantity': 10, 'distribution_center_id': 4 + }, + { + "id": 4, "name": "D", 'quantity': 1, 'distribution_center_id': 3 + }, + { + "id": 5, "name": "C", 'quantity': 100, 'distribution_center_id': 4 + }, + { + "id": 6, "name": "D", 'quantity': 11, 'distribution_center_id': 3 + }, + { + "id": 7, "name": "C", 'quantity': 7, 'distribution_center_id': 1 + }, + { + "id": 8, "name": "D", 'quantity': 4, 'distribution_center_id': 1 + }, + ] + + @classmethod + def create_table(cls, table_name): + fields = [('id', 'INTEGER'), ('name', 'STRING'), ('quantity', 'INTEGER'), + ('distribution_center_id', 'INTEGER')] + table_schema = bigquery.TableSchema() + for name, field_type in fields: + table_field = bigquery.TableFieldSchema() + table_field.name = name + table_field.type = field_type + table_schema.fields.append(table_field) + table = bigquery.Table( + tableReference=bigquery.TableReference( + projectId=cls.project, datasetId=cls.dataset_id, + tableId=table_name), + schema=table_schema) + request = bigquery.BigqueryTablesInsertRequest( + projectId=cls.project, datasetId=cls.dataset_id, table=table) + cls.bigquery_client.client.tables.Insert(request) + cls.bigquery_client.insert_rows( + cls.project, cls.dataset_id, table_name, cls.table_data) + cls.table_name = f"{cls.project}.{cls.dataset_id}.{table_name}" + + @classmethod + def setUpClass(cls): + super(TestBigQueryEnrichmentIT, cls).setUpClass() + cls.create_table('product_details') + def setUp(self) -> None: - self.project = 'apache-beam-testing' - self.condition_template = "id = '{}'" - self.table_name = "`apache-beam-testing.my_ecommerce.product_details`" + self.condition_template = "id = {}" self.retries = 3 self._start_container() @@ -82,123 +165,119 @@ def tearDown(self) -> None: self.client = None def test_bigquery_enrichment(self): - expected_fields = [ - 'id', 'name', 'quantity', 'category', 'brand', 'cost', 'retail_price' + expected_rows = [ + beam.Row(id=1, name="A", quantity=2, distribution_center_id=3), + beam.Row(id=2, name="B", quantity=3, distribution_center_id=1) ] fields = ['id'] requests = [ - beam.Row( - id='13842', - name='low profile dyed cotton twill cap - navy w39s55d', - quantity=2), - beam.Row( - id='15816', - name='low profile dyed cotton twill cap - putty w39s55d', - quantity=1), + beam.Row(id=1, name='A'), + beam.Row(id=2, name='B'), ] handler = BigQueryEnrichmentHandler( project=self.project, - row_restriction_template=self.condition_template, + row_restriction_template="id = {}", table_name=self.table_name, fields=fields, - min_batch_size=2, + min_batch_size=1, max_batch_size=100, ) + with TestPipeline(is_integration_test=True) as test_pipeline: - _ = ( - test_pipeline - | beam.Create(requests) - | Enrichment(handler) - | beam.ParDo(ValidateResponse(expected_fields))) + pcoll = (test_pipeline | beam.Create(requests) | Enrichment(handler)) - def test_bigquery_enrichment_with_query_fn(self): - expected_fields = [ - 'id', 'name', 'quantity', 'category', 'brand', 'cost', 'retail_price' + assert_that(pcoll, equal_to(expected_rows)) + + def test_bigquery_enrichment_batched(self): + expected_rows = [ + beam.Row(id=1, name="A", quantity=2, distribution_center_id=3), + beam.Row(id=2, name="B", quantity=3, distribution_center_id=1) ] + fields = ['id'] requests = [ - beam.Row( - id='13842', - name='low profile dyed cotton twill cap - navy w39s55d', - quantity=2), - beam.Row( - id='15816', - name='low profile dyed cotton twill cap - putty w39s55d', - quantity=1), + beam.Row(id=1, name='A'), + beam.Row(id=2, name='B'), ] - handler = BigQueryEnrichmentHandler(project=self.project, query_fn=query_fn) + handler = BigQueryEnrichmentHandler( + project=self.project, + row_restriction_template="id = {}", + table_name=self.table_name, + fields=fields, + min_batch_size=2, + max_batch_size=100, + ) + with TestPipeline(is_integration_test=True) as test_pipeline: - _ = ( - test_pipeline - | beam.Create(requests) - | Enrichment(handler) - | beam.ParDo(ValidateResponse(expected_fields))) + pcoll = (test_pipeline | beam.Create(requests) | Enrichment(handler)) - def test_bigquery_enrichment_with_condition_value_fn(self): - expected_fields = [ - 'id', 'name', 'quantity', 'category', 'brand', 'cost', 'retail_price' + assert_that(pcoll, equal_to(expected_rows)) + + def test_bigquery_enrichment_batched_multiple_fields(self): + expected_rows = [ + beam.Row(id=1, distribution_center_id=3, name="A", quantity=2), + beam.Row(id=2, distribution_center_id=1, name="B", quantity=3) ] + fields = ['id', 'distribution_center_id'] requests = [ - beam.Row( - id='13842', - name='low profile dyed cotton twill cap - navy w39s55d', - quantity=2), - beam.Row( - id='15816', - name='low profile dyed cotton twill cap - putty w39s55d', - quantity=1), + beam.Row(id=1, distribution_center_id=3), + beam.Row(id=2, distribution_center_id=1), ] handler = BigQueryEnrichmentHandler( project=self.project, - row_restriction_template=self.condition_template, + row_restriction_template="id = {} AND distribution_center_id = {}", table_name=self.table_name, - condition_value_fn=condition_value_fn, - min_batch_size=2, + fields=fields, + min_batch_size=8, max_batch_size=100, ) + with TestPipeline(is_integration_test=True) as test_pipeline: - _ = ( - test_pipeline - | beam.Create(requests) - | Enrichment(handler) - | beam.ParDo(ValidateResponse(expected_fields))) + pcoll = (test_pipeline | beam.Create(requests) | Enrichment(handler)) - def test_bigquery_enrichment_with_condition_without_batch(self): - expected_fields = [ - 'id', 'name', 'quantity', 'category', 'brand', 'cost', 'retail_price' + assert_that(pcoll, equal_to(expected_rows)) + + def test_bigquery_enrichment_with_query_fn(self): + expected_rows = [ + beam.Row(id=1, name="A", quantity=2, distribution_center_id=3), + beam.Row(id=2, name="B", quantity=3, distribution_center_id=1) ] requests = [ - beam.Row( - id='13842', - name='low profile dyed cotton twill cap - navy w39s55d', - quantity=2), - beam.Row( - id='15816', - name='low profile dyed cotton twill cap - putty w39s55d', - quantity=1), + beam.Row(id=1, name='A'), + beam.Row(id=2, name='B'), + ] + fn = functools.partial(query_fn, self.table_name) + handler = BigQueryEnrichmentHandler(project=self.project, query_fn=fn) + with TestPipeline(is_integration_test=True) as test_pipeline: + pcoll = (test_pipeline | beam.Create(requests) | Enrichment(handler)) + + assert_that(pcoll, equal_to(expected_rows)) + + def test_bigquery_enrichment_with_condition_value_fn(self): + expected_rows = [ + beam.Row(id=1, name="A", quantity=2, distribution_center_id=3), + beam.Row(id=2, name="B", quantity=3, distribution_center_id=1) + ] + requests = [ + beam.Row(id=1, name='A'), + beam.Row(id=2, name='B'), ] handler = BigQueryEnrichmentHandler( project=self.project, row_restriction_template=self.condition_template, table_name=self.table_name, condition_value_fn=condition_value_fn, + min_batch_size=2, + max_batch_size=100, ) with TestPipeline(is_integration_test=True) as test_pipeline: - _ = ( - test_pipeline - | beam.Create(requests) - | Enrichment(handler) - | beam.ParDo(ValidateResponse(expected_fields))) + pcoll = (test_pipeline | beam.Create(requests) | Enrichment(handler)) + + assert_that(pcoll, equal_to(expected_rows)) def test_bigquery_enrichment_bad_request(self): requests = [ - beam.Row( - id='13842', - name='low profile dyed cotton twill cap - navy w39s55d', - quantity=2), - beam.Row( - id='15816', - name='low profile dyed cotton twill cap - putty w39s55d', - quantity=1), + beam.Row(id=1, name='A'), + beam.Row(id=2, name='B'), ] handler = BigQueryEnrichmentHandler( project=self.project, @@ -231,18 +310,13 @@ def test_bigquery_enrichment_with_redis(self): requests. Since all requests are cached, it will return from there without making calls to the BigQuery service. """ - expected_fields = [ - 'id', 'name', 'quantity', 'category', 'brand', 'cost', 'retail_price' - ] requests = [ - beam.Row( - id='13842', - name='low profile dyed cotton twill cap - navy w39s55d', - quantity=2), - beam.Row( - id='15816', - name='low profile dyed cotton twill cap - putty w39s55d', - quantity=1), + beam.Row(id=1, name='A'), + beam.Row(id=2, name='B'), + ] + expected_rows = [ + beam.Row(id=1, name="A", quantity=2, distribution_center_id=3), + beam.Row(id=2, name="B", quantity=3, distribution_center_id=1) ] handler = BigQueryEnrichmentHandler( project=self.project, @@ -253,11 +327,12 @@ def test_bigquery_enrichment_with_redis(self): max_batch_size=100, ) with TestPipeline(is_integration_test=True) as test_pipeline: - _ = ( + pcoll_populate_cache = ( test_pipeline | beam.Create(requests) - | Enrichment(handler).with_redis_cache(self.host, self.port) - | beam.ParDo(ValidateResponse(expected_fields))) + | Enrichment(handler).with_redis_cache(self.host, self.port)) + + assert_that(pcoll_populate_cache, equal_to(expected_rows)) # manually check cache entry c = coders.StrUtf8Coder() @@ -268,20 +343,15 @@ def test_bigquery_enrichment_with_redis(self): raise ValueError("No cache entry found for %s" % key) actual = BigQueryEnrichmentHandler.__call__ - BigQueryEnrichmentHandler.__call__ = MagicMock( - return_value=( - beam.Row( - id='15816', - name='low profile dyed cotton twill cap - putty w39s55d', - quantity=1), - beam.Row())) + BigQueryEnrichmentHandler.__call__ = MagicMock(return_value=(beam.Row())) with TestPipeline(is_integration_test=True) as test_pipeline: - _ = ( + pcoll_cached = ( test_pipeline | beam.Create(requests) - | Enrichment(handler).with_redis_cache(self.host, self.port) - | beam.ParDo(ValidateResponse(expected_fields))) + | Enrichment(handler).with_redis_cache(self.host, self.port)) + + assert_that(pcoll_cached, equal_to(expected_rows)) BigQueryEnrichmentHandler.__call__ = actual From 3e49714c9dc9ee0902bfae05719ca94199ce509a Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Wed, 16 Oct 2024 10:06:30 -0400 Subject: [PATCH 58/78] Change to rez --- website/www/site/content/en/blog/beam-summit-2024-overview.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/website/www/site/content/en/blog/beam-summit-2024-overview.md b/website/www/site/content/en/blog/beam-summit-2024-overview.md index 8d5eb209bea2..5cf922d69544 100644 --- a/website/www/site/content/en/blog/beam-summit-2024-overview.md +++ b/website/www/site/content/en/blog/beam-summit-2024-overview.md @@ -8,7 +8,7 @@ aliases: authors: - liferoad - damccorm - - rezarokni + - rez --- + +We are happy to present the new 2.60.0 release of Beam. +This release includes both improvements and new functionality. +See the [download page](/get-started/downloads/#2600-2024-10-17) for this release. + + + +For more information on changes in 2.60.0, check out the [detailed release notes](https://github.com/apache/beam/milestone/24). + +## Highlights + +* Added support for using vLLM in the RunInference transform (Python) ([#32528](https://github.com/apache/beam/issues/32528)) +* [Managed Iceberg] Added support for streaming writes ([#32451](https://github.com/apache/beam/pull/32451)) +* [Managed Iceberg] Added auto-sharding for streaming writes ([#32612](https://github.com/apache/beam/pull/32612)) +* [Managed Iceberg] Added support for writing to dynamic destinations ([#32565](https://github.com/apache/beam/pull/32565)) + +## New Features / Improvements + +* Dataflow worker can install packages from Google Artifact Registry Python repositories (Python) ([#32123](https://github.com/apache/beam/issues/32123)). +* Added support for Zstd codec in SerializableAvroCodecFactory (Java) ([#32349](https://github.com/apache/beam/issues/32349)) +* Added support for using vLLM in the RunInference transform (Python) ([#32528](https://github.com/apache/beam/issues/32528)) +* Prism release binaries and container bootloaders are now being built with the latest Go 1.23 patch. ([#32575](https://github.com/apache/beam/pull/32575)) +* Prism + * Prism now supports Bundle Finalization. ([#32425](https://github.com/apache/beam/pull/32425)) +* Significantly improved performance of Kafka IO reads that enable [commitOffsetsInFinalize](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/kafka/KafkaIO.Read.html#commitOffsetsInFinalize--) by removing the data reshuffle from SDF implementation. ([#31682](https://github.com/apache/beam/pull/31682)). +* Added support for dynamic writing in MqttIO (Java) ([#19376](https://github.com/apache/beam/issues/19376)) +* Optimized Spark Runner parDo transform evaluator (Java) ([#32537](https://github.com/apache/beam/issues/32537)) +* [Managed Iceberg] More efficient manifest file writes/commits ([#32666](https://github.com/apache/beam/issues/32666)) + +## Breaking Changes + +* In Python, assert_that now throws if it is not in a pipeline context instead of silently succeeding ([#30771](https://github.com/apache/beam/pull/30771)) +* In Python and YAML, ReadFromJson now override the dtype from None to + an explicit False. Most notably, string values like `"123"` are preserved + as strings rather than silently coerced (and possibly truncated) to numeric + values. To retain the old behavior, pass `dtype=True` (or any other value + accepted by `pandas.read_json`). +* Users of KafkaIO Read transform that enable [commitOffsetsInFinalize](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/kafka/KafkaIO.Read.html#commitOffsetsInFinalize--) might encounter pipeline graph compatibility issues when updating the pipeline. To mitigate, set the `updateCompatibilityVersion` option to the SDK version used for the original pipeline, example `--updateCompatabilityVersion=2.58.1` + +## Deprecations + +* Python 3.8 is reaching EOL and support is being removed in Beam 2.61.0. The 2.60.0 release will warn users +when running on 3.8. ([#31192](https://github.com/apache/beam/issues/31192)) + +## Bugfixes + +* (Java) Fixed custom delimiter issues in TextIO ([#32249](https://github.com/apache/beam/issues/32249), [#32251](https://github.com/apache/beam/issues/32251)). +* (Java, Python, Go) Fixed PeriodicSequence backlog bytes reporting, which was preventing Dataflow Runner autoscaling from functioning properly ([#32506](https://github.com/apache/beam/issues/32506)). +* (Java) Fix improper decoding of rows with schemas containing nullable fields when encoded with a schema with equal encoding positions but modified field order. ([#32388](https://github.com/apache/beam/issues/32388)). + +## Known Issues + +N/A + +For the most up to date list of known issues, see https://github.com/apache/beam/blob/master/CHANGES.md + +## List of Contributors + +According to git shortlog, the following people contributed to the 2.60.0 release. Thank you to all contributors! + +Ahmed Abualsaud, Aiden Grossman, Arun Pandian, Bartosz Zablocki, Chamikara Jayalath, Claire McGinty, DKPHUONG, Damon Douglass, Danny McCormick, Dip Patel, Ferran Fernández Garrido, Hai Joey Tran, Hyeonho Kim, Igor Bernstein, Israel Herraiz, Jack McCluskey, Jaehyeon Kim, Jeff Kinard, Jeffrey Kinard, Joey Tran, Kenneth Knowles, Kirill Berezin, Michel Davit, Minbo Bae, Naireen Hussain, Niel Markwick, Nito Buendia, Reeba Qureshi, Reuven Lax, Robert Bradshaw, Robert Burke, Rohit Sinha, Ryan Fu, Sam Whittle, Shunping Huang, Svetak Sundhar, Udaya Chathuranga, Vitaly Terentyev, Vlado Djerek, Yi Hu, Claude van der Merwe, XQ Hu, Martin Trieu, Valentyn Tymofieiev, twosom diff --git a/website/www/site/content/en/get-started/downloads.md b/website/www/site/content/en/get-started/downloads.md index 08614b8835c1..ff432996578d 100644 --- a/website/www/site/content/en/get-started/downloads.md +++ b/website/www/site/content/en/get-started/downloads.md @@ -96,10 +96,18 @@ versions denoted `0.x.y`. ## Releases +### 2.60.0 (2024-10-17) + +Official [source code download](https://downloads.apache.org/beam/2.60.0/apache-beam-2.60.0-source-release.zip). +[SHA-512](https://downloads.apache.org/beam/2.60.0/apache-beam-2.60.0-source-release.zip.sha512). +[signature](https://downloads.apache.org/beam/2.60.0/apache-beam-2.60.0-source-release.zip.asc). + +[Release notes](https://github.com/apache/beam/releases/tag/v2.60.0) + ### 2.59.0 (2024-09-11) -Official [source code download](https://downloads.apache.org/beam/2.59.0/apache-beam-2.59.0-source-release.zip). -[SHA-512](https://downloads.apache.org/beam/2.59.0/apache-beam-2.59.0-source-release.zip.sha512). -[signature](https://downloads.apache.org/beam/2.59.0/apache-beam-2.59.0-source-release.zip.asc). +Official [source code download](https://archive.apache.org/dist/beam/2.59.0/apache-beam-2.59.0-source-release.zip). +[SHA-512](https://archive.apache.org/dist/beam/2.59.0/apache-beam-2.59.0-source-release.zip.sha512). +[signature](https://archive.apache.org/dist/beam/2.59.0/apache-beam-2.59.0-source-release.zip.asc). [Release notes](https://github.com/apache/beam/releases/tag/v2.59.0) From 6e3516baf2894b806e9cd3592257ee896c03fe15 Mon Sep 17 00:00:00 2001 From: liferoad Date: Thu, 17 Oct 2024 21:33:33 -0400 Subject: [PATCH 78/78] Revert "Update pyproject.toml by using grpcio-tools==1.65.5" --- sdks/python/pyproject.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/pyproject.toml b/sdks/python/pyproject.toml index a99599a2ce2b..4eb827297019 100644 --- a/sdks/python/pyproject.toml +++ b/sdks/python/pyproject.toml @@ -21,7 +21,7 @@ requires = [ "setuptools", "wheel>=0.36.0", - "grpcio-tools==1.65.5", + "grpcio-tools==1.62.1", "mypy-protobuf==3.5.0", # Avoid https://github.com/pypa/virtualenv/issues/2006 "distlib==0.3.7",