From e9e94a6bfbe918a1f69154c3db90b47cc361ffbb Mon Sep 17 00:00:00 2001 From: Alexey Romanenko Date: Fri, 13 Jan 2023 12:37:42 +0100 Subject: [PATCH 01/13] Use "extensions/avro" instead of avro from"core" in Java SDK modules --- examples/java/build.gradle | 1 + .../beam/examples/complete/AutoComplete.java | 2 +- .../examples/complete/TrafficMaxLaneFlow.java | 2 +- .../beam/examples/complete/TrafficRoutes.java | 2 +- .../examples/complete/game/UserScore.java | 2 +- .../kafkatopubsub/avro/AvroDataClass.java | 2 +- .../transforms/FormatTransform.java | 2 +- .../beam/examples/snippets/Snippets.java | 2 +- .../io/gcp/bigquery/BigQueryMyData.java | 2 +- .../subprocess/utils/ExecutableFile.java | 2 +- .../complete/game/LeaderBoardTest.java | 2 +- .../complete/game/StatefulTeamScoreTest.java | 2 +- examples/kotlin/build.gradle | 1 + .../beam/examples/kotlin/snippets/Snippets.kt | 2 +- .../apache/beam/sdk/schemas/io/Providers.java | 30 +++++++++++++++---- sdks/java/extensions/sketching/build.gradle | 1 + .../sketching/ApproximateDistinctTest.java | 2 +- .../sketching/SketchFrequenciesTest.java | 2 +- sdks/java/extensions/sql/build.gradle | 1 + .../meta/provider/avro/AvroTableProvider.java | 4 +-- .../meta/provider/parquet/ParquetTable.java | 2 +- .../kafka/BeamKafkaTableAvroTest.java | 4 +-- .../provider/kafka/KafkaTableProviderIT.java | 2 +- .../pubsub/PubsubTableProviderIT.java | 4 +-- sdks/java/io/amazon-web-services/build.gradle | 1 + .../io/aws/dynamodb/DynamoDBIOWriteTest.java | 2 +- .../java/io/amazon-web-services2/build.gradle | 1 + .../io/aws2/dynamodb/DynamoDBIOWriteTest.java | 2 +- sdks/java/io/cdap/build.gradle | 1 + .../beam/sdk/io/cdap/TestRowDBWritable.java | 2 +- sdks/java/io/file-based-io-tests/build.gradle | 1 + .../org/apache/beam/sdk/io/avro/AvroIOIT.java | 4 +-- .../beam/sdk/io/parquet/ParquetIOIT.java | 2 +- sdks/java/io/hadoop-format/build.gradle | 1 + .../beam/sdk/io/hadoop/format/Employee.java | 2 +- .../hadoop/format/HadoopFormatIOReadTest.java | 2 +- .../io/hadoop/format/TestRowDBWritable.java | 2 +- sdks/java/io/influxdb/build.gradle | 1 + .../apache/beam/sdk/io/influxdb/Model.java | 2 +- sdks/java/io/jdbc/build.gradle | 1 + .../beam/sdk/io/jdbc/SchemaUtilTest.java | 2 +- sdks/java/io/parquet/build.gradle | 2 ++ .../apache/beam/sdk/io/parquet/ParquetIO.java | 4 +-- .../beam/sdk/io/parquet/ParquetIOTest.java | 4 +-- sdks/java/io/snowflake/build.gradle | 1 + .../test/unit/read/SnowflakeIOReadTest.java | 2 +- .../testing/expansion-service/build.gradle | 1 + .../expansion/TestExpansionService.java | 2 +- sdks/java/testing/nexmark/build.gradle | 1 + .../beam/sdk/nexmark/NexmarkLauncher.java | 2 +- .../apache/beam/sdk/nexmark/NexmarkUtils.java | 2 +- 51 files changed, 81 insertions(+), 47 deletions(-) diff --git a/examples/java/build.gradle b/examples/java/build.gradle index aa51dcfeae85..994001c573f7 100644 --- a/examples/java/build.gradle +++ b/examples/java/build.gradle @@ -56,6 +56,7 @@ dependencies { implementation library.java.vendored_guava_26_0_jre implementation library.java.kafka_clients implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation project(":sdks:java:extensions:avro") implementation project(":sdks:java:extensions:google-cloud-platform-core") implementation project(":sdks:java:extensions:python") implementation project(":sdks:java:io:google-cloud-platform") diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java index 449f19cbf75a..99f509f9f7b8 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java @@ -39,8 +39,8 @@ import org.apache.beam.examples.common.ExampleUtils; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; import org.apache.beam.sdk.io.gcp.datastore.DatastoreIO; diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java index 7d39bdb5e2b6..d3e5144d5338 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java @@ -49,8 +49,8 @@ import org.apache.beam.examples.common.ExampleUtils; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; import org.apache.beam.sdk.options.Default; diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java index d3f66e5ff3c6..4705d461f3c9 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java @@ -55,8 +55,8 @@ import org.apache.beam.examples.common.ExampleUtils; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; import org.apache.beam.sdk.options.Default; diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/game/UserScore.java b/examples/java/src/main/java/org/apache/beam/examples/complete/game/UserScore.java index b1aedacecfbd..8baa82cad8d9 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/game/UserScore.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/game/UserScore.java @@ -23,8 +23,8 @@ import org.apache.avro.reflect.Nullable; import org.apache.beam.examples.complete.game.utils.WriteToText; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/avro/AvroDataClass.java b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/avro/AvroDataClass.java index 8c8702115f65..f7c01e70c632 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/avro/AvroDataClass.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/avro/AvroDataClass.java @@ -17,8 +17,8 @@ */ package org.apache.beam.examples.complete.kafkatopubsub.avro; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; /** * Example of AVRO serialization class. To configure your AVRO schema, change this class to diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/transforms/FormatTransform.java b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/transforms/FormatTransform.java index d493c0648180..1dcf9e196221 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/transforms/FormatTransform.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/transforms/FormatTransform.java @@ -23,10 +23,10 @@ import org.apache.beam.examples.complete.kafkatopubsub.avro.AvroDataClassKafkaAvroDeserializer; import org.apache.beam.examples.complete.kafkatopubsub.kafka.consumer.SslConsumerFactoryFn; import org.apache.beam.examples.complete.kafkatopubsub.options.KafkaToPubsubOptions; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.NullableCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO; import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage; import org.apache.beam.sdk.io.kafka.KafkaIO; diff --git a/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java b/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java index 274f48f11bb7..cf6a968c03be 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java +++ b/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java @@ -45,10 +45,10 @@ import javax.annotation.Nullable; import org.apache.avro.generic.GenericRecord; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.DefaultCoder; import org.apache.beam.sdk.coders.DoubleCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.extensions.ml.AnnotateText; import org.apache.beam.sdk.io.Compression; import org.apache.beam.sdk.io.FileIO; diff --git a/examples/java/src/main/java/org/apache/beam/examples/snippets/transforms/io/gcp/bigquery/BigQueryMyData.java b/examples/java/src/main/java/org/apache/beam/examples/snippets/transforms/io/gcp/bigquery/BigQueryMyData.java index 5641415d7a4f..03b1b4c41425 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/snippets/transforms/io/gcp/bigquery/BigQueryMyData.java +++ b/examples/java/src/main/java/org/apache/beam/examples/snippets/transforms/io/gcp/bigquery/BigQueryMyData.java @@ -30,8 +30,8 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; @SuppressWarnings({ "nullness" // TODO(https://github.com/apache/beam/issues/20497) diff --git a/examples/java/src/main/java/org/apache/beam/examples/subprocess/utils/ExecutableFile.java b/examples/java/src/main/java/org/apache/beam/examples/subprocess/utils/ExecutableFile.java index 036b3d25ca09..1b157d237429 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/subprocess/utils/ExecutableFile.java +++ b/examples/java/src/main/java/org/apache/beam/examples/subprocess/utils/ExecutableFile.java @@ -18,8 +18,8 @@ package org.apache.beam.examples.subprocess.utils; import org.apache.beam.examples.subprocess.configuration.SubProcessConfiguration; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; /** Contains the configuration for the external library. */ @DefaultCoder(AvroCoder.class) diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java index e48c2471cea7..fc449909e139 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java @@ -24,7 +24,7 @@ import org.apache.beam.examples.complete.game.LeaderBoard.CalculateTeamScores; import org.apache.beam.examples.complete.game.LeaderBoard.CalculateUserScores; import org.apache.beam.examples.complete.game.UserScore.GameActionInfo; -import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/game/StatefulTeamScoreTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/game/StatefulTeamScoreTest.java index 6d4c964d8fbc..d94d74d2a85b 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/complete/game/StatefulTeamScoreTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/complete/game/StatefulTeamScoreTest.java @@ -19,9 +19,9 @@ import org.apache.beam.examples.complete.game.StatefulTeamScore.UpdateTeamScoreFn; import org.apache.beam.examples.complete.game.UserScore.GameActionInfo; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestStream; diff --git a/examples/kotlin/build.gradle b/examples/kotlin/build.gradle index 79a1248712d0..6fb3ef64e282 100644 --- a/examples/kotlin/build.gradle +++ b/examples/kotlin/build.gradle @@ -55,6 +55,7 @@ dependencies { // Add the dependency that sdks:java:core that is marked as provided implementation library.java.hamcrest permitUnusedDeclared library.java.hamcrest + implementation project(":sdks:java:extensions:avro") implementation project(":sdks:java:extensions:google-cloud-platform-core") implementation project(":sdks:java:io:google-cloud-platform") implementation library.java.avro diff --git a/examples/kotlin/src/main/java/org/apache/beam/examples/kotlin/snippets/Snippets.kt b/examples/kotlin/src/main/java/org/apache/beam/examples/kotlin/snippets/Snippets.kt index d2e2a5883371..2ba7b3742e16 100644 --- a/examples/kotlin/src/main/java/org/apache/beam/examples/kotlin/snippets/Snippets.kt +++ b/examples/kotlin/src/main/java/org/apache/beam/examples/kotlin/snippets/Snippets.kt @@ -23,7 +23,7 @@ import com.google.api.services.bigquery.model.* import com.google.common.collect.ImmutableList import edu.umd.cs.findbugs.annotations.SuppressFBWarnings import org.apache.beam.sdk.Pipeline -import org.apache.beam.sdk.coders.AvroCoder +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder import org.apache.beam.sdk.coders.DefaultCoder import org.apache.beam.sdk.coders.DoubleCoder import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/io/Providers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/io/Providers.java index dc0f758b4aba..cc1e01b9a4ae 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/io/Providers.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/io/Providers.java @@ -42,12 +42,30 @@ private Providers() {} public static Map loadProviders(Class klass) { Map providers = new HashMap<>(); for (T provider : ServiceLoader.load(klass)) { - checkArgument( - !providers.containsKey(provider.identifier()), - "Duplicate providers exist with identifier `%s` for class %s.", - provider.identifier(), - klass); - providers.put(provider.identifier(), provider); + // Avro provider is treated as a special case until two providers may exist: in "core" + // (deprecated) and in "extensions/avro" (actual). + if (provider.identifier().equals("avro")) { + // Avro provider from "extensions/avro" must have a priority. + if (provider + .toString() + .startsWith( + "org.apache.beam.sdk.extensions.avro.schemas.io.payloads.AvroPayloadSerializerProvider")) { + // Use AvroPayloadSerializerProvider from extensions/avro by any case. + providers.put(provider.identifier(), provider); + } else { + // Load Avro provider from "core" if it was not loaded from Avro extension before. + if (!providers.containsKey(provider.identifier())) { + providers.put(provider.identifier(), provider); + } + } + } else { + checkArgument( + !providers.containsKey(provider.identifier()), + "Duplicate providers exist with identifier `%s` for class %s.", + provider.identifier(), + klass); + providers.put(provider.identifier(), provider); + } } return providers; } diff --git a/sdks/java/extensions/sketching/build.gradle b/sdks/java/extensions/sketching/build.gradle index 1060804713d6..b239771ad274 100644 --- a/sdks/java/extensions/sketching/build.gradle +++ b/sdks/java/extensions/sketching/build.gradle @@ -32,6 +32,7 @@ dependencies { implementation "com.tdunning:t-digest:$tdigest_version" testImplementation library.java.avro testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") + testImplementation project(path: ":sdks:java:extensions:avro", configuration: "testRuntimeMigration") testImplementation library.java.junit testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") } diff --git a/sdks/java/extensions/sketching/src/test/java/org/apache/beam/sdk/extensions/sketching/ApproximateDistinctTest.java b/sdks/java/extensions/sketching/src/test/java/org/apache/beam/sdk/extensions/sketching/ApproximateDistinctTest.java index 2d9933955d31..0cb3e0e5116d 100644 --- a/sdks/java/extensions/sketching/src/test/java/org/apache/beam/sdk/extensions/sketching/ApproximateDistinctTest.java +++ b/sdks/java/extensions/sketching/src/test/java/org/apache/beam/sdk/extensions/sketching/ApproximateDistinctTest.java @@ -29,8 +29,8 @@ import org.apache.avro.SchemaBuilder; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.extensions.sketching.ApproximateDistinct.ApproximateDistinctFn; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.testing.PAssert; diff --git a/sdks/java/extensions/sketching/src/test/java/org/apache/beam/sdk/extensions/sketching/SketchFrequenciesTest.java b/sdks/java/extensions/sketching/src/test/java/org/apache/beam/sdk/extensions/sketching/SketchFrequenciesTest.java index 0f19f81ca890..2a37be8ae193 100644 --- a/sdks/java/extensions/sketching/src/test/java/org/apache/beam/sdk/extensions/sketching/SketchFrequenciesTest.java +++ b/sdks/java/extensions/sketching/src/test/java/org/apache/beam/sdk/extensions/sketching/SketchFrequenciesTest.java @@ -29,9 +29,9 @@ import org.apache.avro.SchemaBuilder; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.extensions.sketching.SketchFrequencies.CountMinSketchFn; import org.apache.beam.sdk.extensions.sketching.SketchFrequencies.Sketch; import org.apache.beam.sdk.testing.CoderProperties; diff --git a/sdks/java/extensions/sql/build.gradle b/sdks/java/extensions/sql/build.gradle index 4f9efec51789..de692d2d651e 100644 --- a/sdks/java/extensions/sql/build.gradle +++ b/sdks/java/extensions/sql/build.gradle @@ -74,6 +74,7 @@ dependencies { fmppTask "org.freemarker:freemarker:2.3.31" fmppTemplates library.java.vendored_calcite_1_28_0 implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation project(":sdks:java:extensions:avro") implementation project(":sdks:java:extensions:join-library") permitUnusedDeclared project(":sdks:java:extensions:join-library") // BEAM-11761 implementation project(":sdks:java:extensions:sql:udf") diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/avro/AvroTableProvider.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/avro/AvroTableProvider.java index 4df6a2b2795c..fb6855a3fb7c 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/avro/AvroTableProvider.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/avro/AvroTableProvider.java @@ -18,10 +18,10 @@ package org.apache.beam.sdk.extensions.sql.meta.provider.avro; import com.google.auto.service.AutoService; +import org.apache.beam.sdk.extensions.avro.io.AvroIO; +import org.apache.beam.sdk.extensions.avro.io.AvroSchemaIOProvider; import org.apache.beam.sdk.extensions.sql.meta.provider.SchemaIOTableProviderWrapper; import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; -import org.apache.beam.sdk.io.AvroIO; -import org.apache.beam.sdk.io.AvroSchemaIOProvider; import org.apache.beam.sdk.schemas.io.SchemaIOProvider; /** diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/parquet/ParquetTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/parquet/ParquetTable.java index c38ed0ccf68e..88d162c02370 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/parquet/ParquetTable.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/parquet/ParquetTable.java @@ -25,6 +25,7 @@ import org.apache.avro.Schema.Field; import org.apache.avro.generic.GenericRecord; import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTableFilter; import org.apache.beam.sdk.extensions.sql.meta.ProjectSupport; import org.apache.beam.sdk.extensions.sql.meta.SchemaBaseBeamTable; @@ -33,7 +34,6 @@ import org.apache.beam.sdk.io.parquet.ParquetIO; import org.apache.beam.sdk.io.parquet.ParquetIO.Read; import org.apache.beam.sdk.schemas.transforms.Convert; -import org.apache.beam.sdk.schemas.utils.AvroUtils; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTableAvroTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTableAvroTest.java index e784865eab89..e468d5b9eef3 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTableAvroTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTableAvroTest.java @@ -23,10 +23,10 @@ import java.util.List; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecordBuilder; -import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.extensions.sql.meta.Table; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.utils.AvroUtils; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProviderIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProviderIT.java index 21f0d4fb3e16..34b7efabb880 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProviderIT.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProviderIT.java @@ -37,6 +37,7 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.extensions.protobuf.PayloadMessages; import org.apache.beam.sdk.extensions.protobuf.ProtoMessageSchema; import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv; @@ -52,7 +53,6 @@ import org.apache.beam.sdk.options.Validation; import org.apache.beam.sdk.schemas.RowMessages; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.utils.AvroUtils; import org.apache.beam.sdk.state.BagState; import org.apache.beam.sdk.state.StateSpec; import org.apache.beam.sdk.state.StateSpecs; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubTableProviderIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubTableProviderIT.java index 7bd872e7c510..2d79dcb0fff3 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubTableProviderIT.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubTableProviderIT.java @@ -49,7 +49,8 @@ import java.util.stream.Collectors; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecordBuilder; -import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.extensions.protobuf.PayloadMessages; import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv; @@ -66,7 +67,6 @@ import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.FieldType; import org.apache.beam.sdk.schemas.SchemaCoder; -import org.apache.beam.sdk.schemas.utils.AvroUtils; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.util.common.ReflectHelpers; import org.apache.beam.sdk.values.PCollection; diff --git a/sdks/java/io/amazon-web-services/build.gradle b/sdks/java/io/amazon-web-services/build.gradle index 23fd55225589..9cc453572e1a 100644 --- a/sdks/java/io/amazon-web-services/build.gradle +++ b/sdks/java/io/amazon-web-services/build.gradle @@ -51,6 +51,7 @@ dependencies { runtimeOnly library.java.commons_codec runtimeOnly "org.apache.httpcomponents:httpclient:4.5.12" 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") testImplementation "io.findify:s3mock_2.12:0.2.6" testImplementation library.java.hamcrest diff --git a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/dynamodb/DynamoDBIOWriteTest.java b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/dynamodb/DynamoDBIOWriteTest.java index f0813ef4eca0..92f31977d8b2 100644 --- a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/dynamodb/DynamoDBIOWriteTest.java +++ b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/dynamodb/DynamoDBIOWriteTest.java @@ -50,8 +50,8 @@ import java.util.stream.IntStream; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.aws.dynamodb.DynamoDBIO.RetryConfiguration; import org.apache.beam.sdk.io.aws.dynamodb.DynamoDBIO.Write.WriteFn; import org.apache.beam.sdk.testing.ExpectedLogs; diff --git a/sdks/java/io/amazon-web-services2/build.gradle b/sdks/java/io/amazon-web-services2/build.gradle index 5b25cde8f0e0..95d0649d8306 100644 --- a/sdks/java/io/amazon-web-services2/build.gradle +++ b/sdks/java/io/amazon-web-services2/build.gradle @@ -66,6 +66,7 @@ dependencies { implementation library.java.commons_lang3 implementation library.java.commons_codec 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") testImplementation "io.findify:s3mock_2.12:0.2.6" testImplementation 'org.elasticmq:elasticmq-rest-sqs_2.12:1.3.5' diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDBIOWriteTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDBIOWriteTest.java index 64cb3f44692f..b410ec916d6d 100644 --- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDBIOWriteTest.java +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDBIOWriteTest.java @@ -41,8 +41,8 @@ import java.util.function.Supplier; import java.util.stream.IntStream; import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.aws2.MockClientBuilderFactory; import org.apache.beam.sdk.io.aws2.common.ClientConfiguration; import org.apache.beam.sdk.io.aws2.common.RetryConfiguration; diff --git a/sdks/java/io/cdap/build.gradle b/sdks/java/io/cdap/build.gradle index 0a1e12a87e14..a9122d1c8f6d 100644 --- a/sdks/java/io/cdap/build.gradle +++ b/sdks/java/io/cdap/build.gradle @@ -69,6 +69,7 @@ dependencies { testImplementation library.java.junit testImplementation library.java.mockito_core testImplementation library.java.testcontainers_postgresql + testImplementation project(path: ":sdks:java:extensions:avro", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:io:hadoop-common", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:io:hadoop-format", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:testing:test-utils", configuration: "testRuntimeMigration") diff --git a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/TestRowDBWritable.java b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/TestRowDBWritable.java index d85c5ea3e69e..ef17d1601dd7 100644 --- a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/TestRowDBWritable.java +++ b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/TestRowDBWritable.java @@ -23,8 +23,8 @@ import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.SQLException; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.common.TestRow; import org.apache.hadoop.io.Writable; import org.apache.hadoop.mapreduce.lib.db.DBWritable; diff --git a/sdks/java/io/file-based-io-tests/build.gradle b/sdks/java/io/file-based-io-tests/build.gradle index 3a23d6ceeabc..14abdbfbae26 100644 --- a/sdks/java/io/file-based-io-tests/build.gradle +++ b/sdks/java/io/file-based-io-tests/build.gradle @@ -26,6 +26,7 @@ ext.summary = "Integration tests for reading/writing using file-based sources/si dependencies { 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") testImplementation project(path: ":sdks:java:io:xml", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:io:parquet", configuration: "testRuntimeMigration") diff --git a/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/avro/AvroIOIT.java b/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/avro/AvroIOIT.java index 832c69af1c44..684cf4d3b639 100644 --- a/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/avro/AvroIOIT.java +++ b/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/avro/AvroIOIT.java @@ -31,8 +31,8 @@ import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecordBuilder; import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.io.AvroIO; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; +import org.apache.beam.sdk.extensions.avro.io.AvroIO; import org.apache.beam.sdk.io.FileIO; import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.io.common.FileBasedIOITHelper; diff --git a/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/parquet/ParquetIOIT.java b/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/parquet/ParquetIOIT.java index 7db703b7b02b..0526b5d66d7b 100644 --- a/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/parquet/ParquetIOIT.java +++ b/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/parquet/ParquetIOIT.java @@ -31,7 +31,7 @@ import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecordBuilder; import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.FileIO; import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.io.common.FileBasedIOITHelper; diff --git a/sdks/java/io/hadoop-format/build.gradle b/sdks/java/io/hadoop-format/build.gradle index 2a920de60fa5..8b938bdc27b6 100644 --- a/sdks/java/io/hadoop-format/build.gradle +++ b/sdks/java/io/hadoop-format/build.gradle @@ -69,6 +69,7 @@ dependencies { // 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") testImplementation project(path: ":sdks:java:testing:test-utils", configuration: "testRuntimeMigration") testImplementation project(":sdks:java:io:jdbc") diff --git a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/Employee.java b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/Employee.java index fe2cda132520..58ff2005594e 100644 --- a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/Employee.java +++ b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/Employee.java @@ -18,8 +18,8 @@ package org.apache.beam.sdk.io.hadoop.format; import java.util.Objects; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.checkerframework.checker.nullness.qual.Nullable; /** diff --git a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOReadTest.java b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOReadTest.java index be3191dfeab9..da12e9d04142 100644 --- a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOReadTest.java +++ b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOReadTest.java @@ -32,11 +32,11 @@ import java.util.ArrayList; import java.util.List; import java.util.stream.Collectors; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.NullableCoder; import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.BoundedSource.BoundedReader; import org.apache.beam.sdk.io.hadoop.SerializableConfiguration; diff --git a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/TestRowDBWritable.java b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/TestRowDBWritable.java index 2d10bdbd269d..f8ef1e71d15a 100644 --- a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/TestRowDBWritable.java +++ b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/TestRowDBWritable.java @@ -23,8 +23,8 @@ import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.SQLException; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.common.TestRow; import org.apache.hadoop.io.Writable; import org.apache.hadoop.mapreduce.lib.db.DBWritable; diff --git a/sdks/java/io/influxdb/build.gradle b/sdks/java/io/influxdb/build.gradle index f890f35a8b09..cca0d01cff05 100644 --- a/sdks/java/io/influxdb/build.gradle +++ b/sdks/java/io/influxdb/build.gradle @@ -34,5 +34,6 @@ dependencies { testImplementation library.java.powermock_mockito testImplementation library.java.mockito_core testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") + testImplementation project(path: ":sdks:java:extensions:avro", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:io:common", configuration: "testRuntimeMigration") } diff --git a/sdks/java/io/influxdb/src/test/java/org/apache/beam/sdk/io/influxdb/Model.java b/sdks/java/io/influxdb/src/test/java/org/apache/beam/sdk/io/influxdb/Model.java index f8eec6f2d029..957ce4837289 100644 --- a/sdks/java/io/influxdb/src/test/java/org/apache/beam/sdk/io/influxdb/Model.java +++ b/sdks/java/io/influxdb/src/test/java/org/apache/beam/sdk/io/influxdb/Model.java @@ -20,8 +20,8 @@ import java.util.HashMap; import java.util.Map; import java.util.concurrent.TimeUnit; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.influxdb.dto.Point; @DefaultCoder(AvroCoder.class) diff --git a/sdks/java/io/jdbc/build.gradle b/sdks/java/io/jdbc/build.gradle index 71f7ff363877..379b073eb786 100644 --- a/sdks/java/io/jdbc/build.gradle +++ b/sdks/java/io/jdbc/build.gradle @@ -37,6 +37,7 @@ dependencies { testImplementation "org.apache.derby:derbyclient:10.14.2.0" testImplementation "org.apache.derby:derbynet:10.14.2.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") testImplementation project(path: ":sdks:java:testing:test-utils", configuration: "testRuntimeMigration") testImplementation library.java.junit diff --git a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/SchemaUtilTest.java b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/SchemaUtilTest.java index 9d0770e2704d..080a451d706f 100644 --- a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/SchemaUtilTest.java +++ b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/SchemaUtilTest.java @@ -37,8 +37,8 @@ import java.sql.Types; import java.time.ZoneId; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.utils.AvroUtils; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; import org.joda.time.DateTime; diff --git a/sdks/java/io/parquet/build.gradle b/sdks/java/io/parquet/build.gradle index 2c46a41120f8..bb33c8b9ba3a 100644 --- a/sdks/java/io/parquet/build.gradle +++ b/sdks/java/io/parquet/build.gradle @@ -40,6 +40,7 @@ def parquet_version = "1.12.0" dependencies { implementation library.java.vendored_guava_26_0_jre implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation project(":sdks:java:extensions:avro") implementation project(":sdks:java:io:hadoop-common") implementation library.java.slf4j_api implementation "org.apache.parquet:parquet-avro:$parquet_version" @@ -52,6 +53,7 @@ dependencies { provided library.java.hadoop_common testImplementation library.java.hadoop_client testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") + testImplementation project(path: ":sdks:java:extensions:avro", configuration: "testRuntimeMigration") testImplementation library.java.junit testRuntimeOnly library.java.slf4j_jdk14 testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") diff --git a/sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java b/sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java index 433a53a20fe1..8a675e2c20d2 100644 --- a/sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java +++ b/sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java @@ -39,18 +39,18 @@ import org.apache.avro.specific.SpecificData; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.io.FileIO; import org.apache.beam.sdk.io.FileIO.ReadableFile; import org.apache.beam.sdk.io.hadoop.SerializableConfiguration; import org.apache.beam.sdk.io.parquet.ParquetIO.ReadFiles.SplitReadFn; import org.apache.beam.sdk.io.range.OffsetRange; import org.apache.beam.sdk.options.ValueProvider; -import org.apache.beam.sdk.schemas.utils.AvroUtils; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; diff --git a/sdks/java/io/parquet/src/test/java/org/apache/beam/sdk/io/parquet/ParquetIOTest.java b/sdks/java/io/parquet/src/test/java/org/apache/beam/sdk/io/parquet/ParquetIOTest.java index 6dd67e3e511c..7ee3ec5050fd 100644 --- a/sdks/java/io/parquet/src/test/java/org/apache/beam/sdk/io/parquet/ParquetIOTest.java +++ b/sdks/java/io/parquet/src/test/java/org/apache/beam/sdk/io/parquet/ParquetIOTest.java @@ -40,12 +40,12 @@ import org.apache.avro.io.EncoderFactory; import org.apache.avro.io.JsonEncoder; import org.apache.avro.reflect.ReflectData; -import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.io.FileIO; import org.apache.beam.sdk.io.parquet.ParquetIO.GenericRecordPassthroughFn; import org.apache.beam.sdk.io.range.OffsetRange; import org.apache.beam.sdk.schemas.SchemaCoder; -import org.apache.beam.sdk.schemas.utils.AvroUtils; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; diff --git a/sdks/java/io/snowflake/build.gradle b/sdks/java/io/snowflake/build.gradle index 59115cef9695..ddb66118241f 100644 --- a/sdks/java/io/snowflake/build.gradle +++ b/sdks/java/io/snowflake/build.gradle @@ -36,6 +36,7 @@ dependencies { implementation "org.bouncycastle:bcprov-jdk15on:1.70" implementation library.java.joda_time 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") testImplementation project(path: ":sdks:java:testing:test-utils", configuration: "testRuntimeMigration") testImplementation 'com.google.cloud:google-cloud-storage:1.102.0' diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/read/SnowflakeIOReadTest.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/read/SnowflakeIOReadTest.java index 10403fda2857..d41e2032e287 100644 --- a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/read/SnowflakeIOReadTest.java +++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/read/SnowflakeIOReadTest.java @@ -23,7 +23,7 @@ import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecordBuilder; import org.apache.beam.sdk.Pipeline.PipelineExecutionException; -import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.AvroGeneratedUser; import org.apache.beam.sdk.io.snowflake.SnowflakeIO; import org.apache.beam.sdk.io.snowflake.services.SnowflakeServices; diff --git a/sdks/java/testing/expansion-service/build.gradle b/sdks/java/testing/expansion-service/build.gradle index 241c107462e9..47b9d6df926e 100644 --- a/sdks/java/testing/expansion-service/build.gradle +++ b/sdks/java/testing/expansion-service/build.gradle @@ -31,6 +31,7 @@ dependencies { testImplementation project(path: ":sdks:java:core", configuration: "shadow") testImplementation project(":sdks:java:io:parquet") testImplementation project(":sdks:java:expansion-service") + testImplementation project(path: ":sdks:java:extensions:avro", configuration: "testRuntimeMigration") testRuntimeOnly project(":sdks:java:extensions:google-cloud-platform-core") testRuntimeOnly library.java.hadoop_client } 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 ebf9f5812b13..821d0e9db9ae 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 @@ -26,9 +26,9 @@ import org.apache.avro.generic.GenericRecord; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.expansion.ExternalTransformRegistrar; import org.apache.beam.sdk.expansion.service.ExpansionService; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.FileIO; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.parquet.ParquetIO; diff --git a/sdks/java/testing/nexmark/build.gradle b/sdks/java/testing/nexmark/build.gradle index 86cf7a5ee3c7..8b989c258ed7 100644 --- a/sdks/java/testing/nexmark/build.gradle +++ b/sdks/java/testing/nexmark/build.gradle @@ -67,6 +67,7 @@ dependencies { implementation library.java.vendored_guava_26_0_jre implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(":sdks:java:io:google-cloud-platform") + implementation project(":sdks:java:extensions:avro") implementation project(":sdks:java:extensions:google-cloud-platform-core") implementation project(":sdks:java:extensions:sql") implementation project(":sdks:java:extensions:sql:zetasql") diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkLauncher.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkLauncher.java index e9afc629315b..fd8563630832 100644 --- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkLauncher.java +++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkLauncher.java @@ -36,7 +36,7 @@ import java.util.concurrent.ThreadLocalRandom; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.io.AvroIO; +import org.apache.beam.sdk.extensions.avro.io.AvroIO; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO; diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkUtils.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkUtils.java index 896c3eab75e1..3d8985df3fab 100644 --- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkUtils.java +++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkUtils.java @@ -31,13 +31,13 @@ import java.util.List; import java.util.stream.Collectors; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.io.Read; From e6ea8930fc3cb49da5b60e55ae3a32965e92c652 Mon Sep 17 00:00:00 2001 From: Alexey Romanenko Date: Tue, 17 Jan 2023 10:32:16 +0100 Subject: [PATCH 02/13] Use "extensions/avro" instead of avro from"core" in sdks/java/io/kafka --- sdks/java/io/kafka/build.gradle | 2 ++ .../io/kafka/ConfluentSchemaRegistryDeserializerProvider.java | 2 +- .../org/apache/beam/sdk/io/kafka/KafkaCheckpointMark.java | 2 +- .../src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java | 2 +- .../beam/sdk/io/kafka/KafkaReadSchemaTransformProvider.java | 4 ++-- .../org/apache/beam/sdk/io/kafka/KafkaUnboundedSource.java | 2 +- .../beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java | 2 +- .../ConfluentSchemaRegistryDeserializerProviderTest.java | 2 +- .../src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java | 2 +- 9 files changed, 11 insertions(+), 9 deletions(-) diff --git a/sdks/java/io/kafka/build.gradle b/sdks/java/io/kafka/build.gradle index 8d64cf0bf971..065c8b83013c 100644 --- a/sdks/java/io/kafka/build.gradle +++ b/sdks/java/io/kafka/build.gradle @@ -59,6 +59,7 @@ dependencies { provided library.java.jackson_dataformat_csv permitUnusedDeclared library.java.jackson_dataformat_csv implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation project(":sdks:java:extensions:avro") implementation project(":runners:core-construction-java") implementation project(":sdks:java:expansion-service") permitUnusedDeclared project(":sdks:java:expansion-service") // BEAM-11761 @@ -90,6 +91,7 @@ dependencies { provided library.java.everit_json_schema testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") testImplementation project(":sdks:java:io:synthetic") + testImplementation project(path: ":sdks:java:extensions:avro", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:io:common", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:testing:test-utils", configuration: "testRuntimeMigration") // For testing Cross-language transforms diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ConfluentSchemaRegistryDeserializerProvider.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ConfluentSchemaRegistryDeserializerProvider.java index 68a0b0522e9f..85c93c863ad2 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ConfluentSchemaRegistryDeserializerProvider.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ConfluentSchemaRegistryDeserializerProvider.java @@ -31,9 +31,9 @@ import org.apache.avro.Schema; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; import org.apache.kafka.common.serialization.Deserializer; diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaCheckpointMark.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaCheckpointMark.java index 0b34262864aa..8e34cfee3d29 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaCheckpointMark.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaCheckpointMark.java @@ -21,8 +21,8 @@ import java.util.List; import java.util.Optional; import org.apache.avro.reflect.AvroIgnore; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Joiner; diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java index 26c895ddc89f..9bfd4723f6c8 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java @@ -44,7 +44,6 @@ import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.coders.AtomicCoder; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; @@ -54,6 +53,7 @@ import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.expansion.ExternalTransformRegistrar; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.Read.Unbounded; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark; diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProvider.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProvider.java index 89c8d986ac1b..86d7c763206c 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProvider.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProvider.java @@ -21,13 +21,13 @@ import java.util.List; import java.util.Objects; import org.apache.avro.generic.GenericRecord; -import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.transforms.Convert; 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.AvroUtils; import org.apache.beam.sdk.schemas.utils.JsonUtils; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedSource.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedSource.java index d35e1c60f5ee..4af13bbf4749 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedSource.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedSource.java @@ -24,8 +24,8 @@ import java.util.Collections; import java.util.Comparator; import java.util.List; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.io.kafka.KafkaIO.Read; import org.apache.beam.sdk.options.PipelineOptions; diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java index 91dd6538e0ed..af7211f6a221 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java @@ -26,13 +26,13 @@ import java.util.Map; import java.util.Set; import javax.annotation.Nullable; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; 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.AvroUtils; import org.apache.beam.sdk.schemas.utils.JsonUtils; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ConfluentSchemaRegistryDeserializerProviderTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ConfluentSchemaRegistryDeserializerProviderTest.java index 4f99fac88680..ee276ae88489 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ConfluentSchemaRegistryDeserializerProviderTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ConfluentSchemaRegistryDeserializerProviderTest.java @@ -29,8 +29,8 @@ import java.util.Map; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecordBuilder; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.kafka.common.serialization.Serializer; import org.junit.Test; diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java index 8f6dc10a95da..a1d5baf49267 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java @@ -40,6 +40,7 @@ import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.io.common.HashingFn; @@ -56,7 +57,6 @@ import org.apache.beam.sdk.options.StreamingOptions; import org.apache.beam.sdk.options.Validation; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.utils.AvroUtils; import org.apache.beam.sdk.schemas.utils.JsonUtils; import org.apache.beam.sdk.testing.ExpectedLogs; import org.apache.beam.sdk.testing.PAssert; From aececbae09c9d5dcd4bc7555290f908f725543fe Mon Sep 17 00:00:00 2001 From: Alexey Romanenko Date: Tue, 17 Jan 2023 15:50:17 +0100 Subject: [PATCH 03/13] Use "extensions/avro" instead of avro from"core" in sdks/java/io/google-cloud-platform --- sdks/java/io/google-cloud-platform/build.gradle | 2 ++ .../java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java | 2 +- .../apache/beam/sdk/io/gcp/bigquery/BigQueryQuerySource.java | 2 +- .../beam/sdk/io/gcp/bigquery/BigQueryQuerySourceDef.java | 2 +- .../apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java | 2 +- .../apache/beam/sdk/io/gcp/bigquery/BigQuerySourceDef.java | 2 +- .../beam/sdk/io/gcp/bigquery/BigQueryStorageArrowReader.java | 2 +- .../beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBase.java | 2 +- .../apache/beam/sdk/io/gcp/bigquery/BigQueryTableSource.java | 2 +- .../beam/sdk/io/gcp/bigquery/BigQueryTableSourceDef.java | 2 +- .../java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClient.java | 2 +- .../main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java | 4 ++-- .../gcp/pubsublite/PubsubLiteReadSchemaTransformProvider.java | 2 +- .../pubsublite/PubsubLiteWriteSchemaTransformProvider.java | 2 +- .../changestreams/model/ChangeStreamRecordMetadata.java | 2 +- .../io/gcp/spanner/changestreams/model/ChildPartition.java | 2 +- .../spanner/changestreams/model/ChildPartitionsRecord.java | 2 +- .../sdk/io/gcp/spanner/changestreams/model/ColumnType.java | 2 +- .../io/gcp/spanner/changestreams/model/DataChangeRecord.java | 2 +- .../io/gcp/spanner/changestreams/model/HeartbeatRecord.java | 2 +- .../beam/sdk/io/gcp/spanner/changestreams/model/Mod.java | 2 +- .../beam/sdk/io/gcp/spanner/changestreams/model/ModType.java | 2 +- .../io/gcp/spanner/changestreams/model/PartitionMetadata.java | 2 +- .../beam/sdk/io/gcp/spanner/changestreams/model/TypeCode.java | 2 +- .../io/gcp/spanner/changestreams/model/ValueCaptureType.java | 2 +- .../beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java | 2 +- .../apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadTest.java | 2 +- .../apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java | 2 +- .../java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java | 2 +- .../io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java | 2 +- .../apache/beam/sdk/io/gcp/pubsub/PubsubRowToMessageTest.java | 2 +- .../pubsub/PubsubSchemaTransformMessageToRowFactoryTest.java | 2 +- .../io/gcp/pubsub/PubsubWriteSchemaTransformProviderTest.java | 2 +- .../spanner/changestreams/encoder/TimestampEncodingTest.java | 2 +- 34 files changed, 36 insertions(+), 34 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/build.gradle b/sdks/java/io/google-cloud-platform/build.gradle index d4a143c61730..076c90c56464 100644 --- a/sdks/java/io/google-cloud-platform/build.gradle +++ b/sdks/java/io/google-cloud-platform/build.gradle @@ -37,6 +37,7 @@ dependencies { implementation project(":runners:core-java") implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(":sdks:java:expansion-service") + implementation project(":sdks:java:extensions:avro") permitUnusedDeclared project(":sdks:java:expansion-service") // BEAM-11761 implementation project(":sdks:java:extensions:google-cloud-platform-core") implementation project(":sdks:java:extensions:protobuf") @@ -155,6 +156,7 @@ dependencies { testImplementation library.java.arrow_memory_netty testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") + testImplementation project(path: ":sdks:java:extensions:avro", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:extensions:google-cloud-platform-core", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:extensions:protobuf", configuration: "testRuntimeMigration") testImplementation project(path: ":runners:direct-java", configuration: "shadow") 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 4f9f0b709262..04a81821d8e9 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 @@ -69,11 +69,11 @@ import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.extensions.avro.io.AvroSource; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.extensions.gcp.util.Transport; import org.apache.beam.sdk.extensions.gcp.util.gcsfs.GcsPath; import org.apache.beam.sdk.extensions.protobuf.ProtoCoder; -import org.apache.beam.sdk.io.AvroSource; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.fs.MoveOptions; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryQuerySource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryQuerySource.java index 8e3b437bec50..8a35f56941fd 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryQuerySource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryQuerySource.java @@ -21,7 +21,7 @@ import com.google.api.services.bigquery.model.TableSchema; import java.io.IOException; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.io.AvroSource; +import org.apache.beam.sdk.extensions.avro.io.AvroSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.display.DisplayData; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryQuerySourceDef.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryQuerySourceDef.java index 606d1d3ad886..628ec6cf7816 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryQuerySourceDef.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryQuerySourceDef.java @@ -30,7 +30,7 @@ import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.io.AvroSource; +import org.apache.beam.sdk.extensions.avro.io.AvroSource; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryResourceNaming.JobType; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.schemas.Schema; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java index 1f16fb440495..c1db05b200c3 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java @@ -30,7 +30,7 @@ import java.io.IOException; import java.util.List; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.io.AvroSource; +import org.apache.beam.sdk.extensions.avro.io.AvroSource; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.fs.MatchResult; import org.apache.beam.sdk.io.fs.ResourceId; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceDef.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceDef.java index 9532a2f4d6f2..733edd268112 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceDef.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceDef.java @@ -22,7 +22,7 @@ import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.io.AvroSource; +import org.apache.beam.sdk.extensions.avro.io.AvroSource; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.transforms.SerializableFunction; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageArrowReader.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageArrowReader.java index 8f23825b9867..70703cf0082e 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageArrowReader.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageArrowReader.java @@ -28,7 +28,7 @@ import org.apache.avro.generic.GenericRecord; import org.apache.beam.sdk.extensions.arrow.ArrowConversion; import org.apache.beam.sdk.extensions.arrow.ArrowConversion.RecordBatchRowIterator; -import org.apache.beam.sdk.schemas.utils.AvroUtils; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.values.Row; class BigQueryStorageArrowReader implements BigQueryStorageReader { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBase.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBase.java index 3c8335ac82b5..27b88dc39600 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBase.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBase.java @@ -30,11 +30,11 @@ import org.apache.avro.Schema; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.extensions.arrow.ArrowConversion; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.StorageClient; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.ValueProvider; -import org.apache.beam.sdk.schemas.utils.AvroUtils; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSource.java index 2d274ed0e511..9d24246405ac 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSource.java @@ -23,7 +23,7 @@ import java.io.IOException; import java.util.concurrent.atomic.AtomicReference; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.io.AvroSource; +import org.apache.beam.sdk.extensions.avro.io.AvroSource; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.SerializableFunction; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSourceDef.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSourceDef.java index e78ea0b5d7c4..9b77f6a7ef9d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSourceDef.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSourceDef.java @@ -27,7 +27,7 @@ import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.io.AvroSource; +import org.apache.beam.sdk.extensions.avro.io.AvroSource; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.schemas.Schema; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClient.java index e864e649e699..e4de1680d5ad 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClient.java @@ -31,8 +31,8 @@ import java.util.List; import java.util.Map; import java.util.concurrent.ThreadLocalRandom; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.utils.AvroUtils; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Objects; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions; 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 a95b456aa52b..7bac875e2375 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 @@ -40,10 +40,11 @@ import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.extensions.protobuf.ProtoCoder; import org.apache.beam.sdk.extensions.protobuf.ProtoDomain; import org.apache.beam.sdk.extensions.protobuf.ProtoDynamicMessageSchema; @@ -55,7 +56,6 @@ import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.SchemaCoder; -import org.apache.beam.sdk.schemas.utils.AvroUtils; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteReadSchemaTransformProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteReadSchemaTransformProvider.java index 263c3fb95005..21885773e8aa 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteReadSchemaTransformProvider.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteReadSchemaTransformProvider.java @@ -26,6 +26,7 @@ import java.util.Collections; import java.util.List; import java.util.Objects; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.Schema; @@ -33,7 +34,6 @@ 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.AvroUtils; import org.apache.beam.sdk.schemas.utils.JsonUtils; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteWriteSchemaTransformProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteWriteSchemaTransformProvider.java index d2d510204ca7..bd083f8892af 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteWriteSchemaTransformProvider.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteWriteSchemaTransformProvider.java @@ -29,13 +29,13 @@ import java.util.List; import java.util.Objects; import java.util.Set; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; 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.AvroUtils; import org.apache.beam.sdk.schemas.utils.JsonUtils; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/ChangeStreamRecordMetadata.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/ChangeStreamRecordMetadata.java index af145b282a1e..22c295550b79 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/ChangeStreamRecordMetadata.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/ChangeStreamRecordMetadata.java @@ -22,8 +22,8 @@ import java.util.Objects; import javax.annotation.Nullable; import org.apache.avro.reflect.AvroEncode; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.gcp.spanner.changestreams.encoder.TimestampEncoding; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/ChildPartition.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/ChildPartition.java index a58434e8770b..84ab846712c1 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/ChildPartition.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/ChildPartition.java @@ -21,8 +21,8 @@ import java.util.HashSet; import java.util.Objects; import javax.annotation.Nullable; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Sets; /** diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/ChildPartitionsRecord.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/ChildPartitionsRecord.java index 7384f6850b96..2dd9d92682b2 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/ChildPartitionsRecord.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/ChildPartitionsRecord.java @@ -22,8 +22,8 @@ import java.util.Objects; import org.apache.avro.reflect.AvroEncode; import org.apache.avro.reflect.Nullable; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.gcp.spanner.changestreams.encoder.TimestampEncoding; /** diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/ColumnType.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/ColumnType.java index fd27934b5638..c54036ff3915 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/ColumnType.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/ColumnType.java @@ -20,8 +20,8 @@ import java.io.Serializable; import java.util.Objects; import javax.annotation.Nullable; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.schemas.annotations.SchemaCreate; /** diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/DataChangeRecord.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/DataChangeRecord.java index 837e334755c3..e00ef9c08ca1 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/DataChangeRecord.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/DataChangeRecord.java @@ -22,8 +22,8 @@ import java.util.Objects; import org.apache.avro.reflect.AvroEncode; import org.apache.avro.reflect.Nullable; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.gcp.spanner.changestreams.encoder.TimestampEncoding; /** diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/HeartbeatRecord.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/HeartbeatRecord.java index 3f8b88afd332..d343dabc69a9 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/HeartbeatRecord.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/HeartbeatRecord.java @@ -21,8 +21,8 @@ import java.util.Objects; import org.apache.avro.reflect.AvroEncode; import org.apache.avro.reflect.Nullable; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.gcp.spanner.changestreams.encoder.TimestampEncoding; /** diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/Mod.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/Mod.java index 60e62f02eb6e..37e80877b2f3 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/Mod.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/Mod.java @@ -20,8 +20,8 @@ import java.io.Serializable; import java.util.Objects; import javax.annotation.Nullable; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; /** * Represents a modification in a table emitted within a {@link DataChangeRecord}. Each mod contains diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/ModType.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/ModType.java index e56dd0af6d65..719e2e7c27f9 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/ModType.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/ModType.java @@ -17,8 +17,8 @@ */ package org.apache.beam.sdk.io.gcp.spanner.changestreams.model; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; /** * Represents the type of modification applied in the {@link DataChangeRecord}. It can be one of the diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/PartitionMetadata.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/PartitionMetadata.java index a32c832b84d0..765675d34743 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/PartitionMetadata.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/PartitionMetadata.java @@ -24,8 +24,8 @@ import java.util.Objects; import javax.annotation.Nullable; import org.apache.avro.reflect.AvroEncode; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.gcp.spanner.changestreams.encoder.TimestampEncoding; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/TypeCode.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/TypeCode.java index df4f520f3738..49498bded2ee 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/TypeCode.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/model/TypeCode.java @@ -20,8 +20,8 @@ import java.io.Serializable; import java.util.Objects; import javax.annotation.Nullable; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; /** * Represents a type of a column within Cloud Spanner. The type itself is encoded in a String code. 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 6a55bdc72d2b..9e336e18eb7b 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 @@ -17,8 +17,8 @@ */ package org.apache.beam.sdk.io.gcp.spanner.changestreams.model; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; /** * Represents the capture type of a change stream. The only supported value at the moment is diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java index f24796837e6d..1a1a67998a14 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java @@ -39,8 +39,8 @@ import org.apache.avro.generic.GenericRecord; import org.apache.avro.reflect.Nullable; import org.apache.avro.util.Utf8; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.BaseEncoding; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadTest.java index 4ca0fa96b5bd..01f58880527b 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadTest.java @@ -49,9 +49,9 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.SerializableCoder; import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.extensions.avro.io.AvroSource; import org.apache.beam.sdk.extensions.protobuf.ByteStringCoder; import org.apache.beam.sdk.extensions.protobuf.ProtoCoder; -import org.apache.beam.sdk.io.AvroSource; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead.QueryPriority; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryResourceNaming.JobType; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java index eacb95a9a683..cbf91d7c5637 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java @@ -48,12 +48,12 @@ import org.apache.avro.LogicalTypes; import org.apache.avro.generic.GenericData; import org.apache.avro.util.Utf8; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.ConversionOptions.TruncateTimestamps; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.FieldType; import org.apache.beam.sdk.schemas.logicaltypes.EnumerationType; import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes; -import org.apache.beam.sdk.schemas.utils.AvroUtils; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; import org.joda.time.DateTime; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java index 09775f7e0768..c7b3ce764a7f 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java @@ -48,10 +48,10 @@ import org.apache.avro.generic.GenericRecord; import org.apache.avro.reflect.AvroSchema; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.extensions.protobuf.Proto3SchemaMessages.Primitive; import org.apache.beam.sdk.extensions.protobuf.ProtoCoder; import org.apache.beam.sdk.extensions.protobuf.ProtoDomain; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java index f91183aa050c..aaceda5342db 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java @@ -37,9 +37,9 @@ import java.util.Objects; import java.util.UUID; import java.util.stream.Collectors; +import org.apache.beam.sdk.extensions.avro.schemas.io.payloads.AvroPayloadSerializerProvider; import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.io.payloads.AvroPayloadSerializerProvider; import org.apache.beam.sdk.schemas.io.payloads.PayloadSerializer; import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.testing.PAssert; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubRowToMessageTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubRowToMessageTest.java index 029c8ef08a4a..2ff0084bc086 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubRowToMessageTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubRowToMessageTest.java @@ -42,6 +42,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.beam.sdk.extensions.avro.schemas.io.payloads.AvroPayloadSerializerProvider; import org.apache.beam.sdk.io.gcp.pubsub.PubsubRowToMessage.FieldMatcher; import org.apache.beam.sdk.io.gcp.pubsub.PubsubRowToMessage.PubsubRowToMessageDoFn; import org.apache.beam.sdk.io.gcp.pubsub.PubsubRowToMessage.SchemaReflection; @@ -52,7 +53,6 @@ import org.apache.beam.sdk.schemas.Schema.Field; import org.apache.beam.sdk.schemas.Schema.FieldType; import org.apache.beam.sdk.schemas.Schema.TypeName; -import org.apache.beam.sdk.schemas.io.payloads.AvroPayloadSerializerProvider; import org.apache.beam.sdk.schemas.io.payloads.JsonPayloadSerializerProvider; import org.apache.beam.sdk.schemas.io.payloads.PayloadSerializer; import org.apache.beam.sdk.testing.PAssert; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubSchemaTransformMessageToRowFactoryTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubSchemaTransformMessageToRowFactoryTest.java index ad75e24ee9ad..709fc35e02ae 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubSchemaTransformMessageToRowFactoryTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubSchemaTransformMessageToRowFactoryTest.java @@ -28,9 +28,9 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import org.apache.beam.sdk.extensions.avro.schemas.io.payloads.AvroPayloadSerializerProvider; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.FieldType; -import org.apache.beam.sdk.schemas.io.payloads.AvroPayloadSerializerProvider; import org.apache.beam.sdk.schemas.io.payloads.JsonPayloadSerializerProvider; import org.apache.beam.sdk.schemas.io.payloads.PayloadSerializer; import org.apache.beam.sdk.schemas.io.payloads.PayloadSerializerProvider; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProviderTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProviderTest.java index b9c912ffea68..98939f7ddc68 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProviderTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubWriteSchemaTransformProviderTest.java @@ -39,6 +39,7 @@ import java.nio.charset.StandardCharsets; import java.util.Map; import org.apache.avro.SchemaParseException; +import org.apache.beam.sdk.extensions.avro.schemas.io.payloads.AvroPayloadSerializerProvider; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.SchemaPath; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.TopicPath; import org.apache.beam.sdk.io.gcp.pubsub.PubsubTestClient.PubsubTestClientFactory; @@ -49,7 +50,6 @@ import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.Field; import org.apache.beam.sdk.schemas.Schema.FieldType; -import org.apache.beam.sdk.schemas.io.payloads.AvroPayloadSerializerProvider; import org.apache.beam.sdk.schemas.io.payloads.JsonPayloadSerializerProvider; import org.apache.beam.sdk.schemas.io.payloads.PayloadSerializer; import org.apache.beam.sdk.testing.PAssert; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/encoder/TimestampEncodingTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/encoder/TimestampEncodingTest.java index 7afdb35a2070..aaa1657b1f28 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/encoder/TimestampEncodingTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/encoder/TimestampEncodingTest.java @@ -32,7 +32,7 @@ import org.apache.avro.io.DecoderFactory; import org.apache.avro.io.EncoderFactory; import org.apache.avro.reflect.AvroEncode; -import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.junit.Before; import org.junit.Test; From daed02af30eb610b121c84295dddb20d0ca1ba6f Mon Sep 17 00:00:00 2001 From: Alexey Romanenko Date: Wed, 15 Feb 2023 10:52:44 +0100 Subject: [PATCH 04/13] Resolve rebase conflicts --- .../structuredstreaming/translation/SparkSessionFactory.java | 4 ++-- runners/spark/spark_runner.gradle | 2 ++ .../org/apache/beam/runners/spark/io/AvroPipelineTest.java | 2 +- 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java index 16eb1131c007..382505c923b1 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java @@ -33,8 +33,6 @@ import org.apache.beam.runners.core.construction.resources.PipelineResources; import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingPipelineOptions; import org.apache.beam.runners.spark.structuredstreaming.translation.batch.functions.SideInputValues; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.coders.AvroGenericCoder; import org.apache.beam.sdk.coders.BigDecimalCoder; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.BigEndianLongCoder; @@ -70,6 +68,8 @@ import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.coders.VoidCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroGenericCoder; import org.apache.beam.sdk.io.FileBasedSink; import org.apache.beam.sdk.io.range.OffsetRange; import org.apache.beam.sdk.transforms.join.CoGbkResult; diff --git a/runners/spark/spark_runner.gradle b/runners/spark/spark_runner.gradle index 7d820c28f83f..319344634de7 100644 --- a/runners/spark/spark_runner.gradle +++ b/runners/spark/spark_runner.gradle @@ -161,6 +161,7 @@ dependencies { implementation project(":runners:java-fn-execution") implementation project(":runners:java-job-service") implementation project(":sdks:java:extensions:google-cloud-platform-core") + implementation project(":sdks:java:extensions:avro") implementation library.java.jackson_annotations implementation library.java.slf4j_api implementation library.java.joda_time @@ -191,6 +192,7 @@ dependencies { testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") // SparkStateInternalsTest extends abstract StateInternalsTest testImplementation project(path: ":runners:core-java", configuration: "testRuntimeMigration") + testImplementation project(path: ":sdks:java:extensions:avro", configuration: "testRuntimeMigration") testImplementation project(":sdks:java:harness") testImplementation library.java.avro testImplementation "org.apache.kafka:kafka_$spark_scala_version:2.4.1" diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java index fa49a9f5d12b..b46f5fafd42e 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java @@ -30,7 +30,7 @@ import org.apache.avro.generic.GenericDatumReader; import org.apache.avro.generic.GenericDatumWriter; import org.apache.avro.generic.GenericRecord; -import org.apache.beam.sdk.io.AvroIO; +import org.apache.beam.sdk.extensions.avro.io.AvroIO; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists; From 560d8fc6409e2adfa9e72a386b3de7690c662c34 Mon Sep 17 00:00:00 2001 From: Alexey Romanenko Date: Wed, 18 Jan 2023 15:31:06 +0100 Subject: [PATCH 05/13] Use "extensions/avro" for Dataflow runner --- runners/google-cloud-dataflow-java/build.gradle | 2 ++ .../runners/dataflow/util/AvroCoderCloudObjectTranslator.java | 2 +- .../apache/beam/runners/dataflow/util/CloudObjectsTest.java | 2 +- .../apache/beam/runners/dataflow/worker/AvroByteReader.java | 4 ++-- 4 files changed, 6 insertions(+), 4 deletions(-) diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index bf200310398e..a37a198cd75b 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -81,6 +81,7 @@ dependencies { implementation library.java.vendored_guava_26_0_jre implementation project(path: ":model:pipeline", configuration: "shadow") implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation project(":sdks:java:extensions:avro") implementation project(":sdks:java:extensions:google-cloud-platform-core") implementation project(":sdks:java:io:kafka") implementation project(":sdks:java:io:google-cloud-platform") @@ -115,6 +116,7 @@ dependencies { testImplementation library.java.junit testImplementation project(path: ":sdks:java:io:google-cloud-platform", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") + testImplementation project(path: ":sdks:java:extensions:avro", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:extensions:google-cloud-platform-core", configuration: "testRuntimeMigration") testImplementation project(path: ":runners:core-construction-java", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:extensions:python", configuration: "testRuntimeMigration") diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/AvroCoderCloudObjectTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/AvroCoderCloudObjectTranslator.java index af657761a34b..cc156aa42ab3 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/AvroCoderCloudObjectTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/AvroCoderCloudObjectTranslator.java @@ -19,7 +19,7 @@ import org.apache.avro.Schema; import org.apache.beam.runners.core.construction.SdkComponents; -import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; /** A {@link CloudObjectTranslator} for {@link AvroCoder}. */ @SuppressWarnings({ diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/CloudObjectsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/CloudObjectsTest.java index 71208f60b205..5a1af2bf3919 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/CloudObjectsTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/CloudObjectsTest.java @@ -36,7 +36,6 @@ import java.util.Set; import org.apache.avro.generic.GenericRecord; import org.apache.beam.runners.core.construction.SdkComponents; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -54,6 +53,7 @@ import org.apache.beam.sdk.coders.StructuredCoder; import org.apache.beam.sdk.coders.TimestampPrefixingWindowCoder; import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.FieldType; import org.apache.beam.sdk.schemas.SchemaCoder; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AvroByteReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AvroByteReader.java index 63e6f904c991..4ff7cc6c2bce 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AvroByteReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AvroByteReader.java @@ -29,8 +29,8 @@ import org.apache.avro.Schema; import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.io.AvroSource; -import org.apache.beam.sdk.io.AvroSource.AvroReader; +import org.apache.beam.sdk.extensions.avro.io.AvroSource; +import org.apache.beam.sdk.extensions.avro.io.AvroSource.AvroReader; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.OffsetBasedSource; From aacae8f7634cfe107523883e9a30cb9f74c329bf Mon Sep 17 00:00:00 2001 From: Alexey Romanenko Date: Wed, 18 Jan 2023 18:04:37 +0100 Subject: [PATCH 06/13] Use "extensions/avro" for core-construction runner --- runners/core-construction-java/build.gradle | 2 ++ .../runners/core/construction/AvroGenericCoderRegistrar.java | 2 +- .../runners/core/construction/AvroGenericCoderTranslator.java | 2 +- .../beam/runners/core/construction/CoderTranslationTest.java | 2 +- 4 files changed, 5 insertions(+), 3 deletions(-) diff --git a/runners/core-construction-java/build.gradle b/runners/core-construction-java/build.gradle index 8e91e156ae8b..c529716b46d0 100644 --- a/runners/core-construction-java/build.gradle +++ b/runners/core-construction-java/build.gradle @@ -53,6 +53,7 @@ dependencies { implementation project(path: ":model:pipeline", configuration: "shadow") implementation project(path: ":model:job-management", configuration: "shadow") implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation project(path: ":sdks:java:extensions:avro") implementation project(path: ":sdks:java:fn-execution") implementation library.java.vendored_grpc_1_48_1 implementation library.java.vendored_guava_26_0_jre @@ -73,5 +74,6 @@ dependencies { testImplementation library.java.jackson_dataformat_yaml testImplementation project(path: ":model:fn-execution", configuration: "shadow") testImplementation project(path: ":sdks:java:core", configuration: "testRuntimeMigration") + testImplementation project(path: ":sdks:java:extensions:avro", configuration: "testRuntimeMigration") testRuntimeOnly library.java.slf4j_jdk14 } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/AvroGenericCoderRegistrar.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/AvroGenericCoderRegistrar.java index 1ccd57935b6e..1797718d6ed1 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/AvroGenericCoderRegistrar.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/AvroGenericCoderRegistrar.java @@ -19,8 +19,8 @@ import com.google.auto.service.AutoService; import java.util.Map; -import org.apache.beam.sdk.coders.AvroGenericCoder; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.extensions.avro.coders.AvroGenericCoder; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; /** Coder registrar for AvroGenericCoder. */ diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/AvroGenericCoderTranslator.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/AvroGenericCoderTranslator.java index 53a566c1b52c..614810abbdbc 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/AvroGenericCoderTranslator.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/AvroGenericCoderTranslator.java @@ -21,8 +21,8 @@ import java.util.List; import org.apache.avro.Schema; import org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext; -import org.apache.beam.sdk.coders.AvroGenericCoder; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.extensions.avro.coders.AvroGenericCoder; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Charsets; /** Coder translator for AvroGenericCoder. */ diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CoderTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CoderTranslationTest.java index 5c28b27148f6..b68887350d49 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CoderTranslationTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CoderTranslationTest.java @@ -33,7 +33,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.Components; import org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext; import org.apache.beam.sdk.coders.AtomicCoder; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.BooleanCoder; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; @@ -48,6 +47,7 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.TimestampPrefixingWindowCoder; import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.Field; import org.apache.beam.sdk.schemas.Schema.FieldType; From 9945b72e9833e490d86c96fcf26332389f225861 Mon Sep 17 00:00:00 2001 From: Alexey Romanenko Date: Wed, 18 Jan 2023 18:54:49 +0100 Subject: [PATCH 07/13] Add avro extension as a dependency for direct runner --- runners/direct-java/build.gradle | 4 +++- sdks/java/harness/build.gradle | 1 + 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/runners/direct-java/build.gradle b/runners/direct-java/build.gradle index 36ada3cc888a..32c1a2304ef1 100644 --- a/runners/direct-java/build.gradle +++ b/runners/direct-java/build.gradle @@ -26,7 +26,8 @@ def dependOnProjects = [":runners:core-construction-java", ":runners:core-java", ":runners:local-java", ":runners:java-fn-execution", - ":sdks:java:fn-execution" + ":sdks:java:fn-execution", + ":sdks:java:extensions:avro" ] applyJavaNature( @@ -96,6 +97,7 @@ dependencies { permitUnusedDeclared library.java.vendored_grpc_1_48_1 permitUnusedDeclared project(":runners:java-fn-execution") permitUnusedDeclared project(":sdks:java:fn-execution") + permitUnusedDeclared project(":sdks:java:extensions:avro") examplesJavaIntegrationTest project(project.path) examplesJavaIntegrationTest project(":examples:java") examplesJavaIntegrationTest project(path: ":examples:java", configuration: "testRuntimeMigration") diff --git a/sdks/java/harness/build.gradle b/sdks/java/harness/build.gradle index 813a32c46178..5ca0a8e2003a 100644 --- a/sdks/java/harness/build.gradle +++ b/sdks/java/harness/build.gradle @@ -53,6 +53,7 @@ applyJavaNature( "org/apache/beam/model/fnexecution/**", "org/apache/beam/runners/core/**", "org/apache/beam/runners/core/construction/**", + "org/apache/beam/sdk/extensions/avro/**", "org/apache/beam/sdk/fn/**", "org/checkerframework/**", "org/hamcrest/**", From 9deb1c2b852fff88bef103f6ac703115eb24e007 Mon Sep 17 00:00:00 2001 From: Alexey Romanenko Date: Mon, 23 Jan 2023 13:55:31 +0100 Subject: [PATCH 08/13] Copy Avro-related tests to Avro extension --- .../worker/build.gradle | 1 + .../apache/beam/sdk/coders/DefaultCoder.java | 2 +- .../apache/beam/sdk/io/CountingSource.java | 48 ++++- .../java/org/apache/beam/sdk/io/ReadTest.java | 3 +- .../apache/beam/sdk/io/TextIOWriteTest.java | 30 ++- .../beam/sdk/transforms/LatestTest.java | 4 +- .../avro/coders/CoderRegistryTest.java | 121 +++++++++++ .../avro/coders/DefaultCoderTest.java | 61 ++++++ .../avro/schemas/SchemaCoderTest.java | 196 ++++++++++++++++++ .../avro/schemas/transforms/ConvertTest.java | 147 +++++++++++++ 10 files changed, 602 insertions(+), 11 deletions(-) create mode 100644 sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/CoderRegistryTest.java create mode 100644 sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/DefaultCoderTest.java create mode 100644 sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/SchemaCoderTest.java create mode 100644 sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/transforms/ConvertTest.java diff --git a/runners/google-cloud-dataflow-java/worker/build.gradle b/runners/google-cloud-dataflow-java/worker/build.gradle index 08769321f1fd..acb846ab3896 100644 --- a/runners/google-cloud-dataflow-java/worker/build.gradle +++ b/runners/google-cloud-dataflow-java/worker/build.gradle @@ -52,6 +52,7 @@ def sdk_provided_shaded_project_dependencies = [ ] def sdk_provided_project_dependencies = [ ":runners:google-cloud-dataflow-java", + ":sdks:java:extensions:avro", ":sdks:java:extensions:google-cloud-platform-core", ":sdks:java:io:google-cloud-platform", ] diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java index bf63103e84c2..f83072626554 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java @@ -74,7 +74,7 @@ public List getCoderProviders() { * A {@link CoderProvider} that uses the {@code @DefaultCoder} annotation to provide {@link * CoderProvider coder providers} that create {@link Coder}s. */ - static class DefaultCoderProvider extends CoderProvider { + public static class DefaultCoderProvider extends CoderProvider { private static final Logger LOG = LoggerFactory.getLogger(DefaultCoderProvider.class); /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java index 3e973cc2e1ad..8babeec98caf 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java @@ -20,13 +20,21 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.EOFException; import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.UTFDataFormatException; import java.util.List; import java.util.NoSuchElementException; import java.util.Objects; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.coders.InstantCoder; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; import org.apache.beam.sdk.metrics.Counter; @@ -354,7 +362,7 @@ public UnboundedReader createReader(PipelineOptions options, CounterMark c @Override public Coder getCheckpointMarkCoder() { - return AvroCoder.of(CountingSource.CounterMark.class); + return CounterMarkCoder.of(); } @Override @@ -481,11 +489,45 @@ public long getSplitBacklogBytes() { } } + public static class CounterMarkCoder extends CustomCoder { + + private static final CounterMarkCoder INSTANCE = new CounterMarkCoder(); + + public static CounterMarkCoder of() { + return INSTANCE; + } + + @Override + public void encode(CountingSource.CounterMark value, OutputStream outStream) + throws CoderException, IOException { + if (value == null) { + throw new CoderException("cannot encode a null CounterMark"); + } + + DataOutputStream stream = new DataOutputStream(outStream); + stream.writeLong(value.getLastEmitted()); + InstantCoder.of().encode(value.getStartTime(), stream); + } + + @Override + public CountingSource.CounterMark decode(InputStream inStream) + throws CoderException, IOException { + try { + DataInputStream stream = new DataInputStream(inStream); + long lastEmitted = stream.readLong(); + Instant startTime = InstantCoder.of().decode(stream); + return new CountingSource.CounterMark(lastEmitted, startTime); + } catch (EOFException | UTFDataFormatException e) { + throw new CoderException(e); + } + } + } + /** * The checkpoint for an unbounded {@link CountingSource} is simply the last value produced. The * associated source object encapsulates the information needed to produce the next value. */ - @DefaultCoder(AvroCoder.class) + @DefaultCoder(CounterMarkCoder.class) public static class CounterMark implements UnboundedSource.CheckpointMark { /** The last value emitted. */ private final long lastEmitted; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java index 315228e77688..b5c2d91996ef 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java @@ -38,7 +38,6 @@ import java.util.function.Consumer; import java.util.stream.Collectors; import java.util.stream.LongStream; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; @@ -343,7 +342,7 @@ public Coder getOutputCoder() { @Override public Coder getCheckpointMarkCoder() { - return AvroCoder.of(CountingSource.CounterMark.class); + return CountingSource.CounterMarkCoder.of(); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOWriteTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOWriteTest.java index f3ecc95005bc..1bb5525af329 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOWriteTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOWriteTest.java @@ -30,9 +30,13 @@ import static org.junit.Assume.assumeFalse; import java.io.BufferedReader; +import java.io.DataInputStream; +import java.io.DataOutputStream; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; import java.nio.channels.Channels; import java.nio.channels.WritableByteChannel; import java.nio.file.Files; @@ -43,8 +47,9 @@ import java.util.List; import java.util.stream.Collectors; import java.util.stream.StreamSupport; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.DefaultCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.FileBasedSink.WritableByteChannelFactory; @@ -209,7 +214,26 @@ private void testDynamicDestinations(boolean customType) throws Exception { DefaultFilenamePolicy.DEFAULT_UNWINDOWED_SHARD_TEMPLATE); } - @DefaultCoder(AvroCoder.class) + public static class UserWriteTypeCoder extends CustomCoder { + + @Override + public void encode(UserWriteType value, OutputStream outStream) + throws CoderException, IOException { + DataOutputStream stream = new DataOutputStream(outStream); + StringUtf8Coder.of().encode(value.destination, stream); + StringUtf8Coder.of().encode(value.metadata, stream); + } + + @Override + public UserWriteType decode(InputStream inStream) throws CoderException, IOException { + DataInputStream stream = new DataInputStream(inStream); + String dest = StringUtf8Coder.of().decode(stream); + String meta = StringUtf8Coder.of().decode(stream); + return new UserWriteType(dest, meta); + } + } + + @DefaultCoder(UserWriteTypeCoder.class) private static class UserWriteType { String destination; String metadata; @@ -279,6 +303,8 @@ public void testDynamicDefaultFilenamePolicy() throws Exception { new UserWriteType("baab", "fourth"), new UserWriteType("caaa", "fifth"), new UserWriteType("caab", "sixth")); + + p.getCoderRegistry().registerCoderForClass(UserWriteType.class, new UserWriteTypeCoder()); PCollection input = p.apply(Create.of(elements)); input.apply( TextIO.writeCustomType() diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java index 179d1ef98a0a..20b04d88bdab 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java @@ -22,7 +22,6 @@ import static org.junit.Assert.assertEquals; import java.io.Serializable; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.BigEndianLongCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; @@ -108,8 +107,7 @@ public void testPerKeyEventTimestamp() { public void testPerKeyOutputCoder() { p.enableAbandonedNodeEnforcement(false); - KvCoder inputCoder = - KvCoder.of(AvroCoder.of(String.class), AvroCoder.of(Long.class)); + KvCoder inputCoder = KvCoder.of(StringUtf8Coder.of(), BigEndianLongCoder.of()); PCollection> output = p.apply(Create.of(KV.of("foo", 1L)).withCoder(inputCoder)).apply(Latest.perKey()); diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/CoderRegistryTest.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/CoderRegistryTest.java new file mode 100644 index 000000000000..28579033f770 --- /dev/null +++ b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/CoderRegistryTest.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.extensions.avro.coders; + +import static org.junit.Assert.assertEquals; + +import com.google.auto.service.AutoService; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.List; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.CoderProvider; +import org.apache.beam.sdk.coders.CoderProviderRegistrar; +import org.apache.beam.sdk.coders.CoderProviders; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.CustomCoder; +import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.testing.ExpectedLogs; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for CoderRegistry and AvroCoder. */ +@RunWith(JUnit4.class) +public class CoderRegistryTest { + + @Rule public TestPipeline pipeline = TestPipeline.create(); + + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Rule public ExpectedLogs expectedLogs = ExpectedLogs.none(CoderRegistry.class); + + @Test + public void testCoderPrecedence() throws Exception { + CoderRegistry registry = CoderRegistry.createDefault(); + + // DefaultCoder precedes CoderProviderRegistrar + assertEquals(AvroCoder.of(MyValueA.class), registry.getCoder(MyValueA.class)); + + // CoderProviderRegistrar precedes SerializableCoder + assertEquals(MyValueBCoder.INSTANCE, registry.getCoder(MyValueB.class)); + + // fallbacks to SerializableCoder at last + assertEquals(SerializableCoder.of(MyValueC.class), registry.getCoder(MyValueC.class)); + } + + @DefaultCoder(AvroCoder.class) + private static class MyValueA implements Serializable {} + + private static class MyValueB implements Serializable {} + + private static class MyValueC implements Serializable {} + + private static class MyValueACoder extends CustomCoder { + private static final MyValueACoder INSTANCE = new MyValueACoder(); + + @Override + public void encode(MyValueA value, OutputStream outStream) throws CoderException, IOException {} + + @Override + public MyValueA decode(InputStream inStream) throws CoderException, IOException { + return null; + } + } + + /** A {@link CoderProviderRegistrar} to demonstrate default {@link Coder} registration. */ + @AutoService(CoderProviderRegistrar.class) + public static class MyValueACoderProviderRegistrar implements CoderProviderRegistrar { + @Override + public List getCoderProviders() { + return ImmutableList.of( + CoderProviders.forCoder(TypeDescriptor.of(MyValueA.class), MyValueACoder.INSTANCE)); + } + } + + private static class MyValueBCoder extends CustomCoder { + private static final MyValueBCoder INSTANCE = new MyValueBCoder(); + + @Override + public void encode(MyValueB value, OutputStream outStream) throws CoderException, IOException {} + + @Override + public MyValueB decode(InputStream inStream) throws CoderException, IOException { + return null; + } + } + + /** A {@link CoderProviderRegistrar} to demonstrate default {@link Coder} registration. */ + @AutoService(CoderProviderRegistrar.class) + public static class MyValueBCoderProviderRegistrar implements CoderProviderRegistrar { + @Override + public List getCoderProviders() { + return ImmutableList.of( + CoderProviders.forCoder(TypeDescriptor.of(MyValueB.class), MyValueBCoder.INSTANCE)); + } + } +} diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/DefaultCoderTest.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/DefaultCoderTest.java new file mode 100644 index 000000000000..82991f191887 --- /dev/null +++ b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/DefaultCoderTest.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.extensions.avro.coders; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.instanceOf; + +import java.util.List; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.coders.DefaultCoder.DefaultCoderProviderRegistrar.DefaultCoderProvider; +import org.apache.beam.sdk.coders.ListCoder; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link DefaultCoder}. */ +@RunWith(JUnit4.class) +public class DefaultCoderTest { + + @Rule public ExpectedException thrown = ExpectedException.none(); + + @DefaultCoder(AvroCoder.class) + private static class AvroRecord {} + + @Test + public void testCodersWithoutComponents() throws Exception { + CoderRegistry registry = CoderRegistry.createDefault(); + registry.registerCoderProvider(new DefaultCoderProvider()); + assertThat(registry.getCoder(AvroRecord.class), instanceOf(AvroCoder.class)); + } + + @Test + public void testDefaultCoderInCollection() throws Exception { + CoderRegistry registry = CoderRegistry.createDefault(); + registry.registerCoderProvider(new DefaultCoderProvider()); + Coder> avroRecordCoder = + registry.getCoder(new TypeDescriptor>() {}); + assertThat(avroRecordCoder, instanceOf(ListCoder.class)); + assertThat(((ListCoder) avroRecordCoder).getElemCoder(), instanceOf(AvroCoder.class)); + } +} diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/SchemaCoderTest.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/SchemaCoderTest.java new file mode 100644 index 000000000000..aa026d441801 --- /dev/null +++ b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/SchemaCoderTest.java @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.extensions.avro.schemas; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.fail; + +import java.time.Instant; +import java.util.Collection; +import java.util.Objects; +import java.util.function.Supplier; +import org.apache.avro.reflect.AvroSchema; +import org.apache.beam.sdk.coders.Coder.NonDeterministicException; +import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.schemas.NoSuchSchemaException; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.Schema.Field; +import org.apache.beam.sdk.schemas.Schema.FieldType; +import org.apache.beam.sdk.schemas.SchemaCoder; +import org.apache.beam.sdk.schemas.SchemaRegistry; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.logicaltypes.NanosInstant; +import org.apache.beam.sdk.testing.CoderProperties; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.hamcrest.Matchers; +import org.joda.time.DateTime; +import org.junit.Test; +import org.junit.experimental.runners.Enclosed; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +/** Unit tests for {@link Schema}. */ +@RunWith(Enclosed.class) +@SuppressWarnings({ + "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) +}) +public class SchemaCoderTest { + + public static final Schema LOGICAL_NANOS_SCHEMA = + Schema.of(Field.of("logicalNanos", FieldType.logicalType(new NanosInstant()))); + public static final Schema FLOATING_POINT_SCHEMA = + Schema.of(Field.of("float", FieldType.FLOAT), Field.of("double", FieldType.DOUBLE)); + + @DefaultSchema(AvroRecordSchema.class) + private static class SimpleAvro { + public String string; + public Integer int32; + public Long int64; + + @AvroSchema("{\"type\": \"long\", \"logicalType\": \"timestamp-millis\"}") + public DateTime datetime; + + public SimpleAvro(String string, Integer int32, Long int64, DateTime datetime) { + this.string = string; + this.int32 = int32; + this.int64 = int64; + this.datetime = datetime; + } + + @Override + public boolean equals(@Nullable Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SimpleAvro that = (SimpleAvro) o; + return string.equals(that.string) + && int32.equals(that.int32) + && int64.equals(that.int64) + && datetime.equals(that.datetime); + } + + @Override + public int hashCode() { + return Objects.hash(string, int32, int64, datetime); + } + } + + private static final SchemaRegistry REGISTRY = SchemaRegistry.createDefault(); + + private static SchemaCoder coderFrom(TypeDescriptor typeDescriptor) throws NoSuchSchemaException { + return SchemaCoder.of( + REGISTRY.getSchema(typeDescriptor), + typeDescriptor, + REGISTRY.getToRowFunction(typeDescriptor), + REGISTRY.getFromRowFunction(typeDescriptor)); + } + + @RunWith(Parameterized.class) + public static class ParameterizedTests { + + @Parameterized.Parameter(0) + public SchemaCoder coder; + + @Parameterized.Parameter(1) + public ImmutableList> testValues; + + @Parameterized.Parameter(2) + public boolean expectDeterministic; + + @Parameterized.Parameters(name = "{index}: coder = {0}") + public static Collection data() throws NoSuchSchemaException { + return ImmutableList.of( + new Object[] { + coderFrom(TypeDescriptor.of(SimpleAvro.class)), + ImmutableList.>of( + () -> + new SimpleAvro( + "foo", + 9001, + 0L, + new DateTime().withDate(1979, 3, 14).withTime(10, 30, 0, 0)), + () -> + new SimpleAvro( + "bar", + 9002, + 1L, + new DateTime().withDate(1989, 3, 14).withTime(10, 30, 0, 0))), + true + }, + new Object[] { + RowCoder.of(LOGICAL_NANOS_SCHEMA), + ImmutableList.>of( + () -> + Row.withSchema(LOGICAL_NANOS_SCHEMA) + .withFieldValue("logicalNanos", Instant.ofEpochMilli(9001)) + .build()), + true + }, + new Object[] { + RowCoder.of(FLOATING_POINT_SCHEMA), + ImmutableList.>of( + () -> + Row.withSchema(FLOATING_POINT_SCHEMA) + .withFieldValue("float", (float) 1.0) + .withFieldValue("double", 2.0) + .build()), + false + }); + } + + @Test + public void coderSerializable() { + CoderProperties.coderSerializable(coder); + } + + @Test + public void coderConsistentWithEquals() throws Exception { + for (Supplier testValueA : testValues) { + for (Supplier testValueB : testValues) { + CoderProperties.coderConsistentWithEquals(coder, testValueA.get(), testValueB.get()); + } + } + } + + @Test + public void verifyDeterministic() throws Exception { + if (expectDeterministic) { + for (Supplier testValue : testValues) { + CoderProperties.coderDeterministic(coder, testValue.get(), testValue.get()); + } + } else { + assertNonDeterministic(coder); + } + } + } + + private static void assertNonDeterministic(SchemaCoder coder) { + try { + coder.verifyDeterministic(); + fail("Expected " + coder + " to be non-deterministic."); + } catch (NonDeterministicException e) { + assertThat(e.getReasons(), Matchers.iterableWithSize(1)); + } + } +} diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/transforms/ConvertTest.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/transforms/ConvertTest.java new file mode 100644 index 000000000000..a33fa013be17 --- /dev/null +++ b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/transforms/ConvertTest.java @@ -0,0 +1,147 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.extensions.avro.schemas.transforms; + +import java.util.Arrays; +import java.util.Map; +import java.util.Objects; +import org.apache.avro.generic.GenericRecord; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; +import org.apache.beam.sdk.schemas.JavaFieldSchema; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.Schema.FieldType; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.transforms.Convert; +import org.apache.beam.sdk.testing.NeedsRunner; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.UsesSchema; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for the {@link Convert} class. */ +@RunWith(JUnit4.class) +@Category(UsesSchema.class) +public class ConvertTest { + @Rule public final transient TestPipeline pipeline = TestPipeline.create(); + + /** Test outer POJO. * */ + @DefaultSchema(JavaFieldSchema.class) + public static class POJO1 { + public String field1 = "field1"; + public long field2 = 42; + public POJO1Nested field3 = new POJO1Nested(); + public POJO1Nested[] field4 = new POJO1Nested[] {new POJO1Nested(), new POJO1Nested()}; + public Map field5 = + ImmutableMap.of( + "first", new POJO1Nested(), + "second", new POJO1Nested()); + + @Override + public boolean equals(@Nullable Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + POJO1 pojo1 = (POJO1) o; + return field2 == pojo1.field2 + && Objects.equals(field1, pojo1.field1) + && Objects.equals(field3, pojo1.field3) + && Arrays.equals(field4, pojo1.field4) + && Objects.equals(field5, pojo1.field5); + } + + @Override + public int hashCode() { + int result = Objects.hash(field1, field2, field3, field5); + result = 31 * result + Arrays.hashCode(field4); + return result; + } + } + + /** Test inner POJO. * */ + @DefaultSchema(JavaFieldSchema.class) + public static class POJO1Nested { + public String yard1 = "yard2"; + public long yard2 = 43; + + @Override + public boolean equals(@Nullable Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + POJO1Nested that = (POJO1Nested) o; + return yard2 == that.yard2 && Objects.equals(yard1, that.yard1); + } + + @Override + public int hashCode() { + return Objects.hash(yard1, yard2); + } + } + + private static final Schema EXPECTED_SCHEMA1_NESTED = + Schema.builder().addStringField("yard1").addInt64Field("yard2").build(); + + private static final Schema EXPECTED_SCHEMA1 = + Schema.builder() + .addStringField("field1") + .addInt64Field("field2") + .addRowField("field3", EXPECTED_SCHEMA1_NESTED) + .addArrayField("field4", FieldType.row(EXPECTED_SCHEMA1_NESTED)) + .addMapField("field5", FieldType.STRING, FieldType.row(EXPECTED_SCHEMA1_NESTED)) + .build(); + + private static final Row EXPECTED_ROW1_NESTED = + Row.withSchema(EXPECTED_SCHEMA1_NESTED).addValues("yard2", 43L).build(); + + private static final Row EXPECTED_ROW1 = + Row.withSchema(EXPECTED_SCHEMA1) + .addValue("field1") + .addValue(42L) + .addValue(EXPECTED_ROW1_NESTED) + .addArray(ImmutableList.of(EXPECTED_ROW1_NESTED, EXPECTED_ROW1_NESTED)) + .addValue(ImmutableMap.of("first", EXPECTED_ROW1_NESTED, "second", EXPECTED_ROW1_NESTED)) + .build(); + + private static final GenericRecord EXPECTED_GENERICRECORD1 = + AvroUtils.toGenericRecord(EXPECTED_ROW1, AvroUtils.toAvroSchema(EXPECTED_SCHEMA1)); + + @Test + @Category(NeedsRunner.class) + public void testToGenericRecords() { + PCollection records = + pipeline.apply(Create.of(new POJO1())).apply(Convert.to(GenericRecord.class)); + PAssert.that(records).containsInAnyOrder(EXPECTED_GENERICRECORD1); + pipeline.run(); + } +} From 691bfa00d1641b802a490de2b89940b835b39a7f Mon Sep 17 00:00:00 2001 From: Alexey Romanenko Date: Thu, 26 Jan 2023 17:02:55 +0100 Subject: [PATCH 09/13] Address review comments --- .../translation/SparkSessionFactory.java | 16 ++- .../apache/beam/sdk/schemas/io/Providers.java | 19 +-- sdks/java/extensions/avro/build.gradle | 2 +- .../avro/coders/CoderRegistryTest.java | 121 ------------------ sdks/java/io/kafka/build.gradle | 2 +- 5 files changed, 24 insertions(+), 136 deletions(-) delete mode 100644 sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/CoderRegistryTest.java diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java index 382505c923b1..763580b6ae72 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java @@ -68,8 +68,6 @@ import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.coders.VoidCoder; -import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; -import org.apache.beam.sdk.extensions.avro.coders.AvroGenericCoder; import org.apache.beam.sdk.io.FileBasedSink; import org.apache.beam.sdk.io.range.OffsetRange; import org.apache.beam.sdk.transforms.join.CoGbkResult; @@ -227,9 +225,11 @@ public void registerClasses(Kryo kryo) { kryo.register(SideInputValues.ByWindow.class); kryo.register(SideInputValues.Global.class); + // avro coders + tryToRegister(kryo, "org.apache.beam.sdk.extensions.avro.coders.AvroCoder"); + tryToRegister(kryo, "org.apache.beam.sdk.extensions.avro.coders.AvroGenericCoder"); + // standard coders of org.apache.beam.sdk.coders - kryo.register(AvroCoder.class); - kryo.register(AvroGenericCoder.class); kryo.register(BigDecimalCoder.class); kryo.register(BigEndianIntegerCoder.class); kryo.register(BigEndianLongCoder.class); @@ -283,5 +283,13 @@ public void registerClasses(Kryo kryo) { kryo.register(TupleTag.class); kryo.register(TupleTagList.class); } + + private void tryToRegister(Kryo kryo, String className) { + try { + kryo.register(Class.forName(className)); + } catch (ClassNotFoundException e) { + LOG.warn("Class {}} was not found on classpath", className); + } + } } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/io/Providers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/io/Providers.java index cc1e01b9a4ae..39a23685eb23 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/io/Providers.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/io/Providers.java @@ -17,7 +17,7 @@ */ package org.apache.beam.sdk.schemas.io; -import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; import java.util.HashMap; import java.util.Map; @@ -42,24 +42,25 @@ private Providers() {} public static Map loadProviders(Class klass) { Map providers = new HashMap<>(); for (T provider : ServiceLoader.load(klass)) { - // Avro provider is treated as a special case until two providers may exist: in "core" - // (deprecated) and in "extensions/avro" (actual). + // Avro provider is treated as a special case since two Avro providers may want to be loaded - + // from "core" (deprecated) and from "extensions/avro" (actual) - but only one must succeed. + // TODO: this check should be removed once once AvroPayloadSerializerProvider from "core" is + // removed if (provider.identifier().equals("avro")) { // Avro provider from "extensions/avro" must have a priority. if (provider - .toString() - .startsWith( + .getClass() + .getName() + .equals( "org.apache.beam.sdk.extensions.avro.schemas.io.payloads.AvroPayloadSerializerProvider")) { // Use AvroPayloadSerializerProvider from extensions/avro by any case. providers.put(provider.identifier(), provider); } else { // Load Avro provider from "core" if it was not loaded from Avro extension before. - if (!providers.containsKey(provider.identifier())) { - providers.put(provider.identifier(), provider); - } + providers.putIfAbsent(provider.identifier(), provider); } } else { - checkArgument( + checkState( !providers.containsKey(provider.identifier()), "Duplicate providers exist with identifier `%s` for class %s.", provider.identifier(), diff --git a/sdks/java/extensions/avro/build.gradle b/sdks/java/extensions/avro/build.gradle index dae13cd99728..1b1b7f824857 100644 --- a/sdks/java/extensions/avro/build.gradle +++ b/sdks/java/extensions/avro/build.gradle @@ -20,7 +20,7 @@ plugins { id 'org.apache.beam.module' } applyJavaNature( automaticModuleName: 'org.apache.beam.sdk.extensions.avro', disableLintWarnings: ['rawtypes'], // Avro-generated test code has raw-type errors - publish: false, + publish: true, exportJavadoc: false, ) applyAvroNature() diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/CoderRegistryTest.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/CoderRegistryTest.java deleted file mode 100644 index 28579033f770..000000000000 --- a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/CoderRegistryTest.java +++ /dev/null @@ -1,121 +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.sdk.extensions.avro.coders; - -import static org.junit.Assert.assertEquals; - -import com.google.auto.service.AutoService; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.util.List; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.CoderProvider; -import org.apache.beam.sdk.coders.CoderProviderRegistrar; -import org.apache.beam.sdk.coders.CoderProviders; -import org.apache.beam.sdk.coders.CoderRegistry; -import org.apache.beam.sdk.coders.CustomCoder; -import org.apache.beam.sdk.coders.DefaultCoder; -import org.apache.beam.sdk.coders.SerializableCoder; -import org.apache.beam.sdk.testing.ExpectedLogs; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Tests for CoderRegistry and AvroCoder. */ -@RunWith(JUnit4.class) -public class CoderRegistryTest { - - @Rule public TestPipeline pipeline = TestPipeline.create(); - - @Rule public ExpectedException thrown = ExpectedException.none(); - - @Rule public ExpectedLogs expectedLogs = ExpectedLogs.none(CoderRegistry.class); - - @Test - public void testCoderPrecedence() throws Exception { - CoderRegistry registry = CoderRegistry.createDefault(); - - // DefaultCoder precedes CoderProviderRegistrar - assertEquals(AvroCoder.of(MyValueA.class), registry.getCoder(MyValueA.class)); - - // CoderProviderRegistrar precedes SerializableCoder - assertEquals(MyValueBCoder.INSTANCE, registry.getCoder(MyValueB.class)); - - // fallbacks to SerializableCoder at last - assertEquals(SerializableCoder.of(MyValueC.class), registry.getCoder(MyValueC.class)); - } - - @DefaultCoder(AvroCoder.class) - private static class MyValueA implements Serializable {} - - private static class MyValueB implements Serializable {} - - private static class MyValueC implements Serializable {} - - private static class MyValueACoder extends CustomCoder { - private static final MyValueACoder INSTANCE = new MyValueACoder(); - - @Override - public void encode(MyValueA value, OutputStream outStream) throws CoderException, IOException {} - - @Override - public MyValueA decode(InputStream inStream) throws CoderException, IOException { - return null; - } - } - - /** A {@link CoderProviderRegistrar} to demonstrate default {@link Coder} registration. */ - @AutoService(CoderProviderRegistrar.class) - public static class MyValueACoderProviderRegistrar implements CoderProviderRegistrar { - @Override - public List getCoderProviders() { - return ImmutableList.of( - CoderProviders.forCoder(TypeDescriptor.of(MyValueA.class), MyValueACoder.INSTANCE)); - } - } - - private static class MyValueBCoder extends CustomCoder { - private static final MyValueBCoder INSTANCE = new MyValueBCoder(); - - @Override - public void encode(MyValueB value, OutputStream outStream) throws CoderException, IOException {} - - @Override - public MyValueB decode(InputStream inStream) throws CoderException, IOException { - return null; - } - } - - /** A {@link CoderProviderRegistrar} to demonstrate default {@link Coder} registration. */ - @AutoService(CoderProviderRegistrar.class) - public static class MyValueBCoderProviderRegistrar implements CoderProviderRegistrar { - @Override - public List getCoderProviders() { - return ImmutableList.of( - CoderProviders.forCoder(TypeDescriptor.of(MyValueB.class), MyValueBCoder.INSTANCE)); - } - } -} diff --git a/sdks/java/io/kafka/build.gradle b/sdks/java/io/kafka/build.gradle index 065c8b83013c..9e3fe0f7341f 100644 --- a/sdks/java/io/kafka/build.gradle +++ b/sdks/java/io/kafka/build.gradle @@ -91,7 +91,7 @@ dependencies { provided library.java.everit_json_schema testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") testImplementation project(":sdks:java:io:synthetic") - testImplementation project(path: ":sdks:java:extensions:avro", configuration: "testRuntimeMigration") + testImplementation project(":sdks:java:extensions:avro") testImplementation project(path: ":sdks:java:io:common", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:testing:test-utils", configuration: "testRuntimeMigration") // For testing Cross-language transforms From 1e7c47185636c53869ef32d381a9a16b152883f9 Mon Sep 17 00:00:00 2001 From: Alexey Romanenko Date: Mon, 30 Jan 2023 18:37:55 +0100 Subject: [PATCH 10/13] Depend javaPreCommit on extensions:avro --- build.gradle.kts | 1 + 1 file changed, 1 insertion(+) diff --git a/build.gradle.kts b/build.gradle.kts index 8a244e6f7ec1..e715ce661fa3 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -263,6 +263,7 @@ tasks.register("javaPreCommit") { dependsOn(":sdks:java:expansion-service:build") dependsOn(":sdks:java:expansion-service:app:build") dependsOn(":sdks:java:extensions:arrow:build") + dependsOn(":sdks:java:extensions:avro:build") dependsOn(":sdks:java:extensions:euphoria:build") dependsOn(":sdks:java:extensions:google-cloud-platform-core:build") dependsOn(":sdks:java:extensions:jackson:build") From 303dea92d609a528317928800967cae4864402df Mon Sep 17 00:00:00 2001 From: Alexey Romanenko Date: Tue, 31 Jan 2023 17:38:18 +0100 Subject: [PATCH 11/13] Address review comments --- runners/core-construction-java/build.gradle | 2 +- runners/google-cloud-dataflow-java/build.gradle | 2 +- .../structuredstreaming/translation/SparkSessionFactory.java | 2 +- sdks/java/io/parquet/build.gradle | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/runners/core-construction-java/build.gradle b/runners/core-construction-java/build.gradle index c529716b46d0..355533fa9f65 100644 --- a/runners/core-construction-java/build.gradle +++ b/runners/core-construction-java/build.gradle @@ -74,6 +74,6 @@ dependencies { testImplementation library.java.jackson_dataformat_yaml testImplementation project(path: ":model:fn-execution", configuration: "shadow") testImplementation project(path: ":sdks:java:core", configuration: "testRuntimeMigration") - testImplementation project(path: ":sdks:java:extensions:avro", configuration: "testRuntimeMigration") + testImplementation project(path: ":sdks:java:extensions:avro") testRuntimeOnly library.java.slf4j_jdk14 } diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index a37a198cd75b..f7f4c32364cf 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -116,7 +116,7 @@ dependencies { testImplementation library.java.junit testImplementation project(path: ":sdks:java:io:google-cloud-platform", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") - testImplementation project(path: ":sdks:java:extensions:avro", configuration: "testRuntimeMigration") + testImplementation project(path: ":sdks:java:extensions:avro") testImplementation project(path: ":sdks:java:extensions:google-cloud-platform-core", configuration: "testRuntimeMigration") testImplementation project(path: ":runners:core-construction-java", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:extensions:python", configuration: "testRuntimeMigration") diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java index 763580b6ae72..34d42c76031f 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java @@ -288,7 +288,7 @@ private void tryToRegister(Kryo kryo, String className) { try { kryo.register(Class.forName(className)); } catch (ClassNotFoundException e) { - LOG.warn("Class {}} was not found on classpath", className); + LOG.info("Class {}} was not found on classpath", className); } } } diff --git a/sdks/java/io/parquet/build.gradle b/sdks/java/io/parquet/build.gradle index bb33c8b9ba3a..84d10d2be175 100644 --- a/sdks/java/io/parquet/build.gradle +++ b/sdks/java/io/parquet/build.gradle @@ -53,7 +53,7 @@ dependencies { provided library.java.hadoop_common testImplementation library.java.hadoop_client testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") - testImplementation project(path: ":sdks:java:extensions:avro", configuration: "testRuntimeMigration") + testImplementation project(path: ":sdks:java:extensions:avro") testImplementation library.java.junit testRuntimeOnly library.java.slf4j_jdk14 testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") From 2df508cbbba1bdfef26ae13d398a7b75ddcfd6e2 Mon Sep 17 00:00:00 2001 From: Alexey Romanenko Date: Wed, 1 Feb 2023 13:11:31 +0100 Subject: [PATCH 12/13] Revert CountingSource.java --- .../apache/beam/sdk/io/CountingSource.java | 48 ++----------------- .../java/org/apache/beam/sdk/io/ReadTest.java | 3 +- 2 files changed, 5 insertions(+), 46 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java index 8babeec98caf..3e973cc2e1ad 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java @@ -20,21 +20,13 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.EOFException; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.UTFDataFormatException; import java.util.List; import java.util.NoSuchElementException; import java.util.Objects; +import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.DefaultCoder; -import org.apache.beam.sdk.coders.InstantCoder; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; import org.apache.beam.sdk.metrics.Counter; @@ -362,7 +354,7 @@ public UnboundedReader createReader(PipelineOptions options, CounterMark c @Override public Coder getCheckpointMarkCoder() { - return CounterMarkCoder.of(); + return AvroCoder.of(CountingSource.CounterMark.class); } @Override @@ -489,45 +481,11 @@ public long getSplitBacklogBytes() { } } - public static class CounterMarkCoder extends CustomCoder { - - private static final CounterMarkCoder INSTANCE = new CounterMarkCoder(); - - public static CounterMarkCoder of() { - return INSTANCE; - } - - @Override - public void encode(CountingSource.CounterMark value, OutputStream outStream) - throws CoderException, IOException { - if (value == null) { - throw new CoderException("cannot encode a null CounterMark"); - } - - DataOutputStream stream = new DataOutputStream(outStream); - stream.writeLong(value.getLastEmitted()); - InstantCoder.of().encode(value.getStartTime(), stream); - } - - @Override - public CountingSource.CounterMark decode(InputStream inStream) - throws CoderException, IOException { - try { - DataInputStream stream = new DataInputStream(inStream); - long lastEmitted = stream.readLong(); - Instant startTime = InstantCoder.of().decode(stream); - return new CountingSource.CounterMark(lastEmitted, startTime); - } catch (EOFException | UTFDataFormatException e) { - throw new CoderException(e); - } - } - } - /** * The checkpoint for an unbounded {@link CountingSource} is simply the last value produced. The * associated source object encapsulates the information needed to produce the next value. */ - @DefaultCoder(CounterMarkCoder.class) + @DefaultCoder(AvroCoder.class) public static class CounterMark implements UnboundedSource.CheckpointMark { /** The last value emitted. */ private final long lastEmitted; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java index b5c2d91996ef..315228e77688 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java @@ -38,6 +38,7 @@ import java.util.function.Consumer; import java.util.stream.Collectors; import java.util.stream.LongStream; +import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; @@ -342,7 +343,7 @@ public Coder getOutputCoder() { @Override public Coder getCheckpointMarkCoder() { - return CountingSource.CounterMarkCoder.of(); + return AvroCoder.of(CountingSource.CounterMark.class); } } From e9c7c88eba54f9d90de31bb7dce04a69be0455ed Mon Sep 17 00:00:00 2001 From: Alexey Romanenko Date: Fri, 17 Feb 2023 15:01:53 +0100 Subject: [PATCH 13/13] Up-to-date Avro usages --- CHANGES.md | 1 + sdks/java/io/file-schema-transform/build.gradle | 1 + .../AvroWriteSchemaTransformFormatProvider.java | 6 +++--- .../FileWriteSchemaTransformFormatProviders.java | 2 +- .../ParquetWriteSchemaTransformFormatProvider.java | 2 +- .../AvroFileWriteSchemaTransformFormatProviderTest.java | 6 +++--- .../ParquetFileWriteSchemaTransformFormatProviderTest.java | 2 +- .../io/gcp/bigquery/AvroGenericRecordToStorageApiProto.java | 2 +- .../StorageApiDynamicDestinationsGenericRecord.java | 2 +- .../bigquery/AvroGenericRecordToStorageApiProtoTest.java | 6 +++--- .../beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java | 2 +- 11 files changed, 17 insertions(+), 15 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 35c14046b0b9..69147cdb6b33 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -61,6 +61,7 @@ * RunInference PTransform will accept model paths as SideInputs in Python SDK. ([#24042](https://github.com/apache/beam/issues/24042)) * RunInference supports ONNX runtime in Python SDK ([#22972](https://github.com/apache/beam/issues/22972)) * Tensorflow Model Handler for RunInference in Python SDK ([#25366](https://github.com/apache/beam/issues/25366)) +* Java SDK modules migrated to use `:sdks:java:extensions:avro` ([#24748](https://github.com/apache/beam/issues/24748)) ## I/Os diff --git a/sdks/java/io/file-schema-transform/build.gradle b/sdks/java/io/file-schema-transform/build.gradle index f018e250cc28..19b10eacf7b1 100644 --- a/sdks/java/io/file-schema-transform/build.gradle +++ b/sdks/java/io/file-schema-transform/build.gradle @@ -41,6 +41,7 @@ dependencies { implementation library.java.joda_time implementation library.java.vendored_guava_26_0_jre implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation project(":sdks:java:extensions:avro") implementation project(path: ":sdks:java:io:csv") implementation project(path: ":sdks:java:io:parquet") implementation "org.apache.parquet:parquet-common:$parquet_version" diff --git a/sdks/java/io/file-schema-transform/src/main/java/org/apache/beam/sdk/io/fileschematransform/AvroWriteSchemaTransformFormatProvider.java b/sdks/java/io/file-schema-transform/src/main/java/org/apache/beam/sdk/io/fileschematransform/AvroWriteSchemaTransformFormatProvider.java index 75cf841beabe..ccf61fddb872 100644 --- a/sdks/java/io/file-schema-transform/src/main/java/org/apache/beam/sdk/io/fileschematransform/AvroWriteSchemaTransformFormatProvider.java +++ b/sdks/java/io/file-schema-transform/src/main/java/org/apache/beam/sdk/io/fileschematransform/AvroWriteSchemaTransformFormatProvider.java @@ -22,10 +22,10 @@ import com.google.auto.service.AutoService; import org.apache.avro.generic.GenericRecord; -import org.apache.beam.sdk.coders.AvroGenericCoder; -import org.apache.beam.sdk.io.AvroIO; +import org.apache.beam.sdk.extensions.avro.coders.AvroGenericCoder; +import org.apache.beam.sdk.extensions.avro.io.AvroIO; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.utils.AvroUtils; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.Values; import org.apache.beam.sdk.values.PCollection; diff --git a/sdks/java/io/file-schema-transform/src/main/java/org/apache/beam/sdk/io/fileschematransform/FileWriteSchemaTransformFormatProviders.java b/sdks/java/io/file-schema-transform/src/main/java/org/apache/beam/sdk/io/fileschematransform/FileWriteSchemaTransformFormatProviders.java index 1148f2ee116e..f1b00db896d7 100644 --- a/sdks/java/io/file-schema-transform/src/main/java/org/apache/beam/sdk/io/fileschematransform/FileWriteSchemaTransformFormatProviders.java +++ b/sdks/java/io/file-schema-transform/src/main/java/org/apache/beam/sdk/io/fileschematransform/FileWriteSchemaTransformFormatProviders.java @@ -23,12 +23,12 @@ import java.util.Optional; import org.apache.avro.generic.GenericRecord; import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.io.Compression; import org.apache.beam.sdk.io.FileIO; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.io.Providers; -import org.apache.beam.sdk.schemas.utils.AvroUtils; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TypeDescriptor; diff --git a/sdks/java/io/file-schema-transform/src/main/java/org/apache/beam/sdk/io/fileschematransform/ParquetWriteSchemaTransformFormatProvider.java b/sdks/java/io/file-schema-transform/src/main/java/org/apache/beam/sdk/io/fileschematransform/ParquetWriteSchemaTransformFormatProvider.java index 6981844d229f..aad54e698bb4 100644 --- a/sdks/java/io/file-schema-transform/src/main/java/org/apache/beam/sdk/io/fileschematransform/ParquetWriteSchemaTransformFormatProvider.java +++ b/sdks/java/io/file-schema-transform/src/main/java/org/apache/beam/sdk/io/fileschematransform/ParquetWriteSchemaTransformFormatProvider.java @@ -24,11 +24,11 @@ import java.util.Optional; import org.apache.avro.generic.GenericRecord; import org.apache.beam.sdk.coders.AvroGenericCoder; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.io.FileIO; import org.apache.beam.sdk.io.fileschematransform.FileWriteSchemaTransformConfiguration.ParquetConfiguration; import org.apache.beam.sdk.io.parquet.ParquetIO; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.utils.AvroUtils; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.Values; import org.apache.beam.sdk.values.PCollection; diff --git a/sdks/java/io/file-schema-transform/src/test/java/org/apache/beam/sdk/io/fileschematransform/AvroFileWriteSchemaTransformFormatProviderTest.java b/sdks/java/io/file-schema-transform/src/test/java/org/apache/beam/sdk/io/fileschematransform/AvroFileWriteSchemaTransformFormatProviderTest.java index 466ea475d310..b304d81a0fdc 100644 --- a/sdks/java/io/file-schema-transform/src/test/java/org/apache/beam/sdk/io/fileschematransform/AvroFileWriteSchemaTransformFormatProviderTest.java +++ b/sdks/java/io/file-schema-transform/src/test/java/org/apache/beam/sdk/io/fileschematransform/AvroFileWriteSchemaTransformFormatProviderTest.java @@ -23,10 +23,10 @@ import java.util.Optional; import java.util.stream.Collectors; import org.apache.avro.generic.GenericRecord; -import org.apache.beam.sdk.coders.AvroGenericCoder; -import org.apache.beam.sdk.io.AvroIO; +import org.apache.beam.sdk.extensions.avro.coders.AvroGenericCoder; +import org.apache.beam.sdk.extensions.avro.io.AvroIO; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.utils.AvroUtils; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; diff --git a/sdks/java/io/file-schema-transform/src/test/java/org/apache/beam/sdk/io/fileschematransform/ParquetFileWriteSchemaTransformFormatProviderTest.java b/sdks/java/io/file-schema-transform/src/test/java/org/apache/beam/sdk/io/fileschematransform/ParquetFileWriteSchemaTransformFormatProviderTest.java index ea51f235c7f0..1bd094cfdad4 100644 --- a/sdks/java/io/file-schema-transform/src/test/java/org/apache/beam/sdk/io/fileschematransform/ParquetFileWriteSchemaTransformFormatProviderTest.java +++ b/sdks/java/io/file-schema-transform/src/test/java/org/apache/beam/sdk/io/fileschematransform/ParquetFileWriteSchemaTransformFormatProviderTest.java @@ -24,9 +24,9 @@ import java.util.Optional; import java.util.stream.Collectors; import org.apache.avro.generic.GenericRecord; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.io.parquet.ParquetIO; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.utils.AvroUtils; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AvroGenericRecordToStorageApiProto.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AvroGenericRecordToStorageApiProto.java index 7becffa6d17f..783ced7d6ffd 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AvroGenericRecordToStorageApiProto.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AvroGenericRecordToStorageApiProto.java @@ -39,7 +39,7 @@ import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; -import org.apache.beam.sdk.schemas.utils.AvroUtils.TypeWithNullability; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils.TypeWithNullability; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Functions; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsGenericRecord.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsGenericRecord.java index 020ea70df539..98684db558bb 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsGenericRecord.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsGenericRecord.java @@ -23,8 +23,8 @@ import com.google.protobuf.Message; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; -import org.apache.beam.sdk.schemas.utils.AvroUtils; import org.apache.beam.sdk.transforms.SerializableFunction; import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/AvroGenericRecordToStorageApiProtoTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/AvroGenericRecordToStorageApiProtoTest.java index 94c58e414251..8b32e1b054df 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/AvroGenericRecordToStorageApiProtoTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/AvroGenericRecordToStorageApiProtoTest.java @@ -41,7 +41,7 @@ import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecordBuilder; -import org.apache.beam.sdk.schemas.utils.AvroUtils.TypeWithNullability; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Functions; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; @@ -365,8 +365,8 @@ void validateDescriptorAgainstSchema(Schema originalSchema, DescriptorProto sche .getFieldList() .forEach( p -> { - TypeWithNullability fieldSchema = - TypeWithNullability.create( + AvroUtils.TypeWithNullability fieldSchema = + AvroUtils.TypeWithNullability.create( originalSchema.getField(nameMapping.get(p.getName())).schema()); Label label = fieldSchema.getType().getType() == Schema.Type.ARRAY diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java index d5dcee095c6a..491c5ada4f3a 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java @@ -84,9 +84,9 @@ import org.apache.avro.io.Encoder; import org.apache.avro.io.EncoderFactory; import org.apache.avro.util.Utf8; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.extensions.protobuf.ByteStringCoder; import org.apache.beam.sdk.extensions.protobuf.ProtoCoder; import org.apache.beam.sdk.io.BoundedSource;