From e9c7c88eba54f9d90de31bb7dce04a69be0455ed Mon Sep 17 00:00:00 2001 From: Alexey Romanenko Date: Fri, 17 Feb 2023 15:01:53 +0100 Subject: [PATCH] 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;