From a41fd2cba03def3b7788ef1ba5a7bd7135630fa0 Mon Sep 17 00:00:00 2001 From: Joey Tran Date: Sun, 7 Jan 2024 22:12:03 -0500 Subject: [PATCH 001/169] Remove INFO from playground python logger --- playground/infrastructure/logger.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/playground/infrastructure/logger.py b/playground/infrastructure/logger.py index 18afd8a1b7ed..aad56f39aa51 100644 --- a/playground/infrastructure/logger.py +++ b/playground/infrastructure/logger.py @@ -37,7 +37,7 @@ def setup_logger(): ) stdout_handler = logging.StreamHandler(sys.stdout) - stdout_handler.addFilter(lambda record: record.levelno in (INFO, WARNING)) + stdout_handler.addFilter(lambda record: record.levelno in (WARNING,)) stdout_handler.setFormatter(formatter) stderr_handler = logging.StreamHandler(sys.stderr) From 847a8f4486e4255a89fb4067bd6d931f53aa60ab Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 12 Jan 2024 14:23:51 -0800 Subject: [PATCH 002/169] [YAML] Improve docs on create. --- sdks/python/apache_beam/yaml/yaml_provider.py | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index fdfcf69ba663..bc025ef5d316 100755 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -554,17 +554,28 @@ def create_builtin_provider(): def create(elements: Iterable[Any], reshuffle: Optional[bool] = True): """Creates a collection containing a specified set of elements. - YAML/JSON-style mappings will be interpreted as Beam rows. For example:: + This transform always produces schema'd data. For example:: type: Create - elements: - - {first: 0, second: {str: "foo", values: [1, 2, 3]}} + config: + elements: [1, 2, 3] + + will result in an output with three elements with a schema of + Row(element=int) whereas YAML/JSON-style mappings will be interpreted + directly as Beam rows, e.g.:: + + type: Create + config: + elements: + - {first: 0, second: {str: "foo", values: [1, 2, 3]}} + - {first: 1, second: {str: "bar", values: [4, 5, 6]}} will result in a schema of the form (int, Row(string, List[int])). Args: elements: The set of elements that should belong to the PCollection. YAML/JSON-style mappings will be interpreted as Beam rows. + Primitives will be mapped to rows with a single "element" field. reshuffle (optional): Whether to introduce a reshuffle (to possibly redistribute the work) if there is more than one element in the collection. Defaults to True. From 8521d2768e05e8ec5013f1c4f5ebc82d8ae36d39 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 12 Jan 2024 14:43:29 -0800 Subject: [PATCH 003/169] [YAML] Better document naming parameter. --- sdks/python/apache_beam/yaml/README.md | 31 +++++++++++++++++++++++++- 1 file changed, 30 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/yaml/README.md b/sdks/python/apache_beam/yaml/README.md index caf3a542b2b0..c790c2d1a082 100644 --- a/sdks/python/apache_beam/yaml/README.md +++ b/sdks/python/apache_beam/yaml/README.md @@ -98,16 +98,45 @@ pipeline: keep: "col3 > 100" input: ReadFromCsv - type: Sql - name: MySqlTransform config: query: "select col1, count(*) as cnt from PCOLLECTION group by col1" input: Filter - type: WriteToJson + config: + path: /path/to/output.json + input: Sql +``` + +Transforms can be named to help with monitoring and debugging. + +``` +pipeline: + transforms: + - type: ReadFromCsv + name: ReadMyData + config: + path: /path/to/input*.csv + - type: Filter + name: KeepBigRecords + config: + language: python + keep: "col3 > 100" + input: ReadMyData + - type: Sql + name: MySqlTransform + config: + query: "select col1, count(*) as cnt from PCOLLECTION group by col1" + input: KeepBigRecords + - type: WriteToJson + name: WriteTheOutput config: path: /path/to/output.json input: MySqlTransform ``` +(This is also needed to disambiguate if more than one transform of the same +type is used.) + If the pipeline is linear, we can let the inputs be implicit by designating the pipeline as a `chain` type. From ba29583d46c1b727a182a5f6f782f2da30fd68e9 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 12 Jan 2024 14:48:28 -0800 Subject: [PATCH 004/169] [YAML] Move pipeline running instructions to the top. This was pointed out as a pain point by multiple people. --- sdks/python/apache_beam/yaml/README.md | 51 ++++++++++++++------------ 1 file changed, 27 insertions(+), 24 deletions(-) diff --git a/sdks/python/apache_beam/yaml/README.md b/sdks/python/apache_beam/yaml/README.md index c790c2d1a082..cbf771406408 100644 --- a/sdks/python/apache_beam/yaml/README.md +++ b/sdks/python/apache_beam/yaml/README.md @@ -48,6 +48,33 @@ It should be noted that everything here is still under development, but any features already included are considered stable. Feedback is welcome at dev@apache.beam.org. +## Running pipelines + +The Beam yaml parser is currently included as part of the Apache Beam Python SDK. +This can be installed (e.g. within a virtual environment) as + +``` +pip install apache_beam[yaml,gcp] +``` + +In addition, several of the provided transforms (such as SQL) are implemented +in Java and their expansion will require a working Java interpeter. (The +requisite artifacts will be automatically downloaded from the apache maven +repositories, so no further installs will be required.) +Docker is also currently required for local execution of these +cross-language-requiring transforms, but not for submission to a non-local +runner such as Flink or Dataflow. + +Once the prerequisites are installed, you can execute a pipeline defined +in a yaml file as + +``` +python -m apache_beam.yaml.main --yaml_pipeline_file=/path/to/pipeline.yaml [other pipeline options such as the runner] +``` + +We intend to support running a pipeline on Dataflow by directly passing the +yaml specification to a template, no local installation of the Beam SDKs required. + ## Example pipelines Here is a simple pipeline that reads some data from csv files and @@ -532,27 +559,3 @@ providers: transforms: MyCustomTransform: "pkg.subpkg.PTransformClassOrCallable" ``` - -## Running pipelines - -The Beam yaml parser is currently included as part of the Apache Beam Python SDK. -This can be installed (e.g. within a virtual environment) as - -``` -pip install apache_beam[yaml,gcp] -``` - -In addition, several of the provided transforms (such as SQL) are implemented -in Java and their expansion will require a working Java interpeter. (The -requisite artifacts will be automatically downloaded from the apache maven -repositories, so no further installs will be required.) -Docker is also currently required for local execution of these -cross-language-requiring transforms, but not for submission to a non-local -runner such as Flink or Dataflow. - -Once the prerequisites are installed, you can execute a pipeline defined -in a yaml file as - -``` -python -m apache_beam.yaml.main --yaml_pipeline_file=/path/to/pipeline.yaml [other pipeline options such as the runner] -``` From e70a28acbd436367275795d685400d30bbe6bab8 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 12 Jan 2024 15:08:06 -0800 Subject: [PATCH 005/169] [YAML] Link to additonal resources. These will get updated when we have proper docs, but it's still good to surface what we have in the meantime. --- sdks/python/apache_beam/yaml/README.md | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/sdks/python/apache_beam/yaml/README.md b/sdks/python/apache_beam/yaml/README.md index cbf771406408..ce991e49c7a1 100644 --- a/sdks/python/apache_beam/yaml/README.md +++ b/sdks/python/apache_beam/yaml/README.md @@ -559,3 +559,16 @@ providers: transforms: MyCustomTransform: "pkg.subpkg.PTransformClassOrCallable" ``` + +## Other Resources + +* [Example pipelines](https://gist.github.com/robertwb/2cb26973f1b1203e8f5f8f88c5764da0) +* [More examples](https://github.com/Polber/beam/tree/jkinard/bug-bash/sdks/python/apache_beam/yaml/examples) +* [Transform glossary](https://gist.github.com/robertwb/64e2f51ff88320eeb6ffd96634202df7) + +Additional documentation in this directory + +* [Mapping](yaml_mapping.md) +* [Aggregation](yaml_combine.md) +* [Error handling](yaml_errors.md) +* [Inlining Python](inline_python.md) From b85e1ebaea3e1467625c495ab331f188278aaf48 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 12 Jan 2024 15:27:53 -0800 Subject: [PATCH 006/169] Some clarification on the mapping docs. --- sdks/python/apache_beam/yaml/yaml_mapping.md | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/yaml/yaml_mapping.md b/sdks/python/apache_beam/yaml/yaml_mapping.md index e760b691e13d..40ced4903f06 100644 --- a/sdks/python/apache_beam/yaml/yaml_mapping.md +++ b/sdks/python/apache_beam/yaml/yaml_mapping.md @@ -37,7 +37,7 @@ To rename fields one can write will result in an output where each record has two fields, `new_col1` and `new_col2`, whose values are those of `col1` and `col2` -respectively. +respectively (which are the names of two fields from the input schema). One can specify the append parameter which indicates the original fields should be retained similar to the use of `*` in an SQL select statement. For example @@ -73,7 +73,8 @@ two new ones. Of course one may want to do transformations beyond just dropping and renaming fields. Beam YAML has the ability to inline simple UDFs. -This requires a language specification. For example +This requires a language specification. For example, we can provide a +Python expression referencing the input fields ``` - type: MapToFields @@ -116,7 +117,7 @@ this up as a dependency and simply refer to it by fully qualified name, e.g. callable: pkg.module.fn ``` -Currently, in addition to Python, SQL expressions are supported as well +Currently, in addition to Python, Java and SQL expressions are supported as well ``` - type: MapToFields From 09aacf1003cb18128b532296a774b02c1943fcf6 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 12 Jan 2024 15:32:15 -0800 Subject: [PATCH 007/169] Another note useful for trying things out. --- sdks/python/apache_beam/yaml/README.md | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/sdks/python/apache_beam/yaml/README.md b/sdks/python/apache_beam/yaml/README.md index ce991e49c7a1..ff49b321bacd 100644 --- a/sdks/python/apache_beam/yaml/README.md +++ b/sdks/python/apache_beam/yaml/README.md @@ -72,6 +72,13 @@ in a yaml file as python -m apache_beam.yaml.main --yaml_pipeline_file=/path/to/pipeline.yaml [other pipeline options such as the runner] ``` +You can do a dry-run of your pipeline using the render runner to see what the +execution graph is, e.g. + +``` +python -m apache_beam.yaml.main --yaml_pipeline_file=/path/to/pipeline.yaml --runner=apache_beam.runners.render.RenderRunner --render_output=out.png [--render_port=0] +``` + We intend to support running a pipeline on Dataflow by directly passing the yaml specification to a template, no local installation of the Beam SDKs required. From ba05f0f0343ba4ebf044d2abdc5474fd6bf33b6f Mon Sep 17 00:00:00 2001 From: Changyu Li Date: Sat, 13 Jan 2024 00:16:48 +0000 Subject: [PATCH 008/169] Add a enum for the new value capture type NEW_ROW_AND_OLD_VALUES. Also add related unit tests. --- .../changestreams/model/ValueCaptureType.java | 1 + .../mapper/ChangeStreamRecordMapperTest.java | 100 ++++++++++++++++++ 2 files changed, 101 insertions(+) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/ValueCaptureType.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/ValueCaptureType.java index 9e336e18eb7b..af1123c118de 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/ValueCaptureType.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/ValueCaptureType.java @@ -30,5 +30,6 @@ public enum ValueCaptureType { NEW_ROW, NEW_VALUES, OLD_AND_NEW_VALUES, + NEW_ROW_AND_OLD_VALUES, UNKNOWN } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/mapper/ChangeStreamRecordMapperTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/mapper/ChangeStreamRecordMapperTest.java index 05ed0bbae6cc..a06fb074e637 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/mapper/ChangeStreamRecordMapperTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/mapper/ChangeStreamRecordMapperTest.java @@ -223,6 +223,44 @@ public void testMappingUpdateStructRowNewValuesToDataChangeRecord() { mapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)); } + /* + * Change streams with NEW_ROW_AND_OLD_VALUES value capture type track both old values for + * modified columns and the whole new row. + */ + @Test + public void testMappingUpdateStructRowNewRowAndOldValuesToDataChangeRecord() { + final DataChangeRecord dataChangeRecord = + new DataChangeRecord( + "partitionToken", + Timestamp.ofTimeSecondsAndNanos(10L, 20), + "serverTransactionId", + true, + "1", + "tableName", + Arrays.asList( + new ColumnType("column1", new TypeCode("{\"code\":\"INT64\"}"), true, 1L), + new ColumnType("column2", new TypeCode("{\"code\":\"BYTES\"}"), false, 2L)), + Collections.singletonList( + new Mod( + "{\"column1\":\"value1\"}", + "{\"column2\":\"oldValue2\"}", + "{\"column2\":\"newValue2\"}")), + ModType.UPDATE, + ValueCaptureType.NEW_ROW, + 10L, + 2L, + "transactionTag", + true, + null); + final Struct jsonFieldsStruct = recordsToStructWithJson(dataChangeRecord); + ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + when(resultSet.getCurrentRowAsStruct()).thenReturn(jsonFieldsStruct); + + assertEquals( + Collections.singletonList(dataChangeRecord), + mapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)); + } + @Test public void testMappingInsertStructRowToDataChangeRecord() { final DataChangeRecord dataChangeRecord = @@ -316,6 +354,37 @@ public void testMappingInsertStructRowNewValuesToDataChangeRecord() { mapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)); } + @Test + public void testMappingInsertStructRowNewRowAndOldValuesToDataChangeRecord() { + final DataChangeRecord dataChangeRecord = + new DataChangeRecord( + "partitionToken", + Timestamp.ofTimeSecondsAndNanos(10L, 20), + "transactionId", + false, + "1", + "tableName", + Arrays.asList( + new ColumnType("column1", new TypeCode("{\"code\":\"INT64\"}"), true, 1L), + new ColumnType("column2", new TypeCode("{\"code\":\"BYTES\"}"), false, 2L)), + Collections.singletonList( + new Mod("{\"column1\":\"value1\"}", null, "{\"column2\":\"newValue2\"}")), + ModType.INSERT, + ValueCaptureType.NEW_ROW_AND_OLD_VALUES, + 10L, + 2L, + "transactionTag", + true, + null); + final Struct jsonFieldsStruct = recordsToStructWithJson(dataChangeRecord); + ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + when(resultSet.getCurrentRowAsStruct()).thenReturn(jsonFieldsStruct); + + assertEquals( + Collections.singletonList(dataChangeRecord), + mapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)); + } + @Test public void testMappingDeleteStructRowToDataChangeRecord() { final DataChangeRecord dataChangeRecord = @@ -407,6 +476,37 @@ public void testMappingDeleteStructRowNewValuesToDataChangeRecord() { mapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)); } + @Test + public void testMappingDeleteStructRowNewRowAndOldValuesToDataChangeRecord() { + final DataChangeRecord dataChangeRecord = + new DataChangeRecord( + "partitionToken", + Timestamp.ofTimeSecondsAndNanos(10L, 20), + "transactionId", + false, + "1", + "tableName", + Arrays.asList( + new ColumnType("column1", new TypeCode("{\"code\":\"INT64\"}"), true, 1L), + new ColumnType("column2", new TypeCode("{\"code\":\"BYTES\"}"), false, 2L)), + Collections.singletonList( + new Mod("{\"column1\":\"value1\"}", "{\"column2\":\"oldValue2\"}", null)), + ModType.DELETE, + ValueCaptureType.NEW_ROW_AND_OLD_VALUES, + 10L, + 2L, + "transactionTag", + true, + null); + final Struct jsonFieldsStruct = recordsToStructWithJson(dataChangeRecord); + ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + when(resultSet.getCurrentRowAsStruct()).thenReturn(jsonFieldsStruct); + + assertEquals( + Collections.singletonList(dataChangeRecord), + mapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)); + } + @Test public void testMappingStructRowWithUnknownModTypeAndValueCaptureTypeToDataChangeRecord() { final DataChangeRecord dataChangeRecord = From 730f1c5edb3a7bd1fc0b7aadf8fdbbaba678591f Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 12 Jan 2024 16:54:44 -0800 Subject: [PATCH 009/169] [YAML] Support comment and delimiter attributes on CSV IO. --- .../csv/providers/CsvWriteTransformProvider.java | 16 ++++++++++++++-- sdks/python/apache_beam/yaml/standard_io.yaml | 3 +++ 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/providers/CsvWriteTransformProvider.java b/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/providers/CsvWriteTransformProvider.java index 4e07a06197f5..f4d54c408cf4 100644 --- a/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/providers/CsvWriteTransformProvider.java +++ b/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/providers/CsvWriteTransformProvider.java @@ -23,6 +23,7 @@ import com.google.auto.value.AutoValue; import java.util.Collections; import java.util.List; +import javax.annotation.Nullable; import org.apache.beam.sdk.io.WriteFilesResult; import org.apache.beam.sdk.io.csv.CsvIO; import org.apache.beam.sdk.schemas.AutoValueSchema; @@ -88,8 +89,10 @@ public List outputCollectionNames() { public abstract static class CsvWriteConfiguration { public void validate() { + checkArgument(!Strings.isNullOrEmpty(getPath()), "Path for a CSV Write must be specified."); checkArgument( - !Strings.isNullOrEmpty(this.getPath()), "Path for a CSV Write must be specified."); + getDelimiter() == null || getDelimiter().length() == 1, + "Only single-character delimiters supported, got '" + getDelimiter() + "'"); } public static Builder builder() { @@ -99,12 +102,17 @@ public static Builder builder() { @SchemaFieldDescription("The file path to write to.") public abstract String getPath(); + @SchemaFieldDescription("The field delimiter to use when writing records. Defaults to a comma.") + public abstract @Nullable String getDelimiter(); + /** Builder for {@link CsvWriteConfiguration}. */ @AutoValue.Builder public abstract static class Builder { public abstract Builder setPath(String path); + public abstract Builder setDelimiter(String delimiter); + /** Builds a {@link CsvWriteConfiguration} instance. */ public abstract CsvWriteConfiguration build(); } @@ -122,10 +130,14 @@ protected static class CsvWriteTransform extends SchemaTransform { @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { + CSVFormat format = CSVFormat.DEFAULT; + if (configuration.getDelimiter() != null) { + format = format.withDelimiter(configuration.getDelimiter().charAt(0)); + } WriteFilesResult result = input .get(INPUT_ROWS_TAG) - .apply(CsvIO.writeRows(configuration.getPath(), CSVFormat.DEFAULT).withSuffix("")); + .apply(CsvIO.writeRows(configuration.getPath(), format).withSuffix("")); Schema outputSchema = Schema.of(Field.of("filename", FieldType.STRING)); return PCollectionRowTuple.of( WRITE_RESULTS, diff --git a/sdks/python/apache_beam/yaml/standard_io.yaml b/sdks/python/apache_beam/yaml/standard_io.yaml index b617a4cbf285..5d1598d2705c 100644 --- a/sdks/python/apache_beam/yaml/standard_io.yaml +++ b/sdks/python/apache_beam/yaml/standard_io.yaml @@ -139,8 +139,11 @@ mappings: 'ReadFromCsv': path: 'path' + delimiter: 'delimiter' + comment: 'comment' 'WriteToCsv': path: 'path' + delimiter: 'delimiter' 'ReadFromJson': path: 'path' 'WriteToJson': From fc626b8b280c9fbd639ce8a52dfacd1f18750ae5 Mon Sep 17 00:00:00 2001 From: Issac Date: Tue, 16 Jan 2024 15:38:58 +0200 Subject: [PATCH 010/169] typo in docs --- website/www/site/content/en/documentation/programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/www/site/content/en/documentation/programming-guide.md b/website/www/site/content/en/documentation/programming-guide.md index 05a3232aa186..6de0ee472782 100644 --- a/website/www/site/content/en/documentation/programming-guide.md +++ b/website/www/site/content/en/documentation/programming-guide.md @@ -7945,7 +7945,7 @@ implementation of your desired logic. You can do this by simply defining both class MultiplyByTwo(beam.DoFn):   def process(self, element: np.int64) -> Iterator[np.int64]: # Multiply an individual int64 by 2 -    yield batch * 2 +    yield element * 2   def process_batch(self, batch: np.ndarray) -> Iterator[np.ndarray]: # Multiply a _batch_ of int64s by 2 From 799c1041473d9d393aba55eef683ddcbc778751a Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Tue, 16 Jan 2024 09:52:31 -0800 Subject: [PATCH 011/169] Avoid SQL keywords for SQL table names. --- sdks/python/apache_beam/yaml/README.md | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/sdks/python/apache_beam/yaml/README.md b/sdks/python/apache_beam/yaml/README.md index ff49b321bacd..1d2104f38fef 100644 --- a/sdks/python/apache_beam/yaml/README.md +++ b/sdks/python/apache_beam/yaml/README.md @@ -243,10 +243,10 @@ pipeline: - type: Sql config: - query: select left.col1, right.col2 from left join right using (col3) + query: select A.col1, B.col2 from A join B using (col3) input: - left: ReadLeft - right: ReadRight + A: ReadLeft + B: ReadRight - type: WriteToJson name: WriteAll @@ -287,10 +287,10 @@ pipeline: - type: Sql config: - query: select left.col1, right.col2 from left join right using (col3) + query: select A.col1, B.col2 from A join B using (col3) input: - left: ReadLeft - right: ReadRight + A: ReadLeft + B: ReadRight - type: WriteToJson name: WriteAll @@ -426,10 +426,10 @@ pipeline: - type: Sql config: - query: select left.col1, right.col2 from left join right using (col3) + query: select A.col1, B.col2 from A join B using (col3) input: - left: ReadLeft - right: ReadRight + A: ReadLeft + B: ReadRight windowing: type: fixed size: 60 From cf201198df96b03c9371dfe9576fb103aea7a61b Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 11 Jan 2024 15:27:56 -0500 Subject: [PATCH 012/169] sickbay new reshuffle test on runners that do not support it --- runners/portability/java/build.gradle | 3 +++ runners/samza/job-server/build.gradle | 2 ++ runners/spark/job-server/spark_job_server.gradle | 2 ++ runners/spark/spark_runner.gradle | 2 ++ 4 files changed, 9 insertions(+) diff --git a/runners/portability/java/build.gradle b/runners/portability/java/build.gradle index 69b93e9d3676..bd7cb5f5be81 100644 --- a/runners/portability/java/build.gradle +++ b/runners/portability/java/build.gradle @@ -200,6 +200,9 @@ def createUlrValidatesRunnerTask = { name, environmentType, dockerImageTask = "" // https://github.com/apache/beam/issues/20374 excludeTestsMatching 'org.apache.beam.sdk.transforms.CombineTest$BasicTests.testHotKeyCombiningWithAccumulationMode' + // TODO(https://github.com/apache/beam/issues/29973) + excludeTestsMatching 'org.apache.beam.sdk.transforms.ReshuffleTest.testReshufflePreservesMetadata' + for (String test : sickbayTests) { excludeTestsMatching test } diff --git a/runners/samza/job-server/build.gradle b/runners/samza/job-server/build.gradle index ad4028709c49..c9401a8aff17 100644 --- a/runners/samza/job-server/build.gradle +++ b/runners/samza/job-server/build.gradle @@ -182,6 +182,8 @@ def portableValidatesRunnerTask(String name, boolean docker) { excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$TimestampTests.testProcessElementSkew' // TODO(https://github.com/apache/beam/issues/22650) excludeTestsMatching 'org.apache.beam.sdk.transforms.GroupByKeyTest$BasicTests.testAfterProcessingTimeContinuationTriggerUsingState' + // TODO(https://github.com/apache/beam/issues/29973) + excludeTestsMatching 'org.apache.beam.sdk.transforms.ReshuffleTest.testReshufflePreservesMetadata' } ) } diff --git a/runners/spark/job-server/spark_job_server.gradle b/runners/spark/job-server/spark_job_server.gradle index 8986d868b2aa..89c5d897b082 100644 --- a/runners/spark/job-server/spark_job_server.gradle +++ b/runners/spark/job-server/spark_job_server.gradle @@ -157,6 +157,8 @@ def portableValidatesRunnerTask(String name, boolean streaming, boolean docker, excludeTestsMatching 'org.apache.beam.sdk.transforms.windowing.WindowingTest' // // Assertion error: incorrect output excludeTestsMatching 'CombineTest$BasicTests.testHotKeyCombining' + // TODO(https://github.com/apache/beam/issues/29973) + excludeTestsMatching 'org.apache.beam.sdk.transforms.ReshuffleTest.testReshufflePreservesMetadata' } } else { diff --git a/runners/spark/spark_runner.gradle b/runners/spark/spark_runner.gradle index 74013de6107d..a58d1bb25ca8 100644 --- a/runners/spark/spark_runner.gradle +++ b/runners/spark/spark_runner.gradle @@ -321,6 +321,8 @@ def validatesRunnerStreaming = tasks.register("validatesRunnerStreaming", Test) // UNBOUNDED View.CreatePCollectionView not supported excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$BundleInvariantsTests.testWatermarkUpdateMidBundle' excludeTestsMatching 'org.apache.beam.sdk.transforms.ViewTest.testWindowedSideInputNotPresent' + // TODO(https://github.com/apache/beam/issues/29973) + excludeTestsMatching 'org.apache.beam.sdk.transforms.ReshuffleTest.testReshufflePreservesMetadata' } // TestStream using processing time is not supported in Spark From 534015c0b8e6b9817ec054c4fe67b3bab423f4a6 Mon Sep 17 00:00:00 2001 From: Cheskel Twersky Date: Wed, 17 Jan 2024 01:22:36 +0200 Subject: [PATCH 013/169] [Typescript] creating dataflow job template fix (#29928) * set template completion status to DONE * job_servicer is not used in serve --- .../apache_beam/runners/dataflow/dataflow_job_service.py | 3 +-- sdks/python/apache_beam/runners/dataflow/dataflow_runner.py | 3 ++- .../apache_beam/runners/portability/local_job_service_main.py | 4 ++-- sdks/python/apache_beam/runners/render.py | 3 +-- 4 files changed, 6 insertions(+), 7 deletions(-) diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_job_service.py b/sdks/python/apache_beam/runners/dataflow/dataflow_job_service.py index 2bd8605c9928..710c71273e34 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_job_service.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_job_service.py @@ -74,8 +74,7 @@ def run(argv, beam_job_type=DataflowBeamJob): options.staging_dir, beam_job_type=beam_job_type) port = job_servicer.start_grpc_server(options.port) try: - local_job_service_main.serve( - "Listening for beam jobs on port %d." % port, job_servicer) + local_job_service_main.serve("Listening for beam jobs on port %d." % port) finally: job_servicer.stop() diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py index dc315119e480..db6a5235ac92 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py @@ -731,7 +731,8 @@ def state(self): A PipelineState object. """ if not self.has_job: - return PipelineState.UNKNOWN + # https://github.com/apache/beam/blob/8f71dc41b30a978095ca0e0699009e4f4445a618/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py#L867-L870 + return PipelineState.DONE self._update_job() diff --git a/sdks/python/apache_beam/runners/portability/local_job_service_main.py b/sdks/python/apache_beam/runners/portability/local_job_service_main.py index 6d9d32f5f23f..efc198e87db1 100644 --- a/sdks/python/apache_beam/runners/portability/local_job_service_main.py +++ b/sdks/python/apache_beam/runners/portability/local_job_service_main.py @@ -145,7 +145,7 @@ def run(argv): with open(options.port_file + '.tmp', 'w') as fout: fout.write(str(port)) os.rename(options.port_file + '.tmp', options.port_file) - serve("Listening for beam jobs on port %d." % port, job_servicer) + serve("Listening for beam jobs on port %d." % port) finally: job_servicer.stop() finally: @@ -155,7 +155,7 @@ def run(argv): os.unlink(options.port_file) -def serve(msg, job_servicer): +def serve(msg): logging_delay = 30 while True: _LOGGER.info(msg) diff --git a/sdks/python/apache_beam/runners/render.py b/sdks/python/apache_beam/runners/render.py index da153d25a4bd..fccfa8aacd61 100644 --- a/sdks/python/apache_beam/runners/render.py +++ b/sdks/python/apache_beam/runners/render.py @@ -559,8 +559,7 @@ def _invoke_runner(self): staging_dir, beam_job_type=RenderBeamJob) port = job_servicer.start_grpc_server(options.job_port) try: - local_job_service_main.serve( - "Listening for beam jobs on port %d." % port, job_servicer) + local_job_service_main.serve("Listening for beam jobs on port %d." % port) finally: job_servicer.stop() From 02b369d87a7424b0c0544dabeda004e2b0c40d2b Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 11 Jan 2024 15:47:40 -0800 Subject: [PATCH 014/169] ReadFromTsv example. --- sdks/python/apache_beam/dataframe/io.py | 3 ++- sdks/python/apache_beam/yaml/inline_python.md | 22 +++++++++++++++++++ sdks/python/apache_beam/yaml/readme_test.py | 5 +++++ 3 files changed, 29 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/dataframe/io.py b/sdks/python/apache_beam/dataframe/io.py index fedc40c60714..b795add1b44e 100644 --- a/sdks/python/apache_beam/dataframe/io.py +++ b/sdks/python/apache_beam/dataframe/io.py @@ -86,7 +86,7 @@ def read_gbq( @frame_base.with_docs_from(pd) -def read_csv(path, *args, splittable=False, **kwargs): +def read_csv(path, *args, splittable=False, binary=True, **kwargs): """If your files are large and records do not contain quoted newlines, you may pass the extra argument ``splittable=True`` to enable dynamic splitting for this read on newlines. Using this option for records that do contain quoted @@ -99,6 +99,7 @@ def read_csv(path, *args, splittable=False, **kwargs): args, kwargs, incremental=True, + binary=binary, splitter=_TextFileSplitter(args, kwargs) if splittable else None) diff --git a/sdks/python/apache_beam/yaml/inline_python.md b/sdks/python/apache_beam/yaml/inline_python.md index 2363b13bfff2..72b8b76c58a2 100644 --- a/sdks/python/apache_beam/yaml/inline_python.md +++ b/sdks/python/apache_beam/yaml/inline_python.md @@ -59,6 +59,28 @@ as an input, e.g. output_type: integer ``` +This can be used to call arbitrary transforms in the Beam SDK, e.g. + +``` +pipeline: + transforms: + - type: PyTransform + name: ReadFromTsv + input: {} + config: + constructor: apache_beam.io.ReadFromCsv + kwargs: + path: '/path/to/*.tsv' + sep: '\t' + skip_blank_lines: True + true_values: ['yes'] + false_values: ['no'] + comment: '#' + on_bad_lines: 'skip' + binary: False + splittable: False +``` + ## Defining a transform inline using `__constructor__` diff --git a/sdks/python/apache_beam/yaml/readme_test.py b/sdks/python/apache_beam/yaml/readme_test.py index da00b01f3ab3..de798c6e4944 100644 --- a/sdks/python/apache_beam/yaml/readme_test.py +++ b/sdks/python/apache_beam/yaml/readme_test.py @@ -154,6 +154,9 @@ def input_file(self, name, content): def input_csv(self): return self.input_file('input.csv', 'col1,col2,col3\nabc,1,2.5\n') + def input_tsv(self): + return self.input_file('input.tsv', 'col1\tcol2\tcol3\nabc\t1\t2.5\n') + def input_json(self): return self.input_file( 'input.json', '{"col1": "abc", "col2": 1, "col3": 2.5"}\n') @@ -192,6 +195,8 @@ def create_test_method(test_type, test_name, test_yaml): def test(self): with TestEnvironment() as env: + nonlocal test_yaml + test_yaml = test_yaml.replace('/path/to/*.tsv', env.input_tsv()) spec = yaml.load(test_yaml, Loader=SafeLoader) if test_type == 'PARSE': return From c62c24dda4c67aa10b51782bfda39366c8b3a533 Mon Sep 17 00:00:00 2001 From: Jeff Kinard <35542536+Polber@users.noreply.github.com> Date: Tue, 16 Jan 2024 19:01:17 -0500 Subject: [PATCH 015/169] [yaml] Revise yaml user label (#29989) --- sdks/python/apache_beam/yaml/yaml_transform.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/yaml/yaml_transform.py b/sdks/python/apache_beam/yaml/yaml_transform.py index 794d7b8c3579..ff20685489c2 100644 --- a/sdks/python/apache_beam/yaml/yaml_transform.py +++ b/sdks/python/apache_beam/yaml/yaml_transform.py @@ -988,10 +988,13 @@ def expand(self, pcolls): self._spec['input'] = {name: name for name in pcolls.keys()} python_provider = yaml_provider.InlineProvider({}) + # Label goog-dataflow-yaml if job is started using Beam YAML. options = pipeline.options.view_as(GoogleCloudOptions) + yaml_version = ('beam-yaml=' + beam.version.__version__.replace('.', '_')) if not options.labels: options.labels = [] - options.labels += ["yaml=true"] + if yaml_version not in options.labels: + options.labels.append(yaml_version) result = expand_transform( self._spec, From 5a4cbe2074f8d0eccdec620d63bda6ecc709b7c4 Mon Sep 17 00:00:00 2001 From: scwhittle Date: Wed, 17 Jan 2024 10:13:08 +0100 Subject: [PATCH 016/169] Additional test for windmill OrderedListState implementation. (#29970) These were attempting to reproduce a bug that ended up being in the test pipeline, but they seem worthwhile keeping to improve coverage. --- .../state/WindmillStateInternalsTest.java | 108 +++++++++++++++++- 1 file changed, 107 insertions(+), 1 deletion(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java index d2590ceb8466..e8eeff3b1d14 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java @@ -47,6 +47,7 @@ import java.util.Objects; import java.util.Random; import java.util.Set; +import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.function.BiConsumer; import java.util.stream.Collectors; @@ -1871,19 +1872,64 @@ public void testOrderedListAddBeforeRead() throws Exception { assertThat(orderedList.read(), Matchers.contains(goodbyeValue, worldValue, helloValue)); } + @Test + public void testOrderedListAddBeforeRangeRead() throws Exception { + StateTag> addr = + StateTags.orderedList("orderedList", StringUtf8Coder.of()); + OrderedListState orderedList = underTest.state(NAMESPACE, addr); + + SettableFuture>> future = SettableFuture.create(); + Range readSubrange = Range.closedOpen(70 * 1000L, 100 * 1000L); + when(mockReader.orderedListFuture( + readSubrange, key(NAMESPACE, "orderedList"), STATE_FAMILY, StringUtf8Coder.of())) + .thenReturn(future); + + orderedList.readRangeLater(Instant.ofEpochMilli(70), Instant.ofEpochMilli(100)); + + final TimestampedValue helloValue = + TimestampedValue.of("hello", Instant.ofEpochMilli(100)); + final TimestampedValue worldValue = + TimestampedValue.of("world", Instant.ofEpochMilli(75)); + final TimestampedValue goodbyeValue = + TimestampedValue.of("goodbye", Instant.ofEpochMilli(50)); + + orderedList.add(helloValue); + waitAndSet(future, Collections.singletonList(worldValue), 200); + orderedList.add(goodbyeValue); + + assertThat( + orderedList.readRange(Instant.ofEpochMilli(70), Instant.ofEpochMilli(100)), + Matchers.contains(worldValue)); + } + @Test public void testOrderedListClearBeforeRead() throws Exception { StateTag> addr = StateTags.orderedList("orderedList", StringUtf8Coder.of()); OrderedListState orderedListState = underTest.state(NAMESPACE, addr); - final TimestampedValue helloElement = TimestampedValue.of("hello", Instant.EPOCH); + final TimestampedValue helloElement = + TimestampedValue.of("hello", Instant.ofEpochSecond(1)); orderedListState.clear(); orderedListState.add(helloElement); assertThat(orderedListState.read(), Matchers.containsInAnyOrder(helloElement)); + // Shouldn't need to read from windmill for this. + Mockito.verifyZeroInteractions(mockReader); + assertThat( + orderedListState.readRange(Instant.ofEpochSecond(1), Instant.ofEpochSecond(2)), + Matchers.containsInAnyOrder(helloElement)); // Shouldn't need to read from windmill for this. Mockito.verifyZeroInteractions(mockReader); + + // Shouldn't need to read from windmill for this. + assertThat( + orderedListState.readRange(Instant.ofEpochSecond(100), Instant.ofEpochSecond(200)), + Matchers.emptyIterable()); + assertThat( + orderedListState.readRange(Instant.EPOCH, Instant.ofEpochSecond(1)), + Matchers.emptyIterable()); + Mockito.verifyZeroInteractions(mockReader); } @Test @@ -2201,6 +2247,66 @@ public void testOrderedListMergePendingAddsAndDeletes() { assertArrayEquals(expected, read); } + @Test + public void testOrderedListInterleavedLocalAddClearReadRange() { + Future, RangeSet>> orderedListFuture = Futures.immediateFuture(null); + Future, RangeSet>> deletionsFuture = Futures.immediateFuture(null); + when(mockReader.valueFuture( + systemKey(NAMESPACE, "orderedList" + IdTracker.IDS_AVAILABLE_STR), + STATE_FAMILY, + IdTracker.IDS_AVAILABLE_CODER)) + .thenReturn(orderedListFuture); + when(mockReader.valueFuture( + systemKey(NAMESPACE, "orderedList" + IdTracker.DELETIONS_STR), + STATE_FAMILY, + IdTracker.SUBRANGE_DELETIONS_CODER)) + .thenReturn(deletionsFuture); + + SettableFuture>> fromStorage = SettableFuture.create(); + + Range readSubrange = Range.closedOpen(1 * 1000000L, 8 * 1000000L); + when(mockReader.orderedListFuture( + readSubrange, key(NAMESPACE, "orderedList"), STATE_FAMILY, StringUtf8Coder.of())) + .thenReturn(fromStorage); + + StateTag> addr = + StateTags.orderedList("orderedList", StringUtf8Coder.of()); + OrderedListState orderedListState = underTest.state(NAMESPACE, addr); + + orderedListState.add(TimestampedValue.of("1", Instant.ofEpochSecond(1))); + orderedListState.add(TimestampedValue.of("2", Instant.ofEpochSecond(2))); + orderedListState.add(TimestampedValue.of("3", Instant.ofEpochSecond(3))); + orderedListState.add(TimestampedValue.of("4", Instant.ofEpochSecond(4))); + + orderedListState.clearRange(Instant.ofEpochSecond(1), Instant.ofEpochSecond(4)); + + orderedListState.add(TimestampedValue.of("5", Instant.ofEpochSecond(5))); + orderedListState.add(TimestampedValue.of("6", Instant.ofEpochSecond(6))); + + orderedListState.add(TimestampedValue.of("3_again", Instant.ofEpochSecond(3))); + + orderedListState.add(TimestampedValue.of("7", Instant.ofEpochSecond(7))); + orderedListState.add(TimestampedValue.of("8", Instant.ofEpochSecond(8))); + + fromStorage.set(ImmutableList.>of()); + + TimestampedValue[] expected = + Iterables.toArray( + ImmutableList.of( + TimestampedValue.of("3_again", Instant.ofEpochSecond(3)), + TimestampedValue.of("4", Instant.ofEpochSecond(4)), + TimestampedValue.of("5", Instant.ofEpochSecond(5)), + TimestampedValue.of("6", Instant.ofEpochSecond(6)), + TimestampedValue.of("7", Instant.ofEpochSecond(7))), + TimestampedValue.class); + + TimestampedValue[] read = + Iterables.toArray( + orderedListState.readRange(Instant.ofEpochSecond(1), Instant.ofEpochSecond(8)), + TimestampedValue.class); + assertArrayEquals(expected, read); + } + @Test public void testOrderedListPersistEmpty() throws Exception { StateTag> addr = From d1a17fa3cbd0d5075de771fac61d524b2f0ea5f1 Mon Sep 17 00:00:00 2001 From: Svetak Sundhar Date: Wed, 17 Jan 2024 16:31:20 +0000 Subject: [PATCH 017/169] Update `google_api_services_healthcare` [01/24] (#30027) * Create HealthcareUtils file with shared resources * revert * update package --- .../main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index f8db63e3a440..c93af317b1a2 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -731,7 +731,7 @@ class BeamModulePlugin implements Plugin { // Keep version consistent with the version in google_cloud_resourcemanager, managed by google_cloud_platform_libraries_bom google_api_services_cloudresourcemanager : "com.google.apis:google-api-services-cloudresourcemanager:v1-rev20230806-$google_clients_version", google_api_services_dataflow : "com.google.apis:google-api-services-dataflow:v1b3-rev20220920-$google_clients_version", - google_api_services_healthcare : "com.google.apis:google-api-services-healthcare:v1-rev20231101-$google_clients_version", + google_api_services_healthcare : "com.google.apis:google-api-services-healthcare:v1-rev20240110-$google_clients_version", google_api_services_pubsub : "com.google.apis:google-api-services-pubsub:v1-rev20220904-$google_clients_version", // Keep version consistent with the version in google_cloud_nio, managed by google_cloud_platform_libraries_bom google_api_services_storage : "com.google.apis:google-api-services-storage:v1-rev20231202-$google_clients_version", From 4b909912ae82aad8dea47319ed8e27ed37290d8f Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Wed, 17 Jan 2024 12:05:44 -0500 Subject: [PATCH 018/169] Vendor grpc 1.60.1 (#29954) * Vendor grpc 1.60.1 * Fix comment --- .../beam/gradle/GrpcVendoring_1_60_1.groovy | 223 ++++++++++++++++++ .../beam/gradle/VendorJavaPlugin.groovy | 9 +- settings.gradle.kts | 1 + vendor/README.md | 4 +- vendor/grpc-1_60_1/build.gradle | 38 +++ 5 files changed, 272 insertions(+), 3 deletions(-) create mode 100644 buildSrc/src/main/groovy/org/apache/beam/gradle/GrpcVendoring_1_60_1.groovy create mode 100644 vendor/grpc-1_60_1/build.gradle diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/GrpcVendoring_1_60_1.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/GrpcVendoring_1_60_1.groovy new file mode 100644 index 000000000000..f12f3bac35c8 --- /dev/null +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/GrpcVendoring_1_60_1.groovy @@ -0,0 +1,223 @@ +/* + * 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.gradle + +/** + * Utilities for working with our vendored version of gRPC. + * + * To update: + * 1. Determine the set of io.grpc libraries we want to include, most likely a superset of + * of the previous vendored gRPC version. + * 2. Use mvn dependency:tree and https://search.maven.org/search?q=g:io.grpc + * to determine dependency tree. You may need to search for optional dependencies + * and determine if they are needed (e.g. conscrypt). + * 3. Validate built artifacts by running linkage tool + * (https://github.com/apache/beam/tree/master/vendor#how-to-validate-the-vendored-dependencies) + * and unit and integration tests in a PR (e.g. https://github.com/apache/beam/pull/16460, + * https://github.com/apache/beam/pull/16459) + */ +class GrpcVendoring_1_60_1 { + static def grpc_version = "1.60.1" + + // See https://github.com/grpc/grpc-java/blob/v1.60.1/gradle/libs.versions.toml + // or https://search.maven.org/search?q=io.grpc%201.60.1 + static def guava_version = "32.0.1-jre" + static def protobuf_version = "3.24.0" + static def gson_version = "2.10.1" + static def google_auth_version = "1.4.0" + static def opencensus_version = "0.31.1" + static def conscrypt_version = "2.5.2" + static def proto_google_common_protos_version = "2.22.0" + + /** Returns the list of implementation time dependencies. */ + static List dependencies() { + return [ + "com.google.guava:guava:$guava_version", + "com.google.protobuf:protobuf-java:$protobuf_version", + "com.google.protobuf:protobuf-java-util:$protobuf_version", + "com.google.code.gson:gson:$gson_version", + "io.grpc:grpc-auth:$grpc_version", + "io.grpc:grpc-core:$grpc_version", + "io.grpc:grpc-context:$grpc_version", + "io.grpc:grpc-netty-shaded:$grpc_version", + "io.grpc:grpc-protobuf:$grpc_version", + "io.grpc:grpc-stub:$grpc_version", + "io.grpc:grpc-alts:$grpc_version", + "io.grpc:grpc-testing:$grpc_version", + "com.google.auth:google-auth-library-credentials:$google_auth_version", + "com.google.api.grpc:proto-google-common-protos:$proto_google_common_protos_version", + "io.opencensus:opencensus-api:$opencensus_version", + "io.opencensus:opencensus-contrib-grpc-metrics:$opencensus_version", + ] + } + + /** + * Returns the list of dependencies that should be exported as runtime + * dependencies within the vendored jar. + */ + static List runtimeDependencies() { + return [ + 'com.google.auto.value:auto-value-annotations:1.8.2', + 'com.google.errorprone:error_prone_annotations:2.20.0', + // transient dependencies of grpc-alts->google-auth-library-oauth2-http->google-http-client: + 'org.apache.httpcomponents:httpclient:4.5.13', + 'org.apache.httpcomponents:httpcore:4.4.15', + // TODO(BEAM-9288): Enable relocation for conscrypt + "org.conscrypt:conscrypt-openjdk-uber:$conscrypt_version" + ] + } + + /** + * Returns the list of test dependencies. + */ + static List testDependencies() { + return [ + 'junit:junit:4.12', + ] + } + + static Map relocations() { + // The relocation paths below specifically use gRPC and the full version string as + // the code relocation prefix. See https://lists.apache.org/thread.html/4c12db35b40a6d56e170cd6fc8bb0ac4c43a99aa3cb7dbae54176815@%3Cdev.beam.apache.org%3E + // for further details. + + // To produce the list of necessary relocations, one needs to start with a set of target + // packages that one wants to vendor, find all necessary transitive dependencies of that + // set and provide relocations for each such that all necessary packages and their + // dependencies are relocated. Any optional dependency that doesn't need relocation + // must be excluded via an 'exclude' rule. There is additional complexity of libraries that use + // JNI or reflection and have to be handled on case by case basis by learning whether + // they support relocation and how would one go about doing it by reading any documentation + // those libraries may provide. The 'validateShadedJarDoesntLeakNonOrgApacheBeamClasses' + // ensures that there are no classes outside of the 'org.apache.beam' namespace. + + String version = "v" + grpc_version.replace(".", "p") + String prefix = "org.apache.beam.vendor.grpc.${version}" + List packagesToRelocate = [ + // guava uses the com.google.common and com.google.thirdparty package namespaces + "com.google.common", + "com.google.thirdparty", + "com.google.protobuf", + "com.google.gson", + "com.google.auth", + "com.google.api", + "com.google.cloud", + "com.google.logging", + "com.google.longrunning", + "com.google.rpc", + "com.google.type", + "com.google.geo.type", + "io.grpc", + "io.opencensus", + "io.perfmark", + ] + + return packagesToRelocate.collectEntries { + [ (it): "${prefix}.${it}" ] + } + [ + // Redirect io.grpc.netty.shaded to top. + // To keep namespace consistency before switching from io.grpc:grpc-netty. + "io.grpc.netty.shaded": "${prefix}", + ] + [ + // Adapted from https://github.com/grpc/grpc-java/blob/e283f70ad91f99c7fee8b31b605ef12a4f9b1690/netty/shaded/build.gradle#L41 + // We have to be careful with these replacements as they must not match any + // string in NativeLibraryLoader, else they cause corruption. Note that + // this includes concatenation of string literals and constants. + 'META-INF/native/io_grpc_netty_shaded_netty': "META-INF/native/org_apache_beam_vendor_grpc_${version}_netty", + 'META-INF/native/libio_grpc_netty_shaded_netty': "META-INF/native/liborg_apache_beam_vendor_grpc_${version}_netty", + ] + } + + static Map> relocationExclusions() { + // sub-package excluded from relocation + return [ + "io.grpc": ["io.grpc.netty.shaded.**"], + ] + } + + /** Returns the list of shading exclusions. */ + static List exclusions() { + return [ + // Don't include in the vendored jar: + // android annotations, autovalue annotations, errorprone, checkerframework, JDK8 annotations, objenesis, junit, + // apache commons, log4j, slf4j and mockito + "android/annotation/**/", + "com/google/auto/value/**", + "com/google/errorprone/**", + "com/google/instrumentation/**", + "com/google/j2objc/annotations/**", + "io/grpc/netty/shaded/io/netty/handler/codec/marshalling/**", + "io/grpc/netty/shaded/io/netty/handler/codec/spdy/**", + "io/grpc/netty/shaded/io/netty/handler/codec/compression/JZlib*", + "io/grpc/netty/shaded/io/netty/handler/codec/compression/Lz4*", + "io/grpc/netty/shaded/io/netty/handler/codec/compression/Lzf*", + "io/grpc/netty/shaded/io/netty/handler/codec/compression/Lzma*", + "io/grpc/netty/shaded/io/netty/handler/codec/protobuf/Protobuf*Nano.class", + "io/grpc/netty/shaded/io/netty/util/internal/logging/CommonsLogger*", + "io/grpc/netty/shaded/io/netty/util/internal/logging/LocationAwareSlf4JLogger*", + "io/grpc/netty/shaded/io/netty/util/internal/logging/Log4JLogger*", + "io/grpc/netty/shaded/io/netty/util/internal/logging/Log4J2Logger*", + "javax/annotation/**", + "junit/**", + "module-info.class", + "org/apache/commons/logging/**", + "org/apache/commons/codec/**", + "org/apache/http/**", + "org/checkerframework/**", + "org/codehaus/mojo/animal_sniffer/**", + "org/conscrypt/**", + "META-INF/native/libconscrypt**", + "META-INF/native/conscrypt**", + "org/hamcrest/**", + "org/junit/**", + "org/mockito/**", + "org/objenesis/**", + ] + } + + /** + * Returns a closure containing the dependencies map used for shading gRPC within the main + * Apache Beam project. + */ + static Object dependenciesClosure() { + return { + dependencies().each { implementation it } + runtimeDependencies().each { shadow it } + } + } + + /** + * Returns a closure with the code relocation configuration for shading gRPC within the main + * Apache Beam project. + */ + static Object shadowClosure() { + def relocExclusions = relocationExclusions() + return { + relocations().each { srcNamespace, destNamespace -> + relocate(srcNamespace, destNamespace) { + if (relocExclusions.containsKey(srcNamespace)) { + relocExclusions.get(srcNamespace).each { toExclude -> + exclude toExclude + } + } + } + } + exclusions().each { exclude it } + } + } +} diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/VendorJavaPlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/VendorJavaPlugin.groovy index 97d96e6cf1eb..82e85d0ed468 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/VendorJavaPlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/VendorJavaPlugin.groovy @@ -54,6 +54,7 @@ class VendorJavaPlugin implements Plugin { List runtimeDependencies List testDependencies Map relocations + Map> relocationExclusions List exclusions String groupId String artifactId @@ -122,7 +123,13 @@ artifactId=${project.name} project.shadowJar { config.relocations.each { srcNamespace, destNamespace -> - relocate(srcNamespace, destNamespace) + relocate(srcNamespace, destNamespace) { + if (config.relocationExclusions?.containsKey(srcNamespace)) { + config.relocationExclusions.get(srcNamespace).each { toExclude -> + exclude toExclude + } + } + } } config.exclusions.each { exclude it } diff --git a/settings.gradle.kts b/settings.gradle.kts index da170d0f8467..c645c1adfd6f 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -313,6 +313,7 @@ include(":sdks:python:test-suites:xlang") include(":sdks:typescript") include(":sdks:typescript:container") include(":vendor:grpc-1_54_0") +include(":vendor:grpc-1_60_1") include(":vendor:calcite-1_28_0") include(":vendor:guava-32_1_2-jre") include(":website") diff --git a/vendor/README.md b/vendor/README.md index 7f37c9283741..7815305fa662 100644 --- a/vendor/README.md +++ b/vendor/README.md @@ -91,8 +91,8 @@ Steps: mkdir -p tempLib/org/apache/beam # Copy files (jar/poms/metadata) to your beam repository -cp -R ~/.m2/repository/org/apache/beam/beam-vendor-grpc-1_54_0/ \ - tempLib/org/apache/beam +cp -R ~/.m2/repository/org/apache/beam/beam-vendor-grpc-1_54_0 \ + tempLib/org/apache/beam/ ``` 2. Add the folder to the expected project repositories: diff --git a/vendor/grpc-1_60_1/build.gradle b/vendor/grpc-1_60_1/build.gradle new file mode 100644 index 000000000000..1617484c1c52 --- /dev/null +++ b/vendor/grpc-1_60_1/build.gradle @@ -0,0 +1,38 @@ +/* + * 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 org.apache.beam.gradle.GrpcVendoring_1_60_1 + +plugins { id 'org.apache.beam.vendor-java' } + +description = "Apache Beam :: Vendored Dependencies :: gRPC :: 1.60.1" + +group = "org.apache.beam" +version = "0.1" + +vendorJava( + dependencies: GrpcVendoring_1_60_1.dependencies(), + runtimeDependencies: GrpcVendoring_1_60_1.runtimeDependencies(), + testDependencies: GrpcVendoring_1_60_1.testDependencies(), + relocations: GrpcVendoring_1_60_1.relocations(), + relocationExclusions: GrpcVendoring_1_60_1.relocationExclusions(), + exclusions: GrpcVendoring_1_60_1.exclusions(), + artifactId: "beam-vendor-grpc-${GrpcVendoring_1_60_1.grpc_version.replace(".", "_")}", + groupId: group, + version: version, +) From afd8b79e8e0e931ce328f66c651ec5131ce5ea1a Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Tue, 16 Jan 2024 09:57:48 -0800 Subject: [PATCH 019/169] Clarification on aggregating transform based on user feedback. --- sdks/python/apache_beam/yaml/README.md | 8 ++++++-- sdks/python/apache_beam/yaml/readme_test.py | 7 +++++-- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/yaml/README.md b/sdks/python/apache_beam/yaml/README.md index 1d2104f38fef..dcedb480c511 100644 --- a/sdks/python/apache_beam/yaml/README.md +++ b/sdks/python/apache_beam/yaml/README.md @@ -348,7 +348,9 @@ pipeline: windowing: type: fixed size: 60 - - type: SomeAggregation + - type: SomeGroupingTransform + config: + arg: ... - type: WriteToPubSub config: topic: anotherPubSubTopic @@ -368,7 +370,9 @@ pipeline: topic: myPubSubTopic format: ... schema: ... - - type: SomeAggregation + - type: SomeGroupingTransform + config: + arg: ... windowing: type: sliding size: 60 diff --git a/sdks/python/apache_beam/yaml/readme_test.py b/sdks/python/apache_beam/yaml/readme_test.py index da00b01f3ab3..f7db478c9096 100644 --- a/sdks/python/apache_beam/yaml/readme_test.py +++ b/sdks/python/apache_beam/yaml/readme_test.py @@ -119,7 +119,10 @@ def expand(self, pcoll): return pcoll -class SomeAggregation(beam.PTransform): +class FakeAggregation(beam.PTransform): + def __init__(self, **unused_kwargs): + pass + def expand(self, pcoll): return pcoll | beam.GroupBy(lambda _: 'key').aggregate_field( lambda _: 1, sum, 'count') @@ -130,7 +133,7 @@ def expand(self, pcoll): 'Sql': FakeSql, 'ReadFromPubSub': FakeReadFromPubSub, 'WriteToPubSub': FakeWriteToPubSub, - 'SomeAggregation': SomeAggregation, + 'SomeGroupingTransform': FakeAggregation, } From b5dc728b677101cf3968e9f94db0898342343f6e Mon Sep 17 00:00:00 2001 From: Shunping Huang Date: Wed, 17 Jan 2024 13:39:35 -0500 Subject: [PATCH 020/169] Fix an edge case of getting duplicated records when using TextIO. (#30026) When processing a CRLF-delimited file and the read buffer has CR as the last character, startOfNextRecord will be set to the position after the CR, i.e. the following LF. Let's say the position of this LF is p. In the next buffer, even though the actual start of record should be p+1, startOfRecord is set to startOfNextRecord, which is p. Then the code processes the next record by skipping the LF and yields a record starting from p+1. It decides whether the record is valid by checking if startOfRecord is in the range defined in RangeTracker. If there is a split right after p, i.e. we have ranges [a, p+1) and [p+1, b), then the above record would be considered as valid in the split [a, p+1), because its startOfRecord is p <= p+1. However, the record is also considered valid when split [p+1, b) is processed, resulting into duplicated records in the output. --- .../org/apache/beam/sdk/io/TextSource.java | 5 +- .../apache/beam/sdk/io/TextIOReadTest.java | 113 ++++++++++++------ 2 files changed, 82 insertions(+), 36 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java index 3d62c677950a..8367b38751c3 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java @@ -323,10 +323,13 @@ private boolean readDefaultLine() throws IOException { // Consume any LF after CR if it is the first character of the next buffer if (skipLineFeedAtStart && buffer[bufferPosn] == LF) { - ++bytesConsumed; ++startPosn; ++bufferPosn; skipLineFeedAtStart = false; + + // Right now, startOfRecord is pointing at the position of LF, but the actual start + // position of the new record should be the position after LF. + ++startOfRecord; } // Search for the newline diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java index 84c05ee6c906..253308d1b93f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java @@ -386,53 +386,96 @@ public void testReadLinesWithDefaultDelimiter() throws Exception { runTestReadWithData(line.getBytes(UTF_8), expected); } + // Placeholder channel that only yields 0- and 1-length buffers. + private static class SlowReadChannel implements ReadableByteChannel { + int readCount = 0; + InputStream stream; + ReadableByteChannel channel; + + public SlowReadChannel(FileBasedSource source) throws IOException { + channel = + FileSystems.open( + FileSystems.matchSingleFileSpec(source.getFileOrPatternSpec()).resourceId()); + stream = Channels.newInputStream(channel); + } + + // Data is read at most one byte at a time from line parameter. + @Override + public int read(ByteBuffer dst) throws IOException { + if (++readCount % 3 == 0) { + if (dst.hasRemaining()) { + int value = stream.read(); + if (value == -1) { + return -1; + } + dst.put((byte) value); + return 1; + } + } + return 0; + } + + @Override + public boolean isOpen() { + return channel.isOpen(); + } + + @Override + public void close() throws IOException { + stream.close(); + } + } + @Test - public void testReadLinesWithDefaultDelimiterAndZeroAndOneLengthReturningChannel() - throws Exception { + public void testReadLinesWithDefaultDelimiterAndSlowReadChannel() throws Exception { Path path = tempFolder.newFile().toPath(); Files.write(path, line.getBytes(UTF_8)); Metadata metadata = FileSystems.matchSingleFileSpec(path.toString()); FileBasedSource source = getTextSource(path.toString(), null, 0) .createForSubrangeOfFile(metadata, 0, metadata.sizeBytes()); + FileBasedReader reader = source.createSingleFileReader(PipelineOptionsFactory.create()); - ReadableByteChannel channel = - FileSystems.open( - FileSystems.matchSingleFileSpec(source.getFileOrPatternSpec()).resourceId()); - InputStream stream = Channels.newInputStream(channel); - reader.startReading( - // Placeholder channel that only yields 0- and 1-length buffers. - // Data is read at most one byte at a time from line parameter. - new ReadableByteChannel() { - int readCount = 0; - @Override - public int read(ByteBuffer dst) throws IOException { - if (++readCount % 3 == 0) { - if (dst.hasRemaining()) { - int value = stream.read(); - if (value == -1) { - return -1; - } - dst.put((byte) value); - return 1; - } - } - return 0; - } + reader.startReading(new SlowReadChannel(source)); + assertEquals(expected, SourceTestUtils.readFromStartedReader(reader)); + } - @Override - public boolean isOpen() { - return channel.isOpen(); - } + @Test + public void testReadLinesWithDefaultDelimiterOnSplittingSourceAndSlowReadChannel() + throws Exception { + Path path = tempFolder.newFile().toPath(); + Files.write(path, line.getBytes(UTF_8)); + Metadata metadata = FileSystems.matchSingleFileSpec(path.toString()); + FileBasedSource source = + getTextSource(path.toString(), null, 0) + .createForSubrangeOfFile(metadata, 0, metadata.sizeBytes()); - @Override - public void close() throws IOException { - stream.close(); - } - }); - assertEquals(expected, SourceTestUtils.readFromStartedReader(reader)); + PipelineOptions options = PipelineOptionsFactory.create(); + + // Check every possible split positions. + for (int i = 0; i < line.length(); ++i) { + double fraction = i * 1.0 / line.length(); + FileBasedReader reader = source.createSingleFileReader(options); + + // Use a slow read channel to read the content byte by byte. This can simulate the scenario + // of a certain character (in our case CR) occurring at the end of the read buffer. + reader.startReading(new SlowReadChannel(source)); + + // In order to get a successful split, we need to read at least one record before calling + // splitAtFraction(). + List totalItems = SourceTestUtils.readNItemsFromStartedReader(reader, 1); + BoundedSource residual = reader.splitAtFraction(fraction); + List primaryItems = SourceTestUtils.readFromStartedReader(reader); + totalItems.addAll(primaryItems); + + if (residual != null) { + List residualItems = SourceTestUtils.readFromSource(residual, options); + totalItems.addAll(residualItems); + } + assertEquals(expected, totalItems); + } } @Test From 3faa8729677ca8937273a78c147b99d6aef76125 Mon Sep 17 00:00:00 2001 From: clmccart Date: Wed, 17 Jan 2024 11:04:18 -0800 Subject: [PATCH 021/169] add #29987 to 2.53 known issues (#29988) Co-authored-by: Claire McCarthy --- CHANGES.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/CHANGES.md b/CHANGES.md index a741b87d429b..81a519b07d7e 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -127,6 +127,10 @@ * Upgraded to go 1.21.5 to build, fixing [CVE-2023-45285](https://security-tracker.debian.org/tracker/CVE-2023-45285) and [CVE-2023-39326](https://security-tracker.debian.org/tracker/CVE-2023-39326) +## Known Issues + +* ([#29987](https://github.com/apache/beam/issues/29987)). + # [2.52.0] - 2023-11-17 ## Highlights From 52b4a9cb58e486745ded7d53a5b6e2d2312e9551 Mon Sep 17 00:00:00 2001 From: Damon Date: Wed, 17 Jan 2024 11:05:10 -0800 Subject: [PATCH 022/169] Update to fix endpoint connectivity (#30022) --- .../google-kubernetes-engine/README.md | 14 ++++++++---- .../google-kubernetes-engine/cluster.tf | 19 +++++++++------- .../us-central1.apache-beam-testing.tfvars | 22 +++++++++++++++++++ ...rs => us-west1.apache-beam-testing.tfvars} | 2 +- 4 files changed, 44 insertions(+), 13 deletions(-) create mode 100644 .test-infra/terraform/google-cloud-platform/google-kubernetes-engine/us-central1.apache-beam-testing.tfvars rename .test-infra/terraform/google-cloud-platform/google-kubernetes-engine/{apache-beam-testing.tfvars => us-west1.apache-beam-testing.tfvars} (97%) diff --git a/.test-infra/terraform/google-cloud-platform/google-kubernetes-engine/README.md b/.test-infra/terraform/google-cloud-platform/google-kubernetes-engine/README.md index cd9558e7739f..3b4a31d2a98e 100644 --- a/.test-infra/terraform/google-cloud-platform/google-kubernetes-engine/README.md +++ b/.test-infra/terraform/google-cloud-platform/google-kubernetes-engine/README.md @@ -27,13 +27,13 @@ Google Cloud Platform (GCP). See [Google Cloud Platform requirements](../../google-cloud-platform/README.md) for details on requirements and usage. -# Prerequisites +# IMPORTANT Prerequisites This module assumes the following pre-existing resources: - [Cloud Resource Manager API Enabled](https://console.cloud.google.com/apis/library/cloudresourcemanager.googleapis.com) - [Virtual Private Cloud (VPC) network and subnetwork](https://cloud.google.com/vpc/docs/create-modify-vpc-networks) -- [GCP Service Account](https://cloud.google.com/iam/docs/service-accounts-create) +- [GCP Service Account](https://cloud.google.com/iam/docs/service-accounts-create) with [minimally permissive IAM roles](https://cloud.google.com/kubernetes-engine/docs/how-to/hardening-your-cluster#use_least_privilege_sa) # Step 1. Create vars.tfvars @@ -53,9 +53,15 @@ variable names. ## If you are provisioning in `apache-beam-testing`: +Set the region: +``` +REGION=us-central1 +``` + +Apply the module: ``` terraform init -terraform apply -var-file=apache-beam-testing.tfvars +terraform apply -var-file=$REGION.apache-beam-testing.tfvars ``` You will be prompted for any remaining variables. @@ -67,4 +73,4 @@ terraform init terraform apply -var-file=vars.tfvars ``` -You will be prompted for any remaining variables. +You will be prompted for any remaining variables. \ No newline at end of file diff --git a/.test-infra/terraform/google-cloud-platform/google-kubernetes-engine/cluster.tf b/.test-infra/terraform/google-cloud-platform/google-kubernetes-engine/cluster.tf index 90c6ac793389..db41632124a8 100644 --- a/.test-infra/terraform/google-cloud-platform/google-kubernetes-engine/cluster.tf +++ b/.test-infra/terraform/google-cloud-platform/google-kubernetes-engine/cluster.tf @@ -23,15 +23,18 @@ resource "random_string" "postfix" { } resource "google_container_cluster" "default" { - depends_on = [google_project_service.required] - name = "${var.cluster_name_prefix}-${random_string.postfix.result}" - location = var.region - enable_autopilot = true - network = data.google_compute_network.default.id - subnetwork = data.google_compute_subnetwork.default.id - master_authorized_networks_config {} + depends_on = [google_project_service.required] + deletion_protection = false + name = "${var.cluster_name_prefix}-${random_string.postfix.result}" + location = var.region + enable_autopilot = true + network = data.google_compute_network.default.id + subnetwork = data.google_compute_subnetwork.default.id private_cluster_config { enable_private_nodes = true enable_private_endpoint = false } -} + node_config { + service_account = data.google_service_account.default.email + } +} \ No newline at end of file diff --git a/.test-infra/terraform/google-cloud-platform/google-kubernetes-engine/us-central1.apache-beam-testing.tfvars b/.test-infra/terraform/google-cloud-platform/google-kubernetes-engine/us-central1.apache-beam-testing.tfvars new file mode 100644 index 000000000000..34d0b6076c81 --- /dev/null +++ b/.test-infra/terraform/google-cloud-platform/google-kubernetes-engine/us-central1.apache-beam-testing.tfvars @@ -0,0 +1,22 @@ +/* + * 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. + */ + +project = "apache-beam-testing" +network = "default" +subnetwork = "default-f91f013bcf8bd369" +region = "us-central1" \ No newline at end of file diff --git a/.test-infra/terraform/google-cloud-platform/google-kubernetes-engine/apache-beam-testing.tfvars b/.test-infra/terraform/google-cloud-platform/google-kubernetes-engine/us-west1.apache-beam-testing.tfvars similarity index 97% rename from .test-infra/terraform/google-cloud-platform/google-kubernetes-engine/apache-beam-testing.tfvars rename to .test-infra/terraform/google-cloud-platform/google-kubernetes-engine/us-west1.apache-beam-testing.tfvars index f1105e180f70..4de029a41c8c 100644 --- a/.test-infra/terraform/google-cloud-platform/google-kubernetes-engine/apache-beam-testing.tfvars +++ b/.test-infra/terraform/google-cloud-platform/google-kubernetes-engine/us-west1.apache-beam-testing.tfvars @@ -19,4 +19,4 @@ project = "apache-beam-testing" network = "default" subnetwork = "default" -region = "us-central1" +region = "us-west1" \ No newline at end of file From 79b9de283feda3c4efea7be4e90c2b7ef4ebfb40 Mon Sep 17 00:00:00 2001 From: Naireen Hussain Date: Wed, 17 Jan 2024 20:00:34 -0800 Subject: [PATCH 023/169] GCS autosharding flag (#29886) GCS autosharding flag Co-authored-by: Naireen --- .../model/pipeline/v1/beam_runner_api.proto | 4 +++ .../construction/WriteFilesTranslation.java | 23 +++++++++++++ .../WriteFilesTranslationTest.java | 6 +++- .../beam/runners/dataflow/DataflowRunner.java | 7 ++++ .../runners/dataflow/DataflowRunnerTest.java | 33 +++++++++++++++++++ .../java/org/apache/beam/sdk/Pipeline.java | 10 +++--- .../java/org/apache/beam/sdk/io/FileIO.java | 14 ++++++++ .../java/org/apache/beam/sdk/io/TextIO.java | 18 ++++++++++ .../org/apache/beam/sdk/io/WriteFiles.java | 12 +++++++ .../org/apache/beam/sdk/PipelineTest.java | 5 ++- .../apache/beam/sdk/io/WriteFilesTest.java | 12 +++++++ 11 files changed, 134 insertions(+), 10 deletions(-) diff --git a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto index 08f05fc51b69..bbd326f91deb 100644 --- a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto +++ b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto @@ -738,6 +738,10 @@ message WriteFilesPayload { bool runner_determined_sharding = 4; map side_inputs = 5; + + // This is different from runner based sharding. This is done by the runner backend, where as runner_determined_sharding + // is by the runner translator + bool auto_sharded = 6; } // Payload used by Google Cloud Pub/Sub read transform. diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java index 3a23ed073776..a32a2a98f410 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java @@ -90,6 +90,11 @@ public boolean isWindowedWrites() { return transform.getWindowedWrites(); } + @Override + public boolean isAutoSharded() { + return transform.getWithAutoSharding(); + } + @Override public boolean isRunnerDeterminedSharding() { return transform.getNumShardsProvider() == null @@ -175,6 +180,16 @@ public static boolean isWindowedWrites( return getWriteFilesPayload(transform).getWindowedWrites(); } + public static boolean isAutoSharded( + AppliedPTransform< + PCollection, + WriteFilesResult, + ? extends PTransform, WriteFilesResult>> + transform) + throws IOException { + return getWriteFilesPayload(transform).getAutoSharded(); + } + public static boolean isRunnerDeterminedSharding( AppliedPTransform< PCollection, @@ -268,6 +283,11 @@ public boolean isWindowedWrites() { return payload.getWindowedWrites(); } + @Override + public boolean isAutoSharded() { + return payload.getAutoSharded(); + } + @Override public boolean isRunnerDeterminedSharding() { return payload.getRunnerDeterminedSharding(); @@ -309,6 +329,8 @@ private interface WriteFilesLike { boolean isWindowedWrites(); + boolean isAutoSharded(); + boolean isRunnerDeterminedSharding(); } @@ -319,6 +341,7 @@ public static WriteFilesPayload payloadForWriteFilesLike( .setSink(writeFiles.translateSink(components)) .putAllSideInputs(writeFiles.translateSideInputs(components)) .setWindowedWrites(writeFiles.isWindowedWrites()) + .setAutoSharded(writeFiles.isAutoSharded()) .setRunnerDeterminedSharding(writeFiles.isRunnerDeterminedSharding()) .build(); } diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WriteFilesTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WriteFilesTranslationTest.java index cc6daee00591..6cea7beb4f46 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WriteFilesTranslationTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WriteFilesTranslationTest.java @@ -57,7 +57,8 @@ public static Iterable> data() { WriteFiles.to(new DummySink()), WriteFiles.to(new DummySink()).withWindowedWrites(), WriteFiles.to(new DummySink()).withNumShards(17), - WriteFiles.to(new DummySink()).withWindowedWrites().withNumShards(42)); + WriteFiles.to(new DummySink()).withWindowedWrites().withNumShards(42), + WriteFiles.to(new DummySink()).withAutoSharding()); } @Parameter(0) @@ -105,6 +106,9 @@ public void testExtractionDirectFromTransform() throws Exception { equalTo( writeFiles.getNumShardsProvider() == null && writeFiles.getComputeNumShards() == null)); + assertThat( + WriteFilesTranslation.isAutoSharded(appliedPTransform), + equalTo(writeFiles.getWithAutoSharding())); assertThat( WriteFilesTranslation.isWindowedWrites(appliedPTransform), equalTo(writeFiles.getWindowedWrites())); diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 5f8098768261..ea7a7d9e7314 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -2446,6 +2446,13 @@ static class StreamingShardedWriteFactory if (WriteFilesTranslation.isWindowedWrites(transform)) { replacement = replacement.withWindowedWrites(); } + + if (WriteFilesTranslation.isAutoSharded(transform)) { + replacement = replacement.withAutoSharding(); + return PTransformReplacement.of( + PTransformReplacements.getSingletonMainInput(transform), replacement); + } + return PTransformReplacement.of( PTransformReplacements.getSingletonMainInput(transform), replacement.withNumShards(numShards)); diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java index bcdea03dba2c..63fccc8f05b9 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java @@ -2151,6 +2151,39 @@ public void testStreamingWriteWithNoShardingReturnsNewTransformMaxWorkersUnset() testStreamingWriteOverride(options, StreamingShardedWriteFactory.DEFAULT_NUM_SHARDS); } + @Test + public void testStreamingWriteWithShardingReturnsSameTransform() { + PipelineOptions options = TestPipeline.testingPipelineOptions(); + + TestPipeline p = TestPipeline.fromOptions(options); + + StreamingShardedWriteFactory factory = + new StreamingShardedWriteFactory<>(p.getOptions()); + WriteFiles original = + WriteFiles.to(new TestSink(tmpFolder.toString())).withAutoSharding(); + PCollection objs = (PCollection) p.apply(Create.empty(VoidCoder.of())); + AppliedPTransform, WriteFilesResult, WriteFiles> + originalApplication = + AppliedPTransform.of( + "writefiles", + PValues.expandInput(objs), + Collections.emptyMap(), + original, + ResourceHints.create(), + p); + + WriteFiles replacement = + (WriteFiles) + factory.getReplacementTransform(originalApplication).getTransform(); + + WriteFilesResult originalResult = objs.apply(original); + WriteFilesResult replacementResult = objs.apply(replacement); + + assertTrue(replacement.getNumShardsProvider() == null); + assertTrue(replacement.getComputeNumShards() == null); + assertTrue(replacement.getWithAutoSharding()); + } + private void verifyMergingStatefulParDoRejected(PipelineOptions options) throws Exception { Pipeline p = Pipeline.create(options); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java index 50158f109cb0..a597013c1c19 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java @@ -210,9 +210,6 @@ public PipelineOptions getOptions() { * *

Replaces all nodes that match a {@link PTransformOverride} in this pipeline. Overrides are * applied in the order they are present within the list. - * - *

After all nodes are replaced, ensures that no nodes in the updated graph match any of the - * overrides. */ @Internal public void replaceAll(List overrides) { @@ -241,9 +238,10 @@ public CompositeBehavior enterCompositeTransform(Node node) { @Override public void leaveCompositeTransform(Node node) { - if (node.isRootNode()) { - checkState( - matched.isEmpty(), "Found nodes that matched overrides. Matches: %s", matched); + if (node.isRootNode() && !matched.isEmpty()) { + LOG.info( + "Found nodes that matched overrides. Matches: {}. The match usually should be empty unless there are runner specific replacement transforms.", + matched); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java index 0bc984877217..c647b0e70bbf 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java @@ -393,6 +393,7 @@ public static Write write() { .setDynamic(false) .setCompression(Compression.UNCOMPRESSED) .setIgnoreWindowing(false) + .setAutoSharding(false) .setNoSpilling(false) .build(); } @@ -406,6 +407,7 @@ public static Write writeDynamic() { .setDynamic(true) .setCompression(Compression.UNCOMPRESSED) .setIgnoreWindowing(false) + .setAutoSharding(false) .setNoSpilling(false) .build(); } @@ -1037,6 +1039,8 @@ public static FileNaming relativeFileNaming( abstract boolean getIgnoreWindowing(); + abstract boolean getAutoSharding(); + abstract boolean getNoSpilling(); abstract @Nullable ErrorHandler getBadRecordErrorHandler(); @@ -1085,6 +1089,8 @@ abstract Builder setSharding( abstract Builder setIgnoreWindowing(boolean ignoreWindowing); + abstract Builder setAutoSharding(boolean autosharding); + abstract Builder setNoSpilling(boolean noSpilling); abstract Builder setBadRecordErrorHandler( @@ -1311,6 +1317,10 @@ public Write withIgnoreWindowing() { return toBuilder().setIgnoreWindowing(true).build(); } + public Write withAutoSharding() { + return toBuilder().setAutoSharding(true).build(); + } + /** See {@link WriteFiles#withNoSpilling()}. */ public Write withNoSpilling() { return toBuilder().setNoSpilling(true).build(); @@ -1412,6 +1422,7 @@ public WriteFilesResult expand(PCollection input) { resolvedSpec.setNumShards(getNumShards()); resolvedSpec.setSharding(getSharding()); resolvedSpec.setIgnoreWindowing(getIgnoreWindowing()); + resolvedSpec.setAutoSharding(getAutoSharding()); resolvedSpec.setNoSpilling(getNoSpilling()); Write resolved = resolvedSpec.build(); @@ -1428,6 +1439,9 @@ public WriteFilesResult expand(PCollection input) { if (!getIgnoreWindowing()) { writeFiles = writeFiles.withWindowedWrites(); } + if (getAutoSharding()) { + writeFiles = writeFiles.withAutoSharding(); + } if (getNoSpilling()) { writeFiles = writeFiles.withNoSpilling(); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java index 96635a37fac1..575ba6dedef7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java @@ -277,6 +277,7 @@ public static TypedWrite writeCustomType() { .setWindowedWrites(false) .setNoSpilling(false) .setSkipIfEmpty(false) + .setAutoSharding(false) .build(); } @@ -702,6 +703,9 @@ public abstract static class TypedWrite /** Whether to write windowed output files. */ abstract boolean getWindowedWrites(); + /** Whether to enable autosharding. */ + abstract boolean getAutoSharding(); + /** Whether to skip the spilling of data caused by having maxNumWritersPerBundle. */ abstract boolean getNoSpilling(); @@ -755,6 +759,8 @@ abstract Builder setNumShards( abstract Builder setWindowedWrites(boolean windowedWrites); + abstract Builder setAutoSharding(boolean windowedWrites); + abstract Builder setNoSpilling(boolean noSpilling); abstract Builder setSkipIfEmpty(boolean noSpilling); @@ -999,6 +1005,10 @@ public TypedWrite withWindowedWrites() { return toBuilder().setWindowedWrites(true).build(); } + public TypedWrite withAutoSharding() { + return toBuilder().setAutoSharding(true).build(); + } + /** See {@link WriteFiles#withNoSpilling()}. */ public TypedWrite withNoSpilling() { return toBuilder().setNoSpilling(true).build(); @@ -1097,6 +1107,9 @@ public WriteFilesResult expand(PCollection input) { if (getWindowedWrites()) { write = write.withWindowedWrites(); } + if (getAutoSharding()) { + write = write.withAutoSharding(); + } if (getNoSpilling()) { write = write.withNoSpilling(); } @@ -1268,6 +1281,11 @@ public Write withWindowedWrites() { return new Write(inner.withWindowedWrites()); } + /** See {@link TypedWrite#withAutoSharding}. */ + public Write withAutoSharding() { + return new Write(inner.withAutoSharding()); + } + /** See {@link TypedWrite#withNoSpilling}. */ public Write withNoSpilling() { return new Write(inner.withNoSpilling()); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java index 7359141c5b87..ff2b7e013fd5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java @@ -169,6 +169,7 @@ public static WriteFiles WriteFiles setNumShardsProvider( abstract Builder setWindowedWrites(boolean windowedWrites); + abstract Builder setWithAutoSharding(boolean withAutoSharding); + abstract Builder setMaxNumWritersPerBundle( int maxNumWritersPerBundle); @@ -308,6 +313,13 @@ public WriteFiles withRunnerDeterminedSharding() { return toBuilder().setComputeNumShards(null).setNumShardsProvider(null).build(); } + public WriteFiles withAutoSharding() { + checkArgument( + getComputeNumShards() == null && getNumShardsProvider() == null, + " sharding should be null if autosharding is specified."); + return toBuilder().setWithAutoSharding(true).build(); + } + /** * Returns a new {@link WriteFiles} that will write to the current {@link FileBasedSink} using the * specified sharding function to assign shard for inputs. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java index 6279f53bb9b1..4a4ef1dca6a1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java @@ -418,7 +418,7 @@ public CompositeBehavior enterCompositeTransform(Node node) { } /** - * Tests that {@link Pipeline#replaceAll(List)} throws when one of the PTransformOverride still + * Tests that {@link Pipeline#replaceAll(List)} succeeds when one of the PTransformOverride still * matches. */ @Test @@ -426,8 +426,7 @@ public void testReplaceAllIncomplete() { pipeline.enableAbandonedNodeEnforcement(false); pipeline.apply(GenerateSequence.from(0)); - // The order is such that the output of the second will match the first, which is not permitted - thrown.expect(IllegalStateException.class); + // The order is such that the output of the second will match the first, which is permitted. pipeline.replaceAll( ImmutableList.of( PTransformOverride.of( diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java index 2db20b92f27f..0eb1036f758f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java @@ -332,6 +332,18 @@ public void testWithRunnerDeterminedShardingUnbounded() throws IOException { true); } + @Test + @Category({NeedsRunner.class, UsesUnboundedPCollections.class}) + public void testWithShardingUnbounded() throws IOException { + runShardedWrite( + Arrays.asList("one", "two", "three", "four", "five", "six"), + Window.into(FixedWindows.of(Duration.standardSeconds(10))), + getBaseOutputFilename(), + WriteFiles.to(makeSimpleSink()).withWindowedWrites().withAutoSharding(), + null, + true); + } + @Test @Category({ NeedsRunner.class, From e867ed7cdce74beda4cce630173e3c53ee28cc50 Mon Sep 17 00:00:00 2001 From: Edward Cheng Date: Thu, 18 Jan 2024 02:47:11 -0800 Subject: [PATCH 024/169] Worker message plumbing (#29879) * use StreamingScalingReport for autoscaling signals * add unit test stub * spotless apply * comment test stub * add unit test * simplify response processing * spotless apply * add more reported metrics * remove byte metrics * fix bug * fix DataflowWorkUnitClient test * fix DataflowWorkUnitClient test * formatting * add check for scheduledtimer * option to options * fix timer check * revert long to int change * refactor timers * var type fix * fix timers refactoring * use arraylist instead of map * add timer to list * fix comment --------- Co-authored-by: scwhittle --- .../beam/gradle/BeamModulePlugin.groovy | 2 +- .../worker/DataflowWorkUnitClient.java | 42 +++++++++ .../worker/StreamingDataflowWorker.java | 89 ++++++++++++------- .../dataflow/worker/WorkUnitClient.java | 17 ++++ .../worker/util/BoundedQueueExecutor.java | 4 +- .../worker/DataflowWorkUnitClientTest.java | 24 +++++ 6 files changed, 145 insertions(+), 33 deletions(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index c93af317b1a2..a26cfaa457d1 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -730,7 +730,7 @@ class BeamModulePlugin implements Plugin { google_api_services_bigquery : "com.google.apis:google-api-services-bigquery:v2-rev20230812-$google_clients_version", // Keep version consistent with the version in google_cloud_resourcemanager, managed by google_cloud_platform_libraries_bom google_api_services_cloudresourcemanager : "com.google.apis:google-api-services-cloudresourcemanager:v1-rev20230806-$google_clients_version", - google_api_services_dataflow : "com.google.apis:google-api-services-dataflow:v1b3-rev20220920-$google_clients_version", + google_api_services_dataflow : "com.google.apis:google-api-services-dataflow:v1b3-rev20231203-$google_clients_version", google_api_services_healthcare : "com.google.apis:google-api-services-healthcare:v1-rev20240110-$google_clients_version", google_api_services_pubsub : "com.google.apis:google-api-services-pubsub:v1-rev20220904-$google_clients_version", // Keep version consistent with the version in google_cloud_nio, managed by google_cloud_platform_libraries_bom diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClient.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClient.java index ffa377fd3f82..bb39e3bd9af4 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClient.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClient.java @@ -31,14 +31,19 @@ import com.google.api.services.dataflow.model.LeaseWorkItemResponse; import com.google.api.services.dataflow.model.ReportWorkItemStatusRequest; import com.google.api.services.dataflow.model.ReportWorkItemStatusResponse; +import com.google.api.services.dataflow.model.SendWorkerMessagesRequest; +import com.google.api.services.dataflow.model.SendWorkerMessagesResponse; +import com.google.api.services.dataflow.model.StreamingScalingReport; import com.google.api.services.dataflow.model.WorkItem; import com.google.api.services.dataflow.model.WorkItemServiceState; import com.google.api.services.dataflow.model.WorkItemStatus; +import com.google.api.services.dataflow.model.WorkerMessage; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Optional; import javax.annotation.concurrent.ThreadSafe; import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions; @@ -48,6 +53,7 @@ import org.apache.beam.sdk.extensions.gcp.util.Transport; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; 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.joda.time.DateTime; import org.joda.time.Duration; import org.slf4j.Logger; @@ -269,4 +275,40 @@ public WorkItemServiceState reportWorkItemStatus(WorkItemStatus workItemStatus) logger.debug("ReportWorkItemStatus result: {}", state); return state; } + + /** Creates WorkerMessage from StreamingScalingReport */ + @Override + public WorkerMessage createWorkerMessageFromStreamingScalingReport( + StreamingScalingReport report) { + DateTime endTime = DateTime.now(); + logger.debug("Reporting WorkMessageResponse"); + Map labels = + ImmutableMap.of("JOB_ID", options.getJobId(), "WORKER_ID", options.getWorkerId()); + WorkerMessage msg = + new WorkerMessage() + .setTime(toCloudTime(endTime)) + .setStreamingScalingReport(report) + .setLabels(labels); + return msg; + } + + /** Reports the autoscaling signals to dataflow */ + @Override + public void reportWorkerMessage(WorkerMessage msg) throws IOException { + SendWorkerMessagesRequest request = + new SendWorkerMessagesRequest() + .setLocation(options.getRegion()) + .setWorkerMessages(Collections.singletonList(msg)); + SendWorkerMessagesResponse result = + dataflow + .projects() + .locations() + .workerMessages(options.getProject(), options.getRegion(), request) + .execute(); + if (result == null) { + logger.warn("Worker Message response is null"); + throw new IOException("Got null Worker Message response"); + } + // Currently no response is expected + } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index f68e5ba26c7f..c9a00ade6d8c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -26,6 +26,7 @@ import com.google.api.services.dataflow.model.Status; import com.google.api.services.dataflow.model.StreamingComputationConfig; import com.google.api.services.dataflow.model.StreamingConfigTask; +import com.google.api.services.dataflow.model.StreamingScalingReport; import com.google.api.services.dataflow.model.WorkItem; import com.google.api.services.dataflow.model.WorkItemStatus; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; @@ -276,13 +277,12 @@ public class StreamingDataflowWorker { private final HotKeyLogger hotKeyLogger; // Periodic sender of debug information to the debug capture service. private final DebugCapture.@Nullable Manager debugCaptureManager; - private ScheduledExecutorService refreshWorkTimer; - private ScheduledExecutorService statusPageTimer; - private ScheduledExecutorService globalWorkerUpdatesTimer; + // Collection of ScheduledExecutorServices that are running periodic functions. + private ArrayList scheduledExecutors = + new ArrayList(); private int retryLocallyDelayMs = 10000; // Periodically fires a global config request to dataflow service. Only used when windmill service // is enabled. - private ScheduledExecutorService globalConfigRefreshTimer; // Possibly overridden by streaming engine config. private int maxWorkItemCommitBytes = Integer.MAX_VALUE; @@ -579,14 +579,25 @@ public void start() { sampler.start(); // Periodically report workers counters and other updates. - globalWorkerUpdatesTimer = executorSupplier.apply("GlobalWorkerUpdatesTimer"); - globalWorkerUpdatesTimer.scheduleWithFixedDelay( + ScheduledExecutorService workerUpdateTimer = executorSupplier.apply("GlobalWorkerUpdates"); + workerUpdateTimer.scheduleWithFixedDelay( this::reportPeriodicWorkerUpdates, 0, options.getWindmillHarnessUpdateReportingPeriod().getMillis(), TimeUnit.MILLISECONDS); + scheduledExecutors.add(workerUpdateTimer); + + ScheduledExecutorService workerMessageTimer = executorSupplier.apply("ReportWorkerMessage"); + if (options.getWindmillHarnessUpdateReportingPeriod().getMillis() > 0) { + workerMessageTimer.scheduleWithFixedDelay( + this::reportPeriodicWorkerMessage, + 0, + options.getWindmillHarnessUpdateReportingPeriod().getMillis(), + TimeUnit.MILLISECONDS); + scheduledExecutors.add(workerMessageTimer); + } - refreshWorkTimer = executorSupplier.apply("RefreshWork"); + ScheduledExecutorService refreshWorkTimer = executorSupplier.apply("RefreshWork"); if (options.getActiveWorkRefreshPeriodMillis() > 0) { refreshWorkTimer.scheduleWithFixedDelay( new Runnable() { @@ -602,15 +613,17 @@ public void run() { options.getActiveWorkRefreshPeriodMillis(), options.getActiveWorkRefreshPeriodMillis(), TimeUnit.MILLISECONDS); + scheduledExecutors.add(refreshWorkTimer); } if (windmillServiceEnabled && options.getStuckCommitDurationMillis() > 0) { int periodMillis = Math.max(options.getStuckCommitDurationMillis() / 10, 100); refreshWorkTimer.scheduleWithFixedDelay( this::invalidateStuckCommits, periodMillis, periodMillis, TimeUnit.MILLISECONDS); + scheduledExecutors.add(refreshWorkTimer); } if (options.getPeriodicStatusPageOutputDirectory() != null) { - statusPageTimer = executorSupplier.apply("DumpStatusPages"); + ScheduledExecutorService statusPageTimer = executorSupplier.apply("DumpStatusPages"); statusPageTimer.scheduleWithFixedDelay( () -> { Collection pages = statusPages.getDebugCapturePages(); @@ -645,6 +658,7 @@ public void run() { 60, 60, TimeUnit.SECONDS); + scheduledExecutors.add(statusPageTimer); } reportHarnessStartup(); @@ -676,25 +690,15 @@ public void addWorkerStatusPage(BaseStatusServlet page) { public void stop() { try { - if (globalConfigRefreshTimer != null) { - globalConfigRefreshTimer.shutdown(); - } - globalWorkerUpdatesTimer.shutdown(); - if (refreshWorkTimer != null) { - refreshWorkTimer.shutdown(); - } - if (statusPageTimer != null) { - statusPageTimer.shutdown(); - } - if (globalConfigRefreshTimer != null) { - globalConfigRefreshTimer.awaitTermination(300, TimeUnit.SECONDS); - } - globalWorkerUpdatesTimer.awaitTermination(300, TimeUnit.SECONDS); - if (refreshWorkTimer != null) { - refreshWorkTimer.awaitTermination(300, TimeUnit.SECONDS); + for (ScheduledExecutorService timer : scheduledExecutors) { + if (timer != null) { + timer.shutdown(); + } } - if (statusPageTimer != null) { - statusPageTimer.awaitTermination(300, TimeUnit.SECONDS); + for (ScheduledExecutorService timer : scheduledExecutors) { + if (timer != null) { + timer.awaitTermination(300, TimeUnit.SECONDS); + } } statusPages.stop(); if (debugCaptureManager != null) { @@ -716,6 +720,7 @@ public void stop() { // one last send reportPeriodicWorkerUpdates(); + reportPeriodicWorkerMessage(); } catch (Exception e) { LOG.warn("Exception while shutting down: ", e); } @@ -1584,12 +1589,14 @@ private void schedulePeriodicGlobalConfigRequests() { LOG.info("windmillServerStub is now ready"); // Now start a thread that periodically refreshes the windmill service endpoint. - globalConfigRefreshTimer = executorSupplier.apply("GlobalConfigRefreshTimer"); - globalConfigRefreshTimer.scheduleWithFixedDelay( + ScheduledExecutorService configRefreshTimer = + executorSupplier.apply("GlobalConfigRefreshTimer"); + configRefreshTimer.scheduleWithFixedDelay( this::getGlobalConfig, 0, options.getGlobalConfigRefreshPeriod().getMillis(), TimeUnit.MILLISECONDS); + scheduledExecutors.add(configRefreshTimer); } private void getGlobalConfig() { @@ -1742,9 +1749,20 @@ private void updateThreadMetrics() { maxOutstandingBytes.getAndReset(); maxOutstandingBytes.addValue(workUnitExecutor.maximumBytesOutstanding()); outstandingBundles.getAndReset(); - outstandingBundles.addValue(workUnitExecutor.elementsOutstanding()); + outstandingBundles.addValue((long) workUnitExecutor.elementsOutstanding()); maxOutstandingBundles.getAndReset(); - maxOutstandingBundles.addValue(workUnitExecutor.maximumElementsOutstanding()); + maxOutstandingBundles.addValue((long) workUnitExecutor.maximumElementsOutstanding()); + } + + private void sendWorkerMessage() throws IOException { + StreamingScalingReport activeThreadsReport = + new StreamingScalingReport() + .setActiveThreadCount(workUnitExecutor.activeCount()) + .setActiveBundleCount(workUnitExecutor.elementsOutstanding()) + .setMaximumThreadCount(chooseMaximumNumberOfThreads()) + .setMaximumBundleCount(workUnitExecutor.maximumElementsOutstanding()); + workUnitClient.reportWorkerMessage( + workUnitClient.createWorkerMessageFromStreamingScalingReport(activeThreadsReport)); } @VisibleForTesting @@ -1760,6 +1778,17 @@ public void reportPeriodicWorkerUpdates() { } } + @VisibleForTesting + public void reportPeriodicWorkerMessage() { + try { + sendWorkerMessage(); + } catch (IOException e) { + LOG.warn("Failed to send worker messages", e); + } catch (Exception e) { + LOG.error("Unexpected exception while trying to send worker messages", e); + } + } + /** * Returns key for a counter update. It is a String in case of legacy counter and * CounterStructuredName in the case of a structured counter. diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkUnitClient.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkUnitClient.java index 82fbcd82c131..6f26f4048070 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkUnitClient.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkUnitClient.java @@ -17,9 +17,11 @@ */ package org.apache.beam.runners.dataflow.worker; +import com.google.api.services.dataflow.model.StreamingScalingReport; import com.google.api.services.dataflow.model.WorkItem; import com.google.api.services.dataflow.model.WorkItemServiceState; import com.google.api.services.dataflow.model.WorkItemStatus; +import com.google.api.services.dataflow.model.WorkerMessage; import java.io.IOException; import java.util.Optional; @@ -49,4 +51,19 @@ interface WorkUnitClient { * @return a {@link WorkItemServiceState} (e.g. a new stop position) */ WorkItemServiceState reportWorkItemStatus(WorkItemStatus workItemStatus) throws IOException; + + /** + * Creates a {@link WorkerMessage} containing the given Streaming Scaling Report + * + * @param report the StreamingScalingReport containing autoscaling metrics + * @return a {@link WorkerMessage} + */ + WorkerMessage createWorkerMessageFromStreamingScalingReport(StreamingScalingReport report); + + /** + * Reports the autoscaling signals with a {@link StreamingScalingReport}. + * + * @param msg the WorkerMessage to report + */ + void reportWorkerMessage(WorkerMessage msg) throws IOException; } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutor.java index dcff1f73f10f..cd4c727e310e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutor.java @@ -127,7 +127,7 @@ public long bytesOutstanding() { return bytesOutstanding; } - public long elementsOutstanding() { + public int elementsOutstanding() { return elementsOutstanding; } @@ -135,7 +135,7 @@ public long maximumBytesOutstanding() { return maximumBytesOutstanding; } - public long maximumElementsOutstanding() { + public int maximumElementsOutstanding() { return maximumElementsOutstanding; } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java index 5329fb0f601c..7720de3563b8 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java @@ -31,8 +31,12 @@ import com.google.api.services.dataflow.model.LeaseWorkItemRequest; import com.google.api.services.dataflow.model.LeaseWorkItemResponse; import com.google.api.services.dataflow.model.MapTask; +import com.google.api.services.dataflow.model.SendWorkerMessagesRequest; +import com.google.api.services.dataflow.model.SendWorkerMessagesResponse; import com.google.api.services.dataflow.model.SeqMapTask; +import com.google.api.services.dataflow.model.StreamingScalingReport; import com.google.api.services.dataflow.model.WorkItem; +import com.google.api.services.dataflow.model.WorkerMessage; import java.io.IOException; import java.util.Optional; import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions; @@ -227,6 +231,26 @@ public void testCloudServiceCallMultipleWorkItems() throws Exception { client.getWorkItem(); } + @Test + public void testReportWorkerMessage() throws Exception { + MockLowLevelHttpResponse response = new MockLowLevelHttpResponse(); + response.setContentType(Json.MEDIA_TYPE); + SendWorkerMessagesResponse workerMessage = new SendWorkerMessagesResponse(); + workerMessage.setFactory(Transport.getJsonFactory()); + response.setContent(workerMessage.toPrettyString()); + when(request.execute()).thenReturn(response); + StreamingScalingReport activeThreadsReport = + new StreamingScalingReport().setActiveThreadCount(1); + WorkUnitClient client = new DataflowWorkUnitClient(pipelineOptions, LOG); + WorkerMessage msg = client.createWorkerMessageFromStreamingScalingReport(activeThreadsReport); + client.reportWorkerMessage(msg); + + SendWorkerMessagesRequest actualRequest = + Transport.getJsonFactory() + .fromString(request.getContentAsString(), SendWorkerMessagesRequest.class); + assertEquals(ImmutableList.of(msg), actualRequest.getWorkerMessages()); + } + private LowLevelHttpResponse generateMockResponse(WorkItem... workItems) throws Exception { MockLowLevelHttpResponse response = new MockLowLevelHttpResponse(); response.setContentType(Json.MEDIA_TYPE); From b4c23b32f2b80ce052c8a235e5064c69f37df992 Mon Sep 17 00:00:00 2001 From: Svetak Sundhar Date: Thu, 18 Jan 2024 14:54:31 +0000 Subject: [PATCH 025/169] Correct Examples of utilizing `UploadToDicomStore` (#30008) * Create HealthcareUtils file with shared resources * revert * This PR introduces the correct transforms to utilize when calling UploadToDicomStore() * This PR introduces the correct transforms to utilize when calling UploadToDicomStore() * indent * indent --- .../apache_beam/io/gcp/healthcare/dicomio.py | 31 ++++++++++--------- 1 file changed, 17 insertions(+), 14 deletions(-) diff --git a/sdks/python/apache_beam/io/gcp/healthcare/dicomio.py b/sdks/python/apache_beam/io/gcp/healthcare/dicomio.py index b31e0d3a78d3..a73de19d5a35 100644 --- a/sdks/python/apache_beam/io/gcp/healthcare/dicomio.py +++ b/sdks/python/apache_beam/io/gcp/healthcare/dicomio.py @@ -77,7 +77,7 @@ In the example above, the pipeline is listening to a pubsub topic and waiting for messages from DICOM API. When a new DICOM file comes into the storage, the -pipeline will receive a pubsub message, convert it to a Qido request dict and +pipeline will receive a pubsub message, convert it to a Qido request dict, and feed it to DicomSearch() PTransform. As a result, users can get the metadata for every new DICOM file. Note that not every pubsub message received is from DICOM API, so we to filter the results first. @@ -85,23 +85,26 @@ Store a DICOM file in a DICOM storage =================================================== UploadToDicomStore() wraps store request API and users can use it to send a -DICOM file to a DICOM store. It supports two types of input: 1.file data in -byte[] 2.fileio object. Users should set the 'input_type' when initialzing +DICOM file to a DICOM store. It supports two types of input: 1. fileio object +2. file data in byte[]. Users should set the 'input_type' when initialzing this PTransform. Here are the examples: + input_dict = {'project_id': 'abc123', 'type': 'instances',...} + str_input = json.dumps(dict_input) + temp_dir = '%s%s' % (self._new_tempdir(), os.sep) + self._create_temp_file(dir=temp_dir, content=str_input) with Pipeline() as p: - input_dict = {'project_id': 'abc123', 'type': 'instances',...} - path = "gcs://bucketname/something/a.dcm" - match = p | fileio.MatchFiles(path) - fileio_obj = match | fileio.ReadAll() - results = fileio_obj | UploadToDicomStore(input_dict, 'fileio') - + results = ( + p | beam.Create([FileSystems.join(temp_dir, '*')]) + | fileio.MatchAll() | fileio.ReadMatches() + | UploadToDicomStore(input_dict, 'fileio')) + + input_dict = {'project_id': 'abc123', 'type': 'instances',...} + str_input = json.dumps(dict_input) + bytes_input = bytes(str_input.encode("utf-8")) with Pipeline() as p: - input_dict = {'project_id': 'abc123', 'type': 'instances',...} - f = open("abc.dcm", "rb") - dcm_file = f.read() - byte_file = p | 'create byte file' >> beam.Create([dcm_file]) - results = byte_file | UploadToDicomStore(input_dict, 'bytes') + results = ( + p | beam.Create([bytes_input]) | UploadToDicomStore(input_dict, 'bytes')) The first example uses a PCollection of fileio objects as input. UploadToDicomStore will read DICOM files from the objects and send them From 65811eed6ea2a7efafd1e596d88d7ef92a9e6247 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 9 Jan 2024 12:50:53 -0500 Subject: [PATCH 026/169] Upgrade checkerframework to 3.42.0 This includes improvements in checker's safety. Files that needed adjustment were adjusted, rather than suppressing the warning. --- .../beam/gradle/BeamModulePlugin.groovy | 2 +- .../dataflow/AbstractPipelineLauncher.java | 17 ++- .../core/metrics/ExecutionStateSampler.java | 3 +- .../beam/runners/flink/metrics/Metrics.java | 3 +- .../runners/dataflow/DataflowMetrics.java | 8 +- runners/spark/spark_runner.gradle | 2 + sdks/java/core/build.gradle | 1 + .../java/org/apache/beam/sdk/io/Read.java | 131 ++++++++++-------- .../beam/sdk/values/RowWithGetters.java | 21 ++- .../protobuf/ProtoSchemaTranslator.java | 9 +- .../sorter/BufferedExternalSorter.java | 6 +- .../beam/sdk/io/aws2/schemas/AwsTypes.java | 1 + .../apache/beam/sdk/io/aws2/sqs/SqsIO.java | 4 +- .../sdk/io/gcp/bigquery/RetryManager.java | 2 +- .../org/apache/beam/sdk/io/jdbc/JdbcIO.java | 27 ++-- .../beam/sdk/io/mongodb/MongoDbGridFSIO.java | 10 +- .../apache/beam/sdk/io/mongodb/MongoDbIO.java | 2 +- .../SingleStoreDefaultRowMapper.java | 5 +- .../beam/sdk/tpcds/TableSchemaJSONLoader.java | 14 +- 19 files changed, 166 insertions(+), 102 deletions(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index f8db63e3a440..f51f51e3468f 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -593,7 +593,7 @@ class BeamModulePlugin implements Plugin { def aws_java_sdk2_version = "2.20.47" def cassandra_driver_version = "3.10.2" def cdap_version = "6.5.1" - def checkerframework_version = "3.27.0" + def checkerframework_version = "3.42.0" def classgraph_version = "4.8.162" def dbcp2_version = "2.9.0" def errorprone_version = "2.10.0" diff --git a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/dataflow/AbstractPipelineLauncher.java b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/dataflow/AbstractPipelineLauncher.java index b3bb29b960b7..55f700f506ee 100644 --- a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/dataflow/AbstractPipelineLauncher.java +++ b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/dataflow/AbstractPipelineLauncher.java @@ -22,6 +22,7 @@ import static org.apache.beam.it.common.PipelineLauncher.JobState.PENDING_STATES; import static org.apache.beam.it.common.logging.LogStrings.formatForLogging; import static org.apache.beam.it.common.utils.RetryUtil.clientRetryPolicy; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import com.google.api.client.util.ArrayMap; import com.google.api.services.dataflow.Dataflow; @@ -215,10 +216,18 @@ public Map getMetrics(String project, String region, String jobI // currently, reporting distribution metrics as 4 separate scalar metrics @SuppressWarnings("rawtypes") ArrayMap distributionMap = (ArrayMap) metricUpdate.getDistribution(); - result.put(metricName + "_COUNT", ((Number) distributionMap.get("count")).doubleValue()); - result.put(metricName + "_MIN", ((Number) distributionMap.get("min")).doubleValue()); - result.put(metricName + "_MAX", ((Number) distributionMap.get("max")).doubleValue()); - result.put(metricName + "_SUM", ((Number) distributionMap.get("sum")).doubleValue()); + result.put( + metricName + "_COUNT", + checkStateNotNull(((Number) distributionMap.get("count"))).doubleValue()); + result.put( + metricName + "_MIN", + checkStateNotNull(((Number) distributionMap.get("min"))).doubleValue()); + result.put( + metricName + "_MAX", + checkStateNotNull(((Number) distributionMap.get("max"))).doubleValue()); + result.put( + metricName + "_SUM", + checkStateNotNull(((Number) distributionMap.get("sum"))).doubleValue()); } else if (metricUpdate.getGauge() != null) { LOG.warn("Gauge metric {} cannot be handled.", metricName); // not sure how to handle gauge metrics diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/ExecutionStateSampler.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/ExecutionStateSampler.java index 9478c218b075..8ecfa3ad1710 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/ExecutionStateSampler.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/ExecutionStateSampler.java @@ -127,6 +127,7 @@ synchronized void start(ExecutorService executor) { } public synchronized void stop() { + final Future executionSamplerFuture = this.executionSamplerFuture; if (executionSamplerFuture == null) { return; } @@ -142,7 +143,7 @@ public synchronized void stop() { } catch (ExecutionException e) { throw new RuntimeException("Exception in state sampler", e); } finally { - executionSamplerFuture = null; + this.executionSamplerFuture = null; } } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/Metrics.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/Metrics.java index 697b22bdad53..116b7fc62f3a 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/Metrics.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/Metrics.java @@ -17,6 +17,7 @@ */ package org.apache.beam.runners.flink.metrics; +import java.util.Objects; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Histogram; @@ -30,7 +31,7 @@ public static String toString(Metric metric) { if (metric instanceof Counter) { return Long.toString(((Counter) metric).getCount()); } else if (metric instanceof Gauge) { - return ((Gauge) metric).getValue().toString(); + return Objects.toString(((Gauge) metric).getValue()); } else if (metric instanceof Meter) { return Double.toString(((Meter) metric).getRate()); } else if (metric instanceof Histogram) { diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowMetrics.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowMetrics.java index 9e6f70a54213..c5023a57d8d6 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowMetrics.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowMetrics.java @@ -187,10 +187,10 @@ private DistributionResult getDistributionValue(MetricUpdate metricUpdate) { return DistributionResult.IDENTITY_ELEMENT; } ArrayMap distributionMap = (ArrayMap) metricUpdate.getDistribution(); - long count = ((Number) distributionMap.get("count")).longValue(); - long min = ((Number) distributionMap.get("min")).longValue(); - long max = ((Number) distributionMap.get("max")).longValue(); - long sum = ((Number) distributionMap.get("sum")).longValue(); + long count = checkArgumentNotNull(((Number) distributionMap.get("count"))).longValue(); + long min = checkArgumentNotNull(((Number) distributionMap.get("min"))).longValue(); + long max = checkArgumentNotNull(((Number) distributionMap.get("max"))).longValue(); + long sum = checkArgumentNotNull(((Number) distributionMap.get("sum"))).longValue(); return DistributionResult.create(sum, count, min, max); } diff --git a/runners/spark/spark_runner.gradle b/runners/spark/spark_runner.gradle index 74013de6107d..a1193987f8e3 100644 --- a/runners/spark/spark_runner.gradle +++ b/runners/spark/spark_runner.gradle @@ -25,9 +25,11 @@ applyJavaNature( archivesBaseName: (project.hasProperty('archives_base_name') ? archives_base_name : archivesBaseName), exportJavadoc: (project.hasProperty('exportJavadoc') ? exportJavadoc : true), classesTriggerCheckerBugs: [ + 'Aggregators': 'https://github.com/typetools/checker-framework/issues/6388#issuecomment-1885532351', 'SparkAssignWindowFn': 'https://github.com/typetools/checker-framework/issues/3793', 'SparkCombineFn' : 'https://github.com/typetools/checker-framework/issues/3793', 'WindowingHelpers' : 'https://github.com/typetools/checker-framework/issues/3793', + 'WindowAssignTranslatorBatch': 'https://github.com/typetools/checker-framework/issues/6388#issuecomment-1885532351', ], ) diff --git a/sdks/java/core/build.gradle b/sdks/java/core/build.gradle index 7c788eaba49d..f0cd084d11a0 100644 --- a/sdks/java/core/build.gradle +++ b/sdks/java/core/build.gradle @@ -21,6 +21,7 @@ applyJavaNature( automaticModuleName: 'org.apache.beam.sdk', classesTriggerCheckerBugs: [ 'DoFnTester': 'https://github.com/typetools/checker-framework/issues/3776', + 'FileIO': 'https://github.com/typetools/checker-framework/issues/6388', 'MergingActiveWindowSetTest': 'https://github.com/typetools/checker-framework/issues/3776', 'WindowFnTestUtils': 'https://github.com/typetools/checker-framework/issues/3776', ], diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java index 715e81c0f930..901856b93dae 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java @@ -69,6 +69,7 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Cache; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.RemovalListener; +import org.checkerframework.checker.nullness.qual.EnsuresNonNull; import org.checkerframework.checker.nullness.qual.Nullable; import org.checkerframework.common.value.qual.ArrayLen; import org.checkerframework.dataflow.qual.Pure; @@ -351,34 +352,7 @@ public boolean tryClaim(TimestampedValue[] position) { return false; } try { - if (currentReader == null) { - currentReader = initialRestriction.createReader(pipelineOptions); - if (!currentReader.start()) { - claimedAll = true; - try { - currentReader.close(); - } finally { - currentReader = null; - } - return false; - } - position[0] = - TimestampedValue.of( - currentReader.getCurrent(), currentReader.getCurrentTimestamp()); - return true; - } - if (!currentReader.advance()) { - claimedAll = true; - try { - currentReader.close(); - } finally { - currentReader = null; - } - return false; - } - position[0] = - TimestampedValue.of(currentReader.getCurrent(), currentReader.getCurrentTimestamp()); - return true; + return tryClaimOrThrow(position); } catch (IOException e) { if (currentReader != null) { try { @@ -393,6 +367,37 @@ public boolean tryClaim(TimestampedValue[] position) { } } + private boolean tryClaimOrThrow(TimestampedValue[] position) throws IOException { + BoundedSource.BoundedReader currentReader = this.currentReader; + if (currentReader == null) { + BoundedSource.BoundedReader newReader = + initialRestriction.createReader(pipelineOptions); + if (!newReader.start()) { + claimedAll = true; + newReader.close(); + return false; + } + position[0] = + TimestampedValue.of(newReader.getCurrent(), newReader.getCurrentTimestamp()); + this.currentReader = newReader; + return true; + } + + if (!currentReader.advance()) { + claimedAll = true; + try { + currentReader.close(); + } finally { + this.currentReader = null; + } + return false; + } + + position[0] = + TimestampedValue.of(currentReader.getCurrent(), currentReader.getCurrentTimestamp()); + return true; + } + @Override protected void finalize() throws Throwable { if (currentReader != null) { @@ -415,6 +420,7 @@ public BoundedSourceT currentRestriction() { @Override public @Nullable SplitResult trySplit(double fractionOfRemainder) { + BoundedSource.BoundedReader currentReader = this.currentReader; if (currentReader == null) { return null; } @@ -859,13 +865,15 @@ private Object createCacheKey( source, checkpoint, BoundedWindow.TIMESTAMP_MIN_VALUE)); } + @EnsuresNonNull("currentReader") private void initializeCurrentReader() throws IOException { checkState(currentReader == null); Object cacheKey = createCacheKey(initialRestriction.getSource(), initialRestriction.getCheckpoint()); - currentReader = cachedReaders.getIfPresent(cacheKey); - if (currentReader == null) { - currentReader = + UnboundedReader cachedReader = cachedReaders.getIfPresent(cacheKey); + + if (cachedReader == null) { + this.currentReader = initialRestriction .getSource() .createReader(pipelineOptions, initialRestriction.getCheckpoint()); @@ -874,6 +882,7 @@ private void initializeCurrentReader() throws IOException { // We also remove this cache entry to avoid eviction. readerHasBeenStarted = true; cachedReaders.invalidate(cacheKey); + this.currentReader = cachedReader; } } @@ -891,50 +900,56 @@ private void cacheCurrentReader( @Override public boolean tryClaim(@Nullable UnboundedSourceValue @ArrayLen(1) [] position) { try { - if (currentReader == null) { - initializeCurrentReader(); - } - checkStateNotNull(currentReader, "currentReader null after initialization"); - if (currentReader instanceof EmptyUnboundedSource.EmptyUnboundedReader) { - return false; - } - if (!readerHasBeenStarted) { - readerHasBeenStarted = true; - if (!currentReader.start()) { - position[0] = null; - return true; - } - } else if (!currentReader.advance()) { - position[0] = null; - return true; - } - position[0] = - UnboundedSourceValue.create( - currentReader.getCurrentRecordId(), - currentReader.getCurrent(), - currentReader.getCurrentTimestamp(), - currentReader.getWatermark()); - return true; + return tryClaimOrThrow(position); } catch (IOException e) { - if (currentReader != null) { + if (this.currentReader != null) { try { currentReader.close(); } catch (IOException closeException) { e.addSuppressed(closeException); } finally { - currentReader = null; + this.currentReader = null; } } throw new RuntimeException(e); } } + private boolean tryClaimOrThrow( + @Nullable UnboundedSourceValue @ArrayLen(1) [] position) throws IOException { + if (this.currentReader == null) { + initializeCurrentReader(); + } + UnboundedSource.UnboundedReader currentReader = this.currentReader; + if (currentReader instanceof EmptyUnboundedSource.EmptyUnboundedReader) { + return false; + } + if (!readerHasBeenStarted) { + readerHasBeenStarted = true; + if (!currentReader.start()) { + position[0] = null; + return true; + } + } else if (!currentReader.advance()) { + position[0] = null; + return true; + } + position[0] = + UnboundedSourceValue.create( + currentReader.getCurrentRecordId(), + currentReader.getCurrent(), + currentReader.getCurrentTimestamp(), + currentReader.getWatermark()); + return true; + } + /** The value is invalid if {@link #tryClaim} has ever thrown an exception. */ @Override public UnboundedSourceRestriction currentRestriction() { if (currentReader == null) { return initialRestriction; } + UnboundedReader currentReader = this.currentReader; Instant watermark = ensureTimestampWithinBounds(currentReader.getWatermark()); // We convert the reader to the empty reader to mark that we are done. if (!(currentReader instanceof EmptyUnboundedSource.EmptyUnboundedReader) @@ -945,7 +960,7 @@ public UnboundedSourceRestriction currentRestriction() { } catch (IOException e) { LOG.warn("Failed to close UnboundedReader.", e); } finally { - currentReader = + this.currentReader = EmptyUnboundedSource.INSTANCE.createReader( PipelineOptionsFactory.create(), checkpointT); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/RowWithGetters.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/RowWithGetters.java index 178ba8923e5a..cb4d83550577 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/RowWithGetters.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/RowWithGetters.java @@ -17,17 +17,21 @@ */ package org.apache.beam.sdk.values; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.TreeMap; +import java.util.function.Function; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.schemas.Factory; import org.apache.beam.sdk.schemas.FieldValueGetter; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.Field; import org.apache.beam.sdk.schemas.Schema.TypeName; +import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; /** @@ -41,7 +45,7 @@ public class RowWithGetters extends Row { private final Object getterTarget; private final List getters; - private @Nullable Map cache = null; + private @Nullable Map cache = null; RowWithGetters( Schema schema, Factory> getterFactory, Object getterTarget) { @@ -65,7 +69,20 @@ public class RowWithGetters extends Row { if (cache == null) { cache = new TreeMap<>(); } - fieldValue = cache.computeIfAbsent(fieldIdx, idx -> getters.get(idx).get(getterTarget)); + fieldValue = + cache.computeIfAbsent( + fieldIdx, + new Function() { + @Override + public Object apply(Integer idx) { + FieldValueGetter getter = getters.get(idx); + checkStateNotNull(getter); + @SuppressWarnings("nullness") + @NonNull + Object value = getter.get(getterTarget); + return value; + } + }); } else { fieldValue = getters.get(fieldIdx).get(getterTarget); } diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoSchemaTranslator.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoSchemaTranslator.java index 9f679e61ecf3..2b06f1e88d73 100644 --- a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoSchemaTranslator.java +++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoSchemaTranslator.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.extensions.protobuf; +import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import com.google.protobuf.Descriptors; @@ -385,10 +386,10 @@ private static Schema.Options.Builder getOptions( case ITERABLE: Field field = Field.of("OPTION", fieldType); ProtoDynamicMessageSchema schema = ProtoDynamicMessageSchema.forSchema(Schema.of(field)); - optionsBuilder.setOption( - prefix + fieldDescriptor.getFullName(), - fieldType, - schema.createConverter(field).convertFromProtoValue(entry.getValue())); + @SuppressWarnings("rawtypes") + ProtoDynamicMessageSchema.Convert convert = schema.createConverter(field); + Object value = checkArgumentNotNull(convert.convertFromProtoValue(entry.getValue())); + optionsBuilder.setOption(prefix + fieldDescriptor.getFullName(), fieldType, value); break; case MAP: case DATETIME: diff --git a/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/BufferedExternalSorter.java b/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/BufferedExternalSorter.java index 6cec1b7c71ff..a4945ef15797 100644 --- a/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/BufferedExternalSorter.java +++ b/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/BufferedExternalSorter.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.extensions.sorter; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import java.io.IOException; @@ -24,7 +25,6 @@ import org.apache.beam.sdk.extensions.sorter.ExternalSorter.Options.SorterType; import org.apache.beam.sdk.values.KV; import org.checkerframework.checker.nullness.qual.Nullable; -import org.checkerframework.checker.nullness.qual.RequiresNonNull; /** * {@link Sorter} that will use in memory sorting until the values can't fit into memory and will @@ -121,7 +121,7 @@ public static BufferedExternalSorter create(Options options) { @Override public void add(KV record) throws IOException { - if (inMemorySorter != null) { + if (this.inMemorySorter != null) { if (inMemorySorter.addIfRoom(record)) { return; } else { @@ -139,8 +139,8 @@ public void add(KV record) throws IOException { * Transfers all of the records loaded so far into the in memory sorter over to the external * sorter. */ - @RequiresNonNull("inMemorySorter") private void transferToExternalSorter() throws IOException { + checkStateNotNull(this.inMemorySorter); for (KV record : inMemorySorter.sort()) { externalSorter.add(record); } diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/schemas/AwsTypes.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/schemas/AwsTypes.java index b941079b121f..f5647c040526 100644 --- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/schemas/AwsTypes.java +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/schemas/AwsTypes.java @@ -177,6 +177,7 @@ boolean needsConversion(SdkField field) { || (type.equals(LIST) && needsConversion(elementField(field))); } + @SuppressWarnings("nullness") private static SerializableFunction andThen( SerializableFunction fn1, SerializableFunction fn2) { return v -> fn2.apply(fn1.apply(v)); diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/SqsIO.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/SqsIO.java index f7f767ab85ee..d8f797e01486 100644 --- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/SqsIO.java +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/SqsIO.java @@ -766,7 +766,7 @@ Batch getLocked(T record) { if (batch == null || !batch.lock(true)) { batch = Batch.createLocked(checkStateNotNull(spec.queueUrl()), spec); } - return batch; + return checkStateNotNull(batch); } @Override @@ -777,7 +777,7 @@ boolean submitAll() { @Override void submitExpired(boolean throwFailures) { if (batch != null && batch.isExpired()) { - lockAndSubmit(batch, throwFailures); + lockAndSubmit(checkStateNotNull(batch), throwFailures); } } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/RetryManager.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/RetryManager.java index b82f39bfd559..1a7202de0a56 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/RetryManager.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/RetryManager.java @@ -169,7 +169,7 @@ void run(Executor executor) { } boolean await() throws Exception { - Preconditions.checkStateNotNull(callback); + Callback callback = Preconditions.checkStateNotNull(this.callback); callback.await(); return callback.getFailed(); } diff --git a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java index e2a4a8e1072a..35d94442f37a 100644 --- a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java +++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java @@ -642,16 +642,23 @@ public DataSource buildDatasource() { String password = getPassword().get(); basicDataSource.setPassword(password); } - if (getConnectionProperties() != null && getConnectionProperties().get() != null) { - basicDataSource.setConnectionProperties(getConnectionProperties().get()); + if (getConnectionProperties() != null) { + String connectionProperties = getConnectionProperties().get(); + if (connectionProperties != null) { + basicDataSource.setConnectionProperties(connectionProperties); + } } - if (getConnectionInitSqls() != null - && getConnectionInitSqls().get() != null - && !getConnectionInitSqls().get().isEmpty()) { - basicDataSource.setConnectionInitSqls(getConnectionInitSqls().get()); + if (getConnectionInitSqls() != null) { + Collection connectionInitSqls = getConnectionInitSqls().get(); + if (connectionInitSqls != null && !connectionInitSqls.isEmpty()) { + basicDataSource.setConnectionInitSqls(connectionInitSqls); + } } - if (getMaxConnections() != null && getMaxConnections().get() != null) { - basicDataSource.setMaxTotal(getMaxConnections().get()); + if (getMaxConnections() != null) { + Integer maxConnections = getMaxConnections().get(); + if (maxConnections != null) { + basicDataSource.setMaxTotal(maxConnections); + } } if (getDriverClassLoader() != null) { basicDataSource.setDriverClassLoader(getDriverClassLoader()); @@ -1769,6 +1776,8 @@ static PCollection> batchElements( @Nullable List outputList; @ProcessElement + @SuppressWarnings( + "nullness") // https://github.com/typetools/checker-framework/issues/6389 public void process(ProcessContext c) { if (outputList == null) { outputList = new ArrayList<>(); @@ -2513,11 +2522,13 @@ public void setup() { } private Connection getConnection() throws SQLException { + Connection connection = this.connection; if (connection == null) { connection = checkStateNotNull(dataSource).getConnection(); connection.setAutoCommit(false); preparedStatement = connection.prepareStatement(checkStateNotNull(spec.getStatement()).get()); + this.connection = connection; } return connection; } diff --git a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java index 1e7dd6dcde0a..07cc238c7e6b 100644 --- a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java +++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java @@ -647,11 +647,13 @@ public void setup() throws Exception { public void startBundle() { GridFS gridfs = Preconditions.checkStateNotNull(this.gridfs); String filename = Preconditions.checkStateNotNull(spec.filename()); - gridFsFile = gridfs.createFile(filename); + GridFSInputFile gridFsFile = gridfs.createFile(filename); if (spec.chunkSize() != null) { gridFsFile.setChunkSize(spec.chunkSize()); } outputStream = gridFsFile.getOutputStream(); + + this.gridFsFile = gridFsFile; } @ProcessElement @@ -664,9 +666,10 @@ public void processElement(ProcessContext context) throws Exception { @FinishBundle public void finishBundle() throws Exception { if (outputStream != null) { + OutputStream outputStream = this.outputStream; outputStream.flush(); outputStream.close(); - outputStream = null; + this.outputStream = null; } if (gridFsFile != null) { gridFsFile = null; @@ -677,9 +680,10 @@ public void finishBundle() throws Exception { public void teardown() throws Exception { try { if (outputStream != null) { + OutputStream outputStream = this.outputStream; outputStream.flush(); outputStream.close(); - outputStream = null; + this.outputStream = null; } if (gridFsFile != null) { gridFsFile = null; diff --git a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java index 53c39e2406fd..13c5c92d87ba 100644 --- a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java +++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java @@ -990,7 +990,7 @@ public void startBundle() { @ProcessElement public void processElement(ProcessContext ctx) { - Preconditions.checkStateNotNull(batch); + List batch = Preconditions.checkStateNotNull(this.batch); // Need to copy the document because mongoCollection.insertMany() will mutate it // before inserting (will assign an id). diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapper.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapper.java index 72d5f3ffa28b..6837f1893d06 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapper.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapper.java @@ -42,6 +42,7 @@ import static org.apache.beam.sdk.schemas.Schema.FieldType.INT32; import static org.apache.beam.sdk.schemas.Schema.FieldType.INT64; import static org.apache.beam.sdk.schemas.Schema.FieldType.STRING; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import java.io.Serializable; import java.sql.ResultSet; @@ -87,9 +88,7 @@ public void init(ResultSetMetaData metaData) throws SQLException { @Override public Row mapRow(ResultSet resultSet) throws Exception { - if (schema == null) { - throw new UnsupportedOperationException("mapRow is called before init"); - } + Schema schema = checkStateNotNull(this.schema, "mapRow is called before init"); Row.Builder rowBuilder = Row.withSchema(schema); diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoader.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoader.java index c408919a53bd..485fa83a4a8e 100644 --- a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoader.java +++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoader.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.tpcds; +import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; + import java.io.IOException; import java.util.ArrayList; import java.util.Iterator; @@ -59,16 +61,16 @@ public static String parseTableSchema(String tableName) throws Exception { StringBuilder schemaStringBuilder = new StringBuilder(); Iterator jsonArrIterator = jsonArray.iterator(); - Iterator recordIterator; while (jsonArrIterator.hasNext()) { - recordIterator = ((Map) jsonArrIterator.next()).entrySet().iterator(); + Map jsonMap = checkArgumentNotNull((Map) jsonArrIterator.next()); + Iterator recordIterator = jsonMap.entrySet().iterator(); while (recordIterator.hasNext()) { - Map.Entry pair = (Map.Entry) recordIterator.next(); - - if (pair.getKey().equals("type")) { + Map.Entry pair = checkArgumentNotNull((Map.Entry) recordIterator.next()); + Object key = checkArgumentNotNull(pair.getKey()); + if (key.equals("type")) { // If the key of the pair is "type", make some modification before appending it to the // schemaStringBuilder, then append a comma. - String typeName = (String) pair.getValue(); + String typeName = checkArgumentNotNull((String) pair.getValue()); if (typeName.equalsIgnoreCase("identifier") || typeName.equalsIgnoreCase("integer")) { // Use long type to represent int, prevent overflow schemaStringBuilder.append("bigint"); From 1de8454ddd215b9659fd049bded6a4b4c484b12d Mon Sep 17 00:00:00 2001 From: Ritesh Ghorse Date: Thu, 18 Jan 2024 21:22:11 -0500 Subject: [PATCH 027/169] [Python] Enrichment Transform with BigTable handler (#30001) * enrichment v1 * add documentation * add doc comment * rerun * update docs, lint * update docs, lint * add generic type * add generic type * adjust doc path * create test row * use request type * use request type * change module name * more tests * remove non-functional params * lint, doc * change types for general use * callable type * dict type * update signatures * fix unit test * bigtable with column family, ids, rrio-throttler * update tests for row filter * convert handler types from dict to Row * update tests for bigtable * ran pydocs * ran pydocs * mark postcommit * remove _test file, fix import * enable postcommit * add more tests * skip tests when dependencies are not installed * add deleted imports from last commit * add skip test condition * fix import order, add TooManyRequests to try-catch * make throttler, repeater non-optional * add exception level and tests * correct pydoc statement * add throttle tests * add bigtable improvements * default app_profile_id * add documentation, ignore None assignment * add to changes.md * change test structure that throws exception, skip http test for now * drop postcommit trigger file --- CHANGES.md | 2 +- sdks/python/apache_beam/io/requestresponse.py | 413 ++++++++++++++++++ ..._it_test.py => requestresponse_it_test.py} | 37 +- .../apache_beam/io/requestresponse_test.py | 156 +++++++ .../apache_beam/io/requestresponseio.py | 218 --------- .../apache_beam/io/requestresponseio_test.py | 88 ---- .../apache_beam/transforms/enrichment.py | 137 ++++++ .../enrichment_handlers/__init__.py | 16 + .../enrichment_handlers/bigtable.py | 151 +++++++ .../enrichment_handlers/bigtable_it_test.py | 300 +++++++++++++ .../transforms/enrichment_it_test.py | 162 +++++++ .../apache_beam/transforms/enrichment_test.py | 41 ++ 12 files changed, 1398 insertions(+), 323 deletions(-) create mode 100644 sdks/python/apache_beam/io/requestresponse.py rename sdks/python/apache_beam/io/{requestresponseio_it_test.py => requestresponse_it_test.py} (86%) create mode 100644 sdks/python/apache_beam/io/requestresponse_test.py delete mode 100644 sdks/python/apache_beam/io/requestresponseio.py delete mode 100644 sdks/python/apache_beam/io/requestresponseio_test.py create mode 100644 sdks/python/apache_beam/transforms/enrichment.py create mode 100644 sdks/python/apache_beam/transforms/enrichment_handlers/__init__.py create mode 100644 sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py create mode 100644 sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py create mode 100644 sdks/python/apache_beam/transforms/enrichment_it_test.py create mode 100644 sdks/python/apache_beam/transforms/enrichment_test.py diff --git a/CHANGES.md b/CHANGES.md index 81a519b07d7e..dbad15f3dba4 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -69,7 +69,7 @@ ## New Features / Improvements -* X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* [Enrichment Transform](https://s.apache.org/enrichment-transform) along with GCP BigTable handler added to Python SDK ([#30001](https://github.com/apache/beam/pull/30001)). ## Breaking Changes diff --git a/sdks/python/apache_beam/io/requestresponse.py b/sdks/python/apache_beam/io/requestresponse.py new file mode 100644 index 000000000000..63ec7061d3e5 --- /dev/null +++ b/sdks/python/apache_beam/io/requestresponse.py @@ -0,0 +1,413 @@ +# +# 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. +# + +"""``PTransform`` for reading from and writing to Web APIs.""" +import abc +import concurrent.futures +import contextlib +import logging +import sys +import time +from typing import Generic +from typing import Optional +from typing import TypeVar + +from google.api_core.exceptions import TooManyRequests + +import apache_beam as beam +from apache_beam.io.components.adaptive_throttler import AdaptiveThrottler +from apache_beam.metrics import Metrics +from apache_beam.ml.inference.vertex_ai_inference import MSEC_TO_SEC +from apache_beam.utils import retry + +RequestT = TypeVar('RequestT') +ResponseT = TypeVar('ResponseT') + +DEFAULT_TIMEOUT_SECS = 30 # seconds + +_LOGGER = logging.getLogger(__name__) + + +class UserCodeExecutionException(Exception): + """Base class for errors related to calling Web APIs.""" + + +class UserCodeQuotaException(UserCodeExecutionException): + """Extends ``UserCodeExecutionException`` to signal specifically that + the Web API client encountered a Quota or API overuse related error. + """ + + +class UserCodeTimeoutException(UserCodeExecutionException): + """Extends ``UserCodeExecutionException`` to signal a user code timeout.""" + + +def retry_on_exception(exception: Exception): + """retry on exceptions caused by unavailability of the remote server.""" + return isinstance( + exception, + (TooManyRequests, UserCodeTimeoutException, UserCodeQuotaException)) + + +class _MetricsCollector: + """A metrics collector that tracks RequestResponseIO related usage.""" + def __init__(self, namespace: str): + """ + Args: + namespace: Namespace for the metrics. + """ + self.requests = Metrics.counter(namespace, 'requests') + self.responses = Metrics.counter(namespace, 'responses') + self.failures = Metrics.counter(namespace, 'failures') + self.throttled_requests = Metrics.counter(namespace, 'throttled_requests') + self.throttled_secs = Metrics.counter( + namespace, 'cumulativeThrottlingSeconds') + self.timeout_requests = Metrics.counter(namespace, 'requests_timed_out') + self.call_counter = Metrics.counter(namespace, 'call_invocations') + self.setup_counter = Metrics.counter(namespace, 'setup_counter') + self.teardown_counter = Metrics.counter(namespace, 'teardown_counter') + self.backoff_counter = Metrics.counter(namespace, 'backoff_counter') + self.sleeper_counter = Metrics.counter(namespace, 'sleeper_counter') + self.should_backoff_counter = Metrics.counter( + namespace, 'should_backoff_counter') + + +class Caller(contextlib.AbstractContextManager, + abc.ABC, + Generic[RequestT, ResponseT]): + """Interface for user custom code intended for API calls. + For setup and teardown of clients when applicable, implement the + ``__enter__`` and ``__exit__`` methods respectively.""" + @abc.abstractmethod + def __call__(self, request: RequestT, *args, **kwargs) -> ResponseT: + """Calls a Web API with the ``RequestT`` and returns a + ``ResponseT``. ``RequestResponseIO`` expects implementations of the + ``__call__`` method to throw either a ``UserCodeExecutionException``, + ``UserCodeQuotaException``, or ``UserCodeTimeoutException``. + """ + pass + + def __enter__(self): + return self + + def __exit__(self, exc_type, exc_val, exc_tb): + return None + + +class ShouldBackOff(abc.ABC): + """ + ShouldBackOff provides mechanism to apply adaptive throttling. + """ + pass + + +class Repeater(abc.ABC): + """Repeater provides mechanism to repeat requests for a + configurable condition.""" + @abc.abstractmethod + def repeat( + self, + caller: Caller[RequestT, ResponseT], + request: RequestT, + timeout: float, + metrics_collector: Optional[_MetricsCollector]) -> ResponseT: + """repeat method is called from the RequestResponseIO when + a repeater is enabled. + + Args: + caller: :class:`apache_beam.io.requestresponse.Caller` object that calls + the API. + request: input request to repeat. + timeout: time to wait for the request to complete. + metrics_collector: (Optional) a + ``:class:`apache_beam.io.requestresponse._MetricsCollector``` object to + collect the metrics for RequestResponseIO. + """ + pass + + +def _execute_request( + caller: Caller[RequestT, ResponseT], + request: RequestT, + timeout: float, + metrics_collector: Optional[_MetricsCollector] = None) -> ResponseT: + with concurrent.futures.ThreadPoolExecutor() as executor: + future = executor.submit(caller, request) + try: + return future.result(timeout=timeout) + except TooManyRequests as e: + _LOGGER.info( + 'request could not be completed. got code %i from the service.', + e.code) + raise e + except concurrent.futures.TimeoutError: + if metrics_collector: + metrics_collector.timeout_requests.inc(1) + raise UserCodeTimeoutException( + f'Timeout {timeout} exceeded ' + f'while completing request: {request}') + except RuntimeError: + if metrics_collector: + metrics_collector.failures.inc(1) + raise UserCodeExecutionException('could not complete request') + + +class ExponentialBackOffRepeater(Repeater): + """Exponential BackOff Repeater uses exponential backoff retry strategy for + exceptions due to the remote service such as TooManyRequests (HTTP 429), + UserCodeTimeoutException, UserCodeQuotaException. + + It utilizes the decorator + :func:`apache_beam.utils.retry.with_exponential_backoff`. + """ + def __init__(self): + pass + + @retry.with_exponential_backoff( + num_retries=2, retry_filter=retry_on_exception) + def repeat( + self, + caller: Caller[RequestT, ResponseT], + request: RequestT, + timeout: float, + metrics_collector: Optional[_MetricsCollector] = None) -> ResponseT: + """repeat method is called from the RequestResponseIO when + a repeater is enabled. + + Args: + caller: :class:`apache_beam.io.requestresponse.Caller` object that + calls the API. + request: input request to repeat. + timeout: time to wait for the request to complete. + metrics_collector: (Optional) a + ``:class:`apache_beam.io.requestresponse._MetricsCollector``` object to + collect the metrics for RequestResponseIO. + """ + return _execute_request(caller, request, timeout, metrics_collector) + + +class NoOpsRepeater(Repeater): + """ + NoOpsRepeater executes a request just once irrespective of any exception. + """ + def repeat( + self, + caller: Caller[RequestT, ResponseT], + request: RequestT, + timeout: float, + metrics_collector: Optional[_MetricsCollector]) -> ResponseT: + return _execute_request(caller, request, timeout, metrics_collector) + + +class CacheReader(abc.ABC): + """CacheReader provides mechanism to read from the cache.""" + pass + + +class CacheWriter(abc.ABC): + """CacheWriter provides mechanism to write to the cache.""" + pass + + +class PreCallThrottler(abc.ABC): + """PreCallThrottler provides a throttle mechanism before sending request.""" + pass + + +class DefaultThrottler(PreCallThrottler): + """Default throttler that uses + :class:`apache_beam.io.components.adaptive_throttler.AdaptiveThrottler` + + Args: + window_ms (int): length of history to consider, in ms, to set throttling. + bucket_ms (int): granularity of time buckets that we store data in, in ms. + overload_ratio (float): the target ratio between requests sent and + successful requests. This is "K" in the formula in + https://landing.google.com/sre/book/chapters/handling-overload.html. + delay_secs (int): minimum number of seconds to throttle a request. + """ + def __init__( + self, + window_ms: int = 1, + bucket_ms: int = 1, + overload_ratio: float = 2, + delay_secs: int = 5): + self.throttler = AdaptiveThrottler( + window_ms=window_ms, bucket_ms=bucket_ms, overload_ratio=overload_ratio) + self.delay_secs = delay_secs + + +class RequestResponseIO(beam.PTransform[beam.PCollection[RequestT], + beam.PCollection[ResponseT]]): + """A :class:`RequestResponseIO` transform to read and write to APIs. + + Processes an input :class:`~apache_beam.pvalue.PCollection` of requests + by making a call to the API as defined in :class:`Caller`'s `__call__` + and returns a :class:`~apache_beam.pvalue.PCollection` of responses. + """ + def __init__( + self, + caller: Caller[RequestT, ResponseT], + timeout: Optional[float] = DEFAULT_TIMEOUT_SECS, + should_backoff: Optional[ShouldBackOff] = None, + repeater: Repeater = ExponentialBackOffRepeater(), + cache_reader: Optional[CacheReader] = None, + cache_writer: Optional[CacheWriter] = None, + throttler: PreCallThrottler = DefaultThrottler(), + ): + """ + Instantiates a RequestResponseIO transform. + + Args: + caller (~apache_beam.io.requestresponse.Caller): an implementation of + `Caller` object that makes call to the API. + timeout (float): timeout value in seconds to wait for response from API. + should_backoff (~apache_beam.io.requestresponse.ShouldBackOff): + (Optional) provides methods for backoff. + repeater (~apache_beam.io.requestresponse.Repeater): provides method to + repeat failed requests to API due to service errors. Defaults to + :class:`apache_beam.io.requestresponse.ExponentialBackOffRepeater` to + repeat requests with exponential backoff. + cache_reader (~apache_beam.io.requestresponse.CacheReader): (Optional) + provides methods to read external cache. + cache_writer (~apache_beam.io.requestresponse.CacheWriter): (Optional) + provides methods to write to external cache. + throttler (~apache_beam.io.requestresponse.PreCallThrottler): + provides methods to pre-throttle a request. Defaults to + :class:`apache_beam.io.requestresponse.DefaultThrottler` for + client-side adaptive throttling using + :class:`apache_beam.io.components.adaptive_throttler.AdaptiveThrottler` + """ + self._caller = caller + self._timeout = timeout + self._should_backoff = should_backoff + if repeater: + self._repeater = repeater + else: + self._repeater = NoOpsRepeater() + self._cache_reader = cache_reader + self._cache_writer = cache_writer + self._throttler = throttler + + def expand( + self, + requests: beam.PCollection[RequestT]) -> beam.PCollection[ResponseT]: + # TODO(riteshghorse): handle Cache and Throttle PTransforms when available. + if isinstance(self._throttler, DefaultThrottler): + return requests | _Call( + caller=self._caller, + timeout=self._timeout, + should_backoff=self._should_backoff, + repeater=self._repeater, + throttler=self._throttler) + else: + return requests | _Call( + caller=self._caller, + timeout=self._timeout, + should_backoff=self._should_backoff, + repeater=self._repeater) + + +class _Call(beam.PTransform[beam.PCollection[RequestT], + beam.PCollection[ResponseT]]): + """(Internal-only) PTransform that invokes a remote function on each element + of the input PCollection. + + This PTransform uses a `Caller` object to invoke the actual API calls, + and uses ``__enter__`` and ``__exit__`` to manage setup and teardown of + clients when applicable. Additionally, a timeout value is specified to + regulate the duration of each call, defaults to 30 seconds. + + Args: + caller (:class:`apache_beam.io.requestresponse.Caller`): a callable + object that invokes API call. + timeout (float): timeout value in seconds to wait for response from API. + should_backoff (~apache_beam.io.requestresponse.ShouldBackOff): + (Optional) provides methods for backoff. + repeater (~apache_beam.io.requestresponse.Repeater): (Optional) provides + methods to repeat requests to API. + throttler (~apache_beam.io.requestresponse.PreCallThrottler): + (Optional) provides methods to pre-throttle a request. + """ + def __init__( + self, + caller: Caller[RequestT, ResponseT], + timeout: Optional[float] = DEFAULT_TIMEOUT_SECS, + should_backoff: Optional[ShouldBackOff] = None, + repeater: Repeater = None, + throttler: PreCallThrottler = None, + ): + self._caller = caller + self._timeout = timeout + self._should_backoff = should_backoff + self._repeater = repeater + self._throttler = throttler + + def expand( + self, + requests: beam.PCollection[RequestT]) -> beam.PCollection[ResponseT]: + return requests | beam.ParDo( + _CallDoFn(self._caller, self._timeout, self._repeater, self._throttler)) + + +class _CallDoFn(beam.DoFn): + def setup(self): + self._caller.__enter__() + self._metrics_collector = _MetricsCollector(self._caller.__str__()) + self._metrics_collector.setup_counter.inc(1) + + def __init__( + self, + caller: Caller[RequestT, ResponseT], + timeout: float, + repeater: Repeater, + throttler: PreCallThrottler): + self._metrics_collector = None + self._caller = caller + self._timeout = timeout + self._repeater = repeater + self._throttler = throttler + + def process(self, request: RequestT, *args, **kwargs): + self._metrics_collector.requests.inc(1) + + is_throttled_request = False + if self._throttler: + while self._throttler.throttler.throttle_request(time.time() * + MSEC_TO_SEC): + _LOGGER.info( + "Delaying request for %d seconds" % self._throttler.delay_secs) + time.sleep(self._throttler.delay_secs) + self._metrics_collector.throttled_secs.inc(self._throttler.delay_secs) + is_throttled_request = True + + if is_throttled_request: + self._metrics_collector.throttled_requests.inc(1) + + try: + req_time = time.time() + response = self._repeater.repeat( + self._caller, request, self._timeout, self._metrics_collector) + self._metrics_collector.responses.inc(1) + self._throttler.throttler.successful_request(req_time * MSEC_TO_SEC) + yield response + except Exception as e: + raise e + + def teardown(self): + self._metrics_collector.teardown_counter.inc(1) + self._caller.__exit__(*sys.exc_info()) diff --git a/sdks/python/apache_beam/io/requestresponseio_it_test.py b/sdks/python/apache_beam/io/requestresponse_it_test.py similarity index 86% rename from sdks/python/apache_beam/io/requestresponseio_it_test.py rename to sdks/python/apache_beam/io/requestresponse_it_test.py index aae6b4e6ef2c..396347c58d16 100644 --- a/sdks/python/apache_beam/io/requestresponseio_it_test.py +++ b/sdks/python/apache_beam/io/requestresponse_it_test.py @@ -16,6 +16,7 @@ # import base64 import sys +import typing import unittest from dataclasses import dataclass from typing import Tuple @@ -24,13 +25,18 @@ import urllib3 import apache_beam as beam -from apache_beam.io.requestresponseio import Caller -from apache_beam.io.requestresponseio import RequestResponseIO -from apache_beam.io.requestresponseio import UserCodeExecutionException -from apache_beam.io.requestresponseio import UserCodeQuotaException from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.testing.test_pipeline import TestPipeline +# pylint: disable=ungrouped-imports +try: + from apache_beam.io.requestresponse import Caller + from apache_beam.io.requestresponse import RequestResponseIO + from apache_beam.io.requestresponse import UserCodeExecutionException + from apache_beam.io.requestresponse import UserCodeQuotaException +except ImportError: + raise unittest.SkipTest('RequestResponseIO dependencies are not installed.') + _HTTP_PATH = '/v1/echo' _PAYLOAD = base64.b64encode(bytes('payload', 'utf-8')) _HTTP_ENDPOINT_ADDRESS_FLAG = '--httpEndpointAddress' @@ -61,28 +67,27 @@ def _add_argparse_args(cls, parser) -> None: help='The ID for an allocated quota that should exceed.') -# TODO(riteshghorse,damondouglas) replace Echo(Request|Response) with proto -# generated classes from .test-infra/mock-apis: @dataclass -class EchoRequest: +class EchoResponse: id: str payload: bytes -@dataclass -class EchoResponse: +# TODO(riteshghorse,damondouglas) replace Echo(Request|Response) with proto +# generated classes from .test-infra/mock-apis: +class Request(typing.NamedTuple): id: str payload: bytes -class EchoHTTPCaller(Caller): +class EchoHTTPCaller(Caller[Request, EchoResponse]): """Implements ``Caller`` to call the ``EchoServiceGrpc``'s HTTP handler. The purpose of ``EchoHTTPCaller`` is to support integration tests. """ def __init__(self, url: str): self.url = url + _HTTP_PATH - def __call__(self, request: EchoRequest, *args, **kwargs) -> EchoResponse: + def __call__(self, request: Request, *args, **kwargs) -> EchoResponse: """Overrides ``Caller``'s call method invoking the ``EchoServiceGrpc``'s HTTP handler with an ``EchoRequest``, returning either a successful ``EchoResponse`` or throwing either a @@ -129,7 +134,7 @@ def setUpClass(cls) -> None: def setUp(self) -> None: client, options = EchoHTTPCallerTestIT._get_client_and_options() - req = EchoRequest(id=options.should_exceed_quota_id, payload=_PAYLOAD) + req = Request(id=options.should_exceed_quota_id, payload=_PAYLOAD) try: # The following is needed to exceed the API client(req) @@ -148,7 +153,7 @@ def _get_client_and_options(cls) -> Tuple[EchoHTTPCaller, EchoITOptions]: def test_given_valid_request_receives_response(self): client, options = EchoHTTPCallerTestIT._get_client_and_options() - req = EchoRequest(id=options.never_exceed_quota_id, payload=_PAYLOAD) + req = Request(id=options.never_exceed_quota_id, payload=_PAYLOAD) response: EchoResponse = client(req) @@ -158,20 +163,20 @@ def test_given_valid_request_receives_response(self): def test_given_exceeded_quota_should_raise(self): client, options = EchoHTTPCallerTestIT._get_client_and_options() - req = EchoRequest(id=options.should_exceed_quota_id, payload=_PAYLOAD) + req = Request(id=options.should_exceed_quota_id, payload=_PAYLOAD) self.assertRaises(UserCodeQuotaException, lambda: client(req)) def test_not_found_should_raise(self): client, _ = EchoHTTPCallerTestIT._get_client_and_options() - req = EchoRequest(id='i-dont-exist-quota-id', payload=_PAYLOAD) + req = Request(id='i-dont-exist-quota-id', payload=_PAYLOAD) self.assertRaisesRegex( UserCodeExecutionException, "Not Found", lambda: client(req)) def test_request_response_io(self): client, options = EchoHTTPCallerTestIT._get_client_and_options() - req = EchoRequest(id=options.never_exceed_quota_id, payload=_PAYLOAD) + req = Request(id=options.never_exceed_quota_id, payload=_PAYLOAD) with TestPipeline(is_integration_test=True) as test_pipeline: output = ( test_pipeline diff --git a/sdks/python/apache_beam/io/requestresponse_test.py b/sdks/python/apache_beam/io/requestresponse_test.py new file mode 100644 index 000000000000..6d807c2a8eb8 --- /dev/null +++ b/sdks/python/apache_beam/io/requestresponse_test.py @@ -0,0 +1,156 @@ +# +# 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 time +import unittest + +import apache_beam as beam +from apache_beam.testing.test_pipeline import TestPipeline + +# pylint: disable=ungrouped-imports +try: + from google.api_core.exceptions import TooManyRequests + from apache_beam.io.requestresponse import Caller, DefaultThrottler + from apache_beam.io.requestresponse import RequestResponseIO + from apache_beam.io.requestresponse import UserCodeExecutionException + from apache_beam.io.requestresponse import UserCodeTimeoutException + from apache_beam.io.requestresponse import retry_on_exception +except ImportError: + raise unittest.SkipTest('RequestResponseIO dependencies are not installed.') + + +class AckCaller(Caller[str, str]): + """AckCaller acknowledges the incoming request by returning a + request with ACK.""" + def __enter__(self): + pass + + def __call__(self, request: str): + return f"ACK: {request}" + + def __exit__(self, exc_type, exc_val, exc_tb): + return None + + +class CallerWithTimeout(AckCaller): + """CallerWithTimeout sleeps for 2 seconds before responding. + Used to test timeout in RequestResponseIO.""" + def __call__(self, request: str, *args, **kwargs): + time.sleep(2) + return f"ACK: {request}" + + +class CallerWithRuntimeError(AckCaller): + """CallerWithRuntimeError raises a `RuntimeError` for RequestResponseIO + to raise a UserCodeExecutionException.""" + def __call__(self, request: str, *args, **kwargs): + if not request: + raise RuntimeError("Exception expected, not an error.") + + +class CallerThatRetries(AckCaller): + def __init__(self): + self.count = -1 + + def __call__(self, request: str, *args, **kwargs): + try: + pass + except Exception as e: + raise e + finally: + self.count += 1 + raise TooManyRequests('retries = %d' % self.count) + + +class TestCaller(unittest.TestCase): + def test_valid_call(self): + caller = AckCaller() + with TestPipeline() as test_pipeline: + output = ( + test_pipeline + | beam.Create(["sample_request"]) + | RequestResponseIO(caller=caller)) + + self.assertIsNotNone(output) + + def test_call_timeout(self): + caller = CallerWithTimeout() + with self.assertRaises(UserCodeTimeoutException): + with TestPipeline() as test_pipeline: + _ = ( + test_pipeline + | beam.Create(["timeout_request"]) + | RequestResponseIO(caller=caller, timeout=1)) + + def test_call_runtime_error(self): + caller = CallerWithRuntimeError() + with self.assertRaises(UserCodeExecutionException): + with TestPipeline() as test_pipeline: + _ = ( + test_pipeline + | beam.Create([""]) + | RequestResponseIO(caller=caller)) + + def test_retry_on_exception(self): + self.assertFalse(retry_on_exception(RuntimeError())) + self.assertTrue(retry_on_exception(TooManyRequests("HTTP 429"))) + + def test_caller_backoff_retry_strategy(self): + caller = CallerThatRetries() + with self.assertRaises(TooManyRequests) as cm: + with TestPipeline() as test_pipeline: + _ = ( + test_pipeline + | beam.Create(["sample_request"]) + | RequestResponseIO(caller=caller)) + self.assertRegex(cm.exception.message, 'retries = 2') + + def test_caller_no_retry_strategy(self): + caller = CallerThatRetries() + with self.assertRaises(TooManyRequests) as cm: + with TestPipeline() as test_pipeline: + _ = ( + test_pipeline + | beam.Create(["sample_request"]) + | RequestResponseIO(caller=caller, repeater=None)) + self.assertRegex(cm.exception.message, 'retries = 0') + + def test_default_throttler(self): + caller = CallerWithTimeout() + throttler = DefaultThrottler( + window_ms=10000, bucket_ms=5000, overload_ratio=1) + # manually override the number of received requests for testing. + throttler.throttler._all_requests.add(time.time() * 1000, 100) + test_pipeline = TestPipeline() + _ = ( + test_pipeline + | beam.Create(['sample_request']) + | RequestResponseIO(caller=caller, throttler=throttler)) + result = test_pipeline.run() + result.wait_until_finish() + metrics = result.metrics().query( + beam.metrics.MetricsFilter().with_name('throttled_requests')) + self.assertEqual(metrics['counters'][0].committed, 1) + metrics = result.metrics().query( + beam.metrics.MetricsFilter().with_name('cumulativeThrottlingSeconds')) + self.assertGreater(metrics['counters'][0].committed, 0) + metrics = result.metrics().query( + beam.metrics.MetricsFilter().with_name('responses')) + self.assertEqual(metrics['counters'][0].committed, 1) + + +if __name__ == '__main__': + unittest.main() diff --git a/sdks/python/apache_beam/io/requestresponseio.py b/sdks/python/apache_beam/io/requestresponseio.py deleted file mode 100644 index 0ec586e64018..000000000000 --- a/sdks/python/apache_beam/io/requestresponseio.py +++ /dev/null @@ -1,218 +0,0 @@ -# -# 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. -# - -"""``PTransform`` for reading from and writing to Web APIs.""" -import abc -import concurrent.futures -import contextlib -import logging -import sys -from typing import Generic -from typing import Optional -from typing import TypeVar - -import apache_beam as beam -from apache_beam.pvalue import PCollection - -RequestT = TypeVar('RequestT') -ResponseT = TypeVar('ResponseT') - -DEFAULT_TIMEOUT_SECS = 30 # seconds - -_LOGGER = logging.getLogger(__name__) - - -class UserCodeExecutionException(Exception): - """Base class for errors related to calling Web APIs.""" - - -class UserCodeQuotaException(UserCodeExecutionException): - """Extends ``UserCodeExecutionException`` to signal specifically that - the Web API client encountered a Quota or API overuse related error. - """ - - -class UserCodeTimeoutException(UserCodeExecutionException): - """Extends ``UserCodeExecutionException`` to signal a user code timeout.""" - - -class Caller(contextlib.AbstractContextManager, abc.ABC): - """Interface for user custom code intended for API calls. - For setup and teardown of clients when applicable, implement the - ``__enter__`` and ``__exit__`` methods respectively.""" - @abc.abstractmethod - def __call__(self, request: RequestT, *args, **kwargs) -> ResponseT: - """Calls a Web API with the ``RequestT`` and returns a - ``ResponseT``. ``RequestResponseIO`` expects implementations of the - ``__call__`` method to throw either a ``UserCodeExecutionException``, - ``UserCodeQuotaException``, or ``UserCodeTimeoutException``. - """ - pass - - def __enter__(self): - return self - - def __exit__(self, exc_type, exc_val, exc_tb): - return None - - -class ShouldBackOff(abc.ABC): - """ - ShouldBackOff provides mechanism to apply adaptive throttling. - """ - pass - - -class Repeater(abc.ABC): - """Repeater provides mechanism to repeat requests for a - configurable condition.""" - pass - - -class CacheReader(abc.ABC): - """CacheReader provides mechanism to read from the cache.""" - pass - - -class CacheWriter(abc.ABC): - """CacheWriter provides mechanism to write to the cache.""" - pass - - -class PreCallThrottler(abc.ABC): - """PreCallThrottler provides a throttle mechanism before sending request.""" - pass - - -class RequestResponseIO(beam.PTransform[beam.PCollection[RequestT], - beam.PCollection[ResponseT]]): - """A :class:`RequestResponseIO` transform to read and write to APIs. - - Processes an input :class:`~apache_beam.pvalue.PCollection` of requests - by making a call to the API as defined in :class:`Caller`'s `__call__` - and returns a :class:`~apache_beam.pvalue.PCollection` of responses. - """ - def __init__( - self, - caller: [Caller], - timeout: Optional[float] = DEFAULT_TIMEOUT_SECS, - should_backoff: Optional[ShouldBackOff] = None, - repeater: Optional[Repeater] = None, - cache_reader: Optional[CacheReader] = None, - cache_writer: Optional[CacheWriter] = None, - throttler: Optional[PreCallThrottler] = None, - ): - """ - Instantiates a RequestResponseIO transform. - - Args: - caller (~apache_beam.io.requestresponseio.Caller): an implementation of - `Caller` object that makes call to the API. - timeout (float): timeout value in seconds to wait for response from API. - should_backoff (~apache_beam.io.requestresponseio.ShouldBackOff): - (Optional) provides methods for backoff. - repeater (~apache_beam.io.requestresponseio.Repeater): (Optional) - provides methods to repeat requests to API. - cache_reader (~apache_beam.io.requestresponseio.CacheReader): (Optional) - provides methods to read external cache. - cache_writer (~apache_beam.io.requestresponseio.CacheWriter): (Optional) - provides methods to write to external cache. - throttler (~apache_beam.io.requestresponseio.PreCallThrottler): - (Optional) provides methods to pre-throttle a request. - """ - self._caller = caller - self._timeout = timeout - self._should_backoff = should_backoff - self._repeater = repeater - self._cache_reader = cache_reader - self._cache_writer = cache_writer - self._throttler = throttler - - def expand(self, requests: PCollection[RequestT]) -> PCollection[ResponseT]: - # TODO(riteshghorse): add Cache and Throttle PTransforms. - return requests | _Call( - caller=self._caller, - timeout=self._timeout, - should_backoff=self._should_backoff, - repeater=self._repeater) - - -class _Call(beam.PTransform[beam.PCollection[RequestT], - beam.PCollection[ResponseT]]): - """(Internal-only) PTransform that invokes a remote function on each element - of the input PCollection. - - This PTransform uses a `Caller` object to invoke the actual API calls, - and uses ``__enter__`` and ``__exit__`` to manage setup and teardown of - clients when applicable. Additionally, a timeout value is specified to - regulate the duration of each call, defaults to 30 seconds. - - Args: - caller (:class:`apache_beam.io.requestresponseio.Caller`): a callable - object that invokes API call. - timeout (float): timeout value in seconds to wait for response from API. - """ - def __init__( - self, - caller: Caller, - timeout: Optional[float] = DEFAULT_TIMEOUT_SECS, - should_backoff: Optional[ShouldBackOff] = None, - repeater: Optional[Repeater] = None, - ): - """Initialize the _Call transform. - Args: - caller (:class:`apache_beam.io.requestresponseio.Caller`): a callable - object that invokes API call. - timeout (float): timeout value in seconds to wait for response from API. - should_backoff (~apache_beam.io.requestresponseio.ShouldBackOff): - (Optional) provides methods for backoff. - repeater (~apache_beam.io.requestresponseio.Repeater): (Optional) provides - methods to repeat requests to API. - """ - self._caller = caller - self._timeout = timeout - self._should_backoff = should_backoff - self._repeater = repeater - - def expand( - self, - requests: beam.PCollection[RequestT]) -> beam.PCollection[ResponseT]: - return requests | beam.ParDo(_CallDoFn(self._caller, self._timeout)) - - -class _CallDoFn(beam.DoFn, Generic[RequestT, ResponseT]): - def setup(self): - self._caller.__enter__() - - def __init__(self, caller: Caller, timeout: float): - self._caller = caller - self._timeout = timeout - - def process(self, request, *args, **kwargs): - with concurrent.futures.ThreadPoolExecutor() as executor: - future = executor.submit(self._caller, request) - try: - yield future.result(timeout=self._timeout) - except concurrent.futures.TimeoutError: - raise UserCodeTimeoutException( - f'Timeout {self._timeout} exceeded ' - f'while completing request: {request}') - except RuntimeError: - raise UserCodeExecutionException('could not complete request') - - def teardown(self): - self._caller.__exit__(*sys.exc_info()) diff --git a/sdks/python/apache_beam/io/requestresponseio_test.py b/sdks/python/apache_beam/io/requestresponseio_test.py deleted file mode 100644 index 2828a3578871..000000000000 --- a/sdks/python/apache_beam/io/requestresponseio_test.py +++ /dev/null @@ -1,88 +0,0 @@ -# -# 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 time -import unittest - -import apache_beam as beam -from apache_beam.io.requestresponseio import Caller -from apache_beam.io.requestresponseio import RequestResponseIO -from apache_beam.io.requestresponseio import UserCodeExecutionException -from apache_beam.io.requestresponseio import UserCodeTimeoutException -from apache_beam.testing.test_pipeline import TestPipeline - - -class AckCaller(Caller): - """AckCaller acknowledges the incoming request by returning a - request with ACK.""" - def __enter__(self): - pass - - def __call__(self, request: str): - return f"ACK: {request}" - - def __exit__(self, exc_type, exc_val, exc_tb): - return None - - -class CallerWithTimeout(AckCaller): - """CallerWithTimeout sleeps for 2 seconds before responding. - Used to test timeout in RequestResponseIO.""" - def __call__(self, request: str, *args, **kwargs): - time.sleep(2) - return f"ACK: {request}" - - -class CallerWithRuntimeError(AckCaller): - """CallerWithRuntimeError raises a `RuntimeError` for RequestResponseIO - to raise a UserCodeExecutionException.""" - def __call__(self, request: str, *args, **kwargs): - if not request: - raise RuntimeError("Exception expected, not an error.") - - -class TestCaller(unittest.TestCase): - def test_valid_call(self): - caller = AckCaller() - with TestPipeline() as test_pipeline: - output = ( - test_pipeline - | beam.Create(["sample_request"]) - | RequestResponseIO(caller=caller)) - - self.assertIsNotNone(output) - - def test_call_timeout(self): - caller = CallerWithTimeout() - with self.assertRaises(UserCodeTimeoutException): - with TestPipeline() as test_pipeline: - _ = ( - test_pipeline - | beam.Create(["timeout_request"]) - | RequestResponseIO(caller=caller, timeout=1)) - - def test_call_runtime_error(self): - caller = CallerWithRuntimeError() - with self.assertRaises(UserCodeExecutionException): - with TestPipeline() as test_pipeline: - _ = ( - test_pipeline - | beam.Create([""]) - | RequestResponseIO(caller=caller)) - - -if __name__ == '__main__': - unittest.main() diff --git a/sdks/python/apache_beam/transforms/enrichment.py b/sdks/python/apache_beam/transforms/enrichment.py new file mode 100644 index 000000000000..a2f961be6437 --- /dev/null +++ b/sdks/python/apache_beam/transforms/enrichment.py @@ -0,0 +1,137 @@ +# +# 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 logging +from typing import Any +from typing import Callable +from typing import Dict +from typing import Optional +from typing import TypeVar + +import apache_beam as beam +from apache_beam.io.requestresponse import DEFAULT_TIMEOUT_SECS +from apache_beam.io.requestresponse import Caller +from apache_beam.io.requestresponse import DefaultThrottler +from apache_beam.io.requestresponse import ExponentialBackOffRepeater +from apache_beam.io.requestresponse import PreCallThrottler +from apache_beam.io.requestresponse import Repeater +from apache_beam.io.requestresponse import RequestResponseIO + +__all__ = [ + "EnrichmentSourceHandler", + "Enrichment", + "cross_join", +] + +InputT = TypeVar('InputT') +OutputT = TypeVar('OutputT') + +JoinFn = Callable[[Dict[str, Any], Dict[str, Any]], beam.Row] + +_LOGGER = logging.getLogger(__name__) + + +def cross_join(left: Dict[str, Any], right: Dict[str, Any]) -> beam.Row: + """cross_join performs a cross join on two `dict` objects. + + Joins the columns of the right row onto the left row. + + Args: + left (Dict[str, Any]): input request dictionary. + right (Dict[str, Any]): response dictionary from the API. + + Returns: + `beam.Row` containing the merged columns. + """ + for k, v in right.items(): + if k not in left: + # Don't override the values in left. + left[k] = v + elif left[k] != v: + _LOGGER.warning( + '%s exists in the input row as well the row fetched ' + 'from API but have different values - %s and %s. Using the input ' + 'value (%s) for the enriched row. You can override this behavior by ' + 'passing a custom `join_fn` to Enrichment transform.' % + (k, left[k], v, left[k])) + return beam.Row(**left) + + +class EnrichmentSourceHandler(Caller[InputT, OutputT]): + """Wrapper class for :class:`apache_beam.io.requestresponse.Caller`. + + Ensure that the implementation of ``__call__`` method returns a tuple + of `beam.Row` objects. + """ + pass + + +class Enrichment(beam.PTransform[beam.PCollection[InputT], + beam.PCollection[OutputT]]): + """A :class:`apache_beam.transforms.enrichment.Enrichment` transform to + enrich elements in a PCollection. + **NOTE:** This transform and its implementation are under development and + do not provide backward compatibility guarantees. + Uses the :class:`apache_beam.transforms.enrichment.EnrichmentSourceHandler` + to enrich elements by joining the metadata from external source. + + Processes an input :class:`~apache_beam.pvalue.PCollection` of `beam.Row` by + applying a :class:`apache_beam.transforms.enrichment.EnrichmentSourceHandler` + to each element and returning the enriched + :class:`~apache_beam.pvalue.PCollection`. + + Args: + source_handler: Handles source lookup and metadata retrieval. + Implements the + :class:`apache_beam.transforms.enrichment.EnrichmentSourceHandler` + join_fn: A lambda function to join original element with lookup metadata. + Defaults to `CROSS_JOIN`. + timeout: (Optional) timeout for source requests. Defaults to 30 seconds. + repeater (~apache_beam.io.requestresponse.Repeater): provides method to + repeat failed requests to API due to service errors. Defaults to + :class:`apache_beam.io.requestresponse.ExponentialBackOffRepeater` to + repeat requests with exponential backoff. + throttler (~apache_beam.io.requestresponse.PreCallThrottler): + provides methods to pre-throttle a request. Defaults to + :class:`apache_beam.io.requestresponse.DefaultThrottler` for + client-side adaptive throttling using + :class:`apache_beam.io.components.adaptive_throttler.AdaptiveThrottler`. + """ + def __init__( + self, + source_handler: EnrichmentSourceHandler, + join_fn: JoinFn = cross_join, + timeout: Optional[float] = DEFAULT_TIMEOUT_SECS, + repeater: Repeater = ExponentialBackOffRepeater(), + throttler: PreCallThrottler = DefaultThrottler(), + ): + self._source_handler = source_handler + self._join_fn = join_fn + self._timeout = timeout + self._repeater = repeater + self._throttler = throttler + + def expand(self, + input_row: beam.PCollection[InputT]) -> beam.PCollection[OutputT]: + fetched_data = input_row | RequestResponseIO( + caller=self._source_handler, + timeout=self._timeout, + repeater=self._repeater, + throttler=self._throttler) + + # EnrichmentSourceHandler returns a tuple of (request,response). + return fetched_data | beam.Map( + lambda x: self._join_fn(x[0]._asdict(), x[1]._asdict())) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/__init__.py b/sdks/python/apache_beam/transforms/enrichment_handlers/__init__.py new file mode 100644 index 000000000000..cce3acad34a4 --- /dev/null +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/__init__.py @@ -0,0 +1,16 @@ +# +# 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. +# diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py new file mode 100644 index 000000000000..86ff2f3b8e7f --- /dev/null +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py @@ -0,0 +1,151 @@ +# +# 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 logging +from enum import Enum +from typing import Any +from typing import Dict +from typing import Optional + +from google.api_core.exceptions import NotFound +from google.cloud import bigtable +from google.cloud.bigtable import Client +from google.cloud.bigtable.row_filters import CellsColumnLimitFilter +from google.cloud.bigtable.row_filters import RowFilter + +import apache_beam as beam +from apache_beam.transforms.enrichment import EnrichmentSourceHandler + +__all__ = [ + 'EnrichWithBigTable', + 'ExceptionLevel', +] + +_LOGGER = logging.getLogger(__name__) + + +class ExceptionLevel(Enum): + """ExceptionLevel defines the exception level options to either + log a warning, or raise an exception, or do nothing when a BigTable query + returns an empty row. + + Members: + - RAISE: Raise the exception. + - WARN: Log a warning for exception without raising it. + - QUIET: Neither log nor raise the exception. + """ + RAISE = 0 + WARN = 1 + QUIET = 2 + + +class EnrichWithBigTable(EnrichmentSourceHandler[beam.Row, beam.Row]): + """EnrichWithBigTable is a handler for + :class:`apache_beam.transforms.enrichment.Enrichment` transform to interact + with GCP BigTable. + + Args: + project_id (str): GCP project-id of the BigTable cluster. + instance_id (str): GCP instance-id of the BigTable cluster. + table_id (str): GCP table-id of the BigTable. + row_key (str): unique row-key field name from the input `beam.Row` object + to use as `row_key` for BigTable querying. + row_filter: a ``:class:`google.cloud.bigtable.row_filters.RowFilter``` to + filter data read with ``read_row()``. + Defaults to `CellsColumnLimitFilter(1)`. + app_profile_id (str): App profile ID to use for BigTable. + See https://cloud.google.com/bigtable/docs/app-profiles for more details. + encoding (str): encoding type to convert the string to bytes and vice-versa + from BigTable. Default is `utf-8`. + exception_level: a `enum.Enum` value from + ``apache_beam.transforms.enrichment_handlers.bigtable.ExceptionLevel`` + to set the level when an empty row is returned from the BigTable query. + Defaults to ``ExceptionLevel.WARN``. + """ + def __init__( + self, + project_id: str, + instance_id: str, + table_id: str, + row_key: str, + row_filter: Optional[RowFilter] = CellsColumnLimitFilter(1), + app_profile_id: str = None, # type: ignore[assignment] + encoding: str = 'utf-8', + exception_level: ExceptionLevel = ExceptionLevel.WARN, + ): + self._project_id = project_id + self._instance_id = instance_id + self._table_id = table_id + self._row_key = row_key + self._row_filter = row_filter + self._app_profile_id = app_profile_id + self._encoding = encoding + self._exception_level = exception_level + + def __enter__(self): + """connect to the Google BigTable cluster.""" + self.client = Client(project=self._project_id) + self.instance = self.client.instance(self._instance_id) + self._table = bigtable.table.Table( + table_id=self._table_id, + instance=self.instance, + app_profile_id=self._app_profile_id) + + def __call__(self, request: beam.Row, *args, **kwargs): + """ + Reads a row from the GCP BigTable and returns + a `Tuple` of request and response. + + Args: + request: the input `beam.Row` to enrich. + """ + response_dict: Dict[str, Any] = {} + row_key_str: str = "" + try: + request_dict = request._asdict() + row_key_str = str(request_dict[self._row_key]) + row_key = row_key_str.encode(self._encoding) + row = self._table.read_row(row_key, filter_=self._row_filter) + if row: + for cf_id, cf_v in row.cells.items(): + response_dict[cf_id] = {} + for k, v in cf_v.items(): + response_dict[cf_id][k.decode(self._encoding)] = \ + v[0].value.decode(self._encoding) + elif self._exception_level == ExceptionLevel.WARN: + _LOGGER.warning( + 'no matching row found for row_key: %s ' + 'with row_filter: %s' % (row_key_str, self._row_filter)) + elif self._exception_level == ExceptionLevel.RAISE: + raise ValueError( + 'no matching row found for row_key: %s ' + 'with row_filter=%s' % (row_key_str, self._row_filter)) + except KeyError: + raise KeyError('row_key %s not found in input PCollection.' % row_key_str) + except NotFound: + raise NotFound( + 'GCP BigTable cluster `%s:%s:%s` not found.' % + (self._project_id, self._instance_id, self._table_id)) + except Exception as e: + raise e + + return request, beam.Row(**response_dict) + + def __exit__(self, exc_type, exc_val, exc_tb): + """Clean the instantiated BigTable client.""" + self.client = None + self.instance = None + self._table = None diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py new file mode 100644 index 000000000000..dd48c8e5ef4d --- /dev/null +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py @@ -0,0 +1,300 @@ +# +# 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 datetime +import unittest +from typing import Dict +from typing import List +from typing import NamedTuple + +import pytest + +import apache_beam as beam +from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.testing.util import BeamAssertException + +# pylint: disable=ungrouped-imports +try: + from google.api_core.exceptions import NotFound + from google.cloud.bigtable import Client + from google.cloud.bigtable.row_filters import ColumnRangeFilter + from apache_beam.transforms.enrichment import Enrichment + from apache_beam.transforms.enrichment_handlers.bigtable import EnrichWithBigTable + from apache_beam.transforms.enrichment_handlers.bigtable import ExceptionLevel +except ImportError: + raise unittest.SkipTest('GCP BigTable dependencies are not installed.') + + +class ValidateResponse(beam.DoFn): + """ValidateResponse validates if a PCollection of `beam.Row` + has the required fields.""" + def __init__( + self, + n_fields: int, + fields: List[str], + enriched_fields: Dict[str, List[str]]): + self.n_fields = n_fields + self._fields = fields + self._enriched_fields = enriched_fields + + def process(self, element: beam.Row, *args, **kwargs): + element_dict = element.as_dict() + if len(element_dict.keys()) != self.n_fields: + raise BeamAssertException( + "Expected %d fields in enriched PCollection:" % self.n_fields) + + for field in self._fields: + if field not in element_dict or element_dict[field] is None: + raise BeamAssertException(f"Expected a not None field: {field}") + + for column_family, columns in self._enriched_fields.items(): + if (len(element_dict[column_family]) != len(columns) or + not all(key in element_dict[column_family] for key in columns)): + raise BeamAssertException( + "Response from bigtable should contain a %s column_family with " + "%s keys." % (column_family, columns)) + + +class _Currency(NamedTuple): + s_id: int + id: str + + +def create_rows(table): + product_id = 'product_id' + product_name = 'product_name' + product_stock = 'product_stock' + + column_family_id = "product" + products = [ + { + 'product_id': 1, 'product_name': 'pixel 5', 'product_stock': 2 + }, + { + 'product_id': 2, 'product_name': 'pixel 6', 'product_stock': 4 + }, + { + 'product_id': 3, 'product_name': 'pixel 7', 'product_stock': 20 + }, + { + 'product_id': 4, 'product_name': 'pixel 8', 'product_stock': 10 + }, + { + 'product_id': 5, 'product_name': 'iphone 11', 'product_stock': 3 + }, + { + 'product_id': 6, 'product_name': 'iphone 12', 'product_stock': 7 + }, + { + 'product_id': 7, 'product_name': 'iphone 13', 'product_stock': 8 + }, + { + 'product_id': 8, 'product_name': 'iphone 14', 'product_stock': 3 + }, + ] + + for item in products: + row_key = str(item[product_id]).encode() + row = table.direct_row(row_key) + row.set_cell( + column_family_id, + product_id.encode(), + str(item[product_id]), + timestamp=datetime.datetime.utcnow()) + row.set_cell( + column_family_id, + product_name.encode(), + item[product_name], + timestamp=datetime.datetime.utcnow()) + row.set_cell( + column_family_id, + product_stock.encode(), + str(item[product_stock]), + timestamp=datetime.datetime.utcnow()) + row.commit() + + +@pytest.mark.it_postcommit +class TestBigTableEnrichment(unittest.TestCase): + def setUp(self): + self.project_id = 'apache-beam-testing' + self.instance_id = 'beam-test' + self.table_id = 'bigtable-enrichment-test' + self.req = [ + beam.Row(sale_id=1, customer_id=1, product_id=1, quantity=1), + beam.Row(sale_id=3, customer_id=3, product_id=2, quantity=3), + beam.Row(sale_id=5, customer_id=5, product_id=4, quantity=2), + beam.Row(sale_id=7, customer_id=7, product_id=1, quantity=1), + ] + self.row_key = 'product_id' + self.column_family_id = 'product' + client = Client(project=self.project_id) + instance = client.instance(self.instance_id) + self.table = instance.table(self.table_id) + create_rows(self.table) + + def tearDown(self) -> None: + self.table = None + + def test_enrichment_with_bigtable(self): + expected_fields = [ + 'sale_id', 'customer_id', 'product_id', 'quantity', 'product' + ] + expected_enriched_fields = { + 'product': ['product_id', 'product_name', 'product_stock'], + } + bigtable = EnrichWithBigTable( + project_id=self.project_id, + instance_id=self.instance_id, + table_id=self.table_id, + row_key=self.row_key) + with TestPipeline(is_integration_test=True) as test_pipeline: + _ = ( + test_pipeline + | "Create" >> beam.Create(self.req) + | "Enrich W/ BigTable" >> Enrichment(bigtable) + | "Validate Response" >> beam.ParDo( + ValidateResponse( + len(expected_fields), + expected_fields, + expected_enriched_fields))) + + def test_enrichment_with_bigtable_row_filter(self): + expected_fields = [ + 'sale_id', 'customer_id', 'product_id', 'quantity', 'product' + ] + expected_enriched_fields = { + 'product': ['product_name', 'product_stock'], + } + start_column = 'product_name'.encode() + column_filter = ColumnRangeFilter(self.column_family_id, start_column) + bigtable = EnrichWithBigTable( + project_id=self.project_id, + instance_id=self.instance_id, + table_id=self.table_id, + row_key=self.row_key, + row_filter=column_filter) + with TestPipeline(is_integration_test=True) as test_pipeline: + _ = ( + test_pipeline + | "Create" >> beam.Create(self.req) + | "Enrich W/ BigTable" >> Enrichment(bigtable) + | "Validate Response" >> beam.ParDo( + ValidateResponse( + len(expected_fields), + expected_fields, + expected_enriched_fields))) + + def test_enrichment_with_bigtable_no_enrichment(self): + # row_key which is product_id=11 doesn't exist, so the enriched field + # won't be added. Hence, the response is same as the request. + expected_fields = ['sale_id', 'customer_id', 'product_id', 'quantity'] + expected_enriched_fields = {} + bigtable = EnrichWithBigTable( + project_id=self.project_id, + instance_id=self.instance_id, + table_id=self.table_id, + row_key=self.row_key) + req = [beam.Row(sale_id=1, customer_id=1, product_id=11, quantity=1)] + with TestPipeline(is_integration_test=True) as test_pipeline: + _ = ( + test_pipeline + | "Create" >> beam.Create(req) + | "Enrich W/ BigTable" >> Enrichment(bigtable) + | "Validate Response" >> beam.ParDo( + ValidateResponse( + len(expected_fields), + expected_fields, + expected_enriched_fields))) + + def test_enrichment_with_bigtable_bad_row_filter(self): + # in case of a bad column filter, that is, incorrect column_family_id and + # columns, no enrichment is done. If the column_family is correct but not + # column names then all columns in that column_family are returned. + start_column = 'car_name'.encode() + column_filter = ColumnRangeFilter('car_name', start_column) + bigtable = EnrichWithBigTable( + project_id=self.project_id, + instance_id=self.instance_id, + table_id=self.table_id, + row_key=self.row_key, + row_filter=column_filter) + with self.assertRaises(NotFound): + test_pipeline = beam.Pipeline() + _ = ( + test_pipeline + | "Create" >> beam.Create(self.req) + | "Enrich W/ BigTable" >> Enrichment(bigtable)) + res = test_pipeline.run() + res.wait_until_finish() + + def test_enrichment_with_bigtable_raises_key_error(self): + """raises a `KeyError` when the row_key doesn't exist in + the input PCollection.""" + bigtable = EnrichWithBigTable( + project_id=self.project_id, + instance_id=self.instance_id, + table_id=self.table_id, + row_key='car_name') + with self.assertRaises(KeyError): + test_pipeline = beam.Pipeline() + _ = ( + test_pipeline + | "Create" >> beam.Create(self.req) + | "Enrich W/ BigTable" >> Enrichment(bigtable)) + res = test_pipeline.run() + res.wait_until_finish() + + def test_enrichment_with_bigtable_raises_not_found(self): + """raises a `NotFound` exception when the GCP BigTable Cluster + doesn't exist.""" + bigtable = EnrichWithBigTable( + project_id=self.project_id, + instance_id=self.instance_id, + table_id='invalid_table', + row_key=self.row_key) + with self.assertRaises(NotFound): + test_pipeline = beam.Pipeline() + _ = ( + test_pipeline + | "Create" >> beam.Create(self.req) + | "Enrich W/ BigTable" >> Enrichment(bigtable)) + res = test_pipeline.run() + res.wait_until_finish() + + def test_enrichment_with_bigtable_exception_level(self): + """raises a `ValueError` exception when the GCP BigTable query returns + an empty row.""" + bigtable = EnrichWithBigTable( + project_id=self.project_id, + instance_id=self.instance_id, + table_id=self.table_id, + row_key=self.row_key, + exception_level=ExceptionLevel.RAISE) + req = [beam.Row(sale_id=1, customer_id=1, product_id=11, quantity=1)] + with self.assertRaises(ValueError): + test_pipeline = beam.Pipeline() + _ = ( + test_pipeline + | "Create" >> beam.Create(req) + | "Enrich W/ BigTable" >> Enrichment(bigtable)) + res = test_pipeline.run() + res.wait_until_finish() + + +if __name__ == '__main__': + unittest.main() diff --git a/sdks/python/apache_beam/transforms/enrichment_it_test.py b/sdks/python/apache_beam/transforms/enrichment_it_test.py new file mode 100644 index 000000000000..89842cb18be0 --- /dev/null +++ b/sdks/python/apache_beam/transforms/enrichment_it_test.py @@ -0,0 +1,162 @@ +# +# 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 time +import unittest +from typing import List +from typing import NamedTuple +from typing import Tuple +from typing import Union + +import pytest +import urllib3 + +import apache_beam as beam +from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.testing.util import BeamAssertException + +# pylint: disable=ungrouped-imports +try: + from apache_beam.io.requestresponse import UserCodeExecutionException + from apache_beam.io.requestresponse import UserCodeQuotaException + from apache_beam.io.requestresponse_it_test import _PAYLOAD + from apache_beam.io.requestresponse_it_test import EchoITOptions + from apache_beam.transforms.enrichment import Enrichment + from apache_beam.transforms.enrichment import EnrichmentSourceHandler +except ImportError: + raise unittest.SkipTest('RequestResponseIO dependencies are not installed.') + + +class Request(NamedTuple): + id: str + payload: bytes + + +def _custom_join(left, right): + """custom_join returns the id and resp_payload along with a timestamp""" + right['timestamp'] = time.time() + return beam.Row(**right) + + +class SampleHTTPEnrichment(EnrichmentSourceHandler[Request, beam.Row]): + """Implements ``EnrichmentSourceHandler`` to call the ``EchoServiceGrpc``'s + HTTP handler. + """ + def __init__(self, url: str): + self.url = url + '/v1/echo' # append path to the mock API. + + def __call__(self, request: Request, *args, **kwargs): + """Overrides ``Caller``'s call method invoking the + ``EchoServiceGrpc``'s HTTP handler with an `dict`, returning + either a successful ``Tuple[dict,dict]`` or throwing either a + ``UserCodeExecutionException``, ``UserCodeTimeoutException``, + or a ``UserCodeQuotaException``. + """ + try: + resp = urllib3.request( + "POST", + self.url, + json={ + "id": request.id, "payload": str(request.payload, 'utf-8') + }, + retries=False) + + if resp.status < 300: + resp_body = resp.json() + resp_id = resp_body['id'] + payload = resp_body['payload'] + return ( + request, beam.Row(id=resp_id, resp_payload=bytes(payload, 'utf-8'))) + + if resp.status == 429: # Too Many Requests + raise UserCodeQuotaException(resp.reason) + elif resp.status != 200: + raise UserCodeExecutionException(resp.status, resp.reason, request) + + except urllib3.exceptions.HTTPError as e: + raise UserCodeExecutionException(e) + + +class ValidateFields(beam.DoFn): + """ValidateFields validates if a PCollection of `beam.Row` + has certain fields.""" + def __init__(self, n_fields: int, fields: List[str]): + self.n_fields = n_fields + self._fields = fields + + def process(self, element: beam.Row, *args, **kwargs): + element_dict = element.as_dict() + if len(element_dict.keys()) != self.n_fields: + raise BeamAssertException( + "Expected %d fields in enriched PCollection:" + " id, payload and resp_payload" % self.n_fields) + + for field in self._fields: + if field not in element_dict or element_dict[field] is None: + raise BeamAssertException(f"Expected a not None field: {field}") + + +@pytest.mark.it_postcommit +class TestEnrichment(unittest.TestCase): + options: Union[EchoITOptions, None] = None + client: Union[SampleHTTPEnrichment, None] = None + + @classmethod + def setUpClass(cls) -> None: + cls.options = EchoITOptions() + http_endpoint_address = cls.options.http_endpoint_address + if not http_endpoint_address or http_endpoint_address == '': + raise unittest.SkipTest('HTTP_ENDPOINT_ADDRESS is required.') + cls.client = SampleHTTPEnrichment(http_endpoint_address) + + @classmethod + def _get_client_and_options( + cls) -> Tuple[SampleHTTPEnrichment, EchoITOptions]: + assert cls.options is not None + assert cls.client is not None + return cls.client, cls.options + + def test_http_enrichment(self): + """Tests Enrichment Transform against the Mock-API HTTP endpoint + with the default cross join.""" + client, options = TestEnrichment._get_client_and_options() + req = Request(id=options.never_exceed_quota_id, payload=_PAYLOAD) + fields = ['id', 'payload', 'resp_payload'] + with TestPipeline(is_integration_test=True) as test_pipeline: + _ = ( + test_pipeline + | 'Create PCollection' >> beam.Create([req]) + | 'Enrichment Transform' >> Enrichment(client) + | 'Assert Fields' >> beam.ParDo( + ValidateFields(len(fields), fields=fields))) + + def test_http_enrichment_custom_join(self): + """Tests Enrichment Transform against the Mock-API HTTP endpoint + with a custom join function.""" + client, options = TestEnrichment._get_client_and_options() + req = Request(id=options.never_exceed_quota_id, payload=_PAYLOAD) + fields = ['id', 'resp_payload', 'timestamp'] + with TestPipeline(is_integration_test=True) as test_pipeline: + _ = ( + test_pipeline + | 'Create PCollection' >> beam.Create([req]) + | 'Enrichment Transform' >> Enrichment(client, join_fn=_custom_join) + | 'Assert Fields' >> beam.ParDo( + ValidateFields(len(fields), fields=fields))) + + +if __name__ == '__main__': + unittest.main() diff --git a/sdks/python/apache_beam/transforms/enrichment_test.py b/sdks/python/apache_beam/transforms/enrichment_test.py new file mode 100644 index 000000000000..23b5f1828c15 --- /dev/null +++ b/sdks/python/apache_beam/transforms/enrichment_test.py @@ -0,0 +1,41 @@ +# +# 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 logging +import unittest + +import apache_beam as beam + +# pylint: disable=ungrouped-imports +try: + from apache_beam.transforms.enrichment import cross_join +except ImportError: + raise unittest.SkipTest('RequestResponseIO dependencies are not installed.') + + +class TestEnrichmentTransform(unittest.TestCase): + def test_cross_join(self): + left = {'id': 1, 'key': 'city'} + right = {'id': 1, 'value': 'durham'} + expected = beam.Row(id=1, key='city', value='durham') + output = cross_join(left, right) + self.assertEqual(expected, output) + + +if __name__ == '__main__': + logging.getLogger().setLevel(logging.INFO) + unittest.main() From 0bfd1a47ff4d7c8b196807173a6f9a3d05bfde3c Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Fri, 19 Jan 2024 13:17:38 -0500 Subject: [PATCH 028/169] Update to libraries-bom 26.30.0 (#30028) --- .../org/apache/beam/gradle/BeamModulePlugin.groovy | 12 ++++++------ .../container/license_scripts/dep_urls_java.yaml | 2 +- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 84630f9647db..7f579a6e24c4 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -598,16 +598,16 @@ class BeamModulePlugin implements Plugin { def dbcp2_version = "2.9.0" def errorprone_version = "2.10.0" // Try to keep gax_version consistent with gax-grpc version in google_cloud_platform_libraries_bom - def gax_version = "2.38.0" + def gax_version = "2.39.0" def google_ads_version = "26.0.0" def google_clients_version = "2.0.0" def google_cloud_bigdataoss_version = "2.2.16" // Try to keep google_cloud_spanner_version consistent with google_cloud_spanner_bom in google_cloud_platform_libraries_bom - def google_cloud_spanner_version = "6.55.0" + def google_cloud_spanner_version = "6.56.0" def google_code_gson_version = "2.10.1" def google_oauth_clients_version = "1.34.1" // Try to keep grpc_version consistent with gRPC version in google_cloud_platform_libraries_bom - def grpc_version = "1.59.1" + def grpc_version = "1.61.0" def guava_version = "32.1.2-jre" def hadoop_version = "2.10.2" def hamcrest_version = "2.1" @@ -626,7 +626,7 @@ class BeamModulePlugin implements Plugin { def postgres_version = "42.2.16" def powermock_version = "2.0.9" // Try to keep protobuf_version consistent with the protobuf version in google_cloud_platform_libraries_bom - def protobuf_version = "3.25.1" + def protobuf_version = "3.25.2" def qpid_jms_client_version = "0.61.0" def quickcheck_version = "1.0" def sbe_tool_version = "1.25.1" @@ -752,9 +752,9 @@ class BeamModulePlugin implements Plugin { google_cloud_pubsub : "com.google.cloud:google-cloud-pubsub", // google_cloud_platform_libraries_bom sets version google_cloud_pubsublite : "com.google.cloud:google-cloud-pubsublite", // google_cloud_platform_libraries_bom sets version // The release notes shows the versions set by the BOM: - // https://github.com/googleapis/java-cloud-bom/releases/tag/v26.21.0 + // https://github.com/googleapis/java-cloud-bom/releases/tag/v26.30.0 // Update libraries-bom version on sdks/java/container/license_scripts/dep_urls_java.yaml - google_cloud_platform_libraries_bom : "com.google.cloud:libraries-bom:26.29.0", + google_cloud_platform_libraries_bom : "com.google.cloud:libraries-bom:26.30.0", google_cloud_spanner : "com.google.cloud:google-cloud-spanner", // google_cloud_platform_libraries_bom sets version google_cloud_spanner_test : "com.google.cloud:google-cloud-spanner:$google_cloud_spanner_version:tests", google_code_gson : "com.google.code.gson:gson:$google_code_gson_version", diff --git a/sdks/java/container/license_scripts/dep_urls_java.yaml b/sdks/java/container/license_scripts/dep_urls_java.yaml index 2ebbecc42c43..42a7c7a0a744 100644 --- a/sdks/java/container/license_scripts/dep_urls_java.yaml +++ b/sdks/java/container/license_scripts/dep_urls_java.yaml @@ -46,7 +46,7 @@ jaxen: '1.1.6': type: "3-Clause BSD" libraries-bom: - '26.29.0': + '26.30.0': license: "https://raw.githubusercontent.com/GoogleCloudPlatform/cloud-opensource-java/master/LICENSE" type: "Apache License 2.0" paranamer: From 81688eb57b36da36fb272224d1a5eda79d8f09e6 Mon Sep 17 00:00:00 2001 From: Jeff Kinard Date: Fri, 19 Jan 2024 13:21:15 -0500 Subject: [PATCH 029/169] [YAML] add time unit suffix to windowing (#30035) * [YAML] add time unit suffix to windowing Signed-off-by: Jeffrey Kinard * add ms and regex Signed-off-by: Jeffrey Kinard * add unit tests Signed-off-by: Jeffrey Kinard --------- Signed-off-by: Jeffrey Kinard --- sdks/python/apache_beam/yaml/README.md | 18 ++-- sdks/python/apache_beam/yaml/yaml_provider.py | 93 ++++++++++++++----- .../yaml/yaml_provider_unit_test.py | 68 ++++++++++++++ .../apache_beam/yaml/yaml_transform_test.py | 12 +-- .../yaml/yaml_transform_unit_test.py | 2 +- 5 files changed, 155 insertions(+), 38 deletions(-) create mode 100644 sdks/python/apache_beam/yaml/yaml_provider_unit_test.py diff --git a/sdks/python/apache_beam/yaml/README.md b/sdks/python/apache_beam/yaml/README.md index caf3a542b2b0..75fe5c11d12a 100644 --- a/sdks/python/apache_beam/yaml/README.md +++ b/sdks/python/apache_beam/yaml/README.md @@ -264,7 +264,7 @@ In order to meaningfully aggregate elements in a streaming pipeline, some kind of windowing is typically required. Beam's [windowing](https://beam.apache.org/documentation/programming-guide/#windowing) and [triggering](https://beam.apache.org/documentation/programming-guide/#triggers) -can be be declared using the same WindowInto transform available in all other +can be declared using the same WindowInto transform available in all other SDKs. ``` @@ -284,7 +284,7 @@ pipeline: - type: WindowInto windowing: type: fixed - size: 60 + size: 60s - type: SomeAggregation - type: WriteToPubSub config: @@ -308,8 +308,8 @@ pipeline: - type: SomeAggregation windowing: type: sliding - size: 60 - period: 10 + size: 60s + period: 10s - type: WriteToPubSub config: topic: anotherPubSubTopic @@ -334,7 +334,7 @@ pipeline: query: "select col1, count(*) as c from PCOLLECTION" windowing: type: sessions - gap: 60 + gap: 60s - type: WriteToPubSub config: topic: anotherPubSubTopic @@ -369,7 +369,7 @@ pipeline: right: ReadRight windowing: type: fixed - size: 60 + size: 60s ``` For a transform with no inputs, the specified windowing is instead applied to @@ -387,7 +387,7 @@ pipeline: schema: ... windowing: type: fixed - size: 60 + size: 60s - type: Sql config: query: "select col1, count(*) as c from PCOLLECTION" @@ -438,7 +438,7 @@ pipeline: schema: ... windowing: type: fixed - size: 10 + size: 10s transforms: - type: Sql @@ -451,7 +451,7 @@ pipeline: path: /path/to/output.json windowing: type: fixed - size: 300 + size: 5m ``` diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index fdfcf69ba663..99c98f891de4 100755 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -208,8 +208,9 @@ def provider_from_spec(cls, spec): urns = SafeLineLoader.strip_metadata(spec['transforms']) type = spec['type'] config = SafeLineLoader.strip_metadata(spec.get('config', {})) - extra_params = set(SafeLineLoader.strip_metadata(spec).keys()) - set( - ['transforms', 'type', 'config']) + extra_params = set(SafeLineLoader.strip_metadata(spec).keys()) - { + 'transforms', 'type', 'config' + } if extra_params: raise ValueError( f'Unexpected parameters in provider of type {type} ' @@ -260,6 +261,7 @@ def maven_jar( urns, lambda: subprocess_server.JavaJarServer.path_to_maven_jar( artifact_id=artifact_id, + group_id=group_id, version=version, repository=repository, classifier=classifier, @@ -550,7 +552,8 @@ def dicts_to_rows(o): return o -def create_builtin_provider(): +class YamlProviders: + @staticmethod def create(elements: Iterable[Any], reshuffle: Optional[bool] = True): """Creates a collection containing a specified set of elements. @@ -565,7 +568,7 @@ def create(elements: Iterable[Any], reshuffle: Optional[bool] = True): Args: elements: The set of elements that should belong to the PCollection. YAML/JSON-style mappings will be interpreted as Beam rows. - reshuffle (optional): Whether to introduce a reshuffle (to possibly + reshuffle: (optional) Whether to introduce a reshuffle (to possibly redistribute the work) if there is more than one element in the collection. Defaults to True. """ @@ -574,6 +577,7 @@ def create(elements: Iterable[Any], reshuffle: Optional[bool] = True): # Or should this be posargs, args? # pylint: disable=dangerous-default-value + @staticmethod def fully_qualified_named_transform( constructor: str, args: Optional[Iterable[Any]] = (), @@ -585,13 +589,13 @@ def fully_qualified_named_transform( via a YAML interface. Note, however, that conversion may be required if this transform does not accept or produce Beam Rows. - For example, + For example:: type: PyTransform config: - constructor: apache_beam.pkg.mod.SomeClass - args: [1, 'foo'] - kwargs: + constructor: apache_beam.pkg.mod.SomeClass + args: [1, 'foo'] + kwargs: baz: 3 can be used to access the transform @@ -651,6 +655,17 @@ class WindowInto(beam.PTransform): See [the Beam documentation on windowing](https://beam.apache.org/documentation/programming-guide/#windowing) for more details. + Sizes, offsets, periods and gaps (where applicable) must be defined using + a time unit suffix 'ms', 's', 'm', 'h' or 'd' for milliseconds, seconds, + minutes, hours or days, respectively. If a time unit is not specified, it + will default to 's'. + + For example:: + + windowing: + type: fixed + size: 30s + Note that any Yaml transform can have a [windowing parameter](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/yaml/README.md#windowing), which is applied to its inputs (if any) or outputs (if there are no inputs) @@ -665,6 +680,29 @@ def __init__(self, windowing): def expand(self, pcoll): return pcoll | self._window_transform + @staticmethod + def _parse_duration(value, name): + time_units = { + 'ms': 0.001, 's': 1, 'm': 60, 'h': 60 * 60, 'd': 60 * 60 * 12 + } + value, suffix = re.match(r'^(.*?)([^\d]*)$', str(value)).groups() + # Default to seconds if time unit suffix is not defined + if not suffix: + suffix = 's' + if not value: + raise ValueError( + f"Invalid windowing {name} value " + f"'{suffix if not value else value}'. " + f"Must provide numeric value.") + if suffix not in time_units: + raise ValueError(( + "Invalid windowing {} time unit '{}'. " + + "Valid time units are {}.").format( + name, + suffix, + ', '.join("'{}'".format(k) for k in time_units.keys()))) + return float(value) * time_units[suffix] + @staticmethod def _parse_window_spec(spec): spec = dict(spec) @@ -673,18 +711,27 @@ def _parse_window_spec(spec): if window_type == 'global': window_fn = window.GlobalWindows() elif window_type == 'fixed': - window_fn = window.FixedWindows(spec.pop('size'), spec.pop('offset', 0)) + window_fn = window.FixedWindows( + YamlProviders.WindowInto._parse_duration(spec.pop('size'), 'size'), + YamlProviders.WindowInto._parse_duration( + spec.pop('offset', 0), 'offset')) elif window_type == 'sliding': window_fn = window.SlidingWindows( - spec.pop('size'), spec.pop('period'), spec.pop('offset', 0)) + YamlProviders.WindowInto._parse_duration(spec.pop('size'), 'size'), + YamlProviders.WindowInto._parse_duration( + spec.pop('period'), 'period'), + YamlProviders.WindowInto._parse_duration( + spec.pop('offset', 0), 'offset')) elif window_type == 'sessions': - window_fn = window.FixedWindows(spec.pop('gap')) + window_fn = window.Sessions( + YamlProviders.WindowInto._parse_duration(spec.pop('gap'), 'gap')) if spec: raise ValueError(f'Unknown parameters {spec.keys()}') # TODO: Triggering, etc. return beam.WindowInto(window_fn) - def LogForTesting(): + @staticmethod + def log_for_testing(): """Logs each element of its input PCollection. The output of this transform is a copy of its input for ease of use in @@ -694,16 +741,18 @@ def log_and_return(x): logging.info(x) return x - return beam.Map(log_and_return) + return "LogForTesting" >> beam.Map(log_and_return) - return InlineProvider({ - 'Create': create, - 'LogForTesting': LogForTesting, - 'PyTransform': fully_qualified_named_transform, - 'Flatten': Flatten, - 'WindowInto': WindowInto, - }, - no_input_transforms=('Create', )) + @staticmethod + def create_builtin_provider(): + return InlineProvider({ + 'Create': YamlProviders.create, + 'LogForTesting': YamlProviders.log_for_testing, + 'PyTransform': YamlProviders.fully_qualified_named_transform, + 'Flatten': YamlProviders.Flatten, + 'WindowInto': YamlProviders.WindowInto, + }, + no_input_transforms=('Create', )) class PypiExpansionService: @@ -943,7 +992,7 @@ def standard_providers(): standard_providers = yaml.load(fin, Loader=SafeLoader) return merge_providers( - create_builtin_provider(), + YamlProviders.create_builtin_provider(), create_mapping_providers(), create_combine_providers(), io_providers(), diff --git a/sdks/python/apache_beam/yaml/yaml_provider_unit_test.py b/sdks/python/apache_beam/yaml/yaml_provider_unit_test.py new file mode 100644 index 000000000000..ec71422fd161 --- /dev/null +++ b/sdks/python/apache_beam/yaml/yaml_provider_unit_test.py @@ -0,0 +1,68 @@ +# +# 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 logging +import unittest + +from apache_beam.yaml.yaml_provider import YamlProviders + + +class WindowIntoTest(unittest.TestCase): + def __init__(self, methodName="runWindowIntoTest"): + unittest.TestCase.__init__(self, methodName) + self.parse_duration = YamlProviders.WindowInto._parse_duration + + def test_parse_duration_ms(self): + value = self.parse_duration('1000ms', 'size') + self.assertEqual(1, value) + + def test_parse_duration_sec(self): + value = self.parse_duration('1s', 'size') + self.assertEqual(1, value) + + def test_parse_duration_min(self): + value = self.parse_duration('1m', 'size') + self.assertEqual(60, value) + + def test_parse_duration_hour(self): + value = self.parse_duration('1h', 'size') + self.assertEqual(3600, value) + + def test_parse_duration_from_decimal(self): + value = self.parse_duration('1.5m', 'size') + self.assertEqual(90, value) + + def test_parse_duration_to_decimal(self): + value = self.parse_duration('1ms', 'size') + self.assertEqual(0.001, value) + + def test_parse_duration_with_missing_suffix(self): + value = self.parse_duration('1', 'size') + self.assertEqual(1, value) + + def test_parse_duration_with_invalid_suffix(self): + with self.assertRaises(ValueError): + self.parse_duration('1x', 'size') + + def test_parse_duration_with_missing_value(self): + with self.assertRaises(ValueError): + self.parse_duration('s', 'size') + + +if __name__ == '__main__': + logging.getLogger().setLevel(logging.INFO) + unittest.main() diff --git a/sdks/python/apache_beam/yaml/yaml_transform_test.py b/sdks/python/apache_beam/yaml/yaml_transform_test.py index 6414f3dbbeb6..9641df0896f5 100644 --- a/sdks/python/apache_beam/yaml/yaml_transform_test.py +++ b/sdks/python/apache_beam/yaml/yaml_transform_test.py @@ -505,7 +505,7 @@ def test_explicit_window_into(self): - type: WindowInto windowing: type: fixed - size: 4 + size: 4s - type: SumGlobally ''', providers=TEST_PROVIDERS) @@ -524,7 +524,7 @@ def test_windowing_on_input(self): - type: SumGlobally windowing: type: fixed - size: 4 + size: 4s ''', providers=TEST_PROVIDERS) assert_that(result, equal_to([6, 9])) @@ -548,7 +548,7 @@ def test_windowing_multiple_inputs(self): input: [Create1, Create2] windowing: type: fixed - size: 4 + size: 4s output: SumGlobally ''', providers=TEST_PROVIDERS) @@ -566,7 +566,7 @@ def test_windowing_on_output(self): elements: [0, 1, 2, 3, 4, 5] windowing: type: fixed - size: 4 + size: 4s - type: SumGlobally ''', providers=TEST_PROVIDERS) @@ -585,7 +585,7 @@ def test_windowing_on_outer(self): - type: SumGlobally windowing: type: fixed - size: 4 + size: 4s ''', providers=TEST_PROVIDERS) assert_that(result, equal_to([6, 9])) @@ -614,7 +614,7 @@ def test_assign_timestamps(self): v: sum windowing: type: fixed - size: 10 + size: 10s ''', providers=TEST_PROVIDERS) assert_that( diff --git a/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py b/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py index d1886ba4dcfb..00e509310d6f 100644 --- a/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py +++ b/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py @@ -209,7 +209,7 @@ def test_expand_composite_transform_with_name_input(self): inputs={'elements': elements}) self.assertRegex( str(expand_composite_transform(spec, scope)['output']), - r"PCollection.*Composite/Map.*") + r"PCollection.*Composite/LogForTesting.*") def test_expand_composite_transform_root(self): with new_pipeline() as p: From e594f57a2d710107a66fb81a5ef3b1b71c6e6799 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Fri, 19 Jan 2024 13:37:16 -0500 Subject: [PATCH 030/169] Undeprecate reshuffle (#30049) * Undeprecate reshuffle * Undeprecate reshuffle * Spotless * Deprecation correction * Wording --- .../main/java/org/apache/beam/sdk/transforms/Reshuffle.java | 6 +++--- .../beam/sdk/transforms/windowing/ReshuffleTrigger.java | 3 --- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reshuffle.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reshuffle.java index a879667ff96f..e06440ae6a9d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reshuffle.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reshuffle.java @@ -44,7 +44,9 @@ * *

A {@link PTransform} that returns a {@link PCollection} equivalent to its input but * operationally provides some of the side effects of a {@link GroupByKey}, in particular - * checkpointing, and preventing fusion of the surrounding transforms. + * redistribution of elements between workers, checkpointing, and preventing fusion of the + * surrounding transforms. Some of these side effects (e.g. checkpointing) are not portable and are + * not guaranteed to occur on all runners. * *

Performs a {@link GroupByKey} so that the data is key-partitioned. Configures the {@link * WindowingStrategy} so that no data is dropped, but doesn't affect the need for the user to @@ -52,10 +54,8 @@ * * @param The type of key being reshuffled on. * @param The type of value being reshuffled. - * @deprecated this transform's intended side effects are not portable; it will likely be removed */ @Internal -@Deprecated public class Reshuffle extends PTransform>, PCollection>> { private Reshuffle() {} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/ReshuffleTrigger.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/ReshuffleTrigger.java index 63103e6a0dff..bf84466ccdbc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/ReshuffleTrigger.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/ReshuffleTrigger.java @@ -29,11 +29,8 @@ * state. * * @param The kind of window that is being reshuffled. - * @deprecated The intended side effect of {@link Reshuffle} is not portable; it will likely be - * removed */ @Internal -@Deprecated public class ReshuffleTrigger extends Trigger { public ReshuffleTrigger() { From 195d8dcb367de4be39d6649702ea5b2e24405eef Mon Sep 17 00:00:00 2001 From: tvalentyn Date: Fri, 19 Jan 2024 10:43:02 -0800 Subject: [PATCH 031/169] Add logging to help understand errors from #28131. (#30052) --- sdks/python/apache_beam/runners/worker/sdk_worker.py | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker.py b/sdks/python/apache_beam/runners/worker/sdk_worker.py index a64fb4c967a6..04b58a6f1987 100644 --- a/sdks/python/apache_beam/runners/worker/sdk_worker.py +++ b/sdks/python/apache_beam/runners/worker/sdk_worker.py @@ -68,6 +68,7 @@ from apache_beam.runners.worker.worker_status import FnApiWorkerStatusHandler from apache_beam.utils import thread_pool_executor from apache_beam.utils.sentinel import Sentinel +from apache_beam.version import __version__ as beam_version if TYPE_CHECKING: from apache_beam.portability.api import endpoints_pb2 @@ -265,6 +266,16 @@ def get_responses(): for work_request in self._control_stub.Control(get_responses()): _LOGGER.debug('Got work %s', work_request.instruction_id) request_type = work_request.WhichOneof('request') + + if request_type is None: + raise RuntimeError( + "Cannot interpret a request received over control channel. " + "This is not expected. " + "Verify that SDK was not accidentally downgraded at runtime. " + f"SDK version: {beam_version}, " + f"instruction id: {work_request.instruction_id}, " + f"raw request: {str(work_request.SerializeToString())}") + # Name spacing the request method with 'request_'. The called method # will be like self.request_register(request) getattr(self, SdkHarness.REQUEST_METHOD_PREFIX + request_type)( From ef8d3faa7dfc63eb068951f6490f43b93a7125a6 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Fri, 19 Jan 2024 14:10:47 -0500 Subject: [PATCH 032/169] Switch back to use official jamm (#30034) --- .../apache/beam/gradle/BeamModulePlugin.groovy | 5 +---- sdks/java/container/agent/build.gradle | 4 ++++ .../beam/sdk/SdkHarnessEnvironmentTest.java | 3 +-- .../java/org/apache/beam/fn/harness/Caches.java | 4 +--- sdks/java/io/hadoop-format/build.gradle | 15 ++++----------- 5 files changed, 11 insertions(+), 20 deletions(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 7f579a6e24c4..613e4453d977 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -813,10 +813,7 @@ class BeamModulePlugin implements Plugin { jackson_datatype_jsr310 : "com.fasterxml.jackson.datatype:jackson-datatype-jsr310:$jackson_version", jackson_module_scala_2_11 : "com.fasterxml.jackson.module:jackson-module-scala_2.11:$jackson_version", jackson_module_scala_2_12 : "com.fasterxml.jackson.module:jackson-module-scala_2.12:$jackson_version", - // Swap to use the officially published version of 0.4.x once available - // instead of relying on a community published copy. See - // https://github.com/jbellis/jamm/issues/44 for additional details. - jamm : 'io.github.stephankoelle:jamm:0.4.1', + jamm : 'com.github.jbellis:jamm:0.4.0', jaxb_api : "jakarta.xml.bind:jakarta.xml.bind-api:$jaxb_api_version", jaxb_impl : "com.sun.xml.bind:jaxb-impl:$jaxb_api_version", jcl_over_slf4j : "org.slf4j:jcl-over-slf4j:$slf4j_version", diff --git a/sdks/java/container/agent/build.gradle b/sdks/java/container/agent/build.gradle index 44338915bf28..ebb27021b41f 100644 --- a/sdks/java/container/agent/build.gradle +++ b/sdks/java/container/agent/build.gradle @@ -21,12 +21,16 @@ plugins { } // the order is intended here +// Overwrite javaVersion global property if corresponding project property specified if (project.hasProperty('java11Home')) { javaVersion = "1.11" } else if (project.hasProperty('java17Home')) { javaVersion = "1.17" } else if (project.hasProperty('java21Home')) { javaVersion = "1.21" +} else if (JavaVersion.VERSION_1_8.compareTo(JavaVersion.current()) < 0) { + // Otherwise, compile the project with java11 spec + javaVersion = "1.11" } applyJavaNature( diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/SdkHarnessEnvironmentTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/SdkHarnessEnvironmentTest.java index d2735265a391..f258529bee2b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/SdkHarnessEnvironmentTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/SdkHarnessEnvironmentTest.java @@ -63,8 +63,7 @@ public class SdkHarnessEnvironmentTest { private static class JammDoFn extends DoFn { @ProcessElement public void processElement(ProcessContext c) { - MemoryMeter memoryMeter = - MemoryMeter.builder().withGuessing(Guess.ALWAYS_INSTRUMENTATION).build(); + MemoryMeter memoryMeter = MemoryMeter.builder().withGuessing(Guess.INSTRUMENTATION).build(); assertThat(memoryMeter.measureDeep(c.element()), greaterThan(0L)); c.output("measured"); } diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/Caches.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/Caches.java index 1d291c65510a..d378c7b86f28 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/Caches.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/Caches.java @@ -37,7 +37,6 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.RemovalNotification; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Weigher; import org.github.jamm.MemoryMeter; -import org.github.jamm.MemoryMeter.Guess; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -65,8 +64,7 @@ public final class Caches { */ private static final long CACHE_SIZE_CHANGE_LIMIT_BYTES = 1 << 16; - private static final MemoryMeter MEMORY_METER = - MemoryMeter.builder().withGuessing(Guess.BEST).build(); + private static final MemoryMeter MEMORY_METER = MemoryMeter.builder().build(); /** The size of a reference. */ public static final long REFERENCE_SIZE = 8; diff --git a/sdks/java/io/hadoop-format/build.gradle b/sdks/java/io/hadoop-format/build.gradle index 986ccbe3e118..fe2a04988808 100644 --- a/sdks/java/io/hadoop-format/build.gradle +++ b/sdks/java/io/hadoop-format/build.gradle @@ -44,16 +44,11 @@ def elastic_search_version = "7.12.0" configurations.testRuntimeClasspath { // Prevent a StackOverflow because of wiring LOG4J -> SLF4J -> LOG4J exclude group: "org.slf4j", module: "log4j-over-slf4j" -} -// Force use the old version of JAMM that cassandra relies on -configurations.all { - resolutionStrategy { - exclude group: "io.github.stephankoelle", module: "jamm" - } + // Force use the old version of JAMM that cassandra relies on + resolutionStrategy.force 'com.github.jbellis:jamm:0.3.0' } - dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") implementation library.java.vendored_guava_32_1_2_jre @@ -65,9 +60,6 @@ dependencies { permitUnusedDeclared library.java.hadoop_hdfs provided library.java.hadoop_hdfs_client provided library.java.hadoop_mapreduce_client_core - // Ensure that the older version of JAMM that cassandra relies on appears - // on the classpath before the one provided by :sdks:java:core shadowTest. - testImplementation "com.github.jbellis:jamm:0.3.0" testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") testImplementation project(path: ":sdks:java:extensions:avro", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:io:common", configuration: "testRuntimeMigration") @@ -94,7 +86,8 @@ dependencies { testImplementation library.java.commons_io testImplementation library.java.cassandra_driver_core testImplementation library.java.cassandra_driver_mapping - testImplementation "org.apache.cassandra:cassandra-all:3.11.8" + // TODO(yathu) bump to cassandra-5.x which uses newer jamm when released & beam runs test on Java11 + testImplementation "org.apache.cassandra:cassandra-all:3.11.10" testImplementation library.java.hadoop_common testImplementation library.java.hadoop_hdfs testImplementation library.java.hadoop_mapreduce_client_core From b4be68e2404e8f87c545f81800a7e83cd9c77df7 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> Date: Fri, 19 Jan 2024 16:25:27 -0500 Subject: [PATCH 033/169] [Python BQ] Substitute final destination schema when no input schema is specified (#30015) * substitute final destination's schema; tests * use cache and only fetch when necessary --- .../apache_beam/io/gcp/bigquery_file_loads.py | 24 +++++++ .../io/gcp/bigquery_file_loads_test.py | 62 +++++++++++++++++++ 2 files changed, 86 insertions(+) diff --git a/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py b/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py index 453cd27dfdaa..48f2ab4b36bf 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py @@ -656,6 +656,7 @@ def start_bundle(self): if not self.bq_io_metadata: self.bq_io_metadata = create_bigquery_io_metadata(self._step_name) self.pending_jobs = [] + self.schema_cache = {} def process( self, @@ -703,6 +704,29 @@ def process( create_disposition = self.create_disposition if self.temporary_tables: + # we need to create temp tables, so we need a schema. + # if there is no input schema, fetch the destination table's schema + if schema is None: + hashed_dest = bigquery_tools.get_hashable_destination(table_reference) + if hashed_dest in self.schema_cache: + schema = self.schema_cache[hashed_dest] + else: + try: + schema = bigquery_tools.table_schema_to_dict( + bigquery_tools.BigQueryWrapper().get_table( + project_id=table_reference.projectId, + dataset_id=table_reference.datasetId, + table_id=table_reference.tableId).schema) + self.schema_cache[hashed_dest] = schema + except Exception as e: + _LOGGER.warning( + "Input schema is absent and could not fetch the final " + "destination table's schema [%s]. Creating temp table [%s] " + "will likely fail: %s", + hashed_dest, + job_name, + e) + # If we are using temporary tables, then we must always create the # temporary tables, so we replace the create_disposition. create_disposition = 'CREATE_IF_NEEDED' diff --git a/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py b/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py index edd92f21e731..345c8e705007 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py @@ -903,6 +903,68 @@ def setUp(self): _LOGGER.info( "Created dataset %s in project %s", self.dataset_id, self.project) + @pytest.mark.it_postcommit + def test_batch_copy_jobs_with_no_input_schema(self): + schema_1 = "col_1:INTEGER" + schema_2 = "col_2:INTEGER" + + # create two tables with different schemas + # test to make sure this works with dynamic destinations too + self.bigquery_client.get_or_create_table( + project_id=self.project, + dataset_id=self.dataset_id, + table_id="output_table_1", + schema=bigquery_tools.get_table_schema_from_string(schema_1), + create_disposition='CREATE_IF_NEEDED', + write_disposition='WRITE_APPEND') + self.bigquery_client.get_or_create_table( + project_id=self.project, + dataset_id=self.dataset_id, + table_id="output_table_2", + schema=bigquery_tools.get_table_schema_from_string(schema_2), + create_disposition='CREATE_IF_NEEDED', + write_disposition='WRITE_APPEND') + + # reduce load job size to induce copy jobs + bqfl._DEFAULT_MAX_FILE_SIZE = 10 + bqfl._MAXIMUM_LOAD_SIZE = 20 + verifiers = [ + BigqueryFullResultMatcher( + project=self.project, + query="SELECT * FROM %s" % (self.output_table + "_1"), + data=[(i, ) for i in range(5)]), + BigqueryFullResultMatcher( + project=self.project, + query="SELECT * FROM %s" % (self.output_table + "_2"), + data=[(i, ) for i in range(5, 10)]) + ] + + output = self.output_table + + def callable_table(el: dict): + dest = output + if "col_1" in el: + dest += "_1" + elif "col_2" in el: + dest += "_2" + return dest + + args = self.test_pipeline.get_full_options_as_args( + on_success_matcher=all_of(verifiers)) + + with beam.Pipeline(argv=args) as p: + # 0...4 going to table 1 + # 5...9 going to table 2 + items = [{"col_1": i} for i in range(5)] + items.extend([{"col_2": i} for i in range(5, 10)]) + _ = ( + p | beam.Create(items) | bigquery.WriteToBigQuery( + table=callable_table, + create_disposition="CREATE_NEVER", + write_disposition="WRITE_APPEND")) + + hamcrest_assert(p, all_of(*verifiers)) + @pytest.mark.it_postcommit def test_multiple_destinations_transform(self): output_table_1 = '%s%s' % (self.output_table, 1) From 23e83092345b21d5fa574fe7ea67f052861faec8 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Fri, 19 Jan 2024 16:45:47 -0500 Subject: [PATCH 034/169] Fix buildx always been true (#30051) --- .../groovy/org/apache/beam/gradle/BeamDockerPlugin.groovy | 3 +++ .../groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 6 +++++- .../job-server-container/flink_job_server_container.gradle | 5 ++--- .../job-server/container/spark_job_server_container.gradle | 5 ++--- sdks/go/container/build.gradle | 5 ++--- sdks/java/container/common.gradle | 5 ++--- sdks/java/expansion-service/container/build.gradle | 5 ++--- .../transform-service/controller-container/build.gradle | 5 ++--- sdks/python/container/common.gradle | 5 ++--- sdks/python/expansion-service-container/build.gradle | 5 ++--- sdks/typescript/container/build.gradle | 5 ++--- 11 files changed, 26 insertions(+), 28 deletions(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamDockerPlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamDockerPlugin.groovy index 2aa37d891271..cd46c1270f83 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamDockerPlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamDockerPlugin.groovy @@ -247,6 +247,9 @@ class BeamDockerPlugin implements Plugin { } } else { buildCommandLine.add 'build' + // TARGETOS and TARGETARCH args not present through `docker build`, add here + ext.buildArgs.put('TARGETOS', 'linux') + ext.buildArgs.put('TARGETARCH', ext.project.nativeArchitecture()) } if (ext.noCache) { buildCommandLine.add '--no-cache' diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 613e4453d977..41bcc70fe532 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -558,7 +558,7 @@ class BeamModulePlugin implements Plugin { project.ext.containerArchitectures = { if (isRelease(project)) { // Ensure we always publish the expected containers. - return ["amd64", "arm64"]; + return ["amd64", "arm64"] } else if (project.rootProject.findProperty("container-architecture-list") != null) { def containerArchitectures = project.rootProject.findProperty("container-architecture-list").split(',') if (containerArchitectures.size() > 1 && !project.rootProject.hasProperty("push-containers")) { @@ -574,6 +574,10 @@ class BeamModulePlugin implements Plugin { return project.containerArchitectures().collect { arch -> "linux/" + arch } } + project.ext.useBuildx = { + return project.containerArchitectures() != [project.nativeArchitecture()] + } + /** ***********************************************************************************************/ // Define and export a map dependencies shared across multiple sub-projects. // diff --git a/runners/flink/job-server-container/flink_job_server_container.gradle b/runners/flink/job-server-container/flink_job_server_container.gradle index f970698e2180..3f30a1aac1fb 100644 --- a/runners/flink/job-server-container/flink_job_server_container.gradle +++ b/runners/flink/job-server-container/flink_job_server_container.gradle @@ -52,7 +52,6 @@ task copyDockerfileDependencies(type: Copy) { into "build" } -def useBuildx = project.containerPlatforms() != [project.nativeArchitecture()] def pushContainers = project.rootProject.hasProperty(["isRelease"]) || project.rootProject.hasProperty("push-containers") docker { @@ -66,9 +65,9 @@ docker { // tags used by dockerTag task tags containerImageTags() files "./build/" - buildx useBuildx + buildx project.useBuildx() platform(*project.containerPlatforms()) - load useBuildx && !pushContainers + load project.useBuildx() && !pushContainers push pushContainers } diff --git a/runners/spark/job-server/container/spark_job_server_container.gradle b/runners/spark/job-server/container/spark_job_server_container.gradle index 3f3cd74bc10f..b73b651e2524 100644 --- a/runners/spark/job-server/container/spark_job_server_container.gradle +++ b/runners/spark/job-server/container/spark_job_server_container.gradle @@ -51,7 +51,6 @@ task copyDockerfileDependencies(type: Copy) { into "build" } -def useBuildx = project.containerPlatforms() != [project.nativeArchitecture()] def pushContainers = project.rootProject.hasProperty(["isRelease"]) || project.rootProject.hasProperty("push-containers") docker { @@ -64,9 +63,9 @@ docker { // tags used by dockerTag task tags containerImageTags() files "./build/" - buildx useBuildx + buildx project.useBuildx() platform(*project.containerPlatforms()) - load useBuildx && !pushContainers + load project.useBuildx() && !pushContainers push pushContainers } diff --git a/sdks/go/container/build.gradle b/sdks/go/container/build.gradle index 8429de133ee7..c3e98d23a422 100644 --- a/sdks/go/container/build.gradle +++ b/sdks/go/container/build.gradle @@ -27,7 +27,6 @@ goBuild { outputLocation = './build/target/${GOOS}_${GOARCH}/boot' } -def useBuildx = project.containerPlatforms() != [project.nativeArchitecture()] def pushContainers = project.rootProject.hasProperty(["isRelease"]) || project.rootProject.hasProperty("push-containers") docker { @@ -41,9 +40,9 @@ docker { files "./build/" buildArgs(['pull_licenses': project.rootProject.hasProperty(["docker-pull-licenses"]) || project.rootProject.hasProperty(["isRelease"])]) - buildx useBuildx + buildx project.useBuildx() platform(*project.containerPlatforms()) - load useBuildx && !pushContainers + load project.useBuildx() && !pushContainers push pushContainers } dockerPrepare.dependsOn tasks.named("goBuild") diff --git a/sdks/java/container/common.gradle b/sdks/java/container/common.gradle index c242f714543e..d8c066993fea 100644 --- a/sdks/java/container/common.gradle +++ b/sdks/java/container/common.gradle @@ -121,7 +121,6 @@ task validateJavaHome { } } -def useBuildx = project.containerPlatforms() != [project.nativeArchitecture()] def pushContainers = project.rootProject.hasProperty(["isRelease"]) || project.rootProject.hasProperty("push-containers") docker { @@ -142,9 +141,9 @@ docker { 'base_image': javaBaseImage, 'java_version': imageJavaVersion, ]) - buildx useBuildx + buildx project.useBuildx() platform(*project.containerPlatforms()) - load useBuildx && !pushContainers + load project.useBuildx() && !pushContainers push pushContainers } diff --git a/sdks/java/expansion-service/container/build.gradle b/sdks/java/expansion-service/container/build.gradle index d5488df3d146..cf81d462f08b 100644 --- a/sdks/java/expansion-service/container/build.gradle +++ b/sdks/java/expansion-service/container/build.gradle @@ -60,7 +60,6 @@ task copyConfigFile(type: Copy) { into "build/target" } -def useBuildx = project.containerPlatforms() != [project.nativeArchitecture()] def pushContainers = project.rootProject.hasProperty(["isRelease"]) || project.rootProject.hasProperty("push-containers") docker { @@ -74,9 +73,9 @@ docker { // tags used by dockerTag task tags containerImageTags() files "./build" - buildx useBuildx + buildx project.useBuildx() platform(*project.containerPlatforms()) - load useBuildx && !pushContainers + load project.useBuildx() && !pushContainers push pushContainers } diff --git a/sdks/java/transform-service/controller-container/build.gradle b/sdks/java/transform-service/controller-container/build.gradle index db5790f7ca66..bf23380c7b53 100644 --- a/sdks/java/transform-service/controller-container/build.gradle +++ b/sdks/java/transform-service/controller-container/build.gradle @@ -49,7 +49,6 @@ task copyConfigFile(type: Copy) { into "build/target" } -def useBuildx = project.containerPlatforms() != [project.nativeArchitecture()] def pushContainers = project.rootProject.hasProperty(["isRelease"]) || project.rootProject.hasProperty("push-containers") docker { @@ -63,9 +62,9 @@ docker { // tags used by dockerTag task tags containerImageTags() files "./build" - buildx useBuildx + buildx project.useBuildx() platform(*project.containerPlatforms()) - load useBuildx && !pushContainers + load project.useBuildx() && !pushContainers push pushContainers } diff --git a/sdks/python/container/common.gradle b/sdks/python/container/common.gradle index 4996dae52db5..0175778a6301 100644 --- a/sdks/python/container/common.gradle +++ b/sdks/python/container/common.gradle @@ -70,7 +70,6 @@ def copyLauncherDependencies = tasks.register("copyLauncherDependencies", Copy) } } -def useBuildx = project.containerPlatforms() != [project.nativeArchitecture()] def pushContainers = project.rootProject.hasProperty(["isRelease"]) || project.rootProject.hasProperty("push-containers") docker { @@ -87,9 +86,9 @@ docker { buildArgs(['py_version': "${project.ext.pythonVersion}", 'pull_licenses': project.rootProject.hasProperty(["docker-pull-licenses"]) || project.rootProject.hasProperty(["isRelease"])]) - buildx useBuildx + buildx project.useBuildx() platform(*project.containerPlatforms()) - load useBuildx && !pushContainers + load project.useBuildx() && !pushContainers push pushContainers } diff --git a/sdks/python/expansion-service-container/build.gradle b/sdks/python/expansion-service-container/build.gradle index ea11f400d380..3edcaee35b4a 100644 --- a/sdks/python/expansion-service-container/build.gradle +++ b/sdks/python/expansion-service-container/build.gradle @@ -57,7 +57,6 @@ def copyLicenseScripts = tasks.register("copyLicenseScripts", Copy){ into "build/target/license_scripts" } -def useBuildx = project.containerPlatforms() != [project.nativeArchitecture()] def pushContainers = project.rootProject.hasProperty(["isRelease"]) || project.rootProject.hasProperty("push-containers") docker { @@ -71,9 +70,9 @@ docker { // tags used by dockerTag task tags containerImageTags() files "./build" - buildx useBuildx + buildx project.useBuildx() platform(*project.containerPlatforms()) - load useBuildx && !pushContainers + load project.useBuildx() && !pushContainers push pushContainers } diff --git a/sdks/typescript/container/build.gradle b/sdks/typescript/container/build.gradle index 248415789834..c6b2093fbc8a 100644 --- a/sdks/typescript/container/build.gradle +++ b/sdks/typescript/container/build.gradle @@ -43,7 +43,6 @@ def copyDockerfileDependencies = tasks.register("copyDockerfileDependencies", Co } } -def useBuildx = project.containerPlatforms() != [project.nativeArchitecture()] def pushContainers = project.rootProject.hasProperty(["isRelease"]) || project.rootProject.hasProperty("push-containers") docker { @@ -57,9 +56,9 @@ docker { // tags used by dockerTag task tags containerImageTags() files "./build" - buildx useBuildx + buildx project.useBuildx() platform(*project.containerPlatforms()) - load useBuildx && !pushContainers + load project.useBuildx() && !pushContainers push pushContainers } From 774082f57a923ae344c63b46324d3e21a038e4ab Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Fri, 19 Jan 2024 16:47:26 -0500 Subject: [PATCH 035/169] Exclude newly added failing validation runner tests (#30030) * Exclude newly added failing validation runner tests * remove temp files --- .github/trigger_files/beam_PostCommit_Go.json | 0 .github/trigger_files/beam_PostCommit_Go_VR_Flink.json | 0 .github/trigger_files/beam_PostCommit_Go_VR_Samza.json | 0 .github/trigger_files/beam_PostCommit_Go_VR_Spark.json | 0 runners/spark/job-server/spark_job_server.gradle | 2 ++ runners/spark/spark_runner.gradle | 2 ++ 6 files changed, 4 insertions(+) delete mode 100644 .github/trigger_files/beam_PostCommit_Go.json delete mode 100644 .github/trigger_files/beam_PostCommit_Go_VR_Flink.json delete mode 100644 .github/trigger_files/beam_PostCommit_Go_VR_Samza.json delete mode 100644 .github/trigger_files/beam_PostCommit_Go_VR_Spark.json diff --git a/.github/trigger_files/beam_PostCommit_Go.json b/.github/trigger_files/beam_PostCommit_Go.json deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/.github/trigger_files/beam_PostCommit_Go_VR_Flink.json b/.github/trigger_files/beam_PostCommit_Go_VR_Flink.json deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/.github/trigger_files/beam_PostCommit_Go_VR_Samza.json b/.github/trigger_files/beam_PostCommit_Go_VR_Samza.json deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/.github/trigger_files/beam_PostCommit_Go_VR_Spark.json b/.github/trigger_files/beam_PostCommit_Go_VR_Spark.json deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/runners/spark/job-server/spark_job_server.gradle b/runners/spark/job-server/spark_job_server.gradle index 89c5d897b082..8945ad3498c7 100644 --- a/runners/spark/job-server/spark_job_server.gradle +++ b/runners/spark/job-server/spark_job_server.gradle @@ -87,6 +87,8 @@ runShadow { def sickbayTests = [ // TODO(BEAM-13498) 'org.apache.beam.sdk.transforms.ParDoTest$TimestampTests.testProcessElementSkew', + // TODO(https://github.com/apache/beam/issues/29973) + 'org.apache.beam.sdk.transforms.ReshuffleTest.testReshufflePreservesMetadata', ] def portableValidatesRunnerTask(String name, boolean streaming, boolean docker, ArrayList sickbayTests) { diff --git a/runners/spark/spark_runner.gradle b/runners/spark/spark_runner.gradle index e3b26d4d1cc5..c336b9966b3b 100644 --- a/runners/spark/spark_runner.gradle +++ b/runners/spark/spark_runner.gradle @@ -424,6 +424,8 @@ tasks.register("validatesStructuredStreamingRunnerBatch", Test) { excludeTestsMatching 'org.apache.beam.sdk.transforms.FlattenTest.testFlattenMultipleCoders' // SDF excludeTestsMatching 'org.apache.beam.sdk.transforms.SplittableDoFnTest.testLifecycleMethodsBounded' + // https://github.com/apache/beam/issues/29972 + excludeTestsMatching 'org.apache.beam.sdk.transforms.CombineTest$CombineWithContextTests.testHotKeyCombineWithSideInputs' } } From d5a7fc92cdfcba199f817cd0bd0793b16c5e105e Mon Sep 17 00:00:00 2001 From: Hai Joey Tran Date: Fri, 19 Jan 2024 17:28:50 -0500 Subject: [PATCH 036/169] Centralize exec stage urn string (#29947) * Move exec stage urn string * Minor ordering update. --------- Co-authored-by: Robert Bradshaw --- sdks/python/apache_beam/portability/common_urns.py | 1 + .../runners/portability/fn_api_runner/translations.py | 2 +- sdks/python/apache_beam/runners/trivial_runner.py | 2 +- 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/portability/common_urns.py b/sdks/python/apache_beam/portability/common_urns.py index e7b086c5a649..4effc91c3d40 100644 --- a/sdks/python/apache_beam/portability/common_urns.py +++ b/sdks/python/apache_beam/portability/common_urns.py @@ -53,6 +53,7 @@ combine_components = StandardPTransforms.CombineComponents sdf_components = StandardPTransforms.SplittableParDoComponents group_into_batches_components = StandardPTransforms.GroupIntoBatchesComponents +executable_stage = "beam:runner:executable_stage:v1" user_state = StandardUserStateTypes.Enum side_inputs = StandardSideInputTypes.Enum diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py b/sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py index cc1494fc7ae2..864f57807da1 100644 --- a/sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py +++ b/sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py @@ -362,7 +362,7 @@ def executable_stage_transform( return beam_runner_api_pb2.PTransform( unique_name=unique_name(None, self.name), spec=beam_runner_api_pb2.FunctionSpec( - urn='beam:runner:executable_stage:v1', + urn=common_urns.executable_stage, payload=exec_payload.SerializeToString()), inputs=named_inputs, outputs={ diff --git a/sdks/python/apache_beam/runners/trivial_runner.py b/sdks/python/apache_beam/runners/trivial_runner.py index 219c84f06624..af8f4f92c4e3 100644 --- a/sdks/python/apache_beam/runners/trivial_runner.py +++ b/sdks/python/apache_beam/runners/trivial_runner.py @@ -114,7 +114,7 @@ def execute_transform(self, transform_id, execution_state): for pc in transform_proto.inputs.values() ], [])) - elif transform_proto.spec.urn == 'beam:runner:executable_stage:v1': + elif transform_proto.spec.urn == common_urns.executable_stage: # This is a collection of user DoFns. self.execute_executable_stage(transform_proto, execution_state) From 175346e350752250b637860865fcb98aedfa0785 Mon Sep 17 00:00:00 2001 From: Chamikara Jayalath Date: Fri, 19 Jan 2024 18:12:03 -0800 Subject: [PATCH 037/169] Automatically enable Dataflow Runner v2 for pipeliens that use the transform upgrade feature --- .../construction/PipelineTranslation.java | 10 +++++++++- .../core/construction/TransformUpgrader.java | 20 ++++++++++++++++--- .../construction/TransformUpgraderTest.java | 4 ++++ .../beam/runners/dataflow/DataflowRunner.java | 19 +++++++++++++++--- 4 files changed, 46 insertions(+), 7 deletions(-) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineTranslation.java index e39a38a74c2c..4433b4b0475d 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineTranslation.java @@ -56,6 +56,14 @@ public static RunnerApi.Pipeline toProto( final Pipeline pipeline, final SdkComponents components, boolean useDeprecatedViewTransforms) { + return toProto(pipeline, components, useDeprecatedViewTransforms, true); + } + + public static RunnerApi.Pipeline toProto( + final Pipeline pipeline, + final SdkComponents components, + boolean useDeprecatedViewTransforms, + boolean upgradeTransforms) { final List rootIds = new ArrayList<>(); pipeline.traverseTopologically( new PipelineVisitor.Defaults() { @@ -106,7 +114,7 @@ public void visitPrimitiveTransform(Node node) { ExternalTranslationOptions externalTranslationOptions = pipeline.getOptions().as(ExternalTranslationOptions.class); List urnsToOverride = externalTranslationOptions.getTransformsToOverride(); - if (urnsToOverride.size() > 0) { + if (urnsToOverride.size() > 0 && upgradeTransforms) { try (TransformUpgrader upgrader = TransformUpgrader.of()) { res = upgrader.upgradeTransformsViaTransformService( diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java index b142ab4af1c9..4f1a02165d23 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java @@ -62,6 +62,8 @@ public class TransformUpgrader implements AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(TransformUpgrader.class); private static final String UPGRADE_NAMESPACE = "transform:upgrade:"; + @VisibleForTesting static final String UPGRADE_KEY = "upgraded_to_version"; + private ExpansionServiceClientFactory clientFactory; private TransformUpgrader() { @@ -148,7 +150,8 @@ public RunnerApi.Pipeline upgradeTransformsViaTransformService( for (String transformId : transformsToOverride) { pipeline = - updateTransformViaTransformService(pipeline, transformId, expansionServiceEndpoint); + updateTransformViaTransformService( + pipeline, transformId, expansionServiceEndpoint, options); } if (service != null) { @@ -165,14 +168,14 @@ public RunnerApi.Pipeline upgradeTransformsViaTransformService( RunnerApi.Pipeline updateTransformViaTransformService( RunnerApi.Pipeline runnerAPIpipeline, String transformId, - Endpoints.ApiServiceDescriptor transformServiceEndpoint) + Endpoints.ApiServiceDescriptor transformServiceEndpoint, + ExternalTranslationOptions options) throws IOException { RunnerApi.PTransform transformToUpgrade = runnerAPIpipeline.getComponents().getTransformsMap().get(transformId); if (transformToUpgrade == null) { throw new IllegalArgumentException("Could not find a transform with the ID " + transformId); } - ByteString configRowBytes = transformToUpgrade.getAnnotationsOrThrow(PTransformTranslation.CONFIG_ROW_KEY); ByteString configRowSchemaBytes = @@ -236,6 +239,17 @@ RunnerApi.Pipeline updateTransformViaTransformService( newEnvironmentsWithDependencies, transformServiceEndpoint)) .build(); RunnerApi.PTransform expandedTransform = response.getTransform(); + + // Adds an annotation that denotes the Beam version the transform was upgraded to. + RunnerApi.PTransform.Builder expandedTransformBuilder = expandedTransform.toBuilder(); + String transformServiceVersion = options.getTransformServiceBeamVersion(); + if (transformServiceVersion == null || transformServiceVersion.isEmpty()) { + transformServiceVersion = "unknown"; + } + expandedTransformBuilder.putAnnotations( + UPGRADE_KEY, ByteString.copyFromUtf8(transformServiceVersion)); + expandedTransform = expandedTransformBuilder.build(); + List expandedRequirements = response.getRequirementsList(); RunnerApi.Components.Builder newComponentsBuilder = expandedComponents.toBuilder(); diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformUpgraderTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformUpgraderTest.java index 6620e780bc16..e14fa556dd90 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformUpgraderTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformUpgraderTest.java @@ -18,6 +18,7 @@ package org.apache.beam.runners.core.construction; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; import com.google.auto.service.AutoService; import java.io.ByteArrayInputStream; @@ -284,6 +285,9 @@ public void testTransformUpgrade() throws Exception { .get("TransformUpgraderTest-TestTransform"); validateTestParam(upgradedTransform, 4); + + // Confirm that the upgraded transform includes the upgrade annotation. + assertTrue(upgradedTransform.getAnnotationsMap().containsKey(TransformUpgrader.UPGRADE_KEY)); } @Test diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index ea7a7d9e7314..725903aaaa27 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -64,6 +64,7 @@ import org.apache.beam.runners.core.construction.EmptyFlattenAsCreateFactory; import org.apache.beam.runners.core.construction.Environments; import org.apache.beam.runners.core.construction.External; +import org.apache.beam.runners.core.construction.ExternalTranslationOptions; import org.apache.beam.runners.core.construction.PTransformMatchers; import org.apache.beam.runners.core.construction.PTransformReplacements; import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; @@ -1101,14 +1102,25 @@ public void visitPrimitiveTransform(Node node) { return visitor.isMultiLanguage; } + private static boolean includesTransformUpgrades(Pipeline pipeline) { + return (pipeline + .getOptions() + .as(ExternalTranslationOptions.class) + .getTransformsToOverride() + .size() + > 0); + } + @Override public DataflowPipelineJob run(Pipeline pipeline) { - if (DataflowRunner.isMultiLanguagePipeline(pipeline)) { + // Multi-language pipelines and pipelines that include upgrades should automatically be upgraded + // to Runner v2. + if (DataflowRunner.isMultiLanguagePipeline(pipeline) || includesTransformUpgrades(pipeline)) { List experiments = firstNonNull(options.getExperiments(), Collections.emptyList()); if (!experiments.contains("use_runner_v2")) { LOG.info( "Automatically enabling Dataflow Runner v2 since the pipeline used cross-language" - + " transforms"); + + " transforms or pipeline needed a transform upgrade."); options.setExperiments( ImmutableList.builder().addAll(experiments).add("use_runner_v2").build()); } @@ -1217,8 +1229,9 @@ public DataflowPipelineJob run(Pipeline pipeline) { .addAllDependencies(getDefaultArtifacts()) .addAllCapabilities(Environments.getJavaCapabilities()) .build()); + // No need to perform transform upgrading for the Runner v1 proto. RunnerApi.Pipeline dataflowV1PipelineProto = - PipelineTranslation.toProto(pipeline, dataflowV1Components, true); + PipelineTranslation.toProto(pipeline, dataflowV1Components, true, false); if (LOG.isDebugEnabled()) { LOG.debug( From e4b81809c133bd7e97dd2bc218c25291d2630720 Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Sat, 20 Jan 2024 12:28:48 -0500 Subject: [PATCH 038/169] fix config translator. If config is set on BigtableIO directly, it should override configs in BigtableOptions (#30039) --- .../bigtable/BigtableConfigTranslator.java | 177 ++++++++++++------ .../gcp/bigtable/BigtableServiceFactory.java | 16 +- .../BigtableConfigTranslatorTest.java | 169 ++++++++++++++--- .../sdk/io/gcp/bigtable/BigtableWriteIT.java | 1 + 4 files changed, 276 insertions(+), 87 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java index 598672797b9c..02273f8334dc 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java @@ -48,6 +48,7 @@ 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.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.Nullable; import org.threeten.bp.Duration; /** @@ -67,21 +68,23 @@ class BigtableConfigTranslator { static BigtableDataSettings translateReadToVeneerSettings( @NonNull BigtableConfig config, @NonNull BigtableReadOptions options, + @Nullable BigtableReadOptions optionsFromBigtableOptions, @NonNull PipelineOptions pipelineOptions) throws IOException { BigtableDataSettings.Builder settings = buildBigtableDataSettings(config, pipelineOptions); - return configureReadSettings(settings, options); + return configureReadSettings(settings, options, optionsFromBigtableOptions); } /** Translate BigtableConfig and BigtableWriteOptions to Veneer settings. */ static BigtableDataSettings translateWriteToVeneerSettings( @NonNull BigtableConfig config, @NonNull BigtableWriteOptions options, + @Nullable BigtableWriteOptions optionsFromBigtableOptions, @NonNull PipelineOptions pipelineOptions) throws IOException { BigtableDataSettings.Builder settings = buildBigtableDataSettings(config, pipelineOptions); - return configureWriteSettings(settings, options); + return configureWriteSettings(settings, options, optionsFromBigtableOptions); } /** Translate BigtableConfig and BigtableWriteOptions to Veneer settings. */ @@ -175,48 +178,93 @@ private static void configureChannelPool( } private static BigtableDataSettings configureWriteSettings( - BigtableDataSettings.Builder settings, BigtableWriteOptions writeOptions) { + BigtableDataSettings.Builder settings, + BigtableWriteOptions writeOptions, + BigtableWriteOptions fromBigtableOptions) { BigtableBatchingCallSettings.Builder callSettings = settings.stubSettings().bulkMutateRowsSettings(); RetrySettings.Builder retrySettings = callSettings.getRetrySettings().toBuilder(); BatchingSettings.Builder batchingSettings = callSettings.getBatchingSettings().toBuilder(); + FlowControlSettings.Builder flowControlSettings = + callSettings.getBatchingSettings().getFlowControlSettings().toBuilder(); + // Settings set directly on WriteOptions overrides settings in BigtableOptions // The default attempt timeout for version <= 2.46.0 is 6 minutes. Reset the timeout to align // with the old behavior. - Duration attemptTimeout = Duration.ofMinutes(6); - - if (writeOptions.getAttemptTimeout() != null) { - attemptTimeout = Duration.ofMillis(writeOptions.getAttemptTimeout().getMillis()); - } - retrySettings.setInitialRpcTimeout(attemptTimeout).setMaxRpcTimeout(attemptTimeout); - // Expand the operation timeout if it's shorter - retrySettings.setTotalTimeout( - Duration.ofMillis( - Math.max(retrySettings.getTotalTimeout().toMillis(), attemptTimeout.toMillis()))); - - if (writeOptions.getOperationTimeout() != null) { - retrySettings.setTotalTimeout( - Duration.ofMillis(writeOptions.getOperationTimeout().getMillis())); - } - - if (writeOptions.getMaxElementsPerBatch() != null) { - batchingSettings.setElementCountThreshold(writeOptions.getMaxElementsPerBatch()); - } - - if (writeOptions.getMaxBytesPerBatch() != null) { - batchingSettings.setRequestByteThreshold(writeOptions.getMaxBytesPerBatch()); + long initialRpcTimeout = + writeOptions.getAttemptTimeout() != null + ? writeOptions.getAttemptTimeout().getMillis() + : (fromBigtableOptions != null && fromBigtableOptions.getAttemptTimeout() != null + ? fromBigtableOptions.getAttemptTimeout().getMillis() + : Duration.ofMinutes(6).toMillis()); + + long totalTimeout = + writeOptions.getOperationTimeout() != null + ? writeOptions.getOperationTimeout().getMillis() + : (fromBigtableOptions != null && fromBigtableOptions.getOperationTimeout() != null + ? fromBigtableOptions.getOperationTimeout().getMillis() + : retrySettings.getTotalTimeout().toMillis()); + + retrySettings + .setInitialRpcTimeout(Duration.ofMillis(initialRpcTimeout)) + .setMaxRpcTimeout(Duration.ofMillis(initialRpcTimeout)) + .setRpcTimeoutMultiplier(1) + .setTotalTimeout(Duration.ofMillis(Math.max(initialRpcTimeout, totalTimeout))); + + long maxElement = + writeOptions.getMaxElementsPerBatch() != null + ? writeOptions.getMaxElementsPerBatch() + : (fromBigtableOptions != null && fromBigtableOptions.getMaxElementsPerBatch() != null + ? fromBigtableOptions.getMaxElementsPerBatch() + : callSettings.getBatchingSettings().getElementCountThreshold()); + + long maxBytes = + writeOptions.getMaxBytesPerBatch() != null + ? writeOptions.getMaxBytesPerBatch() + : (fromBigtableOptions != null && fromBigtableOptions.getMaxBytesPerBatch() != null + ? fromBigtableOptions.getMaxBytesPerBatch() + : callSettings.getBatchingSettings().getRequestByteThreshold()); + + long outstandingElements = + writeOptions.getMaxOutstandingElements() != null + ? writeOptions.getMaxOutstandingElements() + : (fromBigtableOptions != null + && fromBigtableOptions.getMaxOutstandingElements() != null + ? fromBigtableOptions.getMaxOutstandingElements() + : callSettings + .getBatchingSettings() + .getFlowControlSettings() + .getMaxOutstandingElementCount()); + + long outstandingBytes = + writeOptions.getMaxOutstandingBytes() != null + ? writeOptions.getMaxOutstandingBytes() + : (fromBigtableOptions != null && fromBigtableOptions.getMaxOutstandingBytes() != null + ? fromBigtableOptions.getMaxOutstandingBytes() + : callSettings + .getBatchingSettings() + .getFlowControlSettings() + .getMaxOutstandingRequestBytes()); + + retrySettings + .setInitialRpcTimeout(Duration.ofMillis(initialRpcTimeout)) + .setMaxRpcTimeout(Duration.ofMillis(initialRpcTimeout)) + .setRpcTimeoutMultiplier(1) + .setTotalTimeout(Duration.ofMillis(Math.max(initialRpcTimeout, totalTimeout))); + batchingSettings + .setFlowControlSettings( + flowControlSettings + .setMaxOutstandingElementCount(outstandingElements) + .setMaxOutstandingRequestBytes(outstandingBytes) + .build()) + .setElementCountThreshold(maxElement) + .setRequestByteThreshold(maxBytes); + + if (fromBigtableOptions != null && fromBigtableOptions.getThrottlingTargetMs() != null) { + settings.enableBatchMutationLatencyBasedThrottling( + fromBigtableOptions.getThrottlingTargetMs()); } - FlowControlSettings.Builder flowControlSettings = - callSettings.getBatchingSettings().getFlowControlSettings().toBuilder(); - if (writeOptions.getMaxOutstandingElements() != null) { - flowControlSettings.setMaxOutstandingElementCount(writeOptions.getMaxOutstandingElements()); - } - if (writeOptions.getMaxOutstandingBytes() != null) { - flowControlSettings.setMaxOutstandingRequestBytes(writeOptions.getMaxOutstandingBytes()); - } - batchingSettings = batchingSettings.setFlowControlSettings(flowControlSettings.build()); - if (writeOptions.getThrottlingTargetMs() != null) { settings.enableBatchMutationLatencyBasedThrottling(writeOptions.getThrottlingTargetMs()); } @@ -235,33 +283,45 @@ private static BigtableDataSettings configureWriteSettings( } private static BigtableDataSettings configureReadSettings( - BigtableDataSettings.Builder settings, BigtableReadOptions readOptions) { + BigtableDataSettings.Builder settings, + BigtableReadOptions readOptions, + BigtableReadOptions optionsFromBigtableOptions) { RetrySettings.Builder retrySettings = settings.stubSettings().readRowsSettings().getRetrySettings().toBuilder(); - if (readOptions.getAttemptTimeout() != null) { - // Set the user specified attempt timeout and expand the operation timeout if it's shorter - retrySettings.setInitialRpcTimeout( - Duration.ofMillis(readOptions.getAttemptTimeout().getMillis())); - retrySettings.setTotalTimeout( - Duration.ofMillis( - Math.max( - retrySettings.getTotalTimeout().toMillis(), - readOptions.getAttemptTimeout().getMillis()))); - } - - if (readOptions.getOperationTimeout() != null) { - retrySettings.setTotalTimeout( - Duration.ofMillis(readOptions.getOperationTimeout().getMillis())); - } - - if (readOptions.getWaitTimeout() != null) { - settings - .stubSettings() - .readRowsSettings() - .setWaitTimeout(Duration.ofMillis(readOptions.getWaitTimeout().getMillis())); - } + // Options set directly on readOptions overrides Options set in BigtableOptions + long initialRpcTimeout = + readOptions.getAttemptTimeout() != null + ? readOptions.getAttemptTimeout().getMillis() + : (optionsFromBigtableOptions != null + && optionsFromBigtableOptions.getAttemptTimeout() != null + ? optionsFromBigtableOptions.getAttemptTimeout().getMillis() + : retrySettings.getInitialRpcTimeout().toMillis()); + + long totalTimeout = + readOptions.getOperationTimeout() != null + ? readOptions.getOperationTimeout().getMillis() + : (optionsFromBigtableOptions != null + && optionsFromBigtableOptions.getOperationTimeout() != null + ? optionsFromBigtableOptions.getOperationTimeout().getMillis() + : retrySettings.getTotalTimeout().toMillis()); + + long waitTimeout = + readOptions.getWaitTimeout() != null + ? readOptions.getWaitTimeout().getMillis() + : (optionsFromBigtableOptions != null + && optionsFromBigtableOptions.getWaitTimeout() != null + ? optionsFromBigtableOptions.getWaitTimeout().getMillis() + : settings.stubSettings().readRowsSettings().getWaitTimeout().toMillis()); + + retrySettings + .setInitialRpcTimeout(Duration.ofMillis(initialRpcTimeout)) + .setMaxRpcTimeout(Duration.ofMillis(initialRpcTimeout)) + .setRpcTimeoutMultiplier(1) + .setTotalTimeout(Duration.ofMillis(Math.max(initialRpcTimeout, totalTimeout))); + + settings.stubSettings().readRowsSettings().setWaitTimeout(Duration.ofMillis(waitTimeout)); settings.stubSettings().readRowsSettings().setRetrySettings(retrySettings.build()); @@ -372,7 +432,6 @@ static BigtableReadOptions translateToBigtableReadOptions( /** Translate BigtableOptions to BigtableWriteOptions. */ static BigtableWriteOptions translateToBigtableWriteOptions( BigtableWriteOptions writeOptions, BigtableOptions options) { - BigtableWriteOptions.Builder builder = writeOptions.toBuilder(); // configure timeouts if (options.getCallOptionsConfig().getMutateRpcAttemptTimeoutMs().isPresent()) { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java index aa369d1f7ca8..a0910bd75343 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java @@ -115,13 +115,16 @@ BigtableServiceEntry getServiceForReading( } BigtableOptions effectiveOptions = getEffectiveOptions(config); + BigtableReadOptions optsFromBigtableOptions = null; if (effectiveOptions != null) { - // If BigtableOptions is set, convert it to BigtableConfig and BigtableWriteOptions + // If BigtableOptions is set, convert it to BigtableConfig and BigtableReadOptions config = BigtableConfigTranslator.translateToBigtableConfig(config, effectiveOptions); - opts = BigtableConfigTranslator.translateToBigtableReadOptions(opts, effectiveOptions); + optsFromBigtableOptions = + BigtableConfigTranslator.translateToBigtableReadOptions(opts, effectiveOptions); } BigtableDataSettings settings = - BigtableConfigTranslator.translateReadToVeneerSettings(config, opts, pipelineOptions); + BigtableConfigTranslator.translateReadToVeneerSettings( + config, opts, optsFromBigtableOptions, pipelineOptions); if (ExperimentalOptions.hasExperiment(pipelineOptions, BIGTABLE_ENABLE_CLIENT_SIDE_METRICS)) { LOG.info("Enabling client side metrics"); @@ -159,14 +162,17 @@ BigtableServiceEntry getServiceForWriting( } BigtableOptions effectiveOptions = getEffectiveOptions(config); + BigtableWriteOptions optsFromBigtableOptions = null; if (effectiveOptions != null) { // If BigtableOptions is set, convert it to BigtableConfig and BigtableWriteOptions config = BigtableConfigTranslator.translateToBigtableConfig(config, effectiveOptions); - opts = BigtableConfigTranslator.translateToBigtableWriteOptions(opts, effectiveOptions); + optsFromBigtableOptions = + BigtableConfigTranslator.translateToBigtableWriteOptions(opts, effectiveOptions); } BigtableDataSettings settings = - BigtableConfigTranslator.translateWriteToVeneerSettings(config, opts, pipelineOptions); + BigtableConfigTranslator.translateWriteToVeneerSettings( + config, opts, optsFromBigtableOptions, pipelineOptions); if (ExperimentalOptions.hasExperiment(pipelineOptions, BIGTABLE_ENABLE_CLIENT_SIDE_METRICS)) { LOG.info("Enabling client side metrics"); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java index f3a70fd225d3..328617bedf00 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java @@ -92,14 +92,14 @@ public void testBigtableOptionsToBigtableReadOptions() throws Exception { BigtableReadOptions.builder() .setTableId(ValueProvider.StaticValueProvider.of("table")) .build(); + BigtableReadOptions fromBigtableOptions = + BigtableConfigTranslator.translateToBigtableReadOptions(readOptions, options); - readOptions = BigtableConfigTranslator.translateToBigtableReadOptions(readOptions, options); + assertNotNull(fromBigtableOptions.getAttemptTimeout()); + assertNotNull(fromBigtableOptions.getOperationTimeout()); - assertNotNull(readOptions.getAttemptTimeout()); - assertNotNull(readOptions.getOperationTimeout()); - - assertEquals(org.joda.time.Duration.millis(100), readOptions.getAttemptTimeout()); - assertEquals(org.joda.time.Duration.millis(1000), readOptions.getOperationTimeout()); + assertEquals(org.joda.time.Duration.millis(100), fromBigtableOptions.getAttemptTimeout()); + assertEquals(org.joda.time.Duration.millis(1000), fromBigtableOptions.getOperationTimeout()); } @Test @@ -129,21 +129,22 @@ public void testBigtableOptionsToBigtableWriteOptions() throws Exception { .setTableId(ValueProvider.StaticValueProvider.of("table")) .build(); - writeOptions = BigtableConfigTranslator.translateToBigtableWriteOptions(writeOptions, options); - - assertNotNull(writeOptions.getAttemptTimeout()); - assertNotNull(writeOptions.getOperationTimeout()); - assertNotNull(writeOptions.getMaxBytesPerBatch()); - assertNotNull(writeOptions.getMaxElementsPerBatch()); - assertNotNull(writeOptions.getMaxOutstandingElements()); - assertNotNull(writeOptions.getMaxOutstandingBytes()); - - assertEquals(org.joda.time.Duration.millis(200), writeOptions.getAttemptTimeout()); - assertEquals(org.joda.time.Duration.millis(2000), writeOptions.getOperationTimeout()); - assertEquals(20, (long) writeOptions.getMaxBytesPerBatch()); - assertEquals(100, (long) writeOptions.getMaxElementsPerBatch()); - assertEquals(5 * 100, (long) writeOptions.getMaxOutstandingElements()); - assertEquals(5 * 20, (long) writeOptions.getMaxOutstandingBytes()); + BigtableWriteOptions fromBigtableOptions = + BigtableConfigTranslator.translateToBigtableWriteOptions(writeOptions, options); + + assertNotNull(fromBigtableOptions.getAttemptTimeout()); + assertNotNull(fromBigtableOptions.getOperationTimeout()); + assertNotNull(fromBigtableOptions.getMaxBytesPerBatch()); + assertNotNull(fromBigtableOptions.getMaxElementsPerBatch()); + assertNotNull(fromBigtableOptions.getMaxOutstandingElements()); + assertNotNull(fromBigtableOptions.getMaxOutstandingBytes()); + + assertEquals(org.joda.time.Duration.millis(200), fromBigtableOptions.getAttemptTimeout()); + assertEquals(org.joda.time.Duration.millis(2000), fromBigtableOptions.getOperationTimeout()); + assertEquals(20, (long) fromBigtableOptions.getMaxBytesPerBatch()); + assertEquals(100, (long) fromBigtableOptions.getMaxElementsPerBatch()); + assertEquals(5 * 100, (long) fromBigtableOptions.getMaxOutstandingElements()); + assertEquals(5 * 20, (long) fromBigtableOptions.getMaxOutstandingBytes()); } @Test @@ -166,7 +167,7 @@ public void testVeneerReadSettings() throws Exception { BigtableDataSettings settings = BigtableConfigTranslator.translateReadToVeneerSettings( - config, readOptions, pipelineOptions); + config, readOptions, null, pipelineOptions); EnhancedBigtableStubSettings stubSettings = settings.getStubSettings(); @@ -181,6 +182,49 @@ public void testVeneerReadSettings() throws Exception { stubSettings.readRowsSettings().getRetrySettings().getTotalTimeout()); } + @Test + public void testReadOptionsOverride() throws Exception { + BigtableConfig config = + BigtableConfig.builder() + .setProjectId(ValueProvider.StaticValueProvider.of("project")) + .setInstanceId(ValueProvider.StaticValueProvider.of("instance")) + .setAppProfileId(ValueProvider.StaticValueProvider.of("app")) + .setValidate(true) + .build(); + + BigtableReadOptions readOptions = + BigtableReadOptions.builder() + .setTableId(ValueProvider.StaticValueProvider.of("table")) + .setAttemptTimeout(org.joda.time.Duration.millis(101)) + .setOperationTimeout(org.joda.time.Duration.millis(1001)) + .build(); + + BigtableOptions options = + BigtableOptions.builder() + .setCallOptionsConfig( + CallOptionsConfig.builder() + .setReadRowsRpcAttemptTimeoutMs(200) + .setReadRowsRpcTimeoutMs(2001) + .build()) + .build(); + + BigtableReadOptions fromBigtableOptions = + BigtableConfigTranslator.translateToBigtableReadOptions(readOptions, options); + + PipelineOptions pipelineOptions = PipelineOptionsFactory.as(GcpOptions.class); + + BigtableDataSettings settings = + BigtableConfigTranslator.translateReadToVeneerSettings( + config, readOptions, fromBigtableOptions, pipelineOptions); + + assertEquals( + Duration.ofMillis(101), + settings.getStubSettings().readRowsSettings().getRetrySettings().getInitialRpcTimeout()); + assertEquals( + Duration.ofMillis(1001), + settings.getStubSettings().readRowsSettings().getRetrySettings().getTotalTimeout()); + } + @Test public void testVeneerWriteSettings() throws Exception { BigtableConfig config = @@ -205,7 +249,7 @@ public void testVeneerWriteSettings() throws Exception { BigtableDataSettings settings = BigtableConfigTranslator.translateWriteToVeneerSettings( - config, writeOptions, pipelineOptions); + config, writeOptions, null, pipelineOptions); EnhancedBigtableStubSettings stubSettings = settings.getStubSettings(); @@ -245,6 +289,85 @@ public void testVeneerWriteSettings() throws Exception { .getMaxOutstandingRequestBytes()); } + @Test + public void testWriteOptionsOverride() throws Exception { + BigtableConfig config = + BigtableConfig.builder() + .setProjectId(ValueProvider.StaticValueProvider.of("project")) + .setInstanceId(ValueProvider.StaticValueProvider.of("instance")) + .setAppProfileId(ValueProvider.StaticValueProvider.of("app")) + .setValidate(true) + .build(); + BigtableWriteOptions writeOptions = + BigtableWriteOptions.builder() + .setTableId(ValueProvider.StaticValueProvider.of("table")) + .setAttemptTimeout(org.joda.time.Duration.millis(101)) + .setOperationTimeout(org.joda.time.Duration.millis(1001)) + .setMaxElementsPerBatch(105) + .setMaxBytesPerBatch(102) + .setMaxOutstandingElements(10001) + .setMaxOutstandingBytes(100001) + .build(); + + BigtableOptions options = + BigtableOptions.builder() + .setCallOptionsConfig( + CallOptionsConfig.builder() + .setMutateRpcAttemptTimeoutMs(456) + .setMutateRpcTimeoutMs(678) + .build()) + .setBulkOptions( + BulkOptions.builder() + .setMaxInflightRpcs(15) + .setBulkMaxRowKeyCount(100) + .setBulkMaxRequestSize(200) + .build()) + .build(); + + PipelineOptions pipelineOptions = PipelineOptionsFactory.as(GcpOptions.class); + + BigtableDataSettings settings = + BigtableConfigTranslator.translateWriteToVeneerSettings( + config, + writeOptions, + BigtableConfigTranslator.translateToBigtableWriteOptions(writeOptions, options), + pipelineOptions); + + EnhancedBigtableStubSettings stubSettings = settings.getStubSettings(); + + assertEquals( + Duration.ofMillis(101), + stubSettings.bulkMutateRowsSettings().getRetrySettings().getInitialRpcTimeout()); + assertEquals( + Duration.ofMillis(1001), + stubSettings.bulkMutateRowsSettings().getRetrySettings().getTotalTimeout()); + assertEquals( + 105, + (long) + stubSettings.bulkMutateRowsSettings().getBatchingSettings().getElementCountThreshold()); + assertEquals( + 102, + (long) + stubSettings.bulkMutateRowsSettings().getBatchingSettings().getRequestByteThreshold()); + assertEquals( + 10001, + (long) + stubSettings + .bulkMutateRowsSettings() + .getBatchingSettings() + .getFlowControlSettings() + .getMaxOutstandingElementCount()); + + assertEquals( + 100001, + (long) + stubSettings + .bulkMutateRowsSettings() + .getBatchingSettings() + .getFlowControlSettings() + .getMaxOutstandingRequestBytes()); + } + @Test public void testUsingCredentialsFromBigtableOptions() throws Exception { Credentials fakeCredentials = Mockito.mock(Credentials.class); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java index 723bf7e7b190..bf9f7d991fa2 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java @@ -97,6 +97,7 @@ public void setup() throws Exception { BigtableConfigTranslator.translateWriteToVeneerSettings( bigtableConfig, BigtableWriteOptions.builder().build(), + null, PipelineOptionsFactory.create()); BigtableTableAdminSettings adminSettings = From 5eeffc7715fe6823d883906b590d8ccf1acfd14c Mon Sep 17 00:00:00 2001 From: Svetak Sundhar Date: Mon, 22 Jan 2024 14:16:38 +0000 Subject: [PATCH 039/169] Address Feedback from Beam Website Feedback Form. (#30064) * Create HealthcareUtils file with shared resources * revert * Addressing some feedback from the Beam Website Feedback Form --- website/www/site/content/en/documentation/io/connectors.md | 2 +- website/www/site/content/en/documentation/programming-guide.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/website/www/site/content/en/documentation/io/connectors.md b/website/www/site/content/en/documentation/io/connectors.md index ab8ccf935cb7..0ef86123dd04 100644 --- a/website/www/site/content/en/documentation/io/connectors.md +++ b/website/www/site/content/en/documentation/io/connectors.md @@ -802,12 +802,12 @@ This table provides a consolidated, at-a-glance overview of the available built- ✔ native + Not available ✔ native Not available - Not available ✔ ✔ diff --git a/website/www/site/content/en/documentation/programming-guide.md b/website/www/site/content/en/documentation/programming-guide.md index 6de0ee472782..47808063254f 100644 --- a/website/www/site/content/en/documentation/programming-guide.md +++ b/website/www/site/content/en/documentation/programming-guide.md @@ -496,7 +496,7 @@ specifying custom encodings as needed. #### 3.2.2. Element schema {#element-schema} -In many cases, the element type in a `PCollection` has a structure that can introspected. +In many cases, the element type in a `PCollection` has a structure that can be introspected. Examples are JSON, Protocol Buffer, Avro, and database records. Schemas provide a way to express types as a set of named fields, allowing for more-expressive aggregations. From 58650378fe781cb9e859bed3a8d7b5872b617160 Mon Sep 17 00:00:00 2001 From: Jan Lukavsky Date: Mon, 22 Jan 2024 17:33:14 +0100 Subject: [PATCH 040/169] [flink] Fix watermark generation for empty UnboundedSource (#30050) --- .../io/source/FlinkSourceReaderBase.java | 1 + .../unbounded/FlinkUnboundedSourceReader.java | 13 ++- .../io/source/EmptyUnboundedSource.java | 101 ++++++++++++++++++ .../io/source/FlinkSourceReaderTestBase.java | 8 ++ .../FlinkUnboundedSourceReaderTest.java | 33 ++++++ 5 files changed, 154 insertions(+), 2 deletions(-) create mode 100644 runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/EmptyUnboundedSource.java diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceReaderBase.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceReaderBase.java index ce4404f8ce9a..6d4fda74b095 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceReaderBase.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceReaderBase.java @@ -172,6 +172,7 @@ public CompletableFuture isAvailable() { .thenAccept(ignored -> {}); } else if (noMoreSplits) { // All the splits have been read, wait for idle timeout. + LOG.debug("All splits have been read, waiting for shutdown timeout {}", idleTimeoutMs); checkIdleTimeoutAndMaybeStartCountdown(); return idleTimeoutFuture; } else { diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReader.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReader.java index 7b02702e244c..99160a9689eb 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReader.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReader.java @@ -36,6 +36,7 @@ import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; @@ -131,8 +132,8 @@ public InputStatus pollNext(ReaderOutput>> ou if (reader != null) { emitRecord(reader, output); return InputStatus.MORE_AVAILABLE; - } else if (noMoreSplits()) { - LOG.trace("No more splits."); + } else if (noMoreSplits() && isEndOfAllReaders()) { + LOG.info("No more splits and no reader available. Terminating consumption."); return InputStatus.END_OF_INPUT; } else { LOG.trace("No data available for now."); @@ -140,6 +141,14 @@ public InputStatus pollNext(ReaderOutput>> ou } } + private boolean isEndOfAllReaders() { + return allReaders().values().stream() + .mapToLong(r -> asUnbounded(r.reader).getWatermark().getMillis()) + .min() + .orElse(BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis()) + >= BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis(); + } + /** * Check whether there are data available from alive readers. If not, set a future and wait for * the periodically running wake-up task to complete that future when the check interval passes. diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/EmptyUnboundedSource.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/EmptyUnboundedSource.java new file mode 100644 index 000000000000..ade48820dbda --- /dev/null +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/EmptyUnboundedSource.java @@ -0,0 +1,101 @@ +/* + * 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.runners.flink.translation.wrappers.streaming.io.source; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Collections; +import java.util.List; +import java.util.NoSuchElementException; +import org.apache.beam.sdk.coders.Coder; +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.transforms.windowing.BoundedWindow; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Instant; + +public class EmptyUnboundedSource + extends UnboundedSource { + + Instant watermark = BoundedWindow.TIMESTAMP_MIN_VALUE; + + public static class DummyCheckpointMark implements UnboundedSource.CheckpointMark, Serializable { + @Override + public void finalizeCheckpoint() {} + } + + @Override + public List> split( + int desiredNumSplits, PipelineOptions options) throws Exception { + return Collections.singletonList(this); + } + + @Override + public UnboundedReader createReader( + PipelineOptions options, @Nullable DummyCheckpointMark checkpointMark) throws IOException { + return new UnboundedReader() { + @Override + public boolean start() throws IOException { + return advance(); + } + + @Override + public boolean advance() throws IOException { + return false; + } + + @Override + public Instant getWatermark() { + return watermark; + } + + @Override + public CheckpointMark getCheckpointMark() { + return new DummyCheckpointMark(); + } + + @Override + public UnboundedSource getCurrentSource() { + return EmptyUnboundedSource.this; + } + + @Override + public T getCurrent() throws NoSuchElementException { + throw new NoSuchElementException(); + } + + @Override + public Instant getCurrentTimestamp() throws NoSuchElementException { + throw new NoSuchElementException(); + } + + @Override + public void close() {} + }; + } + + @Override + public Coder getCheckpointMarkCoder() { + return SerializableCoder.of(DummyCheckpointMark.class); + } + + public void setWatermark(Instant watermark) { + this.watermark = watermark; + } +} diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceReaderTestBase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceReaderTestBase.java index c635a5778b5c..bb45c3eadb73 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceReaderTestBase.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceReaderTestBase.java @@ -32,6 +32,8 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ScheduledExecutorService; import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import javax.annotation.Nullable; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.TestCountingSource; import org.apache.beam.sdk.io.Source; @@ -294,6 +296,12 @@ protected List>> createSplits( return splitList; } + protected List> createEmptySplits(int numSplits) { + return IntStream.range(0, numSplits) + .mapToObj(i -> new FlinkSourceSplit<>(i, new EmptyUnboundedSource())) + .collect(Collectors.toList()); + } + protected void verifyBeamReaderClosed(List>> splits) { splits.forEach( split -> { diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReaderTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReaderTest.java index b7cba373cf75..0ae5b407a157 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReaderTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReaderTest.java @@ -34,11 +34,13 @@ import java.util.function.Function; import org.apache.beam.runners.flink.FlinkPipelineOptions; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.TestCountingSource; +import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.EmptyUnboundedSource; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.FlinkSourceReaderTestBase; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.FlinkSourceSplit; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.SourceTestCompat.TestMetricGroup; import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.ValueWithRecordId; @@ -222,6 +224,37 @@ public void testWatermark() throws Exception { } } + @Test + public void testWatermarkOnEmptySource() throws Exception { + ManuallyTriggeredScheduledExecutorService executor = + new ManuallyTriggeredScheduledExecutorService(); + try (FlinkUnboundedSourceReader> reader = + (FlinkUnboundedSourceReader>) createReader(executor, -1L)) { + List>> splits = createEmptySplits(2); + reader.start(); + reader.addSplits(splits); + reader.notifyNoMoreSplits(); + + for (int i = 0; i < 4; i++) { + assertEquals(InputStatus.NOTHING_AVAILABLE, reader.pollNext(null)); + } + + // move first reader to end of time + ((EmptyUnboundedSource>) splits.get(0).getBeamSplitSource()) + .setWatermark(BoundedWindow.TIMESTAMP_MAX_VALUE); + + for (int i = 0; i < 4; i++) { + assertEquals(InputStatus.NOTHING_AVAILABLE, reader.pollNext(null)); + } + + // move the second reader to end of time + ((EmptyUnboundedSource>) splits.get(1).getBeamSplitSource()) + .setWatermark(BoundedWindow.TIMESTAMP_MAX_VALUE); + + assertEquals(InputStatus.END_OF_INPUT, reader.pollNext(null)); + } + } + @Test public void testPendingBytesMetric() throws Exception { ManuallyTriggeredScheduledExecutorService executor = From 88bf2fc94fe5bdf7721d0f917c7e2405b53aa177 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 22 Jan 2024 11:09:51 -0800 Subject: [PATCH 041/169] Update sdks/python/apache_beam/yaml/yaml_provider.py Co-authored-by: Jeff Kinard --- sdks/python/apache_beam/yaml/yaml_provider.py | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index bc025ef5d316..2a7b34cafbca 100755 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -571,7 +571,19 @@ def create(elements: Iterable[Any], reshuffle: Optional[bool] = True): - {first: 1, second: {str: "bar", values: [4, 5, 6]}} will result in a schema of the form (int, Row(string, List[int])). - + + This can also be expressed as YAML: + type: Create + config: + elements: + - first: 0 + second: + str: "foo" + values: [1, 2, 3] + - first: 1 + second: + str: "bar" + values: [4, 5, 6] Args: elements: The set of elements that should belong to the PCollection. YAML/JSON-style mappings will be interpreted as Beam rows. From b835111d66ff5451107e0382a86f01b15a8f9a27 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 22 Jan 2024 11:12:36 -0800 Subject: [PATCH 042/169] Make note about requiring Graphviz for rendering pipelines. --- sdks/python/apache_beam/yaml/README.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sdks/python/apache_beam/yaml/README.md b/sdks/python/apache_beam/yaml/README.md index dcedb480c511..f7a8ff43b395 100644 --- a/sdks/python/apache_beam/yaml/README.md +++ b/sdks/python/apache_beam/yaml/README.md @@ -79,6 +79,8 @@ execution graph is, e.g. python -m apache_beam.yaml.main --yaml_pipeline_file=/path/to/pipeline.yaml --runner=apache_beam.runners.render.RenderRunner --render_output=out.png [--render_port=0] ``` +(This requires [Graphviz](https://graphviz.org/download/) to be installed to render the pipeline.) + We intend to support running a pipeline on Dataflow by directly passing the yaml specification to a template, no local installation of the Beam SDKs required. From d014a98936eb95316050474c0714277ccc3fa232 Mon Sep 17 00:00:00 2001 From: Edward Cheng Date: Mon, 22 Jan 2024 14:18:34 -0500 Subject: [PATCH 043/169] Time at max threads fix (#30041) * change counterset * add logs * calculate delta for time at max threads * debugging which worker harness * use atomic long instead of counter to keep track of previous time * fix potential race condition --- .../dataflow/worker/StreamingDataflowWorker.java | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index c9a00ade6d8c..f2d7c02729c5 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -228,6 +228,7 @@ public class StreamingDataflowWorker { private final Thread dispatchThread; private final Thread commitThread; private final AtomicLong activeCommitBytes = new AtomicLong(); + private final AtomicLong previousTimeAtMaxThreads = new AtomicLong(); private final AtomicBoolean running = new AtomicBoolean(); private final SideInputStateFetcher sideInputStateFetcher; private final StreamingDataflowWorkerOptions options; @@ -245,10 +246,10 @@ public class StreamingDataflowWorker { private final Counter windmillStateBytesRead; private final Counter windmillStateBytesWritten; private final Counter windmillQuotaThrottling; + private final Counter timeAtMaxActiveThreads; // Built-in cumulative counters. private final Counter javaHarnessUsedMemory; private final Counter javaHarnessMaxMemory; - private final Counter timeAtMaxActiveThreads; private final Counter activeThreads; private final Counter totalAllocatedThreads; private final Counter outstandingBytes; @@ -331,15 +332,15 @@ public class StreamingDataflowWorker { this.windmillQuotaThrottling = pendingDeltaCounters.longSum( StreamingSystemCounterNames.WINDMILL_QUOTA_THROTTLING.counterName()); + this.timeAtMaxActiveThreads = + pendingDeltaCounters.longSum( + StreamingSystemCounterNames.TIME_AT_MAX_ACTIVE_THREADS.counterName()); this.javaHarnessUsedMemory = pendingCumulativeCounters.longSum( StreamingSystemCounterNames.JAVA_HARNESS_USED_MEMORY.counterName()); this.javaHarnessMaxMemory = pendingCumulativeCounters.longSum( StreamingSystemCounterNames.JAVA_HARNESS_MAX_MEMORY.counterName()); - this.timeAtMaxActiveThreads = - pendingCumulativeCounters.longSum( - StreamingSystemCounterNames.TIME_AT_MAX_ACTIVE_THREADS.counterName()); this.activeThreads = pendingCumulativeCounters.intSum(StreamingSystemCounterNames.ACTIVE_THREADS.counterName()); this.outstandingBytes = @@ -1739,7 +1740,9 @@ private void updateVMMetrics() { private void updateThreadMetrics() { timeAtMaxActiveThreads.getAndReset(); - timeAtMaxActiveThreads.addValue(workUnitExecutor.allThreadsActiveTime()); + long allThreadsActiveTime = workUnitExecutor.allThreadsActiveTime(); + timeAtMaxActiveThreads.addValue(allThreadsActiveTime - previousTimeAtMaxThreads.get()); + previousTimeAtMaxThreads.set(allThreadsActiveTime); activeThreads.getAndReset(); activeThreads.addValue(workUnitExecutor.activeCount()); totalAllocatedThreads.getAndReset(); From 173d8340b205c4c81719e5b2eb4ae635b2b5457e Mon Sep 17 00:00:00 2001 From: johnjcasey <95318300+johnjcasey@users.noreply.github.com> Date: Mon, 22 Jan 2024 14:54:38 -0500 Subject: [PATCH 044/169] split write rename and temp table deletion into to dofns with a shuffle (#30023) * split write rename and temp table deletion into to dofns with a shuffle * fix tests * spotless + checkstyle * remove unserializable data being passed between write-rename and table cleanup * Update sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java Co-authored-by: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> * address comments * spotkess --------- Co-authored-by: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> --- .../beam/sdk/io/gcp/bigquery/BatchLoads.java | 38 +- .../sdk/io/gcp/bigquery/BigQueryHelpers.java | 5 +- .../beam/sdk/io/gcp/bigquery/WriteRename.java | 574 +++++++++++------- .../io/gcp/bigquery/BigQueryIOWriteTest.java | 29 +- 4 files changed, 392 insertions(+), 254 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java index 32ee29738bf8..23acd8e01f7f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java @@ -419,16 +419,15 @@ private WriteResult expandTriggered(PCollection> inpu .withSideInputs(sideInputsForUpdateSchema)) .apply( "WriteRenameTriggered", - ParDo.of( - new WriteRename( - bigQueryServices, - copyJobIdPrefixView, - writeDisposition, - createDisposition, - maxRetryJobs, - kmsKey, - loadJobProjectId)) - .withSideInputs(copyJobIdPrefixView)); + new WriteRename( + bigQueryServices, + copyJobIdPrefixView, + writeDisposition, + createDisposition, + maxRetryJobs, + kmsKey, + loadJobProjectId, + copyJobIdPrefixView)); PCollection successfulSinglePartitionWrites = writeSinglePartition(partitions.get(singlePartitionTag), loadJobIdPrefixView) @@ -517,16 +516,15 @@ public WriteResult expandUntriggered(PCollection> inp .withSideInputs(sideInputsForUpdateSchema)) .apply( "WriteRenameUntriggered", - ParDo.of( - new WriteRename( - bigQueryServices, - copyJobIdPrefixView, - writeDisposition, - createDisposition, - maxRetryJobs, - kmsKey, - loadJobProjectId)) - .withSideInputs(copyJobIdPrefixView)) + new WriteRename( + bigQueryServices, + copyJobIdPrefixView, + writeDisposition, + createDisposition, + maxRetryJobs, + kmsKey, + loadJobProjectId, + copyJobIdPrefixView)) .setCoder(tableDestinationCoder); PCollectionList allSuccessfulWrites = diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java index f4120a013ed6..c4ad09ce6eab 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java @@ -32,6 +32,7 @@ import com.google.api.services.bigquery.model.TimePartitioning; import com.google.cloud.hadoop.util.ApiErrorExtractor; import java.io.IOException; +import java.io.Serializable; import java.math.BigInteger; import java.util.ArrayList; import java.util.Collections; @@ -168,7 +169,7 @@ private static boolean nextBackOff(Sleeper sleeper, BackOff backOff) } } - static class PendingJob { + static class PendingJob implements Serializable { private final SerializableFunction executeJob; private final SerializableFunction pollJob; private final SerializableFunction lookupJob; @@ -275,7 +276,7 @@ boolean shouldRetry() { } } - static class RetryJobId { + static class RetryJobId implements Serializable { private final String jobIdPrefix; private final int retryIndex; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java index 9d798b397070..a7177613c60d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java @@ -22,6 +22,7 @@ import com.google.api.services.bigquery.model.JobReference; import com.google.api.services.bigquery.model.TableReference; import java.io.IOException; +import java.io.Serializable; import java.util.Collection; import java.util.List; import java.util.Map; @@ -32,13 +33,19 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.JobService; +import org.apache.beam.sdk.io.gcp.bigquery.WriteTables.Result; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.Reshuffle; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ArrayListMultimap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; @@ -52,8 +59,9 @@ * provides the list of all temporary tables created for a given {@link TableDestination}. */ class WriteRename - extends DoFn>, TableDestination> { - private static final Logger LOG = LoggerFactory.getLogger(WriteRename.class); + extends PTransform< + PCollection>>, + PCollection> { private final BigQueryServices bqServices; private final PCollectionView jobIdToken; @@ -66,27 +74,7 @@ class WriteRename private final int maxRetryJobs; private final @Nullable String kmsKey; private final @Nullable ValueProvider loadJobProjectId; - private transient @Nullable DatasetService datasetService; - - private static class PendingJobData { - final BigQueryHelpers.PendingJob retryJob; - final TableDestination tableDestination; - final List tempTables; - final BoundedWindow window; - - public PendingJobData( - BigQueryHelpers.PendingJob retryJob, - TableDestination tableDestination, - List tempTables, - BoundedWindow window) { - this.retryJob = retryJob; - this.tableDestination = tableDestination; - this.tempTables = tempTables; - this.window = window; - } - } - // All pending copy jobs. - private List pendingJobs = Lists.newArrayList(); + private final PCollectionView copyJobIdPrefixView; public WriteRename( BigQueryServices bqServices, @@ -95,7 +83,8 @@ public WriteRename( CreateDisposition createDisposition, int maxRetryJobs, @Nullable String kmsKey, - @Nullable ValueProvider loadJobProjectId) { + @Nullable ValueProvider loadJobProjectId, + PCollectionView copyJobIdPrefixView) { this.bqServices = bqServices; this.jobIdToken = jobIdToken; this.firstPaneWriteDisposition = writeDisposition; @@ -103,229 +92,368 @@ public WriteRename( this.maxRetryJobs = maxRetryJobs; this.kmsKey = kmsKey; this.loadJobProjectId = loadJobProjectId; + this.copyJobIdPrefixView = copyJobIdPrefixView; } - @StartBundle - public void startBundle(StartBundleContext c) { - pendingJobs.clear(); + @Override + public PCollection expand( + PCollection>> input) { + return input + .apply( + "WriteRename", + ParDo.of( + new WriteRenameFn( + bqServices, + jobIdToken, + firstPaneWriteDisposition, + firstPaneCreateDisposition, + maxRetryJobs, + kmsKey, + loadJobProjectId)) + .withSideInputs(copyJobIdPrefixView)) + // We apply a fusion break here to ensure that on retries, the temp table renaming won't + // attempt to rename a temp table that was previously deleted in TempTableCleanupFn + .apply(Reshuffle.viaRandomKey()) + .apply("RemoveTempTables", ParDo.of(new TempTableCleanupFn(bqServices))) + .setCoder(TableDestinationCoder.of()); } - @Teardown - public void onTeardown() { - try { - if (datasetService != null) { - datasetService.close(); - datasetService = null; - } - } catch (Exception e) { - throw new RuntimeException(e); + public static class PendingJobData implements Serializable { + + final BigQueryHelpers.PendingJob retryJob; + final TableDestination tableDestination; + final List tempTables; + final BoundedWindow window; + + public PendingJobData( + BigQueryHelpers.PendingJob retryJob, + TableDestination tableDestination, + List tempTables, + BoundedWindow window) { + this.retryJob = retryJob; + this.tableDestination = tableDestination; + this.tempTables = tempTables; + this.window = window; } } - @ProcessElement - public void processElement( - @Element Iterable> element, - ProcessContext c, - BoundedWindow window) - throws Exception { - Multimap tempTables = ArrayListMultimap.create(); - for (KV entry : element) { - tempTables.put(entry.getKey(), entry.getValue()); + public static class WriteRenameFn + extends DoFn< + Iterable>, KV>> { + private static final Logger LOG = LoggerFactory.getLogger(WriteRenameFn.class); + + private final BigQueryServices bqServices; + private final PCollectionView jobIdToken; + + // In the triggered scenario, the user-supplied create and write dispositions only apply to the + // first trigger pane, as that's when when the table is created. Subsequent loads should always + // append to the table, and so use CREATE_NEVER and WRITE_APPEND dispositions respectively. + private final WriteDisposition firstPaneWriteDisposition; + private final CreateDisposition firstPaneCreateDisposition; + private final int maxRetryJobs; + private final @Nullable String kmsKey; + private final @Nullable ValueProvider loadJobProjectId; + private transient @Nullable DatasetService datasetService; + + // All pending copy jobs. + private List pendingJobs = Lists.newArrayList(); + + public WriteRenameFn( + BigQueryServices bqServices, + PCollectionView jobIdToken, + WriteDisposition writeDisposition, + CreateDisposition createDisposition, + int maxRetryJobs, + @Nullable String kmsKey, + @Nullable ValueProvider loadJobProjectId) { + this.bqServices = bqServices; + this.jobIdToken = jobIdToken; + this.firstPaneWriteDisposition = writeDisposition; + this.firstPaneCreateDisposition = createDisposition; + this.maxRetryJobs = maxRetryJobs; + this.kmsKey = kmsKey; + this.loadJobProjectId = loadJobProjectId; + } + + @StartBundle + public void startBundle(StartBundleContext c) { + pendingJobs.clear(); } - for (Map.Entry> entry : - tempTables.asMap().entrySet()) { - // Process each destination table. - // Do not copy if no temp tables are provided. - if (!entry.getValue().isEmpty()) { - pendingJobs.add(startWriteRename(entry.getKey(), entry.getValue(), c, window)); + + @Teardown + public void onTeardown() { + try { + if (datasetService != null) { + datasetService.close(); + datasetService = null; + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @ProcessElement + public void processElement( + @Element Iterable> element, + ProcessContext c, + BoundedWindow window) + throws Exception { + Multimap tempTables = ArrayListMultimap.create(); + for (KV entry : element) { + tempTables.put(entry.getKey(), entry.getValue()); + } + for (Map.Entry> entry : + tempTables.asMap().entrySet()) { + // Process each destination table. + // Do not copy if no temp tables are provided. + if (!entry.getValue().isEmpty()) { + pendingJobs.add(startWriteRename(entry.getKey(), entry.getValue(), c, window)); + } } } - } - @FinishBundle - public void finishBundle(FinishBundleContext c) throws Exception { - DatasetService datasetService = - getDatasetService(c.getPipelineOptions().as(BigQueryOptions.class)); - PendingJobManager jobManager = new PendingJobManager(); - for (PendingJobData pendingJob : pendingJobs) { - jobManager.addPendingJob( - pendingJob.retryJob, - j -> { - try { - if (pendingJob.tableDestination.getTableDescription() != null) { - TableReference ref = pendingJob.tableDestination.getTableReference(); - datasetService.patchTableDescription( - ref.clone() - .setTableId(BigQueryHelpers.stripPartitionDecorator(ref.getTableId())), - pendingJob.tableDestination.getTableDescription()); + @FinishBundle + public void finishBundle(FinishBundleContext c) throws Exception { + DatasetService datasetService = + getDatasetService(c.getPipelineOptions().as(BigQueryOptions.class)); + PendingJobManager jobManager = new PendingJobManager(); + for (PendingJobData pendingJob : pendingJobs) { + jobManager.addPendingJob( + pendingJob.retryJob, + j -> { + try { + if (pendingJob.tableDestination.getTableDescription() != null) { + TableReference ref = pendingJob.tableDestination.getTableReference(); + datasetService.patchTableDescription( + ref.clone() + .setTableId(BigQueryHelpers.stripPartitionDecorator(ref.getTableId())), + pendingJob.tableDestination.getTableDescription()); + } + c.output( + KV.of(pendingJob.tableDestination, pendingJob.tempTables), + pendingJob.window.maxTimestamp(), + pendingJob.window); + return null; + } catch (IOException | InterruptedException e) { + return e; } - c.output( - pendingJob.tableDestination, pendingJob.window.maxTimestamp(), pendingJob.window); - removeTemporaryTables(datasetService, pendingJob.tempTables); - return null; - } catch (IOException | InterruptedException e) { - return e; - } - }); + }); + } + jobManager.waitForDone(); } - jobManager.waitForDone(); - } - private DatasetService getDatasetService(PipelineOptions pipelineOptions) throws IOException { - if (datasetService == null) { - datasetService = bqServices.getDatasetService(pipelineOptions.as(BigQueryOptions.class)); + private DatasetService getDatasetService(PipelineOptions pipelineOptions) throws IOException { + if (datasetService == null) { + datasetService = bqServices.getDatasetService(pipelineOptions.as(BigQueryOptions.class)); + } + return datasetService; } - return datasetService; - } - private PendingJobData startWriteRename( - TableDestination finalTableDestination, - Iterable tempTableNames, - ProcessContext c, - BoundedWindow window) - throws Exception { - // The pane may have advanced either here due to triggering or due to an upstream trigger. We - // check the upstream - // trigger to handle the case where an earlier pane triggered the single-partition path. If this - // happened, then the - // table will already exist so we want to append to the table. - WriteTables.@Nullable Result firstTempTable = Iterables.getFirst(tempTableNames, null); - boolean isFirstPane = - firstTempTable != null && firstTempTable.isFirstPane() && c.pane().isFirst(); - WriteDisposition writeDisposition = - isFirstPane ? firstPaneWriteDisposition : WriteDisposition.WRITE_APPEND; - CreateDisposition createDisposition = - isFirstPane ? firstPaneCreateDisposition : CreateDisposition.CREATE_NEVER; - List tempTables = - StreamSupport.stream(tempTableNames.spliterator(), false) - .map( - result -> - BigQueryHelpers.fromJsonString(result.getTableName(), TableReference.class)) - .collect(Collectors.toList()); - - // Make sure each destination table gets a unique job id. - String jobIdPrefix = - BigQueryResourceNaming.createJobIdWithDestination( - c.sideInput(jobIdToken), finalTableDestination, -1, c.pane().getIndex()); - - BigQueryHelpers.PendingJob retryJob = - startCopy( - bqServices.getJobService(c.getPipelineOptions().as(BigQueryOptions.class)), - getDatasetService(c.getPipelineOptions().as(BigQueryOptions.class)), - jobIdPrefix, - finalTableDestination.getTableReference(), - tempTables, - writeDisposition, - createDisposition, - kmsKey, - loadJobProjectId); - return new PendingJobData(retryJob, finalTableDestination, tempTables, window); - } + private PendingJobData startWriteRename( + TableDestination finalTableDestination, + Iterable tempTableNames, + ProcessContext c, + BoundedWindow window) + throws Exception { + // The pane may have advanced either here due to triggering or due to an upstream trigger. We + // check the upstream + // trigger to handle the case where an earlier pane triggered the single-partition path. If + // this + // happened, then the + // table will already exist so we want to append to the table. + WriteTables.@Nullable Result firstTempTable = Iterables.getFirst(tempTableNames, null); + boolean isFirstPane = + firstTempTable != null && firstTempTable.isFirstPane() && c.pane().isFirst(); + WriteDisposition writeDisposition = + isFirstPane ? firstPaneWriteDisposition : WriteDisposition.WRITE_APPEND; + CreateDisposition createDisposition = + isFirstPane ? firstPaneCreateDisposition : CreateDisposition.CREATE_NEVER; + List tempTables = + StreamSupport.stream(tempTableNames.spliterator(), false) + .map( + result -> + BigQueryHelpers.fromJsonString(result.getTableName(), TableReference.class)) + .collect(Collectors.toList()); - private BigQueryHelpers.PendingJob startCopy( - JobService jobService, - DatasetService datasetService, - String jobIdPrefix, - TableReference ref, - List tempTables, - WriteDisposition writeDisposition, - CreateDisposition createDisposition, - @Nullable String kmsKey, - @Nullable ValueProvider loadJobProjectId) { - JobConfigurationTableCopy copyConfig = - new JobConfigurationTableCopy() - .setSourceTables(tempTables) - .setDestinationTable(ref) - .setWriteDisposition(writeDisposition.name()) - .setCreateDisposition(createDisposition.name()); - if (kmsKey != null) { - copyConfig.setDestinationEncryptionConfiguration( - new EncryptionConfiguration().setKmsKeyName(kmsKey)); + // We maintain string versions of the temp tables in order to make pendingJobData serializable + List tempTableStrings = + StreamSupport.stream(tempTableNames.spliterator(), false) + .map(Result::getTableName) + .collect(Collectors.toList()); + ; + + // Make sure each destination table gets a unique job id. + String jobIdPrefix = + BigQueryResourceNaming.createJobIdWithDestination( + c.sideInput(jobIdToken), finalTableDestination, -1, c.pane().getIndex()); + + BigQueryHelpers.PendingJob retryJob = + startCopy( + bqServices.getJobService(c.getPipelineOptions().as(BigQueryOptions.class)), + getDatasetService(c.getPipelineOptions().as(BigQueryOptions.class)), + jobIdPrefix, + finalTableDestination.getTableReference(), + tempTables, + writeDisposition, + createDisposition, + kmsKey, + loadJobProjectId); + return new PendingJobData(retryJob, finalTableDestination, tempTableStrings, window); } - String bqLocation = - BigQueryHelpers.getDatasetLocation(datasetService, ref.getProjectId(), ref.getDatasetId()); - - String projectId = - loadJobProjectId == null || loadJobProjectId.get() == null - ? ref.getProjectId() - : loadJobProjectId.get(); - BigQueryHelpers.PendingJob retryJob = - new BigQueryHelpers.PendingJob( - jobId -> { - JobReference jobRef = - new JobReference() - .setProjectId(projectId) - .setJobId(jobId.getJobId()) - .setLocation(bqLocation); - LOG.info( - "Starting copy job for table {} using {}, job id iteration {}", - ref, - jobRef, - jobId.getRetryIndex()); - try { - jobService.startCopyJob(jobRef, copyConfig); - } catch (IOException | InterruptedException e) { - LOG.warn("Copy job {} failed.", jobRef, e); - throw new RuntimeException(e); - } - return null; - }, - // Function to poll the result of a load job. - jobId -> { - JobReference jobRef = - new JobReference() - .setProjectId(projectId) - .setJobId(jobId.getJobId()) - .setLocation(bqLocation); - try { - return jobService.pollJob(jobRef, BatchLoads.LOAD_JOB_POLL_MAX_RETRIES); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - }, - // Function to lookup a job. - jobId -> { - JobReference jobRef = - new JobReference() - .setProjectId(projectId) - .setJobId(jobId.getJobId()) - .setLocation(bqLocation); - try { - return jobService.getJob(jobRef); - } catch (InterruptedException | IOException e) { - throw new RuntimeException(e); - } - }, - maxRetryJobs, - jobIdPrefix); - return retryJob; + private BigQueryHelpers.PendingJob startCopy( + JobService jobService, + DatasetService datasetService, + String jobIdPrefix, + TableReference ref, + List tempTables, + WriteDisposition writeDisposition, + CreateDisposition createDisposition, + @Nullable String kmsKey, + @Nullable ValueProvider loadJobProjectId) { + JobConfigurationTableCopy copyConfig = + new JobConfigurationTableCopy() + .setSourceTables(tempTables) + .setDestinationTable(ref) + .setWriteDisposition(writeDisposition.name()) + .setCreateDisposition(createDisposition.name()); + if (kmsKey != null) { + copyConfig.setDestinationEncryptionConfiguration( + new EncryptionConfiguration().setKmsKeyName(kmsKey)); + } + + String bqLocation = + BigQueryHelpers.getDatasetLocation( + datasetService, ref.getProjectId(), ref.getDatasetId()); + + String projectId = + loadJobProjectId == null || loadJobProjectId.get() == null + ? ref.getProjectId() + : loadJobProjectId.get(); + BigQueryHelpers.PendingJob retryJob = + new BigQueryHelpers.PendingJob( + jobId -> { + JobReference jobRef = + new JobReference() + .setProjectId(projectId) + .setJobId(jobId.getJobId()) + .setLocation(bqLocation); + LOG.info( + "Starting copy job for table {} using {}, job id iteration {}", + ref, + jobRef, + jobId.getRetryIndex()); + try { + jobService.startCopyJob(jobRef, copyConfig); + } catch (IOException | InterruptedException e) { + LOG.warn("Copy job {} failed.", jobRef, e); + throw new RuntimeException(e); + } + return null; + }, + // Function to poll the result of a load job. + jobId -> { + JobReference jobRef = + new JobReference() + .setProjectId(projectId) + .setJobId(jobId.getJobId()) + .setLocation(bqLocation); + try { + return jobService.pollJob(jobRef, BatchLoads.LOAD_JOB_POLL_MAX_RETRIES); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + }, + // Function to lookup a job. + jobId -> { + JobReference jobRef = + new JobReference() + .setProjectId(projectId) + .setJobId(jobId.getJobId()) + .setLocation(bqLocation); + try { + return jobService.getJob(jobRef); + } catch (InterruptedException | IOException e) { + throw new RuntimeException(e); + } + }, + maxRetryJobs, + jobIdPrefix); + return retryJob; + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + + builder + .add( + DisplayData.item("firstPaneWriteDisposition", firstPaneWriteDisposition.toString()) + .withLabel("Write Disposition")) + .add( + DisplayData.item("firstPaneCreateDisposition", firstPaneCreateDisposition.toString()) + .withLabel("Create Disposition")) + .add( + DisplayData.item("launchesBigQueryJobs", true) + .withLabel("This transform launches BigQuery jobs to read/write elements.")); + } } - static void removeTemporaryTables(DatasetService tableService, List tempTables) { - for (TableReference tableRef : tempTables) { + public static class TempTableCleanupFn + extends DoFn>, TableDestination> { + + private static final Logger LOG = LoggerFactory.getLogger(TempTableCleanupFn.class); + + private final BigQueryServices bqServices; + private transient @Nullable DatasetService datasetService; + + public TempTableCleanupFn(BigQueryServices bqServices) { + this.bqServices = bqServices; + } + + @ProcessElement + public void processElement( + PipelineOptions pipelineOptions, + @Element KV> tempTable, + OutputReceiver destinationOutputReceiver) { + List tableReferences = + tempTable.getValue().stream() + .map(tableName -> BigQueryHelpers.fromJsonString(tableName, TableReference.class)) + .collect(Collectors.toList()); + removeTemporaryTables(getDatasetService(pipelineOptions), tableReferences); + destinationOutputReceiver.output(tempTable.getKey()); + } + + private DatasetService getDatasetService(PipelineOptions pipelineOptions) { + if (datasetService == null) { + datasetService = bqServices.getDatasetService(pipelineOptions.as(BigQueryOptions.class)); + } + return datasetService; + } + + @VisibleForTesting + public static void removeTemporaryTables( + DatasetService datasetService, List tempTables) { + for (TableReference tableRef : tempTables) { + try { + LOG.debug("Deleting table {}", BigQueryHelpers.toJsonString(tableRef)); + datasetService.deleteTable(tableRef); + } catch (Exception e) { + LOG.warn("Failed to delete the table {}", BigQueryHelpers.toJsonString(tableRef), e); + } + } + } + + @Teardown + public void onTeardown() { try { - LOG.debug("Deleting table {}", BigQueryHelpers.toJsonString(tableRef)); - tableService.deleteTable(tableRef); + if (datasetService != null) { + datasetService.close(); + datasetService = null; + } } catch (Exception e) { - LOG.warn("Failed to delete the table {}", BigQueryHelpers.toJsonString(tableRef), e); + throw new RuntimeException(e); } } } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - - builder - .add( - DisplayData.item("firstPaneWriteDisposition", firstPaneWriteDisposition.toString()) - .withLabel("Write Disposition")) - .add( - DisplayData.item("firstPaneCreateDisposition", firstPaneCreateDisposition.toString()) - .withLabel("Create Disposition")) - .add( - DisplayData.item("launchesBigQueryJobs", true) - .withLabel("This transform launches BigQuery jobs to read/write elements.")); - } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java index 55269342155f..89cbc2cd24b8 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.io.gcp.bigquery; import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.toJsonString; +import static org.apache.beam.sdk.io.gcp.bigquery.WriteTables.ResultCoder.INSTANCE; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; 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; @@ -92,6 +93,7 @@ import org.apache.beam.runners.direct.DirectOptions; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.SerializableCoder; import org.apache.beam.sdk.coders.ShardedKeyCoder; @@ -105,6 +107,7 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.Method; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.SchemaUpdateOption; import org.apache.beam.sdk.io.gcp.bigquery.WritePartition.ResultCoder; +import org.apache.beam.sdk.io.gcp.bigquery.WriteRename.TempTableCleanupFn; import org.apache.beam.sdk.io.gcp.bigquery.WriteTables.Result; import org.apache.beam.sdk.io.gcp.testing.FakeBigQueryServices; import org.apache.beam.sdk.io.gcp.testing.FakeDatasetService; @@ -256,7 +259,9 @@ public void evaluate() throws Throwable { }; @Rule public transient ExpectedException thrown = ExpectedException.none(); - @Rule public transient ExpectedLogs loggedWriteRename = ExpectedLogs.none(WriteRename.class); + + @Rule + public transient ExpectedLogs loggedWriteRename = ExpectedLogs.none(TempTableCleanupFn.class); private FakeDatasetService fakeDatasetService = new FakeDatasetService(); private FakeJobService fakeJobService = new FakeJobService(); @@ -2529,7 +2534,7 @@ public void testWriteTables() throws Exception { PCollection> writeTablesOutput = writeTablesInput .apply(writeTables) - .setCoder(KvCoder.of(StringUtf8Coder.of(), WriteTables.ResultCoder.INSTANCE)) + .setCoder(KvCoder.of(StringUtf8Coder.of(), INSTANCE)) .apply( ParDo.of( new DoFn< @@ -2629,13 +2634,19 @@ public void testWriteRename() throws Exception { BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED, 3, "kms_key", - null); + null, + jobIdTokenView); - DoFnTester>, TableDestination> tester = - DoFnTester.of(writeRename); - tester.setSideInput(jobIdTokenView, GlobalWindow.INSTANCE, jobIdToken); - tester.processElement(tempTablesElement); - tester.finishBundle(); + // Unfortunate hack to have create treat tempTablesElement as a single element, instead of as an + // iterable + p.apply( + Create.of( + ImmutableList.of( + (Iterable>) tempTablesElement)) + .withCoder(IterableCoder.of(KvCoder.of(TableDestinationCoder.of(), INSTANCE)))) + .apply(writeRename); + + p.run().waitUntilFinish(); for (Map.Entry> entry : tempTables.asMap().entrySet()) { TableDestination tableDestination = entry.getKey(); @@ -2683,7 +2694,7 @@ public void testRemoveTemporaryTables() throws Exception { tableRefs.add( BigQueryHelpers.parseTableSpec(String.format("%s:%s.%s", projectId, datasetId, "table4"))); - WriteRename.removeTemporaryTables(datasetService, tableRefs); + WriteRename.TempTableCleanupFn.removeTemporaryTables(datasetService, tableRefs); for (TableReference ref : tableRefs) { loggedWriteRename.verifyDebug("Deleting table " + toJsonString(ref)); From 821a1694c5c0910f3f4df79d52fa482499fb6667 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Mon, 22 Jan 2024 21:19:17 -0500 Subject: [PATCH 045/169] Fix Cassandra Test (#30075) * Fix Cassandra Test * Fix Java CassandraIO PreCommit --- sdks/java/io/cassandra/build.gradle | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/sdks/java/io/cassandra/build.gradle b/sdks/java/io/cassandra/build.gradle index 0a1f2f4accda..22208d6828c5 100644 --- a/sdks/java/io/cassandra/build.gradle +++ b/sdks/java/io/cassandra/build.gradle @@ -31,6 +31,14 @@ ext.summary = "IO to read and write with Apache Cassandra database" // compatible with all Cassandra versions up to 3.11.10 def achilles_version = "6.1.0" +configurations.testRuntimeClasspath { + // Force use the old version of JAMM that embedded cassandra server relies on + // TODO(yathu) bump to cassandra-5.x which uses newer jamm when + // * cassandra-5.x released + // * Beam tests with Java11+ (cassandra 5-beta dropped Java8 release) + resolutionStrategy.force 'com.github.jbellis:jamm:0.3.0' +} + dependencies { implementation library.java.vendored_guava_32_1_2_jre implementation project(path: ":sdks:java:core", configuration: "shadow") From 5e7edc45598b6438761386856e96a66487704b69 Mon Sep 17 00:00:00 2001 From: Andrew Crites Date: Tue, 23 Jan 2024 01:28:20 -0800 Subject: [PATCH 046/169] Heartbeats (#29963) * Adds sending new HeartbeatRequest protos to StreamingDataflowWorker. If any HeartbeatResponses are sent from Windmill containing failed work items, aborts processing those work items as soon as possible. * Adds sending new HeartbeatRequest protos when using streaming RPC's (streaming engine). Also adds a test. * Adds new test for custom source reader exiting early for failed work. Adds special exception for handling failed work. * removes some extra cache invalidations and unneeded log statements. * Added streaming_engine prefix to experiment enabling heartbeats and changed exception in state reader to be WorkItemFailedException. * Adds check that heartbeat response sets failed before failing work. * Adds ability to plumb experiments to test server for GrpcWindmillServerTest so we can test the new style heartbeats. * Changes StreamingDataflowWorkerTest to look for latency attribution in new-style heartbeat requests since that's what FakeWindmillServer returns now. --- .../MetricTrackingWindmillServerStub.java | 32 +++- .../runners/dataflow/worker/PubsubReader.java | 8 + .../worker/StreamingDataflowWorker.java | 51 ++++- .../worker/StreamingModeExecutionContext.java | 14 +- .../worker/UngroupedWindmillReader.java | 8 + .../worker/WorkItemCancelledException.java | 39 ++++ .../dataflow/worker/WorkerCustomSources.java | 3 +- .../worker/streaming/ActiveWorkState.java | 66 ++++++- .../worker/streaming/ComputationState.java | 14 +- .../dataflow/worker/streaming/Work.java | 11 ++ .../worker/windmill/WindmillServerStub.java | 6 + .../windmill/client/WindmillStream.java | 5 +- .../client/grpc/GrpcGetDataStream.java | 107 ++++++++--- .../client/grpc/GrpcWindmillServer.java | 35 +++- .../grpc/GrpcWindmillStreamFactory.java | 16 +- .../windmill/state/WindmillStateCache.java | 5 + .../windmill/state/WindmillStateReader.java | 18 +- .../dataflow/worker/FakeWindmillServer.java | 47 ++++- .../worker/StreamingDataflowWorkerTest.java | 76 +++++++- .../StreamingModeExecutionContextTest.java | 6 +- .../worker/WorkerCustomSourcesTest.java | 82 +++++++- .../worker/streaming/ActiveWorkStateTest.java | 50 ++--- .../client/grpc/GrpcWindmillServerTest.java | 177 +++++++++++++++--- .../windmill/src/main/proto/windmill.proto | 51 ++++- 24 files changed, 801 insertions(+), 126 deletions(-) create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkItemCancelledException.java diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java index 0e929249b3a1..800504f44515 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java @@ -27,7 +27,7 @@ import javax.annotation.concurrent.GuardedBy; import org.apache.beam.runners.dataflow.worker.util.MemoryMonitor; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.HeartbeatRequest; import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStreamPool; @@ -239,25 +239,37 @@ public Windmill.GlobalData getSideInputData(Windmill.GlobalDataRequest request) } /** Tells windmill processing is ongoing for the given keys. */ - public void refreshActiveWork(Map> active) { - activeHeartbeats.set(active.size()); + public void refreshActiveWork(Map> heartbeats) { + activeHeartbeats.set(heartbeats.size()); try { if (useStreamingRequests) { // With streaming requests, always send the request even when it is empty, to ensure that // we trigger health checks for the stream even when it is idle. GetDataStream stream = streamPool.getStream(); try { - stream.refreshActiveWork(active); + stream.refreshActiveWork(heartbeats); } finally { streamPool.releaseStream(stream); } - } else if (!active.isEmpty()) { + } else if (!heartbeats.isEmpty()) { + // This code path is only used by appliance which sends heartbeats (used to refresh active + // work) as KeyedGetDataRequests. So we must translate the HeartbeatRequest to a + // KeyedGetDataRequest here regardless of the value of sendKeyedGetDataRequests. Windmill.GetDataRequest.Builder builder = Windmill.GetDataRequest.newBuilder(); - for (Map.Entry> entry : active.entrySet()) { - builder.addRequests( - Windmill.ComputationGetDataRequest.newBuilder() - .setComputationId(entry.getKey()) - .addAllRequests(entry.getValue())); + for (Map.Entry> entry : heartbeats.entrySet()) { + Windmill.ComputationGetDataRequest.Builder perComputationBuilder = + Windmill.ComputationGetDataRequest.newBuilder(); + perComputationBuilder.setComputationId(entry.getKey()); + for (HeartbeatRequest request : entry.getValue()) { + perComputationBuilder.addRequests( + Windmill.KeyedGetDataRequest.newBuilder() + .setShardingKey(request.getShardingKey()) + .setWorkToken(request.getWorkToken()) + .setCacheToken(request.getCacheToken()) + .addAllLatencyAttribution(request.getLatencyAttributionList()) + .build()); + } + builder.addRequests(perComputationBuilder.build()); } server.getData(builder.build()); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubReader.java index d0931e02cc87..be0bccec0265 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubReader.java @@ -112,6 +112,14 @@ protected PubsubReaderIterator(Windmill.WorkItem work) { super(work); } + @Override + public boolean advance() throws IOException { + if (context.workIsFailed()) { + return false; + } + return super.advance(); + } + @Override protected WindowedValue decodeMessage(Windmill.Message message) throws IOException { T value; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index f2d7c02729c5..a95e78288819 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -85,6 +85,7 @@ import org.apache.beam.runners.dataflow.worker.status.LastExceptionDataProvider; import org.apache.beam.runners.dataflow.worker.status.StatusDataProvider; import org.apache.beam.runners.dataflow.worker.status.WorkerStatusPages; +import org.apache.beam.runners.dataflow.worker.streaming.ActiveWorkState.FailedTokens; import org.apache.beam.runners.dataflow.worker.streaming.Commit; import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; import org.apache.beam.runners.dataflow.worker.streaming.ExecutionState; @@ -422,6 +423,7 @@ public void run() { this.publishCounters = publishCounters; this.windmillServer = options.getWindmillServerStub(); + this.windmillServer.setProcessHeartbeatResponses(this::handleHeartbeatResponses); this.metricTrackingWindmillServer = new MetricTrackingWindmillServerStub(windmillServer, memoryMonitor, windmillServiceEnabled); this.metricTrackingWindmillServer.start(); @@ -982,6 +984,9 @@ private void process( String counterName = "dataflow_source_bytes_processed-" + mapTask.getSystemName(); try { + if (work.isFailed()) { + throw new WorkItemCancelledException(workItem.getShardingKey()); + } executionState = computationState.getExecutionStateQueue().poll(); if (executionState == null) { MutableNetwork mapTaskNetwork = mapTaskToNetwork.apply(mapTask); @@ -1098,7 +1103,8 @@ public void close() { work.setState(State.PROCESSING); } }; - }); + }, + work::isFailed); SideInputStateFetcher localSideInputStateFetcher = sideInputStateFetcher.byteTrackingView(); // If the read output KVs, then we can decode Windmill's byte key into a userland @@ -1136,12 +1142,16 @@ public void close() { synchronizedProcessingTime, stateReader, localSideInputStateFetcher, - outputBuilder); + outputBuilder, + work::isFailed); // Blocks while executing work. executionState.workExecutor().execute(); - // Reports source bytes processed to workitemcommitrequest if available. + if (work.isFailed()) { + throw new WorkItemCancelledException(workItem.getShardingKey()); + } + // Reports source bytes processed to WorkItemCommitRequest if available. try { long sourceBytesProcessed = 0; HashMap counters = @@ -1234,6 +1244,12 @@ public void close() { + "Work will not be retried locally.", computationId, key.toStringUtf8()); + } else if (WorkItemCancelledException.isWorkItemCancelledException(t)) { + LOG.debug( + "Execution of work for computation '{}' on key '{}' failed. " + + "Work will not be retried locally.", + computationId, + workItem.getShardingKey()); } else { LastExceptionDataProvider.reportException(t); LOG.debug("Failed work: {}", work); @@ -1369,6 +1385,10 @@ private void commitLoop() { // Adds the commit to the commitStream if it fits, returning true iff it is consumed. private boolean addCommitToStream(Commit commit, CommitWorkStream commitStream) { Preconditions.checkNotNull(commit); + // Drop commits for failed work. Such commits will be dropped by Windmill anyway. + if (commit.work().isFailed()) { + return true; + } final ComputationState state = commit.computationState(); final Windmill.WorkItemCommitRequest request = commit.request(); final int size = commit.getSize(); @@ -1896,6 +1916,25 @@ private void sendWorkerUpdatesToDataflowService( } } + public void handleHeartbeatResponses(List responses) { + for (Windmill.ComputationHeartbeatResponse computationHeartbeatResponse : responses) { + // Maps sharding key to (work token, cache token) for work that should be marked failed. + Map> failedWork = new HashMap<>(); + for (Windmill.HeartbeatResponse heartbeatResponse : + computationHeartbeatResponse.getHeartbeatResponsesList()) { + if (heartbeatResponse.getFailed()) { + failedWork + .computeIfAbsent(heartbeatResponse.getShardingKey(), key -> new ArrayList<>()) + .add( + new FailedTokens( + heartbeatResponse.getWorkToken(), heartbeatResponse.getCacheToken())); + } + } + ComputationState state = computationMap.get(computationHeartbeatResponse.getComputationId()); + if (state != null) state.failWork(failedWork); + } + } + /** * Sends a GetData request to Windmill for all sufficiently old active work. * @@ -1904,15 +1943,15 @@ private void sendWorkerUpdatesToDataflowService( * StreamingDataflowWorkerOptions#getActiveWorkRefreshPeriodMillis}. */ private void refreshActiveWork() { - Map> active = new HashMap<>(); + Map> heartbeats = new HashMap<>(); Instant refreshDeadline = clock.get().minus(Duration.millis(options.getActiveWorkRefreshPeriodMillis())); for (Map.Entry entry : computationMap.entrySet()) { - active.put(entry.getKey(), entry.getValue().getKeysToRefresh(refreshDeadline, sampler)); + heartbeats.put(entry.getKey(), entry.getValue().getKeyHeartbeats(refreshDeadline, sampler)); } - metricTrackingWindmillServer.refreshActiveWork(active); + metricTrackingWindmillServer.refreshActiveWork(heartbeats); } private void invalidateStuckCommits() { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java index d630601c28a3..83cf49112a8d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java @@ -112,6 +112,7 @@ public class StreamingModeExecutionContext extends DataflowExecutionContext activeReader; private volatile long backlogBytes; + private Supplier workIsFailed; public StreamingModeExecutionContext( CounterFactory counterFactory, @@ -135,6 +136,7 @@ public StreamingModeExecutionContext( this.stateNameMap = ImmutableMap.copyOf(stateNameMap); this.stateCache = stateCache; this.backlogBytes = UnboundedSource.UnboundedReader.BACKLOG_UNKNOWN; + this.workIsFailed = () -> Boolean.FALSE; } @VisibleForTesting @@ -142,6 +144,10 @@ public long getBacklogBytes() { return backlogBytes; } + public boolean workIsFailed() { + return workIsFailed.get(); + } + public void start( @Nullable Object key, Windmill.WorkItem work, @@ -150,9 +156,11 @@ public void start( @Nullable Instant synchronizedProcessingTime, WindmillStateReader stateReader, SideInputStateFetcher sideInputStateFetcher, - Windmill.WorkItemCommitRequest.Builder outputBuilder) { + Windmill.WorkItemCommitRequest.Builder outputBuilder, + @Nullable Supplier workFailed) { this.key = key; this.work = work; + this.workIsFailed = (workFailed != null) ? workFailed : () -> Boolean.FALSE; this.computationKey = WindmillComputationKey.create(computationId, work.getKey(), work.getShardingKey()); this.sideInputStateFetcher = sideInputStateFetcher; @@ -429,7 +437,7 @@ void writePCollectionViewData( /** * Execution states in Streaming are shared between multiple map-task executors. Thus this class - * needs to be thread safe for multiple writers. A single stage could have have multiple executors + * needs to be thread safe for multiple writers. A single stage could have multiple executors * running concurrently. */ public static class StreamingModeExecutionState @@ -670,7 +678,7 @@ class StepContext extends DataflowExecutionContext.DataflowStepContext private NavigableSet modifiedUserSynchronizedProcessingTimersOrdered = null; // A list of timer keys that were modified by user processing earlier in this bundle. This // serves a tombstone, so - // that we know not to fire any bundle tiemrs that were moddified. + // that we know not to fire any bundle timers that were modified. private Table modifiedUserTimerKeys = null; public StepContext(DataflowOperationContext operationContext) { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedWindmillReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedWindmillReader.java index e4e56a96c15a..4aac93ceb3fa 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedWindmillReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedWindmillReader.java @@ -99,6 +99,14 @@ class UngroupedWindmillReaderIterator extends WindmillReaderIteratorBase { super(work); } + @Override + public boolean advance() throws IOException { + if (context.workIsFailed()) { + return false; + } + return super.advance(); + } + @Override protected WindowedValue decodeMessage(Windmill.Message message) throws IOException { Instant timestampMillis = diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkItemCancelledException.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkItemCancelledException.java new file mode 100644 index 000000000000..934977fe0985 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkItemCancelledException.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.runners.dataflow.worker; + +/** Indicates that the work item was cancelled and should not be retried. */ +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +public class WorkItemCancelledException extends RuntimeException { + public WorkItemCancelledException(long sharding_key) { + super("Work item cancelled for key " + sharding_key); + } + + /** Returns whether an exception was caused by a {@link WorkItemCancelledException}. */ + public static boolean isWorkItemCancelledException(Throwable t) { + while (t != null) { + if (t instanceof WorkItemCancelledException) { + return true; + } + t = t.getCause(); + } + return false; + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java index a9050236efc8..2dc3494af5e2 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java @@ -836,7 +836,8 @@ public boolean advance() throws IOException { while (true) { if (elemsRead >= maxElems || Instant.now().isAfter(endTime) - || context.isSinkFullHintSet()) { + || context.isSinkFullHintSet() + || context.workIsFailed()) { return false; } try { 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 16266de9d47c..54942dfeee1f 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 @@ -23,6 +23,7 @@ import java.util.ArrayDeque; import java.util.Deque; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Optional; @@ -34,8 +35,10 @@ import javax.annotation.concurrent.ThreadSafe; import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.HeartbeatRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem; import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -50,7 +53,8 @@ * activate, queue, and complete {@link Work} (including invalidating stuck {@link Work}). */ @ThreadSafe -final class ActiveWorkState { +@Internal +public final class ActiveWorkState { private static final Logger LOG = LoggerFactory.getLogger(ActiveWorkState.class); /* The max number of keys in COMMITTING or COMMIT_QUEUED status to be shown.*/ @@ -120,6 +124,50 @@ synchronized ActivateWorkResult activateWorkForKey(ShardedKey shardedKey, Work w return ActivateWorkResult.QUEUED; } + public static final class FailedTokens { + public long workToken; + public long cacheToken; + + public FailedTokens(long workToken, long cacheToken) { + this.workToken = workToken; + this.cacheToken = cacheToken; + } + } + + /** + * Fails any active work matching an element of the input Map. + * + * @param failedWork a map from sharding_key to tokens for the corresponding work. + */ + synchronized void failWorkForKey(Map> failedWork) { + // Note we can't construct a ShardedKey and look it up in activeWork directly since + // HeartbeatResponse doesn't include the user key. + for (Entry> entry : activeWork.entrySet()) { + List failedTokens = failedWork.get(entry.getKey().shardingKey()); + if (failedTokens == null) continue; + for (FailedTokens failedToken : failedTokens) { + for (Work queuedWork : entry.getValue()) { + WorkItem workItem = queuedWork.getWorkItem(); + if (workItem.getWorkToken() == failedToken.workToken + && workItem.getCacheToken() == failedToken.cacheToken) { + LOG.debug( + "Failing work " + + computationStateCache.getComputation() + + " " + + entry.getKey().shardingKey() + + " " + + failedToken.workToken + + " " + + failedToken.cacheToken + + ". The work will be retried and is not lost."); + queuedWork.setFailed(); + break; + } + } + } + } + } + /** * Removes the complete work from the {@link Queue}. The {@link Work} is marked as completed * if its workToken matches the one that is passed in. Returns the next {@link Work} in the {@link @@ -211,14 +259,14 @@ private synchronized ImmutableMap getStuckCommitsAt( return stuckCommits.build(); } - synchronized ImmutableList getKeysToRefresh( + synchronized ImmutableList getKeyHeartbeats( Instant refreshDeadline, DataflowExecutionStateSampler sampler) { return activeWork.entrySet().stream() - .flatMap(entry -> toKeyedGetDataRequestStream(entry, refreshDeadline, sampler)) + .flatMap(entry -> toHeartbeatRequestStream(entry, refreshDeadline, sampler)) .collect(toImmutableList()); } - private static Stream toKeyedGetDataRequestStream( + private static Stream toHeartbeatRequestStream( Entry> shardedKeyAndWorkQueue, Instant refreshDeadline, DataflowExecutionStateSampler sampler) { @@ -227,12 +275,14 @@ private static Stream toKeyedGetDataRequestStream( return workQueue.stream() .filter(work -> work.getStartTime().isBefore(refreshDeadline)) + // Don't send heartbeats for queued work we already know is failed. + .filter(work -> !work.isFailed()) .map( work -> - Windmill.KeyedGetDataRequest.newBuilder() - .setKey(shardedKey.key()) + Windmill.HeartbeatRequest.newBuilder() .setShardingKey(shardedKey.shardingKey()) .setWorkToken(work.getWorkItem().getWorkToken()) + .setCacheToken(work.getWorkItem().getCacheToken()) .addAllLatencyAttribution( work.getLatencyAttributions(true, work.getLatencyTrackingId(), sampler)) .build()); @@ -250,7 +300,7 @@ synchronized void printActiveWork(PrintWriter writer, Instant now) { for (Map.Entry> entry : activeWork.entrySet()) { Queue workQueue = Preconditions.checkNotNull(entry.getValue()); Work activeWork = Preconditions.checkNotNull(workQueue.peek()); - Windmill.WorkItem workItem = activeWork.getWorkItem(); + WorkItem workItem = activeWork.getWorkItem(); if (activeWork.isCommitPending()) { if (++commitsPendingCount >= MAX_PRINTABLE_COMMIT_PENDING_KEYS) { continue; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java index 4ac1d8bc9fac..8207a6ef2f09 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java @@ -19,12 +19,14 @@ import com.google.api.services.dataflow.model.MapTask; import java.io.PrintWriter; +import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentLinkedQueue; import javax.annotation.Nullable; import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; +import org.apache.beam.runners.dataflow.worker.streaming.ActiveWorkState.FailedTokens; import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.HeartbeatRequest; import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -98,6 +100,10 @@ public boolean activateWork(ShardedKey shardedKey, Work work) { } } + public void failWork(Map> failedWork) { + activeWorkState.failWorkForKey(failedWork); + } + /** * Marks the work for the given shardedKey as complete. Schedules queued work for the key if any. */ @@ -120,10 +126,10 @@ private void forceExecute(Work work) { executor.forceExecute(work, work.getWorkItem().getSerializedSize()); } - /** Adds any work started before the refreshDeadline to the GetDataRequest builder. */ - public ImmutableList getKeysToRefresh( + /** Gets HeartbeatRequests for any work started before refreshDeadline. */ + public ImmutableList getKeyHeartbeats( Instant refreshDeadline, DataflowExecutionStateSampler sampler) { - return activeWorkState.getKeysToRefresh(refreshDeadline, sampler); + return activeWorkState.getKeyHeartbeats(refreshDeadline, sampler); } public void printActiveWork(PrintWriter writer) { 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 3a77a8322b4b..69f2a0dcee76 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 @@ -50,6 +50,8 @@ public class Work implements Runnable { private final Consumer processWorkFn; private TimedState currentState; + private boolean isFailed; + private Work(Windmill.WorkItem workItem, Supplier clock, Consumer processWorkFn) { this.workItem = workItem; this.clock = clock; @@ -57,6 +59,7 @@ private Work(Windmill.WorkItem workItem, Supplier clock, Consumer this.startTime = clock.get(); this.totalDurationPerState = new EnumMap<>(Windmill.LatencyAttribution.State.class); this.currentState = TimedState.initialState(startTime); + this.isFailed = false; } public static Work create( @@ -95,6 +98,10 @@ public void setState(State state) { this.currentState = TimedState.create(state, now); } + public void setFailed() { + this.isFailed = true; + } + public boolean isCommitPending() { return currentState.isCommitPending(); } @@ -180,6 +187,10 @@ private static LatencyAttribution.Builder addActiveLatencyBreakdownToBuilder( return builder; } + public boolean isFailed() { + return isFailed; + } + boolean isStuckCommittingAt(Instant stuckCommitDeadline) { return currentState.state() == Work.State.COMMITTING && currentState.startTime().isBefore(stuckCommitDeadline); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServerStub.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServerStub.java index c327e68d7e91..25581bee2089 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServerStub.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServerStub.java @@ -19,8 +19,11 @@ import java.io.IOException; import java.io.PrintWriter; +import java.util.List; import java.util.Set; +import java.util.function.Consumer; import org.apache.beam.runners.dataflow.worker.status.StatusDataProvider; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationHeartbeatResponse; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.CommitWorkStream; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkStream; @@ -79,6 +82,9 @@ public abstract GetWorkStream getWorkStream( @Override public void appendSummaryHtml(PrintWriter writer) {} + public void setProcessHeartbeatResponses( + Consumer> processHeartbeatResponses) {} + /** Generic Exception type for implementors to use to represent errors while making RPCs. */ public static final class RpcException extends RuntimeException { public RpcException(Throwable cause) { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java index fa1f797a1911..7c22f4fb5765 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java @@ -23,6 +23,7 @@ import java.util.function.Consumer; import javax.annotation.concurrent.ThreadSafe; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.HeartbeatRequest; import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget; import org.joda.time.Instant; @@ -59,7 +60,9 @@ Windmill.KeyedGetDataResponse requestKeyedData( Windmill.GlobalData requestGlobalData(Windmill.GlobalDataRequest request); /** Tells windmill processing is ongoing for the given keys. */ - void refreshActiveWork(Map> active); + void refreshActiveWork(Map> heartbeats); + + void onHeartbeatResponse(List responses); } /** Interface for streaming CommitWorkRequests to Windmill. */ diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java index a04a961ca9c2..b6600e04a09d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java @@ -32,10 +32,15 @@ import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; import java.util.function.Function; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationGetDataRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationHeartbeatRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationHeartbeatResponse; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalData; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalDataRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.HeartbeatRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.JobHeader; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataResponse; @@ -64,6 +69,10 @@ public final class GrpcGetDataStream private final ThrottleTimer getDataThrottleTimer; private final JobHeader jobHeader; private final int streamingRpcBatchLimit; + // If true, then active work refreshes will be sent as KeyedGetDataRequests. Otherwise, use the + // newer ComputationHeartbeatRequests. + private final boolean sendKeyedGetDataRequests; + private Consumer> processHeartbeatResponses; private GrpcGetDataStream( Function, StreamObserver> @@ -75,7 +84,9 @@ private GrpcGetDataStream( ThrottleTimer getDataThrottleTimer, JobHeader jobHeader, AtomicLong idGenerator, - int streamingRpcBatchLimit) { + int streamingRpcBatchLimit, + boolean sendKeyedGetDataRequests, + Consumer> processHeartbeatResponses) { super( startGetDataRpcFn, backoff, streamObserverFactory, streamRegistry, logEveryNStreamFailures); this.idGenerator = idGenerator; @@ -84,6 +95,8 @@ private GrpcGetDataStream( this.streamingRpcBatchLimit = streamingRpcBatchLimit; this.batches = new ConcurrentLinkedDeque<>(); this.pending = new ConcurrentHashMap<>(); + this.sendKeyedGetDataRequests = sendKeyedGetDataRequests; + this.processHeartbeatResponses = processHeartbeatResponses; } public static GrpcGetDataStream create( @@ -96,7 +109,9 @@ public static GrpcGetDataStream create( ThrottleTimer getDataThrottleTimer, JobHeader jobHeader, AtomicLong idGenerator, - int streamingRpcBatchLimit) { + int streamingRpcBatchLimit, + boolean sendKeyedGetDataRequests, + Consumer> processHeartbeatResponses) { GrpcGetDataStream getDataStream = new GrpcGetDataStream( startGetDataRpcFn, @@ -107,7 +122,9 @@ public static GrpcGetDataStream create( getDataThrottleTimer, jobHeader, idGenerator, - streamingRpcBatchLimit); + streamingRpcBatchLimit, + sendKeyedGetDataRequests, + processHeartbeatResponses); getDataStream.startStream(); return getDataStream; } @@ -138,6 +155,7 @@ protected void onResponse(StreamingGetDataResponse chunk) { checkArgument(chunk.getRequestIdCount() == chunk.getSerializedResponseCount()); checkArgument(chunk.getRemainingBytesForResponse() == 0 || chunk.getRequestIdCount() == 1); getDataThrottleTimer.stop(); + onHeartbeatResponse(chunk.getComputationHeartbeatResponseList()); for (int i = 0; i < chunk.getRequestIdCount(); ++i) { AppendableInputStream responseStream = pending.get(chunk.getRequestId(i)); @@ -171,30 +189,71 @@ public GlobalData requestGlobalData(GlobalDataRequest request) { } @Override - public void refreshActiveWork(Map> active) { - long builderBytes = 0; + public void refreshActiveWork(Map> heartbeats) { StreamingGetDataRequest.Builder builder = StreamingGetDataRequest.newBuilder(); - for (Map.Entry> entry : active.entrySet()) { - for (KeyedGetDataRequest request : entry.getValue()) { - // Calculate the bytes with some overhead for proto encoding. - long bytes = (long) entry.getKey().length() + request.getSerializedSize() + 10; - if (builderBytes > 0 - && (builderBytes + bytes > AbstractWindmillStream.RPC_STREAM_CHUNK_SIZE - || builder.getRequestIdCount() >= streamingRpcBatchLimit)) { - send(builder.build()); - builderBytes = 0; - builder.clear(); + if (sendKeyedGetDataRequests) { + long builderBytes = 0; + for (Map.Entry> entry : heartbeats.entrySet()) { + for (HeartbeatRequest request : entry.getValue()) { + // Calculate the bytes with some overhead for proto encoding. + long bytes = (long) entry.getKey().length() + request.getSerializedSize() + 10; + if (builderBytes > 0 + && (builderBytes + bytes > AbstractWindmillStream.RPC_STREAM_CHUNK_SIZE + || builder.getRequestIdCount() >= streamingRpcBatchLimit)) { + send(builder.build()); + builderBytes = 0; + builder.clear(); + } + builderBytes += bytes; + builder.addStateRequest( + ComputationGetDataRequest.newBuilder() + .setComputationId(entry.getKey()) + .addRequests( + Windmill.KeyedGetDataRequest.newBuilder() + .setShardingKey(request.getShardingKey()) + .setWorkToken(request.getWorkToken()) + .setCacheToken(request.getCacheToken()) + .addAllLatencyAttribution(request.getLatencyAttributionList()) + .build())); } - builderBytes += bytes; - builder.addStateRequest( - ComputationGetDataRequest.newBuilder() - .setComputationId(entry.getKey()) - .addRequests(request)); + } + + if (builderBytes > 0) { + send(builder.build()); + } + } else { + // No translation necessary, but we must still respect `RPC_STREAM_CHUNK_SIZE`. + long builderBytes = 0; + for (Map.Entry> entry : heartbeats.entrySet()) { + ComputationHeartbeatRequest.Builder computationHeartbeatBuilder = + ComputationHeartbeatRequest.newBuilder().setComputationId(entry.getKey()); + for (HeartbeatRequest request : entry.getValue()) { + long bytes = (long) entry.getKey().length() + request.getSerializedSize() + 10; + if (builderBytes > 0 + && builderBytes + bytes > AbstractWindmillStream.RPC_STREAM_CHUNK_SIZE) { + if (computationHeartbeatBuilder.getHeartbeatRequestsCount() > 0) { + builder.addComputationHeartbeatRequest(computationHeartbeatBuilder.build()); + } + send(builder.build()); + builderBytes = 0; + builder.clear(); + computationHeartbeatBuilder.clear().setComputationId(entry.getKey()); + } + builderBytes += bytes; + computationHeartbeatBuilder.addHeartbeatRequests(request); + } + builder.addComputationHeartbeatRequest(computationHeartbeatBuilder.build()); + } + + if (builderBytes > 0) { + send(builder.build()); } } - if (builderBytes > 0) { - send(builder.build()); - } + } + + @Override + public void onHeartbeatResponse(List responses) { + processHeartbeatResponses.accept(responses); } @Override @@ -277,7 +336,7 @@ private void queueRequestAndWait(QueuedRequest request) throws InterruptedExcept waitForSendLatch.await(); } // Finalize the batch so that no additional requests will be added. Leave the batch in the - // queue so that a subsequent batch will wait for it's completion. + // queue so that a subsequent batch will wait for its completion. synchronized (batches) { verify(batch == batches.peekFirst()); batch.markFinalized(); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServer.java index 3a881df71462..9f0126a9cc69 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServer.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServer.java @@ -28,13 +28,17 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.function.Consumer; import java.util.function.Supplier; import javax.annotation.Nullable; +import org.apache.beam.runners.dataflow.DataflowRunner; import org.apache.beam.runners.dataflow.worker.options.StreamingDataflowWorkerOptions; import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc; import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.CommitWorkRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.CommitWorkResponse; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationHeartbeatResponse; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetConfigRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetConfigResponse; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetDataRequest; @@ -93,6 +97,10 @@ public final class GrpcWindmillServer extends WindmillServerStub { private final StreamingEngineThrottleTimers throttleTimers; private Duration maxBackoff; private @Nullable WindmillApplianceGrpc.WindmillApplianceBlockingStub syncApplianceStub; + // If true, then active work refreshes will be sent as KeyedGetDataRequests. Otherwise, use the + // newer ComputationHeartbeatRequests. + private final boolean sendKeyedGetDataRequests; + private Consumer> processHeartbeatResponses; private GrpcWindmillServer( StreamingDataflowWorkerOptions options, GrpcDispatcherClient grpcDispatcherClient) { @@ -118,9 +126,21 @@ private GrpcWindmillServer( this.dispatcherClient = grpcDispatcherClient; this.syncApplianceStub = null; + this.sendKeyedGetDataRequests = + !options.isEnableStreamingEngine() + || !DataflowRunner.hasExperiment( + options, "streaming_engine_send_new_heartbeat_requests"); + this.processHeartbeatResponses = (responses) -> {}; } - private static StreamingDataflowWorkerOptions testOptions(boolean enableStreamingEngine) { + @Override + public void setProcessHeartbeatResponses( + Consumer> processHeartbeatResponses) { + this.processHeartbeatResponses = processHeartbeatResponses; + }; + + private static StreamingDataflowWorkerOptions testOptions( + boolean enableStreamingEngine, List additionalExperiments) { StreamingDataflowWorkerOptions options = PipelineOptionsFactory.create().as(StreamingDataflowWorkerOptions.class); options.setProject("project"); @@ -131,6 +151,7 @@ private static StreamingDataflowWorkerOptions testOptions(boolean enableStreamin if (enableStreamingEngine) { experiments.add(GcpOptions.STREAMING_ENGINE_EXPERIMENT); } + experiments.addAll(additionalExperiments); options.setExperiments(experiments); options.setWindmillServiceStreamingRpcBatchLimit(Integer.MAX_VALUE); @@ -162,7 +183,7 @@ public static GrpcWindmillServer create(StreamingDataflowWorkerOptions workerOpt } @VisibleForTesting - static GrpcWindmillServer newTestInstance(String name) { + static GrpcWindmillServer newTestInstance(String name, List experiments) { ManagedChannel inProcessChannel = inProcessChannel(name); CloudWindmillServiceV1Alpha1Stub stub = CloudWindmillServiceV1Alpha1Grpc.newStub(inProcessChannel); @@ -173,14 +194,15 @@ static GrpcWindmillServer newTestInstance(String name) { WindmillStubFactory.inProcessStubFactory(name, unused -> inProcessChannel), dispatcherStubs, dispatcherEndpoints); - return new GrpcWindmillServer(testOptions(/* enableStreamingEngine= */ true), dispatcherClient); + return new GrpcWindmillServer( + testOptions(/* enableStreamingEngine= */ true, experiments), dispatcherClient); } @VisibleForTesting static GrpcWindmillServer newApplianceTestInstance(Channel channel) { GrpcWindmillServer testServer = new GrpcWindmillServer( - testOptions(/* enableStreamingEngine= */ false), + testOptions(/* enableStreamingEngine= */ false, new ArrayList<>()), // No-op, Appliance does not use Dispatcher to call Streaming Engine. GrpcDispatcherClient.create(WindmillStubFactory.inProcessStubFactory("test"))); testServer.syncApplianceStub = createWindmillApplianceStubWithDeadlineInterceptor(channel); @@ -319,7 +341,10 @@ public GetWorkStream getWorkStream(GetWorkRequest request, WorkItemReceiver rece @Override public GetDataStream getDataStream() { return windmillStreamFactory.createGetDataStream( - dispatcherClient.getDispatcherStub(), throttleTimers.getDataThrottleTimer()); + dispatcherClient.getDispatcherStub(), + throttleTimers.getDataThrottleTimer(), + sendKeyedGetDataRequests, + this.processHeartbeatResponses); } @Override diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillStreamFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillStreamFactory.java index 099be8db0fda..7dc43e791e31 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillStreamFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillStreamFactory.java @@ -21,6 +21,7 @@ import com.google.auto.value.AutoBuilder; import java.io.PrintWriter; +import java.util.List; import java.util.Set; import java.util.Timer; import java.util.TimerTask; @@ -32,6 +33,7 @@ import javax.annotation.concurrent.ThreadSafe; import org.apache.beam.runners.dataflow.worker.status.StatusDataProvider; import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationHeartbeatResponse; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.JobHeader; import org.apache.beam.runners.dataflow.worker.windmill.WindmillEndpoints; @@ -152,7 +154,10 @@ public GetWorkStream createDirectGetWorkStream( } public GetDataStream createGetDataStream( - CloudWindmillServiceV1Alpha1Stub stub, ThrottleTimer getDataThrottleTimer) { + CloudWindmillServiceV1Alpha1Stub stub, + ThrottleTimer getDataThrottleTimer, + boolean sendKeyedGetDataRequests, + Consumer> processHeartbeatResponses) { return GrpcGetDataStream.create( responseObserver -> withDeadline(stub).getDataStream(responseObserver), grpcBackOff.get(), @@ -162,7 +167,14 @@ public GetDataStream createGetDataStream( getDataThrottleTimer, jobHeader, streamIdGenerator, - streamingRpcBatchLimit); + streamingRpcBatchLimit, + sendKeyedGetDataRequests, + processHeartbeatResponses); + } + + public GetDataStream createGetDataStream( + CloudWindmillServiceV1Alpha1Stub stub, ThrottleTimer getDataThrottleTimer) { + return createGetDataStream(stub, getDataThrottleTimer, false, (response) -> {}); } public CommitWorkStream createCommitWorkStream( diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCache.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCache.java index 6c1239d6ebd2..5a9e5443a506 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCache.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCache.java @@ -296,6 +296,11 @@ private ForComputation(String computation) { this.computation = computation; } + /** Returns the computation associated to this class. */ + public String getComputation() { + return this.computation; + } + /** Invalidate all cache entries for this computation and {@code processingKey}. */ public void invalidate(ByteString processingKey, long shardingKey) { WindmillComputationKey key = diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReader.java index c28939c59ee2..637b838c7fe2 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReader.java @@ -39,6 +39,7 @@ import org.apache.beam.runners.dataflow.worker.KeyTokenInvalidException; import org.apache.beam.runners.dataflow.worker.MetricTrackingWindmillServerStub; import org.apache.beam.runners.dataflow.worker.WindmillTimeUtils; +import org.apache.beam.runners.dataflow.worker.WorkItemCancelledException; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataResponse; @@ -123,6 +124,7 @@ public class WindmillStateReader { private final MetricTrackingWindmillServerStub metricTrackingWindmillServerStub; private final ConcurrentHashMap, CoderAndFuture> waiting; private long bytesRead = 0L; + private final Supplier workItemIsFailed; public WindmillStateReader( MetricTrackingWindmillServerStub metricTrackingWindmillServerStub, @@ -130,7 +132,8 @@ public WindmillStateReader( ByteString key, long shardingKey, long workToken, - Supplier readWrapperSupplier) { + Supplier readWrapperSupplier, + Supplier workItemIsFailed) { this.metricTrackingWindmillServerStub = metricTrackingWindmillServerStub; this.computation = computation; this.key = key; @@ -139,6 +142,7 @@ public WindmillStateReader( this.readWrapperSupplier = readWrapperSupplier; this.waiting = new ConcurrentHashMap<>(); this.pendingLookups = new ConcurrentLinkedQueue<>(); + this.workItemIsFailed = workItemIsFailed; } public WindmillStateReader( @@ -147,7 +151,14 @@ public WindmillStateReader( ByteString key, long shardingKey, long workToken) { - this(metricTrackingWindmillServerStub, computation, key, shardingKey, workToken, () -> null); + this( + metricTrackingWindmillServerStub, + computation, + key, + shardingKey, + workToken, + () -> null, + () -> Boolean.FALSE); } private Future stateFuture(StateTag stateTag, @Nullable Coder coder) { @@ -404,6 +415,9 @@ public void performReads() { private KeyedGetDataResponse tryGetDataFromWindmill(HashSet> stateTags) throws Exception { + if (workItemIsFailed.get()) { + throw new WorkItemCancelledException(shardingKey); + } KeyedGetDataRequest keyedGetDataRequest = createRequest(stateTags); try (AutoCloseable ignored = readWrapperSupplier.get()) { return Optional.ofNullable( diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java index a434b2001207..2cfec6d3139a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java @@ -46,8 +46,11 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill.CommitWorkResponse; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationCommitWorkRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationGetDataRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationHeartbeatRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationHeartbeatResponse; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetDataRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetDataResponse; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.HeartbeatRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution.State; @@ -80,9 +83,10 @@ class FakeWindmillServer extends WindmillServerStub { private final ErrorCollector errorCollector; private final ConcurrentHashMap> droppedStreamingCommits; private int commitsRequested = 0; - private List getDataRequests = new ArrayList<>(); + private final List getDataRequests = new ArrayList<>(); private boolean isReady = true; private boolean dropStreamingCommits = false; + private Consumer> processHeartbeatResponses; public FakeWindmillServer(ErrorCollector errorCollector) { workToOffer = @@ -91,7 +95,7 @@ public FakeWindmillServer(ErrorCollector errorCollector) { dataToOffer = new ResponseQueue() .returnByDefault(GetDataResponse.getDefaultInstance()) - // Sleep for a little bit to ensure that *-windmill-read state-sampled counters show up. + // Sleep for a bit to ensure that *-windmill-read state-sampled counters show up. .delayEachResponseBy(Duration.millis(500)); commitsToOffer = new ResponseQueue() @@ -102,6 +106,13 @@ public FakeWindmillServer(ErrorCollector errorCollector) { this.errorCollector = errorCollector; statsReceived = new ArrayList<>(); droppedStreamingCommits = new ConcurrentHashMap<>(); + processHeartbeatResponses = (responses) -> {}; + } + + @Override + public void setProcessHeartbeatResponses( + Consumer> processHeartbeatResponses) { + this.processHeartbeatResponses = processHeartbeatResponses; } public void setDropStreamingCommits(boolean dropStreamingCommits) { @@ -116,6 +127,10 @@ public ResponseQueue whenGetDataCalled() { return dataToOffer; } + public void sendFailedHeartbeats(List responses) { + getDataStream().onHeartbeatResponse(responses); + } + public ResponseQueue whenCommitWorkCalled() { return commitsToOffer; @@ -304,17 +319,23 @@ public Windmill.GlobalData requestGlobalData(Windmill.GlobalDataRequest request) } @Override - public void refreshActiveWork(Map> active) { + public void refreshActiveWork(Map> heartbeats) { Windmill.GetDataRequest.Builder builder = Windmill.GetDataRequest.newBuilder(); - for (Map.Entry> entry : active.entrySet()) { - builder.addRequests( - ComputationGetDataRequest.newBuilder() + for (Map.Entry> entry : heartbeats.entrySet()) { + builder.addComputationHeartbeatRequest( + ComputationHeartbeatRequest.newBuilder() .setComputationId(entry.getKey()) - .addAllRequests(entry.getValue())); + .addAllHeartbeatRequests(entry.getValue())); } + getData(builder.build()); } + @Override + public void onHeartbeatResponse(List responses) { + processHeartbeatResponses.accept(responses); + } + @Override public void close() {} @@ -383,6 +404,18 @@ public void waitForEmptyWorkQueue() { } } + public Map waitForAndGetCommitsWithTimeout( + int numCommits, Duration timeout) { + LOG.debug("waitForAndGetCommitsWithTimeout: {} {}", numCommits, timeout); + Instant waitStart = Instant.now(); + while (commitsReceived.size() < commitsRequested + numCommits + && Instant.now().isBefore(waitStart.plus(timeout))) { + Uninterruptibles.sleepUninterruptibly(1000, TimeUnit.MILLISECONDS); + } + commitsRequested += numCommits; + return commitsReceived; + } + public Map waitForAndGetCommits(int numCommits) { LOG.debug("waitForAndGetCommitsRequest: {}", numCommits); int maxTries = 10; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java index 31a9af9004a8..9526c96fd04e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java @@ -109,9 +109,12 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill.CommitStatus; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationGetDataRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationGetDataResponse; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationHeartbeatRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationHeartbeatResponse; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetDataRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetDataResponse; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkResponse; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.HeartbeatRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.InputMessageBundle; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataResponse; @@ -577,8 +580,9 @@ private Windmill.GetWorkResponse makeInput( } /** - * Returns a {@link org.apache.beam.runners.dataflow.windmill.Windmill.WorkItemCommitRequest} - * builder parsed from the provided text format proto. + * Returns a {@link + * org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItemCommitRequest} builder parsed + * from the provided text format proto. */ private WorkItemCommitRequest.Builder parseCommitRequest(String output) throws Exception { WorkItemCommitRequest.Builder builder = Windmill.WorkItemCommitRequest.newBuilder(); @@ -3258,6 +3262,49 @@ public void testActiveWorkRefresh() throws Exception { assertThat(server.numGetDataRequests(), greaterThan(0)); } + @Test + public void testActiveWorkFailure() throws Exception { + List instructions = + Arrays.asList( + makeSourceInstruction(StringUtf8Coder.of()), + makeDoFnInstruction(blockingFn, 0, StringUtf8Coder.of()), + makeSinkInstruction(StringUtf8Coder.of(), 0)); + + FakeWindmillServer server = new FakeWindmillServer(errorCollector); + StreamingDataflowWorkerOptions options = createTestingPipelineOptions(server); + options.setActiveWorkRefreshPeriodMillis(100); + StreamingDataflowWorker worker = makeWorker(instructions, options, true /* publishCounters */); + worker.start(); + + // Queue up two work items for the same key. + server + .whenGetWorkCalled() + .thenReturn(makeInput(0, TimeUnit.MILLISECONDS.toMicros(0), "key", DEFAULT_SHARDING_KEY)) + .thenReturn(makeInput(1, TimeUnit.MILLISECONDS.toMicros(0), "key", DEFAULT_SHARDING_KEY)); + server.waitForEmptyWorkQueue(); + + // Mock Windmill sending a heartbeat response failing the second work item while the first + // is still processing. + ComputationHeartbeatResponse.Builder failedHeartbeat = + ComputationHeartbeatResponse.newBuilder(); + failedHeartbeat + .setComputationId(DEFAULT_COMPUTATION_ID) + .addHeartbeatResponsesBuilder() + .setCacheToken(3) + .setWorkToken(1) + .setShardingKey(DEFAULT_SHARDING_KEY) + .setFailed(true); + server.sendFailedHeartbeats(Collections.singletonList(failedHeartbeat.build())); + + // Release the blocked calls. + BlockingFn.blocker.countDown(); + Map commits = + server.waitForAndGetCommitsWithTimeout(2, Duration.standardSeconds((5))); + assertEquals(1, commits.size()); + + worker.stop(); + } + @Test public void testLatencyAttributionProtobufsPopulated() { FakeClock clock = new FakeClock(); @@ -3573,7 +3620,10 @@ public void testDoFnActiveMessageMetadataReportedOnHeartbeat() throws Exception Windmill.GetDataRequest heartbeat = server.getGetDataRequests().get(2); for (LatencyAttribution la : - heartbeat.getRequests(0).getRequests(0).getLatencyAttributionList()) { + heartbeat + .getComputationHeartbeatRequest(0) + .getHeartbeatRequests(0) + .getLatencyAttributionList()) { if (la.getState() == State.ACTIVE) { assertTrue(la.getActiveLatencyBreakdownCount() > 0); assertTrue(la.getActiveLatencyBreakdown(0).hasActiveMessageMetadata()); @@ -3768,7 +3818,7 @@ public void testStuckCommit() throws Exception { server .whenGetWorkCalled() .thenReturn(makeInput(1, TimeUnit.MILLISECONDS.toMicros(1), DEFAULT_KEY_STRING, 1)); - // Ensure that the this work item processes. + // Ensure that this work item processes. Map result = server.waitForAndGetCommits(1); // Now ensure that nothing happens if a dropped commit actually completes. droppedCommits.values().iterator().next().accept(CommitStatus.OK); @@ -4129,7 +4179,7 @@ public void run() { FakeClock.this.schedule(Duration.millis(unit.toMillis(delay)), this); } }); - FakeClock.this.sleep(Duration.ZERO); // Execute work that has an intial delay of zero. + FakeClock.this.sleep(Duration.ZERO); // Execute work that has an initial delay of zero. return null; } } @@ -4167,6 +4217,7 @@ Duration getLatencyAttributionDuration(long workToken, LatencyAttribution.State } boolean isActiveWorkRefresh(GetDataRequest request) { + if (request.getComputationHeartbeatRequestCount() > 0) return true; for (ComputationGetDataRequest computationRequest : request.getRequestsList()) { if (!computationRequest.getComputationId().equals(DEFAULT_COMPUTATION_ID)) { return false; @@ -4203,6 +4254,21 @@ GetDataResponse getData(GetDataRequest request) { } } } + for (ComputationHeartbeatRequest heartbeatRequest : + request.getComputationHeartbeatRequestList()) { + for (HeartbeatRequest heartbeat : heartbeatRequest.getHeartbeatRequestsList()) { + for (LatencyAttribution la : heartbeat.getLatencyAttributionList()) { + EnumMap durations = + totalDurations.computeIfAbsent( + heartbeat.getWorkToken(), + (Long workToken) -> + new EnumMap( + LatencyAttribution.State.class)); + Duration cur = Duration.millis(la.getTotalDurationMillis()); + durations.compute(la.getState(), (s, d) -> d == null || d.isShorterThan(cur) ? cur : d); + } + } + } return EMPTY_DATA_RESPONDER.apply(request); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java index 60ecaa3e37e0..451ec649aa23 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java @@ -137,7 +137,8 @@ public void testTimerInternalsSetTimer() { null, // synchronized processing time stateReader, sideInputStateFetcher, - outputBuilder); + outputBuilder, + null); TimerInternals timerInternals = stepContext.timerInternals(); @@ -187,7 +188,8 @@ public void testTimerInternalsProcessingTimeSkew() { null, // synchronized processing time stateReader, sideInputStateFetcher, - outputBuilder); + outputBuilder, + null); TimerInternals timerInternals = stepContext.timerInternals(); assertTrue(timerTimestamp.isBefore(timerInternals.currentProcessingTime())); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java index 6fa2ffe711f8..b488641d1ca5 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java @@ -37,6 +37,7 @@ import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.lessThan; import static org.hamcrest.Matchers.lessThanOrEqualTo; @@ -90,9 +91,12 @@ import org.apache.beam.runners.dataflow.worker.counters.CounterSet; import org.apache.beam.runners.dataflow.worker.counters.NameContext; import org.apache.beam.runners.dataflow.worker.profiler.ScopedProfiler.NoopProfileScope; +import org.apache.beam.runners.dataflow.worker.streaming.Work; import org.apache.beam.runners.dataflow.worker.testing.TestCountingSource; import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader; +import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader.NativeReaderIterator; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.Coder; @@ -613,7 +617,8 @@ public void testReadUnboundedReader() throws Exception { null, // synchronized processing time null, // StateReader null, // StateFetcher - Windmill.WorkItemCommitRequest.newBuilder()); + Windmill.WorkItemCommitRequest.newBuilder(), + null); @SuppressWarnings({"unchecked", "rawtypes"}) NativeReader>>> reader = @@ -931,4 +936,79 @@ public void testGetReaderProgressThrowing() { assertNull(progress.getRemainingParallelism()); logged.verifyWarn("remaining parallelism"); } + + @Test + public void testFailedWorkItemsAbort() throws Exception { + CounterSet counterSet = new CounterSet(); + StreamingModeExecutionStateRegistry executionStateRegistry = + new StreamingModeExecutionStateRegistry(null); + StreamingModeExecutionContext context = + new StreamingModeExecutionContext( + counterSet, + "computationId", + new ReaderCache(Duration.standardMinutes(1), Runnable::run), + /*stateNameMap=*/ ImmutableMap.of(), + new WindmillStateCache(options.getWorkerCacheMb()).forComputation("computationId"), + StreamingStepMetricsContainer.createRegistry(), + new DataflowExecutionStateTracker( + ExecutionStateSampler.newForTest(), + executionStateRegistry.getState( + NameContext.forStage("stageName"), "other", null, NoopProfileScope.NOOP), + counterSet, + PipelineOptionsFactory.create(), + "test-work-item-id"), + executionStateRegistry, + Long.MAX_VALUE); + + options.setNumWorkers(5); + int maxElements = 100; + DataflowPipelineDebugOptions debugOptions = options.as(DataflowPipelineDebugOptions.class); + debugOptions.setUnboundedReaderMaxElements(maxElements); + + ByteString state = ByteString.EMPTY; + Windmill.WorkItem workItem = + Windmill.WorkItem.newBuilder() + .setKey(ByteString.copyFromUtf8("0000000000000001")) // key is zero-padded index. + .setWorkToken(0) + .setCacheToken(1) + .setSourceState( + Windmill.SourceState.newBuilder().setState(state).build()) // Source state. + .build(); + Work dummyWork = Work.create(workItem, Instant::now, Collections.emptyList(), unused -> {}); + + context.start( + "key", + workItem, + new Instant(0), // input watermark + null, // output watermark + null, // synchronized processing time + null, // StateReader + null, // StateFetcher + Windmill.WorkItemCommitRequest.newBuilder(), + dummyWork::isFailed); + + @SuppressWarnings({"unchecked", "rawtypes"}) + NativeReader>>> reader = + (NativeReader) + WorkerCustomSources.create( + (CloudObject) + serializeToCloudSource(new TestCountingSource(Integer.MAX_VALUE), options) + .getSpec(), + options, + context); + + NativeReaderIterator>>> readerIterator = + reader.iterator(); + int numReads = 0; + while ((numReads == 0) ? readerIterator.start() : readerIterator.advance()) { + WindowedValue>> value = readerIterator.getCurrent(); + assertEquals(KV.of(0, numReads), value.getValue().getValue()); + numReads++; + // Fail the work item after reading two elements. + if (numReads == 2) { + dummyWork.setFailed(); + } + } + assertThat(numReads, equalTo(2)); + } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java index ea57f687fd95..de30fd0f8d5d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java @@ -36,7 +36,7 @@ import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; import org.apache.beam.runners.dataflow.worker.streaming.ActiveWorkState.ActivateWorkResult; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.HeartbeatRequest; import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -239,7 +239,7 @@ public void testInvalidateStuckCommits() { } @Test - public void testGetKeysToRefresh() { + public void testGetKeyHeartbeats() { Instant refreshDeadline = Instant.now(); Work freshWork = createWork(createWorkItem(3L)); @@ -254,47 +254,51 @@ public void testGetKeysToRefresh() { activeWorkState.activateWorkForKey(shardedKey1, freshWork); activeWorkState.activateWorkForKey(shardedKey2, refreshableWork2); - ImmutableList requests = - activeWorkState.getKeysToRefresh(refreshDeadline, DataflowExecutionStateSampler.instance()); + ImmutableList requests = + activeWorkState.getKeyHeartbeats(refreshDeadline, DataflowExecutionStateSampler.instance()); - ImmutableList expected = + ImmutableList expected = ImmutableList.of( - GetDataRequestKeyShardingKeyAndWorkToken.from(shardedKey1, refreshableWork1), - GetDataRequestKeyShardingKeyAndWorkToken.from(shardedKey2, refreshableWork2)); + HeartbeatRequestShardingKeyWorkTokenAndCacheToken.from(shardedKey1, refreshableWork1), + HeartbeatRequestShardingKeyWorkTokenAndCacheToken.from(shardedKey2, refreshableWork2)); - ImmutableList actual = + ImmutableList actual = requests.stream() - .map(GetDataRequestKeyShardingKeyAndWorkToken::from) + .map(HeartbeatRequestShardingKeyWorkTokenAndCacheToken::from) .collect(toImmutableList()); assertThat(actual).containsExactlyElementsIn(expected); } @AutoValue - abstract static class GetDataRequestKeyShardingKeyAndWorkToken { + abstract static class HeartbeatRequestShardingKeyWorkTokenAndCacheToken { - private static GetDataRequestKeyShardingKeyAndWorkToken create( - ByteString key, long shardingKey, long workToken) { - return new AutoValue_ActiveWorkStateTest_GetDataRequestKeyShardingKeyAndWorkToken( - key, shardingKey, workToken); + private static HeartbeatRequestShardingKeyWorkTokenAndCacheToken create( + long shardingKey, long workToken, long cacheToken) { + return new AutoValue_ActiveWorkStateTest_HeartbeatRequestShardingKeyWorkTokenAndCacheToken( + shardingKey, workToken, cacheToken); } - private static GetDataRequestKeyShardingKeyAndWorkToken from( - KeyedGetDataRequest keyedGetDataRequest) { + private static HeartbeatRequestShardingKeyWorkTokenAndCacheToken from( + HeartbeatRequest heartbeatRequest) { return create( - keyedGetDataRequest.getKey(), - keyedGetDataRequest.getShardingKey(), - keyedGetDataRequest.getWorkToken()); + heartbeatRequest.getShardingKey(), + heartbeatRequest.getWorkToken(), + heartbeatRequest.getCacheToken()); } - private static GetDataRequestKeyShardingKeyAndWorkToken from(ShardedKey shardedKey, Work work) { - return create(shardedKey.key(), shardedKey.shardingKey(), work.getWorkItem().getWorkToken()); + private static HeartbeatRequestShardingKeyWorkTokenAndCacheToken from( + ShardedKey shardedKey, Work work) { + return create( + shardedKey.shardingKey(), + work.getWorkItem().getWorkToken(), + work.getWorkItem().getCacheToken()); } - abstract ByteString key(); - abstract long shardingKey(); abstract long workToken(); + + abstract long cacheToken(); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java index 5f8a452a0433..0ea253027679 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java @@ -44,6 +44,7 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.CommitStatus; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationGetDataRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationHeartbeatRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationWorkItemMetadata; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkStreamTimingInfo; @@ -51,6 +52,7 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalData; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalDataId; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalDataRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.HeartbeatRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.JobHeader; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataResponse; @@ -126,7 +128,7 @@ public void setUp() throws Exception { .build() .start(); - this.client = GrpcWindmillServer.newTestInstance(name); + this.client = GrpcWindmillServer.newTestInstance(name, new ArrayList<>()); } @After @@ -744,7 +746,7 @@ public void onCompleted() { while (true) { Thread.sleep(100); int tmpErrorsBeforeClose = errorsBeforeClose.get(); - // wait for at least 1 errors before close + // wait for at least 1 error before close if (tmpErrorsBeforeClose > 0) { break; } @@ -765,7 +767,7 @@ public void onCompleted() { while (true) { Thread.sleep(100); int tmpErrorsAfterClose = errorsAfterClose.get(); - // wait for at least 1 errors after close + // wait for at least 1 error after close if (tmpErrorsAfterClose > 0) { break; } @@ -786,22 +788,36 @@ public void onCompleted() { assertTrue(stream.awaitTermination(30, TimeUnit.SECONDS)); } - private List makeHeartbeatRequest(List keys) { + private List makeGetDataHeartbeatRequest(List keys) { List result = new ArrayList<>(); for (String key : keys) { result.add( Windmill.KeyedGetDataRequest.newBuilder() - .setKey(ByteString.copyFromUtf8(key)) + .setShardingKey(key.hashCode()) .setWorkToken(0) + .setCacheToken(0) + .build()); + } + return result; + } + + private List makeHeartbeatRequest(List keys) { + List result = new ArrayList<>(); + for (String key : keys) { + result.add( + Windmill.HeartbeatRequest.newBuilder() + .setShardingKey(key.hashCode()) + .setWorkToken(0) + .setCacheToken(0) .build()); } return result; } @Test - public void testStreamingGetDataHeartbeats() throws Exception { + public void testStreamingGetDataHeartbeatsAsKeyedGetDataRequests() throws Exception { // This server records the heartbeats observed but doesn't respond. - final Map> heartbeats = new HashMap<>(); + final Map> getDataHeartbeats = new HashMap<>(); serviceRegistry.addService( new CloudWindmillServiceV1Alpha1ImplBase() { @@ -826,16 +842,17 @@ public void onNext(StreamingGetDataRequest chunk) { .build())); sawHeader = true; } else { - LOG.info("Received {} heartbeats", chunk.getStateRequestCount()); + LOG.info("Received {} getDataHeartbeats", chunk.getStateRequestCount()); errorCollector.checkThat( chunk.getSerializedSize(), Matchers.lessThanOrEqualTo(STREAM_CHUNK_SIZE)); errorCollector.checkThat(chunk.getRequestIdCount(), Matchers.is(0)); - synchronized (heartbeats) { + synchronized (getDataHeartbeats) { for (ComputationGetDataRequest request : chunk.getStateRequestList()) { errorCollector.checkThat(request.getRequestsCount(), Matchers.is(1)); - heartbeats.putIfAbsent(request.getComputationId(), new ArrayList<>()); - heartbeats + getDataHeartbeats.putIfAbsent( + request.getComputationId(), new ArrayList<>()); + getDataHeartbeats .get(request.getComputationId()) .add(request.getRequestsList().get(0)); } @@ -857,7 +874,6 @@ public void onCompleted() { } }); - Map> activeMap = new HashMap<>(); List computation1Keys = new ArrayList<>(); List computation2Keys = new ArrayList<>(); @@ -865,22 +881,141 @@ public void onCompleted() { computation1Keys.add("Computation1Key" + i); computation2Keys.add("Computation2Key" + largeString(i * 20)); } - activeMap.put("Computation1", makeHeartbeatRequest(computation1Keys)); - activeMap.put("Computation2", makeHeartbeatRequest(computation2Keys)); + // We're adding HeartbeatRequests to refreshActiveWork, but expecting to get back + // KeyedGetDataRequests, so make a Map of both types. + Map> expectedKeyedGetDataRequests = new HashMap<>(); + expectedKeyedGetDataRequests.put("Computation1", makeGetDataHeartbeatRequest(computation1Keys)); + expectedKeyedGetDataRequests.put("Computation2", makeGetDataHeartbeatRequest(computation2Keys)); + Map> heartbeatsToRefresh = new HashMap<>(); + heartbeatsToRefresh.put("Computation1", makeHeartbeatRequest(computation1Keys)); + heartbeatsToRefresh.put("Computation2", makeHeartbeatRequest(computation2Keys)); GetDataStream stream = client.getDataStream(); - stream.refreshActiveWork(activeMap); + stream.refreshActiveWork(heartbeatsToRefresh); stream.close(); assertTrue(stream.awaitTermination(60, TimeUnit.SECONDS)); - while (true) { + boolean receivedAllGetDataHeartbeats = false; + while (!receivedAllGetDataHeartbeats) { Thread.sleep(100); - synchronized (heartbeats) { - if (heartbeats.size() != activeMap.size()) { + synchronized (getDataHeartbeats) { + if (getDataHeartbeats.size() != expectedKeyedGetDataRequests.size()) { continue; } - assertEquals(heartbeats, activeMap); - break; + assertEquals(expectedKeyedGetDataRequests, getDataHeartbeats); + receivedAllGetDataHeartbeats = true; + } + } + } + + @Test + public void testStreamingGetDataHeartbeatsAsHeartbeatRequests() throws Exception { + // Create a client and server different from the one in SetUp so we can add an experiment to the + // options passed in. + this.server = + InProcessServerBuilder.forName("TestServer") + .fallbackHandlerRegistry(serviceRegistry) + .executor(Executors.newFixedThreadPool(1)) + .build() + .start(); + this.client = + GrpcWindmillServer.newTestInstance( + "TestServer", + Collections.singletonList("streaming_engine_send_new_heartbeat_requests")); + // This server records the heartbeats observed but doesn't respond. + final List receivedHeartbeats = new ArrayList<>(); + + serviceRegistry.addService( + new CloudWindmillServiceV1Alpha1ImplBase() { + @Override + public StreamObserver getDataStream( + StreamObserver responseObserver) { + return new StreamObserver() { + boolean sawHeader = false; + + @Override + public void onNext(StreamingGetDataRequest chunk) { + try { + if (!sawHeader) { + LOG.info("Received header"); + errorCollector.checkThat( + chunk.getHeader(), + Matchers.equalTo( + JobHeader.newBuilder() + .setJobId("job") + .setProjectId("project") + .setWorkerId("worker") + .build())); + sawHeader = true; + } else { + LOG.info( + "Received {} computationHeartbeatRequests", + chunk.getComputationHeartbeatRequestCount()); + errorCollector.checkThat( + chunk.getSerializedSize(), Matchers.lessThanOrEqualTo(STREAM_CHUNK_SIZE)); + errorCollector.checkThat(chunk.getRequestIdCount(), Matchers.is(0)); + + synchronized (receivedHeartbeats) { + receivedHeartbeats.addAll(chunk.getComputationHeartbeatRequestList()); + } + } + } catch (Exception e) { + errorCollector.addError(e); + } + } + + @Override + public void onError(Throwable throwable) {} + + @Override + public void onCompleted() { + responseObserver.onCompleted(); + } + }; + } + }); + + List computation1Keys = new ArrayList<>(); + List computation2Keys = new ArrayList<>(); + + // When sending heartbeats as HeartbeatRequest protos, all keys for the same computation should + // be batched into the same ComputationHeartbeatRequest. Compare to the KeyedGetDataRequest + // version in the test above, which only sends one key per ComputationGetDataRequest. + List expectedHeartbeats = new ArrayList<>(); + ComputationHeartbeatRequest.Builder comp1Builder = + ComputationHeartbeatRequest.newBuilder().setComputationId("Computation1"); + ComputationHeartbeatRequest.Builder comp2Builder = + ComputationHeartbeatRequest.newBuilder().setComputationId("Computation2"); + for (int i = 0; i < 100; ++i) { + String computation1Key = "Computation1Key" + i; + computation1Keys.add(computation1Key); + comp1Builder.addHeartbeatRequests( + makeHeartbeatRequest(Collections.singletonList(computation1Key)).get(0)); + String computation2Key = "Computation2Key" + largeString(i * 20); + computation2Keys.add(computation2Key); + comp2Builder.addHeartbeatRequests( + makeHeartbeatRequest(Collections.singletonList(computation2Key)).get(0)); + } + expectedHeartbeats.add(comp1Builder.build()); + expectedHeartbeats.add(comp2Builder.build()); + Map> heartbeatRequestMap = new HashMap<>(); + heartbeatRequestMap.put("Computation1", makeHeartbeatRequest(computation1Keys)); + heartbeatRequestMap.put("Computation2", makeHeartbeatRequest(computation2Keys)); + + GetDataStream stream = client.getDataStream(); + stream.refreshActiveWork(heartbeatRequestMap); + stream.close(); + assertTrue(stream.awaitTermination(60, TimeUnit.SECONDS)); + + boolean receivedAllHeartbeatRequests = false; + while (!receivedAllHeartbeatRequests) { + Thread.sleep(100); + synchronized (receivedHeartbeats) { + if (receivedHeartbeats.size() != expectedHeartbeats.size()) { + continue; + } + assertEquals(expectedHeartbeats, receivedHeartbeats); + receivedAllHeartbeatRequests = true; } } } @@ -888,7 +1023,7 @@ public void onCompleted() { @Test public void testThrottleSignal() throws Exception { // This server responds with work items until the throttleMessage limit is hit at which point it - // returns RESROUCE_EXHAUSTED errors for throttleTime msecs after which it resumes sending + // returns RESOURCE_EXHAUSTED errors for throttleTime msecs after which it resumes sending // work items. final int throttleTime = 2000; final int throttleMessage = 15; diff --git a/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill.proto b/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill.proto index 6aaeb57001e0..0c824ca301b3 100644 --- a/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill.proto +++ b/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill.proto @@ -477,9 +477,10 @@ message GetWorkResponse { // GetData message KeyedGetDataRequest { - required bytes key = 1; + optional bytes key = 1; required fixed64 work_token = 2; optional fixed64 sharding_key = 6; + optional fixed64 cache_token = 11; repeated TagValue values_to_fetch = 3; repeated TagValuePrefixRequest tag_value_prefixes_to_fetch = 10; repeated TagBag bags_to_fetch = 8; @@ -507,6 +508,8 @@ message GetDataRequest { // Assigned worker id for the instance. optional string worker_id = 6; + // SE only. Will only be set by compatible client + repeated ComputationHeartbeatRequest computation_heartbeat_request = 7; // DEPRECATED repeated GlobalDataId global_data_to_fetch = 2; } @@ -536,6 +539,44 @@ message ComputationGetDataResponse { message GetDataResponse { repeated ComputationGetDataResponse data = 1; repeated GlobalData global_data = 2; + // Only set if ComputationHeartbeatRequest was sent, prior versions do not + // expect a response for heartbeats. SE only. + repeated ComputationHeartbeatResponse computation_heartbeat_response = 3; +} + +// Heartbeats +// +// Heartbeats are sent over the GetData stream in Streaming Engine and +// indicates the work item that the user worker has previously received from +// GetWork but not yet committed with CommitWork. +// Note that implicit heartbeats not expecting a response may be sent as +// special KeyedGetDataRequests see function KeyedGetDataRequestIsHeartbeat. +// SE only. +message HeartbeatRequest { + optional fixed64 sharding_key = 1; + optional fixed64 work_token = 2; + optional fixed64 cache_token = 3; + repeated LatencyAttribution latency_attribution = 4; +} + +// Responses for heartbeat requests, indicating which work is no longer valid +// on the windmill worker and may be dropped/cancelled in the client. +// SE only. +message HeartbeatResponse { + optional fixed64 sharding_key = 1; + optional fixed64 work_token = 2; + optional fixed64 cache_token = 3; + optional bool failed = 4; +} + +message ComputationHeartbeatRequest { + optional string computation_id = 1; + repeated HeartbeatRequest heartbeat_requests = 2; +} + +message ComputationHeartbeatResponse { + optional string computation_id = 1; + repeated HeartbeatResponse heartbeat_responses = 2; } // CommitWork @@ -772,6 +813,8 @@ message StreamingGetDataRequest { repeated fixed64 request_id = 1; repeated GlobalDataRequest global_data_request = 3; repeated ComputationGetDataRequest state_request = 4; + // Will only be set by compatible client + repeated ComputationHeartbeatRequest computation_heartbeat_request = 5; } message StreamingGetDataResponse { @@ -784,6 +827,12 @@ message StreamingGetDataResponse { repeated bytes serialized_response = 2; // Remaining bytes field applies only to the last serialized_response optional int64 remaining_bytes_for_response = 3; + + // Only set if ComputationHeartbeatRequest was sent, prior versions do not + // expect a response for heartbeats. + repeated ComputationHeartbeatResponse computation_heartbeat_response = 5; + + reserved 4; } message StreamingCommitWorkRequest { From 0a813b996572191bc179c724d3f31d535eb45e7d Mon Sep 17 00:00:00 2001 From: Edward Cheng Date: Tue, 23 Jan 2024 05:05:23 -0500 Subject: [PATCH 047/169] Plumbing remaining autoscaling metrics (#30070) OutstandingBytes and MaximumOutstandingBytes --- .../groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 2 +- .../runners/dataflow/worker/StreamingDataflowWorker.java | 4 +++- .../dataflow/worker/DataflowWorkUnitClientTest.java | 8 +++++++- 3 files changed, 11 insertions(+), 3 deletions(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 41bcc70fe532..51b3cbbf178f 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -734,7 +734,7 @@ class BeamModulePlugin implements Plugin { google_api_services_bigquery : "com.google.apis:google-api-services-bigquery:v2-rev20230812-$google_clients_version", // Keep version consistent with the version in google_cloud_resourcemanager, managed by google_cloud_platform_libraries_bom google_api_services_cloudresourcemanager : "com.google.apis:google-api-services-cloudresourcemanager:v1-rev20230806-$google_clients_version", - google_api_services_dataflow : "com.google.apis:google-api-services-dataflow:v1b3-rev20231203-$google_clients_version", + google_api_services_dataflow : "com.google.apis:google-api-services-dataflow:v1b3-rev20240113-$google_clients_version", google_api_services_healthcare : "com.google.apis:google-api-services-healthcare:v1-rev20240110-$google_clients_version", google_api_services_pubsub : "com.google.apis:google-api-services-pubsub:v1-rev20220904-$google_clients_version", // Keep version consistent with the version in google_cloud_nio, managed by google_cloud_platform_libraries_bom diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index a95e78288819..1ad0ed08c081 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -1782,8 +1782,10 @@ private void sendWorkerMessage() throws IOException { new StreamingScalingReport() .setActiveThreadCount(workUnitExecutor.activeCount()) .setActiveBundleCount(workUnitExecutor.elementsOutstanding()) + .setOutstandingBytes(workUnitExecutor.bytesOutstanding()) .setMaximumThreadCount(chooseMaximumNumberOfThreads()) - .setMaximumBundleCount(workUnitExecutor.maximumElementsOutstanding()); + .setMaximumBundleCount(workUnitExecutor.maximumElementsOutstanding()) + .setMaximumBytes(workUnitExecutor.maximumBytesOutstanding()); workUnitClient.reportWorkerMessage( workUnitClient.createWorkerMessageFromStreamingScalingReport(activeThreadsReport)); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java index 7720de3563b8..7019217b9fbd 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java @@ -240,7 +240,13 @@ public void testReportWorkerMessage() throws Exception { response.setContent(workerMessage.toPrettyString()); when(request.execute()).thenReturn(response); StreamingScalingReport activeThreadsReport = - new StreamingScalingReport().setActiveThreadCount(1); + new StreamingScalingReport() + .setActiveThreadCount(1) + .setActiveBundleCount(2) + .setOutstandingBytes(3L) + .setMaximumThreadCount(4) + .setMaximumBundleCount(5) + .setMaximumBytes(6L); WorkUnitClient client = new DataflowWorkUnitClient(pipelineOptions, LOG); WorkerMessage msg = client.createWorkerMessageFromStreamingScalingReport(activeThreadsReport); client.reportWorkerMessage(msg); From 93339bcf0612681cb5d421e90e0e97f6d5a42ce4 Mon Sep 17 00:00:00 2001 From: Chamikara Jayalath Date: Tue, 23 Jan 2024 10:28:46 -0800 Subject: [PATCH 048/169] Fixes an upgrade compatibility breakage for the BQ write transform (#30032) * Fixes an upgrade compatibility breakage for the BQ write transform * Addressing reviewer comments * Resolve a conflcit --- .../construction/PTransformTranslation.java | 3 +- .../construction/PipelineTranslation.java | 7 ++-- .../core/construction/TransformUpgrader.java | 35 +++++++++++++++---- .../construction/TransformUpgraderTest.java | 3 +- .../expansion/service/ExpansionService.java | 12 ++++--- ...pansionServiceSchemaTransformProvider.java | 3 +- .../JavaClassLookupTransformProvider.java | 3 +- ...ionServiceSchemaTransformProviderTest.java | 8 +++-- .../service/ExpansionServiceTest.java | 2 +- .../sdk/expansion/service/ExternalTest.java | 30 ++++++++-------- .../gcp/bigquery/BigQueryIOTranslation.java | 26 ++++++++++++-- .../bigquery/BigQueryIOTranslationTest.java | 14 ++++++-- .../io/kafka/upgrade/KafkaIOTranslation.java | 5 +-- .../kafka/upgrade/KafkaIOTranslationTest.java | 5 +-- .../expansion/TestExpansionService.java | 5 +-- 15 files changed, 111 insertions(+), 50 deletions(-) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java index 6829e0d6b232..6bdd0fc37394 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java @@ -40,6 +40,7 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.schemas.SchemaTranslation; import org.apache.beam.sdk.transforms.PTransform; @@ -609,7 +610,7 @@ default Row toConfigRow(T transform) { * {@link #toConfigRow(PTransform)} method. * @return a transform represented by the current {@code TransformPayloadTranslator}. */ - default T fromConfigRow(Row configRow) { + default T fromConfigRow(Row configRow, PipelineOptions options) { throw new UnsupportedOperationException("Not implemented"); } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineTranslation.java index 4433b4b0475d..688d7a808647 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineTranslation.java @@ -111,14 +111,13 @@ public void visitPrimitiveTransform(Node node) { res = elideDeprecatedViews(res); } - ExternalTranslationOptions externalTranslationOptions = - pipeline.getOptions().as(ExternalTranslationOptions.class); - List urnsToOverride = externalTranslationOptions.getTransformsToOverride(); + List urnsToOverride = + pipeline.getOptions().as(ExternalTranslationOptions.class).getTransformsToOverride(); if (urnsToOverride.size() > 0 && upgradeTransforms) { try (TransformUpgrader upgrader = TransformUpgrader.of()) { res = upgrader.upgradeTransformsViaTransformService( - res, urnsToOverride, externalTranslationOptions); + res, urnsToOverride, pipeline.getOptions()); } catch (Exception e) { throw new RuntimeException( "Could not override the transforms with URNs " + urnsToOverride, e); diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java index 4f1a02165d23..f07df6052155 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java @@ -41,8 +41,11 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.SchemaApi; import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.StreamingOptions; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transformservice.launcher.TransformServiceLauncher; +import org.apache.beam.sdk.util.ReleaseInfo; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; @@ -97,7 +100,7 @@ static TransformUpgrader of(ExpansionServiceClientFactory clientFactory) { * @throws Exception */ public RunnerApi.Pipeline upgradeTransformsViaTransformService( - RunnerApi.Pipeline pipeline, List urnsToOverride, ExternalTranslationOptions options) + RunnerApi.Pipeline pipeline, List urnsToOverride, PipelineOptions options) throws IOException, TimeoutException { List transformsToOverride = pipeline.getComponents().getTransformsMap().entrySet().stream() @@ -127,13 +130,15 @@ public RunnerApi.Pipeline upgradeTransformsViaTransformService( String serviceAddress; TransformServiceLauncher service = null; - if (options.getTransformServiceAddress() != null) { - serviceAddress = options.getTransformServiceAddress(); - } else if (options.getTransformServiceBeamVersion() != null) { + ExternalTranslationOptions externalTranslationOptions = + options.as(ExternalTranslationOptions.class); + if (externalTranslationOptions.getTransformServiceAddress() != null) { + serviceAddress = externalTranslationOptions.getTransformServiceAddress(); + } else if (externalTranslationOptions.getTransformServiceBeamVersion() != null) { String projectName = UUID.randomUUID().toString(); int port = findAvailablePort(); service = TransformServiceLauncher.forProject(projectName, port, null); - service.setBeamVersion(options.getTransformServiceBeamVersion()); + service.setBeamVersion(externalTranslationOptions.getTransformServiceBeamVersion()); // Starting the transform service. service.start(); @@ -169,7 +174,7 @@ RunnerApi.Pipeline updateTransformViaTransformService( RunnerApi.Pipeline runnerAPIpipeline, String transformId, Endpoints.ApiServiceDescriptor transformServiceEndpoint, - ExternalTranslationOptions options) + PipelineOptions options) throws IOException { RunnerApi.PTransform transformToUpgrade = runnerAPIpipeline.getComponents().getTransformsMap().get(transformId); @@ -207,11 +212,26 @@ RunnerApi.Pipeline updateTransformViaTransformService( ExpansionApi.ExpansionRequest.Builder requestBuilder = ExpansionApi.ExpansionRequest.newBuilder(); + + // Creating a clone here so that we can set properties without modifying the original + // PipelineOptions object. + PipelineOptions optionsClone = + PipelineOptionsTranslation.fromProto(PipelineOptionsTranslation.toProto(options)); + String updateCompatibilityVersion = + optionsClone.as(StreamingOptions.class).getUpdateCompatibilityVersion(); + if (updateCompatibilityVersion == null || updateCompatibilityVersion.isEmpty()) { + // Setting the option 'updateCompatibilityVersion' to the current SDK version so that the + // TransformService uses a compatible schema. + optionsClone + .as(StreamingOptions.class) + .setUpdateCompatibilityVersion(ReleaseInfo.getReleaseInfo().getSdkVersion()); + } ExpansionApi.ExpansionRequest request = requestBuilder .setComponents(runnerAPIpipeline.getComponents()) .setTransform(ptransformBuilder.build()) .setNamespace(UPGRADE_NAMESPACE) + .setPipelineOptions(PipelineOptionsTranslation.toProto(optionsClone)) .addAllRequirements(runnerAPIpipeline.getRequirementsList()) .build(); @@ -242,7 +262,8 @@ RunnerApi.Pipeline updateTransformViaTransformService( // Adds an annotation that denotes the Beam version the transform was upgraded to. RunnerApi.PTransform.Builder expandedTransformBuilder = expandedTransform.toBuilder(); - String transformServiceVersion = options.getTransformServiceBeamVersion(); + String transformServiceVersion = + options.as(ExternalTranslationOptions.class).getTransformServiceBeamVersion(); if (transformServiceVersion == null || transformServiceVersion.isEmpty()) { transformServiceVersion = "unknown"; } diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformUpgraderTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformUpgraderTest.java index e14fa556dd90..2b01bf702465 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformUpgraderTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformUpgraderTest.java @@ -35,6 +35,7 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.schemas.Schema; @@ -93,7 +94,7 @@ public String getUrn() { } @Override - public TestTransform fromConfigRow(Row configRow) { + public TestTransform fromConfigRow(Row configRow, PipelineOptions options) { return new TestTransform(configRow.getInt32("multiplier")); } 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 7760cab64acc..5d46100fe656 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 @@ -152,7 +152,8 @@ public Map knownTransforms() { TransformProvider transformProvider = new TransformProvider() { @Override - public PTransform getTransform(RunnerApi.FunctionSpec spec) { + public PTransform getTransform( + RunnerApi.FunctionSpec spec, PipelineOptions options) { try { Class configClass = getConfigClass(builderInstance); return builderInstance.buildExternal( @@ -222,14 +223,14 @@ public List getDependencies( } final String finalUrn = urn; TransformProvider transformProvider = - spec -> { + (spec, options) -> { try { ExternalConfigurationPayload payload = ExternalConfigurationPayload.parseFrom(spec.getPayload()); Row configRow = RowCoder.of(SchemaTranslation.schemaFromProto(payload.getSchema())) .decode(new ByteArrayInputStream(payload.getPayload().toByteArray())); - PTransform transformFromRow = translator.fromConfigRow(configRow); + PTransform transformFromRow = translator.fromConfigRow(configRow, options); if (transformFromRow != null) { return transformFromRow; } else { @@ -441,7 +442,7 @@ default InputT createInput(Pipeline p, Map> inputs) { } } - PTransform getTransform(RunnerApi.FunctionSpec spec); + PTransform getTransform(RunnerApi.FunctionSpec spec, PipelineOptions options); default Map> extractOutputs(OutputT output) { if (output instanceof PDone) { @@ -485,7 +486,8 @@ default Map> extractOutputs(OutputT output) { default Map> apply( Pipeline p, String name, RunnerApi.FunctionSpec spec, Map> inputs) { return extractOutputs( - Pipeline.applyTransform(name, createInput(p, inputs), getTransform(spec))); + Pipeline.applyTransform( + name, createInput(p, inputs), getTransform(spec, p.getOptions()))); } default String getTransformUniqueID(RunnerApi.FunctionSpec spec) { diff --git a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java index ead1fa67dc98..0bd85fb79e64 100644 --- a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java +++ b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java @@ -29,6 +29,7 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.expansion.service.ExpansionService.TransformProvider; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.SchemaTranslation; import org.apache.beam.sdk.transforms.PTransform; @@ -91,7 +92,7 @@ public Map> extractOutputs(PCollectionRowTuple output) { } @Override - public PTransform getTransform(FunctionSpec spec) { + public PTransform getTransform(FunctionSpec spec, PipelineOptions options) { SchemaTransformPayload payload; try { payload = SchemaTransformPayload.parseFrom(spec.getPayload()); diff --git a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/JavaClassLookupTransformProvider.java b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/JavaClassLookupTransformProvider.java index 9f982f0cd01f..96697c070bec 100644 --- a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/JavaClassLookupTransformProvider.java +++ b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/JavaClassLookupTransformProvider.java @@ -46,6 +46,7 @@ import org.apache.beam.repackaged.core.org.apache.commons.lang3.ClassUtils; import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.expansion.service.ExpansionService.TransformProvider; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.schemas.JavaFieldSchema; import org.apache.beam.sdk.schemas.NoSuchSchemaException; import org.apache.beam.sdk.schemas.Schema; @@ -90,7 +91,7 @@ public JavaClassLookupTransformProvider(AllowList allowList) { @SuppressWarnings("argument") @Override - public PTransform getTransform(FunctionSpec spec) { + public PTransform getTransform(FunctionSpec spec, PipelineOptions options) { JavaClassLookupPayload payload; try { payload = JavaClassLookupPayload.parseFrom(spec.getPayload()); diff --git a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProviderTest.java b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProviderTest.java index 696fed0f8ff4..3e6451b131d3 100644 --- a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProviderTest.java +++ b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProviderTest.java @@ -34,6 +34,7 @@ import org.apache.beam.runners.core.construction.PipelineTranslation; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.schemas.JavaFieldSchema; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.Field; @@ -441,10 +442,13 @@ public void testSchematransformEquivalentConfigSchema() throws CoderException { assertNotEquals(spec.getPayload(), equivalentSpec.getPayload()); TestSchemaTransform transform = - (TestSchemaTransform) ExpansionServiceSchemaTransformProvider.of().getTransform(spec); + (TestSchemaTransform) + ExpansionServiceSchemaTransformProvider.of() + .getTransform(spec, PipelineOptionsFactory.create()); TestSchemaTransform equivalentTransform = (TestSchemaTransform) - ExpansionServiceSchemaTransformProvider.of().getTransform(equivalentSpec); + ExpansionServiceSchemaTransformProvider.of() + .getTransform(equivalentSpec, PipelineOptionsFactory.create()); assertEquals(transform.int1, equivalentTransform.int1); assertEquals(transform.int2, equivalentTransform.int2); diff --git a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceTest.java b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceTest.java index 618fa3333092..b79d91bf6284 100644 --- a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceTest.java +++ b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceTest.java @@ -94,7 +94,7 @@ public static class TestTransformRegistrar implements ExpansionService.Expansion @Override public Map knownTransforms() { - return ImmutableMap.of(TEST_URN, spec -> Count.perElement()); + return ImmutableMap.of(TEST_URN, (spec, options) -> Count.perElement()); } } diff --git a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExternalTest.java b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExternalTest.java index 4b949a597f0a..d2363559473e 100644 --- a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExternalTest.java +++ b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExternalTest.java @@ -167,23 +167,23 @@ public static class TestTransforms public Map knownTransforms() { return ImmutableMap.of( TEST_URN_SIMPLE, - spec -> MapElements.into(TypeDescriptors.strings()).via((String x) -> x + x), + (spec, options) -> MapElements.into(TypeDescriptors.strings()).via((String x) -> x + x), TEST_URN_LE, - spec -> Filter.lessThanEq(Integer.parseInt(spec.getPayload().toStringUtf8())), + (spec, options) -> Filter.lessThanEq(Integer.parseInt(spec.getPayload().toStringUtf8())), TEST_URN_MULTI, - spec -> - ParDo.of( - new DoFn() { - @ProcessElement - public void processElement(ProcessContext c) { - if (c.element() % 2 == 0) { - c.output(c.element()); - } else { - c.output(odd, c.element()); - } - } - }) - .withOutputTags(even, TupleTagList.of(odd))); + (spec, options) -> + ParDo.of( + new DoFn() { + @ProcessElement + public void processElement(ProcessContext c) { + if (c.element() % 2 == 0) { + c.output(c.element()); + } else { + c.output(odd, c.element()); + } + } + }) + .withOutputTags(even, TupleTagList.of(odd))); } } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java index a3a270a315b2..b2d533f69fb3 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java @@ -50,6 +50,8 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.SchemaUpdateOption; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; import org.apache.beam.sdk.io.gcp.bigquery.RowWriterFactory.AvroRowWriterFactory; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.StreamingOptions; import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.schemas.Schema; @@ -188,7 +190,7 @@ public Row toConfigRow(TypedRead transform) { } @Override - public TypedRead fromConfigRow(Row configRow) { + public TypedRead fromConfigRow(Row configRow, PipelineOptions options) { try { BigQueryIO.TypedRead.Builder builder = new AutoValue_BigQueryIO_TypedRead.Builder<>(); @@ -552,7 +554,7 @@ public Row toConfigRow(Write transform) { } @Override - public Write fromConfigRow(Row configRow) { + public Write fromConfigRow(Row configRow, PipelineOptions options) { try { BigQueryIO.Write.Builder builder = new AutoValue_BigQueryIO_Write.Builder<>(); @@ -695,7 +697,25 @@ public Write fromConfigRow(Row configRow) { if (maxBytesPerPartition != null) { builder = builder.setMaxBytesPerPartition(maxBytesPerPartition); } - Duration triggeringFrequency = configRow.getValue("triggering_frequency"); + + String updateCompatibilityBeamVersion = + options.as(StreamingOptions.class).getUpdateCompatibilityVersion(); + + // We need to update the 'triggerring_frequency' field name for pipelines that are upgraded + // from Beam 2.53.0 due to https://github.com/apache/beam/pull/29785. + // We need to set a default 'updateCompatibilityBeamVersion' here since this PipelineOption + // is not correctly passed in for pipelines that use Beam 2.53.0. + // Both above issues are fixed for Beam 2.54.0 and later. + updateCompatibilityBeamVersion = + (updateCompatibilityBeamVersion != null) ? updateCompatibilityBeamVersion : "2.53.0"; + + String triggeringFrequencyFieldName = + (updateCompatibilityBeamVersion != null + && updateCompatibilityBeamVersion.equals("2.53.0")) + ? "triggerring_frequency" + : "triggering_frequency"; + + Duration triggeringFrequency = configRow.getValue(triggeringFrequencyFieldName); if (triggeringFrequency != null) { builder = builder.setTriggeringFrequency( diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslationTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslationTest.java index c46d382bb298..668f4eef4d83 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslationTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslationTest.java @@ -31,6 +31,9 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.options.StreamingOptions; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -142,7 +145,8 @@ public void testReCreateReadTransformFromRowTable() { Row row = translator.toConfigRow(readTransform); BigQueryIO.TypedRead readTransformFromRow = - (BigQueryIO.TypedRead) translator.fromConfigRow(row); + (BigQueryIO.TypedRead) + translator.fromConfigRow(row, PipelineOptionsFactory.create()); assertNotNull(readTransformFromRow.getTable()); assertEquals("dummyproject", readTransformFromRow.getTable().getProjectId()); assertEquals("dummydataset", readTransformFromRow.getTable().getDatasetId()); @@ -172,7 +176,8 @@ public void testReCreateReadTransformFromRowQuery() { new BigQueryIOTranslation.BigQueryIOReadTranslator(); Row row = translator.toConfigRow(readTransform); - BigQueryIO.TypedRead readTransformFromRow = translator.fromConfigRow(row); + BigQueryIO.TypedRead readTransformFromRow = + translator.fromConfigRow(row, PipelineOptionsFactory.create()); assertEquals("dummyquery", readTransformFromRow.getQuery().get()); assertNotNull(readTransformFromRow.getParseFn()); assertTrue(readTransformFromRow.getParseFn() instanceof DummyParseFn); @@ -241,7 +246,10 @@ public void testReCreateWriteTransformFromRowTable() { new BigQueryIOTranslation.BigQueryIOWriteTranslator(); Row row = translator.toConfigRow(writeTransform); - BigQueryIO.Write writeTransformFromRow = (BigQueryIO.Write) translator.fromConfigRow(row); + PipelineOptions options = PipelineOptionsFactory.create(); + options.as(StreamingOptions.class).setUpdateCompatibilityVersion("2.54.0"); + BigQueryIO.Write writeTransformFromRow = + (BigQueryIO.Write) translator.fromConfigRow(row, options); assertNotNull(writeTransformFromRow.getTable()); assertEquals("dummyproject", writeTransformFromRow.getTable().get().getProjectId()); assertEquals("dummydataset", writeTransformFromRow.getTable().get().getDatasetId()); diff --git a/sdks/java/io/kafka/upgrade/src/main/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslation.java b/sdks/java/io/kafka/upgrade/src/main/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslation.java index eedd2282b1f7..a76507a285ba 100644 --- a/sdks/java/io/kafka/upgrade/src/main/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslation.java +++ b/sdks/java/io/kafka/upgrade/src/main/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslation.java @@ -42,6 +42,7 @@ import org.apache.beam.sdk.io.kafka.KafkaIO.WriteRecords; import org.apache.beam.sdk.io.kafka.KafkaIOUtils; import org.apache.beam.sdk.io.kafka.TimestampPolicyFactory; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.FieldType; @@ -215,7 +216,7 @@ public Row toConfigRow(Read transform) { } @Override - public Read fromConfigRow(Row configRow) { + public Read fromConfigRow(Row configRow, PipelineOptions options) { try { Read transform = KafkaIO.read(); @@ -511,7 +512,7 @@ public Row toConfigRow(Write transform) { } @Override - public Write fromConfigRow(Row configRow) { + public Write fromConfigRow(Row configRow, PipelineOptions options) { try { Write transform = KafkaIO.write(); diff --git a/sdks/java/io/kafka/upgrade/src/test/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslationTest.java b/sdks/java/io/kafka/upgrade/src/test/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslationTest.java index be54d7830d5e..a94491d8513b 100644 --- a/sdks/java/io/kafka/upgrade/src/test/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslationTest.java +++ b/sdks/java/io/kafka/upgrade/src/test/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslationTest.java @@ -34,6 +34,7 @@ import org.apache.beam.sdk.io.kafka.KafkaIO.WriteRecords; import org.apache.beam.sdk.io.kafka.upgrade.KafkaIOTranslation.KafkaIOReadWithMetadataTranslator; import org.apache.beam.sdk.io.kafka.upgrade.KafkaIOTranslation.KafkaIOWriteTranslator; +import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.kafka.clients.consumer.ConsumerConfig; @@ -111,7 +112,7 @@ public void testReCreateReadTransformFromRow() throws Exception { Row row = translator.toConfigRow(readTransform); Read readTransformFromRow = - (Read) translator.fromConfigRow(row); + (Read) translator.fromConfigRow(row, PipelineOptionsFactory.create()); assertNotNull( readTransformFromRow.getConsumerConfig().get(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)); assertEquals( @@ -178,7 +179,7 @@ public void testReCreateWriteTransformFromRow() throws Exception { Row row = translator.toConfigRow(writeTransform); Write writeTransformFromRow = - (Write) translator.fromConfigRow(row); + (Write) translator.fromConfigRow(row, PipelineOptionsFactory.create()); WriteRecords writeRecordsTransform = writeTransformFromRow.getWriteRecordsTransform(); assertNotNull( diff --git a/sdks/java/testing/expansion-service/src/test/java/org/apache/beam/sdk/testing/expansion/TestExpansionService.java b/sdks/java/testing/expansion-service/src/test/java/org/apache/beam/sdk/testing/expansion/TestExpansionService.java index b45c922eae7d..9becaf980b8b 100644 --- a/sdks/java/testing/expansion-service/src/test/java/org/apache/beam/sdk/testing/expansion/TestExpansionService.java +++ b/sdks/java/testing/expansion-service/src/test/java/org/apache/beam/sdk/testing/expansion/TestExpansionService.java @@ -32,6 +32,7 @@ import org.apache.beam.sdk.io.FileIO; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.parquet.ParquetIO; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ExternalTransformBuilder; @@ -137,7 +138,7 @@ public KeyedPCollectionTuple createInput( @Override public PTransform, PCollection>>> - getTransform(RunnerApi.FunctionSpec spec) { + getTransform(RunnerApi.FunctionSpec spec, PipelineOptions options) { return new TestCoGroupByKeyTransform(); } } @@ -155,7 +156,7 @@ public PCollectionList createInput(Pipeline p, Map> @Override public PTransform, PCollection> getTransform( - RunnerApi.FunctionSpec spec) { + RunnerApi.FunctionSpec spec, PipelineOptions options) { return Flatten.pCollections(); } } From e8e0ccfdd7f52d466f05ae43dfde2588a6bdc621 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 23 Jan 2024 13:59:37 -0500 Subject: [PATCH 049/169] Bump pillow (#30076) Bumps [pillow](https://github.com/python-pillow/Pillow) from 9.2.0 to 10.2.0. - [Release notes](https://github.com/python-pillow/Pillow/releases) - [Changelog](https://github.com/python-pillow/Pillow/blob/main/CHANGES.rst) - [Commits](https://github.com/python-pillow/Pillow/compare/9.2.0...10.2.0) --- updated-dependencies: - dependency-name: pillow dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .../ml-orchestration/kfp/components/train/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/examples/ml-orchestration/kfp/components/train/requirements.txt b/sdks/python/apache_beam/examples/ml-orchestration/kfp/components/train/requirements.txt index 57beb1844995..2e65f0fba246 100644 --- a/sdks/python/apache_beam/examples/ml-orchestration/kfp/components/train/requirements.txt +++ b/sdks/python/apache_beam/examples/ml-orchestration/kfp/components/train/requirements.txt @@ -15,4 +15,4 @@ torch==1.13.1 numpy==1.22.4 -Pillow==9.2.0 \ No newline at end of file +Pillow==10.2.0 \ No newline at end of file From cffeff78f622c566e87b18ab7994444b07b482c6 Mon Sep 17 00:00:00 2001 From: Svetak Sundhar Date: Tue, 23 Jan 2024 19:04:33 +0000 Subject: [PATCH 050/169] Fix Dicom IO Integration Tests (#30073) * [BEAM-12550] Implement Parallelizable Skew and Kurtosis (Skew implementation) R: @TheNeuralBit * Add failure case to Dicom IO Test * Fix Failing Test * lint * remove job.cancel() * spotless --------- Co-authored-by: svetakvsundhar --- .../sdk/io/gcp/healthcare/DicomIOReadIT.java | 40 +++++++++++++------ 1 file changed, 28 insertions(+), 12 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/DicomIOReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/DicomIOReadIT.java index e550b29541e5..570844011f39 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/DicomIOReadIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/DicomIOReadIT.java @@ -21,14 +21,12 @@ import java.io.IOException; import java.net.URISyntaxException; -import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.junit.After; import org.junit.Assert; import org.junit.Before; -import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; @@ -36,6 +34,8 @@ public class DicomIOReadIT { private static final String TEST_FILE_PATH = "src/test/resources/DICOM/testDicomFile.dcm"; private static final String TEST_FILE_STUDY_ID = "study_000000000"; + private static final String TEST_FILE_SERIES_ID = "series_000000000"; + private static final String TEST_FILE_INSTANCE_ID = "instance_000000000"; @Rule public transient TestPipeline pipeline = TestPipeline.create(); private String healthcareDataset; @@ -61,13 +61,16 @@ public void deleteDicomStore() throws IOException { client.deleteDicomStore(healthcareDataset + "/dicomStores/" + storeName); } - @Ignore("https://github.com/apache/beam/issues/28099") @Test - public void testDicomMetadataRead() throws IOException { + public void testDicomMetadataRead() { String webPath = String.format( - "%s/dicomStores/%s/dicomWeb/studies/%s", - healthcareDataset, storeName, TEST_FILE_STUDY_ID); + "%s/dicomStores/%s/dicomWeb/studies/%s/series/%s/instances/%s", + healthcareDataset, + storeName, + TEST_FILE_STUDY_ID, + TEST_FILE_SERIES_ID, + TEST_FILE_INSTANCE_ID); DicomIO.ReadStudyMetadata.Result result = pipeline.apply(Create.of(webPath)).apply(DicomIO.readStudyMetadata()); @@ -82,12 +85,25 @@ public void testDicomMetadataRead() throws IOException { return null; }); - PipelineResult job = pipeline.run(); + pipeline.run(); + } + + @Test + public void testDicomFailedMetadataRead() { + String badWebPath = "foo"; + + DicomIO.ReadStudyMetadata.Result result = + pipeline.apply(Create.of(badWebPath)).apply(DicomIO.readStudyMetadata()); + + PAssert.that(result.getReadResponse()).empty(); + + PAssert.that(result.getFailedReads()) + .satisfies( + (errors) -> { + Assert.assertTrue(errors.iterator().hasNext()); + return null; + }); - try { - job.cancel(); - } catch (UnsupportedOperationException exc) { - // noop - if runner does not support job.cancel() - } + pipeline.run(); } } From 77285589fe3071df68ddc626c28c0672304c6ab1 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 23 Jan 2024 15:16:04 -0500 Subject: [PATCH 051/169] Use vendored grpc 1.60.1 (#30069) --- .../beam/gradle/BeamModulePlugin.groovy | 12 +- .../beam/gradle/GrpcVendoring_1_54_0.groovy | 205 ------------------ examples/java/build.gradle | 2 +- .../transforms/DataProtectors.java | 6 +- .../kafkatopubsub/kafka/consumer/Utils.java | 4 +- runners/core-construction-java/build.gradle | 2 +- .../runners/core/construction/BeamUrns.java | 2 +- .../core/construction/CoderTranslation.java | 2 +- .../core/construction/CoderTranslators.java | 2 +- .../core/construction/CombineTranslation.java | 2 +- .../CreatePCollectionViewTranslation.java | 2 +- .../DefaultExpansionServiceClientFactory.java | 2 +- .../construction/DisplayDataTranslation.java | 2 +- .../core/construction/Environments.java | 4 +- .../runners/core/construction/External.java | 6 +- .../core/construction/ModelCoders.java | 2 +- .../PCollectionViewTranslation.java | 2 +- .../construction/PTransformTranslation.java | 2 +- .../core/construction/ParDoTranslation.java | 4 +- .../PipelineOptionsTranslation.java | 6 +- .../core/construction/ReadTranslation.java | 4 +- .../core/construction/SdkComponents.java | 2 +- .../construction/TestStreamTranslation.java | 2 +- .../core/construction/TransformUpgrader.java | 4 +- .../construction/WindowIntoTranslation.java | 2 +- .../WindowingStrategyTranslation.java | 8 +- .../construction/WriteFilesTranslation.java | 2 +- .../graph/GreedyPCollectionFusers.java | 2 +- .../construction/graph/QueryablePipeline.java | 2 +- .../core/construction/BaseExternalTest.java | 6 +- .../PipelineOptionsTranslationTest.java | 6 +- .../construction/PipelineTranslationTest.java | 2 +- .../construction/TransformUpgraderTest.java | 2 +- .../WindowIntoTranslationTest.java | 2 +- .../graph/ProtoOverridesTest.java | 2 +- runners/core-java/build.gradle | 2 +- .../core/metrics/MetricsContainerImpl.java | 2 +- .../core/metrics/MetricsContainerStepMap.java | 6 +- .../core/metrics/MonitoringInfoEncodings.java | 2 +- .../beam/runners/core/metrics/ShortIdMap.java | 2 +- .../core/metrics/SimpleExecutionState.java | 2 +- .../core/metrics/SimpleStateRegistry.java | 2 +- .../metrics/MonitoringInfoEncodingsTest.java | 2 +- runners/direct-java/build.gradle | 4 +- runners/flink/flink_runner.gradle | 2 +- .../FlinkBatchPortablePipelineTranslator.java | 2 +- .../beam/runners/flink/FlinkJobInvoker.java | 2 +- .../runners/flink/FlinkPipelineRunner.java | 2 +- ...nkStreamingPortablePipelineTranslator.java | 2 +- .../utils/FlinkPortableRunnerUtils.java | 2 +- .../ExecutableStageDoFnOperator.java | 4 +- .../wrappers/streaming/FlinkKeyUtils.java | 2 +- .../FlinkExecutableStageFunctionTest.java | 2 +- .../ExecutableStageDoFnOperatorTest.java | 4 +- .../wrappers/streaming/FlinkKeyUtilsTest.java | 2 +- .../google-cloud-dataflow-java/build.gradle | 2 +- .../dataflow/DataflowPipelineTranslator.java | 2 +- .../beam/runners/dataflow/DataflowRunner.java | 4 +- .../util/RowCoderCloudObjectTranslator.java | 2 +- .../SchemaCoderCloudObjectTranslator.java | 2 +- .../DataflowPipelineTranslatorTest.java | 2 +- .../runners/dataflow/DataflowRunnerTest.java | 2 +- .../worker/build.gradle | 4 +- .../dataflow/worker/ByteArrayReader.java | 4 +- .../dataflow/worker/ByteStringCoder.java | 2 +- .../worker/ChunkingShuffleBatchReader.java | 2 +- .../worker/DataflowWorkerHarnessHelper.java | 2 +- .../GroupAlsoByWindowParDoFnFactory.java | 2 +- .../dataflow/worker/PubsubDynamicSink.java | 2 +- .../runners/dataflow/worker/PubsubSink.java | 2 +- .../worker/StreamingDataflowWorker.java | 2 +- .../worker/StreamingModeExecutionContext.java | 2 +- .../worker/StreamingSideInputFetcher.java | 2 +- .../worker/UngroupedShuffleReader.java | 2 +- .../worker/WindmillComputationKey.java | 4 +- .../worker/WindmillNamespacePrefix.java | 2 +- .../runners/dataflow/worker/WindmillSink.java | 2 +- .../worker/WindmillTimerInternals.java | 2 +- .../dataflow/worker/WorkerCustomSources.java | 2 +- .../dataflow/worker/streaming/ShardedKey.java | 2 +- .../worker/ByteArrayShufflePosition.java | 4 +- .../worker/GroupingShuffleEntryIterator.java | 2 +- .../worker/KeyGroupedShuffleEntries.java | 2 +- .../util/common/worker/ShuffleEntry.java | 2 +- .../client/AbstractWindmillStream.java | 6 +- .../client/grpc/AppendableInputStream.java | 2 +- .../client/grpc/GrpcCommitWorkStream.java | 4 +- .../grpc/GrpcDeadlineClientInterceptor.java | 10 +- .../client/grpc/GrpcDirectGetWorkStream.java | 4 +- .../client/grpc/GrpcGetDataStream.java | 2 +- .../client/grpc/GrpcGetWorkStream.java | 4 +- .../grpc/GrpcGetWorkerMetadataStream.java | 2 +- .../client/grpc/GrpcWindmillServer.java | 6 +- .../grpc/auth/VendoredCredentialsAdapter.java | 4 +- ...endoredRequestMetadataCallbackAdapter.java | 4 +- .../grpc/observers/DirectStreamObserver.java | 4 +- .../ForwardingClientResponseObserver.java | 6 +- .../grpc/observers/StreamObserverFactory.java | 4 +- .../grpc/stubs/WindmillChannelFactory.java | 12 +- .../grpc/stubs/WindmillStubFactory.java | 4 +- .../worker/windmill/state/StateTag.java | 2 +- .../worker/windmill/state/WindmillBag.java | 2 +- .../worker/windmill/state/WindmillMap.java | 2 +- .../windmill/state/WindmillMultimap.java | 2 +- .../windmill/state/WindmillOrderedList.java | 2 +- .../windmill/state/WindmillStateCache.java | 2 +- .../windmill/state/WindmillStateReader.java | 2 +- .../windmill/state/WindmillStateUtil.java | 2 +- .../worker/windmill/state/WindmillValue.java | 2 +- .../windmill/state/WindmillWatermarkHold.java | 2 +- .../dataflow/harness/test/TestStreams.java | 4 +- .../dataflow/worker/DataflowMatchers.java | 2 +- .../DataflowWorkerHarnessHelperTest.java | 2 +- .../worker/GroupingShuffleReaderTest.java | 2 +- .../worker/PubsubDynamicSinkTest.java | 2 +- .../dataflow/worker/PubsubReaderTest.java | 2 +- .../dataflow/worker/PubsubSinkTest.java | 2 +- .../dataflow/worker/ReaderCacheTest.java | 2 +- .../dataflow/worker/ShuffleSinkTest.java | 2 +- .../worker/StreamingDataflowWorkerTest.java | 4 +- .../StreamingGroupAlsoByWindowFnsTest.java | 2 +- ...ngGroupAlsoByWindowsReshuffleDoFnTest.java | 2 +- .../StreamingModeExecutionContextTest.java | 2 +- .../StreamingSideInputDoFnRunnerTest.java | 2 +- .../worker/StreamingSideInputFetcherTest.java | 2 +- .../dataflow/worker/TestShuffleReader.java | 2 +- .../dataflow/worker/TestShuffleWriter.java | 2 +- .../worker/WindmillKeyedWorkItemTest.java | 2 +- .../WindmillReaderIteratorBaseTest.java | 2 +- .../worker/WorkerCustomSourcesTest.java | 2 +- .../DataflowWorkerLoggingHandlerTest.java | 2 +- .../worker/streaming/ActiveWorkStateTest.java | 2 +- .../sideinput/SideInputStateFetcherTest.java | 2 +- .../BatchingShuffleEntryReaderTest.java | 2 +- .../GroupingShuffleEntryIteratorTest.java | 2 +- .../util/common/worker/ShuffleEntryTest.java | 2 +- .../grpc/GrpcGetWorkerMetadataStreamTest.java | 14 +- .../client/grpc/GrpcWindmillServerTest.java | 30 +-- .../grpc/StreamingEngineClientTest.java | 14 +- .../client/grpc/WindmillStreamSenderTest.java | 6 +- .../state/WindmillStateCacheTest.java | 2 +- .../state/WindmillStateInternalsTest.java | 2 +- .../state/WindmillStateReaderTest.java | 2 +- .../EvenGetWorkBudgetDistributorTest.java | 6 +- runners/java-fn-execution/build.gradle | 2 +- .../artifact/ArtifactRetrievalService.java | 8 +- .../artifact/ArtifactStagingService.java | 10 +- .../control/BundleCheckpointHandlers.java | 2 +- .../control/FnApiControlClient.java | 6 +- .../FnApiControlClientPoolService.java | 6 +- .../control/ProcessBundleDescriptors.java | 2 +- .../fnexecution/data/GrpcDataService.java | 2 +- .../ExternalEnvironmentFactory.java | 2 +- .../logging/GrpcLoggingService.java | 2 +- .../fnexecution/provisioning/JobInfo.java | 2 +- .../StaticGrpcProvisionService.java | 2 +- .../fnexecution/state/GrpcStateService.java | 4 +- .../state/StateRequestHandlers.java | 2 +- .../status/BeamWorkerStatusGrpcService.java | 2 +- .../status/WorkerStatusClient.java | 2 +- .../translation/PipelineTranslatorUtils.java | 2 +- .../fnexecution/wire/ByteStringCoder.java | 4 +- ...GrpcContextHeaderAccessorProviderTest.java | 20 +- .../fnexecution/ServerFactoryTest.java | 12 +- .../ArtifactRetrievalServiceTest.java | 10 +- .../artifact/ArtifactStagingServiceTest.java | 12 +- .../control/DefaultJobBundleFactoryTest.java | 4 +- .../FnApiControlClientPoolServiceTest.java | 8 +- .../control/FnApiControlClientTest.java | 2 +- .../control/RemoteExecutionTest.java | 2 +- .../control/SdkHarnessClientTest.java | 2 +- .../fnexecution/data/GrpcDataServiceTest.java | 8 +- .../logging/GrpcLoggingServiceTest.java | 6 +- .../StaticGrpcProvisionServiceTest.java | 10 +- .../state/GrpcStateServiceTest.java | 4 +- .../state/StateRequestHandlersTest.java | 2 +- .../BeamWorkerStatusGrpcServiceTest.java | 8 +- .../status/WorkerStatusClientTest.java | 2 +- .../fnexecution/wire/ByteStringCoderTest.java | 2 +- .../fnexecution/wire/CommonCoderTest.java | 4 +- runners/java-job-service/build.gradle | 2 +- .../jobsubmission/InMemoryJobService.java | 10 +- .../runners/jobsubmission/JobInvocation.java | 2 +- .../runners/jobsubmission/JobInvoker.java | 2 +- .../runners/jobsubmission/JobPreparation.java | 2 +- .../PortablePipelineJarCreator.java | 4 +- .../PortablePipelineJarUtils.java | 6 +- .../jobsubmission/InMemoryJobServiceTest.java | 6 +- .../jobsubmission/JobInvocationTest.java | 2 +- .../PortablePipelineJarCreatorTest.java | 2 +- runners/portability/java/build.gradle | 2 +- .../portability/JobServicePipelineResult.java | 2 +- .../runners/portability/PortableRunner.java | 4 +- .../portability/testing/TestJobService.java | 2 +- .../portability/PortableRunnerTest.java | 8 +- runners/samza/build.gradle | 2 +- .../beam/runners/samza/SamzaJobInvoker.java | 2 +- .../runtime/SamzaStateRequestHandlers.java | 2 +- .../SamzaTestStreamTranslator.java | 2 +- .../translation/WindowAssignTranslator.java | 2 +- .../samza/util/PipelineJsonRenderer.java | 6 +- ...SamzaMetricsBundleProgressHandlerTest.java | 2 +- runners/spark/spark_runner.gradle | 2 +- .../beam/runners/spark/SparkJobInvoker.java | 2 +- .../runners/spark/SparkPipelineRunner.java | 2 +- runners/twister2/build.gradle | 2 +- .../functions/AssignWindowsFunction.java | 2 +- .../translators/functions/DoFnFunction.java | 2 +- .../functions/GroupByWindowFunction.java | 2 +- sdks/java/core/build.gradle | 2 +- sdks/java/core/jmh/build.gradle | 2 +- .../util/ByteStringOutputStreamBenchmark.java | 4 +- .../org/apache/beam/sdk/io/TextSource.java | 2 +- .../org/apache/beam/sdk/io/range/ByteKey.java | 4 +- .../beam/sdk/schemas/SchemaTranslation.java | 2 +- .../logicaltypes/SchemaLogicalType.java | 2 +- .../resourcehints/ResourceHints.java | 2 +- .../beam/sdk/util/ByteStringOutputStream.java | 4 +- .../org/apache/beam/sdk/util/CoderUtils.java | 2 +- .../sdk/schemas/SchemaTranslationTest.java | 2 +- .../beam/sdk/schemas/utils/JsonUtilsTest.java | 4 +- .../sdk/util/ByteStringOutputStreamTest.java | 2 +- .../apache/beam/sdk/util/CoderUtilsTest.java | 2 +- sdks/java/expansion-service/build.gradle | 2 +- .../expansion/service/ExpansionServer.java | 4 +- .../expansion/service/ExpansionService.java | 10 +- ...pansionServiceSchemaTransformProvider.java | 2 +- .../JavaClassLookupTransformProvider.java | 4 +- ...ionServiceSchemaTransformProviderTest.java | 4 +- .../sdk/expansion/service/ExternalTest.java | 4 +- .../JavaClassLookupTransformProviderTest.java | 4 +- sdks/java/extensions/python/build.gradle | 2 +- .../python/PythonExternalTransform.java | 2 +- .../schemaio-expansion-service/build.gradle | 2 +- .../ExternalSchemaIOTransformRegistrar.java | 2 +- sdks/java/fn-execution/build.gradle | 2 +- .../fn/channel/AddHarnessIdInterceptor.java | 8 +- .../sdk/fn/channel/ManagedChannelFactory.java | 20 +- .../sdk/fn/channel/SocketAddressFactory.java | 2 +- .../fn/data/BeamFnDataGrpcMultiplexer.java | 4 +- .../fn/data/BeamFnDataOutboundAggregator.java | 4 +- .../sdk/fn/data/DecodingFnDataReceiver.java | 2 +- .../beam/sdk/fn/data/RemoteGrpcPortRead.java | 2 +- .../beam/sdk/fn/data/RemoteGrpcPortWrite.java | 2 +- .../apache/beam/sdk/fn/server/FnService.java | 6 +- .../GrpcContextHeaderAccessorProvider.java | 16 +- .../beam/sdk/fn/server/GrpcFnServer.java | 2 +- .../sdk/fn/server/InProcessServerFactory.java | 8 +- .../beam/sdk/fn/server/ServerFactory.java | 20 +- .../fn/stream/BufferingStreamObserver.java | 4 +- .../beam/sdk/fn/stream/DataStreams.java | 2 +- .../sdk/fn/stream/DirectStreamObserver.java | 4 +- .../ForwardingClientResponseObserver.java | 6 +- .../fn/stream/OutboundObserverFactory.java | 4 +- .../fn/stream/SynchronizedStreamObserver.java | 2 +- .../apache/beam/sdk/fn/test/TestStreams.java | 4 +- .../fn/windowing/EncodedBoundedWindow.java | 2 +- .../fn/channel/ManagedChannelFactoryTest.java | 6 +- .../fn/channel/SocketAddressFactoryTest.java | 2 +- .../data/BeamFnDataGrpcMultiplexerTest.java | 2 +- .../sdk/fn/data/RemoteGrpcPortReadTest.java | 2 +- .../sdk/fn/data/RemoteGrpcPortWriteTest.java | 2 +- .../beam/sdk/fn/stream/DataStreamsTest.java | 2 +- .../ForwardingClientResponseObserverTest.java | 6 +- .../stream/OutboundObserverFactoryTest.java | 4 +- .../windowing/EncodedBoundedWindowTest.java | 2 +- sdks/java/harness/build.gradle | 2 +- sdks/java/harness/jmh/build.gradle | 2 +- .../harness/jmh/ProcessBundleBenchmark.java | 2 +- .../ExecutionStateSamplerBenchmark.java | 2 +- .../harness/jmh/control/MetricsBenchmark.java | 2 +- .../logging/BeamFnLoggingClientBenchmark.java | 6 +- .../beam/fn/harness/BeamFnDataReadRunner.java | 2 +- .../fn/harness/ExternalWorkerService.java | 2 +- .../beam/fn/harness/FnApiDoFnRunner.java | 16 +- .../org/apache/beam/fn/harness/FnHarness.java | 4 +- .../harness/control/BeamFnControlClient.java | 4 +- .../control/BundleProgressReporter.java | 2 +- .../control/ExecutionStateSampler.java | 2 +- .../beam/fn/harness/control/Metrics.java | 2 +- .../harness/control/ProcessBundleHandler.java | 4 +- .../fn/harness/data/BeamFnDataGrpcClient.java | 2 +- .../harness/logging/BeamFnLoggingClient.java | 16 +- .../state/BeamFnStateGrpcClientCache.java | 4 +- .../fn/harness/state/FnApiStateAccessor.java | 2 +- .../state/FnApiTimerBundleTracker.java | 2 +- .../fn/harness/state/MultimapSideInput.java | 2 +- .../fn/harness/state/MultimapUserState.java | 2 +- .../fn/harness/state/StateBackedIterable.java | 2 +- .../harness/state/StateFetchingIterators.java | 2 +- .../fn/harness/status/BeamFnStatusClient.java | 4 +- .../HarnessStreamObserverFactories.java | 2 +- .../fn/harness/BeamFnDataReadRunnerTest.java | 2 +- .../fn/harness/BeamFnDataWriteRunnerTest.java | 2 +- .../fn/harness/ExternalWorkerServiceTest.java | 2 +- .../beam/fn/harness/FnApiDoFnRunnerTest.java | 38 ++-- .../apache/beam/fn/harness/FnHarnessTest.java | 8 +- .../control/BeamFnControlClientTest.java | 8 +- .../control/BundleProgressReporterTest.java | 2 +- .../control/ExecutionStateSamplerTest.java | 2 +- .../beam/fn/harness/control/MetricsTest.java | 2 +- .../control/ProcessBundleHandlerTest.java | 4 +- .../data/BeamFnDataGrpcClientTest.java | 14 +- .../data/PCollectionConsumerRegistryTest.java | 14 +- .../fn/harness/debug/DataSamplerTest.java | 2 +- .../fn/harness/debug/OutputSamplerTest.java | 2 +- .../logging/BeamFnLoggingClientTest.java | 32 +-- .../fn/harness/state/BagUserStateTest.java | 2 +- .../state/BeamFnStateGrpcClientCacheTest.java | 12 +- .../harness/state/FakeBeamFnStateClient.java | 2 +- .../harness/state/IterableSideInputTest.java | 2 +- .../harness/state/MultimapSideInputTest.java | 2 +- .../harness/state/MultimapUserStateTest.java | 2 +- .../state/StateBackedIterableTest.java | 2 +- .../state/StateFetchingIteratorsTest.java | 2 +- .../status/BeamFnStatusClientTest.java | 6 +- .../HarnessStreamObserverFactoriesTest.java | 4 +- sdks/java/io/components/build.gradle | 2 +- sdks/java/io/contextualtextio/build.gradle | 2 +- .../ContextualTextIOSource.java | 2 +- sdks/java/io/debezium/build.gradle | 2 +- .../beam/io/debezium/SourceRecordJson.java | 4 +- .../io/google-cloud-platform/build.gradle | 2 +- .../gcp/bigquery/BigQueryIOTranslation.java | 2 +- .../spanner/SpannerTransformRegistrar.java | 2 +- ...ngestreamsReadSchemaTransformProvider.java | 2 +- .../io/gcp/pubsub/PubsubIOExternalTest.java | 2 +- .../SpannerTransformRegistrarTest.java | 2 +- .../sdk/io/kafka/KafkaIOExternalTest.java | 2 +- sdks/java/io/kafka/upgrade/build.gradle | 2 +- .../io/kafka/upgrade/KafkaIOTranslation.java | 2 +- sdks/java/transform-service/build.gradle | 2 +- .../sdk/transformservice/ArtifactService.java | 6 +- .../beam/sdk/transformservice/Controller.java | 4 +- .../transformservice/ExpansionService.java | 4 +- .../transformservice/ArtifactServiceTest.java | 2 +- .../ExpansionServiceTest.java | 2 +- settings.gradle.kts | 1 - vendor/README.md | 8 +- vendor/grpc-1_54_0/build.gradle | 37 ---- 340 files changed, 642 insertions(+), 885 deletions(-) delete mode 100644 buildSrc/src/main/groovy/org/apache/beam/gradle/GrpcVendoring_1_54_0.groovy delete mode 100644 vendor/grpc-1_54_0/build.gradle diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 51b3cbbf178f..3045e2615e4a 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -903,7 +903,7 @@ class BeamModulePlugin implements Plugin { testcontainers_rabbitmq : "org.testcontainers:rabbitmq:$testcontainers_version", truth : "com.google.truth:truth:1.1.5", threetenbp : "org.threeten:threetenbp:1.6.8", - vendored_grpc_1_54_0 : "org.apache.beam:beam-vendor-grpc-1_54_0:0.1", + vendored_grpc_1_60_1 : "org.apache.beam:beam-vendor-grpc-1_60_1:0.1", vendored_guava_32_1_2_jre : "org.apache.beam:beam-vendor-guava-32_1_2-jre:0.1", vendored_calcite_1_28_0 : "org.apache.beam:beam-vendor-calcite-1_28_0:0.2", woodstox_core_asl : "org.codehaus.woodstox:woodstox-core-asl:4.4.1", @@ -2402,10 +2402,10 @@ class BeamModulePlugin implements Plugin { archivesBaseName: configuration.archivesBaseName, automaticModuleName: configuration.automaticModuleName, shadowJarValidationExcludes: it.shadowJarValidationExcludes, - shadowClosure: GrpcVendoring_1_54_0.shadowClosure() << { + shadowClosure: GrpcVendoring_1_60_1.shadowClosure() << { // We perform all the code relocations but don't include // any of the actual dependencies since they will be supplied - // by org.apache.beam:beam-vendor-grpc-v1p54p0 + // by org.apache.beam:beam-vendor-grpc-v1p60p1 dependencies { include(dependency { return false }) } @@ -2422,14 +2422,14 @@ class BeamModulePlugin implements Plugin { project.protobuf { protoc { // The artifact spec for the Protobuf Compiler - artifact = "com.google.protobuf:protoc:${GrpcVendoring_1_54_0.protobuf_version}" } + artifact = "com.google.protobuf:protoc:${GrpcVendoring_1_60_1.protobuf_version}" } // Configure the codegen plugins plugins { // An artifact spec for a protoc plugin, with "grpc" as // the identifier, which can be referred to in the "plugins" // container of the "generateProtoTasks" closure. - grpc { artifact = "io.grpc:protoc-gen-grpc-java:${GrpcVendoring_1_54_0.grpc_version}" } + grpc { artifact = "io.grpc:protoc-gen-grpc-java:${GrpcVendoring_1_60_1.grpc_version}" } } generateProtoTasks { @@ -2443,7 +2443,7 @@ class BeamModulePlugin implements Plugin { } } - project.dependencies GrpcVendoring_1_54_0.dependenciesClosure() << { shadow project.ext.library.java.vendored_grpc_1_54_0 } + project.dependencies GrpcVendoring_1_60_1.dependenciesClosure() << { shadow project.ext.library.java.vendored_grpc_1_60_1 } } /** ***********************************************************************************************/ diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/GrpcVendoring_1_54_0.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/GrpcVendoring_1_54_0.groovy deleted file mode 100644 index 4b8a059109c9..000000000000 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/GrpcVendoring_1_54_0.groovy +++ /dev/null @@ -1,205 +0,0 @@ -/* - * 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.gradle - -/** - * Utilities for working with our vendored version of gRPC. - * - * To update: - * 1. Determine the set of io.grpc libraries we want to include, most likely a superset of - * of the previous vendored gRPC version. - * 2. Use mvn dependency:tree and https://search.maven.org/search?q=g:io.grpc - * to determine dependency tree. You may need to search for optional dependencies - * and determine if they are needed (e.g. conscrypt). - * 3. Validate built artifacts by running linkage tool - * (https://github.com/apache/beam/tree/master/vendor#how-to-validate-the-vendored-dependencies) - * and unit and integration tests in a PR (e.g. https://github.com/apache/beam/pull/16460, - * https://github.com/apache/beam/pull/16459) - */ -class GrpcVendoring_1_54_0 { - static def grpc_version = "1.54.0" - - // See https://github.com/grpc/grpc-java/blob/v1.54.0/gradle/libs.versions.toml - // or https://search.maven.org/search?q=g:io.grpc%201.54.0 - static def guava_version = "31.1-jre" - static def protobuf_version = "3.21.7" - static def gson_version = "2.9.0" - static def google_auth_version = "1.4.0" - static def opencensus_version = "0.31.0" - static def conscrypt_version = "2.5.2" - static def proto_google_common_protos_version = "2.9.0" - static def netty_version = "4.1.87.Final" - static def netty_tcnative_version = "2.0.56.Final" - - /** Returns the list of implementation time dependencies. */ - static List dependencies() { - return [ - "com.google.guava:guava:$guava_version", - "com.google.protobuf:protobuf-java:$protobuf_version", - "com.google.protobuf:protobuf-java-util:$protobuf_version", - "com.google.code.gson:gson:$gson_version", - "io.grpc:grpc-auth:$grpc_version", - "io.grpc:grpc-core:$grpc_version", - "io.grpc:grpc-context:$grpc_version", - "io.grpc:grpc-netty:$grpc_version", - "io.grpc:grpc-protobuf:$grpc_version", - "io.grpc:grpc-stub:$grpc_version", - "io.grpc:grpc-alts:$grpc_version", - "io.grpc:grpc-testing:$grpc_version", - // Use a classifier to ensure we get the jar containing native libraries. In the future - // hopefully netty releases a single jar containing native libraries for all architectures. - "io.netty:netty-transport-native-epoll:$netty_version:linux-x86_64", - "io.netty:netty-tcnative-boringssl-static:$netty_tcnative_version", - "com.google.auth:google-auth-library-credentials:$google_auth_version", - "com.google.api.grpc:proto-google-common-protos:$proto_google_common_protos_version", - "io.opencensus:opencensus-api:$opencensus_version", - "io.opencensus:opencensus-contrib-grpc-metrics:$opencensus_version", - ] - } - - /** - * Returns the list of runtime time dependencies that should be exported as runtime - * dependencies within the vendored jar. - */ - static List runtimeDependencies() { - return [ - 'com.google.errorprone:error_prone_annotations:2.14.0', - // TODO(BEAM-9288): Enable relocation for conscrypt - "org.conscrypt:conscrypt-openjdk-uber:$conscrypt_version" - ] - } - - /** - * Returns the list of test dependencies. - */ - static List testDependencies() { - return [ - 'junit:junit:4.12', - ] - } - - static Map relocations() { - // The relocation paths below specifically use gRPC and the full version string as - // the code relocation prefix. See https://lists.apache.org/thread.html/4c12db35b40a6d56e170cd6fc8bb0ac4c43a99aa3cb7dbae54176815@%3Cdev.beam.apache.org%3E - // for further details. - - // To produce the list of necessary relocations, one needs to start with a set of target - // packages that one wants to vendor, find all necessary transitive dependencies of that - // set and provide relocations for each such that all necessary packages and their - // dependencies are relocated. Any optional dependency that doesn't need relocation - // must be excluded via an 'exclude' rule. There is additional complexity of libraries that use - // JNI or reflection and have to be handled on case by case basis by learning whether - // they support relocation and how would one go about doing it by reading any documentation - // those libraries may provide. The 'validateShadedJarDoesntLeakNonOrgApacheBeamClasses' - // ensures that there are no classes outside of the 'org.apache.beam' namespace. - - String version = "v" + grpc_version.replace(".", "p") - String prefix = "org.apache.beam.vendor.grpc.${version}"; - List packagesToRelocate = [ - // guava uses the com.google.common and com.google.thirdparty package namespaces - "com.google.common", - "com.google.thirdparty", - "com.google.protobuf", - "com.google.gson", - "com.google.auth", - "com.google.api", - "com.google.cloud", - "com.google.logging", - "com.google.longrunning", - "com.google.rpc", - "com.google.type", - "com.google.geo.type", - "io.grpc", - "io.netty", - "io.opencensus", - "io.perfmark", - ] - - return packagesToRelocate.collectEntries { - [ (it): "${prefix}.${it}" ] - } + [ - // Adapted from https://github.com/grpc/grpc-java/blob/e283f70ad91f99c7fee8b31b605ef12a4f9b1690/netty/shaded/build.gradle#L41 - // We "io.netty": "${prefix}.io.netty",have to be careful with these replacements as they must not match any - // string in NativeLibraryLoader, else they cause corruption. Note that - // this includes concatenation of string literals and constants. - 'META-INF/native/libnetty': "META-INF/native/liborg_apache_beam_vendor_grpc_${version}_netty", - 'META-INF/native/netty': "META-INF/native/org_apache_beam_vendor_grpc_${version}_netty", - 'META-INF/native/lib-netty': "META-INF/native/lib-org-apache-beam-vendor-grpc-${version}-netty", - ] - } - - /** Returns the list of shading exclusions. */ - static List exclusions() { - return [ - // Don't include in the vendored jar: - // android annotations, errorprone, checkerframework, JDK8 annotations, objenesis, junit, - // commons-logging, log4j, slf4j and mockito - "android/annotation/**/", - "com/google/errorprone/**", - "com/google/instrumentation/**", - "com/google/j2objc/annotations/**", - "io/netty/handler/codec/marshalling/**", - "io/netty/handler/codec/spdy/**", - "io/netty/handler/codec/compression/JZlib*", - "io/netty/handler/codec/compression/Lz4*", - "io/netty/handler/codec/compression/Lzf*", - "io/netty/handler/codec/compression/Lzma*", - "io/netty/handler/codec/protobuf/Protobuf*Nano.class", - "io/netty/util/internal/logging/CommonsLogger*", - "io/netty/util/internal/logging/LocationAwareSlf4JLogger*", - "io/netty/util/internal/logging/Log4JLogger*", - "io/netty/util/internal/logging/Log4J2Logger*", - "javax/annotation/**", - "junit/**", - "module-info.class", - "org/checkerframework/**", - "org/codehaus/mojo/animal_sniffer/**", - "org/conscrypt/**", - "META-INF/native/libconscrypt**", - "META-INF/native/conscrypt**", - "org/hamcrest/**", - "org/junit/**", - "org/mockito/**", - "org/objenesis/**", - ] - } - - /** - * Returns a closure contaning the dependencies map used for shading gRPC within the main - * Apache Beam project. - */ - static Object dependenciesClosure() { - return { - dependencies().each { implementation it } - runtimeDependencies().each { shadow it } - } - } - - /** - * Returns a closure with the code relocation configuration for shading gRPC within the main - * Apache Beam project. - */ - static Object shadowClosure() { - return { - relocations().each { srcNamespace, destNamespace -> - relocate srcNamespace, destNamespace - } - exclusions().each { exclude it } - } - } -} diff --git a/examples/java/build.gradle b/examples/java/build.gradle index ec08eb45460e..e83737562ab4 100644 --- a/examples/java/build.gradle +++ b/examples/java/build.gradle @@ -87,7 +87,7 @@ dependencies { implementation library.java.commons_io implementation library.java.commons_csv runtimeOnly project(path: ":runners:direct-java", configuration: "shadow") - implementation library.java.vendored_grpc_1_54_0 + implementation library.java.vendored_grpc_1_60_1 implementation library.java.vendored_guava_32_1_2_jre implementation "com.google.api.grpc:proto-google-cloud-language-v1:1.81.4" implementation ("io.confluent:kafka-avro-serializer:5.3.2") { diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java index 287f8be71d5c..c21adc0fb658 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java @@ -48,9 +48,9 @@ import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.gson.Gson; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.gson.JsonArray; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.gson.JsonObject; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.gson.Gson; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.gson.JsonArray; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.gson.JsonObject; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Throwables; import org.apache.commons.io.IOUtils; import org.apache.http.HttpEntity; diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/kafka/consumer/Utils.java b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/kafka/consumer/Utils.java index 41fe2c23a24b..d346fd633d5b 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/kafka/consumer/Utils.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/kafka/consumer/Utils.java @@ -28,8 +28,8 @@ import java.util.Map; import java.util.stream.Collectors; import org.apache.beam.examples.complete.kafkatopubsub.options.KafkaToPubsubOptions; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.gson.JsonObject; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.gson.JsonParser; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.gson.JsonObject; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.gson.JsonParser; import org.apache.http.HttpResponse; import org.apache.http.client.HttpClient; import org.apache.http.client.methods.HttpGet; diff --git a/runners/core-construction-java/build.gradle b/runners/core-construction-java/build.gradle index f593865b3fe9..622bd7048e4b 100644 --- a/runners/core-construction-java/build.gradle +++ b/runners/core-construction-java/build.gradle @@ -56,7 +56,7 @@ dependencies { implementation project(path: ":sdks:java:extensions:avro") implementation project(path: ":sdks:java:fn-execution") implementation project(path: ":sdks:java:transform-service:launcher") - implementation library.java.vendored_grpc_1_54_0 + implementation library.java.vendored_grpc_1_60_1 implementation library.java.vendored_guava_32_1_2_jre implementation library.java.classgraph implementation library.java.jackson_core diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/BeamUrns.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/BeamUrns.java index 0766dcc723e2..a5bbc0103363 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/BeamUrns.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/BeamUrns.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.core.construction; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ProtocolMessageEnum; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ProtocolMessageEnum; /** Returns the standard URN of a given enum annotated with [(standard_urn)]. */ public class BeamUrns { diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java index 5fce0310f7b1..06f5a4e43e59 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java @@ -26,7 +26,7 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BiMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableBiMap; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslators.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslators.java index 769778e0eca4..c175595baaf8 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslators.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslators.java @@ -37,7 +37,7 @@ import org.apache.beam.sdk.util.ShardedKey; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; /** {@link CoderTranslator} implementations for known coder types. */ diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java index fbe4876b414f..d502ba1514db 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java @@ -39,7 +39,7 @@ import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; 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.Iterables; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslation.java index 71038564ec4c..d27599c1ae07 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslation.java @@ -34,7 +34,7 @@ import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; /** * Utility methods for translating a {@link View} transforms to and from {@link RunnerApi} diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DefaultExpansionServiceClientFactory.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DefaultExpansionServiceClientFactory.java index 0341f8ea27ef..106b8b39cbee 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DefaultExpansionServiceClientFactory.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DefaultExpansionServiceClientFactory.java @@ -23,7 +23,7 @@ import org.apache.beam.model.expansion.v1.ExpansionApi; import org.apache.beam.model.expansion.v1.ExpansionServiceGrpc; import org.apache.beam.model.pipeline.v1.Endpoints; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; /** Default factory for ExpansionServiceClient used by External transform. */ public class DefaultExpansionServiceClientFactory implements ExpansionServiceClientFactory { diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DisplayDataTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DisplayDataTranslation.java index e031274e419f..d0e22d5e1df2 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DisplayDataTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DisplayDataTranslation.java @@ -25,7 +25,7 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.StandardDisplayData; import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; 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; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java index 6e05c006d283..cc1f8cbd0701 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java @@ -50,8 +50,8 @@ import org.apache.beam.sdk.util.ReleaseInfo; import org.apache.beam.sdk.util.ZipFiles; import org.apache.beam.sdk.util.common.ReflectHelpers; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java index 3951148d1a58..ea3e0a42e3cf 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java @@ -51,9 +51,9 @@ import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.PValues; import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannelBuilder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; 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; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java index e428d58d4803..e37b52638be8 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java @@ -26,7 +26,7 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.Coder; import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; import org.apache.beam.model.pipeline.v1.RunnerApi.StandardCoders; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; /** Utilities and constants ot interact with coders that are part of the Beam Model. */ diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionViewTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionViewTranslation.java index 6ccf673af0d1..180934fb8b3b 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionViewTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionViewTranslation.java @@ -29,7 +29,7 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; /** Utilities for interacting with PCollection view protos. */ public class PCollectionViewTranslation { diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java index 6bdd0fc37394..758c43b18c29 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java @@ -52,7 +52,7 @@ import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Joiner; 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; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java index 5ef11a747e87..eb4291dc003a 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java @@ -79,8 +79,8 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineOptionsTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineOptionsTranslation.java index 4a23c6237267..840466d6b96b 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineOptionsTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineOptionsTranslation.java @@ -27,9 +27,9 @@ import java.util.Map; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.common.ReflectHelpers; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Struct; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.util.JsonFormat; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.util.JsonFormat; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.CaseFormat; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java index f04d25509593..71cdb18ccf11 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java @@ -36,8 +36,8 @@ import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; /** diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java index feeb5feda1b1..dc24c5540920 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java @@ -41,7 +41,7 @@ import org.apache.beam.sdk.util.NameUtils; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BiMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.HashBiMap; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TestStreamTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TestStreamTranslation.java index 53bb324d03fa..4fcf32bc7486 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TestStreamTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TestStreamTranslation.java @@ -36,7 +36,7 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java index f07df6052155..6e2bce503be1 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java @@ -48,8 +48,8 @@ import org.apache.beam.sdk.util.ReleaseInfo; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannelBuilder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java index 294d89308a31..3beeb043309d 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java @@ -31,7 +31,7 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.checkerframework.checker.nullness.qual.Nullable; /** diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java index 0acf8c97b763..7266b4125cae 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java @@ -43,10 +43,10 @@ import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.util.Durations; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.util.Timestamps; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.util.Durations; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.util.Timestamps; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; import org.joda.time.Duration; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java index a32a2a98f410..a2ad432ce643 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java @@ -45,7 +45,7 @@ import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPCollectionFusers.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPCollectionFusers.java index 4265264f6b5e..0405bfb2ccd5 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPCollectionFusers.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPCollectionFusers.java @@ -31,7 +31,7 @@ import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode; import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode; import org.apache.beam.sdk.transforms.Flatten; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; 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.Maps; import org.slf4j.Logger; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/QueryablePipeline.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/QueryablePipeline.java index 1b89f282e7d5..07d78240446c 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/QueryablePipeline.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/QueryablePipeline.java @@ -61,7 +61,7 @@ import org.apache.beam.runners.core.construction.PTransformTranslation; import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode; import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; 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.Iterables; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/BaseExternalTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/BaseExternalTest.java index 38fc68c5e840..4126e72325f7 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/BaseExternalTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/BaseExternalTest.java @@ -22,9 +22,9 @@ import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ConnectivityState; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ConnectivityState; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannelBuilder; import org.junit.After; import org.junit.Before; import org.junit.BeforeClass; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PipelineOptionsTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PipelineOptionsTranslationTest.java index 497a7388dfbe..4f924e28dcb3 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PipelineOptionsTranslationTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PipelineOptionsTranslationTest.java @@ -30,9 +30,9 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.util.common.ReflectHelpers; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.NullValue; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Struct; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Value; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.NullValue; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Value; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PipelineTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PipelineTranslationTest.java index c17a2866f3c1..d6bfc576b49f 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PipelineTranslationTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PipelineTranslationTest.java @@ -62,7 +62,7 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Duration; import org.junit.Test; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformUpgraderTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformUpgraderTest.java index 2b01bf702465..4735df5397d1 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformUpgraderTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformUpgraderTest.java @@ -47,7 +47,7 @@ import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.checkerframework.checker.nullness.qual.Nullable; import org.junit.Test; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslationTest.java index 113afd4b0f40..2439657a93bb 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslationTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslationTest.java @@ -38,7 +38,7 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.Window.Assign; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ProtoOverridesTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ProtoOverridesTest.java index 17b2a5039c41..cdd00ea5c5db 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ProtoOverridesTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ProtoOverridesTest.java @@ -36,7 +36,7 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline; import org.apache.beam.model.pipeline.v1.RunnerApi.WindowingStrategy; import org.apache.beam.runners.core.construction.graph.ProtoOverrides.TransformReplacement; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/runners/core-java/build.gradle b/runners/core-java/build.gradle index f8663e215fb6..b7dd51e72ed1 100644 --- a/runners/core-java/build.gradle +++ b/runners/core-java/build.gradle @@ -46,7 +46,7 @@ dependencies { implementation project(":sdks:java:fn-execution") implementation library.java.vendored_guava_32_1_2_jre implementation library.java.joda_time - implementation library.java.vendored_grpc_1_54_0 + implementation library.java.vendored_grpc_1_60_1 implementation library.java.slf4j_api testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") testImplementation library.java.junit diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsContainerImpl.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsContainerImpl.java index af18044ce42c..abf3bb2f886b 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsContainerImpl.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsContainerImpl.java @@ -47,7 +47,7 @@ import org.apache.beam.sdk.metrics.MetricsContainer; import org.apache.beam.sdk.util.HistogramData; import org.apache.beam.sdk.values.KV; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; 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; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsContainerStepMap.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsContainerStepMap.java index 7f49ebda2e20..b59e58956a12 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsContainerStepMap.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsContainerStepMap.java @@ -34,9 +34,9 @@ import org.apache.beam.sdk.metrics.MetricKey; import org.apache.beam.sdk.metrics.MetricResult; import org.apache.beam.sdk.metrics.MetricResults; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.util.JsonFormat; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.util.JsonFormat; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoEncodings.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoEncodings.java index be33b305603e..12e7b41650dd 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoEncodings.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoEncodings.java @@ -23,7 +23,7 @@ import org.apache.beam.sdk.coders.DoubleCoder; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.joda.time.Instant; /** A set of functions used to encode and decode common monitoring info types. */ diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/ShortIdMap.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/ShortIdMap.java index d035be86008f..4ac3c150b3e5 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/ShortIdMap.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/ShortIdMap.java @@ -23,7 +23,7 @@ import java.util.Map; import java.util.NoSuchElementException; import org.apache.beam.model.pipeline.v1.MetricsApi.MonitoringInfo; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BiMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.HashBiMap; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleExecutionState.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleExecutionState.java index 4729e8163bc5..2d43e00cbfa0 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleExecutionState.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleExecutionState.java @@ -25,7 +25,7 @@ import java.util.Map; import org.apache.beam.model.pipeline.v1.MetricsApi.MonitoringInfo; import org.apache.beam.runners.core.metrics.ExecutionStateTracker.ExecutionState; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.joda.time.Duration; import org.joda.time.format.PeriodFormatter; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleStateRegistry.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleStateRegistry.java index f75ea85d995c..4215c6a0934b 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleStateRegistry.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleStateRegistry.java @@ -23,7 +23,7 @@ import java.util.Map; import javax.annotation.Nullable; import org.apache.beam.model.pipeline.v1.MetricsApi.MonitoringInfo; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; /** * A Class for registering SimpleExecutionStates with and extracting execution time MonitoringInfos. diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MonitoringInfoEncodingsTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MonitoringInfoEncodingsTest.java index 8e93505e91a9..a1b73781cd6c 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MonitoringInfoEncodingsTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MonitoringInfoEncodingsTest.java @@ -28,7 +28,7 @@ import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.encodeInt64Gauge; import static org.junit.Assert.assertEquals; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/runners/direct-java/build.gradle b/runners/direct-java/build.gradle index 8bb8933051b7..89bf6b30ed9e 100644 --- a/runners/direct-java/build.gradle +++ b/runners/direct-java/build.gradle @@ -73,7 +73,7 @@ dependencies { implementation project(it) } shadow project(path: ":sdks:java:core", configuration: "shadow") - shadow library.java.vendored_grpc_1_54_0 + shadow library.java.vendored_grpc_1_60_1 shadow library.java.joda_time shadow library.java.slf4j_api shadow library.java.jackson_databind @@ -94,7 +94,7 @@ dependencies { validatesRunner project(path: ":sdks:java:core", configuration: "shadowTest") validatesRunner project(path: project.path, configuration: "shadow") validatesRunner project(path: project.path, configuration: "shadowTest") - permitUnusedDeclared library.java.vendored_grpc_1_54_0 + permitUnusedDeclared library.java.vendored_grpc_1_60_1 permitUnusedDeclared project(":runners:java-fn-execution") permitUnusedDeclared project(":sdks:java:fn-execution") permitUnusedDeclared project(":sdks:java:extensions:avro") diff --git a/runners/flink/flink_runner.gradle b/runners/flink/flink_runner.gradle index c5946c509a95..e7fd87067cdf 100644 --- a/runners/flink/flink_runner.gradle +++ b/runners/flink/flink_runner.gradle @@ -151,7 +151,7 @@ dependencies { implementation project(":runners:java-fn-execution") implementation project(":runners:java-job-service") implementation project(":sdks:java:extensions:google-cloud-platform-core") - implementation library.java.vendored_grpc_1_54_0 + implementation library.java.vendored_grpc_1_60_1 implementation library.java.slf4j_api implementation library.java.joda_time implementation library.java.args4j diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPortablePipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPortablePipelineTranslator.java index 92c43ccf75b4..e0bef5e5d73a 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPortablePipelineTranslator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPortablePipelineTranslator.java @@ -71,7 +71,7 @@ import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BiMap; 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.Iterables; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkJobInvoker.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkJobInvoker.java index b6e65cdc6629..07c340fea1a4 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkJobInvoker.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkJobInvoker.java @@ -28,7 +28,7 @@ import org.apache.beam.runners.jobsubmission.PortablePipelineJarCreator; import org.apache.beam.runners.jobsubmission.PortablePipelineRunner; import org.apache.beam.sdk.options.PortablePipelineOptions; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Struct; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; 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.util.concurrent.ListeningExecutorService; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java index 5f6074e2546f..269f21030210 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java @@ -44,7 +44,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.SdkHarnessOptions; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Struct; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.flink.api.common.JobExecutionResult; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java index f6c25b0cce68..7077ea4c3d1a 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java @@ -98,7 +98,7 @@ import org.apache.beam.sdk.values.TypeDescriptors; import org.apache.beam.sdk.values.ValueWithRecordId; import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BiMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.HashMultiset; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/FlinkPortableRunnerUtils.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/FlinkPortableRunnerUtils.java index 5360bf93cc99..359a0c60eb7e 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/FlinkPortableRunnerUtils.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/FlinkPortableRunnerUtils.java @@ -19,7 +19,7 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; /** * Various utilies related to portability. Helps share code between portable batch and streaming diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java index d21316c16160..1bdb57b8f989 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java @@ -106,8 +106,8 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.StatusRuntimeException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.StatusRuntimeException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.flink.api.common.state.ListStateDescriptor; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkKeyUtils.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkKeyUtils.java index 6c81df0bfcd3..28fa37cfa5ae 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkKeyUtils.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkKeyUtils.java @@ -32,7 +32,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StructuredCoder; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; /** * Utility functions for dealing with key encoding. Beam requires keys to be compared in binary diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunctionTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunctionTest.java index 88488d82884f..d00c3d821e86 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunctionTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunctionTest.java @@ -51,7 +51,7 @@ import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Struct; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.api.common.functions.RuntimeContext; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java index 1aedd1af673b..0ad9ef694a20 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java @@ -101,8 +101,8 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Struct; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; 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; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkKeyUtilsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkKeyUtilsTest.java index d1751c047ef1..cab45632ac55 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkKeyUtilsTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkKeyUtilsTest.java @@ -26,7 +26,7 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.junit.Test; diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index 76f4aac6808b..f34db247a0f6 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -116,7 +116,7 @@ dependencies { implementation library.java.jackson_databind implementation library.java.joda_time implementation library.java.slf4j_api - implementation library.java.vendored_grpc_1_54_0 + implementation library.java.vendored_grpc_1_60_1 testImplementation library.java.guava_testlib testImplementation library.java.junit testImplementation project(path: ":sdks:java:io:google-cloud-platform", configuration: "testRuntimeMigration") diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java index 1660f6ee6194..903298458184 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java @@ -107,7 +107,7 @@ import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 725903aaaa27..5d2b7d190227 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -165,8 +165,8 @@ import org.apache.beam.sdk.values.TypeDescriptors; import org.apache.beam.sdk.values.ValueWithRecordId; import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.TextFormat; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.TextFormat; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Joiner; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RowCoderCloudObjectTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RowCoderCloudObjectTranslator.java index e6b1a053b75d..115c7883e77e 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RowCoderCloudObjectTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RowCoderCloudObjectTranslator.java @@ -23,7 +23,7 @@ import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.SchemaTranslation; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.util.JsonFormat; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.util.JsonFormat; /** Translator for row coders. */ public class RowCoderCloudObjectTranslator implements CloudObjectTranslator { diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SchemaCoderCloudObjectTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SchemaCoderCloudObjectTranslator.java index afe38019e143..f611b95358dc 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SchemaCoderCloudObjectTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SchemaCoderCloudObjectTranslator.java @@ -31,7 +31,7 @@ import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.StringUtils; import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.util.JsonFormat; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.util.JsonFormat; /** Translator for Schema coders. */ @SuppressWarnings({ diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java index 6787885d751a..37e50a2d0ebb 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java @@ -130,7 +130,7 @@ import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.sdk.values.TypeDescriptors; import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; 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.ImmutableSet; diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java index 63fccc8f05b9..51af766511fc 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java @@ -170,7 +170,7 @@ import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TypeDescriptors; import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; 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.Iterables; diff --git a/runners/google-cloud-dataflow-java/worker/build.gradle b/runners/google-cloud-dataflow-java/worker/build.gradle index 1ca9eba2b482..6a15dbd5531b 100644 --- a/runners/google-cloud-dataflow-java/worker/build.gradle +++ b/runners/google-cloud-dataflow-java/worker/build.gradle @@ -43,7 +43,7 @@ def sdk_provided_dependencies = [ library.java.jackson_databind, library.java.joda_time, library.java.slf4j_api, - library.java.vendored_grpc_1_54_0, + library.java.vendored_grpc_1_60_1, ] def sdk_provided_shaded_project_dependencies = [ @@ -214,7 +214,7 @@ dependencies { implementation library.java.jackson_databind implementation library.java.joda_time implementation library.java.slf4j_api - implementation library.java.vendored_grpc_1_54_0 + implementation library.java.vendored_grpc_1_60_1 implementation library.java.error_prone_annotations permitUnusedDeclared library.java.error_prone_annotations runtimeOnly library.java.slf4j_jdk14 diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ByteArrayReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ByteArrayReader.java index e3de27baa518..c3e139221c70 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ByteArrayReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ByteArrayReader.java @@ -17,8 +17,8 @@ */ package org.apache.beam.runners.dataflow.worker; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.UnsafeByteOperations; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.UnsafeByteOperations; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.primitives.Ints; class ByteArrayReader { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ByteStringCoder.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ByteStringCoder.java index d48d9ae77a4e..b9b1b45902c8 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ByteStringCoder.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ByteStringCoder.java @@ -23,7 +23,7 @@ import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.util.VarInt; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.ByteStreams; /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ChunkingShuffleBatchReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ChunkingShuffleBatchReader.java index f1924b147247..665e01f5cc97 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ChunkingShuffleBatchReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ChunkingShuffleBatchReader.java @@ -26,7 +26,7 @@ import org.apache.beam.runners.dataflow.worker.util.common.worker.ShuffleBatchReader; import org.apache.beam.runners.dataflow.worker.util.common.worker.ShuffleEntry; import org.apache.beam.runners.dataflow.worker.util.common.worker.ShufflePosition; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.checkerframework.checker.nullness.qual.Nullable; /** ChunkingShuffleBatchReader reads data from a shuffle dataset using a ShuffleReader. */ diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelper.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelper.java index 4280dbb6a5cb..c6d8d727ef4d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelper.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelper.java @@ -29,7 +29,7 @@ import org.apache.beam.runners.dataflow.worker.ExperimentContext.Experiment; import org.apache.beam.runners.dataflow.worker.logging.DataflowWorkerLoggingInitializer; import org.apache.beam.runners.dataflow.worker.logging.DataflowWorkerLoggingMDC; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.TextFormat; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.TextFormat; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; import org.checkerframework.checker.nullness.qual.Nullable; import org.conscrypt.OpenSSLProvider; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowParDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowParDoFnFactory.java index 3d4962691105..df692d83b3f4 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowParDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowParDoFnFactory.java @@ -55,7 +55,7 @@ import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSink.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSink.java index b86f48717890..b79de4f4320d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSink.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSink.java @@ -33,7 +33,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; 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.Maps; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubSink.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubSink.java index c1cc4d2bac62..c21591d91285 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubSink.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubSink.java @@ -37,7 +37,7 @@ import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index 1ad0ed08c081..d915b77995db 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -125,7 +125,7 @@ import org.apache.beam.sdk.util.Sleeper; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java index 83cf49112a8d..5b18e29293ed 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java @@ -64,7 +64,7 @@ import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcher.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcher.java index 4f585e1c01b6..526ccf5161d5 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcher.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcher.java @@ -50,7 +50,7 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Parser; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Parser; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReader.java index 27f3cb9735e8..a2aa994fe690 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReader.java @@ -27,7 +27,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillComputationKey.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillComputationKey.java index b77582503261..a01b1d297c22 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillComputationKey.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillComputationKey.java @@ -18,8 +18,8 @@ package org.apache.beam.runners.dataflow.worker; import com.google.auto.value.AutoValue; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.TextFormat; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.TextFormat; @AutoValue public abstract class WindmillComputationKey { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillNamespacePrefix.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillNamespacePrefix.java index e10f67e0381e..429dfa4a9e18 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillNamespacePrefix.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillNamespacePrefix.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.dataflow.worker; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; /** * A prefix for a Windmill state or timer tag to separate user state and timers from system state diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java index e9b3f252d457..1f26572941a0 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java @@ -41,7 +41,7 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.ValueWithRecordId; import org.apache.beam.sdk.values.ValueWithRecordId.ValueWithRecordIdCoder; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillTimerInternals.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillTimerInternals.java index 5d56186ad94c..cb397a32e552 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillTimerInternals.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillTimerInternals.java @@ -35,7 +35,7 @@ import org.apache.beam.sdk.util.ExposedByteArrayInputStream; import org.apache.beam.sdk.util.ExposedByteArrayOutputStream; import org.apache.beam.sdk.util.VarInt; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.HashBasedTable; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Table; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java index 2dc3494af5e2..8c086016ee95 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java @@ -61,7 +61,7 @@ import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.ValueWithRecordId; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; 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; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ShardedKey.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ShardedKey.java index 86433d9e6752..572125462a33 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ShardedKey.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ShardedKey.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.dataflow.worker.streaming; import com.google.auto.value.AutoValue; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; @AutoValue public abstract class ShardedKey { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ByteArrayShufflePosition.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ByteArrayShufflePosition.java index d74c2fb1b8b7..6d12c08d15fd 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ByteArrayShufflePosition.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ByteArrayShufflePosition.java @@ -20,8 +20,8 @@ import static com.google.api.client.util.Base64.decodeBase64; import static com.google.api.client.util.Base64.encodeBase64URLSafeString; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.UnsafeByteOperations; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.UnsafeByteOperations; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleEntryIterator.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleEntryIterator.java index c7ed7e88e825..d26c92057a23 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleEntryIterator.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleEntryIterator.java @@ -25,7 +25,7 @@ import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterator; import org.apache.beam.sdk.util.common.Reiterable; import org.apache.beam.sdk.util.common.Reiterator; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/KeyGroupedShuffleEntries.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/KeyGroupedShuffleEntries.java index 36b2fe650e0b..4a0b0ed05b36 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/KeyGroupedShuffleEntries.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/KeyGroupedShuffleEntries.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.dataflow.worker.util.common.worker; import org.apache.beam.sdk.util.common.Reiterable; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; /** A collection of ShuffleEntries, all with the same key. */ public class KeyGroupedShuffleEntries { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ShuffleEntry.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ShuffleEntry.java index 1a00722fa03d..65d891cc7f79 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ShuffleEntry.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ShuffleEntry.java @@ -19,7 +19,7 @@ import java.util.Arrays; import java.util.Objects; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.checkerframework.checker.nullness.qual.Nullable; /** Entry written to/read from a shuffle dataset. */ diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java index 4e47676989a6..baafc22e0303 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java @@ -32,9 +32,9 @@ import java.util.function.Supplier; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers.StreamObserverFactory; import org.apache.beam.sdk.util.BackOff; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Status; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.StatusRuntimeException; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Status; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.StatusRuntimeException; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/AppendableInputStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/AppendableInputStream.java index 6a0d0a63d5a9..98545a429461 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/AppendableInputStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/AppendableInputStream.java @@ -29,7 +29,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import javax.annotation.Nullable; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java index 9350b89f182a..b921160e1a90 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java @@ -38,8 +38,8 @@ import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers.StreamObserverFactory; import org.apache.beam.runners.dataflow.worker.windmill.client.throttling.ThrottleTimer; import org.apache.beam.sdk.util.BackOff; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDeadlineClientInterceptor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDeadlineClientInterceptor.java index 629006e23596..a2dd86279c96 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDeadlineClientInterceptor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDeadlineClientInterceptor.java @@ -18,11 +18,11 @@ package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; import java.util.concurrent.TimeUnit; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.CallOptions; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Channel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ClientCall; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ClientInterceptor; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.MethodDescriptor; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.CallOptions; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Channel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ClientCall; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ClientInterceptor; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.MethodDescriptor; /** * Intercepts outgoing calls and attaches a relative deadline to the call. Deadlines are absolute, 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 683f94eb71ee..a56097cc8136 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 @@ -43,8 +43,8 @@ import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.util.BackOff; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Suppliers; import org.joda.time.Instant; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java index b6600e04a09d..992c9fb589b2 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java @@ -53,7 +53,7 @@ import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers.StreamObserverFactory; import org.apache.beam.runners.dataflow.worker.windmill.client.throttling.ThrottleTimer; import org.apache.beam.sdk.util.BackOff; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkStream.java index d7d9bfddffb0..867180fb0d31 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkStream.java @@ -40,8 +40,8 @@ import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemReceiver; import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget; import org.apache.beam.sdk.util.BackOff; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStream.java index 35524dbd2eeb..3672f02c813f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStream.java @@ -32,7 +32,7 @@ import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers.StreamObserverFactory; import org.apache.beam.runners.dataflow.worker.windmill.client.throttling.ThrottleTimer; import org.apache.beam.sdk.util.BackOff; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServer.java index 9f0126a9cc69..64ee5ae0f969 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServer.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServer.java @@ -62,9 +62,9 @@ import org.apache.beam.sdk.util.BackOffUtils; import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.Sleeper; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Channel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.StatusRuntimeException; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Channel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.StatusRuntimeException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/auth/VendoredCredentialsAdapter.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/auth/VendoredCredentialsAdapter.java index 23f6fb801a4f..41fc07edd572 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/auth/VendoredCredentialsAdapter.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/auth/VendoredCredentialsAdapter.java @@ -32,7 +32,7 @@ * delegate to reduce maintenance burden. */ public class VendoredCredentialsAdapter - extends org.apache.beam.vendor.grpc.v1p54p0.com.google.auth.Credentials { + extends org.apache.beam.vendor.grpc.v1p60p1.com.google.auth.Credentials { private final com.google.auth.Credentials credentials; @@ -54,7 +54,7 @@ public Map> getRequestMetadata() throws IOException { public void getRequestMetadata( final URI uri, Executor executor, - final org.apache.beam.vendor.grpc.v1p54p0.com.google.auth.RequestMetadataCallback callback) { + final org.apache.beam.vendor.grpc.v1p60p1.com.google.auth.RequestMetadataCallback callback) { credentials.getRequestMetadata( uri, executor, new VendoredRequestMetadataCallbackAdapter(callback)); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/auth/VendoredRequestMetadataCallbackAdapter.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/auth/VendoredRequestMetadataCallbackAdapter.java index 8b1b695287e7..c2fdf1ac638b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/auth/VendoredRequestMetadataCallbackAdapter.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/auth/VendoredRequestMetadataCallbackAdapter.java @@ -31,11 +31,11 @@ public class VendoredRequestMetadataCallbackAdapter implements com.google.auth.RequestMetadataCallback { - private final org.apache.beam.vendor.grpc.v1p54p0.com.google.auth.RequestMetadataCallback + private final org.apache.beam.vendor.grpc.v1p60p1.com.google.auth.RequestMetadataCallback callback; VendoredRequestMetadataCallbackAdapter( - org.apache.beam.vendor.grpc.v1p54p0.com.google.auth.RequestMetadataCallback callback) { + org.apache.beam.vendor.grpc.v1p60p1.com.google.auth.RequestMetadataCallback callback) { this.callback = callback; } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java index 5fb22476ab3a..9d57df1af317 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java @@ -22,8 +22,8 @@ import java.util.concurrent.TimeoutException; import javax.annotation.concurrent.GuardedBy; import javax.annotation.concurrent.ThreadSafe; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.CallStreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.CallStreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/ForwardingClientResponseObserver.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/ForwardingClientResponseObserver.java index 007717d03b58..e1ef85699a7e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/ForwardingClientResponseObserver.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/ForwardingClientResponseObserver.java @@ -17,9 +17,9 @@ */ package org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.ClientCallStreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.ClientResponseObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.ClientCallStreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.ClientResponseObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; /** * A {@link ClientResponseObserver} which delegates all {@link StreamObserver} calls. diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/StreamObserverFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/StreamObserverFactory.java index e3f12687638d..cb4415bdab18 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/StreamObserverFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/StreamObserverFactory.java @@ -20,8 +20,8 @@ import java.util.function.Function; import org.apache.beam.sdk.fn.stream.AdvancingPhaser; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.CallStreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.CallStreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; /** * Uses {@link PipelineOptions} to configure which underlying {@link StreamObserver} implementation diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillChannelFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillChannelFactory.java index 48cf8ff3f761..484d433ca6e7 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillChannelFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillChannelFactory.java @@ -22,12 +22,12 @@ import java.util.concurrent.TimeUnit; import javax.net.ssl.SSLException; import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Channel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessChannelBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.netty.GrpcSslContexts; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.netty.NegotiationType; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.netty.NettyChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Channel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.netty.GrpcSslContexts; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.netty.NegotiationType; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.netty.NettyChannelBuilder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; /** Utility class used to create different RPC Channels. */ diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillStubFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillStubFactory.java index 0c7719b0bc13..e7a35b83be78 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillStubFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillStubFactory.java @@ -27,8 +27,8 @@ import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub; import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.auth.VendoredCredentialsAdapter; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.auth.MoreCallCredentials; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.auth.MoreCallCredentials; /** * Used to create stubs to talk to Streaming Engine. Stubs are either in-process for testing, or diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/StateTag.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/StateTag.java index 13c2a9e66baa..08275a2ec6c0 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/StateTag.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/StateTag.java @@ -19,7 +19,7 @@ import com.google.auto.value.AutoValue; import javax.annotation.Nullable; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Range; /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillBag.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillBag.java index 7cdb3776dfa1..b3719cc666dd 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillBag.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillBag.java @@ -31,7 +31,7 @@ import org.apache.beam.sdk.state.ReadableState; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.util.Weighted; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @SuppressWarnings({ diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillMap.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillMap.java index 43490a725ac4..9f027af0a870 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillMap.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillMap.java @@ -38,7 +38,7 @@ import org.apache.beam.sdk.state.ReadableStates; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.util.Weighted; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillMultimap.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillMultimap.java index 1c0b3df44c21..75f33e69e0be 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillMultimap.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillMultimap.java @@ -40,7 +40,7 @@ import org.apache.beam.sdk.state.ReadableState; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.util.Weighted; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillOrderedList.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillOrderedList.java index c92e2e93ddfe..11c197ef6e7a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillOrderedList.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillOrderedList.java @@ -36,7 +36,7 @@ import org.apache.beam.sdk.state.ReadableState; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.values.TimestampedValue; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; 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.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Range; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCache.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCache.java index 5a9e5443a506..ba59d1ae8147 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCache.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCache.java @@ -36,7 +36,7 @@ import org.apache.beam.runners.dataflow.worker.status.StatusDataProvider; import org.apache.beam.sdk.state.State; import org.apache.beam.sdk.util.Weighted; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Equivalence; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Cache; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReader.java index 637b838c7fe2..7fe08571d7b4 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReader.java @@ -54,7 +54,7 @@ import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.TimestampedValue; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Function; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateUtil.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateUtil.java index 3cac5c3c5724..114d4335bb1b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateUtil.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateUtil.java @@ -23,7 +23,7 @@ import org.apache.beam.runners.core.StateNamespace; import org.apache.beam.runners.core.StateTag; import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; class WindmillStateUtil { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillValue.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillValue.java index 1ea6e56435d2..923d166c823a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillValue.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillValue.java @@ -29,7 +29,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.state.ValueState; import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Futures; @SuppressWarnings({ diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillWatermarkHold.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillWatermarkHold.java index a800c2eb6dad..9d939c759d27 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillWatermarkHold.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillWatermarkHold.java @@ -30,7 +30,7 @@ import org.apache.beam.sdk.state.ReadableState; import org.apache.beam.sdk.state.WatermarkHoldState; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Optional; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Futures; import org.joda.time.Instant; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/harness/test/TestStreams.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/harness/test/TestStreams.java index 456f63e4532f..3729efb4e79b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/harness/test/TestStreams.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/harness/test/TestStreams.java @@ -19,8 +19,8 @@ import java.util.function.Consumer; import java.util.function.Supplier; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.CallStreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.CallStreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; /** Utility methods which enable testing of {@link StreamObserver}s. */ public class TestStreams { diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowMatchers.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowMatchers.java index 8a97215dc617..a3c2697cec37 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowMatchers.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowMatchers.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.dataflow.worker; import java.io.Serializable; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.hamcrest.Description; import org.hamcrest.TypeSafeMatcher; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelperTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelperTest.java index b2bf0c5aac05..de9d9cf7d155 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelperTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelperTest.java @@ -32,7 +32,7 @@ import org.apache.beam.runners.dataflow.worker.testing.RestoreDataflowLoggingMDC; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.RestoreSystemProperties; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.TextFormat; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.TextFormat; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderTest.java index 49f53ef8e4d0..56603c98e192 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderTest.java @@ -80,7 +80,7 @@ import org.apache.beam.sdk.util.common.Reiterable; import org.apache.beam.sdk.util.common.Reiterator; import org.apache.beam.sdk.values.KV; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSinkTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSinkTest.java index d0af1caa5f7a..782ce87568cb 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSinkTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSinkTest.java @@ -33,7 +33,7 @@ import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.joda.time.Instant; import org.junit.Before; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubReaderTest.java index 0ed7c5ef31ce..f77166c41583 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubReaderTest.java @@ -31,7 +31,7 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.joda.time.Instant; import org.junit.Before; import org.junit.Test; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubSinkTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubSinkTest.java index 98f960e0f858..6ce48b3bab0c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubSinkTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubSinkTest.java @@ -29,7 +29,7 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.joda.time.Instant; import org.junit.Before; import org.junit.Test; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderCacheTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderCacheTest.java index 861e786642ca..cecef25c506c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderCacheTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderCacheTest.java @@ -26,7 +26,7 @@ import java.io.IOException; import java.util.concurrent.TimeUnit; import org.apache.beam.sdk.io.UnboundedSource; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Stopwatch; import org.joda.time.Duration; import org.junit.Before; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkTest.java index 530744af5b0a..fd0d12e4078f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkTest.java @@ -38,7 +38,7 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java index 9526c96fd04e..5de67a371a65 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java @@ -165,8 +165,8 @@ import org.apache.beam.sdk.values.ValueWithRecordId; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.TextFormat; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.TextFormat; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheStats; 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; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowFnsTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowFnsTest.java index 449ed9b178ea..9646c8500b51 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowFnsTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowFnsTest.java @@ -76,7 +76,7 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java index 646859be4f5c..8b2effa08ef1 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java @@ -47,7 +47,7 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Before; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java index 451ec649aa23..40b22a729cba 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java @@ -67,7 +67,7 @@ import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.hamcrest.Matchers; import org.joda.time.Duration; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java index 07c1080d8f23..e46b7ce274c8 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java @@ -63,7 +63,7 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.hamcrest.Matchers; import org.joda.time.Duration; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java index d4fee95ead54..0f0ba199632d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java @@ -49,7 +49,7 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; 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.Sets; import org.hamcrest.Matchers; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/TestShuffleReader.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/TestShuffleReader.java index c6a89f851db3..1a62b62489b5 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/TestShuffleReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/TestShuffleReader.java @@ -29,7 +29,7 @@ import org.apache.beam.runners.dataflow.worker.util.common.worker.ShuffleEntryReader; import org.apache.beam.runners.dataflow.worker.util.common.worker.ShufflePosition; import org.apache.beam.sdk.util.common.Reiterator; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.checkerframework.checker.nullness.qual.Nullable; /** A fake implementation of a ShuffleEntryReader, for testing. */ diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/TestShuffleWriter.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/TestShuffleWriter.java index 9756dc12dd81..02086b968e5f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/TestShuffleWriter.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/TestShuffleWriter.java @@ -23,7 +23,7 @@ import java.util.ArrayList; import java.util.List; import org.apache.beam.runners.dataflow.worker.util.common.worker.ShuffleEntry; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; /** A fake implementation of a ShuffleEntryWriter, for testing. */ public class TestShuffleWriter implements ShuffleWriter { diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItemTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItemTest.java index f604c6f5ef76..c711a94cf985 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItemTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItemTest.java @@ -40,7 +40,7 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Before; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBaseTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBaseTest.java index 3174d0aa1cdc..1ada870312b8 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBaseTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBaseTest.java @@ -26,7 +26,7 @@ import java.util.List; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java index b488641d1ca5..eb94d79692b4 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java @@ -117,7 +117,7 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.ValueWithRecordId; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingHandlerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingHandlerTest.java index b7d6a53f37d5..43503c6adf64 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingHandlerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingHandlerTest.java @@ -37,7 +37,7 @@ import org.apache.beam.runners.dataflow.worker.NameContextsForTests; import org.apache.beam.runners.dataflow.worker.TestOperationContext.TestDataflowExecutionState; import org.apache.beam.runners.dataflow.worker.testing.RestoreDataflowLoggingMDC; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Timestamp; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Timestamp; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; import org.junit.After; import org.junit.Before; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java index de30fd0f8d5d..b384bb03185d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java @@ -38,7 +38,7 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.HeartbeatRequest; import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Instant; import org.junit.Before; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputStateFetcherTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputStateFetcherTest.java index d1e2b3ca60ba..ace5330dd05c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputStateFetcherTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputStateFetcherTest.java @@ -48,7 +48,7 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Cache; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/BatchingShuffleEntryReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/BatchingShuffleEntryReaderTest.java index 0a8d644c61f7..4f8afb4db808 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/BatchingShuffleEntryReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/BatchingShuffleEntryReaderTest.java @@ -30,7 +30,7 @@ import java.util.Collections; import java.util.List; import org.apache.beam.sdk.util.common.Reiterator; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleEntryIteratorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleEntryIteratorTest.java index 4a4b37d93316..2421d7faf824 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleEntryIteratorTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleEntryIteratorTest.java @@ -41,7 +41,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.util.common.Reiterator; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ShuffleEntryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ShuffleEntryTest.java index 12a6f17c5b2e..904d1e10167a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ShuffleEntryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ShuffleEntryTest.java @@ -23,7 +23,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStreamTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStreamTest.java index 253d6ff3a48f..5d17795b28fd 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStreamTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStreamTest.java @@ -42,13 +42,13 @@ import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers.StreamObserverFactory; import org.apache.beam.runners.dataflow.worker.windmill.client.throttling.ThrottleTimer; import org.apache.beam.sdk.util.FluentBackoff; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Server; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessChannelBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessServerBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.testing.GrpcCleanupRule; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.util.MutableHandlerRegistry; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessServerBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.testing.GrpcCleanupRule; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.util.MutableHandlerRegistry; 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.junit.After; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java index 0ea253027679..15610462e014 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java @@ -73,21 +73,21 @@ import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.CommitWorkStream; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkStream; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.CallOptions; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Channel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ClientCall; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ClientInterceptor; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ClientInterceptors; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Deadline; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.MethodDescriptor; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Server; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Status; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.StatusRuntimeException; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessChannelBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessServerBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.util.MutableHandlerRegistry; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.CallOptions; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Channel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ClientCall; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ClientInterceptor; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ClientInterceptors; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Deadline; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.MethodDescriptor; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Status; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.StatusRuntimeException; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessServerBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.util.MutableHandlerRegistry; import org.checkerframework.checker.nullness.qual.Nullable; import org.hamcrest.Matchers; import org.joda.time.Instant; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java index 166b2cd0f031..4831726c49e0 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java @@ -50,13 +50,13 @@ import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget; import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudgetDistributor; import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudgetRefresher; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Server; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessServerBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessSocketAddress; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.testing.GrpcCleanupRule; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.util.MutableHandlerRegistry; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessServerBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessSocketAddress; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.testing.GrpcCleanupRule; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.util.MutableHandlerRegistry; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableCollection; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSenderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSenderTest.java index 5fd1814e511f..2532fca51549 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSenderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSenderTest.java @@ -36,9 +36,9 @@ import org.apache.beam.runners.dataflow.worker.windmill.client.throttling.ThrottleTimer; import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemProcessor; import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessChannelBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.testing.GrpcCleanupRule; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.testing.GrpcCleanupRule; import org.junit.After; import org.junit.Before; import org.junit.Rule; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCacheTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCacheTest.java index 35d01aaffb8a..26448d381373 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCacheTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCacheTest.java @@ -31,7 +31,7 @@ import org.apache.beam.sdk.state.State; import org.apache.beam.sdk.state.StateSpec; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; import org.junit.Before; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java index e8eeff3b1d14..19223124dda3 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java @@ -83,7 +83,7 @@ import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.TimestampedValue; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ArrayListMultimap; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java index 7ef74639bb55..73cbb361f7b4 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java @@ -46,7 +46,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.values.TimestampedValue; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Range; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributorTest.java index 54b605efbf3c..249642aa6d1e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributorTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributorTest.java @@ -32,9 +32,9 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill.JobHeader; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcWindmillStreamFactory; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.WindmillStreamSender; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessChannelBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.testing.GrpcCleanupRule; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.testing.GrpcCleanupRule; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.After; import org.junit.Before; diff --git a/runners/java-fn-execution/build.gradle b/runners/java-fn-execution/build.gradle index 02795add47d4..e055bc991e2a 100644 --- a/runners/java-fn-execution/build.gradle +++ b/runners/java-fn-execution/build.gradle @@ -32,7 +32,7 @@ dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(":sdks:java:fn-execution") implementation project(":runners:core-construction-java") - implementation library.java.vendored_grpc_1_54_0 + implementation library.java.vendored_grpc_1_60_1 implementation library.java.slf4j_api implementation project(path: ":model:job-management", configuration: "shadow") implementation library.java.joda_time diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalService.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalService.java index 5668870bd749..0fa4082b4861 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalService.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalService.java @@ -30,10 +30,10 @@ import org.apache.beam.runners.core.construction.DefaultArtifactResolver; import org.apache.beam.sdk.fn.server.FnService; import org.apache.beam.sdk.io.FileSystems; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Status; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.StatusException; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Status; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.StatusException; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; /** An {@link ArtifactRetrievalService} that uses {@link FileSystems} as its backing storage. */ @SuppressWarnings({ diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingService.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingService.java index 092df738ee6e..a8e5e2ab6a88 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingService.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingService.java @@ -51,11 +51,11 @@ import org.apache.beam.sdk.io.fs.ResolveOptions; import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.util.MimeTypes; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Status; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.StatusException; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Status; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.StatusException; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleCheckpointHandlers.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleCheckpointHandlers.java index c1a9ae0d6071..07906e25bd1e 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleCheckpointHandlers.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleCheckpointHandlers.java @@ -108,7 +108,7 @@ public void onCheckpoint(ProcessBundleResponse response) { // Calculate the watermark hold for the timer. long outputTimestamp = BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis(); if (!residual.getApplication().getOutputWatermarksMap().isEmpty()) { - for (org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Timestamp outputWatermark : + for (org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Timestamp outputWatermark : residual.getApplication().getOutputWatermarksMap().values()) { outputTimestamp = Math.min(outputTimestamp, outputWatermark.getSeconds() * 1000); } diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/FnApiControlClient.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/FnApiControlClient.java index 944ba32fcdcf..4d52d7918e73 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/FnApiControlClient.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/FnApiControlClient.java @@ -29,9 +29,9 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.model.fnexecution.v1.BeamFnApi.InstructionRequest; import org.apache.beam.sdk.fn.stream.SynchronizedStreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Status; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.StatusRuntimeException; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Status; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.StatusRuntimeException; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientPoolService.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientPoolService.java index a2c61f3363b3..ad2ff0e924be 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientPoolService.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientPoolService.java @@ -28,9 +28,9 @@ import org.apache.beam.model.fnexecution.v1.BeamFnControlGrpc; import org.apache.beam.sdk.fn.server.FnService; import org.apache.beam.sdk.fn.server.HeaderAccessor; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Status; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.StatusException; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Status; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.StatusException; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java index 34c95335ad3e..c7e0cd884c4e 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java @@ -58,7 +58,7 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.KV; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; 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.ImmutableTable; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/data/GrpcDataService.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/data/GrpcDataService.java index 15a17e98fee2..7ce8427203ac 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/data/GrpcDataService.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/data/GrpcDataService.java @@ -33,7 +33,7 @@ import org.apache.beam.sdk.fn.server.FnService; import org.apache.beam.sdk.fn.stream.OutboundObserverFactory; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.SettableFuture; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ExternalEnvironmentFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ExternalEnvironmentFactory.java index e6d30f6403f7..461ccceb5a52 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ExternalEnvironmentFactory.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ExternalEnvironmentFactory.java @@ -35,7 +35,7 @@ import org.apache.beam.sdk.fn.channel.ManagedChannelFactory; import org.apache.beam.sdk.fn.server.GrpcFnServer; import org.apache.beam.sdk.fn.server.ServerFactory; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/logging/GrpcLoggingService.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/logging/GrpcLoggingService.java index 37e59f5e58e5..97d0a9b2f3f2 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/logging/GrpcLoggingService.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/logging/GrpcLoggingService.java @@ -24,7 +24,7 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi.LogControl; import org.apache.beam.model.fnexecution.v1.BeamFnLoggingGrpc; import org.apache.beam.sdk.fn.server.FnService; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/provisioning/JobInfo.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/provisioning/JobInfo.java index 844b09663b7f..d8d7cf689353 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/provisioning/JobInfo.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/provisioning/JobInfo.java @@ -20,7 +20,7 @@ import com.google.auto.value.AutoValue; import java.io.Serializable; import org.apache.beam.model.fnexecution.v1.ProvisionApi; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Struct; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; /** * A subset of {@link org.apache.beam.model.fnexecution.v1.ProvisionApi.ProvisionInfo} that diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/provisioning/StaticGrpcProvisionService.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/provisioning/StaticGrpcProvisionService.java index 41f12fe486e2..ba72ba523341 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/provisioning/StaticGrpcProvisionService.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/provisioning/StaticGrpcProvisionService.java @@ -28,7 +28,7 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.sdk.fn.server.FnService; import org.apache.beam.sdk.fn.server.HeaderAccessor; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; /** * A {@link ProvisionServiceImplBase provision service} that returns a static response to all calls. diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/GrpcStateService.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/GrpcStateService.java index 42fbe58fddd7..34d5ef74bd3a 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/GrpcStateService.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/GrpcStateService.java @@ -28,8 +28,8 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateResponse; import org.apache.beam.model.fnexecution.v1.BeamFnStateGrpc; import org.apache.beam.sdk.fn.server.FnService; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.ServerCallStreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.ServerCallStreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; /** An implementation of the Beam Fn State service. */ public class GrpcStateService extends BeamFnStateGrpc.BeamFnStateImplBase diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlers.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlers.java index 0de40dc70365..e2c45850dba9 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlers.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlers.java @@ -53,7 +53,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.common.Reiterable; import org.apache.beam.sdk.values.KV; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/status/BeamWorkerStatusGrpcService.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/status/BeamWorkerStatusGrpcService.java index 1c3e453f893f..68ca36f8a369 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/status/BeamWorkerStatusGrpcService.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/status/BeamWorkerStatusGrpcService.java @@ -35,7 +35,7 @@ import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor; import org.apache.beam.sdk.fn.server.FnService; import org.apache.beam.sdk.fn.server.HeaderAccessor; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; 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.ImmutableSet; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/status/WorkerStatusClient.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/status/WorkerStatusClient.java index 92c3627006f4..db470ffebd99 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/status/WorkerStatusClient.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/status/WorkerStatusClient.java @@ -29,7 +29,7 @@ import org.apache.beam.sdk.fn.IdGenerator; import org.apache.beam.sdk.fn.IdGenerators; import org.apache.beam.sdk.fn.stream.SynchronizedStreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/PipelineTranslatorUtils.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/PipelineTranslatorUtils.java index eaf157e0bd67..fbd3230310f8 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/PipelineTranslatorUtils.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/PipelineTranslatorUtils.java @@ -43,7 +43,7 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BiMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableBiMap; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/ByteStringCoder.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/ByteStringCoder.java index 4319c23fcb2d..8e3ddb6f425f 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/ByteStringCoder.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/ByteStringCoder.java @@ -26,8 +26,8 @@ import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.util.VarInt; import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.UnsafeByteOperations; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.UnsafeByteOperations; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.ByteStreams; /** diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/GrpcContextHeaderAccessorProviderTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/GrpcContextHeaderAccessorProviderTest.java index 8455b23fee90..6590ccae143f 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/GrpcContextHeaderAccessorProviderTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/GrpcContextHeaderAccessorProviderTest.java @@ -26,16 +26,16 @@ import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor; import org.apache.beam.sdk.fn.server.GrpcContextHeaderAccessorProvider; import org.apache.beam.sdk.fn.server.InProcessServerFactory; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.CallOptions; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Channel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ClientCall; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ClientInterceptor; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ForwardingClientCall.SimpleForwardingClientCall; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Metadata; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.MethodDescriptor; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessChannelBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.testing.GrpcCleanupRule; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.CallOptions; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Channel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ClientCall; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ClientInterceptor; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ForwardingClientCall.SimpleForwardingClientCall; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Metadata; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.MethodDescriptor; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.testing.GrpcCleanupRule; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Assert; import org.junit.Rule; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/ServerFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/ServerFactoryTest.java index 0671fd1eb3ca..2dfa49d97989 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/ServerFactoryTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/ServerFactoryTest.java @@ -44,12 +44,12 @@ import org.apache.beam.sdk.fn.channel.ManagedChannelFactory; import org.apache.beam.sdk.fn.server.ServerFactory; import org.apache.beam.sdk.fn.test.TestStreams; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Server; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.CallStreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.testing.GrpcCleanupRule; -import org.apache.beam.vendor.grpc.v1p54p0.io.netty.channel.epoll.Epoll; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.CallStreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.testing.GrpcCleanupRule; +import org.apache.beam.vendor.grpc.v1p60p1.io.netty.channel.epoll.Epoll; 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.net.HostAndPort; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Uninterruptibles; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalServiceTest.java index b7b7b1cfa58e..d6b48a936135 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalServiceTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalServiceTest.java @@ -28,11 +28,11 @@ import org.apache.beam.model.jobmanagement.v1.ArtifactApi; import org.apache.beam.model.jobmanagement.v1.ArtifactRetrievalServiceGrpc; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessChannelBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessServerBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.testing.GrpcCleanupRule; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessServerBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.testing.GrpcCleanupRule; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; 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; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingServiceTest.java index 51c0d99e5933..de5ecf480bb4 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingServiceTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingServiceTest.java @@ -29,12 +29,12 @@ import org.apache.beam.model.jobmanagement.v1.ArtifactRetrievalServiceGrpc; import org.apache.beam.model.jobmanagement.v1.ArtifactStagingServiceGrpc; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessChannelBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessServerBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.testing.GrpcCleanupRule; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessServerBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.testing.GrpcCleanupRule; 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; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactoryTest.java index 823af895d655..799644f66b15 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactoryTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactoryTest.java @@ -69,8 +69,8 @@ import org.apache.beam.sdk.options.ExperimentalOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.PortablePipelineOptions; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Struct; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.junit.Assert; import org.junit.Before; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientPoolServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientPoolServiceTest.java index b605ae407e19..4ae804157478 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientPoolServiceTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientPoolServiceTest.java @@ -41,10 +41,10 @@ import org.apache.beam.sdk.fn.server.InProcessServerFactory; import org.apache.beam.sdk.fn.test.TestStreams; import org.apache.beam.sdk.util.MoreFutures; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Status; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.StatusException; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessChannelBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Status; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.StatusException; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.junit.After; import org.junit.Assert; import org.junit.Before; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientTest.java index 293f847bcc18..77f8b0bef6ac 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientTest.java @@ -33,7 +33,7 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi.InstructionRequest; import org.apache.beam.model.fnexecution.v1.BeamFnApi.InstructionResponse; import org.apache.beam.sdk.util.MoreFutures; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.junit.Before; import org.junit.Rule; import org.junit.Test; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java index d827421d2e81..fff089b8c96c 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java @@ -142,7 +142,7 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Optional; 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.Iterables; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java index 2168854c863d..97f8eeac2f73 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java @@ -88,7 +88,7 @@ import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; 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.Iterables; import org.junit.Before; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/data/GrpcDataServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/data/GrpcDataServiceTest.java index b06c38db7e74..020c131cdc3c 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/data/GrpcDataServiceTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/data/GrpcDataServiceTest.java @@ -50,10 +50,10 @@ import org.apache.beam.sdk.fn.test.TestStreams; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessChannelBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.junit.Rule; import org.junit.Test; import org.junit.rules.Timeout; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/logging/GrpcLoggingServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/logging/GrpcLoggingServiceTest.java index 2fcbde5eaa31..f6b1495b0e68 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/logging/GrpcLoggingServiceTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/logging/GrpcLoggingServiceTest.java @@ -38,9 +38,9 @@ import org.apache.beam.sdk.fn.server.GrpcFnServer; import org.apache.beam.sdk.fn.server.InProcessServerFactory; import org.apache.beam.sdk.fn.test.TestStreams; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessChannelBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.junit.Rule; import org.junit.Test; import org.junit.rules.Timeout; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/provisioning/StaticGrpcProvisionServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/provisioning/StaticGrpcProvisionServiceTest.java index f0d37fe34d3c..72adc49d283f 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/provisioning/StaticGrpcProvisionServiceTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/provisioning/StaticGrpcProvisionServiceTest.java @@ -29,11 +29,11 @@ import org.apache.beam.sdk.fn.server.GrpcContextHeaderAccessorProvider; import org.apache.beam.sdk.fn.server.GrpcFnServer; import org.apache.beam.sdk.fn.server.InProcessServerFactory; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ListValue; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.NullValue; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Struct; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Value; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ListValue; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.NullValue; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Value; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessChannelBuilder; import org.junit.Rule; import org.junit.Test; import org.junit.rules.Timeout; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/state/GrpcStateServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/state/GrpcStateServiceTest.java index fffc0321ca2c..cc6279f6a10a 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/state/GrpcStateServiceTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/state/GrpcStateServiceTest.java @@ -31,8 +31,8 @@ import java.util.concurrent.TimeUnit; import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.sdk.fn.test.TestStreams; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.junit.Before; import org.junit.Rule; import org.junit.Test; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlersTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlersTest.java index 3ef70756760a..2515a4afd644 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlersTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlersTest.java @@ -37,7 +37,7 @@ import org.apache.beam.runners.fnexecution.control.ProcessBundleDescriptors; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.junit.Rule; import org.junit.Test; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/status/BeamWorkerStatusGrpcServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/status/BeamWorkerStatusGrpcServiceTest.java index 49ced362c29a..c4b47da37237 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/status/BeamWorkerStatusGrpcServiceTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/status/BeamWorkerStatusGrpcServiceTest.java @@ -39,10 +39,10 @@ import org.apache.beam.sdk.fn.server.GrpcContextHeaderAccessorProvider; import org.apache.beam.sdk.fn.server.GrpcFnServer; import org.apache.beam.sdk.fn.server.InProcessServerFactory; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessChannelBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.testing.GrpcCleanupRule; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.testing.GrpcCleanupRule; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; import org.junit.After; import org.junit.Before; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/status/WorkerStatusClientTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/status/WorkerStatusClientTest.java index 0a250ef0b8d3..4d0511af0d6b 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/status/WorkerStatusClientTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/status/WorkerStatusClientTest.java @@ -26,7 +26,7 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.model.fnexecution.v1.BeamFnApi.WorkerStatusRequest; import org.apache.beam.model.fnexecution.v1.BeamFnApi.WorkerStatusResponse; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/ByteStringCoderTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/ByteStringCoderTest.java index 28144e68118a..2479482772df 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/ByteStringCoderTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/ByteStringCoderTest.java @@ -31,7 +31,7 @@ import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Rule; import org.junit.Test; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java index 5a775c2c98ec..7a2d54b55064 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java @@ -87,8 +87,8 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableBiMap; diff --git a/runners/java-job-service/build.gradle b/runners/java-job-service/build.gradle index 3c3ff3ad92eb..a1b4aba2e908 100644 --- a/runners/java-job-service/build.gradle +++ b/runners/java-job-service/build.gradle @@ -36,7 +36,7 @@ dependencies { implementation library.java.jackson_databind implementation library.java.joda_time implementation library.java.commons_compress - implementation library.java.vendored_grpc_1_54_0 + implementation library.java.vendored_grpc_1_60_1 implementation library.java.slf4j_api implementation library.java.args4j testImplementation library.java.junit diff --git a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/InMemoryJobService.java b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/InMemoryJobService.java index 41e6135b9207..99ae19e503be 100644 --- a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/InMemoryJobService.java +++ b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/InMemoryJobService.java @@ -57,11 +57,11 @@ import org.apache.beam.sdk.fn.stream.SynchronizedStreamObserver; import org.apache.beam.sdk.function.ThrowingConsumer; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Struct; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Status; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.StatusException; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.StatusRuntimeException; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Status; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.StatusException; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.StatusRuntimeException; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobInvocation.java b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobInvocation.java index b7a5dbf03518..a1d436e5f0ff 100644 --- a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobInvocation.java +++ b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobInvocation.java @@ -35,7 +35,7 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline; import org.apache.beam.runners.fnexecution.provisioning.JobInfo; import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.util.Timestamps; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.util.Timestamps; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.FutureCallback; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Futures; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ListenableFuture; diff --git a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobInvoker.java b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobInvoker.java index 96157421f794..24f82e752b63 100644 --- a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobInvoker.java +++ b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobInvoker.java @@ -22,7 +22,7 @@ import java.util.concurrent.ThreadFactory; import javax.annotation.Nullable; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Struct; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ListeningExecutorService; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.MoreExecutors; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; diff --git a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobPreparation.java b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobPreparation.java index 6082d64e77d0..e31d3954e06e 100644 --- a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobPreparation.java +++ b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobPreparation.java @@ -19,7 +19,7 @@ import com.google.auto.value.AutoValue; import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Struct; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; /** A job that has been prepared, but not invoked. */ @AutoValue diff --git a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarCreator.java b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarCreator.java index b5463354bed5..affc4cc0b09b 100644 --- a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarCreator.java +++ b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarCreator.java @@ -46,8 +46,8 @@ import org.apache.beam.sdk.io.ClassLoaderFileSystem; import org.apache.beam.sdk.metrics.MetricResults; import org.apache.beam.sdk.options.PortablePipelineOptions; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.MessageOrBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.util.JsonFormat; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.MessageOrBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.util.JsonFormat; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; 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.io.ByteStreams; diff --git a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarUtils.java b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarUtils.java index 02333021e53e..b4ace13cc189 100644 --- a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarUtils.java +++ b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarUtils.java @@ -27,9 +27,9 @@ import java.util.jar.JarEntry; import java.util.jar.JarOutputStream; import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Message; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Struct; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.util.JsonFormat; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Message; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.util.JsonFormat; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.ByteStreams; /** diff --git a/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/InMemoryJobServiceTest.java b/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/InMemoryJobServiceTest.java index 45d164c6e605..482a6be569ee 100644 --- a/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/InMemoryJobServiceTest.java +++ b/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/InMemoryJobServiceTest.java @@ -37,9 +37,9 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.runners.fnexecution.artifact.ArtifactStagingService; import org.apache.beam.sdk.fn.server.GrpcFnServer; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Struct; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.StatusException; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.StatusException; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.junit.Before; import org.junit.Test; diff --git a/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/JobInvocationTest.java b/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/JobInvocationTest.java index e45db3a0119c..4793ffcbc8af 100644 --- a/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/JobInvocationTest.java +++ b/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/JobInvocationTest.java @@ -33,7 +33,7 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.metrics.MetricResults; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Struct; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ListeningExecutorService; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.MoreExecutors; import org.joda.time.Duration; diff --git a/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarCreatorTest.java b/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarCreatorTest.java index 8acf07d3c795..9adb2317358b 100644 --- a/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarCreatorTest.java +++ b/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarCreatorTest.java @@ -41,7 +41,7 @@ import java.util.jar.Manifest; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Before; import org.junit.Test; diff --git a/runners/portability/java/build.gradle b/runners/portability/java/build.gradle index bd7cb5f5be81..1cb61bc30cc8 100644 --- a/runners/portability/java/build.gradle +++ b/runners/portability/java/build.gradle @@ -41,7 +41,7 @@ dependencies { permitUnusedDeclared library.java.hamcrest implementation library.java.joda_time implementation library.java.slf4j_api - implementation library.java.vendored_grpc_1_54_0 + implementation library.java.vendored_grpc_1_60_1 implementation library.java.vendored_guava_32_1_2_jre testImplementation project(path: ":runners:core-construction-java", configuration: "testRuntimeMigration") diff --git a/runners/portability/java/src/main/java/org/apache/beam/runners/portability/JobServicePipelineResult.java b/runners/portability/java/src/main/java/org/apache/beam/runners/portability/JobServicePipelineResult.java index eba97bb5f1cf..9b286273a367 100644 --- a/runners/portability/java/src/main/java/org/apache/beam/runners/portability/JobServicePipelineResult.java +++ b/runners/portability/java/src/main/java/org/apache/beam/runners/portability/JobServicePipelineResult.java @@ -33,7 +33,7 @@ import org.apache.beam.model.jobmanagement.v1.JobServiceGrpc.JobServiceBlockingStub; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.metrics.MetricResults; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; import org.slf4j.Logger; diff --git a/runners/portability/java/src/main/java/org/apache/beam/runners/portability/PortableRunner.java b/runners/portability/java/src/main/java/org/apache/beam/runners/portability/PortableRunner.java index 3a3034d92263..b657b8bb49a1 100644 --- a/runners/portability/java/src/main/java/org/apache/beam/runners/portability/PortableRunner.java +++ b/runners/portability/java/src/main/java/org/apache/beam/runners/portability/PortableRunner.java @@ -50,8 +50,8 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsValidator; import org.apache.beam.sdk.options.PortablePipelineOptions; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.slf4j.Logger; diff --git a/runners/portability/java/src/main/java/org/apache/beam/runners/portability/testing/TestJobService.java b/runners/portability/java/src/main/java/org/apache/beam/runners/portability/testing/TestJobService.java index 66f20194aeb5..a6bf32d09b9b 100644 --- a/runners/portability/java/src/main/java/org/apache/beam/runners/portability/testing/TestJobService.java +++ b/runners/portability/java/src/main/java/org/apache/beam/runners/portability/testing/TestJobService.java @@ -27,7 +27,7 @@ import org.apache.beam.model.jobmanagement.v1.JobApi.RunJobResponse; import org.apache.beam.model.jobmanagement.v1.JobServiceGrpc.JobServiceImplBase; import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; /** * A JobService for tests. diff --git a/runners/portability/java/src/test/java/org/apache/beam/runners/portability/PortableRunnerTest.java b/runners/portability/java/src/test/java/org/apache/beam/runners/portability/PortableRunnerTest.java index c3895e4c35cf..25353437a2ec 100644 --- a/runners/portability/java/src/test/java/org/apache/beam/runners/portability/PortableRunnerTest.java +++ b/runners/portability/java/src/test/java/org/apache/beam/runners/portability/PortableRunnerTest.java @@ -45,10 +45,10 @@ import org.apache.beam.sdk.options.PortablePipelineOptions; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Server; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessServerBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.testing.GrpcCleanupRule; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessServerBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.testing.GrpcCleanupRule; 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.io.ByteStreams; import org.joda.time.Duration; diff --git a/runners/samza/build.gradle b/runners/samza/build.gradle index 1dc4332f8ad4..4e410fedeccc 100644 --- a/runners/samza/build.gradle +++ b/runners/samza/build.gradle @@ -66,7 +66,7 @@ dependencies { implementation "org.apache.samza:samza-yarn_2.11:$samza_version" compileOnly library.java.error_prone_annotations runtimeOnly "org.apache.kafka:kafka-clients:2.0.1" - implementation library.java.vendored_grpc_1_54_0 + implementation library.java.vendored_grpc_1_60_1 implementation project(path: ":model:fn-execution", configuration: "shadow") implementation project(path: ":model:job-management", configuration: "shadow") implementation project(path: ":model:pipeline", configuration: "shadow") diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaJobInvoker.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaJobInvoker.java index 5cd4a8ede33d..9c3bdcf4c887 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaJobInvoker.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaJobInvoker.java @@ -26,7 +26,7 @@ import org.apache.beam.runners.jobsubmission.JobInvoker; import org.apache.beam.runners.jobsubmission.PortablePipelineJarCreator; import org.apache.beam.runners.jobsubmission.PortablePipelineRunner; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Struct; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; 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.util.concurrent.ListeningExecutorService; import org.slf4j.Logger; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaStateRequestHandlers.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaStateRequestHandlers.java index 5af189b7e102..f7ef0e5764ba 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaStateRequestHandlers.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaStateRequestHandlers.java @@ -40,7 +40,7 @@ import org.apache.beam.sdk.state.BagState; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.samza.context.TaskContext; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTestStreamTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTestStreamTranslator.java index 77f3b885a09d..06324617b1f5 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTestStreamTranslator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTestStreamTranslator.java @@ -34,7 +34,7 @@ import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.samza.operators.KV; import org.apache.samza.serializers.KVSerde; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/WindowAssignTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/WindowAssignTranslator.java index 2b9fbabb0ba7..32eeaa18a5f6 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/WindowAssignTranslator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/WindowAssignTranslator.java @@ -28,7 +28,7 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.apache.samza.operators.MessageStream; /** diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/PipelineJsonRenderer.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/PipelineJsonRenderer.java index 4c42daab6833..fbdf3c1197b4 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/PipelineJsonRenderer.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/PipelineJsonRenderer.java @@ -44,9 +44,9 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.gson.JsonArray; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.gson.JsonObject; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.gson.JsonParser; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.gson.JsonArray; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.gson.JsonObject; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.gson.JsonParser; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; import org.apache.samza.config.Config; diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/SamzaMetricsBundleProgressHandlerTest.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/SamzaMetricsBundleProgressHandlerTest.java index bc9e7fc58e28..21f67801a589 100644 --- a/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/SamzaMetricsBundleProgressHandlerTest.java +++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/SamzaMetricsBundleProgressHandlerTest.java @@ -34,7 +34,7 @@ import org.apache.beam.runners.core.metrics.MonitoringInfoConstants; import org.apache.beam.runners.samza.metrics.SamzaMetricsContainer; import org.apache.beam.sdk.metrics.MetricName; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.samza.metrics.MetricsRegistryMap; import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/runners/spark/spark_runner.gradle b/runners/spark/spark_runner.gradle index c336b9966b3b..3da819ad2f37 100644 --- a/runners/spark/spark_runner.gradle +++ b/runners/spark/spark_runner.gradle @@ -173,7 +173,7 @@ dependencies { implementation project(path: ":model:fn-execution", configuration: "shadow") implementation project(path: ":model:job-management", configuration: "shadow") implementation project(":sdks:java:fn-execution") - implementation library.java.vendored_grpc_1_54_0 + implementation library.java.vendored_grpc_1_60_1 implementation library.java.vendored_guava_32_1_2_jre spark.components.each { component -> provided "$component:$spark_version" diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkJobInvoker.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkJobInvoker.java index 2d7767ee3bba..dca208c7a1ed 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkJobInvoker.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkJobInvoker.java @@ -26,7 +26,7 @@ import org.apache.beam.runners.jobsubmission.PortablePipelineJarCreator; import org.apache.beam.runners.jobsubmission.PortablePipelineRunner; import org.apache.beam.sdk.options.PortablePipelineOptions; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Struct; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; 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.util.concurrent.ListeningExecutorService; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java index 159cb32a8c5e..e914f0b462d8 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java @@ -52,7 +52,7 @@ import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.metrics.MetricsOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Struct; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.spark.api.java.JavaSparkContext; diff --git a/runners/twister2/build.gradle b/runners/twister2/build.gradle index c9a5e526f337..2db020a7536a 100644 --- a/runners/twister2/build.gradle +++ b/runners/twister2/build.gradle @@ -41,7 +41,7 @@ dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") implementation library.java.jackson_annotations implementation library.java.joda_time - implementation library.java.vendored_grpc_1_54_0 + implementation library.java.vendored_grpc_1_60_1 implementation library.java.slf4j_api implementation "org.mortbay.jetty:jetty-util:6.1.26" implementation "org.twister2:comms-api-java:$twister2_version" diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/AssignWindowsFunction.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/AssignWindowsFunction.java index db27b61c269d..c2f078adc73f 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/AssignWindowsFunction.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/AssignWindowsFunction.java @@ -34,7 +34,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; /** Assign Windows function. */ @SuppressWarnings({ diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/DoFnFunction.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/DoFnFunction.java index ca64e72f4a41..30522c89521b 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/DoFnFunction.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/DoFnFunction.java @@ -59,7 +59,7 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; /** DoFn function. */ @SuppressWarnings({ diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/GroupByWindowFunction.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/GroupByWindowFunction.java index 3bca4e96a6b5..9929b100bd9d 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/GroupByWindowFunction.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/GroupByWindowFunction.java @@ -51,7 +51,7 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.joda.time.Instant; /** GroupBy window function. */ diff --git a/sdks/java/core/build.gradle b/sdks/java/core/build.gradle index f0cd084d11a0..64aae685c380 100644 --- a/sdks/java/core/build.gradle +++ b/sdks/java/core/build.gradle @@ -76,7 +76,7 @@ dependencies { // Required to load constants from the model, e.g. max timestamp for global window shadow project(path: ":model:pipeline", configuration: "shadow") shadow project(path: ":model:job-management", configuration: "shadow") - shadow library.java.vendored_grpc_1_54_0 + shadow library.java.vendored_grpc_1_60_1 shadow library.java.vendored_guava_32_1_2_jre shadow library.java.byte_buddy implementation library.java.antlr_runtime diff --git a/sdks/java/core/jmh/build.gradle b/sdks/java/core/jmh/build.gradle index 67e23e704530..364e8f90600b 100644 --- a/sdks/java/core/jmh/build.gradle +++ b/sdks/java/core/jmh/build.gradle @@ -30,7 +30,7 @@ dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(path: ":sdks:java:core", configuration: "shadowTest") implementation library.java.joda_time - implementation library.java.vendored_grpc_1_54_0 + implementation library.java.vendored_grpc_1_60_1 implementation library.java.vendored_guava_32_1_2_jre implementation library.java.hadoop_common runtimeOnly library.java.slf4j_jdk14 diff --git a/sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/jmh/util/ByteStringOutputStreamBenchmark.java b/sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/jmh/util/ByteStringOutputStreamBenchmark.java index 21fb96d85191..a58565549473 100644 --- a/sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/jmh/util/ByteStringOutputStreamBenchmark.java +++ b/sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/jmh/util/ByteStringOutputStreamBenchmark.java @@ -19,8 +19,8 @@ import java.util.List; import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.UnsafeByteOperations; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.UnsafeByteOperations; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Param; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java index 8367b38751c3..8ab8db033d8c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java @@ -32,7 +32,7 @@ import org.apache.beam.sdk.io.fs.MatchResult; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.ValueProvider; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKey.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKey.java index 5c3473c39001..483498e09336 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKey.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKey.java @@ -22,8 +22,8 @@ import java.io.Serializable; import java.nio.ByteBuffer; import javax.annotation.Nonnull; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString.ByteIterator; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString.ByteIterator; import org.checkerframework.checker.nullness.qual.Nullable; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaTranslation.java index c0683ef44616..1d3f3348f1ed 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaTranslation.java @@ -53,7 +53,7 @@ import org.apache.beam.sdk.schemas.logicaltypes.VariableString; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/SchemaLogicalType.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/SchemaLogicalType.java index 3d5546ec1b67..2ab8c0161d0d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/SchemaLogicalType.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/SchemaLogicalType.java @@ -21,7 +21,7 @@ import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.FieldType; import org.apache.beam.sdk.schemas.SchemaTranslation; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHints.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHints.java index 85cb2df9deab..2f034626acd7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHints.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHints.java @@ -27,7 +27,7 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.StandardResourceHints; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ProtocolMessageEnum; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ProtocolMessageEnum; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ByteStringOutputStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ByteStringOutputStream.java index 2908e637c55a..f3af13a88759 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ByteStringOutputStream.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ByteStringOutputStream.java @@ -19,8 +19,8 @@ import java.io.OutputStream; import javax.annotation.concurrent.NotThreadSafe; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.UnsafeByteOperations; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.UnsafeByteOperations; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java index cbd395ff8807..d889824135b6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java @@ -27,7 +27,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Throwables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.BaseEncoding; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTranslationTest.java index 3020d7e42d05..bd7a0da394ae 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTranslationTest.java @@ -53,7 +53,7 @@ import org.apache.beam.sdk.schemas.logicaltypes.VariableBytes; import org.apache.beam.sdk.schemas.logicaltypes.VariableString; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Instant; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/JsonUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/JsonUtilsTest.java index 251ab804ce14..9984f431bded 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/JsonUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/JsonUtilsTest.java @@ -28,8 +28,8 @@ import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.gson.Gson; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.gson.JsonObject; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.gson.Gson; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.gson.JsonObject; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ByteStringOutputStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ByteStringOutputStreamTest.java index 0f9fdc5c2cfb..31c0f2c4e654 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ByteStringOutputStreamTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ByteStringOutputStreamTest.java @@ -22,7 +22,7 @@ import java.io.ByteArrayOutputStream; import java.io.DataOutputStream; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.UnsafeByteOperations; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.UnsafeByteOperations; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CoderUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CoderUtilsTest.java index a3732a39783c..7a49c502ca32 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CoderUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CoderUtilsTest.java @@ -32,7 +32,7 @@ import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.CoderPropertiesTest.ClosingCoder; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; diff --git a/sdks/java/expansion-service/build.gradle b/sdks/java/expansion-service/build.gradle index 99c515cd4e63..18df128b9386 100644 --- a/sdks/java/expansion-service/build.gradle +++ b/sdks/java/expansion-service/build.gradle @@ -48,7 +48,7 @@ dependencies { implementation library.java.jackson_annotations implementation library.java.jackson_databind implementation library.java.jackson_dataformat_yaml - implementation library.java.vendored_grpc_1_54_0 + implementation library.java.vendored_grpc_1_60_1 implementation library.java.vendored_guava_32_1_2_jre implementation library.java.slf4j_api runtimeOnly library.java.slf4j_jdk14 diff --git a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServer.java b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServer.java index 1b3330b490c7..39cb77b68cee 100644 --- a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServer.java +++ b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServer.java @@ -21,8 +21,8 @@ import java.net.InetSocketAddress; import java.util.concurrent.TimeUnit; import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Server; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.netty.NettyServerBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.netty.NettyServerBuilder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; /** A {@link Server gRPC Server} for an ExpansionService. */ 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 5d46100fe656..0af465cda143 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 @@ -88,11 +88,11 @@ import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Server; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ServerBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ServerBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.CaseFormat; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Converter; diff --git a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java index 0bd85fb79e64..152fc7d48cf0 100644 --- a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java +++ b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java @@ -36,7 +36,7 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/JavaClassLookupTransformProvider.java b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/JavaClassLookupTransformProvider.java index 96697c070bec..cb41e7f1a396 100644 --- a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/JavaClassLookupTransformProvider.java +++ b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/JavaClassLookupTransformProvider.java @@ -60,8 +60,8 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.checkerframework.checker.nullness.qual.Nullable; /** diff --git a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProviderTest.java b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProviderTest.java index 3e6451b131d3..6578287eb4c4 100644 --- a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProviderTest.java +++ b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProviderTest.java @@ -55,8 +55,8 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; 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.Iterables; import org.junit.Test; diff --git a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExternalTest.java b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExternalTest.java index d2363559473e..223ea7e73feb 100644 --- a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExternalTest.java +++ b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExternalTest.java @@ -43,8 +43,8 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.sdk.values.TypeDescriptors; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Server; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ServerBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ServerBuilder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; import org.checkerframework.checker.nullness.qual.RequiresNonNull; diff --git a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/JavaClassLookupTransformProviderTest.java b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/JavaClassLookupTransformProviderTest.java index 9179f1144a2f..e8f475ea66b6 100644 --- a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/JavaClassLookupTransformProviderTest.java +++ b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/JavaClassLookupTransformProviderTest.java @@ -63,8 +63,8 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; 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.io.Resources; diff --git a/sdks/java/extensions/python/build.gradle b/sdks/java/extensions/python/build.gradle index 7058d27fecbf..40729d5f9cab 100644 --- a/sdks/java/extensions/python/build.gradle +++ b/sdks/java/extensions/python/build.gradle @@ -25,7 +25,7 @@ evaluationDependsOn(":runners:core-construction-java") dependencies { implementation library.java.vendored_guava_32_1_2_jre - implementation library.java.vendored_grpc_1_54_0 + implementation library.java.vendored_grpc_1_60_1 implementation library.java.slf4j_api implementation project(path: ":model:pipeline", configuration: "shadow") implementation project(path: ":runners:core-construction-java") 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 5ba3484964c1..bd9ec7aa9816 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 @@ -59,7 +59,7 @@ import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; diff --git a/sdks/java/extensions/schemaio-expansion-service/build.gradle b/sdks/java/extensions/schemaio-expansion-service/build.gradle index 68fb67e93e37..859d01c84575 100644 --- a/sdks/java/extensions/schemaio-expansion-service/build.gradle +++ b/sdks/java/extensions/schemaio-expansion-service/build.gradle @@ -44,7 +44,7 @@ dependencies { permitUnusedDeclared library.java.postgres // BEAM-11761 implementation project(path: ":model:pipeline", configuration: "shadow") implementation project(path: ":sdks:java:core", configuration: "shadow") - implementation library.java.vendored_grpc_1_54_0 + implementation library.java.vendored_grpc_1_60_1 implementation library.java.vendored_guava_32_1_2_jre // Stage default drivers for JdbcIO schema transforms implementation 'mysql:mysql-connector-java:8.0.22' diff --git a/sdks/java/extensions/schemaio-expansion-service/src/main/java/org/apache/beam/sdk/extensions/schemaio/expansion/ExternalSchemaIOTransformRegistrar.java b/sdks/java/extensions/schemaio-expansion-service/src/main/java/org/apache/beam/sdk/extensions/schemaio/expansion/ExternalSchemaIOTransformRegistrar.java index 7bc14afb7832..8d5acd93f850 100644 --- a/sdks/java/extensions/schemaio-expansion-service/src/main/java/org/apache/beam/sdk/extensions/schemaio/expansion/ExternalSchemaIOTransformRegistrar.java +++ b/sdks/java/extensions/schemaio-expansion-service/src/main/java/org/apache/beam/sdk/extensions/schemaio/expansion/ExternalSchemaIOTransformRegistrar.java @@ -39,7 +39,7 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; diff --git a/sdks/java/fn-execution/build.gradle b/sdks/java/fn-execution/build.gradle index b04243a315ba..d1b5c713d092 100644 --- a/sdks/java/fn-execution/build.gradle +++ b/sdks/java/fn-execution/build.gradle @@ -29,7 +29,7 @@ dependencies { implementation project(path: ":model:pipeline", configuration: "shadow") implementation project(path: ":model:fn-execution", configuration: "shadow") implementation project(path: ":sdks:java:core", configuration: "shadow") - implementation library.java.vendored_grpc_1_54_0 + implementation library.java.vendored_grpc_1_60_1 implementation library.java.vendored_guava_32_1_2_jre implementation library.java.slf4j_api implementation library.java.joda_time diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/AddHarnessIdInterceptor.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/AddHarnessIdInterceptor.java index 647e1b4a3d56..84e4fb18e286 100644 --- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/AddHarnessIdInterceptor.java +++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/AddHarnessIdInterceptor.java @@ -19,10 +19,10 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ClientInterceptor; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Metadata; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Metadata.Key; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.MetadataUtils; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ClientInterceptor; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Metadata; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Metadata.Key; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.MetadataUtils; /** A {@link ClientInterceptor} that attaches a provided SDK Harness ID to outgoing messages. */ public class AddHarnessIdInterceptor { diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/ManagedChannelFactory.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/ManagedChannelFactory.java index c2914c3dd8ad..c4944d382557 100644 --- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/ManagedChannelFactory.java +++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/ManagedChannelFactory.java @@ -21,15 +21,15 @@ import java.util.Collections; import java.util.List; import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ClientInterceptor; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannelBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessChannelBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.netty.NettyChannelBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.netty.channel.epoll.EpollDomainSocketChannel; -import org.apache.beam.vendor.grpc.v1p54p0.io.netty.channel.epoll.EpollEventLoopGroup; -import org.apache.beam.vendor.grpc.v1p54p0.io.netty.channel.epoll.EpollSocketChannel; -import org.apache.beam.vendor.grpc.v1p54p0.io.netty.channel.unix.DomainSocketAddress; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ClientInterceptor; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.netty.NettyChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.netty.channel.epoll.EpollDomainSocketChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.netty.channel.epoll.EpollEventLoopGroup; +import org.apache.beam.vendor.grpc.v1p60p1.io.netty.channel.epoll.EpollSocketChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.netty.channel.unix.DomainSocketAddress; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; /** A Factory which creates {@link ManagedChannel} instances. */ @@ -48,7 +48,7 @@ public static ManagedChannelFactory createDefault() { * an {@link EpollSocketChannel}. */ public static ManagedChannelFactory createEpoll() { - org.apache.beam.vendor.grpc.v1p54p0.io.netty.channel.epoll.Epoll.ensureAvailability(); + org.apache.beam.vendor.grpc.v1p60p1.io.netty.channel.epoll.Epoll.ensureAvailability(); return new ManagedChannelFactory(Type.EPOLL, Collections.emptyList(), false); } diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/SocketAddressFactory.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/SocketAddressFactory.java index c835ea8312fa..3825cc8b103b 100644 --- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/SocketAddressFactory.java +++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/SocketAddressFactory.java @@ -23,7 +23,7 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.net.SocketAddress; -import org.apache.beam.vendor.grpc.v1p54p0.io.netty.channel.unix.DomainSocketAddress; +import org.apache.beam.vendor.grpc.v1p60p1.io.netty.channel.unix.DomainSocketAddress; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; /** Creates a {@link SocketAddress} based upon a supplied string. */ diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexer.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexer.java index 3f83af4a8d6b..e946022c4e36 100644 --- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexer.java +++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexer.java @@ -26,8 +26,8 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.model.pipeline.v1.Endpoints; import org.apache.beam.sdk.fn.stream.OutboundObserverFactory; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Status; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Status; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataOutboundAggregator.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataOutboundAggregator.java index ac2069fe264b..c2259c4fbe2a 100644 --- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataOutboundAggregator.java +++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataOutboundAggregator.java @@ -36,8 +36,8 @@ import org.apache.beam.sdk.options.ExperimentalOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.slf4j.Logger; diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/DecodingFnDataReceiver.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/DecodingFnDataReceiver.java index 667dfc7695d9..f1ae2c7e1261 100644 --- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/DecodingFnDataReceiver.java +++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/DecodingFnDataReceiver.java @@ -19,7 +19,7 @@ import java.io.InputStream; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; /** A receiver of encoded data, decoding it and passing it onto a downstream consumer. */ public class DecodingFnDataReceiver implements FnDataReceiver { diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortRead.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortRead.java index ba3ef217d7cf..d9d970049fd4 100644 --- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortRead.java +++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortRead.java @@ -23,7 +23,7 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi.RemoteGrpcPort; import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; /** diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortWrite.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortWrite.java index 3bbd31d8bf25..32401dfc424c 100644 --- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortWrite.java +++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortWrite.java @@ -24,7 +24,7 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection; import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; /** diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/FnService.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/FnService.java index 638ae3abae47..9fea8933fea3 100644 --- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/FnService.java +++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/FnService.java @@ -17,7 +17,7 @@ */ package org.apache.beam.sdk.fn.server; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.BindableService; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.BindableService; /** An interface sharing common behavior with services used during execution of user Fns. */ public interface FnService extends AutoCloseable, BindableService { @@ -26,8 +26,8 @@ public interface FnService extends AutoCloseable, BindableService { * *

There should be no more calls to any service method by the time a call to {@link #close()} * begins. Specifically, this means that a {@link - * org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Server} that this service is bound to should have - * completed a call to the {@link org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Server#shutdown()} + * org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server} that this service is bound to should have + * completed a call to the {@link org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server#shutdown()} * method, and all future incoming calls will be rejected. */ @Override diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/GrpcContextHeaderAccessorProvider.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/GrpcContextHeaderAccessorProvider.java index 2ccae386d1bf..d82e08da26b7 100644 --- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/GrpcContextHeaderAccessorProvider.java +++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/GrpcContextHeaderAccessorProvider.java @@ -17,14 +17,14 @@ */ package org.apache.beam.sdk.fn.server; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Context; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Contexts; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Metadata; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Metadata.Key; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ServerCall; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ServerCall.Listener; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ServerCallHandler; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ServerInterceptor; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Context; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Contexts; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Metadata; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Metadata.Key; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ServerCall; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ServerCall.Listener; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ServerCallHandler; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ServerInterceptor; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; /** diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/GrpcFnServer.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/GrpcFnServer.java index fdf4f9e83068..f0df3c13742c 100644 --- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/GrpcFnServer.java +++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/GrpcFnServer.java @@ -23,7 +23,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Server; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server; 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.Lists; diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/InProcessServerFactory.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/InProcessServerFactory.java index de9025ad4903..b8cb9528b9ee 100644 --- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/InProcessServerFactory.java +++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/InProcessServerFactory.java @@ -21,10 +21,10 @@ import java.util.List; import java.util.concurrent.atomic.AtomicInteger; import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.BindableService; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Server; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ServerInterceptors; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessServerBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.BindableService; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ServerInterceptors; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessServerBuilder; /** * A {@link ServerFactory} which creates {@link Server servers} with the {@link diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/ServerFactory.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/ServerFactory.java index a50e5cbef5bc..fc5d694cd11a 100644 --- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/ServerFactory.java +++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/ServerFactory.java @@ -29,16 +29,16 @@ import java.util.function.Supplier; import org.apache.beam.model.pipeline.v1.Endpoints; import org.apache.beam.sdk.fn.channel.SocketAddressFactory; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.BindableService; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Server; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ServerBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ServerInterceptors; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.netty.NettyServerBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.netty.channel.epoll.EpollEventLoopGroup; -import org.apache.beam.vendor.grpc.v1p54p0.io.netty.channel.epoll.EpollServerDomainSocketChannel; -import org.apache.beam.vendor.grpc.v1p54p0.io.netty.channel.epoll.EpollServerSocketChannel; -import org.apache.beam.vendor.grpc.v1p54p0.io.netty.channel.unix.DomainSocketAddress; -import org.apache.beam.vendor.grpc.v1p54p0.io.netty.util.internal.ThreadLocalRandom; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.BindableService; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ServerBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ServerInterceptors; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.netty.NettyServerBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.netty.channel.epoll.EpollEventLoopGroup; +import org.apache.beam.vendor.grpc.v1p60p1.io.netty.channel.epoll.EpollServerDomainSocketChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.netty.channel.epoll.EpollServerSocketChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.netty.channel.unix.DomainSocketAddress; +import org.apache.beam.vendor.grpc.v1p60p1.io.netty.util.internal.ThreadLocalRandom; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; /** A {@link Server gRPC server} factory. */ diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/BufferingStreamObserver.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/BufferingStreamObserver.java index 4876c6a42011..14cd269389de 100644 --- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/BufferingStreamObserver.java +++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/BufferingStreamObserver.java @@ -22,8 +22,8 @@ import java.util.concurrent.Phaser; import javax.annotation.concurrent.ThreadSafe; import org.apache.beam.sdk.fn.CancellableQueue; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.CallStreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.CallStreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.checkerframework.checker.nullness.qual.NonNull; diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/DataStreams.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/DataStreams.java index 3a541b44fa30..b0d29e2295a8 100644 --- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/DataStreams.java +++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/DataStreams.java @@ -29,7 +29,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.fn.data.WeightedList; import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; /** * {@link DataStreamDecoder} treats multiple {@link ByteString}s as a single input stream decoding diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/DirectStreamObserver.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/DirectStreamObserver.java index 3bc72ac16c7d..05421758ea59 100644 --- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/DirectStreamObserver.java +++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/DirectStreamObserver.java @@ -21,8 +21,8 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import javax.annotation.concurrent.ThreadSafe; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.CallStreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.CallStreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/ForwardingClientResponseObserver.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/ForwardingClientResponseObserver.java index 73ac226aaaca..151c397064b5 100644 --- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/ForwardingClientResponseObserver.java +++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/ForwardingClientResponseObserver.java @@ -17,9 +17,9 @@ */ package org.apache.beam.sdk.fn.stream; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.ClientCallStreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.ClientResponseObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.ClientCallStreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.ClientResponseObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; /** * A {@link ClientResponseObserver} which delegates all {@link StreamObserver} calls. diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactory.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactory.java index e2eeb1a0568d..e040fa57c7d9 100644 --- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactory.java +++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactory.java @@ -18,8 +18,8 @@ package org.apache.beam.sdk.fn.stream; import java.util.concurrent.ExecutorService; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.CallStreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.CallStreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; /** * Creates factories which determine an underlying {@link StreamObserver} implementation to use in diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/SynchronizedStreamObserver.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/SynchronizedStreamObserver.java index cc228ea35106..21f5d5df2077 100644 --- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/SynchronizedStreamObserver.java +++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/SynchronizedStreamObserver.java @@ -17,7 +17,7 @@ */ package org.apache.beam.sdk.fn.stream; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; /** * A {@link StreamObserver} which provides synchronous access access to an underlying {@link diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/test/TestStreams.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/test/TestStreams.java index 500aedce30f2..22f6ddff41df 100644 --- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/test/TestStreams.java +++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/test/TestStreams.java @@ -19,8 +19,8 @@ import java.util.function.Consumer; import java.util.function.Supplier; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.CallStreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.CallStreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; /** Utility methods which enable testing of {@link StreamObserver}s. */ public class TestStreams { diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/windowing/EncodedBoundedWindow.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/windowing/EncodedBoundedWindow.java index f4270f40fc12..c12c17791522 100644 --- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/windowing/EncodedBoundedWindow.java +++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/windowing/EncodedBoundedWindow.java @@ -25,7 +25,7 @@ import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.VarInt; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.ByteStreams; import org.joda.time.Instant; diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/channel/ManagedChannelFactoryTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/channel/ManagedChannelFactoryTest.java index 01915545aef7..f276e719b8ac 100644 --- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/channel/ManagedChannelFactoryTest.java +++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/channel/ManagedChannelFactoryTest.java @@ -22,7 +22,7 @@ import static org.junit.Assume.assumeTrue; import org.apache.beam.model.pipeline.v1.Endpoints; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; import org.apache.commons.lang3.SystemUtils; import org.junit.Rule; import org.junit.Test; @@ -48,7 +48,7 @@ public void testDefaultChannel() { @Test public void testEpollHostPortChannel() { assumeTrue(SystemUtils.IS_OS_LINUX); - assertTrue(org.apache.beam.vendor.grpc.v1p54p0.io.netty.channel.epoll.Epoll.isAvailable()); + assertTrue(org.apache.beam.vendor.grpc.v1p60p1.io.netty.channel.epoll.Epoll.isAvailable()); Endpoints.ApiServiceDescriptor apiServiceDescriptor = Endpoints.ApiServiceDescriptor.newBuilder().setUrl("localhost:123").build(); ManagedChannel channel = @@ -60,7 +60,7 @@ public void testEpollHostPortChannel() { @Test public void testEpollDomainSocketChannel() throws Exception { assumeTrue(SystemUtils.IS_OS_LINUX); - assertTrue(org.apache.beam.vendor.grpc.v1p54p0.io.netty.channel.epoll.Epoll.isAvailable()); + assertTrue(org.apache.beam.vendor.grpc.v1p60p1.io.netty.channel.epoll.Epoll.isAvailable()); Endpoints.ApiServiceDescriptor apiServiceDescriptor = Endpoints.ApiServiceDescriptor.newBuilder() .setUrl("unix://" + tmpFolder.newFile().getAbsolutePath()) diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/channel/SocketAddressFactoryTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/channel/SocketAddressFactoryTest.java index 7912e04d678e..a29ebb40f516 100644 --- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/channel/SocketAddressFactoryTest.java +++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/channel/SocketAddressFactoryTest.java @@ -23,7 +23,7 @@ import java.io.File; import java.net.InetSocketAddress; import java.net.SocketAddress; -import org.apache.beam.vendor.grpc.v1p54p0.io.netty.channel.unix.DomainSocketAddress; +import org.apache.beam.vendor.grpc.v1p60p1.io.netty.channel.unix.DomainSocketAddress; import org.hamcrest.Matchers; import org.junit.Rule; import org.junit.Test; diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexerTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexerTest.java index ee388885a632..a1a8823a0a35 100644 --- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexerTest.java +++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexerTest.java @@ -36,7 +36,7 @@ import org.apache.beam.sdk.fn.test.TestExecutors; import org.apache.beam.sdk.fn.test.TestExecutors.TestExecutorService; import org.apache.beam.sdk.fn.test.TestStreams; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.junit.Rule; import org.junit.Test; diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortReadTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortReadTest.java index e55f154be88d..1150f5e33cbd 100644 --- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortReadTest.java +++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortReadTest.java @@ -24,7 +24,7 @@ import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor; import org.apache.beam.model.pipeline.v1.Endpoints.AuthenticationSpec; import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortWriteTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortWriteTest.java index b05473773c0d..f9a6b8044f0c 100644 --- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortWriteTest.java +++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortWriteTest.java @@ -24,7 +24,7 @@ import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor; import org.apache.beam.model.pipeline.v1.Endpoints.AuthenticationSpec; import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/DataStreamsTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/DataStreamsTest.java index 2c0b99a7ff8e..d15d5bcd503a 100644 --- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/DataStreamsTest.java +++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/DataStreamsTest.java @@ -41,7 +41,7 @@ import org.apache.beam.sdk.fn.stream.DataStreams.ElementDelimitedOutputStream; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.ByteStreams; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.CountingOutputStream; diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/ForwardingClientResponseObserverTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/ForwardingClientResponseObserverTest.java index 8d0e462bd4e8..130b7c996995 100644 --- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/ForwardingClientResponseObserverTest.java +++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/ForwardingClientResponseObserverTest.java @@ -21,9 +21,9 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoMoreInteractions; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.ClientCallStreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.ClientResponseObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.ClientCallStreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.ClientResponseObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactoryTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactoryTest.java index 54cc7a456b51..c3504ddb3691 100644 --- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactoryTest.java +++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactoryTest.java @@ -22,8 +22,8 @@ import static org.junit.Assert.assertEquals; import java.util.concurrent.Executors; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.CallStreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.CallStreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/windowing/EncodedBoundedWindowTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/windowing/EncodedBoundedWindowTest.java index 01c0a69c520b..52dedca8c176 100644 --- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/windowing/EncodedBoundedWindowTest.java +++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/windowing/EncodedBoundedWindowTest.java @@ -19,7 +19,7 @@ import org.apache.beam.sdk.fn.windowing.EncodedBoundedWindow.Coder; import org.apache.beam.sdk.testing.CoderProperties; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/harness/build.gradle b/sdks/java/harness/build.gradle index 937e870a22f4..505cbb38ab13 100644 --- a/sdks/java/harness/build.gradle +++ b/sdks/java/harness/build.gradle @@ -33,7 +33,7 @@ dependencies { provided library.java.avro provided library.java.joda_time provided library.java.slf4j_api - provided library.java.vendored_grpc_1_54_0 + provided library.java.vendored_grpc_1_60_1 provided library.java.vendored_guava_32_1_2_jre provided library.java.jamm diff --git a/sdks/java/harness/jmh/build.gradle b/sdks/java/harness/jmh/build.gradle index 9dfb79bfebb9..90e752947172 100644 --- a/sdks/java/harness/jmh/build.gradle +++ b/sdks/java/harness/jmh/build.gradle @@ -35,7 +35,7 @@ dependencies { implementation project(path: ":sdks:java:harness", configuration: "shadow") implementation project(path: ":runners:java-fn-execution") implementation project(path: ":model:pipeline", configuration: "shadow") - implementation library.java.vendored_grpc_1_54_0 + implementation library.java.vendored_grpc_1_60_1 implementation library.java.vendored_guava_32_1_2_jre implementation library.java.slf4j_api implementation library.java.joda_time diff --git a/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/ProcessBundleBenchmark.java b/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/ProcessBundleBenchmark.java index 7dc6a10b38f8..ce7ae0a1ea45 100644 --- a/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/ProcessBundleBenchmark.java +++ b/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/ProcessBundleBenchmark.java @@ -90,7 +90,7 @@ import org.apache.beam.sdk.transforms.WithKeys; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; 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.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; diff --git a/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/control/ExecutionStateSamplerBenchmark.java b/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/control/ExecutionStateSamplerBenchmark.java index 3aa1ef37eda0..80e1bb81be72 100644 --- a/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/control/ExecutionStateSamplerBenchmark.java +++ b/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/control/ExecutionStateSamplerBenchmark.java @@ -27,7 +27,7 @@ import org.apache.beam.runners.core.metrics.MonitoringInfoConstants.Urns; import org.apache.beam.runners.core.metrics.SimpleExecutionState; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Level; import org.openjdk.jmh.annotations.Scope; diff --git a/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/control/MetricsBenchmark.java b/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/control/MetricsBenchmark.java index d3276e471a48..26d44ce2c112 100644 --- a/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/control/MetricsBenchmark.java +++ b/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/control/MetricsBenchmark.java @@ -26,7 +26,7 @@ import org.apache.beam.runners.core.metrics.CounterCell; import org.apache.beam.runners.core.metrics.MonitoringInfoEncodings; import org.apache.beam.sdk.metrics.MetricName; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Level; import org.openjdk.jmh.annotations.Scope; diff --git a/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/logging/BeamFnLoggingClientBenchmark.java b/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/logging/BeamFnLoggingClientBenchmark.java index 1a5558ed6700..b9e4b20db00b 100644 --- a/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/logging/BeamFnLoggingClientBenchmark.java +++ b/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/logging/BeamFnLoggingClientBenchmark.java @@ -33,9 +33,9 @@ import org.apache.beam.runners.core.metrics.SimpleExecutionState; import org.apache.beam.sdk.fn.channel.ManagedChannelFactory; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Server; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessServerBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessServerBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Level; import org.openjdk.jmh.annotations.Scope; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java index 295818837b77..cc16a4fbbe72 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java @@ -50,7 +50,7 @@ import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.apache.beam.sdk.fn.data.RemoteGrpcPortRead; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ExternalWorkerService.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ExternalWorkerService.java index b688abecee53..09074d7f6cb9 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ExternalWorkerService.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ExternalWorkerService.java @@ -34,7 +34,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PortablePipelineOptions; import org.apache.beam.sdk.util.Sleeper; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java index 69114703a528..a3ae67fc2a56 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java @@ -115,8 +115,8 @@ import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.util.Durations; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.util.Durations; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; 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; @@ -1548,11 +1548,11 @@ static HandlesSplits.SplitResult constructSplitResult .setElement(bytesOut.toByteString()); // We don't want to change the output watermarks or set the checkpoint resume time since // that applies to the current window. - Map + Map outputWatermarkMapForUnprocessedWindows = new HashMap<>(); if (!initialWatermark.equals(GlobalWindow.TIMESTAMP_MIN_VALUE)) { - org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Timestamp outputWatermark = - org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Timestamp.newBuilder() + org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Timestamp outputWatermark = + org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Timestamp.newBuilder() .setSeconds(initialWatermark.getMillis() / 1000) .setNanos((int) (initialWatermark.getMillis() % 1000) * 1000000) .build(); @@ -1592,11 +1592,11 @@ static HandlesSplits.SplitResult constructSplitResult .setTransformId(pTransformId) .setInputId(mainInputId) .setElement(residualBytes.toByteString()); - Map + Map outputWatermarkMap = new HashMap<>(); if (!watermarkAndState.getKey().equals(GlobalWindow.TIMESTAMP_MIN_VALUE)) { - org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Timestamp outputWatermark = - org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Timestamp.newBuilder() + org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Timestamp outputWatermark = + org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Timestamp.newBuilder() .setSeconds(watermarkAndState.getKey().getMillis() / 1000) .setNanos((int) (watermarkAndState.getKey().getMillis() % 1000) * 1000000) .build(); diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java index cada9b12f60b..6d13b3704e16 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java @@ -59,8 +59,8 @@ import org.apache.beam.sdk.options.ExecutorOptions; import org.apache.beam.sdk.options.ExperimentalOptions; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.TextFormat; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.TextFormat; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; 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.ImmutableSet; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/BeamFnControlClient.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/BeamFnControlClient.java index 876a838f1662..68f9c5dcb4f7 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/BeamFnControlClient.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/BeamFnControlClient.java @@ -29,8 +29,8 @@ import org.apache.beam.sdk.fn.channel.ManagedChannelFactory; import org.apache.beam.sdk.fn.stream.OutboundObserverFactory; import org.apache.beam.sdk.function.ThrowingFunction; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Status; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Status; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/BundleProgressReporter.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/BundleProgressReporter.java index 67f1d4a30fda..227f2f1d3fa3 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/BundleProgressReporter.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/BundleProgressReporter.java @@ -22,7 +22,7 @@ import java.util.Map; import javax.annotation.concurrent.NotThreadSafe; import org.apache.beam.fn.harness.control.ProcessBundleHandler.BundleProcessor; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; /** * Reports metrics related to bundle processing. diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ExecutionStateSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ExecutionStateSampler.java index 5509d6380ef6..5d856ee63063 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ExecutionStateSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ExecutionStateSampler.java @@ -46,7 +46,7 @@ import org.apache.beam.sdk.options.ExperimentalOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.HistogramData; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Joiner; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.DateTimeUtils.MillisProvider; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/Metrics.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/Metrics.java index 0415d77b98d7..a2cfb6e0170d 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/Metrics.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/Metrics.java @@ -28,7 +28,7 @@ import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Distribution; import org.apache.beam.sdk.metrics.MetricName; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; public class Metrics { diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java index ecec509bf950..344c84422ecb 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java @@ -90,8 +90,8 @@ import org.apache.beam.sdk.transforms.DoFn.BundleFinalizer; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.common.ReflectHelpers; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.TextFormat; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.TextFormat; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheLoader; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClient.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClient.java index 09dab3479a67..981b115c58e7 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClient.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClient.java @@ -31,7 +31,7 @@ import org.apache.beam.sdk.fn.data.CloseableFnDataReceiver; import org.apache.beam.sdk.fn.stream.OutboundObserverFactory; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClient.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClient.java index c16296be717d..7812d8c0bc30 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClient.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClient.java @@ -50,14 +50,14 @@ import org.apache.beam.sdk.options.ExecutorOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.SdkHarnessOptions; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Struct; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Timestamp; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Value; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.CallStreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.ClientCallStreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.ClientResponseObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Timestamp; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Value; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.CallStreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.ClientCallStreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.ClientResponseObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.initialization.qual.UnderInitialization; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BeamFnStateGrpcClientCache.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BeamFnStateGrpcClientCache.java index 3318caf05335..9cd2aa551183 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BeamFnStateGrpcClientCache.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BeamFnStateGrpcClientCache.java @@ -28,8 +28,8 @@ import org.apache.beam.sdk.fn.IdGenerator; import org.apache.beam.sdk.fn.channel.ManagedChannelFactory; import org.apache.beam.sdk.fn.stream.OutboundObserverFactory; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java index 204c491dc102..2751fade988f 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java @@ -64,7 +64,7 @@ import org.apache.beam.sdk.util.CombineFnUtil; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; 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.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiTimerBundleTracker.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiTimerBundleTracker.java index 01b67c67cf3e..6dbc600fa448 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiTimerBundleTracker.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiTimerBundleTracker.java @@ -33,7 +33,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ComparisonChain; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.HashBasedTable; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java index 0c7726441021..377db38b7015 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java @@ -35,7 +35,7 @@ import org.apache.beam.sdk.transforms.Materializations.MultimapView; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.values.KV; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; /** * An implementation of a multimap side input that utilizes the Beam Fn State API to fetch values. diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapUserState.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapUserState.java index 112ded3161c1..6698216d3e85 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapUserState.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapUserState.java @@ -43,7 +43,7 @@ import org.apache.beam.sdk.fn.stream.PrefetchableIterator; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.values.KV; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; /** diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateBackedIterable.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateBackedIterable.java index 22e0822b6191..358d7924767e 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateBackedIterable.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateBackedIterable.java @@ -46,7 +46,7 @@ import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterable; import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterator; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; 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; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateFetchingIterators.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateFetchingIterators.java index 00abd835e3af..6f8622d736fe 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateFetchingIterators.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateFetchingIterators.java @@ -43,7 +43,7 @@ import org.apache.beam.sdk.fn.stream.PrefetchableIterables; import org.apache.beam.sdk.fn.stream.PrefetchableIterator; import org.apache.beam.sdk.util.Weighted; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Throwables; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/status/BeamFnStatusClient.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/status/BeamFnStatusClient.java index ef8ec626322c..846cf7e4d5a6 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/status/BeamFnStatusClient.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/status/BeamFnStatusClient.java @@ -38,8 +38,8 @@ import org.apache.beam.model.fnexecution.v1.BeamFnWorkerStatusGrpc; import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.DateTimeUtils; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/HarnessStreamObserverFactories.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/HarnessStreamObserverFactories.java index 530e102a23f1..cef15977c8b8 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/HarnessStreamObserverFactories.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/HarnessStreamObserverFactories.java @@ -22,7 +22,7 @@ import org.apache.beam.sdk.options.ExecutorOptions; import org.apache.beam.sdk.options.ExperimentalOptions; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; /** * Uses {@link PipelineOptions} to configure which underlying {@link StreamObserver} implementation diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java index e8ad49eda125..05f720bd24c4 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java @@ -67,7 +67,7 @@ import org.apache.beam.sdk.fn.test.TestExecutors.TestExecutorService; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; 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.Iterables; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java index 13b6d0a7a29b..38c3763a535c 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java @@ -53,7 +53,7 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.hamcrest.collection.IsMapContaining; import org.junit.Before; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/ExternalWorkerServiceTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/ExternalWorkerServiceTest.java index b4f4ccb8f1f1..dde3064e636e 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/ExternalWorkerServiceTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/ExternalWorkerServiceTest.java @@ -27,7 +27,7 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi.StopWorkerResponse; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java index 3bc9ec38e391..7c42104ca7cb 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java @@ -142,8 +142,8 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.util.Durations; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.util.Durations; 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.Iterables; @@ -1697,13 +1697,13 @@ public void testProcessElementForSizedElementAndRestriction() throws Exception { assertEquals( ImmutableMap.of( "output", - org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Timestamp.newBuilder() + org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Timestamp.newBuilder() .setSeconds(expectedOutputWatermark.getMillis() / 1000) .setNanos((int) (expectedOutputWatermark.getMillis() % 1000) * 1000000) .build()), residualRoot.getApplication().getOutputWatermarksMap()); assertEquals( - org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Duration.newBuilder() + org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Duration.newBuilder() .setSeconds(54) .setNanos(321000000) .build(), @@ -1817,7 +1817,7 @@ public void testProcessElementForSizedElementAndRestriction() throws Exception { assertEquals( ImmutableMap.of( "output", - org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Timestamp.newBuilder() + org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Timestamp.newBuilder() .setSeconds(expectedOutputWatermark.getMillis() / 1000) .setNanos((int) (expectedOutputWatermark.getMillis() % 1000) * 1000000) .build()), @@ -2032,27 +2032,27 @@ public void testProcessElementForWindowedSizedElementAndRestriction() throws Exc residualRoot.getApplication().getInputId()); assertEquals(TEST_TRANSFORM_ID, residualRoot.getApplication().getTransformId()); Instant expectedOutputWatermark = GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(7)); - Map + Map expectedOutputWatmermarkMap = ImmutableMap.of( "output", - org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Timestamp.newBuilder() + org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Timestamp.newBuilder() .setSeconds(expectedOutputWatermark.getMillis() / 1000) .setNanos((int) (expectedOutputWatermark.getMillis() % 1000) * 1000000) .build()); Instant initialWatermark = GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(1)); - Map + Map expectedOutputWatmermarkMapForUnprocessedWindows = ImmutableMap.of( "output", - org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Timestamp.newBuilder() + org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Timestamp.newBuilder() .setSeconds(initialWatermark.getMillis() / 1000) .setNanos((int) (initialWatermark.getMillis() % 1000) * 1000000) .build()); assertEquals( expectedOutputWatmermarkMap, residualRoot.getApplication().getOutputWatermarksMap()); assertEquals( - org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Duration.newBuilder() + org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Duration.newBuilder() .setSeconds(54) .setNanos(321000000) .build(), @@ -2261,19 +2261,19 @@ public void testProcessElementForWindowedSizedElementAndRestriction() throws Exc residualRootInUnprocessedWindows.getRequestedTimeDelay()); Instant initialWatermark = GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(1)); Instant expectedOutputWatermark = GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(2)); - Map + Map expectedOutputWatermarkMapInUnprocessedResiduals = ImmutableMap.of( "output", - org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Timestamp.newBuilder() + org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Timestamp.newBuilder() .setSeconds(initialWatermark.getMillis() / 1000) .setNanos((int) (initialWatermark.getMillis() % 1000) * 1000000) .build()); - Map + Map expectedOutputWatermarkMap = ImmutableMap.of( "output", - org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Timestamp.newBuilder() + org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Timestamp.newBuilder() .setSeconds(expectedOutputWatermark.getMillis() / 1000) .setNanos((int) (expectedOutputWatermark.getMillis() % 1000) * 1000000) .build()); @@ -4660,9 +4660,9 @@ private HandlesSplits.SplitResult getProcessElementSplit(String transformId, Str .build())); } - private org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Timestamp toTimestamp( + private org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Timestamp toTimestamp( Instant time) { - return org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Timestamp.newBuilder() + return org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Timestamp.newBuilder() .setSeconds(time.getMillis() / 1000) .setNanos((int) (time.getMillis() % 1000) * 1000000) .build(); @@ -4768,7 +4768,7 @@ public void testConstructSplitResultWithOnlyWindowSplits() throws Exception { assertEquals(1, result.getResidualRoots().size()); DelayedBundleApplication residualRoot = result.getResidualRoots().get(0); assertEquals( - org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Duration.getDefaultInstance(), + org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Duration.getDefaultInstance(), residualRoot.getRequestedTimeDelay()); assertEquals(PROCESS_TRANSFORM_ID, residualRoot.getApplication().getTransformId()); assertEquals(PROCESS_INPUT_ID, residualRoot.getApplication().getInputId()); @@ -4823,7 +4823,7 @@ public void testConstructSplitResultWithElementAndWindowSplitFromProcess() throw DelayedBundleApplication windowResidual = result.getResidualRoots().get(0); DelayedBundleApplication elementResidual = result.getResidualRoots().get(1); assertEquals( - org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Duration.getDefaultInstance(), + org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Duration.getDefaultInstance(), windowResidual.getRequestedTimeDelay()); assertEquals(PROCESS_TRANSFORM_ID, windowResidual.getApplication().getTransformId()); assertEquals(PROCESS_INPUT_ID, windowResidual.getApplication().getInputId()); @@ -4881,7 +4881,7 @@ public void testConstructSplitResultWithElementAndWindowSplitFromTruncate() thro DelayedBundleApplication windowResidual = result.getResidualRoots().get(0); DelayedBundleApplication elementResidual = result.getResidualRoots().get(1); assertEquals( - org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Duration.getDefaultInstance(), + org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Duration.getDefaultInstance(), windowResidual.getRequestedTimeDelay()); assertEquals(TRUNCATE_TRANSFORM_ID, windowResidual.getApplication().getTransformId()); assertEquals(TRUNCATE_INPUT_ID, windowResidual.getApplication().getInputId()); diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnHarnessTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnHarnessTest.java index 4ee781f69a33..65fb4a42c03e 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnHarnessTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnHarnessTest.java @@ -43,10 +43,10 @@ import org.apache.beam.sdk.options.ExecutorOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.TextFormat; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Server; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ServerBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.TextFormat; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ServerBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Uninterruptibles; import org.junit.Rule; import org.junit.Test; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/BeamFnControlClientTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/BeamFnControlClientTest.java index 27799ce2a58a..c18e931b4e24 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/BeamFnControlClientTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/BeamFnControlClientTest.java @@ -44,10 +44,10 @@ import org.apache.beam.sdk.fn.stream.OutboundObserverFactory; import org.apache.beam.sdk.fn.test.TestStreams; import org.apache.beam.sdk.function.ThrowingFunction; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Server; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessServerBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.CallStreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessServerBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.CallStreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Uninterruptibles; import org.junit.Rule; import org.junit.Test; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/BundleProgressReporterTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/BundleProgressReporterTest.java index ee8fca97ccbf..20eab1583280 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/BundleProgressReporterTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/BundleProgressReporterTest.java @@ -24,7 +24,7 @@ import java.util.HashMap; import java.util.Map; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ExecutionStateSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ExecutionStateSamplerTest.java index 47866adc892b..c2fd308205a1 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ExecutionStateSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ExecutionStateSamplerTest.java @@ -45,7 +45,7 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.ExpectedLogs; import org.apache.beam.sdk.util.HistogramData; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.joda.time.DateTimeUtils.MillisProvider; import org.joda.time.Duration; import org.junit.After; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/MetricsTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/MetricsTest.java index ac9916a64739..a3834bfbf6ec 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/MetricsTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/MetricsTest.java @@ -40,7 +40,7 @@ import org.apache.beam.sdk.fn.test.TestExecutors; import org.apache.beam.sdk.fn.test.TestExecutors.TestExecutorService; import org.apache.beam.sdk.metrics.MetricName; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java index ae3adf5ee56f..75c91783076f 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java @@ -143,8 +143,8 @@ import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; 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.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java index af3d1f15cdaf..3489fe766891 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java @@ -49,13 +49,13 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Server; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessChannelBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessServerBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.CallStreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessServerBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.CallStreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistryTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistryTest.java index f75b84e76ad5..2d1aaf0b0179 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistryTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistryTest.java @@ -74,13 +74,13 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterable; import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterator; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Server; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessChannelBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessServerBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.CallStreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessServerBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.CallStreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.junit.After; import org.junit.Before; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java index 1cad5210380b..096bba1bb287 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java @@ -41,7 +41,7 @@ import org.apache.beam.sdk.options.ExperimentalOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java index 26285205bd34..02ab83e1a6a6 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java @@ -32,7 +32,7 @@ import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClientTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClientTest.java index 8c7a40f8db90..9cfa32a40d2b 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClientTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClientTest.java @@ -44,22 +44,22 @@ import org.apache.beam.model.pipeline.v1.Endpoints; import org.apache.beam.sdk.fn.test.TestStreams; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Struct; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Timestamp; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Value; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.CallOptions; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Channel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ClientCall; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ClientInterceptor; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ForwardingClientCall; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.MethodDescriptor; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Server; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Status; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessChannelBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessServerBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.CallStreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Timestamp; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Value; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.CallOptions; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Channel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ClientCall; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ClientInterceptor; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ForwardingClientCall; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.MethodDescriptor; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Status; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessServerBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.CallStreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/BagUserStateTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/BagUserStateTest.java index b4d6f5ae40e2..e3162bdea3de 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/BagUserStateTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/BagUserStateTest.java @@ -31,7 +31,7 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateKey; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; 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.Iterables; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/BeamFnStateGrpcClientCacheTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/BeamFnStateGrpcClientCacheTest.java index dc48a7a3e5bd..2d913ca2df0f 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/BeamFnStateGrpcClientCacheTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/BeamFnStateGrpcClientCacheTest.java @@ -42,12 +42,12 @@ import org.apache.beam.sdk.fn.test.TestExecutors; import org.apache.beam.sdk.fn.test.TestExecutors.TestExecutorService; import org.apache.beam.sdk.fn.test.TestStreams; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Server; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Status; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.StatusRuntimeException; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessServerBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.CallStreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Status; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.StatusRuntimeException; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessServerBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.CallStreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Uninterruptibles; import org.junit.After; import org.junit.Before; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/FakeBeamFnStateClient.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/FakeBeamFnStateClient.java index 5447d6f0c6d6..468b4f6b4251 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/FakeBeamFnStateClient.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/FakeBeamFnStateClient.java @@ -39,7 +39,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.values.KV; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; /** A fake implementation of a {@link BeamFnStateClient} to aid with testing. */ diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/IterableSideInputTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/IterableSideInputTest.java index 2c1f73fcaf0d..0cee6dba58d5 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/IterableSideInputTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/IterableSideInputTest.java @@ -25,7 +25,7 @@ import org.apache.beam.fn.harness.Caches; import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateKey; import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; 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.Iterables; import org.junit.Test; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/MultimapSideInputTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/MultimapSideInputTest.java index 23a572894b40..128e0af6be93 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/MultimapSideInputTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/MultimapSideInputTest.java @@ -32,7 +32,7 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.values.KV; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; 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.Iterables; import org.junit.Test; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/MultimapUserStateTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/MultimapUserStateTest.java index d43056750ad9..60b0a9577823 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/MultimapUserStateTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/MultimapUserStateTest.java @@ -43,7 +43,7 @@ import org.apache.beam.sdk.fn.stream.PrefetchableIterable; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.values.KV; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; 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.Iterables; import org.junit.Test; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/StateBackedIterableTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/StateBackedIterableTest.java index f758c367f73f..ffc1ba62cb56 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/StateBackedIterableTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/StateBackedIterableTest.java @@ -38,7 +38,7 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.FluentIterable; 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; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/StateFetchingIteratorsTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/StateFetchingIteratorsTest.java index bb70b585d4c3..205444007ec6 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/StateFetchingIteratorsTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/StateFetchingIteratorsTest.java @@ -48,7 +48,7 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateResponse; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.fn.stream.PrefetchableIterator; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; 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.primitives.Ints; import org.junit.Test; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/status/BeamFnStatusClientTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/status/BeamFnStatusClientTest.java index 19cd568f6429..b48c939dded8 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/status/BeamFnStatusClientTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/status/BeamFnStatusClientTest.java @@ -47,9 +47,9 @@ import org.apache.beam.sdk.fn.channel.ManagedChannelFactory; import org.apache.beam.sdk.fn.test.TestStreams; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Server; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessServerBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessServerBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Uninterruptibles; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/HarnessStreamObserverFactoriesTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/HarnessStreamObserverFactoriesTest.java index d340fc0ffc19..d402886f43a1 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/HarnessStreamObserverFactoriesTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/HarnessStreamObserverFactoriesTest.java @@ -25,8 +25,8 @@ import org.apache.beam.sdk.fn.stream.DirectStreamObserver; import org.apache.beam.sdk.fn.stream.ForwardingClientResponseObserver; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.CallStreamObserver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.CallStreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/sdks/java/io/components/build.gradle b/sdks/java/io/components/build.gradle index 5b2ad69fb0b0..a0ca06b09f1c 100644 --- a/sdks/java/io/components/build.gradle +++ b/sdks/java/io/components/build.gradle @@ -30,7 +30,7 @@ dependencies { permitUnusedDeclared library.java.protobuf_java // BEAM-11761 implementation library.java.slf4j_api implementation library.java.vendored_guava_32_1_2_jre - implementation library.java.vendored_grpc_1_54_0 + implementation library.java.vendored_grpc_1_60_1 testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") testImplementation library.java.guava_testlib diff --git a/sdks/java/io/contextualtextio/build.gradle b/sdks/java/io/contextualtextio/build.gradle index 0557a1dfa259..41457a9ff81a 100644 --- a/sdks/java/io/contextualtextio/build.gradle +++ b/sdks/java/io/contextualtextio/build.gradle @@ -30,7 +30,7 @@ dependencies { permitUnusedDeclared library.java.protobuf_java // BEAM-11761 implementation library.java.slf4j_api implementation library.java.vendored_guava_32_1_2_jre - implementation library.java.vendored_grpc_1_54_0 + implementation library.java.vendored_grpc_1_60_1 testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") testImplementation library.java.commons_compress diff --git a/sdks/java/io/contextualtextio/src/main/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIOSource.java b/sdks/java/io/contextualtextio/src/main/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIOSource.java index 9be8b5cb4b9b..7ba3761e0e9b 100644 --- a/sdks/java/io/contextualtextio/src/main/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIOSource.java +++ b/sdks/java/io/contextualtextio/src/main/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIOSource.java @@ -30,7 +30,7 @@ import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.schemas.SchemaCoder; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/sdks/java/io/debezium/build.gradle b/sdks/java/io/debezium/build.gradle index 121a43515926..7cbe7fc93c2b 100644 --- a/sdks/java/io/debezium/build.gradle +++ b/sdks/java/io/debezium/build.gradle @@ -32,7 +32,7 @@ ext.summary = "Library to work with Debezium data." dependencies { implementation library.java.vendored_guava_32_1_2_jre - implementation library.java.vendored_grpc_1_54_0 + implementation library.java.vendored_grpc_1_60_1 implementation project(path: ":sdks:java:core", configuration: "shadow") implementation library.java.slf4j_api implementation library.java.joda_time diff --git a/sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/SourceRecordJson.java b/sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/SourceRecordJson.java index 8d2cbebbccc1..4d7de9b0a9a8 100644 --- a/sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/SourceRecordJson.java +++ b/sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/SourceRecordJson.java @@ -20,8 +20,8 @@ import java.io.Serializable; import java.util.HashMap; import java.util.Map; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.gson.Gson; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.gson.GsonBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.gson.Gson; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.gson.GsonBuilder; import org.apache.kafka.connect.data.Field; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.DataException; diff --git a/sdks/java/io/google-cloud-platform/build.gradle b/sdks/java/io/google-cloud-platform/build.gradle index b0122035a015..723cf94f528f 100644 --- a/sdks/java/io/google-cloud-platform/build.gradle +++ b/sdks/java/io/google-cloud-platform/build.gradle @@ -142,7 +142,7 @@ dependencies { implementation library.java.protobuf_java implementation library.java.protobuf_java_util implementation library.java.slf4j_api - implementation library.java.vendored_grpc_1_54_0 + implementation library.java.vendored_grpc_1_60_1 implementation library.java.vendored_guava_32_1_2_jre implementation library.java.arrow_memory_core implementation library.java.arrow_vector diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java index b2d533f69fb3..8f48890ad56d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java @@ -62,7 +62,7 @@ import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.ValueInSingleWindow; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; 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.checkerframework.checker.nullness.qual.Nullable; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerTransformRegistrar.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerTransformRegistrar.java index 809d7a275512..919d84002293 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerTransformRegistrar.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerTransformRegistrar.java @@ -36,7 +36,7 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/SpannerChangestreamsReadSchemaTransformProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/SpannerChangestreamsReadSchemaTransformProvider.java index 51e55dc7664a..f3562e4cd917 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/SpannerChangestreamsReadSchemaTransformProvider.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/SpannerChangestreamsReadSchemaTransformProvider.java @@ -63,7 +63,7 @@ import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.gson.Gson; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.gson.Gson; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; import org.checkerframework.checker.initialization.qual.Initialized; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOExternalTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOExternalTest.java index 020d522b5e2e..a33b17b08e10 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOExternalTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOExternalTest.java @@ -38,7 +38,7 @@ import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.hamcrest.Matchers; import org.hamcrest.text.MatchesPattern; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerTransformRegistrarTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerTransformRegistrarTest.java index 39316e6b688f..1e838d2d2785 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerTransformRegistrarTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerTransformRegistrarTest.java @@ -36,7 +36,7 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.hamcrest.collection.IsMapContaining; import org.junit.Before; import org.junit.Test; diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOExternalTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOExternalTest.java index 38bf723a15a9..71ceb43dbeee 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOExternalTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOExternalTest.java @@ -45,7 +45,7 @@ import org.apache.beam.sdk.transforms.WithKeys; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; 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.Iterables; diff --git a/sdks/java/io/kafka/upgrade/build.gradle b/sdks/java/io/kafka/upgrade/build.gradle index ca2823740dc6..5a62fe230a33 100644 --- a/sdks/java/io/kafka/upgrade/build.gradle +++ b/sdks/java/io/kafka/upgrade/build.gradle @@ -27,7 +27,7 @@ ext.summary = "Library to support upgrading Kafka transforms without upgrading t dependencies { implementation project(path: ":model:pipeline", configuration: "shadow") implementation project(":sdks:java:io:kafka") - implementation library.java.vendored_grpc_1_54_0 + implementation library.java.vendored_grpc_1_60_1 implementation project(path: ":sdks:java:core", configuration: "shadow") implementation library.java.vendored_guava_32_1_2_jre diff --git a/sdks/java/io/kafka/upgrade/src/main/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslation.java b/sdks/java/io/kafka/upgrade/src/main/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslation.java index a76507a285ba..909c919ccdb6 100644 --- a/sdks/java/io/kafka/upgrade/src/main/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslation.java +++ b/sdks/java/io/kafka/upgrade/src/main/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslation.java @@ -52,7 +52,7 @@ import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; 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; import org.apache.kafka.clients.consumer.Consumer; diff --git a/sdks/java/transform-service/build.gradle b/sdks/java/transform-service/build.gradle index 130a6b1d7091..702479fdf30a 100644 --- a/sdks/java/transform-service/build.gradle +++ b/sdks/java/transform-service/build.gradle @@ -43,7 +43,7 @@ dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(path: ":runners:core-construction-java") implementation project(path: ":sdks:java:fn-execution") - implementation library.java.vendored_grpc_1_54_0 + implementation library.java.vendored_grpc_1_60_1 implementation library.java.vendored_guava_32_1_2_jre implementation library.java.jackson_annotations implementation library.java.jackson_databind diff --git a/sdks/java/transform-service/src/main/java/org/apache/beam/sdk/transformservice/ArtifactService.java b/sdks/java/transform-service/src/main/java/org/apache/beam/sdk/transformservice/ArtifactService.java index 714b7b27960e..7e8452cd19cb 100644 --- a/sdks/java/transform-service/src/main/java/org/apache/beam/sdk/transformservice/ArtifactService.java +++ b/sdks/java/transform-service/src/main/java/org/apache/beam/sdk/transformservice/ArtifactService.java @@ -28,9 +28,9 @@ import org.apache.beam.model.jobmanagement.v1.ArtifactRetrievalServiceGrpc.ArtifactRetrievalServiceBlockingStub; import org.apache.beam.model.pipeline.v1.Endpoints; import org.apache.beam.sdk.fn.server.FnService; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannelBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.checkerframework.checker.nullness.qual.Nullable; public class ArtifactService extends ArtifactRetrievalServiceGrpc.ArtifactRetrievalServiceImplBase diff --git a/sdks/java/transform-service/src/main/java/org/apache/beam/sdk/transformservice/Controller.java b/sdks/java/transform-service/src/main/java/org/apache/beam/sdk/transformservice/Controller.java index d1c0e382a838..d082f1f250a8 100644 --- a/sdks/java/transform-service/src/main/java/org/apache/beam/sdk/transformservice/Controller.java +++ b/sdks/java/transform-service/src/main/java/org/apache/beam/sdk/transformservice/Controller.java @@ -22,8 +22,8 @@ import org.apache.beam.model.pipeline.v1.Endpoints; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Server; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ServerBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ServerBuilder; /** * A component that controlls the transform service. diff --git a/sdks/java/transform-service/src/main/java/org/apache/beam/sdk/transformservice/ExpansionService.java b/sdks/java/transform-service/src/main/java/org/apache/beam/sdk/transformservice/ExpansionService.java index 0a2e65099e7d..75515ad75f7c 100644 --- a/sdks/java/transform-service/src/main/java/org/apache/beam/sdk/transformservice/ExpansionService.java +++ b/sdks/java/transform-service/src/main/java/org/apache/beam/sdk/transformservice/ExpansionService.java @@ -30,8 +30,8 @@ import org.apache.beam.model.pipeline.v1.Endpoints; import org.apache.beam.runners.core.construction.DefaultExpansionServiceClientFactory; import org.apache.beam.runners.core.construction.ExpansionServiceClientFactory; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannelBuilder; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Throwables; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/sdks/java/transform-service/src/test/java/org/apache/beam/sdk/transformservice/ArtifactServiceTest.java b/sdks/java/transform-service/src/test/java/org/apache/beam/sdk/transformservice/ArtifactServiceTest.java index 9105e23a5c50..2b0144bfe517 100644 --- a/sdks/java/transform-service/src/test/java/org/apache/beam/sdk/transformservice/ArtifactServiceTest.java +++ b/sdks/java/transform-service/src/test/java/org/apache/beam/sdk/transformservice/ArtifactServiceTest.java @@ -29,7 +29,7 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService; import org.apache.beam.sdk.transformservice.ArtifactService.ArtifactResolver; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Before; import org.junit.Rule; diff --git a/sdks/java/transform-service/src/test/java/org/apache/beam/sdk/transformservice/ExpansionServiceTest.java b/sdks/java/transform-service/src/test/java/org/apache/beam/sdk/transformservice/ExpansionServiceTest.java index 9905abd1d9ba..14cf9a4e0186 100644 --- a/sdks/java/transform-service/src/test/java/org/apache/beam/sdk/transformservice/ExpansionServiceTest.java +++ b/sdks/java/transform-service/src/test/java/org/apache/beam/sdk/transformservice/ExpansionServiceTest.java @@ -33,7 +33,7 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; import org.apache.beam.runners.core.construction.ExpansionServiceClient; import org.apache.beam.runners.core.construction.ExpansionServiceClientFactory; -import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/settings.gradle.kts b/settings.gradle.kts index c645c1adfd6f..b1b03b55add0 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -312,7 +312,6 @@ include(":sdks:python:test-suites:tox:py311") include(":sdks:python:test-suites:xlang") include(":sdks:typescript") include(":sdks:typescript:container") -include(":vendor:grpc-1_54_0") include(":vendor:grpc-1_60_1") include(":vendor:calcite-1_28_0") include(":vendor:guava-32_1_2-jre") diff --git a/vendor/README.md b/vendor/README.md index 7815305fa662..169b67da3e1b 100644 --- a/vendor/README.md +++ b/vendor/README.md @@ -36,8 +36,8 @@ For example, when we upgrade the version of gRPC to 1.54.0 and the version of th we could run the linkage tool as following: ``` -$ ./gradlew -p vendor/grpc-1_54_0 publishMavenJavaPublicationToMavenLocal -Ppublishing -PvendoredDependenciesOnly -$ ./gradlew -PvendoredDependenciesOnly -Ppublishing -PjavaLinkageArtifactIds=beam-vendor-grpc-1_54_0:0.1-SNAPSHOT :checkJavaLinkage +$ ./gradlew -p vendor/grpc-1_60_1 publishMavenJavaPublicationToMavenLocal -Ppublishing -PvendoredDependenciesOnly +$ ./gradlew -PvendoredDependenciesOnly -Ppublishing -PjavaLinkageArtifactIds=beam-vendor-grpc-1_60_1:0.1-SNAPSHOT :checkJavaLinkage ``` ### Known Linkage Errors in the Vendored gRPC Dependencies @@ -86,12 +86,12 @@ Steps: copy to the `tempLib` folder in Beam: ``` -./gradlew -p vendor/grpc-1_54_0 publishMavenJavaPublicationToMavenLocal -Ppublishing -PvendoredDependenciesOnly +./gradlew -p vendor/grpc-1_60_1 publishMavenJavaPublicationToMavenLocal -Ppublishing -PvendoredDependenciesOnly mkdir -p tempLib/org/apache/beam # Copy files (jar/poms/metadata) to your beam repository -cp -R ~/.m2/repository/org/apache/beam/beam-vendor-grpc-1_54_0 \ +cp -R ~/.m2/repository/org/apache/beam/beam-vendor-grpc-1_60_1` \ tempLib/org/apache/beam/ ``` diff --git a/vendor/grpc-1_54_0/build.gradle b/vendor/grpc-1_54_0/build.gradle deleted file mode 100644 index 6dbc9dbd08c4..000000000000 --- a/vendor/grpc-1_54_0/build.gradle +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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 org.apache.beam.gradle.GrpcVendoring_1_54_0 - -plugins { id 'org.apache.beam.vendor-java' } - -description = "Apache Beam :: Vendored Dependencies :: gRPC :: 1.54.0" - -group = "org.apache.beam" -version = "0.2" - -vendorJava( - dependencies: GrpcVendoring_1_54_0.dependencies(), - runtimeDependencies: GrpcVendoring_1_54_0.runtimeDependencies(), - testDependencies: GrpcVendoring_1_54_0.testDependencies(), - relocations: GrpcVendoring_1_54_0.relocations(), - exclusions: GrpcVendoring_1_54_0.exclusions(), - artifactId: "beam-vendor-grpc-${GrpcVendoring_1_54_0.grpc_version.replace(".", "_")}", - groupId: group, - version: version, -) From 0b06cd897d2016e702452e16bc66018528896098 Mon Sep 17 00:00:00 2001 From: JayajP Date: Tue, 23 Jan 2024 12:51:09 -0800 Subject: [PATCH 052/169] Fix BigQuerySinkMetrics constants and increment metrics in more places. (#30067) * Fix bigquerysinkmetrics constants and increment metrics in a few more places * Spotless --- .../io/gcp/bigquery/BigQuerySinkMetrics.java | 12 +++---- .../StorageApiWritesShardedRecords.java | 5 +-- .../gcp/bigquery/BigQuerySinkMetricsTest.java | 35 ++++++++++++------- 3 files changed, 31 insertions(+), 21 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetrics.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetrics.java index a89707c1919f..c0f470b39214 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetrics.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetrics.java @@ -53,9 +53,9 @@ public class BigQuerySinkMetrics { public static final String PAYLOAD_TOO_LARGE = "PayloadTooLarge"; // Base Metric names - private static final String RPC_REQUESTS = "RpcRequests"; + private static final String RPC_REQUESTS = "RpcRequestsCount"; private static final String RPC_LATENCY = "RpcLatency"; - private static final String APPEND_ROWS_ROW_STATUS = "AppendRowsRowStatus"; + private static final String APPEND_ROWS_ROW_STATUS = "RowsAppendedCount"; private static final String THROTTLED_TIME = "ThrottledTime"; // StorageWriteAPI Method names @@ -73,10 +73,10 @@ enum RowStatus { } // Metric labels - private static final String TABLE_ID_LABEL = "TableId"; - private static final String RPC_STATUS_LABEL = "RpcStatus"; - private static final String RPC_METHOD = "Method"; - private static final String ROW_STATUS = "RowStatus"; + private static final String TABLE_ID_LABEL = "table_id"; + private static final String RPC_STATUS_LABEL = "rpc_status"; + private static final String RPC_METHOD = "rpc_method"; + private static final String ROW_STATUS = "row_status"; // Delimiters private static final char LABEL_DELIMITER = ';'; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java index 0f9b07d0c40f..b612f199a29f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java @@ -652,7 +652,7 @@ public void process( AppendRowsContext failedContext = Preconditions.checkStateNotNull(Iterables.getFirst(failedContexts, null)); BigQuerySinkMetrics.reportFailedRPCMetrics( - failedContext, BigQuerySinkMetrics.RpcMethod.APPEND_ROWS); + failedContext, BigQuerySinkMetrics.RpcMethod.APPEND_ROWS, shortTableId); String errorCode = BigQuerySinkMetrics.throwableToGRPCCodeString(failedContext.getError()); @@ -801,7 +801,8 @@ public void process( BigQuerySinkMetrics.reportSuccessfulRpcMetrics( context, BigQuerySinkMetrics.RpcMethod.APPEND_ROWS, shortTableId); BigQuerySinkMetrics.appendRowsRowStatusCounter( - BigQuerySinkMetrics.RowStatus.SUCCESSFUL, BigQuerySinkMetrics.OK, shortTableId); + BigQuerySinkMetrics.RowStatus.SUCCESSFUL, BigQuerySinkMetrics.OK, shortTableId) + .inc(flushedRows); if (successfulRowsTag != null) { for (int i = 0; i < context.protoRows.getSerializedRowsCount(); ++i) { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetricsTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetricsTest.java index 9c6fae164fca..6b04ed0acc15 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetricsTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetricsTest.java @@ -112,7 +112,7 @@ public void testAppendRowsRowStatusCounter() throws Exception { deletesDisabledCounter.inc(); MetricName deletesDisabledCounterName = MetricName.named( - "BigQuerySink", "AppendRowsRowStatus-RowStatus:SUCCESSFUL;RpcStatus:rpcStatus;"); + "BigQuerySink", "RowsAppendedCount-row_status:SUCCESSFUL;rpc_status:rpcStatus;"); assertThat(testContainer.perWorkerCounters, IsMapContaining.hasKey(deletesDisabledCounterName)); assertThat( testContainer.perWorkerCounters.get(deletesDisabledCounterName).getCumulative(), @@ -127,7 +127,7 @@ public void testAppendRowsRowStatusCounter() throws Exception { MetricName deletesEnabledCounterName = MetricName.named( "BigQuerySink", - "AppendRowsRowStatus-RowStatus:SUCCESSFUL;RpcStatus:rpcStatus;TableId:tableId;"); + "RowsAppendedCount-row_status:SUCCESSFUL;rpc_status:rpcStatus;table_id:tableId;"); assertThat(testContainer.perWorkerCounters, IsMapContaining.hasKey(deletesEnabledCounterName)); assertThat( testContainer.perWorkerCounters.get(deletesEnabledCounterName).getCumulative(), @@ -160,8 +160,9 @@ public void testThrottledTimeCounter() throws Exception { appendRowsThrottleCounter.inc(1); assertThat( appendRowsThrottleCounter.getName().getName(), - equalTo("ThrottledTime-Method:APPEND_ROWS;")); - MetricName counterName = MetricName.named("BigQuerySink", "ThrottledTime-Method:APPEND_ROWS;"); + equalTo("ThrottledTime-rpc_method:APPEND_ROWS;")); + MetricName counterName = + MetricName.named("BigQuerySink", "ThrottledTime-rpc_method:APPEND_ROWS;"); assertThat(testContainer.perWorkerCounters, IsMapContaining.hasKey(counterName)); assertThat(testContainer.perWorkerCounters.get(counterName).getCumulative(), equalTo(1L)); } @@ -181,8 +182,9 @@ public void testReportSuccessfulRpcMetrics() throws Exception { BigQuerySinkMetrics.reportSuccessfulRpcMetrics( c, BigQuerySinkMetrics.RpcMethod.APPEND_ROWS, "tableId"); MetricName counterNameDisabledDeletes = - MetricName.named("BigQuerySink", "RpcRequests-Method:APPEND_ROWS;RpcStatus:OK;"); - MetricName histogramName = MetricName.named("BigQuerySink", "RpcLatency-Method:APPEND_ROWS;"); + MetricName.named("BigQuerySink", "RpcRequestsCount-rpc_method:APPEND_ROWS;rpc_status:OK;"); + MetricName histogramName = + MetricName.named("BigQuerySink", "RpcLatency-rpc_method:APPEND_ROWS;"); HistogramData.BucketType bucketType = HistogramData.ExponentialBuckets.of(1, 34); assertThat(testContainer.perWorkerCounters, IsMapContaining.hasKey(counterNameDisabledDeletes)); assertThat( @@ -199,7 +201,8 @@ public void testReportSuccessfulRpcMetrics() throws Exception { c, BigQuerySinkMetrics.RpcMethod.APPEND_ROWS, "tableId"); MetricName counterNameEnabledDeletes = MetricName.named( - "BigQuerySink", "RpcRequests-Method:APPEND_ROWS;RpcStatus:OK;TableId:tableId;"); + "BigQuerySink", + "RpcRequestsCount-rpc_method:APPEND_ROWS;rpc_status:OK;table_id:tableId;"); assertThat(testContainer.perWorkerCounters, IsMapContaining.hasKey(counterNameEnabledDeletes)); assertThat( testContainer.perWorkerCounters.get(counterNameEnabledDeletes).getCumulative(), @@ -228,8 +231,10 @@ public void testReportFailedRPCMetrics_KnownGrpcError() throws Exception { BigQuerySinkMetrics.reportFailedRPCMetrics( c, BigQuerySinkMetrics.RpcMethod.APPEND_ROWS, "tableId"); MetricName counterNameDisabledDeletes = - MetricName.named("BigQuerySink", "RpcRequests-Method:APPEND_ROWS;RpcStatus:NOT_FOUND;"); - MetricName histogramName = MetricName.named("BigQuerySink", "RpcLatency-Method:APPEND_ROWS;"); + MetricName.named( + "BigQuerySink", "RpcRequestsCount-rpc_method:APPEND_ROWS;rpc_status:NOT_FOUND;"); + MetricName histogramName = + MetricName.named("BigQuerySink", "RpcLatency-rpc_method:APPEND_ROWS;"); HistogramData.BucketType bucketType = HistogramData.ExponentialBuckets.of(1, 34); assertThat(testContainer.perWorkerCounters, IsMapContaining.hasKey(counterNameDisabledDeletes)); assertThat( @@ -249,7 +254,8 @@ public void testReportFailedRPCMetrics_KnownGrpcError() throws Exception { c, BigQuerySinkMetrics.RpcMethod.APPEND_ROWS, "tableId"); MetricName counterNameEnabledDeletes = MetricName.named( - "BigQuerySink", "RpcRequests-Method:APPEND_ROWS;RpcStatus:NOT_FOUND;TableId:tableId;"); + "BigQuerySink", + "RpcRequestsCount-rpc_method:APPEND_ROWS;rpc_status:NOT_FOUND;table_id:tableId;"); assertThat(testContainer.perWorkerCounters, IsMapContaining.hasKey(counterNameEnabledDeletes)); assertThat( testContainer.perWorkerCounters.get(counterNameEnabledDeletes).getCumulative(), @@ -277,8 +283,10 @@ public void testReportFailedRPCMetrics_UnknownGrpcError() throws Exception { BigQuerySinkMetrics.reportFailedRPCMetrics( c, BigQuerySinkMetrics.RpcMethod.APPEND_ROWS, "tableId"); MetricName counterNameDisabledDeletes = - MetricName.named("BigQuerySink", "RpcRequests-Method:APPEND_ROWS;RpcStatus:UNKNOWN;"); - MetricName histogramName = MetricName.named("BigQuerySink", "RpcLatency-Method:APPEND_ROWS;"); + MetricName.named( + "BigQuerySink", "RpcRequestsCount-rpc_method:APPEND_ROWS;rpc_status:UNKNOWN;"); + MetricName histogramName = + MetricName.named("BigQuerySink", "RpcLatency-rpc_method:APPEND_ROWS;"); HistogramData.BucketType bucketType = HistogramData.ExponentialBuckets.of(1, 34); assertThat(testContainer.perWorkerCounters, IsMapContaining.hasKey(counterNameDisabledDeletes)); assertThat( @@ -295,7 +303,8 @@ public void testReportFailedRPCMetrics_UnknownGrpcError() throws Exception { c, BigQuerySinkMetrics.RpcMethod.APPEND_ROWS, "tableId"); MetricName counterNameEnabledDeletes = MetricName.named( - "BigQuerySink", "RpcRequests-Method:APPEND_ROWS;RpcStatus:UNKNOWN;TableId:tableId;"); + "BigQuerySink", + "RpcRequestsCount-rpc_method:APPEND_ROWS;rpc_status:UNKNOWN;table_id:tableId;"); assertThat(testContainer.perWorkerCounters, IsMapContaining.hasKey(counterNameEnabledDeletes)); assertThat( testContainer.perWorkerCounters.get(counterNameEnabledDeletes).getCumulative(), From 0c2b1a512db033910af681d7a444c53c44c30722 Mon Sep 17 00:00:00 2001 From: JayajP Date: Tue, 23 Jan 2024 12:57:01 -0800 Subject: [PATCH 053/169] Add converter class to convert metrics to PerStepNamespaceMetrics (#30084) * Add converter class to convert metrics to PerStepNamespaceMetrics * Address comments on null returns --- ...icsToPerStepNamespaceMetricsConverter.java | 199 ++++++++++++ ...oPerStepNamespaceMetricsConverterTest.java | 293 ++++++++++++++++++ 2 files changed, 492 insertions(+) create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverter.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverterTest.java diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverter.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverter.java new file mode 100644 index 000000000000..f3f31ccfd36f --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverter.java @@ -0,0 +1,199 @@ +/* + * 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.runners.dataflow.worker; + +import com.google.api.services.dataflow.model.Base2Exponent; +import com.google.api.services.dataflow.model.BucketOptions; +import com.google.api.services.dataflow.model.DataflowHistogramValue; +import com.google.api.services.dataflow.model.Linear; +import com.google.api.services.dataflow.model.MetricValue; +import com.google.api.services.dataflow.model.OutlierStats; +import com.google.api.services.dataflow.model.PerStepNamespaceMetrics; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Optional; +import org.apache.beam.sdk.io.gcp.bigquery.BigQuerySinkMetrics; +import org.apache.beam.sdk.metrics.MetricName; +import org.apache.beam.sdk.util.HistogramData; + +/** + * Converts metric updates to {@link PerStepNamespaceMetrics} protos. Currently we only support + * converting metrics from {@link BigQuerySinkMetrics} with this converter. + */ +public class MetricsToPerStepNamespaceMetricsConverter { + /** + * @param metricName The {@link MetricName} that represents this counter. + * @param value The counter value. + * @return If the conversion succeeds, {@code MetricValue} that represents this counter. Otherwise + * returns an empty optional + */ + private static Optional convertCounterToMetricValue( + MetricName metricName, Long value) { + if (value == 0 || !metricName.getNamespace().equals(BigQuerySinkMetrics.METRICS_NAMESPACE)) { + return Optional.empty(); + } + + BigQuerySinkMetrics.ParsedMetricName labeledName = + BigQuerySinkMetrics.parseMetricName(metricName.getName()); + if (labeledName == null || labeledName.getBaseName().isEmpty()) { + return Optional.empty(); + } + + return Optional.of( + new MetricValue() + .setMetric(labeledName.getBaseName()) + .setMetricLabels(labeledName.getMetricLabels()) + .setValueInt64(value)); + } + + /** + * @param metricName The {@link MetricName} that represents this Histogram. + * @param value The histogram value. Currently we only support converting histograms that use + * {@code linear} or {@code exponential} buckets. + * @return If this conversion succeeds, a {@code MetricValue} that represents this histogram. + * Otherwise returns an empty optional. + */ + private static Optional convertHistogramToMetricValue( + MetricName metricName, HistogramData value) { + if (value.getTotalCount() == 0L) { + return Optional.empty(); + } + + BigQuerySinkMetrics.ParsedMetricName labeledName = + BigQuerySinkMetrics.parseMetricName(metricName.getName()); + if (labeledName == null || labeledName.getBaseName().isEmpty()) { + return Optional.empty(); + } + + DataflowHistogramValue histogramValue = new DataflowHistogramValue(); + int numberOfBuckets = value.getBucketType().getNumBuckets(); + + if (value.getBucketType() instanceof HistogramData.LinearBuckets) { + HistogramData.LinearBuckets buckets = (HistogramData.LinearBuckets) value.getBucketType(); + Linear linearOptions = + new Linear() + .setNumberOfBuckets(numberOfBuckets) + .setWidth(buckets.getWidth()) + .setStart(buckets.getStart()); + histogramValue.setBucketOptions(new BucketOptions().setLinear(linearOptions)); + } else if (value.getBucketType() instanceof HistogramData.ExponentialBuckets) { + HistogramData.ExponentialBuckets buckets = + (HistogramData.ExponentialBuckets) value.getBucketType(); + Base2Exponent expoenntialOptions = + new Base2Exponent().setNumberOfBuckets(numberOfBuckets).setScale(buckets.getScale()); + histogramValue.setBucketOptions(new BucketOptions().setExponential(expoenntialOptions)); + } else { + return Optional.empty(); + } + + histogramValue.setCount(value.getTotalCount()); + List bucketCounts = new ArrayList<>(value.getBucketType().getNumBuckets()); + + for (int i = 0; i < value.getBucketType().getNumBuckets(); i++) { + bucketCounts.add(value.getCount(i)); + } + + // Remove trailing 0 buckets. + for (int i = bucketCounts.size() - 1; i >= 0; i--) { + if (bucketCounts.get(i) != 0) { + break; + } + bucketCounts.remove(i); + } + + histogramValue.setBucketCounts(bucketCounts); + + OutlierStats outlierStats = + new OutlierStats() + .setOverflowCount(value.getTopBucketCount()) + .setOverflowMean(value.getTopBucketMean()) + .setUnderflowCount(value.getBottomBucketCount()) + .setUnderflowMean(value.getBottomBucketMean()); + + histogramValue.setOutlierStats(outlierStats); + + return Optional.of( + new MetricValue() + .setMetric(labeledName.getBaseName()) + .setMetricLabels(labeledName.getMetricLabels()) + .setValueHistogram(histogramValue)); + } + + /** + * @param stepName The unfused stage that these metrics are associated with. + * @param counters Counter updates to convert. + * @param histograms Histogram updates to convert. + * @return Collection of {@code PerStepNamespaceMetrics} that represent these metric updates. Each + * {@code PerStepNamespaceMetrics} contains a list of {@code MetricUpdates} for a {unfused + * stage, metrics namespace} pair. + */ + public static Collection convert( + String stepName, Map counters, Map histograms) { + + Map metricsByNamespace = new HashMap<>(); + + for (Entry entry : counters.entrySet()) { + MetricName metricName = entry.getKey(); + Optional metricValue = convertCounterToMetricValue(metricName, entry.getValue()); + if (!metricValue.isPresent()) { + continue; + } + + PerStepNamespaceMetrics stepNamespaceMetrics = + metricsByNamespace.get(metricName.getNamespace()); + if (stepNamespaceMetrics == null) { + stepNamespaceMetrics = + new PerStepNamespaceMetrics() + .setMetricValues(new ArrayList<>()) + .setOriginalStep(stepName) + .setMetricsNamespace(metricName.getNamespace()); + metricsByNamespace.put(metricName.getNamespace(), stepNamespaceMetrics); + } + + stepNamespaceMetrics.getMetricValues().add(metricValue.get()); + } + + for (Entry entry : histograms.entrySet()) { + MetricName metricName = entry.getKey(); + Optional metricValue = + convertHistogramToMetricValue(metricName, entry.getValue()); + if (!metricValue.isPresent()) { + continue; + } + + PerStepNamespaceMetrics stepNamespaceMetrics = + metricsByNamespace.get(metricName.getNamespace()); + if (stepNamespaceMetrics == null) { + stepNamespaceMetrics = + new PerStepNamespaceMetrics() + .setMetricValues(new ArrayList<>()) + .setOriginalStep(stepName) + .setMetricsNamespace(metricName.getNamespace()); + metricsByNamespace.put(metricName.getNamespace(), stepNamespaceMetrics); + } + + stepNamespaceMetrics.getMetricValues().add(metricValue.get()); + } + + return metricsByNamespace.values(); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverterTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverterTest.java new file mode 100644 index 000000000000..a37bf643b7ee --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverterTest.java @@ -0,0 +1,293 @@ +/* + * 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.runners.dataflow.worker; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.equalTo; + +import com.google.api.services.dataflow.model.Base2Exponent; +import com.google.api.services.dataflow.model.BucketOptions; +import com.google.api.services.dataflow.model.DataflowHistogramValue; +import com.google.api.services.dataflow.model.Linear; +import com.google.api.services.dataflow.model.MetricValue; +import com.google.api.services.dataflow.model.OutlierStats; +import com.google.api.services.dataflow.model.PerStepNamespaceMetrics; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.beam.sdk.metrics.MetricName; +import org.apache.beam.sdk.util.HistogramData; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class MetricsToPerStepNamespaceMetricsConverterTest { + + public static class TestBucketType implements HistogramData.BucketType { + @Override + public double getRangeFrom() { + return 0.0; + } + + @Override + public double getRangeTo() { + return 5.0; + } + + @Override + public int getNumBuckets() { + return 1; + } + + @Override + public int getBucketIndex(double value) { + return 0; + } + + @Override + public double getBucketSize(int index) { + return 5.0; + } + + @Override + public double getAccumulatedBucketSize(int endIndex) { + return 5.0; + } + } + + @Test + public void testConvert_successfulyConvertCounters() { + String step = "testStepName"; + Map emptyHistograms = new HashMap<>(); + Map counters = new HashMap(); + MetricName bigQueryMetric1 = MetricName.named("BigQuerySink", "metric1"); + MetricName bigQueryMetric2 = + MetricName.named("BigQuerySink", "metric2-label1:val1;label2:val2;"); + MetricName bigQueryMetric3 = MetricName.named("BigQuerySink", "zeroValue"); + + counters.put(bigQueryMetric1, 5L); + counters.put(bigQueryMetric2, 10L); + counters.put(bigQueryMetric3, 0L); + + Collection conversionResult = + MetricsToPerStepNamespaceMetricsConverter.convert(step, counters, emptyHistograms); + + MetricValue expectedVal1 = + new MetricValue().setMetric("metric1").setValueInt64(5L).setMetricLabels(new HashMap<>()); + Map val2LabelMap = new HashMap<>(); + val2LabelMap.put("label1", "val1"); + val2LabelMap.put("label2", "val2"); + MetricValue expectedVal2 = + new MetricValue().setMetric("metric2").setValueInt64(10L).setMetricLabels(val2LabelMap); + + assertThat(conversionResult.size(), equalTo(1)); + PerStepNamespaceMetrics perStepNamespaceMetrics = conversionResult.iterator().next(); + + assertThat(perStepNamespaceMetrics.getOriginalStep(), equalTo(step)); + assertThat(perStepNamespaceMetrics.getMetricsNamespace(), equalTo("BigQuerySink")); + assertThat(perStepNamespaceMetrics.getMetricValues().size(), equalTo(2)); + assertThat( + perStepNamespaceMetrics.getMetricValues(), containsInAnyOrder(expectedVal1, expectedVal2)); + } + + @Test + public void testConvert_skipInvalidMetricNames() { + Map counters = new HashMap<>(); + MetricName bigQueryMetric1 = MetricName.named("BigQuerySink", "invalid-metric-name1"); + counters.put(bigQueryMetric1, 5L); + + Map histograms = new HashMap<>(); + MetricName bigQueryMetric2 = MetricName.named("BigQuerySink", "invalid-metric-name2"); + HistogramData nonEmptyLinearHistogram = HistogramData.linear(0, 10, 10); + nonEmptyLinearHistogram.record(-5.0); + histograms.put(bigQueryMetric2, nonEmptyLinearHistogram); + + Collection conversionResult = + MetricsToPerStepNamespaceMetricsConverter.convert("testStep", counters, histograms); + assertThat(conversionResult.size(), equalTo(0)); + } + + @Test + public void testConvert_successfulConvertHistograms() { + Map histograms = new HashMap(); + MetricName bigQueryMetric1 = MetricName.named("BigQuerySink", "baseLabel"); + MetricName bigQueryMetric2 = + MetricName.named("BigQuerySink", "baseLabel-label1:val1;label2:val2;"); + MetricName bigQueryMetric3 = MetricName.named("BigQuerySink", "zeroValue"); + + HistogramData nonEmptyLinearHistogram = HistogramData.linear(0, 10, 10); + nonEmptyLinearHistogram.record(-5.0, 15.0, 25.0, 35.0, 105.0); + histograms.put(bigQueryMetric1, nonEmptyLinearHistogram); + + HistogramData noEmptyExponentialHistogram = HistogramData.exponential(0, 5); + noEmptyExponentialHistogram.record(-5.0, 15.0, 25.0, 35.0, 105.0); + histograms.put(bigQueryMetric2, noEmptyExponentialHistogram); + + HistogramData emptyHistogram = HistogramData.linear(0, 10, 10); + histograms.put(bigQueryMetric3, emptyHistogram); + + String step = "testStep"; + Map emptyCounters = new HashMap<>(); + Collection conversionResult = + MetricsToPerStepNamespaceMetricsConverter.convert(step, emptyCounters, histograms); + + // Expected value 1 + List bucketCounts1 = ImmutableList.of(0L, 1L, 1L, 1L); + + Linear linearOptions1 = new Linear().setNumberOfBuckets(10).setWidth(10.0).setStart(0.0); + BucketOptions bucketOptions1 = new BucketOptions().setLinear(linearOptions1); + + OutlierStats outlierStats1 = + new OutlierStats() + .setUnderflowCount(1L) + .setUnderflowMean(-5.0) + .setOverflowCount(1L) + .setOverflowMean(105.0); + DataflowHistogramValue linearHistogram1 = + new DataflowHistogramValue() + .setCount(5L) + .setBucketOptions(bucketOptions1) + .setBucketCounts(bucketCounts1) + .setOutlierStats(outlierStats1); + + MetricValue expectedVal1 = + new MetricValue() + .setMetric("baseLabel") + .setMetricLabels(new HashMap<>()) + .setValueHistogram(linearHistogram1); + + // Expected value 2 + List bucketCounts2 = ImmutableList.of(0L, 0L, 0L, 1L, 1L); + OutlierStats outlierStats2 = + new OutlierStats() + .setUnderflowCount(1L) + .setUnderflowMean(-5.0) + .setOverflowCount(2L) + .setOverflowMean(70.0); + Base2Exponent exponentialOptions2 = new Base2Exponent().setNumberOfBuckets(5).setScale(0); + + BucketOptions bucketOptions2 = new BucketOptions().setExponential(exponentialOptions2); + + DataflowHistogramValue exponentialHistogram2 = + new DataflowHistogramValue() + .setCount(5L) + .setBucketOptions(bucketOptions2) + .setBucketCounts(bucketCounts2) + .setOutlierStats(outlierStats2); + + Map metric2Labels = new HashMap<>(); + metric2Labels.put("label1", "val1"); + metric2Labels.put("label2", "val2"); + MetricValue expectedVal2 = + new MetricValue() + .setMetric("baseLabel") + .setValueHistogram(exponentialHistogram2) + .setMetricLabels(metric2Labels); + + assertThat(conversionResult.size(), equalTo(1)); + PerStepNamespaceMetrics perStepNamespaceMetrics = conversionResult.iterator().next(); + + assertThat(perStepNamespaceMetrics.getOriginalStep(), equalTo(step)); + assertThat(perStepNamespaceMetrics.getMetricsNamespace(), equalTo("BigQuerySink")); + assertThat(perStepNamespaceMetrics.getMetricValues().size(), equalTo(2)); + assertThat( + perStepNamespaceMetrics.getMetricValues(), containsInAnyOrder(expectedVal1, expectedVal2)); + } + + @Test + public void testConvert_skipUnknownHistogramBucketType() { + String step = "testStep"; + Map emptyCounters = new HashMap<>(); + Map histograms = new HashMap(); + + HistogramData histogram = new HistogramData(new TestBucketType()); + histogram.record(1.0, 2.0); + MetricName bigQueryMetric1 = MetricName.named("BigQuerySink", "baseLabel"); + histograms.put(bigQueryMetric1, histogram); + + Collection conversionResult = + MetricsToPerStepNamespaceMetricsConverter.convert(step, emptyCounters, histograms); + assertThat(conversionResult.size(), equalTo(0)); + } + + @Test + public void testConvert_convertCountersAndHistograms() { + String step = "testStep"; + Map counters = new HashMap<>(); + Map histograms = new HashMap(); + + MetricName counterMetricName = MetricName.named("BigQuerySink", "counter-label1:val1;"); + counters.put(counterMetricName, 3L); + + MetricName histogramMetricName = MetricName.named("BigQuerySink", "histogram-label2:val2;"); + HistogramData linearHistogram = HistogramData.linear(0, 10, 10); + linearHistogram.record(5.0); + histograms.put(histogramMetricName, linearHistogram); + + Collection conversionResult = + MetricsToPerStepNamespaceMetricsConverter.convert(step, counters, histograms); + + // Expected counter MetricValue + Map counterLabelMap = new HashMap<>(); + counterLabelMap.put("label1", "val1"); + MetricValue expectedCounter = + new MetricValue().setMetric("counter").setValueInt64(3L).setMetricLabels(counterLabelMap); + + // Expected histogram MetricValue + List bucketCounts1 = ImmutableList.of(1L); + + Linear linearOptions1 = new Linear().setNumberOfBuckets(10).setWidth(10.0).setStart(0.0); + BucketOptions bucketOptions1 = new BucketOptions().setLinear(linearOptions1); + + OutlierStats outlierStats1 = + new OutlierStats() + .setUnderflowCount(0L) + .setUnderflowMean(0.0) + .setOverflowCount(0L) + .setOverflowMean(0.0); + DataflowHistogramValue linearHistogram1 = + new DataflowHistogramValue() + .setCount(1L) + .setBucketOptions(bucketOptions1) + .setBucketCounts(bucketCounts1) + .setOutlierStats(outlierStats1); + + Map histogramLabelMap = new HashMap<>(); + histogramLabelMap.put("label2", "val2"); + + MetricValue expectedHistogram = + new MetricValue() + .setMetric("histogram") + .setMetricLabels(histogramLabelMap) + .setValueHistogram(linearHistogram1); + + assertThat(conversionResult.size(), equalTo(1)); + PerStepNamespaceMetrics perStepNamespaceMetrics = conversionResult.iterator().next(); + + assertThat(perStepNamespaceMetrics.getOriginalStep(), equalTo(step)); + assertThat(perStepNamespaceMetrics.getMetricsNamespace(), equalTo("BigQuerySink")); + assertThat(perStepNamespaceMetrics.getMetricValues().size(), equalTo(2)); + assertThat( + perStepNamespaceMetrics.getMetricValues(), + containsInAnyOrder(expectedCounter, expectedHistogram)); + } +} From 405e6c35bb8dc0440c5635000b5375474a3409ae Mon Sep 17 00:00:00 2001 From: Jeffrey Kinard Date: Tue, 23 Jan 2024 16:35:53 -0500 Subject: [PATCH 054/169] [YAML] Use default logging level for main.py Signed-off-by: Jeffrey Kinard --- sdks/python/apache_beam/yaml/main.py | 1 - 1 file changed, 1 deletion(-) diff --git a/sdks/python/apache_beam/yaml/main.py b/sdks/python/apache_beam/yaml/main.py index 26ddcc6340e7..06173aaf60fc 100644 --- a/sdks/python/apache_beam/yaml/main.py +++ b/sdks/python/apache_beam/yaml/main.py @@ -65,7 +65,6 @@ def _pipeline_spec_from_args(known_args): def run(argv=None): - yaml_transform._LOGGER.setLevel('INFO') known_args, pipeline_args = _configure_parser(argv) pipeline_yaml = _pipeline_spec_from_args(known_args) pipeline_spec = yaml.load(pipeline_yaml, Loader=yaml_transform.SafeLineLoader) From 8f3887bb512785893ad3939ac48d459bdfe103aa Mon Sep 17 00:00:00 2001 From: Jeffrey Kinard Date: Fri, 19 Jan 2024 20:24:26 -0500 Subject: [PATCH 055/169] [YAML] Require numStreams for unbounded BigQueryStorageWriteApiSchemaTransform xlang transform Signed-off-by: Jeffrey Kinard --- .../BigQueryStorageWriteApiSchemaTransformProvider.java | 8 ++++++++ sdks/python/apache_beam/yaml/standard_io.yaml | 2 ++ 2 files changed, 10 insertions(+) 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 8c4edd2244b4..52ce97294aa1 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 @@ -349,6 +349,14 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { Long triggeringFrequency = configuration.getTriggeringFrequencySeconds(); Boolean autoSharding = configuration.getAutoSharding(); int numStreams = configuration.getNumStreams() == null ? 0 : configuration.getNumStreams(); + + // TODO(https://github.com/apache/beam/issues/30058): remove once Dataflow supports multiple + // DoFn's per fused step. + if (numStreams < 1) { + throw new IllegalStateException( + "numStreams must be set to a positive integer when input data is unbounded."); + } + boolean useAtLeastOnceSemantics = configuration.getUseAtLeastOnceSemantics() == null ? false diff --git a/sdks/python/apache_beam/yaml/standard_io.yaml b/sdks/python/apache_beam/yaml/standard_io.yaml index 5d1598d2705c..4d26ce96b677 100644 --- a/sdks/python/apache_beam/yaml/standard_io.yaml +++ b/sdks/python/apache_beam/yaml/standard_io.yaml @@ -38,6 +38,8 @@ create_disposition: 'createDisposition' write_disposition: 'writeDisposition' error_handling: 'errorHandling' + # TODO(https://github.com/apache/beam/issues/30058): Required until autosharding support is fixed + num_streams: 'numStreams' underlying_provider: type: beamJar transforms: From 9e5b112be9f5bb0aee950f9119d2be31e29d8a50 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Tue, 23 Jan 2024 21:53:48 +0000 Subject: [PATCH 056/169] Use FileSystems.exist to check the remote the path (#30082) * Add Filesystems.exists instead of os * Update sdks/python/apache_beam/ml/transforms/handlers_test.py * use FileSystems * add / to the path --- .../apache_beam/ml/transforms/handlers.py | 3 ++- .../ml/transforms/handlers_test.py | 20 ++++++++++++++++--- 2 files changed, 19 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/ml/transforms/handlers.py b/sdks/python/apache_beam/ml/transforms/handlers.py index 6c290aa13d58..3c37ddef1ed5 100644 --- a/sdks/python/apache_beam/ml/transforms/handlers.py +++ b/sdks/python/apache_beam/ml/transforms/handlers.py @@ -31,6 +31,7 @@ import apache_beam as beam import tensorflow as tf import tensorflow_transform.beam as tft_beam +from apache_beam.io.filesystems import FileSystems from apache_beam.ml.transforms.base import ArtifactMode from apache_beam.ml.transforms.base import ProcessHandler from apache_beam.ml.transforms.tft import _EXPECTED_TYPES @@ -435,7 +436,7 @@ def expand( path=os.path.join(self.artifact_location, RAW_DATA_METADATA_DIR)) else: # Read the metadata from the artifact_location. - if not os.path.exists(os.path.join( + if not FileSystems.exists(os.path.join( self.artifact_location, RAW_DATA_METADATA_DIR, SCHEMA_FILE)): raise FileNotFoundError( "Artifacts not found at location: %s when using " diff --git a/sdks/python/apache_beam/ml/transforms/handlers_test.py b/sdks/python/apache_beam/ml/transforms/handlers_test.py index f13a916824c4..1331f1308087 100644 --- a/sdks/python/apache_beam/ml/transforms/handlers_test.py +++ b/sdks/python/apache_beam/ml/transforms/handlers_test.py @@ -22,6 +22,7 @@ import tempfile import typing import unittest +import uuid from typing import List from typing import NamedTuple from typing import Union @@ -30,6 +31,7 @@ from parameterized import parameterized import apache_beam as beam +from apache_beam.io.filesystems import FileSystems # pylint: disable=wrong-import-position, ungrouped-imports try: @@ -301,11 +303,14 @@ def test_tft_process_handler_verify_artifacts(self): _ = raw_data | process_handler self.assertTrue( - os.path.exists( + FileSystems.exists( + # To check the gcs directory with FileSystems, the dir path must + # end with / os.path.join( - self.artifact_location, handlers.RAW_DATA_METADATA_DIR))) + self.artifact_location, + handlers.RAW_DATA_METADATA_DIR + '/'))) self.assertTrue( - os.path.exists( + FileSystems.exists( os.path.join( self.artifact_location, handlers.RAW_DATA_METADATA_DIR, @@ -616,5 +621,14 @@ def test_handler_with_same_input_elements(self): label='transformed data') +class TFTProcessHandlerTestWithGCSLocation(TFTProcessHandlerTest): + def setUp(self) -> None: + self.artifact_location = self.gcs_artifact_location = os.path.join( + 'gs://temp-storage-for-perf-tests/tft_handler', uuid.uuid4().hex) + + def tearDown(self): + pass + + if __name__ == '__main__': unittest.main() From 9dd307654768a78cffe1bb0d32e270450a503b75 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 23 Jan 2024 15:31:56 -0800 Subject: [PATCH 057/169] Bump pillow from 10.0.1 to 10.2.0 in /sdks/python/apache_beam/examples/ml-orchestration/kfp/components/preprocessing (#30071) Bumps [pillow](https://github.com/python-pillow/Pillow) from 10.0.1 to 10.2.0. - [Release notes](https://github.com/python-pillow/Pillow/releases) - [Changelog](https://github.com/python-pillow/Pillow/blob/main/CHANGES.rst) - [Commits](https://github.com/python-pillow/Pillow/compare/10.0.1...10.2.0) --- updated-dependencies: - dependency-name: pillow dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .../kfp/components/preprocessing/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/examples/ml-orchestration/kfp/components/preprocessing/requirements.txt b/sdks/python/apache_beam/examples/ml-orchestration/kfp/components/preprocessing/requirements.txt index 706adf9de0aa..5a1b6f30b83e 100644 --- a/sdks/python/apache_beam/examples/ml-orchestration/kfp/components/preprocessing/requirements.txt +++ b/sdks/python/apache_beam/examples/ml-orchestration/kfp/components/preprocessing/requirements.txt @@ -18,4 +18,4 @@ requests==2.31.0 torch==1.13.1 torchvision==0.13.0 numpy==1.22.4 -Pillow==10.0.1 +Pillow==10.2.0 From 7ff25d896250508570b27683bc76523ac2fe3210 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Tue, 23 Jan 2024 15:46:17 -0800 Subject: [PATCH 058/169] [YAML] Allow windowing to be done in Java or Python. (#30055) --- .../beam/sdk/transforms/windowing/Window.java | 12 +- .../service/WindowIntoTransformProvider.java | 127 ++++++++++++++++++ sdks/python/apache_beam/yaml/yaml_provider.py | 77 +++++++++++ .../python/apache_beam/yaml/yaml_transform.py | 9 +- .../yaml/yaml_transform_unit_test.py | 2 +- 5 files changed, 220 insertions(+), 7 deletions(-) create mode 100644 sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/WindowIntoTransformProvider.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java index c22aa35d8f3a..32e218d3cb11 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java @@ -454,7 +454,7 @@ protected String getKindString() { * Pipeline authors should use {@link Window} directly instead. */ public static class Assign extends PTransform, PCollection> { - private final Window original; + private final @Nullable Window original; private final WindowingStrategy updatedStrategy; /** @@ -463,7 +463,7 @@ public static class Assign extends PTransform, PCollection> * #getWindowFn()}. */ @VisibleForTesting - Assign(Window original, WindowingStrategy updatedStrategy) { + Assign(@Nullable Window original, WindowingStrategy updatedStrategy) { this.original = original; this.updatedStrategy = updatedStrategy; } @@ -476,12 +476,18 @@ public PCollection expand(PCollection input) { @Override public void populateDisplayData(DisplayData.Builder builder) { - original.populateDisplayData(builder); + if (original != null) { + original.populateDisplayData(builder); + } } public @Nullable WindowFn getWindowFn() { return updatedStrategy.getWindowFn(); } + + public static Assign createInternal(WindowingStrategy finalStrategy) { + return new Assign(null, finalStrategy); + } } /** diff --git a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/WindowIntoTransformProvider.java b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/WindowIntoTransformProvider.java new file mode 100644 index 000000000000..6272b9445eb8 --- /dev/null +++ b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/WindowIntoTransformProvider.java @@ -0,0 +1,127 @@ +/* + * 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.expansion.service; + +import com.google.auto.service.AutoService; +import com.google.auto.value.AutoValue; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.runners.core.construction.WindowingStrategyTranslation; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.WindowingStrategy; + +/** + * An implementation of {@link TypedSchemaTransformProvider} for WindowInto. + * + *

Internal only: This class is actively being worked on, and it will likely change. We + * provide no backwards compatibility guarantees, and it should not be implemented outside the Beam + * repository. + */ +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +@AutoService(SchemaTransformProvider.class) +public class WindowIntoTransformProvider + extends TypedSchemaTransformProvider { + private static final String INPUT_ROWS_TAG = "input"; + + protected static final String OUTPUT_ROWS_TAG = "output"; + + @Override + protected Class configurationClass() { + return Configuration.class; + } + + @Override + protected SchemaTransform from(Configuration configuration) { + try { + return new WindowIntoStrategy( + (WindowingStrategy) + WindowingStrategyTranslation.fromProto( + RunnerApi.WindowingStrategy.parseFrom( + configuration.getSerializedWindowingStrategy()), + null)); + } catch (IOException exn) { + throw new RuntimeException(exn); + } + } + + @Override + public String identifier() { + return "beam:schematransform:org.apache.beam:yaml:window_into_strategy:v1"; + } + + @Override + public List inputCollectionNames() { + return Collections.singletonList(INPUT_ROWS_TAG); + } + + @Override + public List outputCollectionNames() { + return Collections.singletonList(OUTPUT_ROWS_TAG); + } + + @DefaultSchema(AutoValueSchema.class) + @AutoValue + public abstract static class Configuration { + + @SuppressWarnings({"AutoValueMutable", "mutable"}) + public abstract byte[] getSerializedWindowingStrategy(); + + public static Builder builder() { + return new AutoValue_WindowIntoTransformProvider_Configuration.Builder(); + } + + @AutoValue.Builder + public abstract static class Builder { + + public abstract Builder setSerializedWindowingStrategy(byte[] serializedWindowingStrategy); + + public abstract Configuration build(); + } + } + + private static class WindowIntoStrategy extends SchemaTransform { + + private final WindowingStrategy windowingStrategy; + + WindowIntoStrategy(WindowingStrategy windowingStrategy) { + this.windowingStrategy = windowingStrategy; + } + + @Override + public PCollectionRowTuple expand(PCollectionRowTuple inputTuple) { + PCollection input = inputTuple.get(INPUT_ROWS_TAG); + return PCollectionRowTuple.of( + OUTPUT_ROWS_TAG, + input + .apply(Window.Assign.createInternal(windowingStrategy)) + .setCoder(input.getCoder())); + } + } +} diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index 99c98f891de4..5e9da3b0b05a 100755 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -46,6 +46,7 @@ import apache_beam.io import apache_beam.transforms.util from apache_beam.portability.api import schema_pb2 +from apache_beam.runners import pipeline_context from apache_beam.transforms import external from apache_beam.transforms import window from apache_beam.transforms.fully_qualified_named_transform import FullyQualifiedNamedTransform @@ -755,6 +756,81 @@ def create_builtin_provider(): no_input_transforms=('Create', )) +class TranslatingProvider(Provider): + def __init__( + self, + transforms: Mapping[str, Callable[..., beam.PTransform]], + underlying_provider: Provider): + self._transforms = transforms + self._underlying_provider = underlying_provider + + def provided_transforms(self): + return self._transforms.keys() + + def available(self): + return self._underlying_provider.available() + + def cache_artifacts(self): + return self._underlying_provider.cache_artifacts() + + def underlying_provider(self): + return self._underlying_provider + + def to_json(self): + return {'type': "TranslatingProvider"} + + def create_transform( + self, typ: str, config: Mapping[str, Any], + yaml_create_transform: Any) -> beam.PTransform: + return self._transforms[typ](self._underlying_provider, **config) + + +def create_java_builtin_provider(): + """Exposes built-in transforms from Java as well as Python to maximize + opportunities for fusion. + + This class holds those transforms that require pre-processing of the configs. + For those Java transforms that can consume the user-provided configs directly + (or only need a simple renaming of parameters) a direct or renaming provider + is the simpler choice. + """ + + # An alternative could be examining the capabilities of various environments + # during (or as a pre-processing phase before) fusion to align environments + # where possible. This would also require extra care in skipping these + # common transforms when doing the provider affinity analysis. + + def java_window_into(java_provider, **config): + """Parses the config into a WindowingStrategy and invokes the Java class. + + Though it would not be that difficult to implement this in Java as well, + we prefer to implement it exactly once for consistency (especially as + it evolves). + """ + windowing_strategy = YamlProviders.WindowInto._parse_window_spec( + config).get_windowing(None) + # No context needs to be preserved for the basic WindowFns. + empty_context = pipeline_context.PipelineContext() + return java_provider.create_transform( + 'WindowIntoStrategy', + { + 'serializedWindowingStrategy': windowing_strategy.to_runner_api( + empty_context).SerializeToString() + }, + None) + + return TranslatingProvider( + transforms={'WindowInto': java_window_into}, + underlying_provider=beam_jar( + urns={ + 'WindowIntoStrategy': ( + 'beam:schematransform:' + 'org.apache.beam:yaml:window_into_strategy:v1') + }, + gradle_target= + 'sdks:java:extensions:schemaio-expansion-service:shadowJar')) + + class PypiExpansionService: """Expands transforms by fully qualified name in a virtual environment with the given dependencies. @@ -993,6 +1069,7 @@ def standard_providers(): return merge_providers( YamlProviders.create_builtin_provider(), + create_java_builtin_provider(), create_mapping_providers(), create_combine_providers(), io_providers(), diff --git a/sdks/python/apache_beam/yaml/yaml_transform.py b/sdks/python/apache_beam/yaml/yaml_transform.py index ff20685489c2..03574b5f98ff 100644 --- a/sdks/python/apache_beam/yaml/yaml_transform.py +++ b/sdks/python/apache_beam/yaml/yaml_transform.py @@ -237,8 +237,9 @@ def followers(self, transform_name): for transform in self._transforms: if transform['type'] != 'composite': for input in empty_if_explicitly_empty(transform['input']).values(): - transform_id, _ = self.get_transform_id_and_output_name(input) - self._all_followers[transform_id].append(transform['__uuid__']) + if input not in self._inputs: + transform_id, _ = self.get_transform_id_and_output_name(input) + self._all_followers[transform_id].append(transform['__uuid__']) return self._all_followers[self.get_transform_id(transform_name)] def compute_all(self): @@ -738,7 +739,9 @@ def preprocess_windowing(spec): 'type': 'WindowInto', 'name': f'WindowInto[{key}]', 'windowing': windowing, - 'input': key, + 'input': { + 'input': key + }, '__line__': spec['__line__'], '__uuid__': SafeLineLoader.create_uuid(), } for key in original_inputs.keys()] diff --git a/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py b/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py index 00e509310d6f..a9e4db29d19c 100644 --- a/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py +++ b/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py @@ -653,7 +653,7 @@ def test_preprocess_windowing_custom_type(self): windowing: type: fixed size: 4 - input: input + input: {{input: input}} output: {result['transforms'][0]['__uuid__']} ''' self.assertYaml(expected, result) From b22fe9287b0744973a81013ce51ee5f9aa16fbbc Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Tue, 23 Jan 2024 23:11:24 -0800 Subject: [PATCH 059/169] [YAML] LogForTesting improvements. (#30005) * Prints elements in a more universal format (json rather than BeamSchema_xxxx). * Also adds (basic!) options for log level and a message identifier. * Also add java equivalent. --- .../providers/LoggingTransformProvider.java | 188 ++++++++++++++++++ .../apache_beam/yaml/standard_providers.yaml | 1 + sdks/python/apache_beam/yaml/yaml_provider.py | 36 +++- .../yaml/yaml_transform_unit_test.py | 4 + 4 files changed, 227 insertions(+), 2 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/LoggingTransformProvider.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/LoggingTransformProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/LoggingTransformProvider.java new file mode 100644 index 000000000000..25efaeae2a0e --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/LoggingTransformProvider.java @@ -0,0 +1,188 @@ +/* + * 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.schemas.transforms.providers; + +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 javax.annotation.Nullable; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.schemas.utils.JsonUtils; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; +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.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.slf4j.event.Level; + +/** + * An implementation of {@link TypedSchemaTransformProvider} for Logging. + * + *

Specifically, this is used by YAML's LogForTesting. + * + *

Internal only: This class is actively being worked on, and it will likely change. We + * provide no backwards compatibility guarantees, and it should not be implemented outside the Beam + * repository. + */ +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +@AutoService(SchemaTransformProvider.class) +public class LoggingTransformProvider + extends TypedSchemaTransformProvider { + + protected static final String INPUT_ROWS_TAG = "input"; + protected static final String OUTPUT_ROWS_TAG = "output"; + + @Override + protected Class configurationClass() { + return Configuration.class; + } + + @Override + protected SchemaTransform from(Configuration configuration) { + return new LoggingTransform(configuration); + } + + @Override + public String identifier() { + return "beam:schematransform:org.apache.beam:yaml:log_for_testing:v1"; + } + + @Override + public List inputCollectionNames() { + return Collections.singletonList(INPUT_ROWS_TAG); + } + + @Override + public List outputCollectionNames() { + return Collections.singletonList(OUTPUT_ROWS_TAG); + } + + @DefaultSchema(AutoValueSchema.class) + @AutoValue + public abstract static class Configuration { + private static final Map SUPPORTED_LOG_LEVELS = + ImmutableMap.of("ERROR", Level.ERROR, "INFO", Level.INFO, "DEBUG", Level.DEBUG); + + @Nullable + public abstract String getLevel(); + + public Level getLogLevel() { + if (getLevel() == null) { + return Level.INFO; + } else if (SUPPORTED_LOG_LEVELS.containsKey(getLevel())) { + return SUPPORTED_LOG_LEVELS.get(getLevel()); + } else { + throw new IllegalArgumentException( + "Unknown log level " + + getLevel() + + ". Valid log levels are " + + ImmutableList.copyOf(SUPPORTED_LOG_LEVELS.keySet())); + } + } + + @Nullable + public abstract String getPrefix(); + + public String getNonNullPrefix() { + String prefix = getPrefix(); + return prefix == null ? "" : prefix; + } + + public static Builder builder() { + return new AutoValue_LoggingTransformProvider_Configuration.Builder(); + } + + @AutoValue.Builder + public abstract static class Builder { + + public abstract Builder setLevel(@Nullable String level); + + public abstract Builder setPrefix(@Nullable String prefix); + + public abstract Configuration build(); + } + } + + /** A {@link SchemaTransform} for logging. */ + protected static class LoggingTransform extends SchemaTransform { + + private static final Logger LOG = LoggerFactory.getLogger(LoggingTransform.class); + + private final Configuration configuration; + + LoggingTransform(Configuration configuration) { + this.configuration = configuration; + } + + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + Schema inputSchema = input.get(INPUT_ROWS_TAG).getSchema(); + PCollection result = + input + .get(INPUT_ROWS_TAG) + .apply( + "LogAsJson", + ParDo.of( + createDoFn( + configuration.getLogLevel(), + configuration.getNonNullPrefix(), + inputSchema))) + .setRowSchema(inputSchema); + return PCollectionRowTuple.of(OUTPUT_ROWS_TAG, result); + } + + private static DoFn createDoFn(Level logLevel, String prefix, Schema rowSchema) { + SerializableFunction fn = JsonUtils.getRowToJsonBytesFunction(rowSchema); + return new DoFn() { + @ProcessElement + public void processElement(@Element Row row, OutputReceiver out) { + String msg = prefix + new String(fn.apply(row), Charsets.UTF_8); + // Looks like this is the best we can do. + // https://stackoverflow.com/questions/2621701/setting-log-level-of-message-at-runtime-in-slf4j + switch (logLevel) { + case DEBUG: + LOG.debug(msg); + break; + case INFO: + LOG.info(msg); + break; + case ERROR: + default: + LOG.error(msg); + } + out.output(row); + } + }; + } + } +} diff --git a/sdks/python/apache_beam/yaml/standard_providers.yaml b/sdks/python/apache_beam/yaml/standard_providers.yaml index 67ab8da502f8..c4671fa93fcd 100644 --- a/sdks/python/apache_beam/yaml/standard_providers.yaml +++ b/sdks/python/apache_beam/yaml/standard_providers.yaml @@ -24,6 +24,7 @@ version: BEAM_VERSION transforms: Sql: 'beam:external:java:sql:v1' + LogForTesting: 'beam:schematransform:org.apache.beam:yaml:log_for_testing:v1' - type: renaming transforms: diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index 5e9da3b0b05a..2e543ead43f6 100755 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -732,14 +732,46 @@ def _parse_window_spec(spec): return beam.WindowInto(window_fn) @staticmethod - def log_for_testing(): + def log_for_testing( + level: Optional[str] = 'INFO', prefix: Optional[str] = ''): """Logs each element of its input PCollection. The output of this transform is a copy of its input for ease of use in chain-style pipelines. + + Args: + level: one of ERROR, INFO, or DEBUG, mapped to a corresponding + language-specific logging level + prefix: an optional identifier that will get prepended to the element + being logged """ + # Keeping this simple to be language agnostic. + # The intent is not to develop a logging library (and users can always do) + # their own mappings to get fancier output. + log_levels = { + 'ERROR': logging.error, + 'INFO': logging.info, + 'DEBUG': logging.debug, + } + if level not in log_levels: + raise ValueError( + f'Unknown log level {level} not in {list(log_levels.keys())}') + logger = log_levels[level] + + def to_loggable_json_recursive(o): + if isinstance(o, (str, bytes)): + return o + elif callable(getattr(o, '_asdict', None)): + return to_loggable_json_recursive(o._asdict()) + elif isinstance(o, Mapping) and callable(getattr(o, 'items', None)): + return {str(k): to_loggable_json_recursive(v) for k, v in o.items()} + elif isinstance(o, Iterable): + return [to_loggable_json_recursive(x) for x in o] + else: + return o + def log_and_return(x): - logging.info(x) + logger(prefix + json.dumps(to_loggable_json_recursive(x))) return x return "LogForTesting" >> beam.Map(log_and_return) diff --git a/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py b/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py index a9e4db29d19c..39f31619a741 100644 --- a/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py +++ b/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py @@ -110,6 +110,10 @@ def get_scope_by_spec(self, p, spec, inputs=None): if inputs is None: inputs = {} spec = yaml.load(spec, Loader=SafeLineLoader) + if 'transforms' in spec: + spec['transforms'] = [ + normalize_inputs_outputs(t) for t in spec['transforms'] + ] scope = Scope( beam.pvalue.PBegin(p), From 0eefb797765a8fdabb43443d934cf7bf3fbbcc69 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Tue, 23 Jan 2024 23:16:43 -0800 Subject: [PATCH 060/169] [YAML] Expose flatten implementation from Java. (#30057) Technically Flatten will produce exactly the same result no matter where it is expanded, but simply having this in Java avoids complexities around dealing with affinity optimization. --- .../providers/FlattenTransformProvider.java | 100 ++++++++++++++++++ .../apache_beam/yaml/standard_providers.yaml | 26 +++++ 2 files changed, 126 insertions(+) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/FlattenTransformProvider.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/FlattenTransformProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/FlattenTransformProvider.java new file mode 100644 index 000000000000..baeccd1ac8cc --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/FlattenTransformProvider.java @@ -0,0 +1,100 @@ +/* + * 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.schemas.transforms.providers; + +import com.google.auto.service.AutoService; +import com.google.auto.value.AutoValue; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionList; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; + +/** + * An implementation of {@link TypedSchemaTransformProvider} for Flatten. + * + *

Internal only: This class is actively being worked on, and it will likely change. We + * provide no backwards compatibility guarantees, and it should not be implemented outside the Beam + * repository. + */ +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +@AutoService(SchemaTransformProvider.class) +public class FlattenTransformProvider + extends TypedSchemaTransformProvider { + protected static final String OUTPUT_ROWS_TAG = "output"; + + @Override + protected Class configurationClass() { + return Configuration.class; + } + + @Override + protected SchemaTransform from(Configuration configuration) { + return new SchemaTransform() { + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + return PCollectionRowTuple.of( + OUTPUT_ROWS_TAG, + PCollectionList.of((Iterable>) (Collection) input.expand().values()) + .apply(Flatten.pCollections()) + .setRowSchema( + ((PCollection) input.expand().values().iterator().next()).getSchema())); + } + }; + } + + @Override + public String identifier() { + return "beam:schematransform:org.apache.beam:yaml:flatten:v1"; + } + + @Override + public List inputCollectionNames() { + return Collections.emptyList(); + } + + @Override + public List outputCollectionNames() { + return Collections.singletonList(OUTPUT_ROWS_TAG); + } + + @DefaultSchema(AutoValueSchema.class) + @AutoValue + public abstract static class Configuration { + + public static Builder builder() { + return new AutoValue_FlattenTransformProvider_Configuration.Builder(); + } + + @AutoValue.Builder + public abstract static class Builder { + + public abstract Configuration build(); + } + } +} diff --git a/sdks/python/apache_beam/yaml/standard_providers.yaml b/sdks/python/apache_beam/yaml/standard_providers.yaml index c4671fa93fcd..89b0cc9d553e 100644 --- a/sdks/python/apache_beam/yaml/standard_providers.yaml +++ b/sdks/python/apache_beam/yaml/standard_providers.yaml @@ -24,6 +24,32 @@ version: BEAM_VERSION transforms: Sql: 'beam:external:java:sql:v1' + Flatten: 'beam:schematransform:org.apache.beam:yaml:flatten:v1' + LogForTesting: 'beam:schematransform:org.apache.beam:yaml:log_for_testing:v1' + +# TODO(robertwb): Auto-detect redundantly provided transforms for maximal fusion. +- type: 'beamJar' + config: + gradle_target: 'sdks:java:io:expansion-service:shadowJar' + version: BEAM_VERSION + transforms: + Flatten: 'beam:schematransform:org.apache.beam:yaml:flatten:v1' + LogForTesting: 'beam:schematransform:org.apache.beam:yaml:log_for_testing:v1' + +- type: 'beamJar' + config: + gradle_target: 'sdks:java:io:google-cloud-platform:expansion-service:shadowJar' + version: BEAM_VERSION + transforms: + Flatten: 'beam:schematransform:org.apache.beam:yaml:flatten:v1' + LogForTesting: 'beam:schematransform:org.apache.beam:yaml:log_for_testing:v1' + +- type: 'beamJar' + config: + gradle_target: 'sdks:java:extensions:schemaio-expansion-service:shadowJar' + version: BEAM_VERSION + transforms: + Flatten: 'beam:schematransform:org.apache.beam:yaml:flatten:v1' LogForTesting: 'beam:schematransform:org.apache.beam:yaml:log_for_testing:v1' - type: renaming From 582d5c03569186fbff5b48fc890cd6ed7c1fb283 Mon Sep 17 00:00:00 2001 From: Neeraj Bansal <51872638+nrj179@users.noreply.github.com> Date: Wed, 24 Jan 2024 20:52:08 +0530 Subject: [PATCH 061/169] fix for #30078 (#30079) Co-authored-by: neeraj --- .../main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java index e281e559a544..1080999271d3 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java @@ -566,6 +566,8 @@ public static Read readMessagesWithAttributesAndMessageId() { public static Read readMessagesWithAttributesAndMessageIdAndOrderingKey() { return Read.newBuilder() .setCoder(PubsubMessageWithAttributesAndMessageIdAndOrderingKeyCoder.of()) + .setNeedsAttributes(true) + .setNeedsMessageId(true) .setNeedsOrderingKey(true) .build(); } From 98a86b11cc767774dcbc7a92e03bd19a9656d7b7 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Wed, 24 Jan 2024 16:41:20 +0000 Subject: [PATCH 062/169] [MLTransform]Throw an error when specifiend columns are not present in the input data (#30085) * throw a warning when columns are not present in the input data * Add a check to see if the input dict have same keys --- sdks/python/apache_beam/ml/transforms/base.py | 13 +++++ .../apache_beam/ml/transforms/base_test.py | 47 +++++++++++++++++-- 2 files changed, 57 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/ml/transforms/base.py b/sdks/python/apache_beam/ml/transforms/base.py index 64ad0286e53d..a297f62f5bc5 100644 --- a/sdks/python/apache_beam/ml/transforms/base.py +++ b/sdks/python/apache_beam/ml/transforms/base.py @@ -69,7 +69,14 @@ def _convert_list_of_dicts_to_dict_of_lists( list_of_dicts: Sequence[Dict[str, Any]]) -> Dict[str, List[Any]]: keys_to_element_list = collections.defaultdict(list) + input_keys = list_of_dicts[0].keys() for d in list_of_dicts: + if set(d.keys()) != set(input_keys): + extra_keys = set(d.keys()) - set(input_keys) if len( + d.keys()) > len(input_keys) else set(input_keys) - set(d.keys()) + raise RuntimeError( + f'All the dicts in the input data should have the same keys. ' + f'Got: {extra_keys} instead.') for key, value in d.items(): keys_to_element_list[key].append(value) return keys_to_element_list @@ -629,6 +636,12 @@ def _process_batch( model: ModelT, inference_args: Optional[Dict[str, Any]]) -> Dict[str, List[Any]]: result: Dict[str, List[Any]] = collections.defaultdict(list) + input_keys = dict_batch.keys() + missing_columns_in_data = set(self.columns) - set(input_keys) + if missing_columns_in_data: + raise RuntimeError( + f'Data does not contain the following columns ' + f': {missing_columns_in_data}.') for key, batch in dict_batch.items(): if key in self.columns: self._validate_column_data(batch) diff --git a/sdks/python/apache_beam/ml/transforms/base_test.py b/sdks/python/apache_beam/ml/transforms/base_test.py index 44820c23cc2f..3a83a115e424 100644 --- a/sdks/python/apache_beam/ml/transforms/base_test.py +++ b/sdks/python/apache_beam/ml/transforms/base_test.py @@ -334,8 +334,8 @@ def load_model(self): class FakeEmbeddingsManager(base.EmbeddingsManager): - def __init__(self, columns): - super().__init__(columns=columns) + def __init__(self, columns, **kwargs): + super().__init__(columns=columns, **kwargs) def get_model_handler(self) -> ModelHandler: FakeModelHandler.__repr__ = lambda x: 'FakeEmbeddingsManager' # type: ignore[assignment] @@ -416,7 +416,6 @@ def test_handler_with_batch_sizes(self): ) def test_handler_on_multiple_columns(self): - self.embedding_conig.columns = ['x', 'y'] data = [ { 'x': "Hello world", 'y': "Apache Beam", 'z': 'unchanged' @@ -443,6 +442,26 @@ def test_handler_on_multiple_columns(self): equal_to(expected_data), ) + def test_handler_on_columns_not_exist_in_input_data(self): + data = [ + { + 'x': "Hello world", 'y': "Apache Beam" + }, + { + 'x': "Apache Beam", 'y': "Hello world" + }, + ] + self.embedding_conig.columns = ['x', 'y', 'a'] + + with self.assertRaises(RuntimeError): + with beam.Pipeline() as p: + _ = ( + p + | beam.Create(data) + | base.MLTransform( + write_artifact_location=self.artifact_location).with_transform( + self.embedding_conig)) + def test_handler_with_list_data(self): data = [{ 'x': ['Hello world', 'Apache Beam'], @@ -458,6 +477,28 @@ def test_handler_with_list_data(self): write_artifact_location=self.artifact_location).with_transform( self.embedding_conig)) + def test_handler_with_inconsistent_keys(self): + data = [ + { + 'x': 'foo', 'y': 'bar', 'z': 'baz' + }, + { + 'x': 'foo2', 'y': 'bar2' + }, + { + 'x': 'foo3', 'y': 'bar3', 'z': 'baz3' + }, + ] + self.embedding_conig.min_batch_size = 2 + with self.assertRaises(RuntimeError): + with beam.Pipeline() as p: + _ = ( + p + | beam.Create(data) + | base.MLTransform( + write_artifact_location=self.artifact_location).with_transform( + self.embedding_conig)) + class TestUtilFunctions(unittest.TestCase): def test_list_of_dicts_to_dict_of_lists_normal(self): From f7da756ac6df7b0a1a8578cd5204bb6667011d80 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Wed, 24 Jan 2024 09:54:04 -0800 Subject: [PATCH 063/169] Update container build version to Go 1.21.6 (#30090) Co-authored-by: lostluck <13907733+lostluck@users.noreply.github.com> --- .../main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 3045e2615e4a..f3d79c528f8c 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -2206,7 +2206,8 @@ class BeamModulePlugin implements Plugin { def goRootDir = "${project.rootDir}/sdks/go" // This sets the whole project Go version. - project.ext.goVersion = "go1.21.5" + // The latest stable Go version can be checked at https://go.dev/dl/ + project.ext.goVersion = "go1.21.6" // Minor TODO: Figure out if we can pull out the GOCMD env variable after goPrepare script // completion, and avoid this GOBIN substitution. From e85d070a32b0d16706b56bf7a9c2eba596e2bab2 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Wed, 24 Jan 2024 13:21:50 -0500 Subject: [PATCH 064/169] Ignore DicomMetadataRead tests (#30096) --- .../org/apache/beam/sdk/io/gcp/healthcare/DicomIOReadIT.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/DicomIOReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/DicomIOReadIT.java index 570844011f39..f6f8e8b8fcc8 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/DicomIOReadIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/DicomIOReadIT.java @@ -27,6 +27,7 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; @@ -61,6 +62,7 @@ public void deleteDicomStore() throws IOException { client.deleteDicomStore(healthcareDataset + "/dicomStores/" + storeName); } + @Ignore("https://github.com/apache/beam/issues/28099") @Test public void testDicomMetadataRead() { String webPath = @@ -88,6 +90,7 @@ public void testDicomMetadataRead() { pipeline.run(); } + @Ignore("https://github.com/apache/beam/issues/28099") @Test public void testDicomFailedMetadataRead() { String badWebPath = "foo"; From 27214149d3423fd2b9e9d410bd758dcb97024f9c Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> Date: Wed, 24 Jan 2024 14:05:15 -0500 Subject: [PATCH 065/169] Support dynamic destinations with Python Storage API (#30045) * support dynamic destinations and add tests * put all relevant logic in StorageWriteToBigQuery --- ..._PostCommit_Python_Xlang_Gcp_Dataflow.json | 0 ...am_PostCommit_Python_Xlang_Gcp_Direct.json | 1 + CHANGES.md | 1 + ...torageWriteApiSchemaTransformProvider.java | 59 +++- ...geWriteApiSchemaTransformProviderTest.java | 54 +++- .../io/external/xlang_bigqueryio_it_test.py | 77 ++++- sdks/python/apache_beam/io/gcp/bigquery.py | 293 ++++++++++-------- 7 files changed, 334 insertions(+), 151 deletions(-) create mode 100644 .github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Dataflow.json create mode 100644 .github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Direct.json diff --git a/.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Dataflow.json b/.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Dataflow.json new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Direct.json b/.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Direct.json new file mode 100644 index 000000000000..8b137891791f --- /dev/null +++ b/.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Direct.json @@ -0,0 +1 @@ + diff --git a/CHANGES.md b/CHANGES.md index dbad15f3dba4..d4b9b7d77e07 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -63,6 +63,7 @@ ## I/Os * Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* Added support for writing to BigQuery dynamic destinations with Python's Storage Write API ([#30045](https://github.com/apache/beam/pull/30045)) * Adding support for Tuples DataType in ClickHouse (Java) ([#29715](https://github.com/apache/beam/pull/29715)). * Added support for handling bad records to FileIO, TextIO, AvroIO ([#29670](https://github.com/apache/beam/pull/29670)). * Added support for handling bad records to BigtableIO ([#29885](https://github.com/apache/beam/pull/29885)). 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 52ce97294aa1..d0951cdad1a6 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.TableSchema; import com.google.auto.service.AutoService; import com.google.auto.value.AutoValue; import java.util.Arrays; @@ -35,6 +36,8 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices; 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.TableDestination; import org.apache.beam.sdk.io.gcp.bigquery.WriteResult; import org.apache.beam.sdk.io.gcp.bigquery.providers.BigQueryStorageWriteApiSchemaTransformProvider.BigQueryStorageWriteApiSchemaTransformConfiguration; import org.apache.beam.sdk.metrics.Counter; @@ -56,6 +59,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.sdk.values.ValueInSingleWindow; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; 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; @@ -81,6 +85,8 @@ public class BigQueryStorageWriteApiSchemaTransformProvider private static final String INPUT_ROWS_TAG = "input"; private static final String FAILED_ROWS_TAG = "FailedRows"; 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"; @Override protected Class configurationClass() { @@ -161,7 +167,11 @@ public void validate() { checkArgument( !Strings.isNullOrEmpty(this.getTable()), invalidConfigMessage + "Table spec for a BigQuery Write must be specified."); - checkNotNull(BigQueryHelpers.parseTableSpec(this.getTable())); + + // if we have an input table spec, validate it + if (!this.getTable().equals(DYNAMIC_DESTINATIONS)) { + checkNotNull(BigQueryHelpers.parseTableSpec(this.getTable())); + } // validate create and write dispositions if (!Strings.isNullOrEmpty(this.getCreateDisposition())) { @@ -337,13 +347,36 @@ private static class NoOutputDoFn extends DoFn { public void process(ProcessContext c) {} } + private static class RowDynamicDestinations extends DynamicDestinations { + Schema schema; + + RowDynamicDestinations(Schema schema) { + this.schema = schema; + } + + @Override + public String getDestination(ValueInSingleWindow element) { + return element.getValue().getString("destination"); + } + + @Override + public TableDestination getTable(String destination) { + return new TableDestination(destination, null); + } + + @Override + public TableSchema getSchema(String destination) { + return BigQueryUtils.toTableSchema(schema); + } + } + @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { // Check that the input exists checkArgument(input.has(INPUT_ROWS_TAG), "Missing expected input tag: %s", INPUT_ROWS_TAG); PCollection inputRows = input.get(INPUT_ROWS_TAG); - BigQueryIO.Write write = createStorageWriteApiTransform(); + BigQueryIO.Write write = createStorageWriteApiTransform(inputRows.getSchema()); if (inputRows.isBounded() == IsBounded.UNBOUNDED) { Long triggeringFrequency = configuration.getTriggeringFrequencySeconds(); @@ -358,9 +391,8 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { } boolean useAtLeastOnceSemantics = - configuration.getUseAtLeastOnceSemantics() == null - ? false - : configuration.getUseAtLeastOnceSemantics(); + configuration.getUseAtLeastOnceSemantics() != null + && configuration.getUseAtLeastOnceSemantics(); // Triggering frequency is only applicable for exactly-once if (!useAtLeastOnceSemantics) { write = @@ -433,7 +465,7 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { } } - BigQueryIO.Write createStorageWriteApiTransform() { + BigQueryIO.Write createStorageWriteApiTransform(Schema schema) { Method writeMethod = configuration.getUseAtLeastOnceSemantics() != null && configuration.getUseAtLeastOnceSemantics() @@ -442,12 +474,23 @@ BigQueryIO.Write createStorageWriteApiTransform() { BigQueryIO.Write write = BigQueryIO.write() - .to(configuration.getTable()) .withMethod(writeMethod) - .useBeamSchema() .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."); + write = + write + .to(new RowDynamicDestinations(schema.getField("record").getType().getRowSchema())) + .withFormatFunction(row -> BigQueryUtils.toTableRow(row.getRow("record"))); + } else { + write = write.to(configuration.getTable()).useBeamSchema(); + } + if (!Strings.isNullOrEmpty(configuration.getCreateDisposition())) { CreateDisposition createDisposition = BigQueryStorageWriteApiSchemaTransformConfiguration.CREATE_DISPOSITIONS.get( 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 54c636bde5fe..64ea0b11d1b9 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 @@ -29,6 +29,7 @@ import java.util.Arrays; import java.util.List; import java.util.function.Function; +import java.util.stream.Collectors; 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; @@ -136,8 +137,8 @@ public PCollectionRowTuple runWithConfig( writeTransform.setBigQueryServices(fakeBigQueryServices); String tag = provider.inputCollectionNames().get(0); - - PCollection rows = p.apply(Create.of(inputRows).withRowSchema(SCHEMA)); + PCollection rows = + p.apply(Create.of(inputRows).withRowSchema(inputRows.get(0).getSchema())); PCollectionRowTuple input = PCollectionRowTuple.of(tag, rows); PCollectionRowTuple result = input.apply(writeTransform); @@ -155,16 +156,20 @@ public Boolean rowsEquals(List expectedRows, List actualRows) { TableRow actualRow = actualRows.get(i); Row expectedRow = expectedRows.get(Integer.parseInt(actualRow.get("number").toString()) - 1); - if (!expectedRow.getValue("name").equals(actualRow.get("name")) - || !expectedRow - .getValue("number") - .equals(Long.parseLong(actualRow.get("number").toString()))) { + if (!rowEquals(expectedRow, actualRow)) { return false; } } return true; } + public boolean rowEquals(Row expectedRow, TableRow actualRow) { + return expectedRow.getValue("name").equals(actualRow.get("name")) + && expectedRow + .getValue("number") + .equals(Long.parseLong(actualRow.get("number").toString())); + } + @Test public void testSimpleWrite() throws Exception { String tableSpec = "project:dataset.simple_write"; @@ -179,6 +184,43 @@ public void testSimpleWrite() throws Exception { rowsEquals(ROWS, fakeDatasetService.getAllRows("project", "dataset", "simple_write"))); } + @Test + public void testWriteToDynamicDestinations() throws Exception { + String dynamic = BigQueryStorageWriteApiSchemaTransformProvider.DYNAMIC_DESTINATIONS; + BigQueryStorageWriteApiSchemaTransformConfiguration config = + BigQueryStorageWriteApiSchemaTransformConfiguration.builder().setTable(dynamic).build(); + + String baseTableSpec = "project:dataset.dynamic_write_"; + + Schema schemaWithDestinations = + Schema.builder().addStringField("destination").addRowField("record", SCHEMA).build(); + List rowsWithDestinations = + ROWS.stream() + .map( + row -> + Row.withSchema(schemaWithDestinations) + .withFieldValue("destination", baseTableSpec + row.getInt64("number")) + .withFieldValue("record", row) + .build()) + .collect(Collectors.toList()); + + runWithConfig(config, rowsWithDestinations); + p.run().waitUntilFinish(); + + assertTrue( + rowEquals( + ROWS.get(0), + fakeDatasetService.getAllRows("project", "dataset", "dynamic_write_1").get(0))); + assertTrue( + rowEquals( + ROWS.get(1), + fakeDatasetService.getAllRows("project", "dataset", "dynamic_write_2").get(0))); + assertTrue( + rowEquals( + ROWS.get(2), + fakeDatasetService.getAllRows("project", "dataset", "dynamic_write_3").get(0))); + } + @Test public void testInputElementCount() throws Exception { String tableSpec = "project:dataset.input_count"; 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 5917ca4dc729..c1e9754526e8 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 @@ -28,6 +28,7 @@ import pytest from hamcrest.core import assert_that as hamcrest_assert +from hamcrest.core.core.allof import all_of import apache_beam as beam from apache_beam.io.gcp.bigquery import StorageWriteToBigQuery @@ -52,9 +53,6 @@ @pytest.mark.uses_gcp_java_expansion_service -# @unittest.skipUnless( -# os.environ.get('EXPANSION_PORT'), -# "EXPANSION_PORT environment var is not provided.") class BigQueryXlangStorageWriteIT(unittest.TestCase): BIGQUERY_DATASET = 'python_xlang_storage_write' @@ -114,7 +112,8 @@ def setUp(self): _LOGGER.info( "Created dataset %s in project %s", self.dataset_id, self.project) - _LOGGER.info("expansion port: %s", os.environ.get('EXPANSION_PORT')) + self.assertTrue( + os.environ.get('EXPANSION_PORT'), "Expansion service port not found!") self.expansion_service = ('localhost:%s' % os.environ.get('EXPANSION_PORT')) def tearDown(self): @@ -132,6 +131,8 @@ def tearDown(self): self.project) def parse_expected_data(self, expected_elements): + if not isinstance(expected_elements, list): + expected_elements = [expected_elements] data = [] for row in expected_elements: values = list(row.values()) @@ -246,6 +247,66 @@ def test_write_with_beam_rows(self): table=table_id, expansion_service=self.expansion_service)) 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) + tables = [base_table_spec + str(record['int']) for record in self.ELEMENTS] + + bq_matchers = [ + BigqueryFullResultMatcher( + project=self.project, + query="SELECT * FROM %s" % tables[i], + data=self.parse_expected_data(self.ELEMENTS[i])) + for i in range(len(tables)) + ] + + with beam.Pipeline(argv=self.args) as p: + _ = ( + p + | beam.Create(self.ELEMENTS) + | beam.io.WriteToBigQuery( + table=lambda record: spec_with_project + str(record['int']), + method=beam.io.WriteToBigQuery.Method.STORAGE_WRITE_API, + schema=self.ALL_TYPES_SCHEMA, + use_at_least_once=False, + expansion_service=self.expansion_service)) + hamcrest_assert(p, all_of(*bq_matchers)) + + def test_write_to_dynamic_destinations_with_beam_rows(self): + base_table_spec = '{}.dynamic_dest_'.format(self.dataset_id) + spec_with_project = '{}:{}'.format(self.project, base_table_spec) + tables = [base_table_spec + str(record['int']) for record in self.ELEMENTS] + + bq_matchers = [ + BigqueryFullResultMatcher( + project=self.project, + query="SELECT * FROM %s" % tables[i], + data=self.parse_expected_data(self.ELEMENTS[i])) + for i in range(len(tables)) + ] + + row_elements = [ + beam.Row( + my_int=e['int'], + my_float=e['float'], + my_numeric=e['numeric'], + my_string=e['str'], + my_bool=e['bool'], + my_bytes=e['bytes'], + my_timestamp=e['timestamp']) for e in self.ELEMENTS + ] + + with beam.Pipeline(argv=self.args) as p: + _ = ( + p + | beam.Create(row_elements) + | beam.io.WriteToBigQuery( + table=lambda record: spec_with_project + str(record.my_int), + method=beam.io.WriteToBigQuery.Method.STORAGE_WRITE_API, + use_at_least_once=False, + expansion_service=self.expansion_service)) + hamcrest_assert(p, all_of(*bq_matchers)) + def run_streaming(self, table_name, num_streams=0, use_at_least_once=False): elements = self.ELEMENTS.copy() schema = self.ALL_TYPES_SCHEMA @@ -278,13 +339,16 @@ def run_streaming(self, table_name, num_streams=0, use_at_least_once=False): expansion_service=self.expansion_service)) hamcrest_assert(p, bq_matcher) - def test_streaming_with_fixed_num_streams(self): + def skip_if_not_dataflow_runner(self) -> bool: # skip if dataflow runner is not specified if not self._runner or "dataflowrunner" not in self._runner.lower(): self.skipTest( - "The exactly-once route has the requirement " + "Streaming with exactly-once route has the requirement " "`beam:requirement:pardo:on_window_expiration:v1`, " "which is currently only supported by the Dataflow runner") + + def test_streaming_with_fixed_num_streams(self): + self.skip_if_not_dataflow_runner() table = 'streaming_fixed_num_streams' self.run_streaming(table_name=table, num_streams=4) @@ -292,6 +356,7 @@ def test_streaming_with_fixed_num_streams(self): "Streaming to the Storage Write API sink with autosharding is broken " "with Dataflow Runner V2.") def test_streaming_with_auto_sharding(self): + self.skip_if_not_dataflow_runner() table = 'streaming_with_auto_sharding' self.run_streaming(table_name=table) diff --git a/sdks/python/apache_beam/io/gcp/bigquery.py b/sdks/python/apache_beam/io/gcp/bigquery.py index ac06425e95a9..4643c8ddf0a5 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery.py +++ b/sdks/python/apache_beam/io/gcp/bigquery.py @@ -2222,73 +2222,17 @@ def find_in_nested_dict(schema): BigQueryBatchFileLoads.DESTINATION_COPY_JOBID_PAIRS]) elif method_to_use == WriteToBigQuery.Method.STORAGE_WRITE_API: - if self.schema is None: - try: - schema = schema_from_element_type(pcoll.element_type) - is_rows = True - except TypeError as exn: - raise ValueError( - "A schema is required in order to prepare rows" - "for writing with STORAGE_WRITE_API.") from exn - elif callable(self.schema): - raise NotImplementedError( - "Writing to dynamic destinations is not" - "supported for this write method.") - elif isinstance(self.schema, vp.ValueProvider): - schema = self.schema.get() - is_rows = False - else: - schema = self.schema - is_rows = False - - table = bigquery_tools.get_hashable_destination(self.table_reference) - # None type is not supported - triggering_frequency = self.triggering_frequency or 0 - # SchemaTransform expects Beam Rows, so map to Rows first - if is_rows: - input_beam_rows = pcoll - else: - input_beam_rows = ( - pcoll - | "Convert dict to Beam Row" >> beam.Map( - lambda row: bigquery_tools.beam_row_from_dict(row, schema) - ).with_output_types( - RowTypeConstraint.from_fields( - bigquery_tools.get_beam_typehints_from_tableschema(schema))) - ) - output_beam_rows = ( - input_beam_rows - | StorageWriteToBigQuery( - table=table, - create_disposition=self.create_disposition, - write_disposition=self.write_disposition, - triggering_frequency=triggering_frequency, - 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, - expansion_service=self.expansion_service)) - - if is_rows: - failed_rows = output_beam_rows[StorageWriteToBigQuery.FAILED_ROWS] - failed_rows_with_errors = output_beam_rows[ - StorageWriteToBigQuery.FAILED_ROWS_WITH_ERRORS] - else: - # return back from Beam Rows to Python dict elements - failed_rows = ( - output_beam_rows[StorageWriteToBigQuery.FAILED_ROWS] - | beam.Map(lambda row: row.as_dict())) - failed_rows_with_errors = ( - output_beam_rows[StorageWriteToBigQuery.FAILED_ROWS_WITH_ERRORS] - | beam.Map( - lambda row: { - "error_message": row.error_message, - "failed_row": row.failed_row.as_dict() - })) - - return WriteResult( - method=WriteToBigQuery.Method.STORAGE_WRITE_API, - failed_rows=failed_rows, - failed_rows_with_errors=failed_rows_with_errors) + return pcoll | StorageWriteToBigQuery( + table=self.table_reference, + schema=self.schema, + table_side_inputs=self.table_side_inputs, + create_disposition=self.create_disposition, + write_disposition=self.write_disposition, + triggering_frequency=self.triggering_frequency, + 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, + expansion_service=self.expansion_service) else: raise ValueError(f"Unsupported method {method_to_use}") @@ -2382,7 +2326,7 @@ class WriteResult: """ def __init__( self, - method: WriteToBigQuery.Method = None, + method: str = None, destination_load_jobid_pairs: PCollection[Tuple[str, JobReference]] = None, destination_file_pairs: PCollection[Tuple[str, Tuple[str, int]]] = None, @@ -2505,24 +2449,26 @@ def __getitem__(self, key): return self.attributes[key].__get__(self, WriteResult) -def _default_io_expansion_service(append_args=None): - return BeamJarExpansionService( - 'sdks:java:io:google-cloud-platform:expansion-service:build', - append_args=append_args) - - class StorageWriteToBigQuery(PTransform): """Writes data to BigQuery using Storage API. + Supports dynamic destinations. Dynamic schemas are not supported yet. Experimental; no backwards compatibility guarantees. """ - URN = "beam:schematransform:org.apache.beam:bigquery_storage_write:v2" + IDENTIFIER = "beam:schematransform:org.apache.beam:bigquery_storage_write:v2" FAILED_ROWS = "FailedRows" FAILED_ROWS_WITH_ERRORS = "FailedRowsWithErrors" + # fields for rows sent to Storage API with dynamic destinations + DESTINATION = "destination" + RECORD = "record" + # magic string to tell Java that these rows are going to dynamic destinations + DYNAMIC_DESTINATIONS = "DYNAMIC_DESTINATIONS" def __init__( self, table, + table_side_inputs=None, + schema=None, create_disposition=BigQueryDisposition.CREATE_IF_NEEDED, write_disposition=BigQueryDisposition.WRITE_APPEND, triggering_frequency=0, @@ -2530,71 +2476,156 @@ def __init__( with_auto_sharding=False, num_storage_api_streams=0, expansion_service=None): - """Initialize a StorageWriteToBigQuery transform. - - :param table: - Fully-qualified table ID specified as ``'PROJECT:DATASET.TABLE'``. - :param create_disposition: - String specifying the strategy to take when the table doesn't - exist. Possible values are: - * ``'CREATE_IF_NEEDED'``: create if does not exist. - * ``'CREATE_NEVER'``: fail the write if does not exist. - :param write_disposition: - String specifying the strategy to take when the table already - contains data. Possible values are: - * ``'WRITE_TRUNCATE'``: delete existing rows. - * ``'WRITE_APPEND'``: add to existing rows. - * ``'WRITE_EMPTY'``: fail the write if table not empty. - :param triggering_frequency: - The time in seconds between write commits. Should only be specified - for streaming pipelines. Defaults to 5 seconds. - :param use_at_least_once: - Use at-least-once semantics. Is cheaper and provides lower latency, - but will potentially duplicate records. - :param with_auto_sharding: - Experimental. If true, enables using a dynamically determined number of - shards to write to BigQuery. Only applicable to unbounded input. - :param expansion_service: - The address (host:port) of the expansion service. If no expansion - service is provided, will attempt to run the default GCP expansion - service. - """ - super().__init__() self._table = table + self._table_side_inputs = table_side_inputs + self._schema = schema self._create_disposition = create_disposition self._write_disposition = write_disposition self._triggering_frequency = triggering_frequency 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._expansion_service = ( - expansion_service or _default_io_expansion_service()) - self.schematransform_config = SchemaAwareExternalTransform.discover_config( - self._expansion_service, self.URN) + self._expansion_service = expansion_service or BeamJarExpansionService( + 'sdks:java:io:google-cloud-platform:expansion-service:build') def expand(self, input): - external_storage_write = SchemaAwareExternalTransform( - identifier=self.schematransform_config.identifier, - expansion_service=self._expansion_service, - rearrange_based_on_discovery=True, - autoSharding=self._with_auto_sharding, - numStreams=self._num_storage_api_streams, - createDisposition=self._create_disposition, - table=self._table, - triggeringFrequencySeconds=self._triggering_frequency, - useAtLeastOnceSemantics=self._use_at_least_once, - writeDisposition=self._write_disposition, - errorHandling={ - 'output': StorageWriteToBigQuery.FAILED_ROWS_WITH_ERRORS - }) - - input_tag = self.schematransform_config.inputs[0] - - result = {input_tag: input} | external_storage_write - result[StorageWriteToBigQuery.FAILED_ROWS] = result[ - StorageWriteToBigQuery.FAILED_ROWS_WITH_ERRORS] | beam.Map( - lambda row_and_error: row_and_error[0]) - return result + if self._schema is None: + try: + schema = schema_from_element_type(input.element_type) + is_rows = True + except TypeError as exn: + raise ValueError( + "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" + "supported for this write method.") + elif isinstance(self._schema, vp.ValueProvider): + schema = self._schema.get() + is_rows = False + else: + schema = self._schema + is_rows = False + + table = bigquery_tools.get_hashable_destination(self._table) + + # if writing to one destination, just convert to Beam rows and send over + if not callable(table): + if is_rows: + input_beam_rows = input + else: + input_beam_rows = ( + input + | "Convert dict to Beam Row" >> self.ConvertToBeamRows( + schema, False).with_output_types()) + + # For dynamic destinations, we first figure out where each row is going. + # Then we send (destination, record) rows over to Java SchemaTransform. + # We need to do this here because there are obstacles to passing the + # destinations function to Java + else: + # call function and append destination to each row + input_rows = ( + input + | "Append dynamic destinations" >> beam.ParDo( + bigquery_tools.AppendDestinationsFn(table), + *self._table_side_inputs)) + # if input type is Beam Row, just wrap everything in another Row + if is_rows: + input_beam_rows = ( + input_rows + | "Wrap in Beam Row" >> beam.Map( + lambda row: beam.Row( + **{ + StorageWriteToBigQuery.DESTINATION: row[0], + StorageWriteToBigQuery.RECORD: row[1] + })).with_output_types( + RowTypeConstraint.from_fields([ + (StorageWriteToBigQuery.DESTINATION, str), + (StorageWriteToBigQuery.RECORD, input.element_type) + ]))) + # otherwise, convert to Beam Rows + else: + input_beam_rows = ( + input_rows + | "Convert dict to Beam Row" >> self.ConvertToBeamRows( + schema, True).with_output_types()) + # communicate to Java that this write should use dynamic destinations + table = StorageWriteToBigQuery.DYNAMIC_DESTINATIONS + + output = ( + input_beam_rows + | SchemaAwareExternalTransform( + identifier=StorageWriteToBigQuery.IDENTIFIER, + expansion_service=self._expansion_service, + rearrange_based_on_discovery=True, + table=table, + createDisposition=self._create_disposition, + writeDisposition=self._write_disposition, + triggeringFrequencySeconds=self._triggering_frequency, + autoSharding=self._with_auto_sharding, + numStreams=self._num_storage_api_streams, + useAtLeastOnceSemantics=self._use_at_least_once, + errorHandling={ + 'output': StorageWriteToBigQuery.FAILED_ROWS_WITH_ERRORS + })) + + failed_rows_with_errors = output[ + StorageWriteToBigQuery.FAILED_ROWS_WITH_ERRORS] + failed_rows = failed_rows_with_errors | beam.Map( + lambda row_and_error: row_and_error[0]) + if not is_rows: + # return back from Beam Rows to Python dict elements + failed_rows = failed_rows | beam.Map(lambda row: row.as_dict()) + failed_rows_with_errors = failed_rows_with_errors | beam.Map( + lambda row: { + "error_message": row.error_message, + "failed_row": row.failed_row.as_dict() + }) + + return WriteResult( + method=WriteToBigQuery.Method.STORAGE_WRITE_API, + failed_rows=failed_rows, + failed_rows_with_errors=failed_rows_with_errors) + + class ConvertToBeamRows(PTransform): + def __init__(self, schema, dynamic_destinations): + self.schema = schema + self.dynamic_destinations = dynamic_destinations + + def expand(self, input_dicts): + if self.dynamic_destinations: + return ( + input_dicts + | "Convert dict to Beam Row" >> beam.Map( + lambda row: beam.Row( + **{ + StorageWriteToBigQuery.DESTINATION: row[0], + StorageWriteToBigQuery.RECORD: bigquery_tools. + beam_row_from_dict(row[1], self.schema) + }))) + else: + return ( + input_dicts + | "Convert dict to Beam Row" >> beam.Map( + lambda row: bigquery_tools.beam_row_from_dict(row, self.schema)) + ) + + def with_output_types(self): + row_type_hints = bigquery_tools.get_beam_typehints_from_tableschema( + self.schema) + if self.dynamic_destinations: + type_hint = RowTypeConstraint.from_fields([ + (StorageWriteToBigQuery.DESTINATION, str), + ( + StorageWriteToBigQuery.RECORD, + RowTypeConstraint.from_fields(row_type_hints)) + ]) + else: + type_hint = RowTypeConstraint.from_fields(row_type_hints) + + return super().with_output_types(type_hint) class ReadFromBigQuery(PTransform): From 674fe77df47cadca16196af0d9d759592d9ef04f Mon Sep 17 00:00:00 2001 From: Ritesh Ghorse Date: Wed, 24 Jan 2024 15:11:05 -0500 Subject: [PATCH 066/169] [Python] Add an option to retain timestamp returned from the BigTable row (#30088) * retain timestamp returned from bigtable row * make it optional to retain timestamp * fix import order, remove trigger file * ignore arg type in isinstance --- .../enrichment_handlers/bigtable.py | 24 +++++-- .../enrichment_handlers/bigtable_it_test.py | 63 +++++++++++++++---- 2 files changed, 70 insertions(+), 17 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py index 86ff2f3b8e7f..873dd156cb87 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py @@ -30,7 +30,7 @@ from apache_beam.transforms.enrichment import EnrichmentSourceHandler __all__ = [ - 'EnrichWithBigTable', + 'BigTableEnrichmentHandler', 'ExceptionLevel', ] @@ -52,8 +52,8 @@ class ExceptionLevel(Enum): QUIET = 2 -class EnrichWithBigTable(EnrichmentSourceHandler[beam.Row, beam.Row]): - """EnrichWithBigTable is a handler for +class BigTableEnrichmentHandler(EnrichmentSourceHandler[beam.Row, beam.Row]): + """BigTableEnrichmentHandler is a handler for :class:`apache_beam.transforms.enrichment.Enrichment` transform to interact with GCP BigTable. @@ -74,6 +74,10 @@ class EnrichWithBigTable(EnrichmentSourceHandler[beam.Row, beam.Row]): ``apache_beam.transforms.enrichment_handlers.bigtable.ExceptionLevel`` to set the level when an empty row is returned from the BigTable query. Defaults to ``ExceptionLevel.WARN``. + include_timestamp (bool): If enabled, the timestamp associated with the + value is returned as `(value, timestamp)` for each `row_key`. + Defaults to `False` - only the latest value without + the timestamp is returned. """ def __init__( self, @@ -82,9 +86,11 @@ def __init__( table_id: str, row_key: str, row_filter: Optional[RowFilter] = CellsColumnLimitFilter(1), + *, app_profile_id: str = None, # type: ignore[assignment] encoding: str = 'utf-8', exception_level: ExceptionLevel = ExceptionLevel.WARN, + include_timestamp: bool = False, ): self._project_id = project_id self._instance_id = instance_id @@ -94,6 +100,7 @@ def __init__( self._app_profile_id = app_profile_id self._encoding = encoding self._exception_level = exception_level + self._include_timestamp = include_timestamp def __enter__(self): """connect to the Google BigTable cluster.""" @@ -122,9 +129,14 @@ def __call__(self, request: beam.Row, *args, **kwargs): if row: for cf_id, cf_v in row.cells.items(): response_dict[cf_id] = {} - for k, v in cf_v.items(): - response_dict[cf_id][k.decode(self._encoding)] = \ - v[0].value.decode(self._encoding) + for col_id, col_v in cf_v.items(): + if self._include_timestamp: + response_dict[cf_id][col_id.decode(self._encoding)] = [ + (v.value.decode(self._encoding), v.timestamp) for v in col_v + ] + else: + response_dict[cf_id][col_id.decode( + self._encoding)] = col_v[0].value.decode(self._encoding) elif self._exception_level == ExceptionLevel.WARN: _LOGGER.warning( 'no matching row found for row_key: %s ' diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py index dd48c8e5ef4d..86fc438960d3 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py @@ -20,6 +20,7 @@ from typing import Dict from typing import List from typing import NamedTuple +from typing import Tuple import pytest @@ -33,7 +34,7 @@ from google.cloud.bigtable import Client from google.cloud.bigtable.row_filters import ColumnRangeFilter from apache_beam.transforms.enrichment import Enrichment - from apache_beam.transforms.enrichment_handlers.bigtable import EnrichWithBigTable + from apache_beam.transforms.enrichment_handlers.bigtable import BigTableEnrichmentHandler from apache_beam.transforms.enrichment_handlers.bigtable import ExceptionLevel except ImportError: raise unittest.SkipTest('GCP BigTable dependencies are not installed.') @@ -46,10 +47,13 @@ def __init__( self, n_fields: int, fields: List[str], - enriched_fields: Dict[str, List[str]]): + enriched_fields: Dict[str, List[str]], + include_timestamp: bool = False, + ): self.n_fields = n_fields self._fields = fields self._enriched_fields = enriched_fields + self._include_timestamp = include_timestamp def process(self, element: beam.Row, *args, **kwargs): element_dict = element.as_dict() @@ -62,12 +66,22 @@ def process(self, element: beam.Row, *args, **kwargs): raise BeamAssertException(f"Expected a not None field: {field}") for column_family, columns in self._enriched_fields.items(): - if (len(element_dict[column_family]) != len(columns) or - not all(key in element_dict[column_family] for key in columns)): + if len(element_dict[column_family]) != len(columns): raise BeamAssertException( "Response from bigtable should contain a %s column_family with " "%s keys." % (column_family, columns)) + for key in columns: + if key not in element_dict[column_family]: + raise BeamAssertException( + "Response from bigtable should contain a %s column_family with " + "%s columns." % (column_family, columns)) + if (self._include_timestamp and + not isinstance(element_dict[column_family][key][0], Tuple)): # type: ignore[arg-type] + raise BeamAssertException( + "Response from bigtable should contain timestamp associated with " + "its value.") + class _Currency(NamedTuple): s_id: int @@ -157,7 +171,7 @@ def test_enrichment_with_bigtable(self): expected_enriched_fields = { 'product': ['product_id', 'product_name', 'product_stock'], } - bigtable = EnrichWithBigTable( + bigtable = BigTableEnrichmentHandler( project_id=self.project_id, instance_id=self.instance_id, table_id=self.table_id, @@ -182,7 +196,7 @@ def test_enrichment_with_bigtable_row_filter(self): } start_column = 'product_name'.encode() column_filter = ColumnRangeFilter(self.column_family_id, start_column) - bigtable = EnrichWithBigTable( + bigtable = BigTableEnrichmentHandler( project_id=self.project_id, instance_id=self.instance_id, table_id=self.table_id, @@ -204,7 +218,7 @@ def test_enrichment_with_bigtable_no_enrichment(self): # won't be added. Hence, the response is same as the request. expected_fields = ['sale_id', 'customer_id', 'product_id', 'quantity'] expected_enriched_fields = {} - bigtable = EnrichWithBigTable( + bigtable = BigTableEnrichmentHandler( project_id=self.project_id, instance_id=self.instance_id, table_id=self.table_id, @@ -227,7 +241,7 @@ def test_enrichment_with_bigtable_bad_row_filter(self): # column names then all columns in that column_family are returned. start_column = 'car_name'.encode() column_filter = ColumnRangeFilter('car_name', start_column) - bigtable = EnrichWithBigTable( + bigtable = BigTableEnrichmentHandler( project_id=self.project_id, instance_id=self.instance_id, table_id=self.table_id, @@ -245,7 +259,7 @@ def test_enrichment_with_bigtable_bad_row_filter(self): def test_enrichment_with_bigtable_raises_key_error(self): """raises a `KeyError` when the row_key doesn't exist in the input PCollection.""" - bigtable = EnrichWithBigTable( + bigtable = BigTableEnrichmentHandler( project_id=self.project_id, instance_id=self.instance_id, table_id=self.table_id, @@ -262,7 +276,7 @@ def test_enrichment_with_bigtable_raises_key_error(self): def test_enrichment_with_bigtable_raises_not_found(self): """raises a `NotFound` exception when the GCP BigTable Cluster doesn't exist.""" - bigtable = EnrichWithBigTable( + bigtable = BigTableEnrichmentHandler( project_id=self.project_id, instance_id=self.instance_id, table_id='invalid_table', @@ -279,7 +293,7 @@ def test_enrichment_with_bigtable_raises_not_found(self): def test_enrichment_with_bigtable_exception_level(self): """raises a `ValueError` exception when the GCP BigTable query returns an empty row.""" - bigtable = EnrichWithBigTable( + bigtable = BigTableEnrichmentHandler( project_id=self.project_id, instance_id=self.instance_id, table_id=self.table_id, @@ -295,6 +309,33 @@ def test_enrichment_with_bigtable_exception_level(self): res = test_pipeline.run() res.wait_until_finish() + def test_enrichment_with_bigtable_with_timestamp(self): + """test whether the `(value,timestamp)` is returned when the + `include_timestamp` is enabled.""" + expected_fields = [ + 'sale_id', 'customer_id', 'product_id', 'quantity', 'product' + ] + expected_enriched_fields = { + 'product': ['product_id', 'product_name', 'product_stock'], + } + bigtable = BigTableEnrichmentHandler( + project_id=self.project_id, + instance_id=self.instance_id, + table_id=self.table_id, + row_key=self.row_key, + include_timestamp=True) + with TestPipeline(is_integration_test=True) as test_pipeline: + _ = ( + test_pipeline + | "Create" >> beam.Create(self.req) + | "Enrich W/ BigTable" >> Enrichment(bigtable) + | "Validate Response" >> beam.ParDo( + ValidateResponse( + len(expected_fields), + expected_fields, + expected_enriched_fields, + include_timestamp=True))) + if __name__ == '__main__': unittest.main() From e3f452496202b989a2f0203dc12eb029e73984e2 Mon Sep 17 00:00:00 2001 From: lostluck Date: Wed, 24 Jan 2024 22:36:31 +0000 Subject: [PATCH 067/169] Moving to 2.55.0-SNAPSHOT on master branch. --- .asf.yaml | 1 + gradle.properties | 4 ++-- sdks/go/pkg/beam/core/core.go | 2 +- sdks/python/apache_beam/version.py | 2 +- sdks/typescript/package.json | 2 +- 5 files changed, 6 insertions(+), 5 deletions(-) diff --git a/.asf.yaml b/.asf.yaml index d126c562838f..63e05f72052c 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -49,6 +49,7 @@ github: protected_branches: master: {} + release-2.54.0: {} release-2.53.0: {} release-2.52.0: {} release-2.51.0: {} diff --git a/gradle.properties b/gradle.properties index 8f8736a100f6..b76e071981af 100644 --- a/gradle.properties +++ b/gradle.properties @@ -30,8 +30,8 @@ signing.gnupg.useLegacyGpg=true # buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy. # To build a custom Beam version make sure you change it in both places, see # https://github.com/apache/beam/issues/21302. -version=2.54.0-SNAPSHOT -sdk_version=2.54.0.dev +version=2.55.0-SNAPSHOT +sdk_version=2.55.0.dev javaVersion=1.8 diff --git a/sdks/go/pkg/beam/core/core.go b/sdks/go/pkg/beam/core/core.go index 862b6eee94ba..176c98b0f07e 100644 --- a/sdks/go/pkg/beam/core/core.go +++ b/sdks/go/pkg/beam/core/core.go @@ -27,7 +27,7 @@ const ( // SdkName is the human readable name of the SDK for UserAgents. SdkName = "Apache Beam SDK for Go" // SdkVersion is the current version of the SDK. - SdkVersion = "2.54.0.dev" + SdkVersion = "2.55.0.dev" // DefaultDockerImage represents the associated image for this release. DefaultDockerImage = "apache/beam_go_sdk:" + SdkVersion diff --git a/sdks/python/apache_beam/version.py b/sdks/python/apache_beam/version.py index f0df92e31e57..73b908e0ef77 100644 --- a/sdks/python/apache_beam/version.py +++ b/sdks/python/apache_beam/version.py @@ -17,4 +17,4 @@ """Apache Beam SDK version information and utilities.""" -__version__ = '2.54.0.dev' +__version__ = '2.55.0.dev' diff --git a/sdks/typescript/package.json b/sdks/typescript/package.json index 8fc999bc9c9d..023d3d10131b 100644 --- a/sdks/typescript/package.json +++ b/sdks/typescript/package.json @@ -1,6 +1,6 @@ { "name": "apache-beam", - "version": "2.54.0-SNAPSHOT", + "version": "2.55.0-SNAPSHOT", "devDependencies": { "@google-cloud/bigquery": "^5.12.0", "@types/mocha": "^9.0.0", From d759f97013931d6e4694e7b5341d5b9d043a04f9 Mon Sep 17 00:00:00 2001 From: JayajP Date: Wed, 24 Jan 2024 14:38:45 -0800 Subject: [PATCH 068/169] Add StreamingStepMetricsContainer::extractPerWorkerMetrics method. (#30089) * Add support for StreamingStepMetricsContainer to extract perworkermetrics updates * Address comments on testing patterns --- .../worker/StreamingStepMetricsContainer.java | 128 +++++++++++++- .../StreamingStepMetricsContainerTest.java | 159 ++++++++++++++++++ 2 files changed, 284 insertions(+), 3 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainer.java index df1599f47189..af3b3e51071f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainer.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainer.java @@ -18,9 +18,15 @@ package org.apache.beam.runners.dataflow.worker; import com.google.api.services.dataflow.model.CounterUpdate; +import com.google.api.services.dataflow.model.PerStepNamespaceMetrics; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.time.Clock; +import java.time.Duration; +import java.time.Instant; +import java.util.HashSet; import java.util.Map; import java.util.Map.Entry; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; import javax.annotation.Nonnull; @@ -37,6 +43,7 @@ import org.apache.beam.sdk.metrics.MetricsContainer; import org.apache.beam.sdk.util.HistogramData; import org.apache.beam.sdk.values.KV; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Function; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Predicates; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.FluentIterable; @@ -50,15 +57,14 @@ "nullness" // TODO(https://github.com/apache/beam/issues/20497) }) public class StreamingStepMetricsContainer implements MetricsContainer { - private final String stepName; - private static Boolean enablePerWorkerMetrics; + private static boolean enablePerWorkerMetrics = false; private MetricsMap counters = new MetricsMap<>(DeltaCounterCell::new); - private ConcurrentHashMap perWorkerCounters = new ConcurrentHashMap<>(); + private final ConcurrentHashMap perWorkerCounters; private MetricsMap gauges = new MetricsMap<>(GaugeCell::new); @@ -68,8 +74,19 @@ public class StreamingStepMetricsContainer implements MetricsContainer { private MetricsMap, HistogramCell> perWorkerHistograms = new MetricsMap<>(HistogramCell::new); + private final Map perWorkerCountersByFirstStaleTime; + + // PerWorkerCounters that have been longer than this value will be removed from the underlying + // metrics map. + private final Duration maximumPerWorkerCounterStaleness = Duration.ofMinutes(5); + + private final Clock clock; + private StreamingStepMetricsContainer(String stepName) { this.stepName = stepName; + this.perWorkerCountersByFirstStaleTime = new ConcurrentHashMap<>(); + this.clock = Clock.systemUTC(); + perWorkerCounters = new ConcurrentHashMap<>(); } public static MetricsContainerRegistry createRegistry() { @@ -81,6 +98,28 @@ protected StreamingStepMetricsContainer createContainer(String stepName) { }; } + // Testing constructor. + private StreamingStepMetricsContainer( + String stepName, + Map perWorkerCountersByFirstStaleTime, + ConcurrentHashMap perWorkerCounters, + Clock clock) { + this.stepName = stepName; + this.perWorkerCountersByFirstStaleTime = perWorkerCountersByFirstStaleTime; + this.perWorkerCounters = perWorkerCounters; + this.clock = clock; + } + + @VisibleForTesting + static StreamingStepMetricsContainer forTesting( + String stepName, + Map perWorkerCountersByFirstStaleTime, + ConcurrentHashMap perWorkerCounters, + Clock clock) { + return new StreamingStepMetricsContainer( + stepName, perWorkerCountersByFirstStaleTime, perWorkerCounters, clock); + } + @Override public Counter getCounter(MetricName metricName) { return counters.get(metricName); @@ -178,4 +217,87 @@ public static Iterable extractMetricUpdates( public static void setEnablePerWorkerMetrics(Boolean enablePerWorkerMetrics) { StreamingStepMetricsContainer.enablePerWorkerMetrics = enablePerWorkerMetrics; } + + /** + * Updates {@code perWorkerCountersByFirstStaleTime} with the current zero-valued metrics and + * removes metrics that have been stale for longer than {@code maximumPerWorkerCounterStaleness} + * from {@code perWorkerCounters}. + * + * @param currentZeroValuedCounters Current zero-valued perworker counters. + * @param extractionTime Time {@code currentZeroValuedCounters} were discovered to be zero-valued. + */ + private void deleteStaleCounters( + Set currentZeroValuedCounters, Instant extractionTime) { + // perWorkerCountersByFirstStaleTime should only contain metrics that are currently zero-valued. + perWorkerCountersByFirstStaleTime.keySet().retainAll(currentZeroValuedCounters); + + // Delete metrics that have been longer than 'maximumPerWorkerCounterStaleness'. + Set deletedMetricNames = new HashSet(); + for (Entry entry : perWorkerCountersByFirstStaleTime.entrySet()) { + if (Duration.between(entry.getValue(), extractionTime) + .compareTo(maximumPerWorkerCounterStaleness) + > 0) { + RemoveSafeDeltaCounterCell cell = + new RemoveSafeDeltaCounterCell(entry.getKey(), perWorkerCounters); + cell.deleteIfZero(); + deletedMetricNames.add(entry.getKey()); + } + } + + // Insert new zero-valued metrics into `perWorkerCountersByFirstStaleTime`. + currentZeroValuedCounters.forEach( + name -> perWorkerCountersByFirstStaleTime.putIfAbsent(name, extractionTime)); + + // Metrics in 'deletedMetricNames' have either been removed from 'perWorkerCounters' or are no + // longer zero-valued. + perWorkerCountersByFirstStaleTime.keySet().removeAll(deletedMetricNames); + } + + /** + * Extracts metric updates for all PerWorker metrics that have changed in this Container since the + * last time this function was called. Additionally, deletes any PerWorker counters that have been + * zero valued for more than {@code maximumPerWorkerCounterStaleness}. + */ + @VisibleForTesting + Iterable extractPerWorkerMetricUpdates() { + ConcurrentHashMap counters = new ConcurrentHashMap(); + ConcurrentHashMap histograms = + new ConcurrentHashMap(); + HashSet currentZeroValuedCounters = new HashSet(); + + // Extract metrics updates. + perWorkerCounters.forEach( + (k, v) -> { + Long val = v.getAndSet(0); + if (val == 0) { + currentZeroValuedCounters.add(k); + return; + } + counters.put(k, val); + }); + perWorkerHistograms.forEach( + (k, v) -> { + HistogramData val = v.getCumulative().getAndReset(); + if (val.getTotalCount() == 0) { + return; + } + histograms.put(k.getKey(), val); + }); + + deleteStaleCounters(currentZeroValuedCounters, Instant.now(clock)); + + return MetricsToPerStepNamespaceMetricsConverter.convert(stepName, counters, histograms); + } + + /** + * @param metricsContainerRegistry Metrics will be extracted for all containers in this registry. + * @return An iterable of {@link PerStepNamespaceMetrics} representing the changes to all + * PerWorkerMetrics that have changed since the last time this function was invoked. + */ + public static Iterable extractPerWorkerMetricUpdates( + MetricsContainerRegistry metricsContainerRegistry) { + return metricsContainerRegistry + .getContainers() + .transformAndConcat(StreamingStepMetricsContainer::extractPerWorkerMetricUpdates); + } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainerTest.java index 9be65c198ac7..cc0722e7a8e6 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainerTest.java @@ -19,16 +19,35 @@ import static org.apache.beam.runners.dataflow.worker.counters.DataflowCounterUpdateExtractor.longToSplitInt; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.sameInstance; +import com.google.api.services.dataflow.model.BucketOptions; import com.google.api.services.dataflow.model.CounterMetadata; import com.google.api.services.dataflow.model.CounterStructuredName; import com.google.api.services.dataflow.model.CounterStructuredNameAndMetadata; import com.google.api.services.dataflow.model.CounterUpdate; +import com.google.api.services.dataflow.model.DataflowHistogramValue; import com.google.api.services.dataflow.model.DistributionUpdate; +import com.google.api.services.dataflow.model.Linear; +import com.google.api.services.dataflow.model.MetricValue; +import com.google.api.services.dataflow.model.OutlierStats; +import com.google.api.services.dataflow.model.PerStepNamespaceMetrics; +import java.time.Clock; +import java.time.Duration; +import java.time.Instant; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; import org.apache.beam.runners.dataflow.worker.MetricsToCounterUpdateConverter.Kind; import org.apache.beam.runners.dataflow.worker.MetricsToCounterUpdateConverter.Origin; import org.apache.beam.sdk.metrics.Distribution; @@ -37,6 +56,7 @@ import org.apache.beam.sdk.metrics.NoOpCounter; import org.apache.beam.sdk.metrics.NoOpHistogram; import org.apache.beam.sdk.util.HistogramData; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.junit.Rule; import org.junit.Test; import org.junit.rules.Timeout; @@ -202,4 +222,143 @@ public void testPerWorkerMetrics() { metricsContainer.getPerWorkerHistogram(name1, testBucket), not(instanceOf(NoOpHistogram.class))); } + + @Test + public void testExtractPerWorkerMetricUpdates() { + StreamingStepMetricsContainer.setEnablePerWorkerMetrics(true); + MetricName counterMetricName = MetricName.named("BigQuerySink", "counter-"); + c1.getPerWorkerCounter(counterMetricName).inc(3); + + MetricName histogramMetricName = MetricName.named("BigQuerySink", "histogram-"); + HistogramData.LinearBuckets linearBuckets = HistogramData.LinearBuckets.of(0, 10, 10); + c2.getPerWorkerHistogram(histogramMetricName, linearBuckets).update(5.0); + + Iterable updates = + StreamingStepMetricsContainer.extractPerWorkerMetricUpdates(registry); + + // Expected counter metric. + MetricValue expectedCounter = + new MetricValue().setMetric("counter").setMetricLabels(new HashMap<>()).setValueInt64(3L); + + PerStepNamespaceMetrics counters = + new PerStepNamespaceMetrics() + .setOriginalStep("s1") + .setMetricsNamespace("BigQuerySink") + .setMetricValues(Collections.singletonList(expectedCounter)); + + // Expected histogram metric + List bucketCounts = Collections.singletonList(1L); + + Linear linearOptions = new Linear().setNumberOfBuckets(10).setWidth(10.0).setStart(0.0); + BucketOptions bucketOptions = new BucketOptions().setLinear(linearOptions); + + OutlierStats outlierStats = + new OutlierStats() + .setUnderflowCount(0L) + .setUnderflowMean(0.0) + .setOverflowCount(0L) + .setOverflowMean(0.0); + DataflowHistogramValue linearHistogram = + new DataflowHistogramValue() + .setCount(1L) + .setBucketOptions(bucketOptions) + .setBucketCounts(bucketCounts) + .setOutlierStats(outlierStats); + + MetricValue expectedHistogram = + new MetricValue() + .setMetric("histogram") + .setMetricLabels(new HashMap<>()) + .setValueHistogram(linearHistogram); + + PerStepNamespaceMetrics histograms = + new PerStepNamespaceMetrics() + .setOriginalStep("s2") + .setMetricsNamespace("BigQuerySink") + .setMetricValues(Collections.singletonList(expectedHistogram)); + + assertThat(updates, containsInAnyOrder(histograms, counters)); + } + + public class TestClock extends Clock { + private Instant currentTime; + + public void advance(Duration amount) { + currentTime = currentTime.plus(amount); + } + + TestClock(Instant startTime) { + currentTime = startTime; + } + + @Override + public Instant instant() { + return currentTime; + } + + @Override + public ZoneId getZone() { + return ZoneOffset.UTC; + } + + // Currently not supported. + @Override + public Clock withZone(ZoneId zone) { + return new TestClock(currentTime); + } + } + + @Test + public void testDeleteStaleCounters() { + TestClock clock = new TestClock(Instant.now()); + Map countersByFirstStaleTime = new HashMap<>(); + ConcurrentHashMap perWorkerCounters = new ConcurrentHashMap<>(); + + StreamingStepMetricsContainer metricsContainer = + StreamingStepMetricsContainer.forTesting( + "s1", countersByFirstStaleTime, perWorkerCounters, clock); + + MetricName counterMetricName1 = MetricName.named("BigQuerySink", "counter1-"); + MetricName counterMetricName2 = MetricName.named("BigQuerySink", "counter2-"); + metricsContainer.getPerWorkerCounter(counterMetricName1).inc(3); + metricsContainer.getPerWorkerCounter(counterMetricName2).inc(3); + + List updatesList = + Lists.newArrayList(metricsContainer.extractPerWorkerMetricUpdates()); + assertThat(updatesList.size(), equalTo(1)); + + assertThat(perWorkerCounters.get(counterMetricName1).get(), equalTo(0L)); + assertThat(countersByFirstStaleTime.size(), equalTo(0)); + + // At minute 1 both metrics are discovered to be zero-valued. + updatesList = Lists.newArrayList(metricsContainer.extractPerWorkerMetricUpdates()); + assertThat(updatesList.size(), equalTo(0)); + + assertThat( + countersByFirstStaleTime.keySet(), + containsInAnyOrder(counterMetricName1, counterMetricName2)); + assertThat( + perWorkerCounters.keySet(), containsInAnyOrder(counterMetricName1, counterMetricName2)); + + // At minute 2 metric1 is zero-valued, metric2 has been updated. + metricsContainer.getPerWorkerCounter(counterMetricName2).inc(3); + clock.advance(Duration.ofSeconds(60)); + + updatesList = Lists.newArrayList(metricsContainer.extractPerWorkerMetricUpdates()); + assertThat(updatesList.size(), equalTo(1)); + + assertThat(countersByFirstStaleTime.keySet(), contains(counterMetricName1)); + assertThat( + perWorkerCounters.keySet(), containsInAnyOrder(counterMetricName1, counterMetricName2)); + + // After minute 6 metric1 is still zero valued and should be cleaned up. + metricsContainer.getPerWorkerCounter(counterMetricName2).inc(3); + clock.advance(Duration.ofSeconds(4 * 60 + 1)); + + updatesList = Lists.newArrayList(metricsContainer.extractPerWorkerMetricUpdates()); + assertThat(updatesList.size(), equalTo(1)); + + assertThat(countersByFirstStaleTime.size(), equalTo(0)); + assertThat(perWorkerCounters.keySet(), contains(counterMetricName2)); + } } From d62ae0144ccfb388a48986c6ec23d716703f3966 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Wed, 24 Jan 2024 16:06:50 -0800 Subject: [PATCH 069/169] Add section for 2.55.0, cleanup 2.54.0 due to cut. (#30101) --- CHANGES.md | 44 ++++++++++++++++++++++++++++++++++++++------ 1 file changed, 38 insertions(+), 6 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index d4b9b7d77e07..2d9c249bf6b3 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -52,8 +52,7 @@ * ([#X](https://github.com/apache/beam/issues/X)). --> - -# [2.54.0] - Unreleased +# [2.55.0] - Unreleased ## Highlights @@ -63,6 +62,39 @@ ## I/Os * Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). + +## New Features / Improvements + +* X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). + +## Breaking Changes + +* X behavior was changed ([#X](https://github.com/apache/beam/issues/X)). + +## Deprecations + +* X behavior is deprecated and will be removed in X versions ([#X](https://github.com/apache/beam/issues/X)). + +## Bugfixes + +* Fixed X (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). + +## Security Fixes +* Fixed (CVE-YYYY-NNNN)[https://www.cve.org/CVERecord?id=CVE-YYYY-NNNN] (Java/Python/Go) ([#X](https://github.com/apache/beam/issues/X)). + +## Known Issues + +* ([#X](https://github.com/apache/beam/issues/X)). + +# [2.54.0] - Cut,Unreleased + +## Highlights + +* 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)). + +## I/Os + * Added support for writing to BigQuery dynamic destinations with Python's Storage Write API ([#30045](https://github.com/apache/beam/pull/30045)) * Adding support for Tuples DataType in ClickHouse (Java) ([#29715](https://github.com/apache/beam/pull/29715)). * Added support for handling bad records to FileIO, TextIO, AvroIO ([#29670](https://github.com/apache/beam/pull/29670)). @@ -74,18 +106,18 @@ ## Breaking Changes -* X behavior was changed ([#X](https://github.com/apache/beam/issues/X)). +* N/A ## Deprecations -* X behavior is deprecated and will be removed in X versions ([#X](https://github.com/apache/beam/issues/X)). +* N/A ## Bugfixes -* Fixed X (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* N/A ## Security Fixes -* Fixed (CVE-YYYY-NNNN)[https://www.cve.org/CVERecord?id=CVE-YYYY-NNNN] (Java/Python/Go) ([#X](https://github.com/apache/beam/issues/X)). +* N/A ## Known Issues From 61a62e19c4a822c92b3db5018ed1d522aad67423 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> Date: Wed, 24 Jan 2024 20:34:04 -0500 Subject: [PATCH 070/169] rename ExternalSchemaTransform to ExternalTransform (#30102) --- ...ider.py => external_transform_provider.py} | 34 +++++++++---------- ...py => external_transform_provider_test.py} | 18 +++++----- 2 files changed, 26 insertions(+), 26 deletions(-) rename sdks/python/apache_beam/transforms/{external_schematransform_provider.py => external_transform_provider.py} (90%) rename sdks/python/apache_beam/transforms/{external_schematransform_provider_test.py => external_transform_provider_test.py} (87%) diff --git a/sdks/python/apache_beam/transforms/external_schematransform_provider.py b/sdks/python/apache_beam/transforms/external_transform_provider.py similarity index 90% rename from sdks/python/apache_beam/transforms/external_schematransform_provider.py rename to sdks/python/apache_beam/transforms/external_transform_provider.py index fd650087893a..26cc31471e69 100644 --- a/sdks/python/apache_beam/transforms/external_schematransform_provider.py +++ b/sdks/python/apache_beam/transforms/external_transform_provider.py @@ -29,7 +29,7 @@ from apache_beam.typehints.schemas import named_tuple_to_schema from apache_beam.typehints.schemas import typing_from_runner_api -__all__ = ['ExternalSchemaTransform', 'ExternalSchemaTransformProvider'] +__all__ = ['ExternalTransform', 'ExternalTransformProvider'] def snake_case_to_upper_camel_case(string): @@ -84,7 +84,7 @@ def get_config_with_descriptions( return fields_with_descriptions -class ExternalSchemaTransform(PTransform): +class ExternalTransform(PTransform): """Template for a wrapper class of an external SchemaTransform This is a superclass for dynamically generated SchemaTransform wrappers and @@ -93,7 +93,7 @@ class ExternalSchemaTransform(PTransform): Experimental; no backwards compatibility guarantees.""" # These attributes need to be set when - # creating an ExternalSchemaTransform type + # creating an ExternalTransform type default_expansion_service = None description: str = "" identifier: str = "" @@ -138,21 +138,21 @@ def infer_name_from_identifier(identifier: str, pattern: str): return ''.join(components) -class ExternalSchemaTransformProvider: +class ExternalTransformProvider: """Dynamically discovers Schema-aware external transforms from a given list of expansion services and provides them as ready PTransforms. - A :class:`ExternalSchemaTransform` subclass is generated for each external + A :class:`ExternalTransform` subclass is generated for each external transform, and is named based on what can be inferred from the URN (see :param urn_pattern). - These classes are generated when :class:`ExternalSchemaTransformProvider` is + These classes are generated when :class:`ExternalTransformProvider` is initialized. We need to give it one or more expansion service addresses that are already up and running: - >>> provider = ExternalSchemaTransformProvider(["localhost:12345", + >>> provider = ExternalTransformProvider(["localhost:12345", ... "localhost:12121"]) We can also give it the gradle target of a standard Beam expansion service: - >>> provider = ExternalSchemaTransform(BeamJarExpansionService( + >>> provider = ExternalTransform(BeamJarExpansionService( ... "sdks:java:io:google-cloud-platform:expansion-service:shadowJar")) Let's take a look at the output of :func:`get_available()` to know the available transforms in the expansion service(s) we provided: @@ -162,7 +162,7 @@ class ExternalSchemaTransformProvider: ...] Then retrieve a transform by :func:`get()`, :func:`get_urn()`, or by directly - accessing it as an attribute of :class:`ExternalSchemaTransformProvider`. + accessing it as an attribute of :class:`ExternalTransformProvider`. All of the following commands do the same thing: >>> provider.get('BigqueryStorageRead') >>> provider.get_urn( @@ -194,7 +194,7 @@ class ExternalSchemaTransformProvider: Experimental; no backwards compatibility guarantees. """ def __init__(self, expansion_services, urn_pattern=STANDARD_URN_PATTERN): - f"""Initialize an ExternalSchemaTransformProvider + f"""Initialize an ExternalTransformProvider :param expansion_services: A list of expansion services to discover transforms from. @@ -207,7 +207,7 @@ def __init__(self, expansion_services, urn_pattern=STANDARD_URN_PATTERN): By default, the following pattern is used: [{STANDARD_URN_PATTERN}] """ self._urn_pattern = urn_pattern - self._transforms: Dict[str, type(ExternalSchemaTransform)] = {} + self._transforms: Dict[str, type(ExternalTransform)] = {} self._name_to_urn: Dict[str, str] = {} if isinstance(expansion_services, set): @@ -245,7 +245,7 @@ def _create_wrappers(self): continue self._transforms[identifier] = type( - name, (ExternalSchemaTransform, ), + name, (ExternalTransform, ), dict( identifier=identifier, default_expansion_service=service, @@ -265,13 +265,13 @@ def _create_wrappers(self): setattr(self, transform.__name__, transform) def get_available(self) -> List[Tuple[str, str]]: - """Get a list of available ExternalSchemaTransform names and identifiers""" + """Get a list of available ExternalTransform names and identifiers""" return list(self._name_to_urn.items()) - def get(self, name) -> ExternalSchemaTransform: - """Get an ExternalSchemaTransform by its inferred class name""" + def get(self, name) -> ExternalTransform: + """Get an ExternalTransform by its inferred class name""" return self._transforms[self._name_to_urn[name]] - def get_urn(self, identifier) -> ExternalSchemaTransform: - """Get an ExternalSchemaTransform by its SchemaTransform identifier""" + def get_urn(self, identifier) -> ExternalTransform: + """Get an ExternalTransform by its SchemaTransform identifier""" return self._transforms[identifier] diff --git a/sdks/python/apache_beam/transforms/external_schematransform_provider_test.py b/sdks/python/apache_beam/transforms/external_transform_provider_test.py similarity index 87% rename from sdks/python/apache_beam/transforms/external_schematransform_provider_test.py rename to sdks/python/apache_beam/transforms/external_transform_provider_test.py index bf951e671c2a..36fe9b5c4bd6 100644 --- a/sdks/python/apache_beam/transforms/external_schematransform_provider_test.py +++ b/sdks/python/apache_beam/transforms/external_transform_provider_test.py @@ -25,12 +25,12 @@ from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to from apache_beam.transforms.external import BeamJarExpansionService -from apache_beam.transforms.external_schematransform_provider import STANDARD_URN_PATTERN -from apache_beam.transforms.external_schematransform_provider import ExternalSchemaTransformProvider -from apache_beam.transforms.external_schematransform_provider import camel_case_to_snake_case -from apache_beam.transforms.external_schematransform_provider import infer_name_from_identifier -from apache_beam.transforms.external_schematransform_provider import snake_case_to_lower_camel_case -from apache_beam.transforms.external_schematransform_provider import snake_case_to_upper_camel_case +from apache_beam.transforms.external_transform_provider import STANDARD_URN_PATTERN +from apache_beam.transforms.external_transform_provider import ExternalTransformProvider +from apache_beam.transforms.external_transform_provider import camel_case_to_snake_case +from apache_beam.transforms.external_transform_provider import infer_name_from_identifier +from apache_beam.transforms.external_transform_provider import snake_case_to_lower_camel_case +from apache_beam.transforms.external_transform_provider import snake_case_to_upper_camel_case class NameUtilsTest(unittest.TestCase): @@ -101,12 +101,12 @@ def test_infer_name_from_identifier(self): @unittest.skipUnless( os.environ.get('EXPANSION_PORT'), "EXPANSION_PORT environment var is not provided.") -class ExternalSchemaTransformProviderTest(unittest.TestCase): +class ExternalTransformProviderTest(unittest.TestCase): def setUp(self): self.test_pipeline = TestPipeline(is_integration_test=True) def test_generate_sequence_config_schema_and_description(self): - provider = ExternalSchemaTransformProvider( + provider = ExternalTransformProvider( BeamJarExpansionService(":sdks:java:io:expansion-service:shadowJar")) self.assertTrue(( @@ -125,7 +125,7 @@ def test_generate_sequence_config_schema_and_description(self): self.assertTrue(description_substring in GenerateSequence.description) def test_run_generate_sequence(self): - provider = ExternalSchemaTransformProvider( + provider = ExternalTransformProvider( BeamJarExpansionService(":sdks:java:io:expansion-service:shadowJar")) with beam.Pipeline() as p: From 93073e74e11aa3c5bc8448e7547bd4b7995e5c7b Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Thu, 25 Jan 2024 09:39:49 -0500 Subject: [PATCH 071/169] Decommission all remaining jenkins jobs (#30113) * Decommission all remaining jenkins jobs * Readme --- .test-infra/jenkins/README.md | 149 +----------------- .test-infra/jenkins/job_00_seed.groovy | 115 -------------- .test-infra/jenkins/job_Inventory.groovy | 83 ---------- ...CrossLanguageValidatesRunner_Direct.groovy | 51 ------ ...alidatesRunner_GoUsingJava_Dataflow.groovy | 44 ------ ...atesRunner_PythonUsingJava_Dataflow.groovy | 49 ------ .../job_PostCommit_Java_DataflowV2.groovy | 48 ------ .../job_PostCommit_Java_InfluxDBIO_IT.groovy | 55 ------- ...stCommit_Java_ValidatesRunner_Flink.groovy | 45 ------ ...t_Java_ValidatesRunner_Flink_Java11.groovy | 61 ------- .../jenkins/job_PostCommit_Python.groovy | 50 ------ ...it_Python_CrossLanguage_IO_Dataflow.groovy | 55 ------- ...b_PostCommit_Python_Examples_Direct.groovy | 43 ----- ...ob_PostCommit_Python_Examples_Flink.groovy | 43 ----- ...Commit_Python_ValidatesRunner_Flink.groovy | 38 ----- ...Commit_Python_ValidatesRunner_Spark.groovy | 43 ----- .../job_PostRelease_NightlySnapshot.groovy | 59 ------- .../jenkins/job_seed_standalone.groovy | 115 -------------- 18 files changed, 1 insertion(+), 1145 deletions(-) delete mode 100644 .test-infra/jenkins/job_00_seed.groovy delete mode 100644 .test-infra/jenkins/job_Inventory.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_Direct.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_GoUsingJava_Dataflow.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_PythonUsingJava_Dataflow.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_DataflowV2.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_InfluxDBIO_IT.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Flink.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Flink_Java11.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Python.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Python_CrossLanguage_IO_Dataflow.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Python_Examples_Direct.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Python_Examples_Flink.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Python_ValidatesRunner_Flink.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Python_ValidatesRunner_Spark.groovy delete mode 100644 .test-infra/jenkins/job_PostRelease_NightlySnapshot.groovy delete mode 100644 .test-infra/jenkins/job_seed_standalone.groovy diff --git a/.test-infra/jenkins/README.md b/.test-infra/jenkins/README.md index 4d08aedd7c62..aa1a35741b21 100644 --- a/.test-infra/jenkins/README.md +++ b/.test-infra/jenkins/README.md @@ -23,151 +23,4 @@ **DEPRECATED:** As of November 2023, Beam CI has migrated to self-hosted GitHub Action: [link](https://github.com/apache/beam/blob/master/.github/workflows/README.md). New tests should be setup by GitHub Action, and the Jenkins jobs listed below is planned to be shutdown. -Beam Jenkins overview page: [link](https://ci-beam.apache.org/) - -### PreCommit Jobs - -All PreCommit Jobs have been migrated to GitHub Action. - -### PostCommit Jobs - -| Name | Link | PR Trigger Phrase | Cron Status | -|------|------|-------------------|-------------| -| beam_PostCommit_BeamMetrics_Publish | [cron](https://ci-beam.apache.org/job/beam_PostCommit_BeamMetrics_Publish/) | N/A | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_BeamMetrics_Publish/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_BeamMetrics_Publish) | -| beam_PostCommit_XVR_Flink | [cron](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Flink/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Flink_PR/) | `Run XVR_Flink PostCommit` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Flink/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Flink) | -| beam_PostCommit_XVR_Samza | [cron](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Samza/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Samza_PR/) | `Run XVR_Samza PostCommit` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Samza/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Samza) | -| beam_PostCommit_XVR_Spark | [cron](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Spark/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Spark_PR/) | `Run XVR_Spark PostCommit` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Spark/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Spark) | -| beam_PostCommit_XVR_Spark3 | [cron](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_XVR_Spark3/), [phrase](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_XVR_Spark3_PR) | `Run XVR_Spark3 PostCommit` | [![Build Status](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_XVR_Spark3/badge/icon)](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_XVR_Spark3/) | -| beam_PostCommit_XVR_Direct | [cron](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Direct/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Direct_PR/) | `Run XVR_Direct PostCommit` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Direct/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Direct) | -| beam_PostCommit_XVR_Dataflow | [cron](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_XVR_Dataflow/), [phrase](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_XVR_Dataflow_PR/) | `Run XVR_Dataflow PostCommit` | [![Build Status](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_XVR_Dataflow/badge/icon)](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_XVR_Dataflow/) | -| beam_PostCommit_Go | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Go/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Go_PR/) | `Run Go PostCommit` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Go/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Go) | -| beam_PostCommit_Go_VR_Flink | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Flink/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Flink_PR/) | `Run Go Flink ValidatesRunner` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Flink/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Flink/) | -| beam_PostCommit_Go_VR_Samza | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Samza/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Samza_PR/) | `Run Go Samza ValidatesRunner` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Samza/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Samza/) | -| beam_PostCommit_Go_VR_Spark | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Spark/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Spark_PR/) | `Run Go Spark ValidatesRunner` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Spark/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Spark/) | -| beam_PostCommit_Java_Hadoop_Versions | [cron](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_Hadoop_Versions/), [phrase](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_Hadoop_Versions_PR/) | `Run PostCommit_Java_Hadoop_Versions` | [![Build Status](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_Hadoop_Versions/badge/icon)](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_Hadoop_Versions/) | -| beam_PostCommit_Java | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Java/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Java_PR/) | `Run Java PostCommit` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java) | -| beam_PostCommit_Java_DataflowV1 | [cron](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_DataflowV1/), [phrase](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_DataflowV1_PR/) | `Run PostCommit_Java_Dataflow` | [![Build Status](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_DataflowV1/badge/icon)](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_DataflowV1/) | -| beam_PostCommit_Java_DataflowV2 | [cron](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_DataflowV2/), [phrase](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_DataflowV2_PR/) | `Run PostCommit_Java_DataflowV2` | [![Build Status](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_DataflowV2/badge/icon)](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_DataflowV2/) | -| beam_PostCommit_Java_InfluxDbIO_IT | [cron](https://ci-beam.apache.org/view/PerformanceTests/job/beam_PostCommit_Java_InfluxDbIO_IT/), [phrase](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_SingleStoreIO_IT_PR/) | `Run Java InfluxDbIO_IT` | [![Build Status](https://ci-beam.apache.org/view/PerformanceTests/job/beam_PostCommit_Java_InfluxDbIO_IT/badge/icon)](https://ci-beam.apache.org/view/PerformanceTests/job/beam_PostCommit_Java_InfluxDbIO_IT/) | -| beam_PostCommit_Java_Nexmark_Dataflow | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Java_Nexmark_Dataflow/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Java_Nexmark_Dataflow_PR/) | `Dataflow Runner Nexmark Tests` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_Nexmark_Dataflow/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_Nexmark_Dataflow) | -| beam_PostCommit_Java_Nexmark_Dataflow_V2 | [cron](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_Nexmark_Dataflow_V2/), [phrase](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_Nexmark_DataflowV2_PR/) | `Run Dataflow Runner V2 Nexmark Tests` | [![Build Status](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_Nexmark_Dataflow_V2/badge/icon)](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_Nexmark_Dataflow_V2/) | -| beam_PostCommit_Java_Nexmark_Dataflow_V2_Java11 | [cron](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java11/), [phrase](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java11/badge/icon) | `Dataflow Runner V2 Java 11 Nexmark Tests` | [![Build Status](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java11/badge/icon)](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java11/) | -| beam_PostCommit_Java_Nexmark_Direct | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Java_Nexmark_Direct/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Java_Nexmark_Direct_PR/) | `Direct Runner Nexmark Tests` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_Nexmark_Direct/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_Nexmark_Direct) | -| beam_PostCommit_Java_Nexmark_Flink | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Java_Nexmark_Flink/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Java_Nexmark_Flink_PR/) | `Flink Runner Nexmark Tests` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_Nexmark_Flink/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_Nexmark_Flink) | -| beam_PostCommit_Java_Nexmark_Spark | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Java_Nexmark_Spark/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Java_Nexmark_Spark_PR/) | `Spark Runner Nexmark Tests` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_Nexmark_Spark/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_Nexmark_Spark) | -| beam_PostCommit_Java_PVR_Flink_Batch | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch_PR/) | `Run Java Flink PortableValidatesRunner Batch` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch) | -| beam_PostCommit_Java_PVR_Flink_Streaming | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming_PR/) | `Run Java Flink PortableValidatesRunner Streaming` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming) | -| beam_PostCommit_Java_PVR_Samza | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Samza/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Samza_PR/) | `Run Java Samza PortableValidatesRunner` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Samza/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Samza) | -| beam_PostCommit_Java_PVR_Spark_Batch | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch_PR/) | `Run Java Spark PortableValidatesRunner Batch` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch) | -| beam_PostCommit_Java_PVR_Spark2_Streaming | [cron](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_PVR_Spark2_Streaming/), [phrase](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_PVR_Spark2_Streaming_PR/) | `Run Java Spark v2 PortableValidatesRunner Streaming` | [![Build Status](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_PVR_Spark2_Streaming/badge/icon)](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_PVR_Spark2_Streaming/) | -| beam_PostCommit_Java_PVR_Spark3_Streaming | [cron](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_PVR_Spark3_Streaming/), [phrase](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_PVR_Spark3_Streaming_PR/) | `Run Java Spark v3 PortableValidatesRunner Streaming` | [![Build Status](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_PVR_Spark3_Streaming/badge/icon)](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_PVR_Spark3_Streaming/) | -| beam_PostCommit_Java_SingleStoreIO_IT | [cron](https://ci-beam.apache.org/view/PerformanceTests/job/beam_PostCommit_Java_SingleStoreIO_IT/), [phrase](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_SingleStoreIO_IT_PR/) | `Run Java SingleStoreIO_IT` | [![Build Status](https://ci-beam.apache.org/view/PerformanceTests/job/beam_PostCommit_Java_SingleStoreIO_IT/badge/icon)](https://ci-beam.apache.org/view/PerformanceTests/job/beam_PostCommit_Java_SingleStoreIO_IT/) | -| beam_PostCommit_Java_Dataflow_Examples_Java11 | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Java_Examples_Dataflow_Java11/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Java_Examples_Dataflow_Java11_PR/) | `Run Java examples on Dataflow Java 11` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_Examples_Dataflow_Java11/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_Examples_Dataflow_Java11) | -| beam_PostCommit_Java_Examples_Dataflow_V2 | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Java_Examples_Dataflow_V2/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Java_Examples_Dataflow_V2/) | `Run Java Examples on Dataflow Runner V2` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_Examples_Dataflow_V2/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_Examples_Dataflow_V2) | -| beam_PostCommit_Java_Examples_Dataflow_V2_Java11 | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Java_Examples__Dataflow_V2_Java11/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Java_Examples__Dataflow_V2_Java11/) | `Run Java 11 Examples on Dataflow Runner V2` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_Examples_Dataflow_V2_Java11/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_Examples_Dataflow_V2_Java11) | -| beam_PostCommit_Java_Jpms_Dataflow_Java11 | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Java_Jpms_Dataflow_Java11/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Java_Jpms_Dataflow_Java11_PR/) | `Run Jpms Dataflow Java 11 PostCommit` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_Jpms_Dataflow_Java11/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_Jpms_Dataflow_Java11/) | -| beam_PostCommit_Java_Jpms_Direct_Java11 | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Java_Jpms_Direct_Java11/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Java_Jpms_Direct_Java11_PR/) | `Run Jpms Direct Java 11 PostCommit` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_Jpms_Direct_Java11/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_Jpms_Direct_Java11/) | -| beam_PostCommit_Java_Jpms_Flink_Java11 | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Java_Jpms_Flink_Java11/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Java_Jpms_Flink_Java11_PR/) | `Run Jpms Flink Java 11 PostCommit` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_Jpms_Flink_Java11/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_Jpms_Flink_Java11/) | -| beam_PostCommit_Java_Jpms_Spark_Java11 | [cron](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_Jpms_Spark_Java11/), [phrase](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_Jpms_Spark_Java11_PR/) | `Run Jpms Spark Java 11 PostCommit` | [![Build Status](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_Jpms_Spark_Java11/badge/icon)](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_Jpms_Spark_Java11/badge/icon) | -| beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11 | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11_PR/) | `Run Dataflow ValidatesRunner Java 11` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11) | -| beam_PostCommit_Java_ValidatesRunner_Direct | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Direct/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Direct_PR/) | `Run Direct ValidatesRunner` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Direct/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Direct/) | -| beam_PostCommit_Java_ValidatesRunner_Direct_Java11 | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Direct_Java11), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Direct_Java11_PR) | `Run Direct ValidatesRunner Java 11` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Direct_Java11/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Direct_Java11) | -| beam_PostCommit_Java_ValidatesRunner_Dataflow | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_PR/) | `Run Dataflow ValidatesRunner` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow) | -| beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming | [cron](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming/), [phrase](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming_PR/) | `Run Dataflow Streaming ValidatesRunner` | [![Build Status](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming/badge/icon)](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming/) | -| beam_PostCommit_Java_ValidatesRunner_Flink | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink_PR/) | `Run Flink ValidatesRunner` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink) | -| beam_PostCommit_Java_ValidatesRunner_Flink_Java11 | [cron](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_ValidatesRunner_Flink_Java11/), [phrase](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_ValidatesRunner_Flink_Java11_PR/) | `Run Flink ValidatesRunner Java 11` | [![Build Status](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_ValidatesRunner_Flink_Java11/badge/icon)](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_ValidatesRunner_Flink_Java11/) | -| beam_PostCommit_Java_ValidatesRunner_Samza | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza_PR/) | `Run Samza ValidatesRunner` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza) | -| beam_PostCommit_Java_ValidatesRunner_Twister2 | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Twister2/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Twister2_PR/) | `Run Twister2 ValidatesRunner` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Twister2/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Twister2) | -| beam_PostCommit_Java_ValidatesRunner_Spark | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark_PR/) | `Run Spark ValidatesRunner` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark) | -| beam_PostCommit_Java_ValidatesRunner_Spark_Java11 | [cron](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_ValidatesRunner_Spark_Java11/), [phrase](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_ValidatesRunner_Spark_Java11_PR) | `Run Spark ValidatesRunner Java 11` | [![Build Status](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_ValidatesRunner_Spark_Java11/badge/icon)](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_ValidatesRunner_Spark_Java11/) | -| beam_PostCommit_Java_ValidatesRunner_ULR | [cron](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_ValidatesRunner_ULR/), [phrase](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_ValidatesRunner_ULR_PR) | `Run ULR Loopback ValidatesRunner` | [![Build Status](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_ValidatesRunner_ULR/badge/icon)](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_ValidatesRunner_ULR/) | -| beam_PostCommit_Java_VR_Dataflow_V2 | [cron](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_VR_Dataflow_V2/), [phrase](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_VR_Dataflow_V2_PR/) | `Run Java Dataflow V2 ValidatesRunner` | [![Build Status](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_VR_Dataflow_V2/badge/icon)](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_VR_Dataflow_V2/) | -| beam_PostCommit_Java_VR_Dataflow_V2_Streaming | [cron](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_VR_Dataflow_V2_Streaming/), [phrase](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_VR_Dataflow_V2_Streaming_PR/) | `Run Java Dataflow V2 ValidatesRunner Streaming` | [![Build Status](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_VR_Dataflow_V2_Streaming/badge/icon)](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_VR_Dataflow_V2_Streaming/) | -| beam_PostCommit_Javadoc | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Javadoc/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Javadoc_PR/) | `Run Javadoc PostCommit` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Javadoc/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Javadoc) | -| beam_PostCommit_PortableJar_Flink | [cron](https://ci-beam.apache.org/job/beam_PostCommit_PortableJar_Flink/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_PortableJar_Flink_PR/) | `Run PortableJar_Flink PostCommit` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_PortableJar_Flink/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_PortableJar_Flink) | -| beam_PostCommit_PortableJar_Spark | [cron](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_PortableJar_Spark/), [phrase](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_PortableJar_Spark_PR/) | `Run PortableJar_Spark PostCommit` | [![Build Status](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_PortableJar_Spark/badge/icon)](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_PortableJar_Spark/) | -| beam_PostCommit_Java_Sickbay | [cron](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_Sickbay/), [phrase](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_Sickbay_PR/) | `Run Java Sickbay` | [![Build Status](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_Sickbay/badge/icon)](https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_Sickbay/) | -| beam_PostCommit_Py_VR_Dataflow | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow_PR/) | `Run Python Dataflow ValidatesRunner` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow) | -| beam_PostCommit_Py_ValCont | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Py_ValCont/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Py_ValCont_PR/) | `Run Python Dataflow ValidatesContainer` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Py_ValCont/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Py_ValCont) | -| beam_PostCommit_Python_VR_Flink | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Flink/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Flink_PR/) | `Run Python Flink ValidatesRunner` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Flink/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Flink) | -| beam_PostCommit_Python_VR_Samza | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Samza/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Samza_PR/) | `Run Python Samza ValidatesRunner` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Samza/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Samza) | -| beam_PostCommit_Py_ValCont_with_RC | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Py_ValCont_with_RC/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Py_ValCont_with_RC_PR/) | `Run Python RC Dataflow ValidatesContainer` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Py_ValCont_with_RC/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Py_ValCont_with_RC) | -| beam_PostCommit_Python_Examples_Direct | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Python_Examples_Direct/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Python_Examples_Direct_PR/) | `Run Python Examples_Direct` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python_Examples_Direct/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python_Examples_Direct/) | -| beam_PostCommit_Python_Examples_Dataflow | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Python_Examples_Dataflow/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Python_Examples_Dataflow_PR/) | `Run Python Examples_Dataflow` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python_Examples_Dataflow/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python_Examples_Dataflow/) | -| beam_PostCommit_Python_Examples_Flink | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Python_Examples_Flink/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Python_Examples_Flink_PR/) | `Run Python Examples_Flink` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python_Examples_Flink/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python_Examples_Flink/) | -| beam_PostCommit_Python_Examples_Spark | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Python_Examples_Spark/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Python_Examples_Spark_PR/) | `Run Python Examples_Spark` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python_Examples_Spark/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python_Examples_Spark/) | -| beam_PostCommit_Python_Chicago_Taxi_Example_Dataflow | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Python_Chicago_Taxi_Dataflow/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Python_Chicago_Taxi_Dataflow_PR/) | `Run Chicago Taxi on Dataflow` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python_Chicago_Taxi_Dataflow/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python_Chicago_Taxi_Dataflow) | -| beam_PostCommit_Python_Chicago_Taxi_Example_Flink | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Python_Chicago_Taxi_Flink/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Python_Chicago_Taxi_Flink_PR/) | `Run Chicago Taxi on Flink` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python_Chicago_Taxi_Flink/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python_Chicago_Taxi_Flink) | -| beam_PostCommit_Python_MongoDBIO_IT | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Python_MongoDBIO_IT), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Python_MongoDBIO_IT_PR/) | `Run Python MongoDBIO_IT` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python_MongoDBIO_IT/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python_MongoDBIO_IT) | -| beam_PostCommit_Python_VR_Spark | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Spark/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Spark/) | `Run Python Spark ValidatesRunner` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Spark/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Spark) | -| beam_PostCommit_Python_Xlang_Gcp_Direct | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Python_Xlang_Gcp_Direct/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Python_Xlang_Gcp_Direct_PR/) | `Run Python_Xlang_Gcp_Direct PostCommit` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python_Xlang_Gcp_Direct/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python_Xlang_Gcp_Direct/) | -| beam_PostCommit_Python_Xlang_Gcp_Dataflow | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Python_Xlang_Gcp_Dataflow/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Python_Xlang_Gcp_Dataflow_PR/) | `Run Python_Xlang_Gcp_Dataflow PostCommit` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python_Xlang_Gcp_Dataflow/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python_Xlang_Gcp_Dataflow/) | -| beam_PostCommit_Python_Xlang_IO_Dataflow | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Python_Xlang_IO_Dataflow/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Python_Xlang_IO_Dataflow_PR/) | `Run Python_Xlang_IO_Dataflow PostCommit` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python_Xlang_IO_Dataflow/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python_Xlang_IO_Dataflow/| -| beam_PostCommit_Python38 | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Python38), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Python38_PR/) | `Run Python 3.8 PostCommit` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python38/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python38) | -| beam_PostCommit_Python39 | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Python39), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Python39_PR/) | `Run Python 3.9 PostCommit` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python39/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python39) | -| beam_PostCommit_Python310 | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Python310), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Python310_PR/) | `Run Python 3.10 PostCommit` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python310/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python310) | -| beam_PostCommit_Python311 | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Python311), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_Python311_PR/) | `Run Python 3.11 PostCommit` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python311/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python311) | -| beam_PostCommit_Sickbay_Python38 | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Sickbay_Python38), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_SickBay_Python38_PR/) | `Run Python 3.8 PostCommit Sickbay tests` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Sickbay_Python38/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Sickbay_Python38) | -| beam_PostCommit_Sickbay_Python39 | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Sickbay_Python39), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_SickBay_Python39_PR/) | `Run Python 3.9 PostCommit Sickbay tests` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Sickbay_Python39/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Sickbay_Python39) | -| beam_PostCommit_Sickbay_Python310 | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Sickbay_Python310), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_SickBay_Python310_PR/) | `Run Python 3.10 PostCommit Sickbay tests` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Sickbay_Python310/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Sickbay_Python310) | -| beam_PostCommit_Sickbay_Python311 | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Sickbay_Python311), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_SickBay_Python311_PR/) | `Run Python 3.11 PostCommit Sickbay tests` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Sickbay_Python311/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Sickbay_Python311) | -| beam_PostCommit_SQL | [cron](https://ci-beam.apache.org/job/beam_PostCommit_SQL/), [phrase](https://ci-beam.apache.org/job/beam_PostCommit_SQL_PR/) | `Run SQL PostCommit` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_SQL/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_SQL) | -| beam_PostCommit_Website_Publish | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Website_Publish/) | N/A | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Website_Publish/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Website_Publish) | -| beam_PostCommit_Website_Test | [cron](https://ci-beam.apache.org/job/beam_PostCommit_Website_Test/) | `Run Full Website Test` | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Website_Test/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Website_Test) | - -### Performance Jobs - -All Performance Jobs have been migrated to GitHub Action. - -### Load test Jobs - -All LoadTest Jobs have been migrated to GitHub Action. - -### Inventory Jobs - -| Name | Link | PR Trigger Phrase | Cron Status | -|------|------|-------------------|-------------| -| beam_Inventory_apache-beam-jenkins-1 | [cron](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-1/) | `Run inventory apache-beam-jenkins-1` | [![Build Status](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-1/badge/icon)](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-1) | -| beam_Inventory_apache-beam-jenkins-2 | [cron](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-2/) | `Run inventory apache-beam-jenkins-2` | [![Build Status](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-2/badge/icon)](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-2) | -| beam_Inventory_apache-beam-jenkins-3 | [cron](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-3/) | `Run inventory apache-beam-jenkins-3` | [![Build Status](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-3/badge/icon)](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-3) | -| beam_Inventory_apache-beam-jenkins-4 | [cron](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-4/) | `Run inventory apache-beam-jenkins-4` | [![Build Status](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-4/badge/icon)](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-4) | -| beam_Inventory_apache-beam-jenkins-5 | [cron](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-5/) | `Run inventory apache-beam-jenkins-5` | [![Build Status](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-5/badge/icon)](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-5) | -| beam_Inventory_apache-beam-jenkins-6 | [cron](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-6/) | `Run inventory apache-beam-jenkins-6` | [![Build Status](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-6/badge/icon)](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-6) | -| beam_Inventory_apache-beam-jenkins-7 | [cron](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-7/) | `Run inventory apache-beam-jenkins-7` | [![Build Status](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-7/badge/icon)](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-7) | -| beam_Inventory_apache-beam-jenkins-8 | [cron](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-8/) | `Run inventory apache-beam-jenkins-8` | [![Build Status](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-8/badge/icon)](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-8) | -| beam_Inventory_apache-beam-jenkins-9 | [cron](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-9/) | `Run inventory apache-beam-jenkins-9` | [![Build Status](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-9/badge/icon)](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-9) | -| beam_Inventory_apache-beam-jenkins-10 | [cron](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-10/) | `Run inventory apache-beam-jenkins-10` | [![Build Status](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-10/badge/icon)](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-10) | -| beam_Inventory_apache-beam-jenkins-11 | [cron](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-11/) | `Run inventory apache-beam-jenkins-11` | [![Build Status](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-11/badge/icon)](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-11) | -| beam_Inventory_apache-beam-jenkins-12 | [cron](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-12/) | `Run inventory apache-beam-jenkins-12` | [![Build Status](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-12/badge/icon)](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-12) | -| beam_Inventory_apache-beam-jenkins-13 | [cron](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-13/) | `Run inventory apache-beam-jenkins-13` | [![Build Status](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-13/badge/icon)](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-13) | -| beam_Inventory_apache-beam-jenkins-14 | [cron](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-14/) | `Run inventory apache-beam-jenkins-14` | [![Build Status](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-14/badge/icon)](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-14) | -| beam_Inventory_apache-beam-jenkins-15 | [cron](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-15/) | `Run inventory apache-beam-jenkins-15` | [![Build Status](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-15/badge/icon)](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-15) | -| beam_Inventory_apache-beam-jenkins-16 | [cron](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-16/) | `Run inventory apache-beam-jenkins-16` | [![Build Status](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-16/badge/icon)](https://ci-beam.apache.org/job/beam_Inventory_apache-beam-jenkins-16) | - -### Other Jobs - -| Name | Link | PR Trigger Phrase | Cron Status | -|------|------|-------------------|-------------| -| beam_Metrics_Report | [cron](https://ci-beam.apache.org/job/beam_Metrics_Report/) | `Run Metrics Report` | [![Build Status](https://ci-beam.apache.org/job/beam_Metrics_Report/badge/icon)](https://ci-beam.apache.org/job/beam_Metrics_Report) | -| beam_Release_NightlySnapshot | [cron](https://ci-beam.apache.org/job/beam_Release_NightlySnapshot/) | `Run Gradle Publish` | [![Build Status](https://ci-beam.apache.org/job/beam_Release_NightlySnapshot/badge/icon)](https://ci-beam.apache.org/job/beam_Release_NightlySnapshot) | -| beam_Release_Python_NightlySnapshot | [cron](https://ci-beam.apache.org/job/beam_Release_Python_NightlySnapshot/) | `Run Python Publish` | [![Build Status](https://ci-beam.apache.org/job/beam_Release_Python_NightlySnapshot/badge/icon)](https://ci-beam.apache.org/job/beam_Release_Python_NightlySnapshot) | -| beam_PostRelease_NightlySnapshot | [cron](https://ci-beam.apache.org/job/beam_PostRelease_NightlySnapshot/) | `Run Dataflow PostRelease` | [![Build Status](https://ci-beam.apache.org/job/beam_PostRelease_NightlySnapshot/badge/icon)](https://ci-beam.apache.org/job/beam_PostRelease_NightlySnapshot) | -| beam_Prober_CommunityMetrics | [cron](https://ci-beam.apache.org/job/beam_Prober_CommunityMetrics/) | `Run Community Metrics Prober` | [![Build Status](https://ci-beam.apache.org/job/beam_Prober_CommunityMetrics/badge/icon)](https://ci-beam.apache.org/job/beam_Prober_CommunityMetrics) | -| beam_SeedJob | [cron](https://ci-beam.apache.org/job/beam_SeedJob/), [standalone](https://ci-beam.apache.org/job/beam_SeedJob_Standalone/) | `Run Seed Job` | [![Build Status](https://ci-beam.apache.org/job/beam_SeedJob/badge/icon)](https://ci-beam.apache.org/job/beam_SeedJob) | -| beam_sonarqube_report | [cron](https://ci-beam.apache.org/job/beam_sonarqube_report/)| N/A | [![Build Status](https://ci-beam.apache.org/job/beam_sonarqube_report/badge/icon)](https://ci-beam.apache.org/job/beam_sonarqube_report/) | -| beam_CancelStaleDataflowJobs | [cron](https://ci-beam.apache.org/job/beam_CancelStaleDataflowJobs/)| `Run Cancel Stale Dataflow Jobs` | [![Build Status](https://ci-beam.apache.org/job/beam_CancelStaleDataflowJobs/badge/icon)](https://ci-beam.apache.org/job/beam_CancelStaleDataflowJobs/) | -| beam_CleanUpGCPResources | [cron](https://ci-beam.apache.org/job/beam_CleanUpGCPResources/)| `Run Clean GCP Resources` | [![Build Status](https://ci-beam.apache.org/job/beam_CleanUpGCPResources/badge/icon)](https://ci-beam.apache.org/job/beam_CleanUpGCPResources/) | -| beam_Clean_tmp_directory | [cron](https://ci-beam.apache.org/job/beam_Clean_tmp_directory/)| N/A | [![Build Status](https://ci-beam.apache.org/job/beam_Clean_tmp_directory/badge/icon)](https://ci-beam.apache.org/job/beam_Clean_tmp_directory/) | -| beam_Publish_Beam_SDK_Snapshots | [cron](https://ci-beam.apache.org/job/beam_Publish_Beam_SDK_Snapshots/)| N/A | [![Build Status](https://ci-beam.apache.org/job/beam_Publish_Beam_SDK_Snapshots/badge/icon)](https://ci-beam.apache.org/job/beam_Publish_Beam_SDK_Snapshots/) | -| beam_Publish_Docker_Snapshots | [cron](https://ci-beam.apache.org/job/beam_Publish_Docker_Snapshots/)| N/A | [![Build Status](https://ci-beam.apache.org/job/beam_Publish_Docker_Snapshots/badge/icon)](https://ci-beam.apache.org/job/beam_Publish_Docker_Snapshots/) | -| beam_PostRelease_Python_Candidate | [cron](https://ci-beam.apache.org/job/beam_PostRelease_Python_Candidate/)| `Run Python ReleaseCandidate` | [![Build Status](https://ci-beam.apache.org/job/beam_PostRelease_Python_Candidate/badge/icon)](https://ci-beam.apache.org/job/beam_PostRelease_Python_Candidate/) | - -### Notes: - -* To rerun full set of PreCommit jobs in PR, comment following trigger phrase: - -``` -retest this please -``` - -* Last update (mm/dd/yyyy): 11/27/2023 +All jobs have been migrated as of January 2023, this folder just contains remaining test resources which should be moved/cleaned up (https://github.com/apache/beam/issues/30112). diff --git a/.test-infra/jenkins/job_00_seed.groovy b/.test-infra/jenkins/job_00_seed.groovy deleted file mode 100644 index 441b1eaf6cb9..000000000000 --- a/.test-infra/jenkins/job_00_seed.groovy +++ /dev/null @@ -1,115 +0,0 @@ -/* - * 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. - */ - -// Defines the seed job, which creates or updates all other Jenkins projects. - -import Committers as committers - -job('beam_SeedJob') { - description('Automatically configures all Apache Beam Jenkins projects based' + - ' on Jenkins DSL groovy files checked into the code repository.') - - properties { - githubProjectUrl('https://github.com/apache/beam/') - } - - // Restrict to only run on Jenkins executors labeled 'beam' - label('beam') - - logRotator { - daysToKeep(30) - } - - scm { - git { - remote { - github('apache/beam') - - // ${ghprbPullId} is not interpolated by groovy, but passed through to Jenkins where it - // refers to the environment variable - refspec([ - '+refs/heads/*:refs/remotes/origin/*', - '+refs/pull/${ghprbPullId}/*:refs/remotes/origin/pr/${ghprbPullId}/*'] - .join(' ')) - - // The variable ${sha1} is not interpolated by groovy, but a parameter of the Jenkins job - branch('${sha1}') - - extensions { - cleanAfterCheckout() - } - } - } - } - - parameters { - // Setup for running this job from a pull request - stringParam( - 'sha1', - 'master', - 'Commit id or refname (eg: origin/pr/4001/head) you want to build against.') - } - - wrappers { - timeout { - absolute(60) - abortBuild() - } - } - - triggers { - // Run every six hours, alternate job_seed_standalone has same freq - cron('H 0,6,12,18 * * *') - - githubPullRequest { - admins(['asfbot']) - useGitHubHooks() - userWhitelist(committers.GITHUB_USERNAMES) - - // Also run when manually kicked on a pull request - triggerPhrase('Run Seed Job') - onlyTriggerPhrase() - - extensions { - commitStatus { - context("Jenkins: Seed Job") - } - - buildStatus { - completedStatus('SUCCESS', '--none--') - completedStatus('FAILURE', '--none--') - completedStatus('ERROR', '--none--') - } - } - } - } - - publishers { - mailer('builds@beam.apache.org', false, true) - } - - steps { - dsl { - // A list or a glob of other groovy files to process. - external('.test-infra/jenkins/job_*.groovy') - - // If a job is removed from the script, disable it (rather than deleting). - removeAction('DISABLE') - } - } -} diff --git a/.test-infra/jenkins/job_Inventory.groovy b/.test-infra/jenkins/job_Inventory.groovy deleted file mode 100644 index e115603d0d42..000000000000 --- a/.test-infra/jenkins/job_Inventory.groovy +++ /dev/null @@ -1,83 +0,0 @@ -/* - * 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 CommonJobProperties as commonJobProperties - -import static PythonTestProperties.ALL_SUPPORTED_VERSIONS - -// These jobs list details about each beam runner, to clarify what software -// is on each machine. -def nums = 1..16 -nums.each { - def machine = "apache-beam-jenkins-${it}" - job("beam_Inventory_${machine}") { - description("Run inventory on ${machine}") - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Sets that this is a cron job. - commonJobProperties.setCronJob(delegate, '45 */8 * * *') - - // Allows triggering this build against pull requests. - commonJobProperties.enablePhraseTriggeringFromPullRequest( - delegate, - "Machine Inventory ${machine}", - "Run Inventory ${machine}") - - parameters { - nodeParam('TEST_HOST') { - description("Select test host ${machine}") - defaultNodes([machine]) - allowedNodes([machine]) - trigger('multiSelectionDisallowed') - eligibility('AllNodeEligibility') - } - stringParam { - name("tmp_unaccessed_for") - defaultValue("24") - description("Files from /tmp dir that were not accessed for last `tmp_unaccessed_for` hours will be deleted.") - trim(true) - } - } - - steps { - shell('ls /home/jenkins/tools') - shell('ls /home/jenkins/tools/*') - shell('python --version || echo "python not found"') - shell('python3 --version || echo "python3 not found"') - ALL_SUPPORTED_VERSIONS.each { version -> - shell("python${version} --version || echo \"python${version} not found\"") - } - shell('gcloud -v || echo "gcloud not found"') - shell('kubectl version || echo "kubectl not found"') - ALL_SUPPORTED_VERSIONS.each { version -> - def versionSuffix = version.replace('.', '') - shell("python${version} -m venv test${versionSuffix} && . ./test${versionSuffix}/bin/activate && python --version && deactivate || echo \"python ${version} not found\"") - } - shell('echo "Maven home $MAVEN_HOME"') - shell('env') - shell('docker system prune --all --filter until=24h --force') - shell('docker volume prune --force') - shell('echo "Current size of /tmp dir is \$(sudo du -sh /tmp)"') - shell('echo "Deleting files accessed later than \${tmp_unaccessed_for} hours ago"') - shell('sudo find /tmp -type f -amin +\$((60*\${tmp_unaccessed_for})) -print -delete') - shell('echo "Size of /tmp dir after cleanup is \$(sudo du -sh /tmp)"') - } - } -} diff --git a/.test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_Direct.groovy b/.test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_Direct.groovy deleted file mode 100644 index 0bc3ca684016..000000000000 --- a/.test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_Direct.groovy +++ /dev/null @@ -1,51 +0,0 @@ -/* - * 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 CommonJobProperties as commonJobProperties -import PostcommitJobBuilder - -import static PythonTestProperties.CROSS_LANGUAGE_VALIDATES_RUNNER_PYTHON_VERSIONS - -// This job runs the suite of ValidatesRunner tests against the Direct runner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_XVR_Direct', - 'Run XVR_Direct PostCommit', 'Direct CrossLanguageValidatesRunner Tests', this) { - description('Runs the CrossLanguageValidatesRunner suite on the Direct runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 120) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - CROSS_LANGUAGE_VALIDATES_RUNNER_PYTHON_VERSIONS.each { pythonVersion -> - shell("echo \"*** RUN CROSS-LANGUAGE DIRECT USING PYTHON ${pythonVersion} ***\"") - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:python:test-suites:direct:xlang:validatesCrossLanguageRunner') - commonJobProperties.setGradleSwitches(delegate) - switches("-PpythonVersion=${pythonVersion}") - // only run non-python task (e.g. GoUsingJava) once - switches("-PskipNonPythonTask=${pythonVersion != CROSS_LANGUAGE_VALIDATES_RUNNER_PYTHON_VERSIONS[0]}") - } - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_GoUsingJava_Dataflow.groovy b/.test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_GoUsingJava_Dataflow.groovy deleted file mode 100644 index c13e1e342012..000000000000 --- a/.test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_GoUsingJava_Dataflow.groovy +++ /dev/null @@ -1,44 +0,0 @@ -/* - * 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 CommonJobProperties as commonJobProperties -import PostcommitJobBuilder - -// This job runs the suite of ValidatesRunner tests against the Dataflow runner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_XVR_GoUsingJava_Dataflow', - 'Run XVR_GoUsingJava_Dataflow PostCommit', 'Dataflow CrossLanguageValidatesRunner GoUsingJava Tests', this) { - description('Runs the CrossLanguageValidatesRunner GoUsingJava suite on the Dataflow runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - shell("echo \"*** RUN CROSS-LANGUAGE DATAFLOW GO WITH JAVA EXTERNAL TRANSFORMS ***\"") - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:google-cloud-dataflow-java:validatesCrossLanguageRunnerGoUsingJava') - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_PythonUsingJava_Dataflow.groovy b/.test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_PythonUsingJava_Dataflow.groovy deleted file mode 100644 index 03745685a299..000000000000 --- a/.test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_PythonUsingJava_Dataflow.groovy +++ /dev/null @@ -1,49 +0,0 @@ -/* - * 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 CommonJobProperties as commonJobProperties -import PostcommitJobBuilder - -import static PythonTestProperties.CROSS_LANGUAGE_VALIDATES_RUNNER_PYTHON_VERSIONS - -// This job runs the suite of ValidatesRunner tests against the Dataflow runner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_XVR_PythonUsingJava_Dataflow', - 'Run XVR_PythonUsingJava_Dataflow PostCommit', 'Dataflow CrossLanguageValidatesRunner PythonUsingJava Tests', this) { - description('Runs the CrossLanguageValidatesRunner PythonUsingJava suite on the Dataflow runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/pytest*.xml') - } - - // Gradle goals for this job. - steps { - CROSS_LANGUAGE_VALIDATES_RUNNER_PYTHON_VERSIONS.each { pythonVersion -> - shell("echo \"*** RUN CROSS-LANGUAGE DATAFLOW PYTHON WITH JAVA EXTERNAL TRANSFORMS USING PYTHON ${pythonVersion} ***\"") - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:google-cloud-dataflow-java:validatesCrossLanguageRunnerPythonUsingJava') - commonJobProperties.setGradleSwitches(delegate) - switches("-PpythonVersion=${pythonVersion}") - } - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_DataflowV2.groovy b/.test-infra/jenkins/job_PostCommit_Java_DataflowV2.groovy deleted file mode 100644 index 03d82d48974e..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_DataflowV2.groovy +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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 CommonJobProperties as commonJobProperties -import PostcommitJobBuilder - - -// This job runs the Java postcommit tests, including the suite of integration -// tests. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_DataflowV2', 'Run PostCommit_Java_DataflowV2', - 'Dataflow V2 Java Post Commit Tests', this) { - - description('Dataflow V2 Java Post Commit Tests') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(":runners:google-cloud-dataflow-java:postCommitRunnerV2") - commonJobProperties.setGradleSwitches(delegate) - // Specify maven home on Jenkins, needed by Maven archetype integration tests. - switches('-Pmaven_home=/home/jenkins/tools/maven/apache-maven-3.5.4') - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_InfluxDBIO_IT.groovy b/.test-infra/jenkins/job_PostCommit_Java_InfluxDBIO_IT.groovy deleted file mode 100644 index cb74cbf3228f..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_InfluxDBIO_IT.groovy +++ /dev/null @@ -1,55 +0,0 @@ -/* - * 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 CommonJobProperties as common -import PostcommitJobBuilder -import Kubernetes - -String jobName = "beam_PostCommit_Java_InfluxDbIO_IT" - -PostcommitJobBuilder.postCommitJob(jobName, 'Run Java InfluxDbIO_IT', 'Java InfluxDbIO Integration Test', this) { - description('Runs the Java InfluxDbIO Integration Test.') - previousNames(/beam_PerformanceTests_InfluxDbIO_IT/) - // Set common parameters. - common.setTopLevelMainJobProperties(delegate) - - // Deploy InfluxDb cluster - String namespace = common.getKubernetesNamespace(jobName) - String kubeconfigPath = common.getKubeconfigLocationForNamespace(namespace) - Kubernetes k8s = Kubernetes.create(delegate, kubeconfigPath, namespace) - - k8s.apply(common.makePathAbsolute("src/.test-infra/kubernetes/influxdb/influxdb.yml")) - String influxDBHostName = "LOAD_BALANCER_IP" - k8s.loadBalancerIP("influxdb-load-balancer-service", influxDBHostName) - Map pipelineOptions = [ - influxDBURL : "http://\$${influxDBHostName}:8086", - influxDBUserName : "superadmin", - influxDBPassword : "supersecretpassword", - databaseName : "db1" - ] - - steps { - gradle { - rootBuildScriptDir(common.checkoutDir) - common.setGradleSwitches(delegate) - switches("--info") - switches("-DintegrationTestPipelineOptions=\'${common.joinPipelineOptions(pipelineOptions)}\'") - switches("-DintegrationTestRunner=direct") - tasks(":sdks:java:io:influxdb:integrationTest --tests org.apache.beam.sdk.io.influxdb.InfluxDbIOIT") - } - } -} diff --git a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Flink.groovy b/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Flink.groovy deleted file mode 100644 index 6f24f9fe6a47..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Flink.groovy +++ /dev/null @@ -1,45 +0,0 @@ -/* - * 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 CommonJobProperties as commonJobProperties -import CommonTestProperties -import PostcommitJobBuilder - -// This job runs the suite of ValidatesRunner tests against the Flink runner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_ValidatesRunner_Flink', - 'Run Flink ValidatesRunner', 'Apache Flink Runner ValidatesRunner Tests', this) { - description('Runs the ValidatesRunner suite on the Flink runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - previousNames(/beam_PostCommit_Java_ValidatesRunner_Flink_Gradle/) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(":runners:flink:${CommonTestProperties.getFlinkVersion()}:validatesRunner") - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Flink_Java11.groovy b/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Flink_Java11.groovy deleted file mode 100644 index 9178caebd813..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Flink_Java11.groovy +++ /dev/null @@ -1,61 +0,0 @@ -/* - * 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 CommonJobProperties as commonJobProperties -import CommonTestProperties -import PostcommitJobBuilder - - -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_ValidatesRunner_Flink_Java11', - 'Run Flink ValidatesRunner Java 11', 'Apache Flink Runner ValidatesRunner Tests On Java 11', this) { - - description('Runs the ValidatesRunner suite on the Flink runner with Java 11.') - - def JAVA_11_HOME = '/usr/lib/jvm/java-11-openjdk-amd64' - def JAVA_8_HOME = '/usr/lib/jvm/java-8-openjdk-amd64' - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 270) - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(":runners:flink:${CommonTestProperties.getFlinkVersion()}:jar") - tasks(":runners:flink:${CommonTestProperties.getFlinkVersion()}:testJar") - switches("-Dorg.gradle.java.home=${JAVA_8_HOME}") - } - - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(":runners:flink:${CommonTestProperties.getFlinkVersion()}:validatesRunner") - switches('-x shadowJar') - switches('-x shadowTestJar') - switches('-x compileJava') - switches('-x compileTestJava') - switches('-x jar') - switches('-x testJar') - switches('-x classes') - switches('-x testClasses') - switches("-Dorg.gradle.java.home=${JAVA_11_HOME}") - - commonJobProperties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Python.groovy b/.test-infra/jenkins/job_PostCommit_Python.groovy deleted file mode 100644 index 6e8f5e81b07d..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Python.groovy +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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 CommonJobProperties as commonJobProperties -import PostcommitJobBuilder - -import static PythonTestProperties.ALL_SUPPORTED_VERSIONS - -// This job defines the Python postcommit tests. -ALL_SUPPORTED_VERSIONS.each { pythonVersion -> - def versionSuffix = pythonVersion.replace('.', '') - PostcommitJobBuilder.postCommitJob("beam_PostCommit_Python${versionSuffix}", - "Run Python ${pythonVersion} PostCommit", - "Python${versionSuffix}_PC(\"Run Python ${pythonVersion} PostCommit\")", this) { - description("Runs Python postcommit tests using Python ${pythonVersion}.") - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - - publishers { - archiveJunit('**/pytest*.xml') - } - - // Execute shell command to test Python SDK. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(":python${versionSuffix}PostCommit") - commonJobProperties.setGradleSwitches(delegate) - switches("-PuseWheelDistribution") - } - } - } -} - diff --git a/.test-infra/jenkins/job_PostCommit_Python_CrossLanguage_IO_Dataflow.groovy b/.test-infra/jenkins/job_PostCommit_Python_CrossLanguage_IO_Dataflow.groovy deleted file mode 100644 index 9a37e76cbde8..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Python_CrossLanguage_IO_Dataflow.groovy +++ /dev/null @@ -1,55 +0,0 @@ -/* - * 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 CommonJobProperties as commonJobProperties -import PostcommitJobBuilder - - -import static PythonTestProperties.CROSS_LANGUAGE_VALIDATES_RUNNER_PYTHON_VERSIONS - - -// This job runs end-to-end cross language GCP IO tests with DataflowRunner. -// Collects tests with the @pytest.mark.uses_io_java_expansion_service decorator -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Python_Xlang_IO_Dataflow', - 'Run Python_Xlang_IO_Dataflow PostCommit', 'Python_Xlang_IO_Dataflow (\"Run Python_Xlang_IO_Dataflow PostCommit\")', this) { - description('Runs end-to-end cross language non-GCP IO tests on the Dataflow runner.') - - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 180) - - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/pytest*.xml') - } - - - // Gradle goals for this job. - - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(":sdks:python:test-suites:dataflow:ioCrossLanguagePostCommit") - commonJobProperties.setGradleSwitches(delegate) - switches("-PuseWheelDistribution") - switches("-PkafkaBootstrapServer=10.128.0.40:9094,10.128.0.28:9094,10.128.0.165:9094") - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Python_Examples_Direct.groovy b/.test-infra/jenkins/job_PostCommit_Python_Examples_Direct.groovy deleted file mode 100644 index e420b046271d..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Python_Examples_Direct.groovy +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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 CommonJobProperties as commonJobProperties -import PostcommitJobBuilder - -// This job runs the Python examples tests with DirectRunner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Python_Examples_Direct', - 'Run Python Examples_Direct', 'Python Direct Runner Examples', this) { - - description('Runs the Python Examples with DirectRunner') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 120) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/pytest*.xml') - } - - // Execute shell command to run examples. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(":sdks:python:test-suites:direct:examplesPostCommit") - commonJobProperties.setGradleSwitches(delegate) - } - } - } \ No newline at end of file diff --git a/.test-infra/jenkins/job_PostCommit_Python_Examples_Flink.groovy b/.test-infra/jenkins/job_PostCommit_Python_Examples_Flink.groovy deleted file mode 100644 index e9150ccae011..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Python_Examples_Flink.groovy +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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 CommonJobProperties as commonJobProperties -import PostcommitJobBuilder - -// This job runs the Python examples tests with FlinkRunner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Python_Examples_Flink', - 'Run Python Examples_Flink', 'Python Flink Runner Examples', this) { - - description('Runs the Python Examples with Flink Runner') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 120, true, 'beam', true, 40) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/pytest*.xml') - } - - // Execute shell command to run examples. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(":sdks:python:test-suites:portable:flinkExamplesPostCommit") - commonJobProperties.setGradleSwitches(delegate) - } - } - } \ No newline at end of file diff --git a/.test-infra/jenkins/job_PostCommit_Python_ValidatesRunner_Flink.groovy b/.test-infra/jenkins/job_PostCommit_Python_ValidatesRunner_Flink.groovy deleted file mode 100644 index 5f0aa8f13b9d..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Python_ValidatesRunner_Flink.groovy +++ /dev/null @@ -1,38 +0,0 @@ -/* - * 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 CommonJobProperties as commonJobProperties -import PostcommitJobBuilder - -// This job runs the suite of Python ValidatesRunner tests against the Flink runner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Python_VR_Flink', - 'Run Python Flink ValidatesRunner', 'Run Python Flink ValidatesRunner', this) { - description('Runs the Python ValidatesRunner suite on the Flink runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:python:test-suites:portable:flinkValidatesRunner') - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Python_ValidatesRunner_Spark.groovy b/.test-infra/jenkins/job_PostCommit_Python_ValidatesRunner_Spark.groovy deleted file mode 100644 index ce71871da7eb..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Python_ValidatesRunner_Spark.groovy +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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 CommonJobProperties as commonJobProperties -import PostcommitJobBuilder - -// This job runs the suite of Python ValidatesRunner tests against the Spark runner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Python_VR_Spark', - 'Run Python Spark ValidatesRunner', 'Python Spark ValidatesRunner Tests', this) { - description('Runs the Python ValidatesRunner suite on the Spark runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Publish all test results to Jenkins. - publishers { - archiveJunit('**/pytest*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':pythonSparkPostCommit') - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/.test-infra/jenkins/job_PostRelease_NightlySnapshot.groovy b/.test-infra/jenkins/job_PostRelease_NightlySnapshot.groovy deleted file mode 100644 index 45290152f830..000000000000 --- a/.test-infra/jenkins/job_PostRelease_NightlySnapshot.groovy +++ /dev/null @@ -1,59 +0,0 @@ -/* - * 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 CommonJobProperties as commonJobProperties - -// This verifies the nightly snapshot build. -// From https://repository.apache.org/content/groups/snapshots/org/apache/beam. -job('beam_PostRelease_NightlySnapshot') { - description('Runs post release verification of the nightly snapshot.') - - // Set common parameters. Timeout is longer, to avoid [BEAM-5774]. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 200) - - parameters { - stringParam('snapshot_version', - '', - 'Version of the repository snapshot to install') - stringParam('snapshot_url', - '', - 'Repository URL to install from') - } - - // This is a post-commit job that runs once per day, not for every push. - commonJobProperties.setAutoJob( - delegate, - '@daily') - - - // Allows triggering this build against pull requests. - commonJobProperties.enablePhraseTriggeringFromPullRequest( - delegate, - './gradlew :release:runJavaExamplesValidationTask', - 'Run Dataflow PostRelease') - - steps { - // Run a quickstart from https://beam.apache.org/get-started/quickstart-java - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':release:runJavaExamplesValidationTask') - commonJobProperties.setGradleSwitches(delegate) - switches('-Pver=$snapshot_version -Prepourl=$snapshot_url') - } - } -} diff --git a/.test-infra/jenkins/job_seed_standalone.groovy b/.test-infra/jenkins/job_seed_standalone.groovy deleted file mode 100644 index 6f88448da4ce..000000000000 --- a/.test-infra/jenkins/job_seed_standalone.groovy +++ /dev/null @@ -1,115 +0,0 @@ -/* - * 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. - */ - -// Defines the seed job, which creates or updates all other Jenkins projects. - -import Committers as committers - -job('beam_SeedJob_Standalone') { - description('Automatically configures all Apache Beam Jenkins projects based' + - ' on Jenkins DSL groovy files checked into the code repository.') - - properties { - githubProjectUrl('https://github.com/apache/beam/') - } - - // Restrict to only run on Jenkins executors labeled 'beam' - label('beam') - - logRotator { - daysToKeep(30) - } - - scm { - git { - remote { - github('apache/beam') - - // ${ghprbPullId} is not interpolated by groovy, but passed through to Jenkins where it - // refers to the environment variable - refspec([ - '+refs/heads/*:refs/remotes/origin/*', - '+refs/pull/${ghprbPullId}/*:refs/remotes/origin/pr/${ghprbPullId}/*'] - .join(' ')) - - // The variable ${sha1} is not interpolated by groovy, but a parameter of the Jenkins job - branch('${sha1}') - - extensions { - cleanAfterCheckout() - } - } - } - } - - parameters { - // Setup for running this job from a pull request - stringParam( - 'sha1', - 'master', - 'Commit id or refname (eg: origin/pr/4001/head) you want to build against.') - } - - wrappers { - timeout { - absolute(60) - abortBuild() - } - } - - triggers { - // Run every six hours, alternate job_00_seed has same freq - cron('H 3,9,15,21 * * *') - - githubPullRequest { - admins(['asfbot']) - useGitHubHooks() - userWhitelist(committers.GITHUB_USERNAMES) - - // Also run when manually kicked on a pull request - triggerPhrase('Run Standalone Seed Job') - onlyTriggerPhrase() - - extensions { - commitStatus { - context("Jenkins: Standalone Seed Job") - } - - buildStatus { - completedStatus('SUCCESS', '--none--') - completedStatus('FAILURE', '--none--') - completedStatus('ERROR', '--none--') - } - } - } - } - - publishers { - mailer('builds@beam.apache.org', false, true) - } - - steps { - dsl { - // A list or a glob of other groovy files to process. - external('.test-infra/jenkins/job_*.groovy') - - // If a job is removed from the script, disable it (rather than deleting). - removeAction('DISABLE') - } - } -} From 772cf0a5c01ca64de9392b17f1acaa7caf7a3498 Mon Sep 17 00:00:00 2001 From: Svetak Sundhar Date: Thu, 25 Jan 2024 18:38:39 +0000 Subject: [PATCH 072/169] Don't run DicomIO Integration Tests on GCP DF Runner (#30103) * [BEAM-12550] Implement Parallelizable Skew and Kurtosis (Skew implementation) R: @TheNeuralBit * dont run on Dataflow Java V2 * spotless * run PostCommit * delete postcommit trigger --------- Co-authored-by: svetakvsundhar --- runners/google-cloud-dataflow-java/build.gradle | 2 ++ .../org/apache/beam/sdk/io/gcp/healthcare/DicomIOReadIT.java | 3 --- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index f34db247a0f6..8a89c489ceb9 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -566,6 +566,7 @@ task googleCloudPlatformLegacyWorkerIntegrationTest(type: Test, dependsOn: copyG exclude '**/BigQueryIOStorageReadTableRowIT.class' exclude '**/PubsubReadIT.class' exclude '**/FhirIOReadIT.class' + exclude '**/DicomIOReadIT.class' exclude '**/gcp/spanner/changestreams/it/*.class' maxParallelForks 4 classpath = configurations.googleCloudPlatformIntegrationTest @@ -610,6 +611,7 @@ task googleCloudPlatformRunnerV2IntegrationTest(type: Test) { exclude '**/SpannerWriteIT.class' exclude '**/*KmsKeyIT.class' exclude '**/FhirIOReadIT.class' + exclude '**/DicomIOReadIT.class' exclude '**/FhirIOWriteIT.class' exclude '**/FhirIOLROIT.class' exclude '**/FhirIOSearchIT.class' diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/DicomIOReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/DicomIOReadIT.java index f6f8e8b8fcc8..570844011f39 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/DicomIOReadIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/DicomIOReadIT.java @@ -27,7 +27,6 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; -import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; @@ -62,7 +61,6 @@ public void deleteDicomStore() throws IOException { client.deleteDicomStore(healthcareDataset + "/dicomStores/" + storeName); } - @Ignore("https://github.com/apache/beam/issues/28099") @Test public void testDicomMetadataRead() { String webPath = @@ -90,7 +88,6 @@ public void testDicomMetadataRead() { pipeline.run(); } - @Ignore("https://github.com/apache/beam/issues/28099") @Test public void testDicomFailedMetadataRead() { String badWebPath = "foo"; From 3ae851876e22be8ed09c4434c73dd654874e7ae4 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> Date: Thu, 25 Jan 2024 14:03:37 -0500 Subject: [PATCH 073/169] Fix failing python BQ test (#30099) --- .github/trigger_files/beam_PostCommit_Python.json | 0 sdks/python/apache_beam/io/gcp/bigquery.py | 5 +++++ .../apache_beam/io/gcp/bigquery_file_loads.py | 1 + .../io/gcp/bigquery_file_loads_test.py | 15 +++++++-------- sdks/python/test-suites/dataflow/common.gradle | 2 +- 5 files changed, 14 insertions(+), 9 deletions(-) create mode 100644 .github/trigger_files/beam_PostCommit_Python.json diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/sdks/python/apache_beam/io/gcp/bigquery.py b/sdks/python/apache_beam/io/gcp/bigquery.py index 4643c8ddf0a5..bba8b8a4af75 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery.py +++ b/sdks/python/apache_beam/io/gcp/bigquery.py @@ -1854,6 +1854,7 @@ def __init__( kms_key=None, batch_size=None, max_file_size=None, + max_partition_size=None, max_files_per_bundle=None, test_client=None, custom_gcs_temp_location=None, @@ -1934,6 +1935,8 @@ def __init__( max_file_size (int): The maximum size for a file to be written and then loaded into BigQuery. The default value is 4TB, which is 80% of the limit of 5TB for BigQuery to load any file. + max_partition_size (int): Maximum byte size for each load job to + BigQuery. Defaults to 15TB. Applicable to FILE_LOADS only. max_files_per_bundle(int): The maximum number of files to be concurrently written by a worker. The default here is 20. Larger values will allow writing to multiple destinations without having to reshard - but they @@ -2059,6 +2062,7 @@ def __init__( # TODO(pabloem): Consider handling ValueProvider for this location. self.custom_gcs_temp_location = custom_gcs_temp_location self.max_file_size = max_file_size + self.max_partition_size = max_partition_size self.max_files_per_bundle = max_files_per_bundle self.method = method or WriteToBigQuery.Method.DEFAULT self.triggering_frequency = triggering_frequency @@ -2202,6 +2206,7 @@ def find_in_nested_dict(schema): with_auto_sharding=self.with_auto_sharding, temp_file_format=self._temp_file_format, max_file_size=self.max_file_size, + max_partition_size=self.max_partition_size, max_files_per_bundle=self.max_files_per_bundle, custom_gcs_temp_location=self.custom_gcs_temp_location, test_client=self.test_client, diff --git a/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py b/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py index 48f2ab4b36bf..e1a4af31f1c2 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py @@ -747,6 +747,7 @@ def process( ) if not self.bq_io_metadata: self.bq_io_metadata = create_bigquery_io_metadata(self._step_name) + job_reference = self.bq_wrapper.perform_load_job( destination=table_reference, source_uris=files, diff --git a/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py b/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py index 345c8e705007..0605206714ed 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py @@ -925,9 +925,6 @@ def test_batch_copy_jobs_with_no_input_schema(self): create_disposition='CREATE_IF_NEEDED', write_disposition='WRITE_APPEND') - # reduce load job size to induce copy jobs - bqfl._DEFAULT_MAX_FILE_SIZE = 10 - bqfl._MAXIMUM_LOAD_SIZE = 20 verifiers = [ BigqueryFullResultMatcher( project=self.project, @@ -949,8 +946,7 @@ def callable_table(el: dict): dest += "_2" return dest - args = self.test_pipeline.get_full_options_as_args( - on_success_matcher=all_of(verifiers)) + args = self.test_pipeline.get_full_options_as_args() with beam.Pipeline(argv=args) as p: # 0...4 going to table 1 @@ -961,7 +957,10 @@ def callable_table(el: dict): p | beam.Create(items) | bigquery.WriteToBigQuery( table=callable_table, create_disposition="CREATE_NEVER", - write_disposition="WRITE_APPEND")) + write_disposition="WRITE_APPEND", + # reduce load job size to induce copy jobs + max_file_size=10, + max_partition_size=20)) hamcrest_assert(p, all_of(*verifiers)) @@ -1001,8 +1000,7 @@ def test_multiple_destinations_transform(self): if 'foundation' in d]) ] - args = self.test_pipeline.get_full_options_as_args( - on_success_matcher=all_of(*pipeline_verifiers)) + args = self.test_pipeline.get_full_options_as_args() with beam.Pipeline(argv=args) as p: input = p | beam.Create(_ELEMENTS, reshuffle=False) @@ -1044,6 +1042,7 @@ def test_multiple_destinations_transform(self): write_disposition=beam.io.BigQueryDisposition.WRITE_EMPTY, max_file_size=20, max_files_per_bundle=-1)) + hamcrest_assert(p, all_of(*pipeline_verifiers)) @pytest.mark.it_postcommit def test_bqfl_streaming(self): diff --git a/sdks/python/test-suites/dataflow/common.gradle b/sdks/python/test-suites/dataflow/common.gradle index a713b82400e7..5fc1751a9686 100644 --- a/sdks/python/test-suites/dataflow/common.gradle +++ b/sdks/python/test-suites/dataflow/common.gradle @@ -133,7 +133,7 @@ task postCommitIT { "test_opts": testOpts, "sdk_location": project.ext.sdkLocation, "suite": "postCommitIT-df${pythonVersionSuffix}", - "collect": "it_postcommit" + "collect": "it_postcommit", ] def cmdArgs = mapToArgString(argMap) exec { From 6b6fc98ec93cfabc9cdbb088a38adbb650ee00d3 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Thu, 25 Jan 2024 16:43:46 -0800 Subject: [PATCH 074/169] Don't depend on the go 1.21 standard lib yet (#30120) Co-authored-by: lostluck <13907733+lostluck@users.noreply.github.com> --- sdks/go/pkg/beam/core/runtime/contextreg/contextreg.go | 3 ++- sdks/go/pkg/beam/core/runtime/harness/init/init.go | 4 ++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/sdks/go/pkg/beam/core/runtime/contextreg/contextreg.go b/sdks/go/pkg/beam/core/runtime/contextreg/contextreg.go index d91141477576..c5b0f1af7159 100644 --- a/sdks/go/pkg/beam/core/runtime/contextreg/contextreg.go +++ b/sdks/go/pkg/beam/core/runtime/contextreg/contextreg.go @@ -22,8 +22,9 @@ package contextreg import ( "context" - "maps" "sync" + + "golang.org/x/exp/maps" ) var defaultReg = &Registry{} diff --git a/sdks/go/pkg/beam/core/runtime/harness/init/init.go b/sdks/go/pkg/beam/core/runtime/harness/init/init.go index 27d44f7029e5..468708b2917b 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/init/init.go +++ b/sdks/go/pkg/beam/core/runtime/harness/init/init.go @@ -22,15 +22,15 @@ import ( "context" "encoding/json" "flag" - "slices" "strings" "time" "fmt" "os" - "runtime/debug" + "golang.org/x/exp/slices" + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime/graphx" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime/harness" From 445151694fef810d5112ea7e034659f91fceceac Mon Sep 17 00:00:00 2001 From: liferoad Date: Fri, 26 Jan 2024 10:45:11 -0500 Subject: [PATCH 075/169] Check the issue number type (#30127) --- sdks/python/apache_beam/testing/analyzers/perf_analysis.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/testing/analyzers/perf_analysis.py b/sdks/python/apache_beam/testing/analyzers/perf_analysis.py index 5802fe041454..a12b06c8c3eb 100644 --- a/sdks/python/apache_beam/testing/analyzers/perf_analysis.py +++ b/sdks/python/apache_beam/testing/analyzers/perf_analysis.py @@ -173,8 +173,10 @@ def run_change_point_analysis( constants._CHANGE_POINT_TIMESTAMP_LABEL].tolist() last_reported_issue_number = existing_issue_data[ constants._ISSUE_NUMBER].tolist()[0] - # convert numpy.int64 to int - last_reported_issue_number = last_reported_issue_number.item() + + if not isinstance(last_reported_issue_number, int): + # convert numpy.int64 to int + last_reported_issue_number = last_reported_issue_number.item() is_valid_change_point = is_sibling_change_point( previous_change_point_timestamps=existing_issue_timestamps, From 3fd7bf29dcd421979a0ce1d6baa7df724e625186 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Fri, 26 Jan 2024 11:27:46 -0500 Subject: [PATCH 076/169] Bump Dataflow dev container version (#30124) --- runners/google-cloud-dataflow-java/build.gradle | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index 8a89c489ceb9..0f1af6aab269 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -51,8 +51,8 @@ evaluationDependsOn(":sdks:java:container:java11") ext.dataflowLegacyEnvironmentMajorVersion = '8' ext.dataflowFnapiEnvironmentMajorVersion = '8' -ext.dataflowLegacyContainerVersion = 'beam-master-20231212' -ext.dataflowFnapiContainerVersion = 'beam-master-20240109' +ext.dataflowLegacyContainerVersion = 'beam-master-20240125' +ext.dataflowFnapiContainerVersion = 'beam-master-20240125' ext.dataflowContainerBaseRepository = 'gcr.io/cloud-dataflow/v1beta3' processResources { From 1ab504c34a6385304059fa0822db3a701b2cd168 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Fri, 26 Jan 2024 11:37:12 -0500 Subject: [PATCH 077/169] Update update_python_dependencies.yml with all python versions (#30126) * Update update_python_dependencies.yml with all python versions * Give write permissions * Add token to cli step * Syntax --- .github/workflows/update_python_dependencies.yml | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) diff --git a/.github/workflows/update_python_dependencies.yml b/.github/workflows/update_python_dependencies.yml index b4b839c3204c..014ead8d6739 100644 --- a/.github/workflows/update_python_dependencies.yml +++ b/.github/workflows/update_python_dependencies.yml @@ -24,7 +24,14 @@ on: schedule: - cron: "0 0 * * 0" workflow_dispatch: -permissions: read-all +permissions: + contents: write + pull-requests: write + checks: read + issues: read + statuses: read + + env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} @@ -55,6 +62,8 @@ jobs: python-version: | 3.8 3.9 + 3.10 + 3.11 java-version: 8 go-version: 1.21 - name: Update Python Dependencies @@ -68,6 +77,8 @@ jobs: git config user.name $GITHUB_ACTOR git config user.email actions@"$RUNNER_NAME".local - name: Commit Changes and create new PR + env: + GH_TOKEN: ${{ secrets.GITHUB_TOKEN }} run: | branchName=weekly_update_python_dependencies_$(date +%s) git checkout -b $branchName @@ -81,4 +92,4 @@ jobs: echo "Link of the new PR $GITHUB_PR_URL" else echo "No changes on the files" - fi \ No newline at end of file + fi From 4981c0c3abc26ec2e5bfce94c612411462b43723 Mon Sep 17 00:00:00 2001 From: tvalentyn Date: Fri, 26 Jan 2024 10:10:13 -0800 Subject: [PATCH 078/169] Upgrade numpy dependency (#29645) * Upgrade numpy dependency * Regenerate dependencies. --- .../py310/base_image_requirements.txt | 39 ++++++++++--------- .../py311/base_image_requirements.txt | 39 ++++++++++--------- .../py38/base_image_requirements.txt | 33 ++++++++-------- .../py39/base_image_requirements.txt | 39 ++++++++++--------- sdks/python/pyproject.toml | 4 +- sdks/python/setup.py | 2 +- 6 files changed, 80 insertions(+), 76 deletions(-) diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index a58d61d703c6..ebc96e17b122 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -22,8 +22,8 @@ # Reach out to a committer if you need help. attrs==23.2.0 -beautifulsoup4==4.12.2 -bs4==0.0.1 +beautifulsoup4==4.12.3 +bs4==0.0.2 build==1.0.3 cachetools==5.3.2 certifi==2023.11.17 @@ -32,11 +32,12 @@ charset-normalizer==3.3.2 click==8.1.7 cloudpickle==2.2.1 crcmod==1.7 -cryptography==41.0.7 +cryptography==42.0.1 Cython==0.29.37 +Deprecated==1.2.14 deprecation==2.1.0 dill==0.3.1.1 -dnspython==2.4.2 +dnspython==2.5.0 docker==7.0.0 docopt==0.6.2 docstring-parser==0.15 @@ -47,12 +48,12 @@ fasteners==0.19 freezegun==1.4.0 future==0.18.3 google-api-core==2.15.0 -google-api-python-client==2.113.0 +google-api-python-client==2.115.0 google-apitools==0.5.31 -google-auth==2.26.1 +google-auth==2.27.0 google-auth-httplib2==0.1.1 -google-cloud-aiplatform==1.39.0 -google-cloud-bigquery==3.15.0 +google-cloud-aiplatform==1.40.0 +google-cloud-bigquery==3.17.1 google-cloud-bigquery-storage==2.24.0 google-cloud-bigtable==2.22.0 google-cloud-core==2.4.1 @@ -64,7 +65,7 @@ google-cloud-pubsub==2.19.0 google-cloud-pubsublite==1.9.0 google-cloud-recommendations-ai==0.10.6 google-cloud-resource-manager==1.11.0 -google-cloud-spanner==3.40.1 +google-cloud-spanner==3.41.0 google-cloud-storage==2.14.0 google-cloud-videointelligence==2.12.0 google-cloud-vision==3.5.0 @@ -78,29 +79,29 @@ grpcio-status==1.60.0 guppy3==3.1.4.post1 hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.92.7 +hypothesis==6.97.0 idna==3.6 iniconfig==2.0.0 joblib==1.3.2 Js2Py==0.74 jsonpickle==3.0.2 -jsonschema==4.20.0 +jsonschema==4.21.1 jsonschema-specifications==2023.12.1 mmh3==4.1.0 mock==5.1.0 nltk==3.8.1 nose==1.3.7 -numpy==1.24.4 +numpy==1.26.3 oauth2client==4.1.3 objsize==0.7.0 -orjson==3.9.10 -overrides==7.4.0 +orjson==3.9.12 +overrides==7.6.0 packaging==23.2 pandas==2.0.3 parameterized==0.9.0 -pluggy==1.3.0 +pluggy==1.4.0 proto-plus==1.23.0 -protobuf==4.25.1 +protobuf==4.25.2 psycopg2-binary==2.9.9 pyarrow==14.0.2 pyarrow-hotfix==0.6 @@ -125,10 +126,10 @@ referencing==0.32.1 regex==2023.12.25 requests==2.31.0 requests-mock==1.11.0 -rpds-py==0.16.2 +rpds-py==0.17.1 rsa==4.9 -scikit-learn==1.3.2 -scipy==1.11.4 +scikit-learn==1.4.0 +scipy==1.12.0 shapely==2.0.2 six==1.16.0 sortedcontainers==2.4.0 diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt index e4c3552d71e1..37c1a30f673c 100644 --- a/sdks/python/container/py311/base_image_requirements.txt +++ b/sdks/python/container/py311/base_image_requirements.txt @@ -22,8 +22,8 @@ # Reach out to a committer if you need help. attrs==23.2.0 -beautifulsoup4==4.12.2 -bs4==0.0.1 +beautifulsoup4==4.12.3 +bs4==0.0.2 build==1.0.3 cachetools==5.3.2 certifi==2023.11.17 @@ -32,11 +32,12 @@ charset-normalizer==3.3.2 click==8.1.7 cloudpickle==2.2.1 crcmod==1.7 -cryptography==41.0.7 +cryptography==42.0.1 Cython==0.29.37 +Deprecated==1.2.14 deprecation==2.1.0 dill==0.3.1.1 -dnspython==2.4.2 +dnspython==2.5.0 docker==7.0.0 docopt==0.6.2 docstring-parser==0.15 @@ -46,12 +47,12 @@ fasteners==0.19 freezegun==1.4.0 future==0.18.3 google-api-core==2.15.0 -google-api-python-client==2.113.0 +google-api-python-client==2.115.0 google-apitools==0.5.31 -google-auth==2.26.1 +google-auth==2.27.0 google-auth-httplib2==0.1.1 -google-cloud-aiplatform==1.39.0 -google-cloud-bigquery==3.15.0 +google-cloud-aiplatform==1.40.0 +google-cloud-bigquery==3.17.1 google-cloud-bigquery-storage==2.24.0 google-cloud-bigtable==2.22.0 google-cloud-core==2.4.1 @@ -63,7 +64,7 @@ google-cloud-pubsub==2.19.0 google-cloud-pubsublite==1.9.0 google-cloud-recommendations-ai==0.10.6 google-cloud-resource-manager==1.11.0 -google-cloud-spanner==3.40.1 +google-cloud-spanner==3.41.0 google-cloud-storage==2.14.0 google-cloud-videointelligence==2.12.0 google-cloud-vision==3.5.0 @@ -77,29 +78,29 @@ grpcio-status==1.60.0 guppy3==3.1.4.post1 hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.92.7 +hypothesis==6.97.0 idna==3.6 iniconfig==2.0.0 joblib==1.3.2 Js2Py==0.74 jsonpickle==3.0.2 -jsonschema==4.20.0 +jsonschema==4.21.1 jsonschema-specifications==2023.12.1 mmh3==4.1.0 mock==5.1.0 nltk==3.8.1 nose==1.3.7 -numpy==1.24.4 +numpy==1.26.3 oauth2client==4.1.3 objsize==0.7.0 -orjson==3.9.10 -overrides==7.4.0 +orjson==3.9.12 +overrides==7.6.0 packaging==23.2 pandas==2.0.3 parameterized==0.9.0 -pluggy==1.3.0 +pluggy==1.4.0 proto-plus==1.23.0 -protobuf==4.25.1 +protobuf==4.25.2 psycopg2-binary==2.9.9 pyarrow==14.0.2 pyarrow-hotfix==0.6 @@ -123,10 +124,10 @@ referencing==0.32.1 regex==2023.12.25 requests==2.31.0 requests-mock==1.11.0 -rpds-py==0.16.2 +rpds-py==0.17.1 rsa==4.9 -scikit-learn==1.3.2 -scipy==1.11.4 +scikit-learn==1.4.0 +scipy==1.12.0 shapely==2.0.2 six==1.16.0 sortedcontainers==2.4.0 diff --git a/sdks/python/container/py38/base_image_requirements.txt b/sdks/python/container/py38/base_image_requirements.txt index 76c67798e27c..eeb814f8a99c 100644 --- a/sdks/python/container/py38/base_image_requirements.txt +++ b/sdks/python/container/py38/base_image_requirements.txt @@ -23,8 +23,8 @@ attrs==23.2.0 backports.zoneinfo==0.2.1 -beautifulsoup4==4.12.2 -bs4==0.0.1 +beautifulsoup4==4.12.3 +bs4==0.0.2 build==1.0.3 cachetools==5.3.2 certifi==2023.11.17 @@ -33,11 +33,12 @@ charset-normalizer==3.3.2 click==8.1.7 cloudpickle==2.2.1 crcmod==1.7 -cryptography==41.0.7 +cryptography==42.0.1 Cython==0.29.37 +Deprecated==1.2.14 deprecation==2.1.0 dill==0.3.1.1 -dnspython==2.4.2 +dnspython==2.5.0 docker==7.0.0 docopt==0.6.2 docstring-parser==0.15 @@ -48,12 +49,12 @@ fasteners==0.19 freezegun==1.4.0 future==0.18.3 google-api-core==2.15.0 -google-api-python-client==2.113.0 +google-api-python-client==2.115.0 google-apitools==0.5.31 -google-auth==2.26.1 +google-auth==2.27.0 google-auth-httplib2==0.1.1 -google-cloud-aiplatform==1.39.0 -google-cloud-bigquery==3.15.0 +google-cloud-aiplatform==1.40.0 +google-cloud-bigquery==3.17.1 google-cloud-bigquery-storage==2.24.0 google-cloud-bigtable==2.22.0 google-cloud-core==2.4.1 @@ -65,7 +66,7 @@ google-cloud-pubsub==2.19.0 google-cloud-pubsublite==1.9.0 google-cloud-recommendations-ai==0.10.6 google-cloud-resource-manager==1.11.0 -google-cloud-spanner==3.40.1 +google-cloud-spanner==3.41.0 google-cloud-storage==2.14.0 google-cloud-videointelligence==2.12.0 google-cloud-vision==3.5.0 @@ -79,7 +80,7 @@ grpcio-status==1.60.0 guppy3==3.1.4.post1 hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.92.7 +hypothesis==6.97.0 idna==3.6 importlib-metadata==7.0.1 importlib-resources==6.1.1 @@ -87,7 +88,7 @@ iniconfig==2.0.0 joblib==1.3.2 Js2Py==0.74 jsonpickle==3.0.2 -jsonschema==4.20.0 +jsonschema==4.21.1 jsonschema-specifications==2023.12.1 mmh3==4.1.0 mock==5.1.0 @@ -96,15 +97,15 @@ nose==1.3.7 numpy==1.24.4 oauth2client==4.1.3 objsize==0.7.0 -orjson==3.9.10 -overrides==7.4.0 +orjson==3.9.12 +overrides==7.6.0 packaging==23.2 pandas==2.0.3 parameterized==0.9.0 pkgutil_resolve_name==1.3.10 -pluggy==1.3.0 +pluggy==1.4.0 proto-plus==1.23.0 -protobuf==4.25.1 +protobuf==4.25.2 psycopg2-binary==2.9.9 pyarrow==14.0.2 pyarrow-hotfix==0.6 @@ -129,7 +130,7 @@ referencing==0.32.1 regex==2023.12.25 requests==2.31.0 requests-mock==1.11.0 -rpds-py==0.16.2 +rpds-py==0.17.1 rsa==4.9 scikit-learn==1.3.2 scipy==1.10.1 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index fd1feee62b67..acf6a106c0dc 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -22,8 +22,8 @@ # Reach out to a committer if you need help. attrs==23.2.0 -beautifulsoup4==4.12.2 -bs4==0.0.1 +beautifulsoup4==4.12.3 +bs4==0.0.2 build==1.0.3 cachetools==5.3.2 certifi==2023.11.17 @@ -32,11 +32,12 @@ charset-normalizer==3.3.2 click==8.1.7 cloudpickle==2.2.1 crcmod==1.7 -cryptography==41.0.7 +cryptography==42.0.1 Cython==0.29.37 +Deprecated==1.2.14 deprecation==2.1.0 dill==0.3.1.1 -dnspython==2.4.2 +dnspython==2.5.0 docker==7.0.0 docopt==0.6.2 docstring-parser==0.15 @@ -47,12 +48,12 @@ fasteners==0.19 freezegun==1.4.0 future==0.18.3 google-api-core==2.15.0 -google-api-python-client==2.113.0 +google-api-python-client==2.115.0 google-apitools==0.5.31 -google-auth==2.26.1 +google-auth==2.27.0 google-auth-httplib2==0.1.1 -google-cloud-aiplatform==1.39.0 -google-cloud-bigquery==3.15.0 +google-cloud-aiplatform==1.40.0 +google-cloud-bigquery==3.17.1 google-cloud-bigquery-storage==2.24.0 google-cloud-bigtable==2.22.0 google-cloud-core==2.4.1 @@ -64,7 +65,7 @@ google-cloud-pubsub==2.19.0 google-cloud-pubsublite==1.9.0 google-cloud-recommendations-ai==0.10.6 google-cloud-resource-manager==1.11.0 -google-cloud-spanner==3.40.1 +google-cloud-spanner==3.41.0 google-cloud-storage==2.14.0 google-cloud-videointelligence==2.12.0 google-cloud-vision==3.5.0 @@ -78,30 +79,30 @@ grpcio-status==1.60.0 guppy3==3.1.4.post1 hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.92.7 +hypothesis==6.97.0 idna==3.6 importlib-metadata==7.0.1 iniconfig==2.0.0 joblib==1.3.2 Js2Py==0.74 jsonpickle==3.0.2 -jsonschema==4.20.0 +jsonschema==4.21.1 jsonschema-specifications==2023.12.1 mmh3==4.1.0 mock==5.1.0 nltk==3.8.1 nose==1.3.7 -numpy==1.24.4 +numpy==1.26.3 oauth2client==4.1.3 objsize==0.7.0 -orjson==3.9.10 -overrides==7.4.0 +orjson==3.9.12 +overrides==7.6.0 packaging==23.2 pandas==2.0.3 parameterized==0.9.0 -pluggy==1.3.0 +pluggy==1.4.0 proto-plus==1.23.0 -protobuf==4.25.1 +protobuf==4.25.2 psycopg2-binary==2.9.9 pyarrow==14.0.2 pyarrow-hotfix==0.6 @@ -126,10 +127,10 @@ referencing==0.32.1 regex==2023.12.25 requests==2.31.0 requests-mock==1.11.0 -rpds-py==0.16.2 +rpds-py==0.17.1 rsa==4.9 -scikit-learn==1.3.2 -scipy==1.11.4 +scikit-learn==1.4.0 +scipy==1.12.0 shapely==2.0.2 six==1.16.0 sortedcontainers==2.4.0 diff --git a/sdks/python/pyproject.toml b/sdks/python/pyproject.toml index d185c45f6191..f1a65c842d53 100644 --- a/sdks/python/pyproject.toml +++ b/sdks/python/pyproject.toml @@ -26,11 +26,11 @@ requires = [ # Avoid https://github.com/pypa/virtualenv/issues/2006 "distlib==0.3.7", # Numpy headers - "numpy>=1.14.3,<1.25", # Update setup.py as well. + "numpy>=1.14.3,<1.27", # Update setup.py as well. # having cython here will create wheels that are platform dependent. "cython==0.29.36", ] # legacy installation is needed to generate `apache_beam.portability.api` package. -build-backend = "setuptools.build_meta" \ No newline at end of file +build-backend = "setuptools.build_meta" diff --git a/sdks/python/setup.py b/sdks/python/setup.py index bafebc97f053..5e78c491598f 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -301,7 +301,7 @@ def get_portability_package_data(): 'jsonpickle>=3.0.0,<4.0.0', # numpy can have breaking changes in minor versions. # Use a strict upper bound. - 'numpy>=1.14.3,<1.25.0', # Update pyproject.toml as well. + 'numpy>=1.14.3,<1.27.0', # Update pyproject.toml as well. 'objsize>=0.6.1,<0.8.0', 'packaging>=22.0', 'pymongo>=3.8.0,<5.0.0', From 1d4413d8e95361ba974676df05223676f1467c3e Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Fri, 26 Jan 2024 13:26:25 -0500 Subject: [PATCH 079/169] Update project root documentations reflecting Jenkins shutdown (#30130) --- CI.md | 37 ++++++++++++++++++++++++------------- README.md | 3 --- 2 files changed, 24 insertions(+), 16 deletions(-) diff --git a/CI.md b/CI.md index 200e1208e4d7..f5254e49e4b0 100644 --- a/CI.md +++ b/CI.md @@ -23,19 +23,30 @@ Continuous Integration is important component of making Apache Beam robust and stable. -Our execution environment for CI is mainly the Jenkins which is available at -[https://ci-beam.apache.org/](https://ci-beam.apache.org/). See -[.test-infra/jenkins/README](.test-infra/jenkins/README.md) -for trigger phrase, status and link of all Jenkins jobs. See Apache Beam Developer Guide for -[Jenkins Tips](https://cwiki.apache.org/confluence/display/BEAM/Jenkins+Tips). - -An additional execution environment for CI is [GitHub Actions](https://github.com/features/actions). GitHub Actions -(GA) are very well integrated with GitHub code and Workflow and it has evolved fast in 2019/2020 to become -a fully-fledged CI environment, easy to use and develop for, so we decided to use it for building python source -distribution and wheels. +Our execution environment for CI is the [GitHub Actions](https://github.com/features/actions). +See [.github/workflow/README](.github/workflow/README.md) for trigger phrase, +status and link of all GHA jobs. + +GitHub Actions (GHA) are very well integrated with GitHub code and Workflow and +it has evolved fast in 2019/2020 to become a fully-fledged CI environment, easy +to use and develop for, so we decided to use it firstly for a few workflows then +migrated all workflows previously run on Jenkins. + +For this reason there are mainly two types of GHA workflows running +- Self-hosted runner GHAs. These were mifrated from Jenkins with workflow name + prefix (beam_*.yml) as well as new workflows added following the same naming + convention, including PreCommit, PostCommit, LoadTest, PerformanceTest, and + several infrastructure jobs. See [.github/workflow/README](.github/workflow/README.md) + for the complete list. +- GitHub-hosted runner GHAs. Most of these jobs exercises the workflow in different + OS (linux, macOS, Windows). They were added prior to Jenkins migration. + Some Linux jobs later migrated to use self-hosted runner. ## GitHub Actions +This section applies to GitHub-hosted runner GHAs. New workflows unless intended +to run on a matrix of OS should refer to Self-hosted runner GHAs [.github/workflow/README](.github/workflow/README.md). + ### GitHub actions run types The following GA CI Job runs are currently run for Apache Beam, and each of the runs have different @@ -43,7 +54,7 @@ purpose and context. #### Pull request run -Those runs are results of PR from the forks made by contributors. Most builds for Apache Beam fall +Those runs are results of PR from the forks made by contributors. Most builds for Apache Beam fall into this category. They are executed in the context of the "Fork", not main Beam Code Repository which means that they have only "read" permission to all the GitHub resources (container registry, code repository). This is necessary as the code in those PRs (including CI job @@ -57,7 +68,7 @@ the PR is ready to review and merge. Those runs are results of direct pushes done by the committers or as result of merge of a Pull Request by the committers. Those runs execute in the context of the Apache Beam Code Repository and have also write permission for GitHub resources (container registry, code repository). -The main purpose for the run is to check if the code after merge still holds all the assertions - like +The main purpose for the run is to check if the code after merge still holds all the assertions - like whether it still builds, all tests are green. This is needed because some of the conflicting changes from multiple PRs might cause build and test failures @@ -65,7 +76,7 @@ after merge even if they do not fail in isolation. #### Scheduled runs -Those runs are results of (nightly) triggered job - only for `master` branch. The +Those runs are results of (nightly) triggered job - only for `master` branch. The main purpose of the job is to check if there was no impact of external dependency changes on the Apache Beam code (for example transitive dependencies released that fail the build). Another reason for the nightly build is that the builds tags most recent master with `nightly-master`. diff --git a/README.md b/README.md index 0bd0e8b83cc4..0d564024f87d 100644 --- a/README.md +++ b/README.md @@ -30,9 +30,6 @@ [![Build python source distribution and wheels](https://github.com/apache/beam/workflows/Build%20python%20source%20distribution%20and%20wheels/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Build+python+source+distribution+and+wheels%22+branch%3Amaster+event%3Aschedule) [![Python tests](https://github.com/apache/beam/workflows/Python%20tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Python+Tests%22+branch%3Amaster+event%3Aschedule) [![Java tests](https://github.com/apache/beam/workflows/Java%20Tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Java+Tests%22+branch%3Amaster+event%3Aschedule) -[![Go tests (Jenkins)](https://ci-beam.apache.org/job/beam_PreCommit_Go_Cron/lastCompletedBuild/badge/icon?subject=Go%20Tests%28Jenkins%29)](https://ci-beam.apache.org/job/beam_PreCommit_Go_Cron/lastCompletedBuild/) -[![Java tests (Jenkins)](https://ci-beam.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/badge/icon?subject=Java%20Tests%28Jenkins%29)](https://ci-beam.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/) -[![Python tests (Jenkins)](https://ci-beam.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/badge/icon?subject=Python%20Tests%28Jenkins%29)](https://ci-beam.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/) ## Overview From e0e20a10e9925edf36ba3fdd911f04f3ebcc8992 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Fri, 26 Jan 2024 10:59:05 -0800 Subject: [PATCH 080/169] [#28142][Go SDK] Evict closed readers from the cache. (#30119) Co-authored-by: lostluck <13907733+lostluck@users.noreply.github.com> --- .../pkg/beam/core/runtime/harness/datamgr.go | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/sdks/go/pkg/beam/core/runtime/harness/datamgr.go b/sdks/go/pkg/beam/core/runtime/harness/datamgr.go index 0f2de99dd2ad..6385098d013c 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/datamgr.go +++ b/sdks/go/pkg/beam/core/runtime/harness/datamgr.go @@ -425,16 +425,20 @@ func (c *DataChannel) read(ctx context.Context) { continue // we've already closed this cached reader, skip } r.PTransformDone() - if r.Closed() { - // Clean up local bookkeeping. We'll never see another message - // for it again. We have to be careful not to remove the real - // one, because readers may be initialized after we've seen - // the full stream. - delete(cache, id.instID) - } } seenLast = seenLast[:0] // reset for re-use c.mu.Unlock() + // Scan through the cache and check for any closed readers, and evict them from the cache. + // Readers might be closed out of band from the data messages because we received all data + // for all transforms in an instruction before the instruction even begun. However, we can't + // know this until we received the Control instruction which knows how many transforms for which + // we need to receive data. So we check the cache directly every so often and evict closed + // readers. We will never recieve data for these instructions again. + for instID, r := range cache { + if r.Closed() { + delete(cache, instID) + } + } } } } From 8485ab40e12150957b9b0cedee6d1b3b0d3c76ed Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Fri, 26 Jan 2024 14:39:04 -0500 Subject: [PATCH 081/169] Allow JSON type in TableSchema for BigQuery FILE_LOAD (#29923) * Allow JSON type in TableSchema for BigQuery FILE_LOAD * Warn instead of fail validation and link to documentation * Add test for this use case --- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 45 ++++++---- .../sdk/io/gcp/bigquery/BigQueryIOJsonIT.java | 87 ++++++++++++------- 2 files changed, 82 insertions(+), 50 deletions(-) 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 4424b53f83ce..3d38ef0e83ff 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 @@ -605,7 +605,7 @@ public class BigQueryIO { * PCollection} directly to BigQueryIO.Write. */ static final SerializableFunction TABLE_ROW_IDENTITY_FORMATTER = - SerializableFunctions.identity();; + SerializableFunctions.identity(); /** * A formatting function that maps a GenericRecord to itself. This allows sending a {@code @@ -2231,9 +2231,7 @@ public enum Method { * of load jobs allowed per day, so be careful not to set the triggering frequency too * frequent. For more information, see Loading Data from - * Cloud Storage. Note: Load jobs currently do not support BigQuery's - * JSON data type. + * Cloud Storage. */ FILE_LOADS, @@ -3575,11 +3573,25 @@ private WriteResult continueExpandTyped( !getPropagateSuccessfulStorageApiWrites(), "withPropagateSuccessfulStorageApiWrites only supported when using storage api writes."); - // Batch load jobs currently support JSON data insertion only with CSV files + // Batch load handles wrapped json string value differently than the other methods. Raise a + // warning when applies. if (getJsonSchema() != null && getJsonSchema().isAccessible()) { JsonElement schema = JsonParser.parseString(getJsonSchema().get()); - if (!schema.getAsJsonObject().keySet().isEmpty()) { - validateNoJsonTypeInSchema(schema); + if (!schema.getAsJsonObject().keySet().isEmpty() && hasJsonTypeInSchema(schema)) { + if (rowWriterFactory.getOutputType() == OutputType.JsonTableRow) { + LOG.warn( + "Found JSON type in TableSchema for 'FILE_LOADS' write method. \n" + + "Make sure the TableRow value is a parsed JSON to ensure the read as a " + + "JSON type. Otherwise it will read as a raw (escaped) string.\n" + + "See https://cloud.google.com/bigquery/docs/loading-data-cloud-storage-json#limitations " + + "for limitations."); + } else if (rowWriterFactory.getOutputType() == OutputType.AvroGenericRecord) { + LOG.warn( + "Found JSON type in TableSchema for 'FILE_LOADS' write method. \n" + + " check steps in https://cloud.google.com/bigquery/docs/loading-data-cloud-storage-avro#extract_json_data_from_avro_data " + + " to ensure the read as a JSON type. Otherwise it will read as a raw " + + "(escaped) string."); + } } } @@ -3725,28 +3737,27 @@ private WriteResult continueExpandTyped( } } - private void validateNoJsonTypeInSchema(JsonElement schema) { + private boolean hasJsonTypeInSchema(JsonElement schema) { JsonElement fields = schema.getAsJsonObject().get("fields"); if (!fields.isJsonArray() || fields.getAsJsonArray().isEmpty()) { - return; + return false; } JsonArray fieldArray = fields.getAsJsonArray(); for (int i = 0; i < fieldArray.size(); i++) { JsonObject field = fieldArray.get(i).getAsJsonObject(); - checkArgument( - !field.get("type").getAsString().equals("JSON"), - "Found JSON type in TableSchema. JSON data insertion is currently " - + "not supported with 'FILE_LOADS' write method. This is supported with the " - + "other write methods, however. For more information, visit: " - + "https://cloud.google.com/bigquery/docs/reference/standard-sql/" - + "json-data#ingest_json_data"); + if (field.get("type").getAsString().equals("JSON")) { + return true; + } if (field.get("type").getAsString().equals("STRUCT")) { - validateNoJsonTypeInSchema(field); + if (hasJsonTypeInSchema(field)) { + return true; + } } } + return false; } @Override diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOJsonIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOJsonIT.java index 9da050a059cc..04775338576c 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOJsonIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOJsonIT.java @@ -52,6 +52,7 @@ import org.apache.beam.sdk.values.PCollection; 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.commons.lang3.StringEscapeUtils; import org.json.JSONArray; import org.json.JSONObject; import org.junit.BeforeClass; @@ -77,7 +78,7 @@ public class BigQueryIOJsonIT { @Rule public final transient TestPipeline p = TestPipeline.fromOptions(testOptions); - @Rule public final transient TestPipeline pWrite = TestPipeline.create(); + @Rule public final transient TestPipeline pWrite = TestPipeline.fromOptions(testOptions); private BigQueryIOJsonOptions options; @@ -85,6 +86,7 @@ public class BigQueryIOJsonIT { private static final String DATASET_ID = "bq_jsontype_test_nodelete"; private static final String JSON_TABLE_NAME = "json_data"; + @SuppressWarnings("unused") // persistent test fixture, though unused for the moment private static final String JSON_TABLE_DESTINATION = String.format("%s:%s.%s", project, DATASET_ID, JSON_TABLE_NAME); @@ -135,6 +137,9 @@ public class BigQueryIOJsonIT { public static final String STORAGE_WRITE_TEST_TABLE = "storagewrite_test" + System.currentTimeMillis() + "_" + new SecureRandom().nextInt(32); + public static final String FILE_LOAD_TEST_TABLE = + "fileload_test" + System.currentTimeMillis() + "_" + new SecureRandom().nextInt(32); + public static final String STREAMING_TEST_TABLE = "streaming_test" + System.currentTimeMillis() + "_" + new SecureRandom().nextInt(32); @@ -203,9 +208,21 @@ public void processElement(@Element TableRow row, OutputReceiver>, Void> { Map expected; + // Unescape actual string or not. This is to handle (currently) inconsistent behavior of same + // input data for different write methods. + // + // When feed a string to BigQuery JSON field, FILE_LOAD gives escaped string (e.g. "[\"a\"]" ) + // while other write methods (STORAGE_WRITE_API, STREAMING_INSET) convert it to JSON string + // (e.g. ["a"]) + final boolean unescape; public CompareJsonStrings(Map expected) { + this(expected, false); + } + + public CompareJsonStrings(Map expected, boolean unescape) { this.expected = expected; + this.unescape = unescape; } @Override @@ -223,6 +240,14 @@ public Void apply(Iterable> input) throws RuntimeException { key)); } String jsonStringActual = actual.getValue(); + + // TODO(yathu) remove this conversion if FILE_LOAD produces unescaped JSON string + if (unescape && jsonStringActual.length() > 1) { + jsonStringActual = + StringEscapeUtils.unescapeEcmaScript( + jsonStringActual.substring(1, jsonStringActual.length() - 1)); + } + JsonElement jsonActual = JsonParser.parseString(jsonStringActual); String jsonStringExpected = expected.get(key); @@ -240,7 +265,7 @@ public Void apply(Iterable> input) throws RuntimeException { } // Writes with given write method then reads back and validates with original test data. - public void runTestWrite(BigQueryIOJsonOptions options) { + public void runTestWriteRead(BigQueryIOJsonOptions options) { List countries = Arrays.asList("usa", "aus", "special"); List rowsToWrite = new ArrayList<>(); for (Map.Entry> element : JSON_TEST_DATA.entrySet()) { @@ -299,55 +324,37 @@ public void readAndValidateRows(BigQueryIOJsonOptions options) { return; } + final boolean unescape = options.getWriteMethod() == Write.Method.FILE_LOADS; + // Testing countries (straight json) PCollection> countries = jsonRows.apply( "Convert countries to KV JSON Strings", ParDo.of(new CountryToKVJsonString())); - PAssert.that(countries).satisfies(new CompareJsonStrings(getTestData("countries"))); + PAssert.that(countries).satisfies(new CompareJsonStrings(getTestData("countries"), unescape)); // Testing stats (json in struct) PCollection> stats = jsonRows.apply("Convert stats to KV JSON Strings", ParDo.of(new StatsToKVJsonString())); - PAssert.that(stats).satisfies(new CompareJsonStrings(getTestData("stats"))); + PAssert.that(stats).satisfies(new CompareJsonStrings(getTestData("stats"), unescape)); // Testing cities (json in array of structs) PCollection> cities = jsonRows.apply("Convert cities to KV JSON Strings", ParDo.of(new CitiesToKVJsonString())); - PAssert.that(cities).satisfies(new CompareJsonStrings(getTestData("cities"))); + PAssert.that(cities).satisfies(new CompareJsonStrings(getTestData("cities"), unescape)); // Testing landmarks (json in array) PCollection> landmarks = jsonRows.apply( "Convert landmarks to KV JSON Strings", ParDo.of(new LandmarksToKVJsonString())); - PAssert.that(landmarks).satisfies(new CompareJsonStrings(getTestData("landmarks"))); + PAssert.that(landmarks).satisfies(new CompareJsonStrings(getTestData("landmarks"), unescape)); p.run().waitUntilFinish(); } - @Test - public void testDirectRead() throws Exception { - LOG.info("Testing DIRECT_READ read method with JSON data"); - options = TestPipeline.testingPipelineOptions().as(BigQueryIOJsonOptions.class); - options.setReadMethod(TypedRead.Method.DIRECT_READ); - options.setInputTable(JSON_TABLE_DESTINATION); - - readAndValidateRows(options); - } - - @Test - public void testExportRead() throws Exception { - LOG.info("Testing EXPORT read method with JSON data"); - options = TestPipeline.testingPipelineOptions().as(BigQueryIOJsonOptions.class); - options.setReadMethod(TypedRead.Method.EXPORT); - options.setInputTable(JSON_TABLE_DESTINATION); - - readAndValidateRows(options); - } - @Test public void testQueryRead() throws Exception { LOG.info("Testing querying JSON data with DIRECT_READ read method"); @@ -368,35 +375,49 @@ public void testQueryRead() throws Exception { } @Test - public void testStorageWrite() throws Exception { - LOG.info("Testing writing JSON data with Storage API"); - + public void testStorageWriteRead() { options = TestPipeline.testingPipelineOptions().as(BigQueryIOJsonOptions.class); options.setWriteMethod(Write.Method.STORAGE_WRITE_API); + options.setReadMethod(TypedRead.Method.DIRECT_READ); String storageDestination = String.format("%s:%s.%s", project, DATASET_ID, STORAGE_WRITE_TEST_TABLE); options.setOutput(storageDestination); options.setInputTable(storageDestination); - runTestWrite(options); + runTestWriteRead(options); + } + + @Test + public void testFileLoadWriteExportRead() { + options = TestPipeline.testingPipelineOptions().as(BigQueryIOJsonOptions.class); + options.setWriteMethod(Write.Method.FILE_LOADS); + options.setReadMethod(TypedRead.Method.EXPORT); + + String storageDestination = + String.format("%s:%s.%s", project, DATASET_ID, FILE_LOAD_TEST_TABLE); + options.setOutput(storageDestination); + options.setInputTable(storageDestination); + + runTestWriteRead(options); } @Test - public void testLegacyStreamingWrite() throws Exception { + public void testLegacyStreamingWriteDefaultRead() { options = TestPipeline.testingPipelineOptions().as(BigQueryIOJsonOptions.class); options.setWriteMethod(Write.Method.STREAMING_INSERTS); + options.setReadMethod(TypedRead.Method.DEFAULT); String streamingDestination = String.format("%s:%s.%s", project, DATASET_ID, STREAMING_TEST_TABLE); options.setOutput(streamingDestination); options.setInputTable(streamingDestination); - runTestWrite(options); + runTestWriteRead(options); } @BeforeClass - public static void setupTestEnvironment() throws Exception { + public static void setupTestEnvironment() { PipelineOptionsFactory.register(BigQueryIOJsonOptions.class); } From c1c81b3618730aab8c5b533ebf8336d064ffa6f4 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 25 Jan 2024 18:07:08 -0800 Subject: [PATCH 082/169] Respect gcloud active configuration, if any, rather than always using "default." --- .../sdk/extensions/gcp/options/GcpOptions.java | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptions.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptions.java index 9137b5594ec5..d4cff72c43f3 100644 --- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptions.java +++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptions.java @@ -241,7 +241,7 @@ public List create(PipelineOptions options) { /** * Attempts to infer the default project based upon the environment this application is executing - * within. Currently this only supports getting the default project from gcloud. + * within. Currently this only supports getting the active project from gcloud. */ class DefaultProjectFactory implements DefaultValueFactory { private static final Logger LOG = LoggerFactory.getLogger(DefaultProjectFactory.class); @@ -256,9 +256,19 @@ public String create(PipelineOptions options) { configFile = new File(getEnvironment().get("APPDATA"), "gcloud/properties"); } else { // New versions of gcloud use this file + String activeConfig; + File activeConfigFile = + new File(System.getProperty("user.home"), ".config/gcloud/active_config"); + if (activeConfigFile.exists()) { + activeConfig = + Files.asCharSource(activeConfigFile, StandardCharsets.UTF_8).readFirstLine(); + } else { + activeConfig = "default"; + } configFile = new File( - System.getProperty("user.home"), ".config/gcloud/configurations/config_default"); + System.getProperty("user.home"), + ".config/gcloud/configurations/config_" + activeConfig); if (!configFile.exists()) { // Old versions of gcloud use this file configFile = new File(System.getProperty("user.home"), ".config/gcloud/properties"); From 3172736aaa3fb871fc4d7462e6057d971f3b151f Mon Sep 17 00:00:00 2001 From: "github-actions[bot]" <41898282+github-actions[bot]@users.noreply.github.com> Date: Sat, 27 Jan 2024 22:21:06 -0500 Subject: [PATCH 083/169] Update Python Dependencies (#30140) Co-authored-by: damccorm --- sdks/python/container/py310/base_image_requirements.txt | 4 ++-- sdks/python/container/py311/base_image_requirements.txt | 4 ++-- sdks/python/container/py38/base_image_requirements.txt | 4 ++-- sdks/python/container/py39/base_image_requirements.txt | 4 ++-- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index ebc96e17b122..4e5dd72248a2 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -79,7 +79,7 @@ grpcio-status==1.60.0 guppy3==3.1.4.post1 hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.97.0 +hypothesis==6.97.1 idna==3.6 iniconfig==2.0.0 joblib==1.3.2 @@ -95,7 +95,7 @@ numpy==1.26.3 oauth2client==4.1.3 objsize==0.7.0 orjson==3.9.12 -overrides==7.6.0 +overrides==7.7.0 packaging==23.2 pandas==2.0.3 parameterized==0.9.0 diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt index 37c1a30f673c..af61fd7c4794 100644 --- a/sdks/python/container/py311/base_image_requirements.txt +++ b/sdks/python/container/py311/base_image_requirements.txt @@ -78,7 +78,7 @@ grpcio-status==1.60.0 guppy3==3.1.4.post1 hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.97.0 +hypothesis==6.97.1 idna==3.6 iniconfig==2.0.0 joblib==1.3.2 @@ -94,7 +94,7 @@ numpy==1.26.3 oauth2client==4.1.3 objsize==0.7.0 orjson==3.9.12 -overrides==7.6.0 +overrides==7.7.0 packaging==23.2 pandas==2.0.3 parameterized==0.9.0 diff --git a/sdks/python/container/py38/base_image_requirements.txt b/sdks/python/container/py38/base_image_requirements.txt index eeb814f8a99c..57a71dbac928 100644 --- a/sdks/python/container/py38/base_image_requirements.txt +++ b/sdks/python/container/py38/base_image_requirements.txt @@ -80,7 +80,7 @@ grpcio-status==1.60.0 guppy3==3.1.4.post1 hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.97.0 +hypothesis==6.97.1 idna==3.6 importlib-metadata==7.0.1 importlib-resources==6.1.1 @@ -98,7 +98,7 @@ numpy==1.24.4 oauth2client==4.1.3 objsize==0.7.0 orjson==3.9.12 -overrides==7.6.0 +overrides==7.7.0 packaging==23.2 pandas==2.0.3 parameterized==0.9.0 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index acf6a106c0dc..abefdb119fb8 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -79,7 +79,7 @@ grpcio-status==1.60.0 guppy3==3.1.4.post1 hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.97.0 +hypothesis==6.97.1 idna==3.6 importlib-metadata==7.0.1 iniconfig==2.0.0 @@ -96,7 +96,7 @@ numpy==1.26.3 oauth2client==4.1.3 objsize==0.7.0 orjson==3.9.12 -overrides==7.6.0 +overrides==7.7.0 packaging==23.2 pandas==2.0.3 parameterized==0.9.0 From dde81384ef26c9bb5b03e01ba2bf9cd79389688c Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Mon, 29 Jan 2024 13:32:55 -0500 Subject: [PATCH 084/169] Fix clean up stale container workflow (#30132) * Fix clean up stale container workflow * fix comment * Setup GCloud Docker credential helper * Fix log spam * Fix expression * Fix no such manifest * Bump timeout to 3h --- .../beam_CancelStaleDataflowJobs.yml | 2 + .../workflows/beam_CleanUpGCPResources.yml | 2 + .../beam_CleanUpPrebuiltSDKImages.yml | 8 ++- .../stale_dataflow_prebuilt_image_cleaner.sh | 72 ++++++++++++------- 4 files changed, 58 insertions(+), 26 deletions(-) diff --git a/.github/workflows/beam_CancelStaleDataflowJobs.yml b/.github/workflows/beam_CancelStaleDataflowJobs.yml index 78cfe67da851..12c8a6b4f238 100644 --- a/.github/workflows/beam_CancelStaleDataflowJobs.yml +++ b/.github/workflows/beam_CancelStaleDataflowJobs.yml @@ -71,6 +71,8 @@ jobs: github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - name: Setup environment uses: ./.github/actions/setup-environment-action + with: + disable-cache: true - name: Authenticate on GCP id: auth uses: google-github-actions/auth@v1 diff --git a/.github/workflows/beam_CleanUpGCPResources.yml b/.github/workflows/beam_CleanUpGCPResources.yml index 7fe322ec84e0..08eb672cd7a4 100644 --- a/.github/workflows/beam_CleanUpGCPResources.yml +++ b/.github/workflows/beam_CleanUpGCPResources.yml @@ -71,6 +71,8 @@ jobs: github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - name: Setup environment uses: ./.github/actions/setup-environment-action + with: + disable-cache: true - name: Authenticate on GCP id: auth uses: google-github-actions/setup-gcloud@v0 diff --git a/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml b/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml index 4cc4b09a84ca..dcb80a980417 100644 --- a/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml +++ b/.github/workflows/beam_CleanUpPrebuiltSDKImages.yml @@ -52,7 +52,7 @@ jobs: beam_CleanUpPrebuiltSDKImages: name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] - timeout-minutes: 100 + timeout-minutes: 180 strategy: matrix: job_name: [beam_CleanUpPrebuiltSDKImages] @@ -71,12 +71,18 @@ jobs: github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - name: Setup environment uses: ./.github/actions/setup-environment-action + with: + disable-cache: true - name: Authenticate on GCP id: auth uses: google-github-actions/auth@v1 with: credentials_json: ${{ secrets.GCP_SA_KEY }} project_id: ${{ secrets.GCP_PROJECT_ID }} + - name: GCloud Docker credential helper + run: | + gcloud auth configure-docker gcr.io && \ + gcloud auth configure-docker us.gcr.io - name: run remove stale sdk container images uses: ./.github/actions/gradle-command-self-hosted-action with: diff --git a/.test-infra/tools/stale_dataflow_prebuilt_image_cleaner.sh b/.test-infra/tools/stale_dataflow_prebuilt_image_cleaner.sh index 2838d6245cec..804f6e2f39d4 100755 --- a/.test-infra/tools/stale_dataflow_prebuilt_image_cleaner.sh +++ b/.test-infra/tools/stale_dataflow_prebuilt_image_cleaner.sh @@ -25,7 +25,12 @@ set -euo pipefail PUBLIC_REPOSITORIES=(beam-sdk beam_portability beamgrafana beammetricssyncjenkins beammetricssyncgithub) PRIVATE_REPOSITORIES=(java-postcommit-it python-postcommit-it jenkins github-actions) # set as the same as 6-week release period -DELETE_BEFORE_DAY=$(date --iso-8601=s -d '6 weeks ago') +if [[ $OSTYPE == "linux-gnu"* ]]; then + # date command usage depending on OS + DELETE_BEFORE_DAY=$(date --iso-8601=s -d '6 weeks ago') +elif [[ $OSTYPE == "darwin"* ]]; then + DELETE_BEFORE_DAY=$(date -j -v-6w '+%Y-%m-%dT%H:%M:%S') +fi REPOSITORIES=("${PUBLIC_REPOSITORIES[@]/#/gcr.io/apache-beam-testing/}" "${PRIVATE_REPOSITORIES[@]/#/us.gcr.io/apache-beam-testing/}") @@ -49,6 +54,9 @@ while [ -n "$REPOSITORIES" ]; do REPOSITORIES=("${PENDING_REPOSITORIES[@]}") done +STALE_IMAGES="" +FAILED_INSPECT="" + for image_name in ${IMAGE_NAMES[@]}; do echo IMAGES FOR image ${image_name} FAILED_TO_DELETE="" @@ -70,24 +78,31 @@ for image_name in ${IMAGE_NAMES[@]}; do # do not delete the one with latest label and the newest image without latest label # this make sure we leave at least one container under each image name, either labelled "latest" or not if [ "$LATEST_IN_TIME" != "$current" ]; then - # Check to see if this image is built on top of earlier images. This is the case for multiarch images, - # they will have a virtual size of 0 and a created date at the start of the epoch, but their manifests will - # point to active images. These images should only be deleted when all of their dependencies can be safely - # deleted. - MANIFEST=$(docker manifest inspect ${image_name}@"${current}") - SHOULD_DELETE=0 - DIGEST=$(echo $MANIFEST | jq -r '.manifests[0].digest') - if [ "$DIGEST" != "null" ] - then - SHOULD_DELETE=1 - for i in ${STALE_IMAGES_CURRENT[@]} - do - echo "$i" - if [ "$i" = "$DIGEST" ] - then - SHOULD_DELETE=0 - fi - done + if [[ $image_name =~ 'beamgrafana' || $image_name =~ 'beammetricssyncjenkins' || $image_name =~ 'beammetricssyncgithub' ]]; then + # Skip docker manifest inspect for known single arch images, workaround permission issue & saving API call + SHOULD_DELETE=0 + else + # Check to see if this image is built on top of earlier images. This is the case for multiarch images, + # they will have a virtual size of 0 and a created date at the start of the epoch, but their manifests will + # point to active images. These images should only be deleted when all of their dependencies can be safely + # deleted. + MANIFEST=$(docker manifest inspect ${image_name}@"${current}" || echo "") + if [ -z "$MANIFEST" ]; then + # Sometimes "no such manifest" seen. Skip current if command hit error + FAILED_INSPECT+=" $current" + continue + fi + SHOULD_DELETE=0 + DIGEST=$(echo $MANIFEST | jq -r '.manifests[0].digest') + if [ "$DIGEST" != "null" ]; then + SHOULD_DELETE=1 + for i in ${STALE_IMAGES_CURRENT[@]}; do + if [ "$i" = "$DIGEST" ]; then + SHOULD_DELETE=0 + break + fi + done + fi fi if [ $SHOULD_DELETE = 0 ] @@ -100,12 +115,14 @@ for image_name in ${IMAGE_NAMES[@]}; do fi # Some images may not be successfully deleted the first time due to flakiness or having a dependency that hasn't been deleted yet. - RETRY_DELETE=("${FAILED_TO_DELETE[@]}") - echo "Failed to delete the following images: ${FAILED_TO_DELETE}. Retrying each of them." - for current in $RETRY_DELETE; do - echo "Trying again to delete image ${image_name}@"${current}". Command: gcloud container images delete ${image_name}@"${current}" --force-delete-tags -q" - gcloud container images delete ${image_name}@"${current}" --force-delete-tags -q - done + if [ -n "$FAILED_TO_DELETE" ]; then + RETRY_DELETE=("${FAILED_TO_DELETE[@]}") + echo "Failed to delete the following images: ${FAILED_TO_DELETE}. Retrying each of them." + for current in $RETRY_DELETE; do + echo "Trying again to delete image ${image_name}@"${current}". Command: gcloud container images delete ${image_name}@"${current}" --force-delete-tags -q" + gcloud container images delete ${image_name}@"${current}" --force-delete-tags -q + done + fi done if [[ ${STALE_IMAGES} ]]; then @@ -113,3 +130,8 @@ if [[ ${STALE_IMAGES} ]]; then else echo "No stale prebuilt container images found." fi + +if [ -n "$FAILED_INSPECT" ]; then + echo "Failed delete images $FAILED_INSPECT" + exit 1 +fi \ No newline at end of file From c9135e00041a3604ed493e2c8f933ca39c97f5a8 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Mon, 29 Jan 2024 13:46:21 -0500 Subject: [PATCH 085/169] Add info on getting started contributing (#30143) --- website/www/site/content/en/contribute/_index.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/website/www/site/content/en/contribute/_index.md b/website/www/site/content/en/contribute/_index.md index 6a2497627a9b..ce66d4de363a 100644 --- a/website/www/site/content/en/contribute/_index.md +++ b/website/www/site/content/en/contribute/_index.md @@ -82,6 +82,8 @@ Here’s a list of things you can do to get started contributing:

  • Comment “.take-issue” on the issue you'd like to work on. This will cause the issue to be assigned to you.
  • If appropriate, make your change and open a Pull Request.
  • Comment “.close-issue” to close the issue once all work has been completed.
  • +
  • You can find good first issues using the "good first issue" label.
  • +
  • You can also find more developer documentation on the wiki.
  • From b72eacfba37d8e5a2bc821b86b0bcf42900df087 Mon Sep 17 00:00:00 2001 From: clmccart Date: Mon, 29 Jan 2024 11:13:05 -0800 Subject: [PATCH 086/169] make getProcessingDistributionsForWorkId threadsafe (#29979) * make getProcessingDistributionsForWorkId synchronized to avoid race conditions with remove tracker * correctly access the concurrent hashmap to avoid race conditions --------- Co-authored-by: Claire McCarthy --- .../dataflow/worker/DataflowExecutionStateSampler.java | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateSampler.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateSampler.java index 61b7e01e49a0..1ff9a9be40d3 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateSampler.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateSampler.java @@ -118,13 +118,10 @@ public Optional getActiveMessageMetadataForWorkId(String } public Map getProcessingDistributionsForWorkId(String workId) { - if (!activeTrackersByWorkId.containsKey(workId)) { - if (completedProcessingMetrics.containsKey(workId)) { - return completedProcessingMetrics.get(workId); - } - return new HashMap<>(); - } DataflowExecutionStateTracker tracker = activeTrackersByWorkId.get(workId); + if (tracker == null) { + return completedProcessingMetrics.getOrDefault(workId, new HashMap<>()); + } return mergeStepStatsMaps( completedProcessingMetrics.getOrDefault(workId, new HashMap<>()), tracker.getProcessingTimesByStepCopy()); From b5dc54da26fcddd3b99c63ddd86f64ef80acf099 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 29 Jan 2024 11:52:12 -0800 Subject: [PATCH 087/169] Also mention JavaScript. --- sdks/python/apache_beam/yaml/yaml_mapping.md | 3 ++- sdks/python/apache_beam/yaml/yaml_provider.py | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/yaml/yaml_mapping.md b/sdks/python/apache_beam/yaml/yaml_mapping.md index 40ced4903f06..74b95d3cab2a 100644 --- a/sdks/python/apache_beam/yaml/yaml_mapping.md +++ b/sdks/python/apache_beam/yaml/yaml_mapping.md @@ -117,7 +117,8 @@ this up as a dependency and simply refer to it by fully qualified name, e.g. callable: pkg.module.fn ``` -Currently, in addition to Python, Java and SQL expressions are supported as well +Currently, in addition to Python, Java, SQL, and JavaScript (experimental) +expressions are supported as well ``` - type: MapToFields diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index e8eaa296398e..fc449b8eb589 100755 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -575,7 +575,8 @@ def create(elements: Iterable[Any], reshuffle: Optional[bool] = True): will result in a schema of the form (int, Row(string, List[int])). - This can also be expressed as YAML: + This can also be expressed as YAML:: + type: Create config: elements: From e199e42a3415969258020e6e1f4a91b89f7bfdae Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 29 Jan 2024 13:23:30 -0800 Subject: [PATCH 088/169] Fix whitespace in create suggestion. --- sdks/python/apache_beam/yaml/yaml_provider.py | 1 + 1 file changed, 1 insertion(+) diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index fc449b8eb589..c4db889f4b76 100755 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -588,6 +588,7 @@ def create(elements: Iterable[Any], reshuffle: Optional[bool] = True): second: str: "bar" values: [4, 5, 6] + Args: elements: The set of elements that should belong to the PCollection. YAML/JSON-style mappings will be interpreted as Beam rows. From f9029616a911a22f4da0725f4e5a8dc15e1f3312 Mon Sep 17 00:00:00 2001 From: tvalentyn Date: Tue, 30 Jan 2024 06:26:44 -0800 Subject: [PATCH 089/169] Update environments.md (#30149) Fix a typo in instructions. --- .../www/site/content/en/documentation/runtime/environments.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/www/site/content/en/documentation/runtime/environments.md b/website/www/site/content/en/documentation/runtime/environments.md index 3d730700e1f3..c860816e300d 100644 --- a/website/www/site/content/en/documentation/runtime/environments.md +++ b/website/www/site/content/en/documentation/runtime/environments.md @@ -228,7 +228,7 @@ Other runners, such as Dataflow, support specifying containers with different fl {{< runner direct >}} export IMAGE="my-repo/beam_python_sdk_custom" export TAG="X.Y.Z" -export IMAGE_URL = "${IMAGE}:${TAG}" +export IMAGE_URL="${IMAGE}:${TAG}" python -m apache_beam.examples.wordcount \ --input=/path/to/inputfile \ From 097e6e27bfec25671ea867aff545447f5784988f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 30 Jan 2024 09:30:46 -0500 Subject: [PATCH 090/169] Bump actions/cache from 3 to 4 (#30033) Bumps [actions/cache](https://github.com/actions/cache) from 3 to 4. - [Release notes](https://github.com/actions/cache/releases) - [Changelog](https://github.com/actions/cache/blob/main/RELEASES.md) - [Commits](https://github.com/actions/cache/compare/v3...v4) --- updated-dependencies: - dependency-name: actions/cache dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/playground_frontend_test.yml | 2 +- .github/workflows/tour_of_beam_frontend_test.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/playground_frontend_test.yml b/.github/workflows/playground_frontend_test.yml index 4d70197a11f7..8729d5d87563 100644 --- a/.github/workflows/playground_frontend_test.yml +++ b/.github/workflows/playground_frontend_test.yml @@ -48,7 +48,7 @@ jobs: - uses: actions/checkout@v4 - name: 'Cache Flutter Dependencies' - uses: actions/cache@v3 + uses: actions/cache@v4 with: path: /opt/hostedtoolcache/flutter key: ${{ runner.OS }}-flutter-install-cache-${{ env.FLUTTER_VERSION }} diff --git a/.github/workflows/tour_of_beam_frontend_test.yml b/.github/workflows/tour_of_beam_frontend_test.yml index 49bc9ef357d6..ca6853ac96c6 100644 --- a/.github/workflows/tour_of_beam_frontend_test.yml +++ b/.github/workflows/tour_of_beam_frontend_test.yml @@ -50,7 +50,7 @@ jobs: - uses: actions/checkout@v4 - name: 'Cache Flutter Dependencies' - uses: actions/cache@v3 + uses: actions/cache@v4 with: path: /opt/hostedtoolcache/flutter key: ${{ runner.OS }}-flutter-install-cache-${{ env.FLUTTER_VERSION }} From f9c9788815eb42d39ff8ff117285a3eacb5adb35 Mon Sep 17 00:00:00 2001 From: liferoad Date: Tue, 30 Jan 2024 09:45:28 -0500 Subject: [PATCH 091/169] Update 02_io_pubsub.md (#29986) --- learning/prompts/code-explaination/02_io_pubsub.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/learning/prompts/code-explaination/02_io_pubsub.md b/learning/prompts/code-explaination/02_io_pubsub.md index de02bd58e4d8..22f625acfb62 100644 --- a/learning/prompts/code-explaination/02_io_pubsub.md +++ b/learning/prompts/code-explaination/02_io_pubsub.md @@ -9,13 +9,13 @@ with beam.Pipeline(options=options) as p: | "Write" >> WriteToPubSub(topic=output_topic)) ``` Response: -This code reads messages from the [Pub/Sub](https://cloud.google.com/pubsub/docs/overview) subscription `input_subscription` using the `ReadFromPubSub`, processes them. The code then writes the resulting collection to the Pub/Sub topic `output_topic` using the `WriteToPubSub` transform. +This code reads messages from the [Pub/Sub](https://cloud.google.com/pubsub/docs/overview) subscription `input_subscription` using the `ReadFromPubSub` function and processes them. The code then writes the resulting collection to the Pub/Sub topic `output_topic` using the `WriteToPubSub` transform. Pub/Sub is only supported in streaming pipelines. -Reading messages directly from a topic is also supported. In this case, a temporary subscription will be created automatically. +Reading messages directly from a topic is also supported. In this case, a temporary subscription is created automatically. -`ReadFromPubSub` produces a `PCollection` of `PubsubMessage` objects or a `PCollection` of byte sequences. Behavior is controlled by the `with_attributes` parameter, with byte sequences being the default. +`ReadFromPubSub` produces a `PCollection` of `PubsubMessage` objects or a `PCollection` of byte sequences. The behavior is controlled by the `with_attributes` parameter, with byte sequences being the default. For more information about PCollections see the [Beam Programming Guide](https://beam.apache.org/documentation/basics/#pcollection). The `ProcessMessage` class processes the messages. This class is a subclass of the `DoFn` class. @@ -30,4 +30,4 @@ For more information about the `DoFn` class, see the [Beam Programming Guide](ht For more details, see the [Pub/Sub I/O transform documentation](https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.pubsub.html). -For a common pattern for configuring pipeline options, see the [Pipeline option pattern](https://beam.apache.org/documentation/patterns/pipeline-options/). \ No newline at end of file +For a common pattern for configuring pipeline options, see the [Pipeline option pattern](https://beam.apache.org/documentation/patterns/pipeline-options/). From 5a9a27cbafa5b08f4380e74beb91292ac2e8f85f Mon Sep 17 00:00:00 2001 From: Andrew Crites Date: Tue, 30 Jan 2024 06:55:16 -0800 Subject: [PATCH 092/169] Volatile heartbeat (#30148) --- .../org/apache/beam/runners/dataflow/worker/streaming/Work.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 69f2a0dcee76..8d4ba33a1abc 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 @@ -50,7 +50,7 @@ public class Work implements Runnable { private final Consumer processWorkFn; private TimedState currentState; - private boolean isFailed; + private volatile boolean isFailed; private Work(Windmill.WorkItem workItem, Supplier clock, Consumer processWorkFn) { this.workItem = workItem; From b9fd39cb16b1900120ea1d4b1b2844c3468321d0 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 30 Jan 2024 13:03:37 -0500 Subject: [PATCH 093/169] Fix Dataproc cleanup race condition (#30154) --- .github/workflows/README.md | 1 + .github/workflows/beam_CleanUpDataprocResources.yml | 2 +- .test-infra/dataproc/cleanup.sh | 4 ++-- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/.github/workflows/README.md b/.github/workflows/README.md index b39dfc9d5924..f882553ef9bf 100644 --- a/.github/workflows/README.md +++ b/.github/workflows/README.md @@ -79,6 +79,7 @@ jobs: github.event_name == 'push' || github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || + github.event_name == 'workflow_dispatch' || startsWith(github.event.comment.body, 'Run Job With Matrix') steps: - uses: actions/checkout@v3 diff --git a/.github/workflows/beam_CleanUpDataprocResources.yml b/.github/workflows/beam_CleanUpDataprocResources.yml index 86a3e42c3893..9e56230ffdbe 100644 --- a/.github/workflows/beam_CleanUpDataprocResources.yml +++ b/.github/workflows/beam_CleanUpDataprocResources.yml @@ -17,7 +17,7 @@ name: Cleanup Dataproc Resources on: schedule: - - cron: '0 */6 * * *' + - cron: '0 0 * * *' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.test-infra/dataproc/cleanup.sh b/.test-infra/dataproc/cleanup.sh index ac12a0bff05c..95eea525a2e1 100755 --- a/.test-infra/dataproc/cleanup.sh +++ b/.test-infra/dataproc/cleanup.sh @@ -44,8 +44,8 @@ function filterClusters(){ elapsedHours=$((($currentDate - $clusterStartTime)/3600)) - #Most of the jobs commonly finish in less than one hour - if [[ $elapsedHours -ge 2 ]]; then + # teardown clusters >= 6 hr old + if [[ $elapsedHours -ge 6 ]]; then for name in ${generatedResources[@]}; do # Only resources generated by the groovy jobs set are queued for deletion if [[ "$cluster" == *${name}* && ! ("$cluster" =~ nokill) ]]; then From 22fefebacabad865ead0f09d75bd0ab672d44cbb Mon Sep 17 00:00:00 2001 From: GStravinsky <52568826+GStravinsky@users.noreply.github.com> Date: Tue, 30 Jan 2024 22:35:53 +0100 Subject: [PATCH 094/169] BigQuery: Decouple clustering from time partitioning when writing (#30094) * Decouple clustering from time partitioning when writing * Update sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java Co-authored-by: Michel Davit * Refactor the tests, remove redundant input validations and reuse existing variables * add to CHANGES.md * add PR and not issue in CHANGES.md --------- Co-authored-by: Michel Davit --- CHANGES.md | 3 +- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 10 ++--- .../io/gcp/bigquery/CreateTableHelpers.java | 10 +++-- .../bigquery/DynamicDestinationsHelpers.java | 42 ++++++++++--------- .../gcp/bigquery/UpdateSchemaDestination.java | 9 ++-- .../io/gcp/bigquery/BigQueryIOWriteTest.java | 38 +++++++++-------- 6 files changed, 62 insertions(+), 50 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 2d9c249bf6b3..c63464a6e01e 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -65,7 +65,8 @@ ## New Features / Improvements -* X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* [Enrichment Transform](https://s.apache.org/enrichment-transform) along with GCP BigTable handler added to Python SDK ([#30001](https://github.com/apache/beam/pull/30001)). +* Allow writing clustered and not time partitioned BigQuery tables (Java) ([#30094](https://github.com/apache/beam/pull/30094)). ## Breaking Changes 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 3d38ef0e83ff..cd62c5810d81 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 @@ -96,7 +96,7 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.StorageClient; import org.apache.beam.sdk.io.gcp.bigquery.BigQuerySourceBase.ExtractResult; import org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinationsHelpers.ConstantSchemaDestinations; -import org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinationsHelpers.ConstantTimePartitioningDestinations; +import org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinationsHelpers.ConstantTimePartitioningClusteringDestinations; import org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinationsHelpers.SchemaFromViewDestinations; import org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinationsHelpers.TableFunctionDestinations; import org.apache.beam.sdk.io.gcp.bigquery.PassThroughThenCleanup.CleanupOperation; @@ -2744,8 +2744,7 @@ public Write withJsonTimePartitioning(ValueProvider partitioning) { } /** - * Specifies the clustering fields to use when writing to a single output table. Can only be - * used when {@link#withTimePartitioning(TimePartitioning)} is set. If {@link + * Specifies the clustering fields to use when writing to a single output table. If {@link * #to(SerializableFunction)} or {@link #to(DynamicDestinations)} is used to write to dynamic * tables, the fields here will be ignored; call {@link #withClustering()} instead. */ @@ -3357,9 +3356,10 @@ && getStorageApiTriggeringFrequency(bqOptions) != null) { } // Wrap with a DynamicDestinations class that will provide the proper TimePartitioning. - if (getJsonTimePartitioning() != null) { + if (getJsonTimePartitioning() != null + || Optional.ofNullable(getClustering()).map(Clustering::getFields).isPresent()) { dynamicDestinations = - new ConstantTimePartitioningDestinations<>( + new ConstantTimePartitioningClusteringDestinations<>( (DynamicDestinations) dynamicDestinations, getJsonTimePartitioning(), StaticValueProvider.of(BigQueryHelpers.toJsonString(getClustering()))); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTableHelpers.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTableHelpers.java index 6edd3f71cc71..7a94657107ec 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTableHelpers.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTableHelpers.java @@ -179,11 +179,13 @@ private static void tryCreateTable( TimePartitioning timePartitioning = tableDestination.getTimePartitioning(); if (timePartitioning != null) { table.setTimePartitioning(timePartitioning); - Clustering clustering = tableDestination.getClustering(); - if (clustering != null) { - table.setClustering(clustering); - } } + + Clustering clustering = tableDestination.getClustering(); + if (clustering != null) { + table.setClustering(clustering); + } + if (kmsKey != null) { table.setEncryptionConfiguration(new EncryptionConfiguration().setKmsKeyName(kmsKey)); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java index 62355fd9417d..1f042a81eb9d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java @@ -272,23 +272,28 @@ public String toString() { } } - static class ConstantTimePartitioningDestinations + static class ConstantTimePartitioningClusteringDestinations extends DelegatingDynamicDestinations { - private final ValueProvider jsonTimePartitioning; + private final @Nullable ValueProvider jsonTimePartitioning; private final @Nullable ValueProvider jsonClustering; - ConstantTimePartitioningDestinations( + ConstantTimePartitioningClusteringDestinations( DynamicDestinations inner, ValueProvider jsonTimePartitioning, ValueProvider jsonClustering) { super(inner); - Preconditions.checkArgumentNotNull( - jsonTimePartitioning, "jsonTimePartitioning provider can not be null"); - if (jsonTimePartitioning.isAccessible()) { - Preconditions.checkArgumentNotNull( - jsonTimePartitioning.get(), "jsonTimePartitioning can not be null"); - } + + checkArgument( + (jsonTimePartitioning != null + && jsonTimePartitioning.isAccessible() + && jsonTimePartitioning.get() != null) + || (jsonClustering != null + && jsonClustering.isAccessible() + && jsonClustering.get() != null), + "at least one of jsonTimePartitioning or jsonClustering must be non-null, accessible " + + "and present"); + this.jsonTimePartitioning = jsonTimePartitioning; this.jsonClustering = jsonClustering; } @@ -296,13 +301,12 @@ static class ConstantTimePartitioningDestinations @Override public TableDestination getDestination(@Nullable ValueInSingleWindow element) { TableDestination destination = super.getDestination(element); - String partitioning = this.jsonTimePartitioning.get(); - checkArgument(partitioning != null, "jsonTimePartitioning can not be null"); + String partitioning = + Optional.ofNullable(jsonTimePartitioning).map(ValueProvider::get).orElse(null); + String clustering = Optional.ofNullable(jsonClustering).map(ValueProvider::get).orElse(null); + return new TableDestination( - destination.getTableSpec(), - destination.getTableDescription(), - partitioning, - Optional.ofNullable(jsonClustering).map(ValueProvider::get).orElse(null)); + destination.getTableSpec(), destination.getTableDescription(), partitioning, clustering); } @Override @@ -316,10 +320,10 @@ public Coder getDestinationCoder() { @Override public String toString() { - MoreObjects.ToStringHelper helper = - MoreObjects.toStringHelper(this) - .add("inner", inner) - .add("jsonTimePartitioning", jsonTimePartitioning); + MoreObjects.ToStringHelper helper = MoreObjects.toStringHelper(this).add("inner", inner); + if (jsonTimePartitioning != null) { + helper.add("jsonTimePartitioning", jsonTimePartitioning); + } if (jsonClustering != null) { helper.add("jsonClustering", jsonClustering); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/UpdateSchemaDestination.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/UpdateSchemaDestination.java index 51e61fe41953..65bb3bf11b1b 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/UpdateSchemaDestination.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/UpdateSchemaDestination.java @@ -288,11 +288,12 @@ private BigQueryHelpers.PendingJob startZeroLoadJob( } if (timePartitioning != null) { loadConfig.setTimePartitioning(timePartitioning); - // only set clustering if timePartitioning is set - if (clustering != null) { - loadConfig.setClustering(clustering); - } } + + if (clustering != null) { + loadConfig.setClustering(clustering); + } + if (kmsKey != null) { loadConfig.setDestinationEncryptionConfiguration( new EncryptionConfiguration().setKmsKeyName(kmsKey)); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java index 89cbc2cd24b8..21d3e53a0701 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java @@ -500,7 +500,7 @@ private void verifySideInputs() { } } - void testTimePartitioningClustering( + void testTimePartitioningAndClustering( BigQueryIO.Write.Method insertMethod, boolean enablePartitioning, boolean enableClustering) throws Exception { TableRow row1 = new TableRow().set("date", "2018-01-01").set("number", "1"); @@ -545,16 +545,8 @@ void testTimePartitioningClustering( } } - void testTimePartitioning(BigQueryIO.Write.Method insertMethod) throws Exception { - testTimePartitioningClustering(insertMethod, true, false); - } - - void testClustering(BigQueryIO.Write.Method insertMethod) throws Exception { - testTimePartitioningClustering(insertMethod, true, true); - } - - @Test - public void testTimePartitioning() throws Exception { + void testTimePartitioningAndClusteringWithAllMethods( + Boolean enablePartitioning, Boolean enableClustering) throws Exception { BigQueryIO.Write.Method method; if (useStorageApi) { method = @@ -564,15 +556,27 @@ public void testTimePartitioning() throws Exception { } else { method = Method.FILE_LOADS; } - testTimePartitioning(method); + testTimePartitioningAndClustering(method, enablePartitioning, enableClustering); } @Test - public void testClusteringStorageApi() throws Exception { - if (useStorageApi) { - testClustering( - useStorageApiApproximate ? Method.STORAGE_API_AT_LEAST_ONCE : Method.STORAGE_WRITE_API); - } + public void testTimePartitioningWithoutClustering() throws Exception { + testTimePartitioningAndClusteringWithAllMethods(true, false); + } + + @Test + public void testTimePartitioningWithClustering() throws Exception { + testTimePartitioningAndClusteringWithAllMethods(true, true); + } + + @Test + public void testClusteringWithoutPartitioning() throws Exception { + testTimePartitioningAndClusteringWithAllMethods(false, true); + } + + @Test + public void testNoClusteringNoPartitioning() throws Exception { + testTimePartitioningAndClusteringWithAllMethods(false, false); } @Test From 415b1658f5022117f2f5c161d775d51b1591ab0d Mon Sep 17 00:00:00 2001 From: Chris Gray Date: Tue, 30 Jan 2024 15:10:28 -0800 Subject: [PATCH 095/169] Merge pull request #30137: Fix a misspelling in zetasql overview page * Fix a misspelling * Apply suggested style change --- .../content/en/documentation/dsls/sql/zetasql/overview.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/website/www/site/content/en/documentation/dsls/sql/zetasql/overview.md b/website/www/site/content/en/documentation/dsls/sql/zetasql/overview.md index 03ab6e79177d..b68324a53677 100644 --- a/website/www/site/content/en/documentation/dsls/sql/zetasql/overview.md +++ b/website/www/site/content/en/documentation/dsls/sql/zetasql/overview.md @@ -16,7 +16,7 @@ See the License for the specific language governing permissions and limitations under the License. --> # Beam ZetaSQL overview -Beam SQL supports a varient of the [ZetaSQL](https://github.com/google/zetasql) language. ZetaSQL is similar to the language in BigQuery's SQL framework. This Beam SQL dialect is especially useful in pipelines that [write to or read from BigQuery tables](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.html). +Beam SQL supports a variant of the [ZetaSQL](https://github.com/google/zetasql) language. ZetaSQL is similar to the language in BigQuery's SQL framework. This Beam SQL dialect is especially useful in pipelines that [write to or read from BigQuery tables](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.html). Beam SQL has additional extensions leveraging Beam’s unified batch/streaming model and processing complex data types. You can use these extensions with all Beam SQL dialects, including Beam ZetaSQL. @@ -30,4 +30,4 @@ A Beam SQL statement comprises a series of tokens. For more information about to Beam SQL supports standard SQL scalar data types as well as extensions including arrays, maps, and nested rows. For more information about scalar data in Beam ZetaSQL, see the [Data types](/documentation/dsls/sql/zetasql/data-types) reference. ## Functions and operators -For a list of the builtin functions and operators supported in Beam ZetaSQL, see [SupportedZetaSqlBuiltinFunctions.java](https://github.com/apache/beam/blob/master/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SupportedZetaSqlBuiltinFunctions.java) (commented-out entries are not yet supported). For documentation on how these functions work, see the [ZetaSQL functions and operators](https://github.com/google/zetasql/blob/master/docs/functions-and-operators.md) reference. \ No newline at end of file +For a list of the built-in functions and operators supported in Beam ZetaSQL, see [SupportedZetaSqlBuiltinFunctions.java](https://github.com/apache/beam/blob/master/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SupportedZetaSqlBuiltinFunctions.java) (commented-out entries are not yet supported). For documentation on how these functions work, see the [ZetaSQL functions and operators](https://github.com/google/zetasql/blob/master/docs/functions-and-operators.md) reference. From 1df9d6bba9150ff62276405e4e13d232411c0294 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 30 Jan 2024 18:45:22 -0500 Subject: [PATCH 096/169] Bumping timeout for beam_PostCommit_Java_DataflowV2 (#30160) --- .github/workflows/beam_PostCommit_Java_DataflowV2.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml index 58ecc37361d4..3abbe9cdc6c6 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml @@ -19,7 +19,7 @@ name: PostCommit Java Dataflow V2 on: schedule: - - cron: '30 3/6 * * *' + - cron: '30 3/8 * * *' pull_request_target: paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_DataflowV2.json'] workflow_dispatch: @@ -54,7 +54,7 @@ jobs: beam_PostCommit_Java_DataflowV2: name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] - timeout-minutes: 240 + timeout-minutes: 360 strategy: matrix: job_name: [beam_PostCommit_Java_DataflowV2] From 4c5ee349f9bfcdecbce9885c35ba5124066c4253 Mon Sep 17 00:00:00 2001 From: Jan Lukavsky Date: Wed, 31 Jan 2024 08:59:03 +0100 Subject: [PATCH 097/169] [sdks-java-core] Test for Reiterator outside loop in CoGbkResult --- .../java/org/apache/beam/sdk/transforms/join/CoGbkResult.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java index ba5bce20d090..ba0430e39daf 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java @@ -95,8 +95,9 @@ public CoGbkResult( // according to their tag. final Iterator taggedIter = taggedValues.iterator(); int elementCount = 0; + boolean isReiterator = taggedIter instanceof Reiterator; while (taggedIter.hasNext()) { - if (elementCount++ >= inMemoryElementCount && taggedIter instanceof Reiterator) { + if (isReiterator && elementCount++ >= inMemoryElementCount) { // Let the tails be lazy. break; } From 41dee464db458fa72eeab7ddc902b242ebc894eb Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> Date: Wed, 31 Jan 2024 13:07:02 -0500 Subject: [PATCH 098/169] Make ReadFromBigQueryRequest id more randomized (#30156) * make ReadFromBigQueryRequest id more randomized --- sdks/python/apache_beam/io/gcp/bigquery.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/io/gcp/bigquery.py b/sdks/python/apache_beam/io/gcp/bigquery.py index bba8b8a4af75..7648ab4064da 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery.py +++ b/sdks/python/apache_beam/io/gcp/bigquery.py @@ -361,6 +361,7 @@ def chain_after(result): import json import logging import random +import secrets import time import uuid import warnings @@ -2925,8 +2926,9 @@ def __init__( self.table = table self.validate() - # We use this internal object ID to generate BigQuery export directories. - self.obj_id = random.randint(0, 100000) + # We use this internal object ID to generate BigQuery export directories + # and to create BigQuery job names + self.obj_id = '%d_%s' % (int(time.time()), secrets.token_hex(3)) def validate(self): if self.table is not None and self.query is not None: From 81513f10d60f65c0873fc2484fb27a8e38fabc56 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 31 Jan 2024 13:36:13 -0500 Subject: [PATCH 099/169] Bump github.com/aws/aws-sdk-go-v2/credentials in /sdks (#30162) Bumps [github.com/aws/aws-sdk-go-v2/credentials](https://github.com/aws/aws-sdk-go-v2) from 1.16.13 to 1.16.16. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Changelog](https://github.com/aws/aws-sdk-go-v2/blob/v1.16.16/CHANGELOG.md) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/v1.16.13...v1.16.16) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go-v2/credentials 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 | 18 +++++++++--------- sdks/go.sum | 36 ++++++++++++++++++------------------ 2 files changed, 27 insertions(+), 27 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 1c5874a49c1e..de7b36930455 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -30,9 +30,9 @@ require ( cloud.google.com/go/pubsub v1.33.0 cloud.google.com/go/spanner v1.53.1 cloud.google.com/go/storage v1.36.0 - github.com/aws/aws-sdk-go-v2 v1.24.0 + github.com/aws/aws-sdk-go-v2 v1.24.1 github.com/aws/aws-sdk-go-v2/config v1.26.2 - github.com/aws/aws-sdk-go-v2/credentials v1.16.13 + github.com/aws/aws-sdk-go-v2/credentials v1.16.16 github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.8 github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2 github.com/aws/smithy-go v1.19.0 @@ -113,18 +113,18 @@ require ( github.com/apache/thrift v0.16.0 // indirect github.com/aws/aws-sdk-go v1.34.0 // indirect github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1 // indirect - github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.10 // indirect - github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.9 // indirect - github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.9 // indirect + github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.11 // indirect + github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.10 // indirect + github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.10 // indirect github.com/aws/aws-sdk-go-v2/internal/ini v1.7.2 // indirect github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.3 // indirect github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.4 // indirect github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.9 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.10 // indirect github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3 // indirect - github.com/aws/aws-sdk-go-v2/service/sso v1.18.5 // indirect - github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.5 // indirect - github.com/aws/aws-sdk-go-v2/service/sts v1.26.6 // indirect + github.com/aws/aws-sdk-go-v2/service/sso v1.18.7 // indirect + github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.7 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.26.7 // 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.2.0 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 7dc312f9cba2..60b711ab0595 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -81,26 +81,26 @@ 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.24.0 h1:890+mqQ+hTpNuw0gGP6/4akolQkSToDJgHfQE7AwGuk= -github.com/aws/aws-sdk-go-v2 v1.24.0/go.mod h1:LNh45Br1YAkEKaAqvmE1m8FUx6a5b/V0oAKV7of29b4= +github.com/aws/aws-sdk-go-v2 v1.24.1 h1:xAojnj+ktS95YZlDf0zxWBkbFtymPeDP+rvUQIH3uAU= +github.com/aws/aws-sdk-go-v2 v1.24.1/go.mod h1:LNh45Br1YAkEKaAqvmE1m8FUx6a5b/V0oAKV7of29b4= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1 h1:ZY3108YtBNq96jNZTICHxN1gSBSbnvIdYwwqnvCV4Mc= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1/go.mod h1:t8PYl/6LzdAqsU4/9tz28V/kU+asFePvpOMkdul0gEQ= 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.26.2 h1:+RWLEIWQIGgrz2pBPAUoGgNGs1TOyF4Hml7hCnYj2jc= github.com/aws/aws-sdk-go-v2/config v1.26.2/go.mod h1:l6xqvUxt0Oj7PI/SUXYLNyZ9T/yBPn3YTQcJLLOdtR8= github.com/aws/aws-sdk-go-v2/credentials v1.3.1/go.mod h1:r0n73xwsIVagq8RsxmZbGSRQFj9As3je72C2WzUIToc= -github.com/aws/aws-sdk-go-v2/credentials v1.16.13 h1:WLABQ4Cp4vXtXfOWOS3MEZKr6AAYUpMczLhgKtAjQ/8= -github.com/aws/aws-sdk-go-v2/credentials v1.16.13/go.mod h1:Qg6x82FXwW0sJHzYruxGiuApNo31UEtJvXVSZAXeWiw= +github.com/aws/aws-sdk-go-v2/credentials v1.16.16 h1:8q6Rliyv0aUFAVtzaldUEcS+T5gbadPbWdV1WcAddK8= +github.com/aws/aws-sdk-go-v2/credentials v1.16.16/go.mod h1:UHVZrdUsv63hPXFo1H7c5fEneoVo9UXiz36QG1GEPi0= 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.14.10 h1:w98BT5w+ao1/r5sUuiH6JkVzjowOKeOJRHERyy1vh58= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.10/go.mod h1:K2WGI7vUvkIv1HoNbfBA1bvIZ+9kL3YVmWxeKuLQsiw= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.11 h1:c5I5iH+DZcH3xOIMlz3/tCKJDaHFwYEmxvlh2fAcFo8= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.11/go.mod h1:cRrYDYAMUohBJUtUnOhydaMHtiK/1NZ0Otc9lIb6O0Y= 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.13.8 h1:wuOjvalpd2CnXffks74Vq6n3yv9vunKCoy4R1sjStGk= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.8/go.mod h1:vywwjy6VnrR48Izg136JoSUXC4mH9QeUi3g0EH9DSrA= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.9 h1:v+HbZaCGmOwnTTVS86Fleq0vPzOd7tnJGbFhP0stNLs= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.9/go.mod h1:Xjqy+Nyj7VDLBtCMkQYOw1QYfAEZCVLrfI0ezve8wd4= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.9 h1:N94sVhRACtXyVcjXxrwK1SKFIJrA9pOJ5yu2eSHnmls= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.9/go.mod h1:hqamLz7g1/4EJP+GH5NBhcUMLjW+gKLQabgyz6/7WAU= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.10 h1:vF+Zgd9s+H4vOXd5BMaPWykta2a6Ih0AKLq/X6NYKn4= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.10/go.mod h1:6BkRjejp/GR4411UGqkX8+wFMbFbqsUIimfK4XjOKR4= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.10 h1:nYPe006ktcqUji8S2mqXf9c/7NdiKriOwMvWQHgYztw= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.10/go.mod h1:6UV4SZkVvmODfXKql4LCbaZUpF7HO2BX38FgBf9ZOLw= 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.7.2 h1:GrSw8s0Gs/5zZ0SX+gX4zQjRnRsMJDJ2sLur1gRBhEM= github.com/aws/aws-sdk-go-v2/internal/ini v1.7.2/go.mod h1:6fQQgfuGmw8Al/3M2IgIllycxV7ZW7WCdVSqfBeUiCY= @@ -112,8 +112,8 @@ github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.4/go.mod h1: github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3 h1:xbwRyCy7kXrOj89iIKLB6NfE2WCpP9HoKyk8dMDvnIQ= github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3/go.mod h1:R+/S1O4TYpcktbVwddeOYg+uwUfLhADP2S/x4QwsCTM= 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.10.9 h1:Nf2sHxjMJR8CSImIVCONRi4g0Su3J+TSTbS7G0pUeMU= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.9/go.mod h1:idky4TER38YIjr2cADF1/ugFMKvZV7p//pVeV5LZbF0= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.10 h1:DBYTXwIGQSGs9w4jKm60F5dmCQ3EEruxdc0MFh+3EY4= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.10/go.mod h1:wohMUQiFdzo0NtxbBg0mSRGZ4vL3n0dKjLTINdcIino= 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.16.3 h1:KV0z2RDc7euMtg8aUT1czv5p29zcLlXALNFsd3jkkEc= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3/go.mod h1:KZgs2ny8HsxRIRbDwgvJcHHBZPOzQr/+NtGwnP+w2ec= @@ -121,13 +121,13 @@ github.com/aws/aws-sdk-go-v2/service/s3 v1.11.1/go.mod h1:XLAGFrEjbvMCLvAtWLLP32 github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2 h1:NnduxUd9+Fq9DcCDdJK8v6l9lR1xDX4usvog+JuQAno= github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2/go.mod h1:NXRKkiRF+erX2hnybnVU660cYT5/KChRD4iUgJ97cI8= 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.18.5 h1:ldSFWz9tEHAwHNmjx2Cvy1MjP5/L9kNoR0skc6wyOOM= -github.com/aws/aws-sdk-go-v2/service/sso v1.18.5/go.mod h1:CaFfXLYL376jgbP7VKC96uFcU8Rlavak0UlAwk1Dlhc= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.5 h1:2k9KmFawS63euAkY4/ixVNsYYwrwnd5fIvgEKkfZFNM= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.5/go.mod h1:W+nd4wWDVkSUIox9bacmkBP5NMFQeTJ/xqNabpzSR38= +github.com/aws/aws-sdk-go-v2/service/sso v1.18.7 h1:eajuO3nykDPdYicLlP3AGgOyVN3MOlFmZv7WGTuJPow= +github.com/aws/aws-sdk-go-v2/service/sso v1.18.7/go.mod h1:+mJNDdF+qiUlNKNC3fxn74WWNN+sOiGOEImje+3ScPM= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.7 h1:QPMJf+Jw8E1l7zqhZmMlFw6w1NmfkfiSK8mS4zOx3BA= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.7/go.mod h1:ykf3COxYI0UJmxcfcxcVuz7b6uADi1FkiUz6Eb7AgM8= 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.26.6 h1:HJeiuZ2fldpd0WqngyMR6KW7ofkXNLyOaHwEIGm39Cs= -github.com/aws/aws-sdk-go-v2/service/sts v1.26.6/go.mod h1:XX5gh4CB7wAs4KhcF46G6C8a2i7eupU19dcAAE+EydU= +github.com/aws/aws-sdk-go-v2/service/sts v1.26.7 h1:NzO4Vrau795RkUdSHKEwiR01FaGzGOH1EETJ+5QHnm0= +github.com/aws/aws-sdk-go-v2/service/sts v1.26.7/go.mod h1:6h2YuIoxaMSCFf5fi1EgZAwdfkGMgDY+DVfa61uLe4U= github.com/aws/smithy-go v1.6.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= github.com/aws/smithy-go v1.19.0 h1:KWFKQV80DpP3vJrrA9sVAHQ5gc2z8i4EzrLhLlWXcBM= github.com/aws/smithy-go v1.19.0/go.mod h1:NukqUGpCZIILqqiV0NIjeFh24kd/FAa4beRb6nbIUPE= From 89d1c06e1eab530c4e95a807a035b450c340e5f8 Mon Sep 17 00:00:00 2001 From: Svetak Sundhar Date: Wed, 31 Jan 2024 19:26:26 +0000 Subject: [PATCH 100/169] Fix Fhir IO Search IT (#30158) * [BEAM-12550] Implement Parallelizable Skew and Kurtosis (Skew implementation) R: @TheNeuralBit * fix fhirio search tests * spotless * reduce query count and size * postcommit * remove clear --------- Co-authored-by: svetakvsundhar --- .../trigger_files/beam_PostCommit_Java_DataflowV2.json | 0 .../beam/sdk/io/gcp/healthcare/FhirIOSearchIT.java | 9 +++------ 2 files changed, 3 insertions(+), 6 deletions(-) create mode 100644 .github/trigger_files/beam_PostCommit_Java_DataflowV2.json diff --git a/.github/trigger_files/beam_PostCommit_Java_DataflowV2.json b/.github/trigger_files/beam_PostCommit_Java_DataflowV2.json new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOSearchIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOSearchIT.java index fc92a568ba4d..ebd94123d499 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOSearchIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOSearchIT.java @@ -43,7 +43,6 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.junit.After; import org.junit.Before; -import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; @@ -66,7 +65,7 @@ public static Collection versions() { private static final String BASE_STORE_ID = "FHIR_store_search_it_" + System.currentTimeMillis() + "_" + new SecureRandom().nextInt(32); private String fhirStoreId; - private static final int MAX_NUM_OF_SEARCHES = 50; + private static final int MAX_NUM_OF_SEARCHES = 20; private List> input = new ArrayList<>(); private List>> genericParametersInput = new ArrayList<>(); private static final String KEY = "key"; @@ -96,9 +95,9 @@ public void setup() throws Exception { JsonArray fhirResources = JsonParser.parseString(bundles.get(0)).getAsJsonObject().getAsJsonArray("entry"); - Map searchParameters = ImmutableMap.of("_count", "50"); + Map searchParameters = ImmutableMap.of("_count", "20"); Map> genericSearchParameters = - ImmutableMap.of("_count", Arrays.asList(50)); + ImmutableMap.of("_count", Arrays.asList(20)); // Include a non-resource type search. input.add(FhirSearchParameter.of("", KEY, searchParameters)); @@ -127,7 +126,6 @@ public void teardown() throws IOException { } } - @Ignore("https://github.com/apache/beam/issues/28505") @Test public void testFhirIOSearch() { pipeline.getOptions().as(DirectOptions.class).setBlockOnRun(false); @@ -157,7 +155,6 @@ public void testFhirIOSearch() { pipeline.run().waitUntilFinish(); } - @Ignore("https://github.com/apache/beam/issues/28505") @Test public void testFhirIOSearchWithGenericParameters() { pipeline.getOptions().as(DirectOptions.class).setBlockOnRun(false); From c281852a58ffb666fdc14badac93deed60682f48 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Tue, 30 Jan 2024 16:17:01 -0800 Subject: [PATCH 101/169] [YAML] Add documentation to Beam site. --- .../en/documentation/sdks/yaml-combine.md | 166 +++++ .../en/documentation/sdks/yaml-errors.md | 200 ++++++ .../documentation/sdks/yaml-inline-python.md | 202 ++++++ .../content/en/documentation/sdks/yaml-udf.md | 248 ++++++++ .../content/en/documentation/sdks/yaml.md | 587 ++++++++++++++++++ website/www/site/data/pipelines.yaml | 3 + .../partials/section-menu/en/sdks.html | 11 + .../site/static/images/logos/sdks/yaml.png | Bin 0 -> 6197 bytes 8 files changed, 1417 insertions(+) create mode 100644 website/www/site/content/en/documentation/sdks/yaml-combine.md create mode 100644 website/www/site/content/en/documentation/sdks/yaml-errors.md create mode 100644 website/www/site/content/en/documentation/sdks/yaml-inline-python.md create mode 100644 website/www/site/content/en/documentation/sdks/yaml-udf.md create mode 100644 website/www/site/content/en/documentation/sdks/yaml.md create mode 100644 website/www/site/static/images/logos/sdks/yaml.png diff --git a/website/www/site/content/en/documentation/sdks/yaml-combine.md b/website/www/site/content/en/documentation/sdks/yaml-combine.md new file mode 100644 index 000000000000..e2fef304fb0a --- /dev/null +++ b/website/www/site/content/en/documentation/sdks/yaml-combine.md @@ -0,0 +1,166 @@ + + +# Beam YAML Aggregations + +Beam YAML has EXPERIMENTAL ability to do aggregations to group and combine +values across records. The is accomplished via the `Combine` transform type. +Currently `Combine` needs to be in the `yaml_experimental_features` +option to use this transform. + +For example, one can write + +``` +- type: Combine + config: + group_by: col1 + combine: + total: + value: col2 + fn: + type: sum +``` + +If the function has no configuration requirements, it can be provided directly +as a string + +``` +- type: Combine + config: + group_by: col1 + combine: + total: + value: col2 + fn: sum +``` + +This can be simplified further if the output field name is the same as the input +field name + +``` +- type: Combine + config: + group_by: col1 + combine: + col2: sum +``` + +One can aggregate over may fields at once + +``` +- type: Combine + config: + group_by: col1 + combine: + col2: sum + col3: max +``` + +and/or group by more than one field + +``` +- type: Combine + config: + group_by: [col1, col2] + combine: + col3: sum +``` + +or none at all (which will result in a global combine with a single output) + +``` +- type: Combine + config: + group_by: [] + combine: + col2: sum + col3: max +``` + +## Windowed aggregation + +As with all transforms, `Combine` can take a windowing parameter + +``` +- type: Combine + windowing: + type: fixed + size: 60 + config: + group_by: col1 + combine: + col2: sum + col3: max +``` + +If no windowing specification is provided, it inherits the windowing +parameters from upstream, e.g. + +``` +- type: WindowInto + windowing: + type: fixed + size: 60 +- type: Combine + config: + group_by: col1 + combine: + col2: sum + col3: max +``` + +is equivalent to the previous example. + + +## Custom aggregation functions + +One can use aggregation functions defined in Python by setting the language +parameter. + +``` +- type: Combine + config: + language: python + group_by: col1 + combine: + biggest: + value: "col2 + col2" + fn: + type: 'apache_beam.transforms.combiners.TopCombineFn' + config: + n: 10 +``` + +## SQL-style aggregations + +By setting the language to SQL, one can provide full SQL snippets as the +combine fn. + +``` +- type: Combine + config: + language: sql + group_by: col1 + combine: + num_values: "count(*)" + total: "sum(col2)" +``` + +One can of course also use the `Sql` transform type and provide a query +directly. diff --git a/website/www/site/content/en/documentation/sdks/yaml-errors.md b/website/www/site/content/en/documentation/sdks/yaml-errors.md new file mode 100644 index 000000000000..aec602393674 --- /dev/null +++ b/website/www/site/content/en/documentation/sdks/yaml-errors.md @@ -0,0 +1,200 @@ + + +# Beam YAML Error Handling + +The larger one's pipeline gets, the more common it is to encounter "exceptional" +data that is malformatted, doesn't handle the proper preconditions, or otherwise +breaks during processing. Generally any such record will cause the pipeline to +permanently fail, but often it is desirable to allow the pipeline to continue, +re-directing bad records to another path for special handling or simply +recording them for later off-line analysis. This is often called the +"dead letter queue" pattern. + +Beam YAML has special support for this pattern if the transform supports a +`error_handling` config parameter with an `output` field. For example, +the following code will write all "good" processed records to one file and +any "bad" records to a separate file. + +``` +pipeline: + transforms: + - type: ReadFromCsv + config: + path: /path/to/input*.csv + + - type: MapToFields + input: ReadFromCsv + config: + language: python + fields: + col1: col1 + # This could raise a divide-by-zero error. + ratio: col2 / col3 + error_handling: + output: my_error_output + + - type: WriteToJson + input: MapToFields + config: + path: /path/to/output.json + + - type: WriteToJson + name: WriteErrorsToJson + input: MapToFields.my_error_output + config: + path: /path/to/errors.json +``` + +Note that with `error_handling` declared, `MapToFields.my_error_output` +**must** be consumed; to ignore it will be an error. Any use is fine, e.g. +logging the bad records to stdout would be sufficient (though not recommended +for a robust pipeline). + +Note also that the exact format of the error outputs is still being finalized. +They can be safely printed and written to outputs, but their precise schema +may change in a future version of Beam and should not yet be depended on. + +Some transforms allow for extra arguments in their error_handling config, e.g. +for Python functions one can give a `threshold` which limits the relative number +of records that can be bad before considering the entire pipeline a failure + +``` +pipeline: + transforms: + - type: ReadFromCsv + config: + path: /path/to/input*.csv + + - type: MapToFields + input: ReadFromCsv + config: + language: python + fields: + col1: col1 + # This could raise a divide-by-zero error. + ratio: col2 / col3 + error_handling: + output: my_error_output + # If more than 10% of records throw an error, stop the pipeline. + threshold: 0.1 + + - type: WriteToJson + input: MapToFields + config: + path: /path/to/output.json + + - type: WriteToJson + name: WriteErrorsToJson + input: MapToFields.my_error_output + config: + path: /path/to/errors.json +``` + +One can do arbitrary further processing on these failed records if desired, +e.g. + +``` +pipeline: + transforms: + - type: ReadFromCsv + config: + path: /path/to/input*.csv + + - type: MapToFields + name: ComputeRatio + input: ReadFromCsv + config: + language: python + fields: + col1: col1 + # This could raise a divide-by-zero error. + ratio: col2 / col3 + error_handling: + output: my_error_output + + - type: MapToFields + name: ComputeRatioForBadRecords + input: ComputeRatio.my_error_output + config: + language: python + fields: + col1: col1 + ratio: col2 / (col3 + 1) + error_handling: + output: still_bad + + - type: WriteToJson + # Takes as input everything from the "success" path of both transforms. + input: [ComputeRatio, ComputeRatioForBadRecords] + config: + path: /path/to/output.json + + - type: WriteToJson + name: WriteErrorsToJson + # These failed the first and the second transform. + input: ComputeRatioForBadRecords.still_bad + config: + path: /path/to/errors.json +``` + +When using the `chain` syntax, the required error consumption can happen +in an `extra_transforms` block. + +``` +pipeline: + type: chain + transforms: + - type: ReadFromCsv + config: + path: /path/to/input*.csv + + - type: MapToFields + name: SomeStep + config: + language: python + fields: + col1: col1 + # This could raise a divide-by-zero error. + ratio: col2 / col3 + error_handling: + output: errors + + - type: MapToFields + name: AnotherStep + config: + language: python + fields: + col1: col1 + # This could raise a divide-by-zero error. + inverse_ratio: 1 / ratio + error_handling: + output: errors + + - type: WriteToJson + config: + path: /path/to/output.json + + extra_transforms: + - type: WriteToJson + name: WriteErrors + input: [SomeStep.errors, AnotherStep.errors] + config: + path: /path/to/errors.json +``` diff --git a/website/www/site/content/en/documentation/sdks/yaml-inline-python.md b/website/www/site/content/en/documentation/sdks/yaml-inline-python.md new file mode 100644 index 000000000000..72b8b76c58a2 --- /dev/null +++ b/website/www/site/content/en/documentation/sdks/yaml-inline-python.md @@ -0,0 +1,202 @@ + + +# Using PyTransform form YAML + +Beam YAML provides the ability to easily invoke Python transforms via the +`PyTransform` type, simply referencing them by fully qualified name. +For example, + +``` +- type: PyTransform + config: + constructor: apache_beam.pkg.module.SomeTransform + args: [1, 'foo'] + kwargs: + baz: 3 +``` + +will invoke the transform `apache_beam.pkg.mod.SomeTransform(1, 'foo', baz=3)`. +This fully qualified name can be any PTransform class or other callable that +returns a PTransform. Note, however, that PTransforms that do not accept or +return schema'd data may not be as useable to use from YAML. +Restoring the schema-ness after a non-schema returning transform can be done +by using the `callable` option on `MapToFields` which takes the entire element +as an input, e.g. + +``` +- type: PyTransform + config: + constructor: apache_beam.pkg.module.SomeTransform + args: [1, 'foo'] + kwargs: + baz: 3 +- type: MapToFields + config: + language: python + fields: + col1: + callable: 'lambda element: element.col1' + output_type: string + col2: + callable: 'lambda element: element.col2' + output_type: integer +``` + +This can be used to call arbitrary transforms in the Beam SDK, e.g. + +``` +pipeline: + transforms: + - type: PyTransform + name: ReadFromTsv + input: {} + config: + constructor: apache_beam.io.ReadFromCsv + kwargs: + path: '/path/to/*.tsv' + sep: '\t' + skip_blank_lines: True + true_values: ['yes'] + false_values: ['no'] + comment: '#' + on_bad_lines: 'skip' + binary: False + splittable: False +``` + + +## Defining a transform inline using `__constructor__` + +If the desired transform does not exist, one can define it inline as well. +This is done with the special `__constructor__` keywords, +similar to how cross-language transforms are done. + +With the `__constuctor__` keyword, one defines a Python callable that, on +invocation, *returns* the desired transform. The first argument (or `source` +keyword argument, if there are no positional arguments) +is interpreted as the Python code. For example + +``` +- type: PyTransform + config: + constructor: __constructor__ + kwargs: + source: | + import apache_beam as beam + + def create_my_transform(inc): + return beam.Map(lambda x: beam.Row(a=x.col2 + inc)) + + inc: 10 +``` + +will apply `beam.Map(lambda x: beam.Row(a=x.col2 + 10))` to the incoming +PCollection. + +As a class object can be invoked as its own constructor, this allows one to +define a `beam.PTransform` inline, e.g. + +``` +- type: PyTransform + config: + constructor: __constructor__ + kwargs: + source: | + class MyPTransform(beam.PTransform): + def __init__(self, inc): + self._inc = inc + def expand(self, pcoll): + return pcoll | beam.Map(lambda x: beam.Row(a=x.col2 + self._inc)) + + inc: 10 +``` + +which works exactly as one would expect. + + +## Defining a transform inline using `__callable__` + +The `__callable__` keyword works similarly, but instead of defining a +callable that returns an applicable `PTransform` one simply defines the +expansion to be performed as a callable. This is analogous to BeamPython's +`ptransform.ptransform_fn` decorator. + +In this case one can simply write + +``` +- type: PyTransform + config: + constructor: __callable__ + kwargs: + source: | + def my_ptransform(pcoll, inc): + return pcoll | beam.Map(lambda x: beam.Row(a=x.col2 + inc)) + + inc: 10 +``` + + +# External transforms + +One can also invoke PTransforms define elsewhere via a `python` provider, +for example + +``` +pipeline: + transforms: + - ... + - type: MyTransform + config: + kwarg: whatever + +providers: + - ... + - type: python + input: ... + config: + packages: + - 'some_pypi_package>=version' + transforms: + MyTransform: 'pkg.module.MyTransform' +``` + +These can be defined inline as well, with or without dependencies, e.g. + +``` +pipeline: + transforms: + - ... + - type: ToCase + input: ... + config: + upper: True + +providers: + - type: python + config: {} + transforms: + 'ToCase': | + @beam.ptransform_fn + def ToCase(pcoll, upper): + if upper: + return pcoll | beam.Map(lambda x: str(x).upper()) + else: + return pcoll | beam.Map(lambda x: str(x).lower()) +``` diff --git a/website/www/site/content/en/documentation/sdks/yaml-udf.md b/website/www/site/content/en/documentation/sdks/yaml-udf.md new file mode 100644 index 000000000000..74b95d3cab2a --- /dev/null +++ b/website/www/site/content/en/documentation/sdks/yaml-udf.md @@ -0,0 +1,248 @@ + + +# Beam YAML mappings + +Beam YAML has the ability to do simple transformations which can be used to +get data into the correct shape. The simplest of these is `MaptoFields` +which creates records with new fields defined in terms of the input fields. + +## Field renames + +To rename fields one can write + +``` +- type: MapToFields + config: + fields: + new_col1: col1 + new_col2: col2 +``` + +will result in an output where each record has two fields, +`new_col1` and `new_col2`, whose values are those of `col1` and `col2` +respectively (which are the names of two fields from the input schema). + +One can specify the append parameter which indicates the original fields should +be retained similar to the use of `*` in an SQL select statement. For example + +``` +- type: MapToFields + config: + append: true + fields: + new_col1: col1 + new_col2: col2 +``` + +will output records that have `new_col1` and `new_col2` as *additional* +fields. When the append field is specified, one can drop fields as well, e.g. + +``` +- type: MapToFields + config: + append: true + drop: + - col3 + fields: + new_col1: col1 + new_col2: col2 +``` + +which includes all original fiels *except* col3 in addition to outputting the +two new ones. + + +## Mapping functions + +Of course one may want to do transformations beyond just dropping and renaming +fields. Beam YAML has the ability to inline simple UDFs. +This requires a language specification. For example, we can provide a +Python expression referencing the input fields + +``` +- type: MapToFields + config: + language: python + fields: + new_col: "col1.upper()" + another_col: "col2 + col3" +``` + +In addition, one can provide a full Python callable that takes the row as an +argument to do more complex mappings +(see [PythonCallableSource](https://beam.apache.org/releases/pydoc/current/apache_beam.utils.python_callable.html#apache_beam.utils.python_callable.PythonCallableWithSource) +for acceptable formats). Thus one can write + +``` +- type: MapToFields + config: + language: python + fields: + new_col: + callable: | + import re + def my_mapping(row): + if re.match("[0-9]+", row.col1) and row.col2 > 0: + return "good" + else: + return "bad" +``` + +Once one reaches a certain level of complexity, it may be preferable to package +this up as a dependency and simply refer to it by fully qualified name, e.g. + +``` +- type: MapToFields + config: + language: python + fields: + new_col: + callable: pkg.module.fn +``` + +Currently, in addition to Python, Java, SQL, and JavaScript (experimental) +expressions are supported as well + +``` +- type: MapToFields + config: + language: sql + fields: + new_col: "UPPER(col1)" + another_col: "col2 + col3" +``` + +## FlatMap + +Sometimes it may be desirable to emit more (or less) than one record for each +input record. This can be accomplished by mapping to an iterable type and +following the mapping with an Explode operation, e.g. + +``` +- type: MapToFields + config: + language: python + fields: + new_col: "[col1.upper(), col1.lower(), col1.title()]" + another_col: "col2 + col3" +- type: Explode + config: + fields: new_col +``` + +will result in three output records for every input record. + +If more than one record is to be exploded, one must specify whether the cross +product over all fields should be taken. For example + +``` +- type: MapToFields + config: + language: python + fields: + new_col: "[col1.upper(), col1.lower(), col1.title()]" + another_col: "[col2 - 1, col2, col2 + 1]" +- type: Explode + config: + fields: [new_col, another_col] + cross_product: true +``` + +will emit nine records whereas + +``` +- type: MapToFields + config: + language: python + fields: + new_col: "[col1.upper(), col1.lower(), col1.title()]" + another_col: "[col2 - 1, col2, col2 + 1]" +- type: Explode + config: + fields: [new_col, another_col] + cross_product: false +``` + +will only emit three. + +The `Explode` operation can be used on its own if the field in question is +already an iterable type. + +``` +- type: Explode + config: + fields: [col1] +``` + +## Filtering + +Sometimes it can be desirable to only keep records that satisfy a certain +criteria. This can be accomplished with a `Filter` transform, e.g. + +``` +- type: Filter + config: + language: sql + keep: "col2 > 0" +``` + +## Types + +Beam will try to infer the types involved in the mappings, but sometimes this +is not possible. In these cases one can explicitly denote the expected output +type, e.g. + +``` +- type: MapToFields + config: + language: python + fields: + new_col: + expression: "col1.upper()" + output_type: string +``` + +The expected type is given in json schema notation, with the addition that +a top-level basic types may be given as a literal string rather than requiring +a `{type: 'basic_type_name'}` nesting. + +``` +- type: MapToFields + config: + language: python + fields: + new_col: + expression: "col1.upper()" + output_type: string + another_col: + expression: "beam.Row(a=col1, b=[col2])" + output_type: + type: 'object' + properties: + a: + type: 'string' + b: + type: 'array' + items: + type: 'number' +``` + +This can be especially useful to resolve errors involving the inability to +handle the `beam:logical:pythonsdk_any:v1` type. diff --git a/website/www/site/content/en/documentation/sdks/yaml.md b/website/www/site/content/en/documentation/sdks/yaml.md new file mode 100644 index 000000000000..c278c7755ba1 --- /dev/null +++ b/website/www/site/content/en/documentation/sdks/yaml.md @@ -0,0 +1,587 @@ + + +# Beam YAML API + +While Beam provides powerful APIs for authoring sophisticated data +processing pipelines, it often still has too high a barrier for +getting started and authoring simple pipelines. Even setting up the +environment, installing the dependencies, and setting up the project +can be an overwhelming amount of boilerplate for some (though +https://beam.apache.org/blog/beam-starter-projects/ has gone a long +way in making this easier). + +Here we provide a simple declarative syntax for describing pipelines +that does not require coding experience or learning how to use an +SDK—any text editor will do. +Some installation may be required to actually *execute* a pipeline, but +we envision various services (such as Dataflow) to accept yaml pipelines +directly obviating the need for even that in the future. +We also anticipate the ability to generate code directly from these +higher-level yaml descriptions, should one want to graduate to a full +Beam SDK (and possibly the other direction as well as far as possible). + +Though we intend this syntax to be easily authored (and read) directly by +humans, this may also prove a useful intermediate representation for +tools to use as well, either as output (e.g. a pipeline authoring GUI) +or consumption (e.g. a lineage analysis tool) and expect it to be more +easily manipulated and semantically meaningful than the Beam protos +themselves (which concern themselves more with execution). + +It should be noted that everything here is still under development, but any +features already included are considered stable. Feedback is welcome at +dev@apache.beam.org. + +## Running pipelines + +The Beam yaml parser is currently included as part of the Apache Beam Python SDK. +This can be installed (e.g. within a virtual environment) as + +``` +pip install apache_beam[yaml,gcp] +``` + +In addition, several of the provided transforms (such as SQL) are implemented +in Java and their expansion will require a working Java interpeter. (The +requisite artifacts will be automatically downloaded from the apache maven +repositories, so no further installs will be required.) +Docker is also currently required for local execution of these +cross-language-requiring transforms, but not for submission to a non-local +runner such as Flink or Dataflow. + +Once the prerequisites are installed, you can execute a pipeline defined +in a yaml file as + +``` +python -m apache_beam.yaml.main --yaml_pipeline_file=/path/to/pipeline.yaml [other pipeline options such as the runner] +``` + +You can do a dry-run of your pipeline using the render runner to see what the +execution graph is, e.g. + +``` +python -m apache_beam.yaml.main --yaml_pipeline_file=/path/to/pipeline.yaml --runner=apache_beam.runners.render.RenderRunner --render_output=out.png [--render_port=0] +``` + +(This requires [Graphviz](https://graphviz.org/download/) to be installed to render the pipeline.) + +We intend to support running a pipeline on Dataflow by directly passing the +yaml specification to a template, no local installation of the Beam SDKs required. + +## Example pipelines + +Here is a simple pipeline that reads some data from csv files and +writes it out in json format. + +``` +pipeline: + transforms: + - type: ReadFromCsv + config: + path: /path/to/input*.csv + - type: WriteToJson + config: + path: /path/to/output.json + input: ReadFromCsv +``` + +We can also add a transformation + +``` +pipeline: + transforms: + - type: ReadFromCsv + config: + path: /path/to/input*.csv + - type: Filter + config: + language: python + keep: "col3 > 100" + input: ReadFromCsv + - type: WriteToJson + config: + path: /path/to/output.json + input: Filter +``` + +or two. + +``` +pipeline: + transforms: + - type: ReadFromCsv + config: + path: /path/to/input*.csv + - type: Filter + config: + language: python + keep: "col3 > 100" + input: ReadFromCsv + - type: Sql + config: + query: "select col1, count(*) as cnt from PCOLLECTION group by col1" + input: Filter + - type: WriteToJson + config: + path: /path/to/output.json + input: Sql +``` + +Transforms can be named to help with monitoring and debugging. + +``` +pipeline: + transforms: + - type: ReadFromCsv + name: ReadMyData + config: + path: /path/to/input*.csv + - type: Filter + name: KeepBigRecords + config: + language: python + keep: "col3 > 100" + input: ReadMyData + - type: Sql + name: MySqlTransform + config: + query: "select col1, count(*) as cnt from PCOLLECTION group by col1" + input: KeepBigRecords + - type: WriteToJson + name: WriteTheOutput + config: + path: /path/to/output.json + input: MySqlTransform +``` + +(This is also needed to disambiguate if more than one transform of the same +type is used.) + +If the pipeline is linear, we can let the inputs be implicit by designating +the pipeline as a `chain` type. + +``` +pipeline: + type: chain + + transforms: + - type: ReadFromCsv + config: + path: /path/to/input*.csv + - type: Filter + config: + language: python + keep: "col3 > 100" + - type: Sql + name: MySqlTransform + config: + query: "select col1, count(*) as cnt from PCOLLECTION group by col1" + - type: WriteToJson + config: + path: /path/to/output.json +``` + +As syntactic sugar, we can name the first and last transforms in our pipeline +as `source` and `sink`. + +``` +pipeline: + type: chain + + source: + type: ReadFromCsv + config: + path: /path/to/input*.csv + + transforms: + - type: Filter + config: + language: python + keep: "col3 > 100" + + - type: Sql + name: MySqlTransform + config: + query: "select col1, count(*) as cnt from PCOLLECTION group by col1" + + sink: + type: WriteToJson + config: + path: /path/to/output.json +``` + +Arbitrary non-linear pipelines are supported as well, though in this case +inputs must be explicitly named. +Here we read two sources, join them, and write two outputs. + +``` +pipeline: + transforms: + - type: ReadFromCsv + name: ReadLeft + config: + path: /path/to/left*.csv + + - type: ReadFromCsv + name: ReadRight + config: + path: /path/to/right*.csv + + - type: Sql + config: + query: select A.col1, B.col2 from A join B using (col3) + input: + A: ReadLeft + B: ReadRight + + - type: WriteToJson + name: WriteAll + input: Sql + config: + path: /path/to/all.json + + - type: Filter + name: FilterToBig + input: Sql + config: + language: python + keep: "col2 > 100" + + - type: WriteToCsv + name: WriteBig + input: FilterToBig + config: + path: /path/to/big.csv +``` + +One can, however, nest `chains` within a non-linear pipeline. +For example, here `ExtraProcessingForBigRows` is itself a "chain" transform +that has a single input and contains its own sink. + +``` +pipeline: + transforms: + - type: ReadFromCsv + name: ReadLeft + config: + path: /path/to/left*.csv + + - type: ReadFromCsv + name: ReadRight + config: + path: /path/to/right*.csv + + - type: Sql + config: + query: select A.col1, B.col2 from A join B using (col3) + input: + A: ReadLeft + B: ReadRight + + - type: WriteToJson + name: WriteAll + input: Sql + config: + path: /path/to/all.json + + - type: chain + name: ExtraProcessingForBigRows + input: Sql + transforms: + - type: Filter + config: + language: python + keep: "col2 > 100" + - type: Filter + config: + language: python + keep: "len(col1) > 10" + - type: Filter + config: + language: python + keep: "col1 > 'z'" + sink: + type: WriteToCsv + config: + path: /path/to/big.csv +``` + +## Windowing + +This API can be used to define both streaming and batch pipelines. +In order to meaningfully aggregate elements in a streaming pipeline, +some kind of windowing is typically required. Beam's +[windowing](https://beam.apache.org/documentation/programming-guide/#windowing) +and [triggering](https://beam.apache.org/documentation/programming-guide/#triggers) +can be declared using the same WindowInto transform available in all other +SDKs. + +``` +pipeline: + type: chain + transforms: + - type: ReadFromPubSub + config: + topic: myPubSubTopic + format: json + schema: + type: object + properties: + col1: {type: string} + col2: {type: integer} + col3: {type: number} + - type: WindowInto + windowing: + type: fixed + size: 60s + - type: SomeGroupingTransform + config: + arg: ... + - type: WriteToPubSub + config: + topic: anotherPubSubTopic + format: json +``` + +Rather than using an explicit `WindowInto` operation, one may instead tag a +transform itself with a specified windowing which will cause its inputs +(and hence the transform itself) to be applied with that windowing. + +``` +pipeline: + type: chain + transforms: + - type: ReadFromPubSub + config: + topic: myPubSubTopic + format: ... + schema: ... + - type: SomeGroupingTransform + config: + arg: ... + windowing: + type: sliding + size: 60s + period: 10s + - type: WriteToPubSub + config: + topic: anotherPubSubTopic + format: json +``` + +Note that the `Sql` operation itself is often a from of aggregation, and +applying a windowing (or consuming an already windowed input) will cause all +grouping to be done per window. + +``` +pipeline: + type: chain + transforms: + - type: ReadFromPubSub + config: + topic: myPubSubTopic + format: ... + schema: ... + - type: Sql + config: + query: "select col1, count(*) as c from PCOLLECTION" + windowing: + type: sessions + gap: 60s + - type: WriteToPubSub + config: + topic: anotherPubSubTopic + format: json +``` + +The specified windowing is applied to all inputs, in this case resulting in +a join per window. + +``` +pipeline: + transforms: + - type: ReadFromPubSub + name: ReadLeft + config: + topic: leftTopic + format: ... + schema: ... + + - type: ReadFromPubSub + name: ReadRight + config: + topic: rightTopic + format: ... + schema: ... + + - type: Sql + config: + query: select A.col1, B.col2 from A join B using (col3) + input: + A: ReadLeft + B: ReadRight + windowing: + type: fixed + size: 60s +``` + +For a transform with no inputs, the specified windowing is instead applied to +its output(s). As per the Beam model, the windowing is then inherited by all +consuming operations. This is especially useful for root operations like Read. + +``` +pipeline: + type: chain + transforms: + - type: ReadFromPubSub + config: + topic: myPubSubTopic + format: ... + schema: ... + windowing: + type: fixed + size: 60s + - type: Sql + config: + query: "select col1, count(*) as c from PCOLLECTION" + - type: WriteToPubSub + config: + topic: anotherPubSubTopic + format: json +``` + +One can also specify windowing at the top level of a pipeline (or composite), +which is a shorthand to simply applying this same windowing to all root +operations (that don't otherwise specify their own windowing), +and can be an effective way to apply it everywhere. + +``` +pipeline: + type: chain + transforms: + - type: ReadFromPubSub + config: + topic: myPubSubTopic + format: ... + schema: ... + - type: Sql + config: + query: "select col1, count(*) as c from PCOLLECTION" + - type: WriteToPubSub + config: + topic: anotherPubSubTopic + format: json + windowing: + type: fixed + size: 60 +``` + +Note that all these windowing specifications are compatible with the `source` +and `sink` syntax as well + +``` +pipeline: + type: chain + + source: + type: ReadFromPubSub + config: + topic: myPubSubTopic + format: ... + schema: ... + windowing: + type: fixed + size: 10s + + transforms: + - type: Sql + config: + query: "select col1, count(*) as c from PCOLLECTION" + + sink: + type: WriteToCsv + config: + path: /path/to/output.json + windowing: + type: fixed + size: 5m +``` + + +## Providers + +Though we aim to offer a large suite of built-in transforms, it is inevitable +that people will want to be able to author their own. This is made possible +through the notion of Providers which leverage expansion services and +schema transforms. + +For example, one could build a jar that vends a +[cross language transform](https://beam.apache.org/documentation/sdks/python-multi-language-pipelines/) +or [schema transform](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/schemas/transforms/SchemaTransformProvider.html) +and then use it in a transform as follows + +``` +pipeline: + type: chain + source: + type: ReadFromCsv + config: + path: /path/to/input*.csv + + transforms: + - type: MyCustomTransform + config: + arg: whatever + + sink: + type: WriteToJson + config: + path: /path/to/output.json + +providers: + - type: javaJar + jar: /path/or/url/to/myExpansionService.jar + transforms: + MyCustomTransform: "urn:registered:in:expansion:service" +``` + +Arbitrary Python transforms can be provided as well, using the syntax + +``` +providers: + - type: pythonPackage + packages: + - my_pypi_package>=version + - /path/to/local/package.zip + transforms: + MyCustomTransform: "pkg.subpkg.PTransformClassOrCallable" +``` + +## Other Resources + +* [Example pipelines](https://gist.github.com/robertwb/2cb26973f1b1203e8f5f8f88c5764da0) +* [More examples](https://github.com/Polber/beam/tree/jkinard/bug-bash/sdks/python/apache_beam/yaml/examples) +* [Transform glossary](https://gist.github.com/robertwb/64e2f51ff88320eeb6ffd96634202df7) + +Additional documentation in this directory + +* [Mapping](yaml_mapping.md) +* [Aggregation](yaml_combine.md) +* [Error handling](yaml_errors.md) +* [Inlining Python](inline_python.md) diff --git a/website/www/site/data/pipelines.yaml b/website/www/site/data/pipelines.yaml index 42fcd2955648..e1dbb67c6a24 100644 --- a/website/www/site/data/pipelines.yaml +++ b/website/www/site/data/pipelines.yaml @@ -25,6 +25,9 @@ - title: Scala image_url: /images/logos/sdks/scala_pipeline.png url: /documentation/sdks/scala/ +- title: YAML + image_url: /images/logos/sdks/yaml.png + url: /documentation/sdks/yaml/ - title: SQL image_url: /images/logos/sdks/sql_pipelines.png url: /documentation/dsls/sql/overview/ diff --git a/website/www/site/layouts/partials/section-menu/en/sdks.html b/website/www/site/layouts/partials/section-menu/en/sdks.html index 73bea15a28d1..4405c9e4d3ee 100644 --- a/website/www/site/layouts/partials/section-menu/en/sdks.html +++ b/website/www/site/layouts/partials/section-menu/en/sdks.html @@ -84,6 +84,17 @@ +
  • + Yaml + +
  • +
  • SQL