From 37609ba70fab2216edc338121bf2f3a056a1e490 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> Date: Mon, 22 Apr 2024 19:36:50 -0400 Subject: [PATCH] Managed Transform protos & translation; Iceberg SchemaTransforms & translation (#30910) * iceberg write schematransform and test * IcebergIO translation and tests * unify iceberg urns and identifiers; update some comments * replace icebergIO translation with iceberg schematransform translation; fix Schema::sorted to do recursive sorting * removed new proto file and moved Managed URNs to beam_runner_api.proto; we now use SchemaTransformPayload for all schematransforms, including Managed; adding a version number to FileWriteResult encoding so that we can use it to fork in the future whhen needed * Row and Schema snake_case <-> camelCase conversion logic * Row sorted() util * use Row::sorted to fetch Managed & Iceberg row configs * use snake_case convention when translating transforms to spec; remove Managed and Iceberg urns from proto and use SCHEMA_TRANSFORM URN * perform snake_case <-> camelCase conversions directly in TypedSchemaTransformProvider * add SchemaTransformTranslation abstraction. when encountering a SCHEMA_TRANSFORM urn, fetch underlying identifier * prioritize registered providers; remove snake_case <-> camelCase conversions from python side --- ...am_PostCommit_Python_Xlang_Gcp_Direct.json | 3 +- ...m_PostCommit_Python_Xlang_IO_Dataflow.json | 3 +- ...m_PreCommit_Xlang_Generated_Transforms.yml | 2 +- .../pipeline/v1/external_transforms.proto | 4 + .../org/apache/beam/sdk/schemas/Schema.java | 54 +++- .../beam/sdk/schemas/SchemaRegistry.java | 23 ++ .../schemas/annotations/DefaultSchema.java | 18 ++ .../SchemaTransformTranslation.java | 79 ++++++ .../TypedSchemaTransformProvider.java | 35 ++- .../beam/sdk/schemas/utils/YamlUtils.java | 21 +- .../construction/PTransformTranslation.java | 11 + .../java/org/apache/beam/sdk/values/Row.java | 49 ++++ .../beam/sdk/schemas/SchemaRegistryTest.java | 49 ++++ .../apache/beam/sdk/schemas/SchemaTest.java | 100 +++++++ .../TypedSchemaTransformProviderTest.java | 24 +- .../apache/beam/sdk/util/YamlUtilsTest.java | 33 +++ .../org/apache/beam/sdk/values/RowTest.java | 90 +++++++ .../expansion/service/ExpansionService.java | 41 ++- sdks/java/io/expansion-service/build.gradle | 2 + .../FileWriteSchemaTransformProviderTest.java | 32 +-- ...xportReadSchemaTransformConfiguration.java | 14 - ...oadsWriteSchemaTransformConfiguration.java | 14 - ...ExportReadSchemaTransformProviderTest.java | 25 +- ...LoadsWriteSchemaTransformProviderTest.java | 12 +- .../bigquery/BigQueryIOTranslationTest.java | 2 +- sdks/java/io/iceberg/build.gradle | 1 + .../org/apache/beam/io/iceberg/IcebergIO.java | 93 ------- .../io/iceberg/AppendFilesToTables.java | 17 +- .../io/iceberg/AssignDestinations.java | 2 +- .../io/iceberg/DynamicDestinations.java | 2 +- .../{ => sdk}/io/iceberg/FileWriteResult.java | 12 +- .../io/iceberg/IcebergCatalogConfig.java | 2 +- .../io/iceberg/IcebergDestination.java | 2 +- .../apache/beam/sdk/io/iceberg/IcebergIO.java | 136 ++++++++++ .../IcebergReadSchemaTransformProvider.java | 134 ++++++++++ .../io/iceberg/IcebergScanConfig.java | 2 +- .../IcebergSchemaTransformCatalogConfig.java | 107 ++++++++ .../IcebergSchemaTransformTranslation.java | 88 +++++++ .../io/iceberg/IcebergTableCreateConfig.java | 2 +- .../io/iceberg/IcebergWriteResult.java | 13 +- .../IcebergWriteSchemaTransformProvider.java | 179 +++++++++++++ .../iceberg/OneTableDynamicDestinations.java | 31 ++- .../{ => sdk}/io/iceberg/PropertyBuilder.java | 2 +- .../{ => sdk}/io/iceberg/RecordWriter.java | 4 +- .../beam/{ => sdk}/io/iceberg/ScanSource.java | 2 +- .../{ => sdk}/io/iceberg/ScanTaskReader.java | 2 +- .../{ => sdk}/io/iceberg/ScanTaskSource.java | 2 +- .../io/iceberg/SchemaAndRowConversions.java | 2 +- .../beam/sdk/io/iceberg/SnapshotInfo.java | 118 +++++++++ .../io/iceberg/WriteGroupedRowsToFiles.java | 2 +- .../io/iceberg/WriteToDestinations.java | 5 +- .../io/iceberg/WriteUngroupedRowsToFiles.java | 2 +- .../{ => sdk}/io/iceberg/package-info.java | 2 +- .../io/iceberg/FileWriteResultTest.java | 17 +- .../io/iceberg/IcebergIOReadTest.java | 4 +- .../io/iceberg/IcebergIOWriteTest.java | 16 +- ...cebergReadSchemaTransformProviderTest.java | 183 +++++++++++++ ...IcebergSchemaTransformTranslationTest.java | 248 ++++++++++++++++++ ...ebergWriteSchemaTransformProviderTest.java | 175 ++++++++++++ .../{ => sdk}/io/iceberg/ScanSourceTest.java | 2 +- .../iceberg/SchemaAndRowConversionsTest.java | 2 +- .../io/iceberg/TestDataWarehouse.java | 2 +- .../{ => sdk}/io/iceberg/TestFixtures.java | 2 +- .../KafkaReadSchemaTransformProviderTest.java | 16 +- sdks/java/managed/build.gradle | 1 - .../org/apache/beam/sdk/managed/Managed.java | 39 +-- .../ManagedSchemaTransformProvider.java | 119 +++++---- .../ManagedSchemaTransformTranslation.java | 59 +++++ .../managed/ManagedTransformConstants.java | 25 ++ .../testing}/TestSchemaTransformProvider.java | 8 +- .../sdk/managed/testing/package-info.java | 20 ++ .../ManagedSchemaTransformProviderTest.java | 54 +++- ...ManagedSchemaTransformTranslationTest.java | 185 +++++++++++++ .../apache/beam/sdk/managed/ManagedTest.java | 1 + sdks/python/apache_beam/io/gcp/bigquery.py | 14 +- sdks/python/apache_beam/io/gcp/bigtableio.py | 12 +- .../transforms/external_transform_provider.py | 35 +-- .../external_transform_provider_it_test.py | 22 -- sdks/python/apache_beam/yaml/yaml_provider.py | 2 +- sdks/python/build.gradle | 2 +- sdks/python/gen_xlang_wrappers.py | 21 +- sdks/standard_expansion_services.yaml | 3 + sdks/standard_external_transforms.yaml | 5 +- 83 files changed, 2566 insertions(+), 434 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/SchemaTransformTranslation.java delete mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIO.java rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/AppendFilesToTables.java (87%) rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/AssignDestinations.java (98%) rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/DynamicDestinations.java (96%) rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/FileWriteResult.java (94%) rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/IcebergCatalogConfig.java (99%) rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/IcebergDestination.java (98%) create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/IcebergScanConfig.java (99%) create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformCatalogConfig.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslation.java rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/IcebergTableCreateConfig.java (97%) rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/IcebergWriteResult.java (85%) create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/OneTableDynamicDestinations.java (66%) rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/PropertyBuilder.java (97%) rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/RecordWriter.java (96%) rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/ScanSource.java (99%) rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/ScanTaskReader.java (99%) rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/ScanTaskSource.java (98%) rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/SchemaAndRowConversions.java (99%) create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotInfo.java rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/WriteGroupedRowsToFiles.java (98%) rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/WriteToDestinations.java (97%) rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/WriteUngroupedRowsToFiles.java (99%) rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/package-info.java (95%) rename sdks/java/io/iceberg/src/test/java/org/apache/beam/{ => sdk}/io/iceberg/FileWriteResultTest.java (91%) rename sdks/java/io/iceberg/src/test/java/org/apache/beam/{ => sdk}/io/iceberg/IcebergIOReadTest.java (97%) rename sdks/java/io/iceberg/src/test/java/org/apache/beam/{ => sdk}/io/iceberg/IcebergIOWriteTest.java (95%) create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProviderTest.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslationTest.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProviderTest.java rename sdks/java/io/iceberg/src/test/java/org/apache/beam/{ => sdk}/io/iceberg/ScanSourceTest.java (99%) rename sdks/java/io/iceberg/src/test/java/org/apache/beam/{ => sdk}/io/iceberg/SchemaAndRowConversionsTest.java (99%) rename sdks/java/io/iceberg/src/test/java/org/apache/beam/{ => sdk}/io/iceberg/TestDataWarehouse.java (99%) rename sdks/java/io/iceberg/src/test/java/org/apache/beam/{ => sdk}/io/iceberg/TestFixtures.java (99%) create mode 100644 sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedSchemaTransformTranslation.java create mode 100644 sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java rename sdks/java/managed/src/{test/java/org/apache/beam/sdk/managed => main/java/org/apache/beam/sdk/managed/testing}/TestSchemaTransformProvider.java (91%) create mode 100644 sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/testing/package-info.java create mode 100644 sdks/java/managed/src/test/java/org/apache/beam/sdk/managed/ManagedSchemaTransformTranslationTest.java diff --git a/.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Direct.json b/.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Direct.json index c4edaa85a89d..e3d6056a5de9 100644 --- a/.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Direct.json +++ b/.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Direct.json @@ -1,3 +1,4 @@ { - "comment": "Modify this file in a trivial way to cause this test suite to run" + "comment": "Modify this file in a trivial way to cause this test suite to run", + "modification": 1 } diff --git a/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Dataflow.json b/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Dataflow.json index c4edaa85a89d..e3d6056a5de9 100644 --- a/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Dataflow.json +++ b/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Dataflow.json @@ -1,3 +1,4 @@ { - "comment": "Modify this file in a trivial way to cause this test suite to run" + "comment": "Modify this file in a trivial way to cause this test suite to run", + "modification": 1 } diff --git a/.github/workflows/beam_PreCommit_Xlang_Generated_Transforms.yml b/.github/workflows/beam_PreCommit_Xlang_Generated_Transforms.yml index f8d64eb5d4a6..e8b9a466e26f 100644 --- a/.github/workflows/beam_PreCommit_Xlang_Generated_Transforms.yml +++ b/.github/workflows/beam_PreCommit_Xlang_Generated_Transforms.yml @@ -111,4 +111,4 @@ jobs: - name: run Cross-Language Wrapper Validation script uses: ./.github/actions/gradle-command-self-hosted-action with: - gradle-command: :sdks:python:test-suites:direct:crossLanguageWrapperValidationPreCommit \ No newline at end of file + gradle-command: :sdks:python:test-suites:direct:crossLanguageWrapperValidationPreCommit --info diff --git a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/external_transforms.proto b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/external_transforms.proto index 18cd02e3942c..aa9e70c7a871 100644 --- a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/external_transforms.proto +++ b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/external_transforms.proto @@ -111,6 +111,10 @@ message BuilderMethod { bytes payload = 3; } +// Payload for a Schema-aware PTransform. +// This is a transform that is aware of its input and output PCollection schemas +// and is configured using Beam Schema-compatible parameters. +// The information available in the payload can be used to instantiate the schema-aware transform. message SchemaTransformPayload { // The identifier of the SchemaTransform (typically a URN). string identifier = 1; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java index 39bee9ebaea4..acf233faf341 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java @@ -37,6 +37,7 @@ import java.util.stream.Collectors; import javax.annotation.concurrent.Immutable; import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.CaseFormat; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BiMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.HashBiMap; @@ -326,7 +327,10 @@ public static Schema of(Field... fields) { return Schema.builder().addFields(fields).build(); } - /** Returns an identical Schema with sorted fields. */ + /** + * Returns an identical Schema with lexicographically sorted fields. Recursively sorts nested + * fields. + */ public Schema sorted() { // Create a new schema and copy over the appropriate Schema object attributes: // {fields, uuid, options} @@ -336,6 +340,16 @@ public Schema sorted() { Schema sortedSchema = this.fields.stream() .sorted(Comparator.comparing(Field::getName)) + .map( + field -> { + FieldType innerType = field.getType(); + if (innerType.getRowSchema() != null) { + Schema innerSortedSchema = innerType.getRowSchema().sorted(); + innerType = innerType.toBuilder().setRowSchema(innerSortedSchema).build(); + return field.toBuilder().setType(innerType).build(); + } + return field; + }) .collect(Schema.toSchema()) .withOptions(getOptions()); sortedSchema.setUUID(getUUID()); @@ -1451,4 +1465,42 @@ public int getFieldCount() { public Options getOptions() { return this.options; } + + /** Recursively converts all field names to `snake_case`. */ + public Schema toSnakeCase() { + return this.getFields().stream() + .map( + field -> { + FieldType innerType = field.getType(); + if (innerType.getRowSchema() != null) { + Schema innerSnakeCaseSchema = innerType.getRowSchema().toSnakeCase(); + innerType = innerType.toBuilder().setRowSchema(innerSnakeCaseSchema).build(); + field = field.toBuilder().setType(innerType).build(); + } + return field + .toBuilder() + .setName(CaseFormat.LOWER_CAMEL.to(CaseFormat.LOWER_UNDERSCORE, field.getName())) + .build(); + }) + .collect(toSchema()); + } + + /** Recursively converts all field names to `lowerCamelCase`. */ + public Schema toCamelCase() { + return this.getFields().stream() + .map( + field -> { + FieldType innerType = field.getType(); + if (innerType.getRowSchema() != null) { + Schema innerCamelCaseSchema = innerType.getRowSchema().toCamelCase(); + innerType = innerType.toBuilder().setRowSchema(innerCamelCaseSchema).build(); + field = field.toBuilder().setType(innerType).build(); + } + return field + .toBuilder() + .setName(CaseFormat.LOWER_UNDERSCORE.to(CaseFormat.LOWER_CAMEL, field.getName())) + .build(); + }) + .collect(toSchema()); + } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaRegistry.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaRegistry.java index 372e2d00b768..679a1fcf54fc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaRegistry.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaRegistry.java @@ -302,6 +302,29 @@ public SchemaCoder getSchemaCoder(TypeDescriptor typeDescriptor) getFromRowFunction(typeDescriptor)); } + /** + * Retrieve a registered {@link SchemaProvider} for a given {@link TypeDescriptor}. If no schema + * exists, throws {@link * NoSuchSchemaException}. + */ + public SchemaProvider getSchemaProvider(TypeDescriptor typeDescriptor) + throws NoSuchSchemaException { + for (SchemaProvider provider : providers) { + Schema schema = provider.schemaFor(typeDescriptor); + if (schema != null) { + return provider; + } + } + throw new NoSuchSchemaException(); + } + + /** + * Retrieve a registered {@link SchemaProvider} for a given {@link Class}. If no schema exists, + * throws {@link * NoSuchSchemaException}. + */ + public SchemaProvider getSchemaProvider(Class clazz) throws NoSuchSchemaException { + return getSchemaProvider(TypeDescriptor.of(clazz)); + } + private ReturnT getProviderResult(Function f) throws NoSuchSchemaException { for (SchemaProvider provider : providers) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/annotations/DefaultSchema.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/annotations/DefaultSchema.java index 2ad3feb686fa..ddebbeb2bffe 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/annotations/DefaultSchema.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/annotations/DefaultSchema.java @@ -123,6 +123,24 @@ public ProviderAndDescriptor( }); } + /** + * Retrieves the underlying {@link SchemaProvider} for the given {@link TypeDescriptor}. If no + * provider is found, returns null. + */ + public @Nullable SchemaProvider getUnderlyingSchemaProvider( + TypeDescriptor typeDescriptor) { + ProviderAndDescriptor providerAndDescriptor = getSchemaProvider(typeDescriptor); + return providerAndDescriptor != null ? providerAndDescriptor.schemaProvider : null; + } + + /** + * Retrieves the underlying {@link SchemaProvider} for the given {@link Class}. If no provider + * is found, returns null. + */ + public @Nullable SchemaProvider getUnderlyingSchemaProvider(Class clazz) { + return getUnderlyingSchemaProvider(TypeDescriptor.of(clazz)); + } + @Override public Schema schemaFor(TypeDescriptor typeDescriptor) { ProviderAndDescriptor providerAndDescriptor = getSchemaProvider(typeDescriptor); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/SchemaTransformTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/SchemaTransformTranslation.java new file mode 100644 index 000000000000..15553411f4c1 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/SchemaTransformTranslation.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.schemas.transforms; + +import static org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods.Enum.SCHEMA_TRANSFORM; +import static org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; +import static org.apache.beam.sdk.util.construction.PTransformTranslation.TransformPayloadTranslator; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import org.apache.beam.model.pipeline.v1.ExternalTransforms; +import org.apache.beam.model.pipeline.v1.SchemaApi; +import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.schemas.SchemaTranslation; +import org.apache.beam.sdk.util.construction.BeamUrns; +import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * A {@link TransformPayloadTranslator} implementation that translates between a Java {@link + * SchemaTransform} and a protobuf payload for that transform. + */ +public class SchemaTransformTranslation { + public abstract static class SchemaTransformPayloadTranslator + implements TransformPayloadTranslator { + public abstract SchemaTransformProvider provider(); + + @Override + public String getUrn() { + return BeamUrns.getUrn(SCHEMA_TRANSFORM); + } + + @Override + @SuppressWarnings("argument") + public @Nullable FunctionSpec translate( + AppliedPTransform application, SdkComponents components) throws IOException { + SchemaApi.Schema expansionSchema = + SchemaTranslation.schemaToProto(provider().configurationSchema(), true); + Row configRow = toConfigRow(application.getTransform()); + ByteArrayOutputStream os = new ByteArrayOutputStream(); + RowCoder.of(provider().configurationSchema()).encode(configRow, os); + + return FunctionSpec.newBuilder() + .setUrn(getUrn()) + .setPayload( + ExternalTransforms.SchemaTransformPayload.newBuilder() + .setIdentifier(provider().identifier()) + .setConfigurationSchema(expansionSchema) + .setConfigurationRow(ByteString.copyFrom(os.toByteArray())) + .build() + .toByteString()) + .build(); + } + + @Override + public T fromConfigRow(Row configRow, PipelineOptions options) { + return (T) provider().from(configRow); + } + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/TypedSchemaTransformProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/TypedSchemaTransformProvider.java index e75fa27d2d16..cfd298ae87ee 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/TypedSchemaTransformProvider.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/TypedSchemaTransformProvider.java @@ -17,8 +17,10 @@ */ package org.apache.beam.sdk.schemas.transforms; +import static org.apache.beam.sdk.schemas.annotations.DefaultSchema.DefaultSchemaProvider; import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; import java.lang.reflect.ParameterizedType; import java.util.List; @@ -26,8 +28,10 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.NoSuchSchemaException; import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.SchemaProvider; import org.apache.beam.sdk.schemas.SchemaRegistry; import org.apache.beam.sdk.values.Row; @@ -37,6 +41,13 @@ * *

ConfigT should be available in the SchemaRegistry. * + *

{@link #configurationSchema()} produces a configuration {@link Schema} that is inferred from + * {@code ConfigT} using the SchemaRegistry. A Beam {@link Row} can still be used produce a {@link + * SchemaTransform} using {@link #from(Row)}, as long as the Row fits the configuration Schema. + * + *

NOTE: The inferred field names in the configuration {@link Schema} and {@link Row} follow the + * {@code snake_case} naming convention. + * *

Internal only: This interface is actively being worked on and it will likely change as * we provide implementations for more standard Beam transforms. We provide no backwards * compatibility guarantees and it should not be implemented outside of the Beam repository. @@ -77,7 +88,8 @@ Optional> dependencies(ConfigT configuration, PipelineOptions optio public final Schema configurationSchema() { try { // Sort the fields by name to ensure a consistent schema is produced - return SchemaRegistry.createDefault().getSchema(configurationClass()).sorted(); + // We also establish a `snake_case` convention for all SchemaTransform configurations + return SchemaRegistry.createDefault().getSchema(configurationClass()).sorted().toSnakeCase(); } catch (NoSuchSchemaException e) { throw new RuntimeException( "Unable to find schema for " @@ -86,6 +98,10 @@ public final Schema configurationSchema() { } } + /** + * Produces a {@link SchemaTransform} from a Row configuration. Row fields are expected to have + * `snake_case` naming convention. + */ @Override public final SchemaTransform from(Row configuration) { return from(configFromRow(configuration)); @@ -98,9 +114,20 @@ public final Optional> dependencies(Row configuration, PipelineOpti private ConfigT configFromRow(Row configuration) { try { - return SchemaRegistry.createDefault() - .getFromRowFunction(configurationClass()) - .apply(configuration); + SchemaRegistry registry = SchemaRegistry.createDefault(); + + // Configuration objects handled by the AutoValueSchema provider will expect Row fields with + // camelCase naming convention + SchemaProvider schemaProvider = registry.getSchemaProvider(configurationClass()); + if (schemaProvider.getClass().equals(DefaultSchemaProvider.class) + && checkNotNull( + ((DefaultSchemaProvider) schemaProvider) + .getUnderlyingSchemaProvider(configurationClass())) + .getClass() + .equals(AutoValueSchema.class)) { + configuration = configuration.toCamelCase(); + } + return registry.getFromRowFunction(configurationClass()).apply(configuration); } catch (NoSuchSchemaException e) { throw new RuntimeException( "Unable to find schema for " + identifier() + "SchemaTransformProvider's config"); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/YamlUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/YamlUtils.java index 5c05b2bed396..122f2d1963b9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/YamlUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/YamlUtils.java @@ -147,14 +147,27 @@ public static Row toBeamRow( } @SuppressWarnings("nullness") - public static Row toBeamRow(Map yamlMap, Schema rowSchema, boolean toCamelCase) { + public static Row toBeamRow( + @Nullable Map map, Schema rowSchema, boolean toCamelCase) { + if (map == null || map.isEmpty()) { + List requiredFields = + rowSchema.getFields().stream() + .filter(field -> !field.getType().getNullable()) + .collect(Collectors.toList()); + if (requiredFields.isEmpty()) { + return Row.nullRow(rowSchema); + } else { + throw new IllegalArgumentException( + String.format( + "Received an empty Map, but output schema contains required fields: %s", + requiredFields)); + } + } return rowSchema.getFields().stream() .map( field -> toBeamValue( - field, - yamlMap.get(maybeGetSnakeCase(field.getName(), toCamelCase)), - toCamelCase)) + field, map.get(maybeGetSnakeCase(field.getName(), toCamelCase)), toCamelCase)) .collect(toRow(rowSchema)); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PTransformTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PTransformTranslation.java index 3167c2f22b0d..69d4353d5009 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PTransformTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PTransformTranslation.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.util.construction; +import static org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods.Enum.SCHEMA_TRANSFORM; import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; @@ -30,6 +31,7 @@ import java.util.Map.Entry; import java.util.ServiceLoader; import java.util.Set; +import org.apache.beam.model.pipeline.v1.ExternalTransforms; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; import org.apache.beam.model.pipeline.v1.RunnerApi.StandardPTransforms; @@ -100,6 +102,7 @@ public class PTransformTranslation { public static final String CONFIG_ROW_KEY = "config_row"; public static final String CONFIG_ROW_SCHEMA_KEY = "config_row_schema"; + public static final String SCHEMATRANSFORM_URN_KEY = "schematransform_urn"; // DeprecatedPrimitives /** @@ -509,6 +512,14 @@ public RunnerApi.PTransform translate( components.getEnvironmentIdFor(appliedPTransform.getResourceHints())); } } + + if (spec.getUrn().equals(BeamUrns.getUrn(SCHEMA_TRANSFORM))) { + transformBuilder.putAnnotations( + SCHEMATRANSFORM_URN_KEY, + ByteString.copyFromUtf8( + ExternalTransforms.SchemaTransformPayload.parseFrom(spec.getPayload()) + .getIdentifier())); + } } Row configRow = null; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java index f5c6c7fcf34a..ee3852d70bbe 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java @@ -893,4 +893,53 @@ public static Row nullRow(Schema schema) { .addValues(Collections.nCopies(schema.getFieldCount(), null)) .build(); } + + /** Returns an equivalent {@link Row} with fields lexicographically sorted by their name. */ + public Row sorted() { + Schema sortedSchema = getSchema().sorted(); + return sortedSchema.getFields().stream() + .map( + field -> { + if (field.getType().getRowSchema() != null) { + Row innerRow = getValue(field.getName()); + if (innerRow != null) { + return innerRow.sorted(); + } + } + return (Object) getValue(field.getName()); + }) + .collect(Row.toRow(sortedSchema)); + } + + /** Returns an equivalent {@link Row} with `snake_case` field names. */ + public Row toSnakeCase() { + return getSchema().getFields().stream() + .map( + field -> { + if (field.getType().getRowSchema() != null) { + Row innerRow = getValue(field.getName()); + if (innerRow != null) { + return innerRow.toSnakeCase(); + } + } + return (Object) getValue(field.getName()); + }) + .collect(toRow(getSchema().toSnakeCase())); + } + + /** Returns an equivalent {@link Row} with `lowerCamelCase` field names. */ + public Row toCamelCase() { + return getSchema().getFields().stream() + .map( + field -> { + if (field.getType().getRowSchema() != null) { + Row innerRow = getValue(field.getName()); + if (innerRow != null) { + return innerRow.toCamelCase(); + } + } + return (Object) getValue(field.getName()); + }) + .collect(toRow(getSchema().toCamelCase())); + } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaRegistryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaRegistryTest.java index 1946dfd1fdb7..55a16e9faf39 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaRegistryTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaRegistryTest.java @@ -17,12 +17,14 @@ */ package org.apache.beam.sdk.schemas; +import static org.apache.beam.sdk.schemas.annotations.DefaultSchema.DefaultSchemaProvider; import static org.apache.beam.sdk.schemas.utils.TestJavaBeans.SIMPLE_BEAN_SCHEMA; import static org.apache.beam.sdk.schemas.utils.TestPOJOs.SIMPLE_POJO_SCHEMA; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import com.google.auto.service.AutoService; +import com.google.auto.value.AutoValue; import java.util.List; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.schemas.utils.TestJavaBeans.SimpleBean; @@ -248,4 +250,51 @@ public void testGetSchemaCoder() throws NoSuchSchemaException { thrown.expect(NoSuchSchemaException.class); registry.getSchemaCoder(Double.class); } + + @Test + public void testGetSchemaProvider() throws NoSuchSchemaException { + SchemaRegistry registry = SchemaRegistry.createDefault(); + + SchemaProvider testDefaultSchemaProvider = + registry.getSchemaProvider(TestDefaultSchemaClass.class); + assertEquals(DefaultSchemaProvider.class, testDefaultSchemaProvider.getClass()); + assertEquals( + TestDefaultSchemaProvider.class, + ((DefaultSchemaProvider) testDefaultSchemaProvider) + .getUnderlyingSchemaProvider(TestDefaultSchemaClass.class) + .getClass()); + + SchemaProvider autoValueSchemaProvider = registry.getSchemaProvider(TestAutoValue.class); + assertEquals(DefaultSchemaProvider.class, autoValueSchemaProvider.getClass()); + assertEquals( + AutoValueSchema.class, + ((DefaultSchemaProvider) autoValueSchemaProvider) + .getUnderlyingSchemaProvider(TestAutoValue.class) + .getClass()); + + SchemaProvider simpleBeanSchemaProvider = registry.getSchemaProvider(SimpleBean.class); + assertEquals(DefaultSchemaProvider.class, simpleBeanSchemaProvider.getClass()); + assertEquals( + JavaBeanSchema.class, + ((DefaultSchemaProvider) simpleBeanSchemaProvider) + .getUnderlyingSchemaProvider(SimpleBean.class) + .getClass()); + } + + @DefaultSchema(AutoValueSchema.class) + @AutoValue + public abstract static class TestAutoValue { + public static Builder builder() { + return new AutoValue_SchemaRegistryTest_TestAutoValue.Builder(); + } + + public abstract String getStr(); + + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setStr(String str); + + public abstract TestAutoValue build(); + } + } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTest.java index 9797556618ea..5f3749120496 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTest.java @@ -194,6 +194,71 @@ public void testCollector() { assertEquals(FieldType.STRING, schema.getField(1).getType()); } + @Test + public void testToSnakeCase() { + Schema innerSchema = + Schema.builder() + .addStringField("myFirstNestedStringField") + .addStringField("mySecondNestedStringField") + .build(); + Schema schema = + Schema.builder() + .addStringField("myFirstStringField") + .addStringField("mySecondStringField") + .addRowField("myRowField", innerSchema) + .build(); + + Schema expectedInnerSnakeCaseSchema = + Schema.builder() + .addStringField("my_first_nested_string_field") + .addStringField("my_second_nested_string_field") + .build(); + Schema expectedSnakeCaseSchema = + Schema.builder() + .addStringField("my_first_string_field") + .addStringField("my_second_string_field") + .addRowField("my_row_field", expectedInnerSnakeCaseSchema) + .build(); + + assertEquals( + expectedInnerSnakeCaseSchema, + schema.toSnakeCase().getField("my_row_field").getType().getRowSchema()); + assertEquals(expectedSnakeCaseSchema, schema.toSnakeCase()); + } + + @Test + public void testToCamelCase() { + Schema innerSchema = + Schema.builder() + .addStringField("my_first_nested_string_field") + .addStringField("my_second_nested_string_field") + .build(); + Schema schema = + Schema.builder() + .addStringField("my_first_string_field") + .addStringField("my_second_string_field") + .addRowField("my_row_field", innerSchema) + .build(); + + Schema expectedInnerCamelCaseSchema = + Schema.builder() + .addStringField("myFirstNestedStringField") + .addStringField("mySecondNestedStringField") + .build(); + Schema expectedCamelCaseSchema = + Schema.builder() + .addStringField("myFirstStringField") + .addStringField("mySecondStringField") + .addRowField("myRowField", expectedInnerCamelCaseSchema) + .build(); + + assertTrue(schema.toCamelCase().hasField("myRowField")); + assertEquals( + expectedInnerCamelCaseSchema, + schema.toCamelCase().getField("myRowField").getType().getRowSchema()); + assertEquals(expectedCamelCaseSchema, schema.toCamelCase()); + } + @Test public void testSorted() { Options testOptions = @@ -232,6 +297,41 @@ public void testSorted() { sortedSchema.getEncodingPositions())); } + @Test + public void testNestedSorted() { + Schema unsortedNestedSchema = + Schema.builder().addStringField("bb").addInt32Field("aa").addStringField("cc").build(); + Schema unsortedSchema = + Schema.builder() + .addStringField("d") + .addInt32Field("c") + .addRowField("e", unsortedNestedSchema) + .addStringField("b") + .addByteField("a") + .build(); + + Schema sortedSchema = unsortedSchema.sorted(); + + Schema expectedInnerSortedSchema = + Schema.builder().addInt32Field("aa").addStringField("bb").addStringField("cc").build(); + Schema expectedSortedSchema = + Schema.builder() + .addByteField("a") + .addStringField("b") + .addInt32Field("c") + .addStringField("d") + .addRowField("e", expectedInnerSortedSchema) + .build(); + + assertTrue(unsortedSchema.equivalent(sortedSchema)); + assertEquals(expectedSortedSchema.getFields(), sortedSchema.getFields()); + assertEquals(expectedSortedSchema.getEncodingPositions(), sortedSchema.getEncodingPositions()); + assertEquals(expectedInnerSortedSchema, sortedSchema.getField("e").getType().getRowSchema()); + assertEquals( + expectedInnerSortedSchema.getEncodingPositions(), + sortedSchema.getField("e").getType().getRowSchema().getEncodingPositions()); + } + @Test public void testSortedMethodIncludesAllSchemaFields() { // This test is most likely to break when new Schema object attributes are added. It is designed diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/TypedSchemaTransformProviderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/TypedSchemaTransformProviderTest.java index 6b5ccbff4e42..2eef0e30f805 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/TypedSchemaTransformProviderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/TypedSchemaTransformProviderTest.java @@ -43,12 +43,13 @@ public class TypedSchemaTransformProviderTest { @DefaultSchema(AutoValueSchema.class) @AutoValue abstract static class Configuration { - abstract String getField1(); + abstract String getStringField(); - abstract Integer getField2(); + abstract Integer getIntegerField(); - static Configuration create(String field1, int field2) { - return new AutoValue_TypedSchemaTransformProviderTest_Configuration(field1, field2); + static Configuration create(String stringField, int integerField) { + return new AutoValue_TypedSchemaTransformProviderTest_Configuration( + stringField, integerField); } }; @@ -90,7 +91,8 @@ public List outputCollectionNames() { public Optional> dependencies( Configuration configuration, PipelineOptions options) { return Optional.of( - Arrays.asList(configuration.getField1(), String.valueOf(configuration.getField2()))); + Arrays.asList( + configuration.getStringField(), String.valueOf(configuration.getIntegerField()))); } } @@ -128,8 +130,8 @@ public void testFrom() { Row inputConfig = Row.withSchema(provider.configurationSchema()) - .withFieldValue("field1", "field1") - .withFieldValue("field2", Integer.valueOf(13)) + .withFieldValue("string_field", "field1") + .withFieldValue("integer_field", Integer.valueOf(13)) .build(); Configuration outputConfig = ((FakeSchemaTransform) provider.from(inputConfig)).config; @@ -137,8 +139,8 @@ public void testFrom() { ((FakeSchemaTransform) minimalProvider.from(inputConfig)).config; for (Configuration config : Arrays.asList(outputConfig, minimalOutputConfig)) { - assertEquals("field1", config.getField1()); - assertEquals(13, config.getField2().intValue()); + assertEquals("field1", config.getStringField()); + assertEquals(13, config.getIntegerField().intValue()); } assertEquals("Description of fake provider", provider.description()); } @@ -148,8 +150,8 @@ public void testDependencies() { SchemaTransformProvider provider = new FakeTypedSchemaIOProvider(); Row inputConfig = Row.withSchema(provider.configurationSchema()) - .withFieldValue("field1", "field1") - .withFieldValue("field2", Integer.valueOf(13)) + .withFieldValue("string_field", "field1") + .withFieldValue("integer_field", Integer.valueOf(13)) .build(); assertEquals(Arrays.asList("field1", "13"), provider.dependencies(inputConfig, null).get()); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/YamlUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/YamlUtilsTest.java index 6e6984dde3a6..bf032aed7b5c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/YamlUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/YamlUtilsTest.java @@ -21,11 +21,13 @@ import java.math.BigDecimal; import java.util.Arrays; +import java.util.Map; import java.util.stream.Collectors; import java.util.stream.IntStream; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.utils.YamlUtils; import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.CaseFormat; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.BaseEncoding; import org.junit.Rule; import org.junit.Test; @@ -225,4 +227,35 @@ public void testNestedArray() { assertEquals(expectedRow, YamlUtils.toBeamRow(yamlString, schema)); } + + private static final Schema FLAT_SCHEMA_CAMEL_CASE = + Schema.builder() + .addFields( + FLAT_SCHEMA.getFields().stream() + .map( + field -> + field.withName( + CaseFormat.LOWER_UNDERSCORE.to( + CaseFormat.LOWER_CAMEL, field.getName()))) + .collect(Collectors.toList())) + .build(); + + private static final Map FLAT_MAP = + FLAT_SCHEMA.getFields().stream() + .collect( + Collectors.toMap( + Schema.Field::getName, + field -> Preconditions.checkArgumentNotNull(FLAT_ROW.getValue(field.getName())))); + + @Test + public void testSnakeCaseMapToCamelCaseRow() { + Row expectedRow = + FLAT_SCHEMA.getFields().stream() + .map(field -> Preconditions.checkStateNotNull(FLAT_ROW.getValue(field.getName()))) + .collect(Row.toRow(FLAT_SCHEMA_CAMEL_CASE)); + + Row convertedRow = YamlUtils.toBeamRow(FLAT_MAP, FLAT_SCHEMA_CAMEL_CASE, true); + + assertEquals(expectedRow, convertedRow); + } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/RowTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/RowTest.java index 6182a6791577..37583f0ceedc 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/RowTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/RowTest.java @@ -806,4 +806,94 @@ public void testWithFieldValues() { assertEquals( enumerationType.valueOf("zero"), row.getLogicalTypeValue(0, EnumerationType.Value.class)); } + + @Test + public void testSorted() { + Schema unsortedNestedSchema = + Schema.builder().addStringField("bb").addStringField("aa").addStringField("cc").build(); + Schema unsortedSchema = + Schema.builder() + .addStringField("d") + .addStringField("c") + .addRowField("e", unsortedNestedSchema) + .addStringField("b") + .addStringField("a") + .build(); + Row unsortedNestedRow = + Row.withSchema(unsortedNestedSchema).addValues("bb_val", "aa_val", "cc_val").build(); + Row unsortedRow = + Row.withSchema(unsortedSchema) + .addValues("d_val", "c_val", unsortedNestedRow, "b_val", "a_val") + .build(); + + Row expectedSortedNestedRow = + Row.withSchema(unsortedNestedSchema.sorted()) + .addValues("aa_val", "bb_val", "cc_val") + .build(); + Row expectedSortedRow = + Row.withSchema(unsortedSchema.sorted()) + .addValues("a_val", "b_val", "c_val", "d_val", expectedSortedNestedRow) + .build(); + + Row sortedRow = unsortedRow.sorted(); + assertEquals(expectedSortedNestedRow, sortedRow.getRow("e")); + assertEquals(expectedSortedRow, sortedRow); + assertNotEquals(unsortedRow, sortedRow); + } + + @Test + public void testToSnakeCase() { + Schema innerSchema = + Schema.builder() + .addStringField("myFirstNestedStringField") + .addStringField("mySecondNestedStringField") + .build(); + Schema schema = + Schema.builder() + .addStringField("myFirstStringField") + .addStringField("mySecondStringField") + .addRowField("myRowField", innerSchema) + .build(); + + Row innerRow = Row.withSchema(innerSchema).addValues("nested1", "nested2").build(); + Row row = Row.withSchema(schema).addValues("str1", "str2", innerRow).build(); + + Row expectedSnakeCaseInnerRow = + Row.withSchema(innerSchema.toSnakeCase()).addValues("nested1", "nested2").build(); + Row expectedSnakeCaseRow = + Row.withSchema(schema.toSnakeCase()) + .addValues("str1", "str2", expectedSnakeCaseInnerRow) + .build(); + + assertEquals(expectedSnakeCaseInnerRow, row.toSnakeCase().getRow("my_row_field")); + assertEquals(expectedSnakeCaseRow, row.toSnakeCase()); + } + + @Test + public void testToCamelCase() { + Schema innerSchema = + Schema.builder() + .addStringField("my_first_nested_string_field") + .addStringField("my_second_nested_string_field") + .build(); + Schema schema = + Schema.builder() + .addStringField("my_first_string_field") + .addStringField("my_second_string_field") + .addRowField("my_row_field", innerSchema) + .build(); + + Row innerRow = Row.withSchema(innerSchema).addValues("nested1", "nested2").build(); + Row row = Row.withSchema(schema).addValues("str1", "str2", innerRow).build(); + + Row expectedCamelCaseInnerRow = + Row.withSchema(innerSchema.toCamelCase()).addValues("nested1", "nested2").build(); + Row expectedCamelCaseRow = + Row.withSchema(schema.toCamelCase()) + .addValues("str1", "str2", expectedCamelCaseInnerRow) + .build(); + + assertEquals(expectedCamelCaseInnerRow, row.toCamelCase().getRow("myRowField")); + assertEquals(expectedCamelCaseRow, row.toCamelCase()); + } } diff --git a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java index 8dd50794df39..4a272a1ef595 100644 --- a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java +++ b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.expansion.service; +import static org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods.Enum.SCHEMA_TRANSFORM; +import static org.apache.beam.sdk.schemas.transforms.SchemaTransformTranslation.SchemaTransformPayloadTranslator; import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn; import static org.apache.beam.sdk.util.construction.PTransformTranslation.READ_TRANSFORM_URN; @@ -41,6 +43,7 @@ import org.apache.beam.model.expansion.v1.ExpansionApi.DiscoverSchemaTransformResponse; import org.apache.beam.model.expansion.v1.ExpansionApi.SchemaTransformConfig; import org.apache.beam.model.expansion.v1.ExpansionServiceGrpc; +import org.apache.beam.model.pipeline.v1.ExternalTransforms; import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods; import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExternalConfigurationPayload; import org.apache.beam.model.pipeline.v1.RunnerApi; @@ -68,6 +71,7 @@ import org.apache.beam.sdk.transforms.ExternalTransformBuilder; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.util.construction.BeamUrns; import org.apache.beam.sdk.util.construction.Environments; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.PTransformTranslation.TransformPayloadTranslator; @@ -81,6 +85,7 @@ import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ServerBuilder; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; @@ -170,6 +175,9 @@ public Map knownTransforms() { + translator + " to the Expansion Service since it did not produce a unique URN."); continue; + } else if (urn.equals(BeamUrns.getUrn(SCHEMA_TRANSFORM)) + && translator instanceof SchemaTransformPayloadTranslator) { + urn = ((SchemaTransformPayloadTranslator) translator).provider().identifier(); } } catch (Exception e) { LOG.info( @@ -584,17 +592,28 @@ private Map loadRegisteredTransforms() { String urn = request.getTransform().getSpec().getUrn(); - TransformProvider transformProvider = null; - if (getUrn(ExpansionMethods.Enum.JAVA_CLASS_LOOKUP).equals(urn)) { - AllowList allowList = - pipelineOptions.as(ExpansionServiceOptions.class).getJavaClassLookupAllowlist(); - assert allowList != null; - transformProvider = new JavaClassLookupTransformProvider(allowList); - } else if (getUrn(ExpansionMethods.Enum.SCHEMA_TRANSFORM).equals(urn)) { - transformProvider = ExpansionServiceSchemaTransformProvider.of(); - } else { - transformProvider = getRegisteredTransforms().get(urn); - if (transformProvider == null) { + TransformProvider transformProvider = getRegisteredTransforms().get(urn); + if (transformProvider == null) { + if (getUrn(ExpansionMethods.Enum.JAVA_CLASS_LOOKUP).equals(urn)) { + AllowList allowList = + pipelineOptions.as(ExpansionServiceOptions.class).getJavaClassLookupAllowlist(); + assert allowList != null; + transformProvider = new JavaClassLookupTransformProvider(allowList); + } else if (getUrn(SCHEMA_TRANSFORM).equals(urn)) { + try { + String underlyingIdentifier = + ExternalTransforms.SchemaTransformPayload.parseFrom( + request.getTransform().getSpec().getPayload()) + .getIdentifier(); + transformProvider = getRegisteredTransforms().get(underlyingIdentifier); + } catch (InvalidProtocolBufferException e) { + throw new RuntimeException(e); + } + transformProvider = + transformProvider != null + ? transformProvider + : ExpansionServiceSchemaTransformProvider.of(); + } else { throw new UnsupportedOperationException( "Unknown urn: " + request.getTransform().getSpec().getUrn()); } diff --git a/sdks/java/io/expansion-service/build.gradle b/sdks/java/io/expansion-service/build.gradle index 9ab71ff16d3d..15e0baaa7318 100644 --- a/sdks/java/io/expansion-service/build.gradle +++ b/sdks/java/io/expansion-service/build.gradle @@ -33,6 +33,8 @@ ext.summary = "Expansion service serving several Java IOs" dependencies { implementation project(":sdks:java:expansion-service") permitUnusedDeclared project(":sdks:java:expansion-service") // BEAM-11761 + implementation project(":sdks:java:io:iceberg") + permitUnusedDeclared project(":sdks:java:io:iceberg") // BEAM-11761 implementation project(":sdks:java:io:kafka") permitUnusedDeclared project(":sdks:java:io:kafka") // BEAM-11761 implementation project(":sdks:java:io:kafka:upgrade") diff --git a/sdks/java/io/file-schema-transform/src/test/java/org/apache/beam/sdk/io/fileschematransform/FileWriteSchemaTransformProviderTest.java b/sdks/java/io/file-schema-transform/src/test/java/org/apache/beam/sdk/io/fileschematransform/FileWriteSchemaTransformProviderTest.java index c8494446deda..e733969eb3d1 100644 --- a/sdks/java/io/file-schema-transform/src/test/java/org/apache/beam/sdk/io/fileschematransform/FileWriteSchemaTransformProviderTest.java +++ b/sdks/java/io/file-schema-transform/src/test/java/org/apache/beam/sdk/io/fileschematransform/FileWriteSchemaTransformProviderTest.java @@ -30,15 +30,12 @@ import static org.junit.Assert.assertTrue; import org.apache.beam.sdk.io.fileschematransform.FileWriteSchemaTransformProvider.FileWriteSchemaTransform; -import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.sdk.values.Row; -import org.apache.beam.sdk.values.TypeDescriptor; import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; @@ -47,11 +44,6 @@ /** Tests for {@link FileWriteSchemaTransformProvider}. */ @RunWith(JUnit4.class) public class FileWriteSchemaTransformProviderTest { - private static final AutoValueSchema AUTO_VALUE_SCHEMA = new AutoValueSchema(); - private static final TypeDescriptor TYPE_DESCRIPTOR = - TypeDescriptor.of(FileWriteSchemaTransformConfiguration.class); - private static final SerializableFunction TO_ROW_FN = - AUTO_VALUE_SCHEMA.toRowFunction(TYPE_DESCRIPTOR); private static final FileWriteSchemaTransformProvider PROVIDER = new FileWriteSchemaTransformProvider(); @@ -60,8 +52,7 @@ public class FileWriteSchemaTransformProviderTest { @Test public void receivedUnexpectedInputTagsThrowsAnError() { - SchemaTransform transform = - PROVIDER.from(rowConfiguration(defaultConfiguration().setFormat(JSON).build())); + SchemaTransform transform = PROVIDER.from(defaultConfiguration().setFormat(JSON).build()); PCollectionRowTuple empty = PCollectionRowTuple.empty(errorPipeline); IllegalArgumentException emptyInputError = assertThrows(IllegalArgumentException.class, () -> empty.apply(transform)); @@ -94,31 +85,28 @@ public void receivedUnexpectedInputTagsThrowsAnError() { @Test public void formatMapsToFileWriteSchemaFormatTransform() { - Row avro = rowConfiguration(defaultConfiguration().setFormat(AVRO).build()); FileWriteSchemaTransformFormatProvider avroFormatProvider = - ((FileWriteSchemaTransform) PROVIDER.from(avro)).getProvider(); + ((FileWriteSchemaTransform) PROVIDER.from(defaultConfiguration().setFormat(AVRO).build())) + .getProvider(); assertTrue(avroFormatProvider instanceof AvroWriteSchemaTransformFormatProvider); - Row json = rowConfiguration(defaultConfiguration().setFormat(JSON).build()); FileWriteSchemaTransformFormatProvider jsonFormatProvider = - ((FileWriteSchemaTransform) PROVIDER.from(json)).getProvider(); + ((FileWriteSchemaTransform) PROVIDER.from(defaultConfiguration().setFormat(JSON).build())) + .getProvider(); assertTrue(jsonFormatProvider instanceof JsonWriteSchemaTransformFormatProvider); - Row parquet = rowConfiguration(defaultConfiguration().setFormat(PARQUET).build()); FileWriteSchemaTransformFormatProvider parquetFormatProvider = - ((FileWriteSchemaTransform) PROVIDER.from(parquet)).getProvider(); + ((FileWriteSchemaTransform) + PROVIDER.from(defaultConfiguration().setFormat(PARQUET).build())) + .getProvider(); assertTrue(parquetFormatProvider instanceof ParquetWriteSchemaTransformFormatProvider); - Row xml = rowConfiguration(defaultConfiguration().setFormat(XML).build()); FileWriteSchemaTransformFormatProvider xmlFormatProvider = - ((FileWriteSchemaTransform) PROVIDER.from(xml)).getProvider(); + ((FileWriteSchemaTransform) PROVIDER.from(defaultConfiguration().setFormat(XML).build())) + .getProvider(); assertTrue(xmlFormatProvider instanceof XmlWriteSchemaTransformFormatProvider); } - private static Row rowConfiguration(FileWriteSchemaTransformConfiguration configuration) { - return TO_ROW_FN.apply(configuration); - } - private static FileWriteSchemaTransformConfiguration.Builder defaultConfiguration() { return FileWriteSchemaTransformConfiguration.builder() .setFilenamePrefix(FileWriteSchemaTransformProviderTest.class.getSimpleName()); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryExportReadSchemaTransformConfiguration.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryExportReadSchemaTransformConfiguration.java index 9eb4ac87ff98..a5ca8e1a2bbb 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryExportReadSchemaTransformConfiguration.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryExportReadSchemaTransformConfiguration.java @@ -21,9 +21,6 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.values.Row; -import org.apache.beam.sdk.values.TypeDescriptor; /** * Configuration for reading from BigQuery. @@ -46,17 +43,6 @@ public static Builder builder() { return new AutoValue_BigQueryExportReadSchemaTransformConfiguration.Builder(); } - private static final AutoValueSchema AUTO_VALUE_SCHEMA = new AutoValueSchema(); - private static final TypeDescriptor - TYPE_DESCRIPTOR = TypeDescriptor.of(BigQueryExportReadSchemaTransformConfiguration.class); - private static final SerializableFunction - ROW_SERIALIZABLE_FUNCTION = AUTO_VALUE_SCHEMA.toRowFunction(TYPE_DESCRIPTOR); - - /** Serializes configuration to a {@link Row}. */ - Row toBeamRow() { - return ROW_SERIALIZABLE_FUNCTION.apply(this); - } - /** Configures the BigQuery read job with the SQL query. */ @Nullable public abstract String getQuery(); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryFileLoadsWriteSchemaTransformConfiguration.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryFileLoadsWriteSchemaTransformConfiguration.java index 0c91ce1cd078..f634b5ec6f60 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryFileLoadsWriteSchemaTransformConfiguration.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryFileLoadsWriteSchemaTransformConfiguration.java @@ -20,9 +20,6 @@ import com.google.auto.value.AutoValue; import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.values.Row; -import org.apache.beam.sdk.values.TypeDescriptor; /** * Configuration for writing to BigQuery. @@ -42,12 +39,6 @@ public static Builder builder() { return new AutoValue_BigQueryFileLoadsWriteSchemaTransformConfiguration.Builder(); } - private static final AutoValueSchema AUTO_VALUE_SCHEMA = new AutoValueSchema(); - private static final TypeDescriptor - TYPE_DESCRIPTOR = TypeDescriptor.of(BigQueryFileLoadsWriteSchemaTransformConfiguration.class); - private static final SerializableFunction - ROW_SERIALIZABLE_FUNCTION = AUTO_VALUE_SCHEMA.toRowFunction(TYPE_DESCRIPTOR); - /** * Writes to the given table specification. See {@link BigQueryIO.Write#to(String)}} for the * expected format. @@ -60,11 +51,6 @@ public static Builder builder() { /** Specifies what to do with existing data in the table, in case the table already exists. */ public abstract String getWriteDisposition(); - /** Serializes configuration to a {@link Row}. */ - Row toBeamRow() { - return ROW_SERIALIZABLE_FUNCTION.apply(this); - } - @AutoValue.Builder public abstract static class Builder { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryExportReadSchemaTransformProviderTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryExportReadSchemaTransformProviderTest.java index c732434b2bac..ab6dd8b6c737 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryExportReadSchemaTransformProviderTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryExportReadSchemaTransformProviderTest.java @@ -39,7 +39,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.transforms.SchemaTransformProvider; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; @@ -153,11 +152,11 @@ public void testQuery() { for (Pair> caze : cases) { Map want = DisplayData.from(caze.getRight()).asMap(); - SchemaTransformProvider provider = new BigQueryExportReadSchemaTransformProvider(); + BigQueryExportReadSchemaTransformProvider provider = + new BigQueryExportReadSchemaTransformProvider(); BigQueryExportReadSchemaTransformConfiguration configuration = caze.getLeft().build(); - Row configurationRow = configuration.toBeamRow(); BigQueryExportSchemaTransform schemaTransform = - (BigQueryExportSchemaTransform) provider.from(configurationRow); + (BigQueryExportSchemaTransform) provider.from(configuration); Map got = DisplayData.from(schemaTransform.toTypedRead()).asMap(); assertEquals(want, got); } @@ -165,12 +164,12 @@ public void testQuery() { @Test public void testExtract() { - SchemaTransformProvider provider = new BigQueryExportReadSchemaTransformProvider(); + BigQueryExportReadSchemaTransformProvider provider = + new BigQueryExportReadSchemaTransformProvider(); BigQueryExportReadSchemaTransformConfiguration configuration = BigQueryExportReadSchemaTransformConfiguration.builder().setTableSpec(TABLE_SPEC).build(); - Row configurationRow = configuration.toBeamRow(); BigQueryExportSchemaTransform schemaTransform = - (BigQueryExportSchemaTransform) provider.from(configurationRow); + (BigQueryExportSchemaTransform) provider.from(configuration); schemaTransform.setTestBigQueryServices(fakeBigQueryServices); PCollectionRowTuple input = PCollectionRowTuple.empty(p); @@ -185,7 +184,8 @@ public void testExtract() { @Test public void testInvalidConfiguration() { - SchemaTransformProvider provider = new BigQueryExportReadSchemaTransformProvider(); + BigQueryExportReadSchemaTransformProvider provider = + new BigQueryExportReadSchemaTransformProvider(); for (Pair< BigQueryExportReadSchemaTransformConfiguration.Builder, ? extends Class> @@ -207,9 +207,8 @@ public void testInvalidConfiguration() { BigQueryExportReadSchemaTransformConfiguration.builder() .setUseStandardSql(true), IllegalArgumentException.class))) { - Row configurationRow = caze.getLeft().build().toBeamRow(); BigQueryExportSchemaTransform schemaTransform = - (BigQueryExportSchemaTransform) provider.from(configurationRow); + (BigQueryExportSchemaTransform) provider.from(caze.getLeft().build()); schemaTransform.setTestBigQueryServices(fakeBigQueryServices); PCollectionRowTuple empty = PCollectionRowTuple.empty(p); assertThrows(caze.getRight(), () -> empty.apply(schemaTransform)); @@ -218,12 +217,12 @@ public void testInvalidConfiguration() { @Test public void testInvalidInput() { - SchemaTransformProvider provider = new BigQueryExportReadSchemaTransformProvider(); + BigQueryExportReadSchemaTransformProvider provider = + new BigQueryExportReadSchemaTransformProvider(); BigQueryExportReadSchemaTransformConfiguration configuration = BigQueryExportReadSchemaTransformConfiguration.builder().setTableSpec(TABLE_SPEC).build(); - Row configurationRow = configuration.toBeamRow(); BigQueryExportSchemaTransform schemaTransform = - (BigQueryExportSchemaTransform) provider.from(configurationRow); + (BigQueryExportSchemaTransform) provider.from(configuration); schemaTransform.setTestBigQueryServices(fakeBigQueryServices); PCollectionRowTuple input = PCollectionRowTuple.of("badinput", p.apply(Create.of(ROWS))); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryFileLoadsWriteSchemaTransformProviderTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryFileLoadsWriteSchemaTransformProviderTest.java index 194746d9825a..dd8bb9fc8664 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryFileLoadsWriteSchemaTransformProviderTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryFileLoadsWriteSchemaTransformProviderTest.java @@ -41,7 +41,6 @@ import org.apache.beam.sdk.schemas.Schema.Field; import org.apache.beam.sdk.schemas.Schema.FieldType; import org.apache.beam.sdk.schemas.io.InvalidConfigurationException; -import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.display.DisplayData; @@ -108,16 +107,16 @@ public void tearDown() { @Test public void testLoad() throws IOException, InterruptedException { - SchemaTransformProvider provider = new BigQueryFileLoadsWriteSchemaTransformProvider(); + BigQueryFileLoadsWriteSchemaTransformProvider provider = + new BigQueryFileLoadsWriteSchemaTransformProvider(); BigQueryFileLoadsWriteSchemaTransformConfiguration configuration = BigQueryFileLoadsWriteSchemaTransformConfiguration.builder() .setTableSpec(BigQueryHelpers.toTableSpec(TABLE_REFERENCE)) .setWriteDisposition(WriteDisposition.WRITE_TRUNCATE.name()) .setCreateDisposition(CreateDisposition.CREATE_IF_NEEDED.name()) .build(); - Row configurationRow = configuration.toBeamRow(); BigQueryWriteSchemaTransform schemaTransform = - (BigQueryWriteSchemaTransform) provider.from(configurationRow); + (BigQueryWriteSchemaTransform) provider.from(configuration); schemaTransform.setTestBigQueryServices(fakeBigQueryServices); String tag = provider.inputCollectionNames().get(0); PCollectionRowTuple input = @@ -254,9 +253,10 @@ public void validatePCollectionRowTupleInput() { private BigQueryWriteSchemaTransform transformFrom( BigQueryFileLoadsWriteSchemaTransformConfiguration configuration) { - SchemaTransformProvider provider = new BigQueryFileLoadsWriteSchemaTransformProvider(); + BigQueryFileLoadsWriteSchemaTransformProvider provider = + new BigQueryFileLoadsWriteSchemaTransformProvider(); BigQueryWriteSchemaTransform transform = - (BigQueryWriteSchemaTransform) provider.from(configuration.toBeamRow()); + (BigQueryWriteSchemaTransform) provider.from(configuration); transform.setTestBigQueryServices(fakeBigQueryServices); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslationTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslationTest.java index ce4c80adb954..3654304f4390 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslationTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslationTest.java @@ -43,7 +43,7 @@ public class BigQueryIOTranslationTest { // A mapping from Read transform builder methods to the corresponding schema fields in - // KafkaIOTranslation. + // BigQueryIOTranslation. static final Map READ_TRANSFORM_SCHEMA_MAPPING = new HashMap<>(); static { diff --git a/sdks/java/io/iceberg/build.gradle b/sdks/java/io/iceberg/build.gradle index caa72bef7536..e721b98f1029 100644 --- a/sdks/java/io/iceberg/build.gradle +++ b/sdks/java/io/iceberg/build.gradle @@ -43,6 +43,7 @@ def hive_version = "3.1.3" dependencies { implementation library.java.vendored_guava_32_1_2_jre implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation project(":sdks:java:managed") implementation library.java.slf4j_api implementation library.java.joda_time implementation "org.apache.parquet:parquet-column:$parquet_version" diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIO.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIO.java deleted file mode 100644 index 8b8d852e106b..000000000000 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIO.java +++ /dev/null @@ -1,93 +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.io.iceberg; - -import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; - -import org.apache.beam.sdk.io.Read; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.values.PBegin; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.Row; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; -import org.checkerframework.checker.nullness.qual.Nullable; - -public class IcebergIO { - - public static WriteRows writeToDynamicDestinations( - IcebergCatalogConfig catalog, DynamicDestinations dynamicDestinations) { - return new WriteRows(catalog, dynamicDestinations); - } - - public static ReadTable readTable(IcebergCatalogConfig catalogConfig, TableIdentifier tableId) { - return new ReadTable(catalogConfig, tableId); - } - - static class WriteRows extends PTransform, IcebergWriteResult> { - - private final IcebergCatalogConfig catalog; - private final DynamicDestinations dynamicDestinations; - - private WriteRows(IcebergCatalogConfig catalog, DynamicDestinations dynamicDestinations) { - this.catalog = catalog; - this.dynamicDestinations = dynamicDestinations; - } - - @Override - public IcebergWriteResult expand(PCollection input) { - - return input - .apply("Set Destination Metadata", new AssignDestinations(dynamicDestinations)) - .apply( - "Write Rows to Destinations", new WriteToDestinations(catalog, dynamicDestinations)); - } - } - - public static class ReadTable extends PTransform> { - - private final IcebergCatalogConfig catalogConfig; - private final transient @Nullable TableIdentifier tableId; - - private TableIdentifier getTableId() { - return checkStateNotNull( - tableId, "Transient field tableId null; it should not be accessed after serialization"); - } - - private ReadTable(IcebergCatalogConfig catalogConfig, TableIdentifier tableId) { - this.catalogConfig = catalogConfig; - this.tableId = tableId; - } - - @Override - public PCollection expand(PBegin input) { - - Table table = catalogConfig.catalog().loadTable(getTableId()); - - return input.apply( - Read.from( - new ScanSource( - IcebergScanConfig.builder() - .setCatalogConfig(catalogConfig) - .setScanType(IcebergScanConfig.ScanType.TABLE) - .setTableIdentifier(getTableId()) - .setSchema(SchemaAndRowConversions.icebergSchemaToBeamSchema(table.schema())) - .build()))); - } - } -} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/AppendFilesToTables.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/AppendFilesToTables.java similarity index 87% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/AppendFilesToTables.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/AppendFilesToTables.java index 468bdcf5790d..e4ba60001824 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/AppendFilesToTables.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/AppendFilesToTables.java @@ -15,10 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg; +package org.apache.beam.sdk.io.iceberg; import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.coders.SerializableCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.GroupByKey; @@ -30,14 +29,13 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; class AppendFilesToTables - extends PTransform, PCollection>> { + extends PTransform, PCollection>> { private final IcebergCatalogConfig catalogConfig; @@ -46,7 +44,7 @@ class AppendFilesToTables } @Override - public PCollection> expand(PCollection writtenFiles) { + public PCollection> expand(PCollection writtenFiles) { // Apply any sharded writes and flatten everything for catalog updates return writtenFiles @@ -63,11 +61,11 @@ public String apply(FileWriteResult input) { .apply( "Append metadata updates to tables", ParDo.of(new AppendFilesToTablesDoFn(catalogConfig))) - .setCoder(KvCoder.of(StringUtf8Coder.of(), SerializableCoder.of(Snapshot.class))); + .setCoder(KvCoder.of(StringUtf8Coder.of(), SnapshotInfo.CODER)); } private static class AppendFilesToTablesDoFn - extends DoFn>, KV> { + extends DoFn>, KV> { private final IcebergCatalogConfig catalogConfig; @@ -87,7 +85,7 @@ private Catalog getCatalog() { @ProcessElement public void processElement( @Element KV> element, - OutputReceiver> out, + OutputReceiver> out, BoundedWindow window) { Table table = getCatalog().loadTable(TableIdentifier.parse(element.getKey())); AppendFiles update = table.newAppend(); @@ -96,7 +94,8 @@ public void processElement( } update.commit(); out.outputWithTimestamp( - KV.of(element.getKey(), table.currentSnapshot()), window.maxTimestamp()); + KV.of(element.getKey(), SnapshotInfo.fromSnapshot(table.currentSnapshot())), + window.maxTimestamp()); } } } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/AssignDestinations.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/AssignDestinations.java similarity index 98% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/AssignDestinations.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/AssignDestinations.java index a9c4e82a7767..4008fcc6f5bd 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/AssignDestinations.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/AssignDestinations.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg; +package org.apache.beam.sdk.io.iceberg; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.transforms.DoFn; diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/DynamicDestinations.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/DynamicDestinations.java similarity index 96% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/DynamicDestinations.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/DynamicDestinations.java index a395086403f6..6fc3c139bdc1 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/DynamicDestinations.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/DynamicDestinations.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg; +package org.apache.beam.sdk.io.iceberg; import java.io.Serializable; import org.apache.beam.sdk.schemas.Schema; diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/FileWriteResult.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/FileWriteResult.java similarity index 94% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/FileWriteResult.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/FileWriteResult.java index 09645cdd827b..c12febc03f48 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/FileWriteResult.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/FileWriteResult.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg; +package org.apache.beam.sdk.io.iceberg; import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; @@ -72,6 +72,7 @@ abstract static class Builder { } public static class FileWriteResultCoder extends StructuredCoder { + static final int VERSION = 0; private static final FileWriteResultCoder SINGLETON = new FileWriteResultCoder(); private static final Coder tableIdentifierCoder = StringUtf8Coder.of(); @@ -92,6 +93,12 @@ private static Schema getDataFileAvroSchema(FileWriteResult fileWriteResult) { @Override public void encode(FileWriteResult value, OutputStream outStream) throws CoderException, IOException { + // "version" of this coder. + // If breaking changes are introduced (e.g. from Beam, Iceberg, Avro, etc..), + // then update this version and create a fork in decode() below for the new decode logic. + // This helps keep the pipeline update-compatible + outStream.write(VERSION); + tableIdentifierCoder.encode(value.getTableIdentifier().toString(), outStream); partitionSpecCoder.encode(value.getPartitionSpec(), outStream); dataFileBytesCoder.encode( @@ -100,6 +107,9 @@ public void encode(FileWriteResult value, OutputStream outStream) @Override public FileWriteResult decode(InputStream inStream) throws CoderException, IOException { + // Forking logic can be added here depending on the version of this coder + assert inStream.read() == 0; + TableIdentifier tableId = TableIdentifier.parse(tableIdentifierCoder.decode(inStream)); PartitionSpec partitionSpec = partitionSpecCoder.decode(inStream); DataFile dataFile = diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergCatalogConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java similarity index 99% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergCatalogConfig.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java index 06a29ac14652..fefef4aa4917 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergCatalogConfig.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg; +package org.apache.beam.sdk.io.iceberg; import com.google.auto.value.AutoValue; import java.io.Serializable; diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergDestination.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergDestination.java similarity index 98% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergDestination.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergDestination.java index dcd69c87e9d0..78cf5f8e2985 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergDestination.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergDestination.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg; +package org.apache.beam.sdk.io.iceberg; import com.google.auto.value.AutoValue; import org.apache.iceberg.FileFormat; diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java new file mode 100644 index 000000000000..75a35e6f8a30 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.iceberg; + +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import com.google.auto.value.AutoValue; +import java.util.Arrays; +import java.util.List; +import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Predicates; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.checkerframework.checker.nullness.qual.Nullable; + +public class IcebergIO { + + public static WriteRows writeRows(IcebergCatalogConfig catalog) { + return new AutoValue_IcebergIO_WriteRows.Builder().setCatalogConfig(catalog).build(); + } + + @AutoValue + public abstract static class WriteRows extends PTransform, IcebergWriteResult> { + + abstract IcebergCatalogConfig getCatalogConfig(); + + abstract @Nullable TableIdentifier getTableIdentifier(); + + abstract @Nullable DynamicDestinations getDynamicDestinations(); + + abstract Builder toBuilder(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setCatalogConfig(IcebergCatalogConfig config); + + abstract Builder setTableIdentifier(TableIdentifier identifier); + + abstract Builder setDynamicDestinations(DynamicDestinations destinations); + + abstract WriteRows build(); + } + + public WriteRows to(TableIdentifier identifier) { + return toBuilder().setTableIdentifier(identifier).build(); + } + + public WriteRows to(DynamicDestinations destinations) { + return toBuilder().setDynamicDestinations(destinations).build(); + } + + @Override + public IcebergWriteResult expand(PCollection input) { + List allToArgs = Arrays.asList(getTableIdentifier(), getDynamicDestinations()); + Preconditions.checkArgument( + 1 == allToArgs.stream().filter(Predicates.notNull()).count(), + "Must set exactly one of table identifier or dynamic destinations object."); + + DynamicDestinations destinations = getDynamicDestinations(); + if (destinations == null) { + destinations = + DynamicDestinations.singleTable(Preconditions.checkNotNull(getTableIdentifier())); + } + return input + .apply("Set Destination Metadata", new AssignDestinations(destinations)) + .apply( + "Write Rows to Destinations", + new WriteToDestinations(getCatalogConfig(), destinations)); + } + } + + public static ReadRows readRows(IcebergCatalogConfig catalogConfig) { + return new AutoValue_IcebergIO_ReadRows.Builder().setCatalogConfig(catalogConfig).build(); + } + + @AutoValue + public abstract static class ReadRows extends PTransform> { + + abstract IcebergCatalogConfig getCatalogConfig(); + + abstract @Nullable TableIdentifier getTableIdentifier(); + + abstract Builder toBuilder(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setCatalogConfig(IcebergCatalogConfig config); + + abstract Builder setTableIdentifier(TableIdentifier identifier); + + abstract ReadRows build(); + } + + public ReadRows from(TableIdentifier tableIdentifier) { + return toBuilder().setTableIdentifier(tableIdentifier).build(); + } + + @Override + public PCollection expand(PBegin input) { + TableIdentifier tableId = + checkStateNotNull(getTableIdentifier(), "Must set a table to read from."); + + Table table = getCatalogConfig().catalog().loadTable(tableId); + + return input.apply( + Read.from( + new ScanSource( + IcebergScanConfig.builder() + .setCatalogConfig(getCatalogConfig()) + .setScanType(IcebergScanConfig.ScanType.TABLE) + .setTableIdentifier(tableId) + .setSchema(SchemaAndRowConversions.icebergSchemaToBeamSchema(table.schema())) + .build()))); + } + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java new file mode 100644 index 000000000000..fb32e18d9374 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.iceberg; + +import com.google.auto.service.AutoService; +import com.google.auto.value.AutoValue; +import java.util.Collections; +import java.util.List; +import org.apache.beam.sdk.io.iceberg.IcebergReadSchemaTransformProvider.Config; +import org.apache.beam.sdk.managed.ManagedTransformConstants; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.NoSuchSchemaException; +import org.apache.beam.sdk.schemas.SchemaRegistry; +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.values.PCollection; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; +import org.apache.iceberg.catalog.TableIdentifier; + +/** + * SchemaTransform implementation for {@link IcebergIO#readRows}. Reads records from Iceberg and + * outputs a {@link org.apache.beam.sdk.values.PCollection} of Beam {@link + * org.apache.beam.sdk.values.Row}s. + */ +@AutoService(SchemaTransformProvider.class) +public class IcebergReadSchemaTransformProvider extends TypedSchemaTransformProvider { + static final String OUTPUT_TAG = "output"; + + @Override + protected SchemaTransform from(Config configuration) { + configuration.validate(); + return new IcebergReadSchemaTransform(configuration); + } + + @Override + public List outputCollectionNames() { + return Collections.singletonList(OUTPUT_TAG); + } + + @Override + public String identifier() { + return ManagedTransformConstants.ICEBERG_READ; + } + + @DefaultSchema(AutoValueSchema.class) + @AutoValue + public abstract static class Config { + public static Builder builder() { + return new AutoValue_IcebergReadSchemaTransformProvider_Config.Builder(); + } + + public abstract String getTable(); + + public abstract IcebergSchemaTransformCatalogConfig getCatalogConfig(); + + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setTable(String tables); + + public abstract Builder setCatalogConfig(IcebergSchemaTransformCatalogConfig catalogConfig); + + public abstract Config build(); + } + + public void validate() { + getCatalogConfig().validate(); + } + } + + static class IcebergReadSchemaTransform extends SchemaTransform { + private final Config configuration; + + IcebergReadSchemaTransform(Config configuration) { + this.configuration = configuration; + } + + Row getConfigurationRow() { + try { + // To stay consistent with our SchemaTransform configuration naming conventions, + // we sort lexicographically and convert field names to snake_case + return SchemaRegistry.createDefault() + .getToRowFunction(Config.class) + .apply(configuration) + .sorted() + .toSnakeCase(); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e); + } + } + + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + IcebergSchemaTransformCatalogConfig catalogConfig = configuration.getCatalogConfig(); + + IcebergCatalogConfig.Builder catalogBuilder = + IcebergCatalogConfig.builder().setName(catalogConfig.getCatalogName()); + + if (!Strings.isNullOrEmpty(catalogConfig.getCatalogType())) { + catalogBuilder = catalogBuilder.setIcebergCatalogType(catalogConfig.getCatalogType()); + } + if (!Strings.isNullOrEmpty(catalogConfig.getWarehouseLocation())) { + catalogBuilder = catalogBuilder.setWarehouseLocation(catalogConfig.getWarehouseLocation()); + } + + PCollection output = + input + .getPipeline() + .apply( + IcebergIO.readRows(catalogBuilder.build()) + .from(TableIdentifier.parse(configuration.getTable()))); + + return PCollectionRowTuple.of(OUTPUT_TAG, output); + } + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergScanConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java similarity index 99% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergScanConfig.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java index c2bda838997a..60372b172af7 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergScanConfig.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg; +package org.apache.beam.sdk.io.iceberg; import com.google.auto.value.AutoValue; import java.io.Serializable; diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformCatalogConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformCatalogConfig.java new file mode 100644 index 000000000000..473bcaa3cfd9 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformCatalogConfig.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.iceberg; + +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; + +import com.google.auto.value.AutoValue; +import java.util.Set; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.NoSuchSchemaException; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.SchemaRegistry; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; +import org.apache.beam.sdk.util.Preconditions; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; +import org.apache.iceberg.CatalogUtil; +import org.checkerframework.checker.nullness.qual.Nullable; + +@DefaultSchema(AutoValueSchema.class) +@AutoValue +public abstract class IcebergSchemaTransformCatalogConfig { + public static Builder builder() { + return new AutoValue_IcebergSchemaTransformCatalogConfig.Builder(); + } + + public abstract String getCatalogName(); + + @SchemaFieldDescription("Valid types are: {hadoop, hive, rest}") + public abstract @Nullable String getCatalogType(); + + public abstract @Nullable String getCatalogImplementation(); + + public abstract @Nullable String getWarehouseLocation(); + + @AutoValue.Builder + public abstract static class Builder { + + public abstract Builder setCatalogName(String catalogName); + + public abstract Builder setCatalogType(String catalogType); + + public abstract Builder setCatalogImplementation(String catalogImplementation); + + public abstract Builder setWarehouseLocation(String warehouseLocation); + + public abstract IcebergSchemaTransformCatalogConfig build(); + } + + public static final Schema SCHEMA; + + static { + try { + // To stay consistent with our SchemaTransform configuration naming conventions, + // we sort lexicographically and convert field names to snake_case + SCHEMA = + SchemaRegistry.createDefault() + .getSchema(IcebergSchemaTransformCatalogConfig.class) + .sorted() + .toSnakeCase(); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e); + } + } + + @SuppressWarnings("argument") + public Row toRow() { + return Row.withSchema(SCHEMA) + .withFieldValue("catalog_name", getCatalogName()) + .withFieldValue("catalog_type", getCatalogType()) + .withFieldValue("catalog_implementation", getCatalogImplementation()) + .withFieldValue("warehouse_location", getWarehouseLocation()) + .build(); + } + + public static final Set VALID_CATALOG_TYPES = + Sets.newHashSet( + CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP, + CatalogUtil.ICEBERG_CATALOG_TYPE_HIVE, + CatalogUtil.ICEBERG_CATALOG_TYPE_REST); + + public void validate() { + if (Strings.isNullOrEmpty(getCatalogType())) { + checkArgument( + VALID_CATALOG_TYPES.contains(Preconditions.checkArgumentNotNull(getCatalogType())), + "Invalid catalog type. Please pick one of %s", + VALID_CATALOG_TYPES); + } + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslation.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslation.java new file mode 100644 index 000000000000..c33f7d6261e8 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslation.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.iceberg; + +import static org.apache.beam.sdk.io.iceberg.IcebergReadSchemaTransformProvider.IcebergReadSchemaTransform; +import static org.apache.beam.sdk.io.iceberg.IcebergWriteSchemaTransformProvider.IcebergWriteSchemaTransform; +import static org.apache.beam.sdk.schemas.transforms.SchemaTransformTranslation.SchemaTransformPayloadTranslator; + +import com.google.auto.service.AutoService; +import java.util.Map; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.PTransformTranslation.TransformPayloadTranslator; +import org.apache.beam.sdk.util.construction.TransformPayloadTranslatorRegistrar; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; + +@SuppressWarnings({"rawtypes", "nullness"}) +public class IcebergSchemaTransformTranslation { + static class IcebergReadSchemaTransformTranslator + extends SchemaTransformPayloadTranslator { + @Override + public SchemaTransformProvider provider() { + return new IcebergReadSchemaTransformProvider(); + } + + @Override + public Row toConfigRow(IcebergReadSchemaTransform transform) { + return transform.getConfigurationRow(); + } + } + + @AutoService(TransformPayloadTranslatorRegistrar.class) + public static class ReadRegistrar implements TransformPayloadTranslatorRegistrar { + @Override + @SuppressWarnings({ + "rawtypes", + }) + public Map, ? extends TransformPayloadTranslator> + getTransformPayloadTranslators() { + return ImmutableMap., TransformPayloadTranslator>builder() + .put(IcebergReadSchemaTransform.class, new IcebergReadSchemaTransformTranslator()) + .build(); + } + } + + static class IcebergWriteSchemaTransformTranslator + extends SchemaTransformPayloadTranslator { + @Override + public SchemaTransformProvider provider() { + return new IcebergWriteSchemaTransformProvider(); + } + + @Override + public Row toConfigRow(IcebergWriteSchemaTransform transform) { + return transform.getConfigurationRow(); + } + } + + @AutoService(TransformPayloadTranslatorRegistrar.class) + public static class WriteRegistrar implements TransformPayloadTranslatorRegistrar { + @Override + @SuppressWarnings({ + "rawtypes", + }) + public Map, ? extends TransformPayloadTranslator> + getTransformPayloadTranslators() { + return ImmutableMap., TransformPayloadTranslator>builder() + .put(IcebergWriteSchemaTransform.class, new IcebergWriteSchemaTransformTranslator()) + .build(); + } + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergTableCreateConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergTableCreateConfig.java similarity index 97% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergTableCreateConfig.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergTableCreateConfig.java index c1041f026c31..eeddf280d0af 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergTableCreateConfig.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergTableCreateConfig.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg; +package org.apache.beam.sdk.io.iceberg; import com.google.auto.value.AutoValue; import org.apache.iceberg.PartitionSpec; diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteResult.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteResult.java similarity index 85% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteResult.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteResult.java index 94ac576674ed..8e2549b5dadb 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteResult.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteResult.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg; +package org.apache.beam.sdk.io.iceberg; import java.util.Map; import org.apache.beam.sdk.Pipeline; @@ -27,22 +27,21 @@ import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.Snapshot; public final class IcebergWriteResult implements POutput { - private static final TupleTag> SNAPSHOTS_TAG = - new TupleTag>() {}; + private static final TupleTag> SNAPSHOTS_TAG = + new TupleTag>() {}; private final Pipeline pipeline; - private final PCollection> snapshots; + private final PCollection> snapshots; - public PCollection> getSnapshots() { + public PCollection> getSnapshots() { return snapshots; } - IcebergWriteResult(Pipeline pipeline, PCollection> snapshots) { + IcebergWriteResult(Pipeline pipeline, PCollection> snapshots) { this.pipeline = pipeline; this.snapshots = snapshots; } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java new file mode 100644 index 000000000000..b490693a9adb --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.iceberg; + +import com.google.auto.service.AutoService; +import com.google.auto.value.AutoValue; +import java.util.Collections; +import java.util.List; +import org.apache.beam.sdk.io.iceberg.IcebergWriteSchemaTransformProvider.Config; +import org.apache.beam.sdk.managed.ManagedTransformConstants; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.NoSuchSchemaException; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.SchemaRegistry; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; +import org.apache.iceberg.catalog.TableIdentifier; + +/** + * SchemaTransform implementation for {@link IcebergIO#writeRows}. Writes Beam Rows to Iceberg and + * outputs a {@code PCollection} representing snapshots created in the process. + */ +@AutoService(SchemaTransformProvider.class) +public class IcebergWriteSchemaTransformProvider extends TypedSchemaTransformProvider { + + static final String INPUT_TAG = "input"; + static final String OUTPUT_TAG = "output"; + + static final Schema OUTPUT_SCHEMA = + Schema.builder().addStringField("table").addFields(SnapshotInfo.SCHEMA.getFields()).build(); + + @Override + public String description() { + return "Writes Beam Rows to Iceberg.\n" + + "Returns a PCollection representing the snapshots produced in the process, with the following schema:\n" + + "{\"table\" (str), \"operation\" (str), \"summary\" (map[str, str]), \"manifestListLocation\" (str)}"; + } + + @Override + protected SchemaTransform from(Config configuration) { + configuration.validate(); + return new IcebergWriteSchemaTransform(configuration); + } + + @Override + public List inputCollectionNames() { + return Collections.singletonList(INPUT_TAG); + } + + @Override + public List outputCollectionNames() { + return Collections.singletonList(OUTPUT_TAG); + } + + @Override + public String identifier() { + return ManagedTransformConstants.ICEBERG_WRITE; + } + + @DefaultSchema(AutoValueSchema.class) + @AutoValue + public abstract static class Config { + public static Builder builder() { + return new AutoValue_IcebergWriteSchemaTransformProvider_Config.Builder(); + } + + @SchemaFieldDescription("Identifier of the Iceberg table to write to.") + public abstract String getTable(); + + @SchemaFieldDescription("Configuration parameters used to set up the Iceberg catalog.") + public abstract IcebergSchemaTransformCatalogConfig getCatalogConfig(); + + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setTable(String tables); + + public abstract Builder setCatalogConfig(IcebergSchemaTransformCatalogConfig catalogConfig); + + public abstract Config build(); + } + + public void validate() { + getCatalogConfig().validate(); + } + } + + static class IcebergWriteSchemaTransform extends SchemaTransform { + private final Config configuration; + + IcebergWriteSchemaTransform(Config configuration) { + this.configuration = configuration; + } + + Row getConfigurationRow() { + try { + // To stay consistent with our SchemaTransform configuration naming conventions, + // we sort lexicographically and convert field names to snake_case + return SchemaRegistry.createDefault() + .getToRowFunction(Config.class) + .apply(configuration) + .sorted() + .toSnakeCase(); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e); + } + } + + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + + PCollection rows = input.get(INPUT_TAG); + + IcebergSchemaTransformCatalogConfig catalogConfig = configuration.getCatalogConfig(); + + IcebergCatalogConfig.Builder catalogBuilder = + IcebergCatalogConfig.builder().setName(catalogConfig.getCatalogName()); + + if (!Strings.isNullOrEmpty(catalogConfig.getCatalogType())) { + catalogBuilder = catalogBuilder.setIcebergCatalogType(catalogConfig.getCatalogType()); + } + if (!Strings.isNullOrEmpty(catalogConfig.getWarehouseLocation())) { + catalogBuilder = catalogBuilder.setWarehouseLocation(catalogConfig.getWarehouseLocation()); + } + + // TODO: support dynamic destinations + IcebergWriteResult result = + rows.apply( + IcebergIO.writeRows(catalogBuilder.build()) + .to(TableIdentifier.parse(configuration.getTable()))); + + PCollection snapshots = + result + .getSnapshots() + .apply(MapElements.via(new SnapshotToRow())) + .setRowSchema(OUTPUT_SCHEMA); + + return PCollectionRowTuple.of(OUTPUT_TAG, snapshots); + } + + @VisibleForTesting + static class SnapshotToRow extends SimpleFunction, Row> { + @Override + public Row apply(KV input) { + SnapshotInfo snapshot = input.getValue(); + + return Row.withSchema(OUTPUT_SCHEMA) + .addValue(input.getKey()) + .addValues(snapshot.toRow().getValues()) + .build(); + } + } + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/OneTableDynamicDestinations.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/OneTableDynamicDestinations.java similarity index 66% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/OneTableDynamicDestinations.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/OneTableDynamicDestinations.java index dc6815235a06..e09fdf171fd6 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/OneTableDynamicDestinations.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/OneTableDynamicDestinations.java @@ -15,27 +15,34 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg; +package org.apache.beam.sdk.io.iceberg; +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.iceberg.FileFormat; import org.apache.iceberg.catalog.TableIdentifier; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; -class OneTableDynamicDestinations implements DynamicDestinations { +class OneTableDynamicDestinations implements DynamicDestinations, Externalizable { private static final Schema EMPTY_SCHEMA = Schema.builder().build(); private static final Row EMPTY_ROW = Row.nullRow(EMPTY_SCHEMA); // TableId represented as String for serializability - private final String tableIdString; + private transient @MonotonicNonNull String tableIdString; private transient @MonotonicNonNull TableIdentifier tableId; - private TableIdentifier getTableIdentifier() { + @VisibleForTesting + TableIdentifier getTableIdentifier() { if (tableId == null) { - tableId = TableIdentifier.parse(tableIdString); + tableId = TableIdentifier.parse(Preconditions.checkNotNull(tableIdString)); } return tableId; } @@ -62,4 +69,18 @@ public IcebergDestination instantiateDestination(Row dest) { .setFileFormat(FileFormat.PARQUET) .build(); } + + // Need a public default constructor for custom serialization + public OneTableDynamicDestinations() {} + + @Override + public void writeExternal(ObjectOutput out) throws IOException { + out.writeUTF(Preconditions.checkNotNull(tableIdString)); + } + + @Override + public void readExternal(ObjectInput in) throws IOException { + tableIdString = in.readUTF(); + tableId = TableIdentifier.parse(tableIdString); + } } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/PropertyBuilder.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/PropertyBuilder.java similarity index 97% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/PropertyBuilder.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/PropertyBuilder.java index 53184c70dfca..a0901b5c0691 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/PropertyBuilder.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/PropertyBuilder.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg; +package org.apache.beam.sdk.io.iceberg; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/RecordWriter.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriter.java similarity index 96% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/RecordWriter.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriter.java index 51df91ccadd3..aa203eb6eb66 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/RecordWriter.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriter.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg; +package org.apache.beam.sdk.io.iceberg; -import static org.apache.beam.io.iceberg.SchemaAndRowConversions.rowToRecord; +import static org.apache.beam.sdk.io.iceberg.SchemaAndRowConversions.rowToRecord; import java.io.IOException; import org.apache.beam.sdk.values.Row; diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/ScanSource.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ScanSource.java similarity index 99% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/ScanSource.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ScanSource.java index fb77c79281fa..ed2f2eda767e 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/ScanSource.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ScanSource.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg; +package org.apache.beam.sdk.io.iceberg; import java.io.IOException; import java.util.ArrayList; diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/ScanTaskReader.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ScanTaskReader.java similarity index 99% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/ScanTaskReader.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ScanTaskReader.java index 3b9aae56f69a..52e6d60c1fbd 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/ScanTaskReader.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ScanTaskReader.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg; +package org.apache.beam.sdk.io.iceberg; import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/ScanTaskSource.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ScanTaskSource.java similarity index 98% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/ScanTaskSource.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ScanTaskSource.java index 8c44d174b625..66a355819169 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/ScanTaskSource.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ScanTaskSource.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg; +package org.apache.beam.sdk.io.iceberg; import java.io.IOException; import java.util.List; diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/SchemaAndRowConversions.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SchemaAndRowConversions.java similarity index 99% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/SchemaAndRowConversions.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SchemaAndRowConversions.java index e0210c96d685..27c591ad81fa 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/SchemaAndRowConversions.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SchemaAndRowConversions.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg; +package org.apache.beam.sdk.io.iceberg; import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotInfo.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotInfo.java new file mode 100644 index 000000000000..03240eff9f7b --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotInfo.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.iceberg; + +import com.google.auto.value.AutoValue; +import java.util.Map; +import javax.annotation.Nullable; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.NoSuchSchemaException; +import org.apache.beam.sdk.schemas.Schema; +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.values.Row; +import org.apache.iceberg.Snapshot; + +/** + * This is an AutoValue representation of an Iceberg {@link Snapshot}. + * + *

Note: this only includes the subset of fields in {@link Snapshot} that are Beam + * Schema-compatible. + */ +@DefaultSchema(AutoValueSchema.class) +@AutoValue +public abstract class SnapshotInfo { + public static SnapshotInfo fromSnapshot(Snapshot snapshot) { + return SnapshotInfo.builder() + .setSequenceNumber(snapshot.sequenceNumber()) + .setSnapshotId(snapshot.snapshotId()) + .setParentId(snapshot.parentId()) + .setTimestampMillis(snapshot.timestampMillis()) + .setOperation(snapshot.operation()) + .setSummary(snapshot.summary()) + .setManifestListLocation(snapshot.manifestListLocation()) + .setSchemaId(snapshot.schemaId()) + .build(); + } + + public Row toRow() { + try { + return SchemaRegistry.createDefault() + .getToRowFunction(SnapshotInfo.class) + .apply(this) + .sorted(); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e); + } + } + + public static final SchemaCoder CODER; + public static final Schema SCHEMA; + + static { + try { + SchemaRegistry registry = SchemaRegistry.createDefault(); + CODER = registry.getSchemaCoder(SnapshotInfo.class); + SCHEMA = registry.getSchema(SnapshotInfo.class).sorted(); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e); + } + } + + public static Builder builder() { + return new AutoValue_SnapshotInfo.Builder(); + } + + public abstract long getSequenceNumber(); + + public abstract long getSnapshotId(); + + public abstract @Nullable Long getParentId(); + + public abstract long getTimestampMillis(); + + public abstract @Nullable String getOperation(); + + public abstract @Nullable Map getSummary(); + + public abstract @Nullable String getManifestListLocation(); + + public abstract @Nullable Integer getSchemaId(); + + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setSequenceNumber(long sequenceNumber); + + public abstract Builder setSnapshotId(long snapshotId); + + public abstract Builder setParentId(Long parentId); + + public abstract Builder setTimestampMillis(long timestampMillis); + + public abstract Builder setOperation(String operation); + + public abstract Builder setSummary(Map summary); + + public abstract Builder setManifestListLocation(String manifestListLocation); + + public abstract Builder setSchemaId(Integer schemaId); + + public abstract SnapshotInfo build(); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteGroupedRowsToFiles.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteGroupedRowsToFiles.java similarity index 98% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteGroupedRowsToFiles.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteGroupedRowsToFiles.java index 7a152b602581..731a9fefb49d 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteGroupedRowsToFiles.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteGroupedRowsToFiles.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg; +package org.apache.beam.sdk.io.iceberg; import java.io.IOException; import java.util.UUID; diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteToDestinations.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteToDestinations.java similarity index 97% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteToDestinations.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteToDestinations.java index 22e27a4df7ca..65fd551c782a 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteToDestinations.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteToDestinations.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg; +package org.apache.beam.sdk.io.iceberg; import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; @@ -33,7 +33,6 @@ import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.ShardedKey; -import org.apache.iceberg.Snapshot; class WriteToDestinations extends PTransform, IcebergWriteResult> { @@ -115,7 +114,7 @@ public KV, Row> apply(Row elem) { .apply("Flatten Written Files", Flatten.pCollections()); // Apply any sharded writes and flatten everything for catalog updates - PCollection> snapshots = + PCollection> snapshots = allWrittenFiles.apply(new AppendFilesToTables(catalogConfig)); return new IcebergWriteResult(input.getPipeline(), snapshots); diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteUngroupedRowsToFiles.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteUngroupedRowsToFiles.java similarity index 99% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteUngroupedRowsToFiles.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteUngroupedRowsToFiles.java index 240678c83697..917aab9e55c5 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteUngroupedRowsToFiles.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteUngroupedRowsToFiles.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg; +package org.apache.beam.sdk.io.iceberg; import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/package-info.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/package-info.java similarity index 95% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/package-info.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/package-info.java index f97ff98677f9..e32a9eb6fbe8 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/package-info.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/package-info.java @@ -17,4 +17,4 @@ */ /** Iceberg connectors. */ -package org.apache.beam.io.iceberg; +package org.apache.beam.sdk.io.iceberg; diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/FileWriteResultTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/FileWriteResultTest.java similarity index 91% rename from sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/FileWriteResultTest.java rename to sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/FileWriteResultTest.java index 6a19c510d52d..644130593152 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/FileWriteResultTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/FileWriteResultTest.java @@ -15,13 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg; +package org.apache.beam.sdk.io.iceberg; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertEquals; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; import java.io.Serializable; import java.util.List; import java.util.UUID; @@ -141,6 +144,18 @@ public void testDecodeEncodeEqual() throws Exception { } } + @Test + public void testDecodeEncodeVersionNumber() throws Exception { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + ByteArrayInputStream in; + for (FileWriteResult value : getTestValues()) { + TEST_CODER.encode(value, out); + in = new ByteArrayInputStream(out.toByteArray()); + + assertEquals(FileWriteResult.FileWriteResultCoder.VERSION, in.read()); + } + } + @Rule public ExpectedException thrown = ExpectedException.none(); @Test diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergIOReadTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOReadTest.java similarity index 97% rename from sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergIOReadTest.java rename to sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOReadTest.java index 0ae63439f76e..12d86811e604 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergIOReadTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOReadTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg; +package org.apache.beam.sdk.io.iceberg; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -102,7 +102,7 @@ public void testSimpleScan() throws Exception { PCollection output = testPipeline - .apply(IcebergIO.readTable(catalogConfig, tableId)) + .apply(IcebergIO.readRows(catalogConfig).from(tableId)) .apply(ParDo.of(new PrintRow())) .setCoder( RowCoder.of( diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergIOWriteTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOWriteTest.java similarity index 95% rename from sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergIOWriteTest.java rename to sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOWriteTest.java index 011ab2662457..e04eaf48cb3d 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergIOWriteTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOWriteTest.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg; +package org.apache.beam.sdk.io.iceberg; -import static org.apache.beam.io.iceberg.SchemaAndRowConversions.rowToRecord; +import static org.apache.beam.sdk.io.iceberg.SchemaAndRowConversions.rowToRecord; import static org.hamcrest.MatcherAssert.assertThat; import java.io.Serializable; @@ -82,12 +82,10 @@ public void testSimpleAppend() throws Exception { .setWarehouseLocation(warehouse.location) .build(); - DynamicDestinations destination = DynamicDestinations.singleTable(tableId); - testPipeline .apply("Records To Add", Create.of(TestFixtures.asRows(TestFixtures.FILE1SNAPSHOT1))) .setRowSchema(SchemaAndRowConversions.icebergSchemaToBeamSchema(TestFixtures.SCHEMA)) - .apply("Append To Table", IcebergIO.writeToDynamicDestinations(catalog, destination)); + .apply("Append To Table", IcebergIO.writeRows(catalog).to(tableId)); LOG.info("Executing pipeline"); testPipeline.run().waitUntilFinish(); @@ -118,7 +116,7 @@ public void testDynamicDestinationsWithoutSpillover() throws Exception { .setWarehouseLocation(warehouse.location) .build(); - DynamicDestinations destination = + DynamicDestinations dynamicDestinations = new DynamicDestinations() { private final Schema schema = Schema.builder().addInt64Field("tableNumber").build(); @@ -154,7 +152,7 @@ public IcebergDestination instantiateDestination(Row dest) { TestFixtures.FILE1SNAPSHOT2, TestFixtures.FILE1SNAPSHOT3)))) .setRowSchema(SchemaAndRowConversions.icebergSchemaToBeamSchema(TestFixtures.SCHEMA)) - .apply("Append To Table", IcebergIO.writeToDynamicDestinations(catalog, destination)); + .apply("Append To Table", IcebergIO.writeRows(catalog).to(dynamicDestinations)); LOG.info("Executing pipeline"); testPipeline.run().waitUntilFinish(); @@ -208,7 +206,7 @@ public void testDynamicDestinationsWithSpillover() throws Exception { .setWarehouseLocation(warehouse.location) .build(); - DynamicDestinations destination = + DynamicDestinations dynamicDestinations = new DynamicDestinations() { private final Schema schema = Schema.builder().addInt64Field("tableNumber").build(); @@ -237,7 +235,7 @@ public IcebergDestination instantiateDestination(Row dest) { testPipeline .apply("Records To Add", Create.of(TestFixtures.asRows(elements))) .setRowSchema(SchemaAndRowConversions.icebergSchemaToBeamSchema(TestFixtures.SCHEMA)) - .apply("Append To Table", IcebergIO.writeToDynamicDestinations(catalog, destination)); + .apply("Append To Table", IcebergIO.writeRows(catalog).to(dynamicDestinations)); LOG.info("Executing pipeline"); testPipeline.run().waitUntilFinish(); diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProviderTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProviderTest.java new file mode 100644 index 000000000000..22514b035450 --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProviderTest.java @@ -0,0 +1,183 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.iceberg; + +import static org.apache.beam.sdk.io.iceberg.IcebergReadSchemaTransformProvider.OUTPUT_TAG; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; + +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.beam.sdk.managed.Managed; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.yaml.snakeyaml.Yaml; + +public class IcebergReadSchemaTransformProviderTest { + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule public TestDataWarehouse warehouse = new TestDataWarehouse(TEMPORARY_FOLDER, "default"); + + @Rule public TestPipeline testPipeline = TestPipeline.create(); + + @Test + public void testBuildTransformWithRow() { + Row catalogConfigRow = + Row.withSchema(IcebergSchemaTransformCatalogConfig.SCHEMA) + .withFieldValue("catalog_name", "test_name") + .withFieldValue("catalog_type", "test_type") + .withFieldValue("catalog_implementation", "testImplementation") + .withFieldValue("warehouse_location", "test_location") + .build(); + Row transformConfigRow = + Row.withSchema(new IcebergReadSchemaTransformProvider().configurationSchema()) + .withFieldValue("table", "test_table_identifier") + .withFieldValue("catalog_config", catalogConfigRow) + .build(); + + new IcebergReadSchemaTransformProvider().from(transformConfigRow); + } + + @Test + public void testSimpleScan() throws Exception { + String identifier = "default.table_" + Long.toString(UUID.randomUUID().hashCode(), 16); + TableIdentifier tableId = TableIdentifier.parse(identifier); + + Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); + final Schema schema = SchemaAndRowConversions.icebergSchemaToBeamSchema(TestFixtures.SCHEMA); + + simpleTable + .newFastAppend() + .appendFile( + warehouse.writeRecords( + "file1s1.parquet", simpleTable.schema(), TestFixtures.FILE1SNAPSHOT1)) + .appendFile( + warehouse.writeRecords( + "file2s1.parquet", simpleTable.schema(), TestFixtures.FILE2SNAPSHOT1)) + .appendFile( + warehouse.writeRecords( + "file3s1.parquet", simpleTable.schema(), TestFixtures.FILE3SNAPSHOT1)) + .commit(); + + final List expectedRows = + Stream.of( + TestFixtures.FILE1SNAPSHOT1, + TestFixtures.FILE2SNAPSHOT1, + TestFixtures.FILE3SNAPSHOT1) + .flatMap(List::stream) + .map(record -> SchemaAndRowConversions.recordToRow(schema, record)) + .collect(Collectors.toList()); + + IcebergSchemaTransformCatalogConfig catalogConfig = + IcebergSchemaTransformCatalogConfig.builder() + .setCatalogName("hadoop") + .setCatalogType(CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .setWarehouseLocation(warehouse.location) + .build(); + + IcebergReadSchemaTransformProvider.Config readConfig = + IcebergReadSchemaTransformProvider.Config.builder() + .setTable(identifier) + .setCatalogConfig(catalogConfig) + .build(); + + PCollection output = + PCollectionRowTuple.empty(testPipeline) + .apply(new IcebergReadSchemaTransformProvider().from(readConfig)) + .get(OUTPUT_TAG); + + PAssert.that(output) + .satisfies( + (Iterable rows) -> { + assertThat(rows, containsInAnyOrder(expectedRows.toArray())); + return null; + }); + + testPipeline.run(); + } + + @Test + public void testReadUsingManagedTransform() throws Exception { + String identifier = "default.table_" + Long.toString(UUID.randomUUID().hashCode(), 16); + TableIdentifier tableId = TableIdentifier.parse(identifier); + + Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); + final Schema schema = SchemaAndRowConversions.icebergSchemaToBeamSchema(TestFixtures.SCHEMA); + + simpleTable + .newFastAppend() + .appendFile( + warehouse.writeRecords( + "file1s1.parquet", simpleTable.schema(), TestFixtures.FILE1SNAPSHOT1)) + .appendFile( + warehouse.writeRecords( + "file2s1.parquet", simpleTable.schema(), TestFixtures.FILE2SNAPSHOT1)) + .appendFile( + warehouse.writeRecords( + "file3s1.parquet", simpleTable.schema(), TestFixtures.FILE3SNAPSHOT1)) + .commit(); + + final List expectedRows = + Stream.of( + TestFixtures.FILE1SNAPSHOT1, + TestFixtures.FILE2SNAPSHOT1, + TestFixtures.FILE3SNAPSHOT1) + .flatMap(List::stream) + .map(record -> SchemaAndRowConversions.recordToRow(schema, record)) + .collect(Collectors.toList()); + + String yamlConfig = + String.format( + "table: %s\n" + + "catalog_config: \n" + + " catalog_name: hadoop\n" + + " catalog_type: %s\n" + + " warehouse_location: %s", + identifier, CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP, warehouse.location); + Map configMap = new Yaml().load(yamlConfig); + + PCollection output = + PCollectionRowTuple.empty(testPipeline) + .apply(Managed.read(Managed.ICEBERG).withConfig(configMap)) + .get(OUTPUT_TAG); + + PAssert.that(output) + .satisfies( + (Iterable rows) -> { + assertThat(rows, containsInAnyOrder(expectedRows.toArray())); + return null; + }); + + testPipeline.run(); + } +} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslationTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslationTest.java new file mode 100644 index 000000000000..d643e194dcba --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslationTest.java @@ -0,0 +1,248 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.iceberg; + +import static org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods.Enum.SCHEMA_TRANSFORM; +import static org.apache.beam.sdk.io.iceberg.IcebergReadSchemaTransformProvider.IcebergReadSchemaTransform; +import static org.apache.beam.sdk.io.iceberg.IcebergWriteSchemaTransformProvider.INPUT_TAG; +import static org.apache.beam.sdk.io.iceberg.IcebergWriteSchemaTransformProvider.IcebergWriteSchemaTransform; +import static org.junit.Assert.assertEquals; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload; +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.SchemaTranslation; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.util.construction.BeamUrns; +import org.apache.beam.sdk.util.construction.PipelineTranslation; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.catalog.TableIdentifier; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; + +public class IcebergSchemaTransformTranslationTest { + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public transient TestDataWarehouse warehouse = new TestDataWarehouse(TEMPORARY_FOLDER, "default"); + + @Rule public transient ExpectedException thrown = ExpectedException.none(); + + static final IcebergWriteSchemaTransformProvider WRITE_PROVIDER = + new IcebergWriteSchemaTransformProvider(); + static final IcebergReadSchemaTransformProvider READ_PROVIDER = + new IcebergReadSchemaTransformProvider(); + + @Test + public void testReCreateWriteTransformFromRow() { + Row catalogConfigRow = + Row.withSchema(IcebergSchemaTransformCatalogConfig.SCHEMA) + .withFieldValue("catalog_name", "test_name") + .withFieldValue("catalog_type", "test_type") + .withFieldValue("catalog_implementation", "testImplementation") + .withFieldValue("warehouse_location", "test_location") + .build(); + Row transformConfigRow = + Row.withSchema(WRITE_PROVIDER.configurationSchema()) + .withFieldValue("table", "test_table_identifier") + .withFieldValue("catalog_config", catalogConfigRow) + .build(); + IcebergWriteSchemaTransform writeTransform = + (IcebergWriteSchemaTransform) WRITE_PROVIDER.from(transformConfigRow); + + IcebergSchemaTransformTranslation.IcebergWriteSchemaTransformTranslator translator = + new IcebergSchemaTransformTranslation.IcebergWriteSchemaTransformTranslator(); + Row row = translator.toConfigRow(writeTransform); + + IcebergWriteSchemaTransform writeTransformFromRow = + translator.fromConfigRow(row, PipelineOptionsFactory.create()); + + assertEquals(transformConfigRow, writeTransformFromRow.getConfigurationRow()); + } + + @Test + public void testWriteTransformProtoTranslation() + throws InvalidProtocolBufferException, IOException { + // First build a pipeline + Pipeline p = Pipeline.create(); + Schema inputSchema = Schema.builder().addStringField("str").build(); + PCollection input = + p.apply( + Create.of( + Collections.singletonList(Row.withSchema(inputSchema).addValue("a").build()))) + .setRowSchema(inputSchema); + + Row catalogConfigRow = + Row.withSchema(IcebergSchemaTransformCatalogConfig.SCHEMA) + .withFieldValue("catalog_name", "test_catalog") + .withFieldValue("catalog_type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .withFieldValue("catalog_implementation", "test_implementation") + .withFieldValue("warehouse_location", warehouse.location) + .build(); + Row transformConfigRow = + Row.withSchema(WRITE_PROVIDER.configurationSchema()) + .withFieldValue("table", "test_identifier") + .withFieldValue("catalog_config", catalogConfigRow) + .build(); + + IcebergWriteSchemaTransform writeTransform = + (IcebergWriteSchemaTransform) WRITE_PROVIDER.from(transformConfigRow); + PCollectionRowTuple.of(INPUT_TAG, input).apply(writeTransform); + + // Then translate the pipeline to a proto and extract IcebergWriteSchemaTransform proto + RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p); + List writeTransformProto = + pipelineProto.getComponents().getTransformsMap().values().stream() + .filter( + tr -> { + RunnerApi.FunctionSpec spec = tr.getSpec(); + try { + return spec.getUrn().equals(BeamUrns.getUrn(SCHEMA_TRANSFORM)) + && SchemaTransformPayload.parseFrom(spec.getPayload()) + .getIdentifier() + .equals(WRITE_PROVIDER.identifier()); + } catch (InvalidProtocolBufferException e) { + throw new RuntimeException(e); + } + }) + .collect(Collectors.toList()); + assertEquals(1, writeTransformProto.size()); + RunnerApi.FunctionSpec spec = writeTransformProto.get(0).getSpec(); + + // Check that the proto contains correct values + SchemaTransformPayload payload = SchemaTransformPayload.parseFrom(spec.getPayload()); + Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getConfigurationSchema()); + assertEquals(WRITE_PROVIDER.configurationSchema(), schemaFromSpec); + Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getConfigurationRow().newInput()); + + assertEquals(transformConfigRow, rowFromSpec); + + // Use the information in the proto to recreate the IcebergWriteSchemaTransform + IcebergSchemaTransformTranslation.IcebergWriteSchemaTransformTranslator translator = + new IcebergSchemaTransformTranslation.IcebergWriteSchemaTransformTranslator(); + IcebergWriteSchemaTransform writeTransformFromSpec = + translator.fromConfigRow(rowFromSpec, PipelineOptionsFactory.create()); + + assertEquals(transformConfigRow, writeTransformFromSpec.getConfigurationRow()); + } + + @Test + public void testReCreateReadTransformFromRow() { + // setting a subset of fields here. + Row catalogConfigRow = + Row.withSchema(IcebergSchemaTransformCatalogConfig.SCHEMA) + .withFieldValue("catalog_name", "test_name") + .withFieldValue("catalog_type", "test_type") + .withFieldValue("catalog_implementation", "testImplementation") + .withFieldValue("warehouse_location", "test_location") + .build(); + Row transformConfigRow = + Row.withSchema(READ_PROVIDER.configurationSchema()) + .withFieldValue("table", "test_table_identifier") + .withFieldValue("catalog_config", catalogConfigRow) + .build(); + + IcebergReadSchemaTransform readTransform = + (IcebergReadSchemaTransform) READ_PROVIDER.from(transformConfigRow); + + IcebergSchemaTransformTranslation.IcebergReadSchemaTransformTranslator translator = + new IcebergSchemaTransformTranslation.IcebergReadSchemaTransformTranslator(); + Row row = translator.toConfigRow(readTransform); + + IcebergReadSchemaTransform readTransformFromRow = + translator.fromConfigRow(row, PipelineOptionsFactory.create()); + + assertEquals(transformConfigRow, readTransformFromRow.getConfigurationRow()); + } + + @Test + public void testReadTransformProtoTranslation() + throws InvalidProtocolBufferException, IOException { + // First build a pipeline + Pipeline p = Pipeline.create(); + Row catalogConfigRow = + Row.withSchema(IcebergSchemaTransformCatalogConfig.SCHEMA) + .withFieldValue("catalog_name", "test_catalog") + .withFieldValue("catalog_type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .withFieldValue("warehouse_location", warehouse.location) + .build(); + String identifier = "default.table_" + Long.toString(UUID.randomUUID().hashCode(), 16); + warehouse.createTable(TableIdentifier.parse(identifier), TestFixtures.SCHEMA); + + Row transformConfigRow = + Row.withSchema(READ_PROVIDER.configurationSchema()) + .withFieldValue("table", identifier) + .withFieldValue("catalog_config", catalogConfigRow) + .build(); + + IcebergReadSchemaTransform readTransform = + (IcebergReadSchemaTransform) READ_PROVIDER.from(transformConfigRow); + + PCollectionRowTuple.empty(p).apply(readTransform); + + // Then translate the pipeline to a proto and extract IcebergReadSchemaTransform proto + RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p); + List readTransformProto = + pipelineProto.getComponents().getTransformsMap().values().stream() + .filter( + tr -> { + RunnerApi.FunctionSpec spec = tr.getSpec(); + try { + return spec.getUrn().equals(BeamUrns.getUrn(SCHEMA_TRANSFORM)) + && SchemaTransformPayload.parseFrom(spec.getPayload()) + .getIdentifier() + .equals(READ_PROVIDER.identifier()); + } catch (InvalidProtocolBufferException e) { + throw new RuntimeException(e); + } + }) + .collect(Collectors.toList()); + assertEquals(1, readTransformProto.size()); + RunnerApi.FunctionSpec spec = readTransformProto.get(0).getSpec(); + + // Check that the proto contains correct values + SchemaTransformPayload payload = SchemaTransformPayload.parseFrom(spec.getPayload()); + Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getConfigurationSchema()); + assertEquals(READ_PROVIDER.configurationSchema(), schemaFromSpec); + Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getConfigurationRow().newInput()); + assertEquals(transformConfigRow, rowFromSpec); + + // Use the information in the proto to recreate the IcebergReadSchemaTransform + IcebergSchemaTransformTranslation.IcebergReadSchemaTransformTranslator translator = + new IcebergSchemaTransformTranslation.IcebergReadSchemaTransformTranslator(); + IcebergReadSchemaTransform readTransformFromSpec = + translator.fromConfigRow(rowFromSpec, PipelineOptionsFactory.create()); + + assertEquals(transformConfigRow, readTransformFromSpec.getConfigurationRow()); + } +} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProviderTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProviderTest.java new file mode 100644 index 000000000000..5089a58f0a97 --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProviderTest.java @@ -0,0 +1,175 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.iceberg; + +import static org.apache.beam.sdk.io.iceberg.IcebergWriteSchemaTransformProvider.Config; +import static org.apache.beam.sdk.io.iceberg.IcebergWriteSchemaTransformProvider.INPUT_TAG; +import static org.apache.beam.sdk.io.iceberg.IcebergWriteSchemaTransformProvider.OUTPUT_TAG; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertEquals; + +import java.util.List; +import java.util.Map; +import java.util.UUID; +import org.apache.beam.sdk.managed.Managed; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.Record; +import org.hamcrest.Matchers; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.yaml.snakeyaml.Yaml; + +@RunWith(JUnit4.class) +public class IcebergWriteSchemaTransformProviderTest { + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public transient TestDataWarehouse warehouse = new TestDataWarehouse(TEMPORARY_FOLDER, "default"); + + @Rule public transient TestPipeline testPipeline = TestPipeline.create(); + + @Test + public void testBuildTransformWithRow() { + Row catalogConfigRow = + Row.withSchema(IcebergSchemaTransformCatalogConfig.SCHEMA) + .withFieldValue("catalog_name", "test_name") + .withFieldValue("catalog_type", "test_type") + .withFieldValue("catalog_implementation", "testImplementation") + .withFieldValue("warehouse_location", "test_location") + .build(); + Row transformConfigRow = + Row.withSchema(new IcebergWriteSchemaTransformProvider().configurationSchema()) + .withFieldValue("table", "test_table_identifier") + .withFieldValue("catalog_config", catalogConfigRow) + .build(); + + new IcebergWriteSchemaTransformProvider().from(transformConfigRow); + } + + @Test + public void testSimpleAppend() { + String identifier = "default.table_" + Long.toString(UUID.randomUUID().hashCode(), 16); + + TableIdentifier tableId = TableIdentifier.parse(identifier); + + // Create a table and add records to it. + Table table = warehouse.createTable(tableId, TestFixtures.SCHEMA); + + Config config = + Config.builder() + .setTable(identifier) + .setCatalogConfig( + IcebergSchemaTransformCatalogConfig.builder() + .setCatalogName("hadoop") + .setCatalogType(CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .setWarehouseLocation(warehouse.location) + .build()) + .build(); + + PCollectionRowTuple input = + PCollectionRowTuple.of( + INPUT_TAG, + testPipeline + .apply( + "Records To Add", Create.of(TestFixtures.asRows(TestFixtures.FILE1SNAPSHOT1))) + .setRowSchema( + SchemaAndRowConversions.icebergSchemaToBeamSchema(TestFixtures.SCHEMA))); + + PCollection result = + input + .apply("Append To Table", new IcebergWriteSchemaTransformProvider().from(config)) + .get(OUTPUT_TAG); + + PAssert.that(result).satisfies(new VerifyOutputs(identifier, "append")); + + testPipeline.run().waitUntilFinish(); + + List writtenRecords = ImmutableList.copyOf(IcebergGenerics.read(table).build()); + + assertThat(writtenRecords, Matchers.containsInAnyOrder(TestFixtures.FILE1SNAPSHOT1.toArray())); + } + + @Test + public void testWriteUsingManagedTransform() { + String identifier = "default.table_" + Long.toString(UUID.randomUUID().hashCode(), 16); + Table table = warehouse.createTable(TableIdentifier.parse(identifier), TestFixtures.SCHEMA); + + String yamlConfig = + String.format( + "table: %s\n" + + "catalog_config: \n" + + " catalog_name: hadoop\n" + + " catalog_type: %s\n" + + " warehouse_location: %s", + identifier, CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP, warehouse.location); + Map configMap = new Yaml().load(yamlConfig); + + PCollectionRowTuple input = + PCollectionRowTuple.of( + INPUT_TAG, + testPipeline + .apply( + "Records To Add", Create.of(TestFixtures.asRows(TestFixtures.FILE1SNAPSHOT1))) + .setRowSchema( + SchemaAndRowConversions.icebergSchemaToBeamSchema(TestFixtures.SCHEMA))); + PCollection result = + input.apply(Managed.write(Managed.ICEBERG).withConfig(configMap)).get(OUTPUT_TAG); + + PAssert.that(result).satisfies(new VerifyOutputs(identifier, "append")); + + testPipeline.run().waitUntilFinish(); + + List writtenRecords = ImmutableList.copyOf(IcebergGenerics.read(table).build()); + assertThat(writtenRecords, Matchers.containsInAnyOrder(TestFixtures.FILE1SNAPSHOT1.toArray())); + } + + private static class VerifyOutputs implements SerializableFunction, Void> { + private final String tableId; + private final String operation; + + public VerifyOutputs(String identifier, String operation) { + this.tableId = identifier; + this.operation = operation; + } + + @Override + public Void apply(Iterable input) { + Row row = input.iterator().next(); + + assertEquals(tableId, row.getString("table")); + assertEquals(operation, row.getString("operation")); + return null; + } + } +} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/ScanSourceTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ScanSourceTest.java similarity index 99% rename from sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/ScanSourceTest.java rename to sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ScanSourceTest.java index 21ee3fd50a6c..c7d5353428c2 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/ScanSourceTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ScanSourceTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg; +package org.apache.beam.sdk.io.iceberg; import static org.hamcrest.MatcherAssert.assertThat; diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/SchemaAndRowConversionsTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/SchemaAndRowConversionsTest.java similarity index 99% rename from sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/SchemaAndRowConversionsTest.java rename to sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/SchemaAndRowConversionsTest.java index 225e7136811e..3ff7239a805c 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/SchemaAndRowConversionsTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/SchemaAndRowConversionsTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg; +package org.apache.beam.sdk.io.iceberg; import static org.apache.iceberg.types.Types.NestedField.required; import static org.hamcrest.MatcherAssert.assertThat; diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/TestDataWarehouse.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/TestDataWarehouse.java similarity index 99% rename from sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/TestDataWarehouse.java rename to sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/TestDataWarehouse.java index 0fc704cfc087..a8f63383801b 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/TestDataWarehouse.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/TestDataWarehouse.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg; +package org.apache.beam.sdk.io.iceberg; import static org.apache.iceberg.hadoop.HadoopOutputFile.fromPath; diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/TestFixtures.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/TestFixtures.java similarity index 99% rename from sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/TestFixtures.java rename to sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/TestFixtures.java index e6bb42d6a242..4048e88398a9 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/TestFixtures.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/TestFixtures.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg; +package org.apache.beam.sdk.io.iceberg; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProviderTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProviderTest.java index f6e231c758a5..bf9895e36b84 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProviderTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProviderTest.java @@ -112,17 +112,17 @@ public void testFindTransformAndMakeItWork() { assertEquals( Sets.newHashSet( - "bootstrapServers", + "bootstrap_servers", "topic", "schema", - "autoOffsetResetConfig", - "consumerConfigUpdates", + "auto_offset_reset_config", + "consumer_config_updates", "format", - "confluentSchemaRegistrySubject", - "confluentSchemaRegistryUrl", - "errorHandling", - "fileDescriptorPath", - "messageName"), + "confluent_schema_registry_subject", + "confluent_schema_registry_url", + "error_handling", + "file_descriptor_path", + "message_name"), kafkaProvider.configurationSchema().getFields().stream() .map(field -> field.getName()) .collect(Collectors.toSet())); diff --git a/sdks/java/managed/build.gradle b/sdks/java/managed/build.gradle index 88e537d66f8c..f06df27429b1 100644 --- a/sdks/java/managed/build.gradle +++ b/sdks/java/managed/build.gradle @@ -29,7 +29,6 @@ ext.summary = """Library that provides managed IOs.""" dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") implementation library.java.vendored_guava_32_1_2_jre -// implementation library.java.vendored_grpc_1_60_1 testImplementation library.java.junit testRuntimeOnly "org.yaml:snakeyaml:2.0" diff --git a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/Managed.java b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/Managed.java index b2b010b1e434..d24a3fd88ddc 100644 --- a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/Managed.java +++ b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/Managed.java @@ -17,6 +17,9 @@ */ package org.apache.beam.sdk.managed; +import static org.apache.beam.sdk.managed.ManagedTransformConstants.ICEBERG_READ; +import static org.apache.beam.sdk.managed.ManagedTransformConstants.ICEBERG_WRITE; + import com.google.auto.value.AutoValue; import java.util.ArrayList; import java.util.List; @@ -25,6 +28,7 @@ import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; import org.apache.beam.sdk.schemas.utils.YamlUtils; +import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; @@ -36,8 +40,8 @@ * *

Available transforms

* - *

This API currently supports two operations: {@link Read} and {@link Write}. Each one - * enumerates the available transforms in a {@code TRANSFORMS} map. + *

This API currently supports two operations: {@link Managed#read} and {@link Managed#write}. + * Each one enumerates the available transforms in a {@code TRANSFORMS} map. * *

Building a Managed turnkey transform

* @@ -48,7 +52,7 @@ *
{@code
  * PCollectionRowTuple output = PCollectionRowTuple.empty(pipeline).apply(
  *       Managed.read(ICEBERG)
- *           .withConfig(ImmutableMap..builder()
+ *           .withConfig(ImmutableMap..builder()
  *               .put("foo", "abc")
  *               .put("bar", 123)
  *               .build()));
@@ -77,25 +81,21 @@ public class Managed {
   // TODO: Dynamically generate a list of supported transforms
   public static final String ICEBERG = "iceberg";
 
+  // Supported SchemaTransforms
   public static final Map READ_TRANSFORMS =
-      ImmutableMap.builder()
-          .put(ICEBERG, "beam:schematransform:org.apache.beam:iceberg_read:v1")
-          .build();
+      ImmutableMap.builder().put(ICEBERG, ICEBERG_READ).build();
   public static final Map WRITE_TRANSFORMS =
-      ImmutableMap.builder()
-          .put(ICEBERG, "beam:schematransform:org.apache.beam:iceberg_write:v1")
-          .build();
+      ImmutableMap.builder().put(ICEBERG, ICEBERG_WRITE).build();
 
   /**
-   * Instantiates a {@link Managed.Read} transform for the specified source. The supported managed
-   * sources are:
+   * Instantiates a {@link Managed.ManagedTransform} transform for the specified source. The
+   * supported managed sources are:
    *
    * 
    *
  • {@link Managed#ICEBERG} : Read from Apache Iceberg *
*/ public static ManagedTransform read(String source) { - return new AutoValue_Managed_ManagedTransform.Builder() .setIdentifier( Preconditions.checkNotNull( @@ -108,8 +108,8 @@ public static ManagedTransform read(String source) { } /** - * Instantiates a {@link Managed.Write} transform for the specified sink. The supported managed - * sinks are: + * Instantiates a {@link Managed.ManagedTransform} transform for the specified sink. The supported + * managed sinks are: * *
    *
  • {@link Managed#ICEBERG} : Write to Apache Iceberg @@ -128,10 +128,11 @@ public static ManagedTransform write(String sink) { } @AutoValue - public abstract static class ManagedTransform extends SchemaTransform { + public abstract static class ManagedTransform + extends PTransform { abstract String getIdentifier(); - abstract @Nullable String getConfig(); + abstract @Nullable Map getConfig(); abstract @Nullable String getConfigUrl(); @@ -144,7 +145,7 @@ public abstract static class ManagedTransform extends SchemaTransform { abstract static class Builder { abstract Builder setIdentifier(String identifier); - abstract Builder setConfig(@Nullable String config); + abstract Builder setConfig(@Nullable Map config); abstract Builder setConfigUrl(@Nullable String configUrl); @@ -161,7 +162,7 @@ abstract static class Builder { * SchemaTransformProvider#configurationSchema()}) to see which parameters are available. */ public ManagedTransform withConfig(Map config) { - return toBuilder().setConfig(YamlUtils.yamlStringFromMap(config)).build(); + return toBuilder().setConfig(config).build(); } /** @@ -182,7 +183,7 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { ManagedSchemaTransformProvider.ManagedConfig managedConfig = ManagedSchemaTransformProvider.ManagedConfig.builder() .setTransformIdentifier(getIdentifier()) - .setConfig(getConfig()) + .setConfig(YamlUtils.yamlStringFromMap(getConfig())) .setConfigUrl(getConfigUrl()) .build(); diff --git a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedSchemaTransformProvider.java b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedSchemaTransformProvider.java index 1ee2b11a90ff..cb5088a24cca 100644 --- a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedSchemaTransformProvider.java +++ b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedSchemaTransformProvider.java @@ -24,15 +24,19 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; +import java.util.Arrays; import java.util.Collection; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.ServiceLoader; import javax.annotation.Nullable; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.fs.MatchResult; import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.NoSuchSchemaException; import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.SchemaRegistry; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; import org.apache.beam.sdk.schemas.transforms.SchemaTransform; @@ -43,6 +47,7 @@ import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Predicates; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; @AutoService(SchemaTransformProvider.class) @@ -51,14 +56,14 @@ public class ManagedSchemaTransformProvider @Override public String identifier() { - return "beam:schematransform:org.apache.beam:managed:v1"; + return "beam:transform:managed:v1"; } private final Map schemaTransformProviders = new HashMap<>(); public ManagedSchemaTransformProvider() {} - ManagedSchemaTransformProvider(Collection supportedIdentifiers) { + ManagedSchemaTransformProvider(@Nullable Collection supportedIdentifiers) { try { for (SchemaTransformProvider schemaTransformProvider : ServiceLoader.load(SchemaTransformProvider.class)) { @@ -67,13 +72,15 @@ public ManagedSchemaTransformProvider() {} "Found multiple SchemaTransformProvider implementations with the same identifier " + schemaTransformProvider.identifier()); } - schemaTransformProviders.put(schemaTransformProvider.identifier(), schemaTransformProvider); + if (supportedIdentifiers == null + || supportedIdentifiers.contains(schemaTransformProvider.identifier())) { + schemaTransformProviders.put( + schemaTransformProvider.identifier(), schemaTransformProvider); + } } } catch (Exception e) { throw new RuntimeException(e.getMessage()); } - - schemaTransformProviders.entrySet().removeIf(e -> !supportedIdentifiers.contains(e.getKey())); } @DefaultSchema(AutoValueSchema.class) @@ -84,13 +91,15 @@ public static Builder builder() { return new AutoValue_ManagedSchemaTransformProvider_ManagedConfig.Builder(); } - @SchemaFieldDescription("Identifier of the underlying IO to instantiate.") + @SchemaFieldDescription( + "Identifier of the underlying SchemaTransform to discover and instantiate.") public abstract String getTransformIdentifier(); - @SchemaFieldDescription("URL path to the YAML config file used to build the underlying IO.") + @SchemaFieldDescription( + "URL path to the YAML config file used to build the underlying SchemaTransform.") public abstract @Nullable String getConfigUrl(); - @SchemaFieldDescription("YAML string config used to build the underlying IO.") + @SchemaFieldDescription("YAML string config used to build the underlying SchemaTransform.") public abstract @Nullable String getConfig(); @AutoValue.Builder @@ -99,7 +108,7 @@ public abstract static class Builder { public abstract Builder setConfigUrl(@Nullable String configUrl); - public abstract Builder setConfig(@Nullable String config); + public abstract Builder setConfig(@Nullable String yamlConfig); public abstract ManagedConfig build(); } @@ -107,10 +116,28 @@ public abstract static class Builder { protected void validate() { boolean configExists = !Strings.isNullOrEmpty(getConfig()); boolean configUrlExists = !Strings.isNullOrEmpty(getConfigUrl()); + List configs = Arrays.asList(configExists, configUrlExists); checkArgument( - !(configExists && configUrlExists) && (configExists || configUrlExists), + 1 == configs.stream().filter(Predicates.equalTo(true)).count(), "Please specify a config or a config URL, but not both."); } + + public @Nullable String resolveUnderlyingConfig() { + String yamlTransformConfig = getConfig(); + // If YAML string is empty, then attempt to read from YAML file + if (Strings.isNullOrEmpty(yamlTransformConfig)) { + try { + MatchResult.Metadata fileMetaData = + FileSystems.matchSingleFileSpec(Preconditions.checkNotNull(getConfigUrl())); + ByteBuffer buffer = ByteBuffer.allocate((int) fileMetaData.sizeBytes()); + FileSystems.open(fileMetaData.resourceId()).read(buffer); + yamlTransformConfig = new String(buffer.array(), StandardCharsets.UTF_8); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + return yamlTransformConfig; + } } @Override @@ -122,61 +149,63 @@ protected SchemaTransform from(ManagedConfig managedConfig) { "Could not find transform with identifier %s, or it may not be supported", managedConfig.getTransformIdentifier()); - // parse config before expansion to check if it matches underlying transform's config schema - Schema transformConfigSchema = schemaTransformProvider.configurationSchema(); - Row transformConfig; - try { - transformConfig = getRowConfig(managedConfig, transformConfigSchema); - } catch (Exception e) { - throw new IllegalArgumentException( - String.format( - "Specified configuration does not align with the underlying transform's configuration schema [%s].", - transformConfigSchema), - e); - } - - return new ManagedSchemaTransform(transformConfig, schemaTransformProvider); + return new ManagedSchemaTransform(managedConfig, schemaTransformProvider); } - private static class ManagedSchemaTransform extends SchemaTransform { - private final Row transformConfig; + static class ManagedSchemaTransform extends SchemaTransform { + private final ManagedConfig managedConfig; + private final Row underlyingTransformConfig; private final SchemaTransformProvider underlyingTransformProvider; ManagedSchemaTransform( - Row transformConfig, SchemaTransformProvider underlyingTransformProvider) { - this.transformConfig = transformConfig; + ManagedConfig managedConfig, SchemaTransformProvider underlyingTransformProvider) { + // parse config before expansion to check if it matches underlying transform's config schema + Schema transformConfigSchema = underlyingTransformProvider.configurationSchema(); + Row underlyingTransformConfig; + try { + underlyingTransformConfig = getRowConfig(managedConfig, transformConfigSchema); + } catch (Exception e) { + throw new IllegalArgumentException( + "Encountered an error when retrieving a Row configuration", e); + } + + this.managedConfig = managedConfig; + this.underlyingTransformConfig = underlyingTransformConfig; this.underlyingTransformProvider = underlyingTransformProvider; } @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { - SchemaTransform underlyingTransform = underlyingTransformProvider.from(transformConfig); + return input.apply(underlyingTransformProvider.from(underlyingTransformConfig)); + } - return input.apply(underlyingTransform); + public ManagedConfig getManagedConfig() { + return this.managedConfig; } - } - @VisibleForTesting - static Row getRowConfig(ManagedConfig config, Schema transformSchema) { - String transformYamlConfig; - if (!Strings.isNullOrEmpty(config.getConfigUrl())) { + Row getConfigurationRow() { try { - MatchResult.Metadata fileMetaData = - FileSystems.matchSingleFileSpec(Preconditions.checkNotNull(config.getConfigUrl())); - ByteBuffer buffer = ByteBuffer.allocate((int) fileMetaData.sizeBytes()); - FileSystems.open(fileMetaData.resourceId()).read(buffer); - transformYamlConfig = new String(buffer.array(), StandardCharsets.UTF_8); - } catch (IOException e) { + // To stay consistent with our SchemaTransform configuration naming conventions, + // we sort lexicographically and convert field names to snake_case + return SchemaRegistry.createDefault() + .getToRowFunction(ManagedConfig.class) + .apply(managedConfig) + .sorted() + .toSnakeCase(); + } catch (NoSuchSchemaException e) { throw new RuntimeException(e); } - } else { - transformYamlConfig = config.getConfig(); } - - return YamlUtils.toBeamRow(transformYamlConfig, transformSchema, true); } + /** */ @VisibleForTesting + static Row getRowConfig(ManagedConfig config, Schema transformSchema) { + // May return an empty row (perhaps the underlying transform doesn't have any required + // parameters) + return YamlUtils.toBeamRow(config.resolveUnderlyingConfig(), transformSchema, false); + } + Map getAllProviders() { return schemaTransformProviders; } diff --git a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedSchemaTransformTranslation.java b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedSchemaTransformTranslation.java new file mode 100644 index 000000000000..2b1e6544ef8b --- /dev/null +++ b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedSchemaTransformTranslation.java @@ -0,0 +1,59 @@ +/* + * 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.managed; + +import static org.apache.beam.sdk.managed.ManagedSchemaTransformProvider.ManagedSchemaTransform; +import static org.apache.beam.sdk.schemas.transforms.SchemaTransformTranslation.SchemaTransformPayloadTranslator; +import static org.apache.beam.sdk.util.construction.PTransformTranslation.TransformPayloadTranslator; + +import com.google.auto.service.AutoService; +import java.util.Map; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.TransformPayloadTranslatorRegistrar; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; + +public class ManagedSchemaTransformTranslation { + static class ManagedSchemaTransformTranslator + extends SchemaTransformPayloadTranslator { + @Override + public SchemaTransformProvider provider() { + return new ManagedSchemaTransformProvider(null); + } + + @Override + public Row toConfigRow(ManagedSchemaTransform transform) { + return transform.getConfigurationRow(); + } + } + + @AutoService(TransformPayloadTranslatorRegistrar.class) + public static class ManagedTransformRegistrar implements TransformPayloadTranslatorRegistrar { + @Override + @SuppressWarnings({ + "rawtypes", + }) + public Map, ? extends TransformPayloadTranslator> + getTransformPayloadTranslators() { + return ImmutableMap., TransformPayloadTranslator>builder() + .put(ManagedSchemaTransform.class, new ManagedSchemaTransformTranslator()) + .build(); + } + } +} diff --git a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java new file mode 100644 index 000000000000..48735d8c33a3 --- /dev/null +++ b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java @@ -0,0 +1,25 @@ +/* + * 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.managed; + +/** This class contains constants for supported managed transform identifiers. */ +public class ManagedTransformConstants { + public static final String ICEBERG_READ = "beam:schematransform:org.apache.beam:iceberg_read:v1"; + public static final String ICEBERG_WRITE = + "beam:schematransform:org.apache.beam:iceberg_write:v1"; +} diff --git a/sdks/java/managed/src/test/java/org/apache/beam/sdk/managed/TestSchemaTransformProvider.java b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/testing/TestSchemaTransformProvider.java similarity index 91% rename from sdks/java/managed/src/test/java/org/apache/beam/sdk/managed/TestSchemaTransformProvider.java rename to sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/testing/TestSchemaTransformProvider.java index 136d98d468d0..2771ecd01643 100644 --- a/sdks/java/managed/src/test/java/org/apache/beam/sdk/managed/TestSchemaTransformProvider.java +++ b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/testing/TestSchemaTransformProvider.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.managed; +package org.apache.beam.sdk.managed.testing; import com.google.auto.service.AutoService; import com.google.auto.value.AutoValue; @@ -35,7 +35,9 @@ @AutoService(SchemaTransformProvider.class) public class TestSchemaTransformProvider extends TypedSchemaTransformProvider { - static final String IDENTIFIER = "beam:schematransform:org.apache.beam:test_transform:v1"; + private static final TestSchemaTransformProvider INSTANCE = new TestSchemaTransformProvider(); + public static final String IDENTIFIER = INSTANCE.identifier(); + public static final Schema SCHEMA = INSTANCE.configurationSchema(); @DefaultSchema(AutoValueSchema.class) @AutoValue @@ -93,6 +95,6 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { @Override public String identifier() { - return IDENTIFIER; + return "beam:test_schematransform:v1"; } } diff --git a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/testing/package-info.java b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/testing/package-info.java new file mode 100644 index 000000000000..e90e3ce8a94e --- /dev/null +++ b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/testing/package-info.java @@ -0,0 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** Test transform for Managed API. */ +package org.apache.beam.sdk.managed.testing; diff --git a/sdks/java/managed/src/test/java/org/apache/beam/sdk/managed/ManagedSchemaTransformProviderTest.java b/sdks/java/managed/src/test/java/org/apache/beam/sdk/managed/ManagedSchemaTransformProviderTest.java index 0c495d0d2c5c..e9edf8751e34 100644 --- a/sdks/java/managed/src/test/java/org/apache/beam/sdk/managed/ManagedSchemaTransformProviderTest.java +++ b/sdks/java/managed/src/test/java/org/apache/beam/sdk/managed/ManagedSchemaTransformProviderTest.java @@ -24,6 +24,7 @@ import java.net.URISyntaxException; import java.nio.file.Paths; import java.util.Arrays; +import org.apache.beam.sdk.managed.testing.TestSchemaTransformProvider; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.values.Row; import org.junit.Rule; @@ -49,28 +50,28 @@ public void testFailWhenNoConfigSpecified() { } @Test - public void testGetRowFromYamlConfig() { + public void testGetConfigRowFromYamlString() { String yamlString = "extra_string: abc\n" + "extra_integer: 123"; ManagedConfig config = ManagedConfig.builder() .setTransformIdentifier(TestSchemaTransformProvider.IDENTIFIER) .setConfig(yamlString) .build(); - Schema configSchema = new TestSchemaTransformProvider().configurationSchema(); + Row expectedRow = - Row.withSchema(configSchema) - .withFieldValue("extraString", "abc") - .withFieldValue("extraInteger", 123) + Row.withSchema(TestSchemaTransformProvider.SCHEMA) + .withFieldValue("extra_string", "abc") + .withFieldValue("extra_integer", 123) .build(); - Row configRow = - ManagedSchemaTransformProvider.getRowConfig( - config, new TestSchemaTransformProvider().configurationSchema()); - assertEquals(expectedRow, configRow); + Row returnedRow = + ManagedSchemaTransformProvider.getRowConfig(config, TestSchemaTransformProvider.SCHEMA); + + assertEquals(expectedRow, returnedRow); } @Test - public void testGetRowFromConfigUrl() throws URISyntaxException { + public void testGetConfigRowFromYamlFile() throws URISyntaxException { String yamlConfigPath = Paths.get(getClass().getClassLoader().getResource("test_config.yaml").toURI()) .toFile() @@ -83,8 +84,8 @@ public void testGetRowFromConfigUrl() throws URISyntaxException { Schema configSchema = new TestSchemaTransformProvider().configurationSchema(); Row expectedRow = Row.withSchema(configSchema) - .withFieldValue("extraString", "abc") - .withFieldValue("extraInteger", 123) + .withFieldValue("extra_string", "abc") + .withFieldValue("extra_integer", 123) .build(); Row configRow = ManagedSchemaTransformProvider.getRowConfig( @@ -93,6 +94,35 @@ public void testGetRowFromConfigUrl() throws URISyntaxException { assertEquals(expectedRow, configRow); } + @Test + public void testBuildWithYamlString() { + String yamlString = "extra_string: abc\n" + "extra_integer: 123"; + + ManagedConfig config = + ManagedConfig.builder() + .setTransformIdentifier(TestSchemaTransformProvider.IDENTIFIER) + .setConfig(yamlString) + .build(); + + new ManagedSchemaTransformProvider(null).from(config); + } + + @Test + public void testBuildWithYamlFile() throws URISyntaxException { + String yamlConfigPath = + Paths.get(getClass().getClassLoader().getResource("test_config.yaml").toURI()) + .toFile() + .getAbsolutePath(); + + ManagedConfig config = + ManagedConfig.builder() + .setTransformIdentifier(TestSchemaTransformProvider.IDENTIFIER) + .setConfigUrl(yamlConfigPath) + .build(); + + new ManagedSchemaTransformProvider(null).from(config); + } + @Test public void testDiscoverTestProvider() { ManagedSchemaTransformProvider provider = diff --git a/sdks/java/managed/src/test/java/org/apache/beam/sdk/managed/ManagedSchemaTransformTranslationTest.java b/sdks/java/managed/src/test/java/org/apache/beam/sdk/managed/ManagedSchemaTransformTranslationTest.java new file mode 100644 index 000000000000..b4b41ded841c --- /dev/null +++ b/sdks/java/managed/src/test/java/org/apache/beam/sdk/managed/ManagedSchemaTransformTranslationTest.java @@ -0,0 +1,185 @@ +/* + * 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.managed; + +import static org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods.Enum.SCHEMA_TRANSFORM; +import static org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload; +import static org.apache.beam.sdk.managed.ManagedSchemaTransformProvider.ManagedConfig; +import static org.apache.beam.sdk.managed.ManagedSchemaTransformProvider.ManagedSchemaTransform; +import static org.apache.beam.sdk.managed.ManagedSchemaTransformTranslation.ManagedSchemaTransformTranslator; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +import java.net.URISyntaxException; +import java.nio.file.Paths; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.managed.testing.TestSchemaTransformProvider; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.SchemaTranslation; +import org.apache.beam.sdk.schemas.utils.YamlUtils; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.util.construction.BeamUrns; +import org.apache.beam.sdk.util.construction.PipelineTranslation; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.junit.Test; + +public class ManagedSchemaTransformTranslationTest { + static final ManagedSchemaTransformProvider PROVIDER = new ManagedSchemaTransformProvider(null); + + @Test + public void testReCreateTransformFromRowWithConfigUrl() throws URISyntaxException { + String yamlConfigPath = + Paths.get(getClass().getClassLoader().getResource("test_config.yaml").toURI()) + .toFile() + .getAbsolutePath(); + + ManagedConfig originalConfig = + ManagedConfig.builder() + .setTransformIdentifier(TestSchemaTransformProvider.IDENTIFIER) + .setConfigUrl(yamlConfigPath) + .build(); + + ManagedSchemaTransform originalTransform = + (ManagedSchemaTransform) PROVIDER.from(originalConfig); + + ManagedSchemaTransformTranslator translator = new ManagedSchemaTransformTranslator(); + Row configRow = translator.toConfigRow(originalTransform); + + ManagedSchemaTransform transformFromRow = + translator.fromConfigRow(configRow, PipelineOptionsFactory.create()); + ManagedConfig configFromRow = transformFromRow.getManagedConfig(); + + assertNotNull(transformFromRow.getManagedConfig()); + assertEquals(originalConfig.getTransformIdentifier(), configFromRow.getTransformIdentifier()); + assertEquals(originalConfig.getConfigUrl(), configFromRow.getConfigUrl()); + assertNull(configFromRow.getConfig()); + } + + @Test + public void testReCreateTransformFromRowWithConfig() { + String yamlString = "extra_string: abc\n" + "extra_integer: 123"; + + ManagedConfig originalConfig = + ManagedConfig.builder() + .setTransformIdentifier(TestSchemaTransformProvider.IDENTIFIER) + .setConfig(yamlString) + .build(); + + ManagedSchemaTransform originalTransform = + (ManagedSchemaTransform) PROVIDER.from(originalConfig); + + ManagedSchemaTransformTranslator translator = new ManagedSchemaTransformTranslator(); + Row configRow = translator.toConfigRow(originalTransform); + + ManagedSchemaTransform transformFromRow = + translator.fromConfigRow(configRow, PipelineOptionsFactory.create()); + ManagedConfig configFromRow = transformFromRow.getManagedConfig(); + + assertNotNull(transformFromRow.getManagedConfig()); + assertEquals(originalConfig.getTransformIdentifier(), configFromRow.getTransformIdentifier()); + assertEquals(configFromRow.getConfig(), yamlString); + assertNull(originalConfig.getConfigUrl()); + } + + @Test + public void testProtoTranslation() throws Exception { + // First build a pipeline + Pipeline p = Pipeline.create(); + Schema inputSchema = Schema.builder().addStringField("str").build(); + PCollection input = + p.apply( + Create.of( + Arrays.asList( + Row.withSchema(inputSchema).addValue("a").build(), + Row.withSchema(inputSchema).addValue("b").build(), + Row.withSchema(inputSchema).addValue("c").build()))) + .setRowSchema(inputSchema); + Map underlyingConfig = + ImmutableMap.builder() + .put("extra_string", "abc") + .put("extra_integer", 123) + .build(); + String yamlStringConfig = YamlUtils.yamlStringFromMap(underlyingConfig); + Managed.ManagedTransform transform = + Managed.read(Managed.ICEBERG) // give a supported source to get around the check + .withSupportedIdentifiers(Arrays.asList(TestSchemaTransformProvider.IDENTIFIER)) + .toBuilder() + .setIdentifier(TestSchemaTransformProvider.IDENTIFIER) + .build() + .withConfig(underlyingConfig); + PCollectionRowTuple.of("input", input).apply(transform).get("output"); + + // Then translate the pipeline to a proto and extract the ManagedSchemaTransform's proto + RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p); + List managedTransformProto = + pipelineProto.getComponents().getTransformsMap().values().stream() + .filter( + tr -> { + RunnerApi.FunctionSpec spec = tr.getSpec(); + try { + return spec.getUrn().equals(BeamUrns.getUrn(SCHEMA_TRANSFORM)) + && SchemaTransformPayload.parseFrom(spec.getPayload()) + .getIdentifier() + .equals(PROVIDER.identifier()); + } catch (InvalidProtocolBufferException e) { + throw new RuntimeException(e); + } + }) + .collect(Collectors.toList()); + assertEquals(1, managedTransformProto.size()); + RunnerApi.FunctionSpec spec = managedTransformProto.get(0).getSpec(); + + // Check that the proto contains correct values + SchemaTransformPayload payload = SchemaTransformPayload.parseFrom(spec.getPayload()); + assertEquals(PROVIDER.identifier(), payload.getIdentifier()); + Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getConfigurationSchema()); + assertEquals(PROVIDER.configurationSchema(), schemaFromSpec); + Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getConfigurationRow().newInput()); + // Translation logic outputs a Row with snake_case naming convention + Row expectedRow = + Row.withSchema(PROVIDER.configurationSchema()) + .withFieldValue("transform_identifier", TestSchemaTransformProvider.IDENTIFIER) + .withFieldValue("config_url", null) + .withFieldValue("config", yamlStringConfig) + .build(); + assertEquals(expectedRow, rowFromSpec); + + // Use the information in the proto to recreate the ManagedSchemaTransform + ManagedSchemaTransformTranslator translator = new ManagedSchemaTransformTranslator(); + ManagedSchemaTransform transformFromSpec = + translator.fromConfigRow(rowFromSpec, PipelineOptionsFactory.create()); + + assertEquals( + TestSchemaTransformProvider.IDENTIFIER, + transformFromSpec.getManagedConfig().getTransformIdentifier()); + assertEquals(yamlStringConfig, transformFromSpec.getManagedConfig().getConfig()); + assertNull(transformFromSpec.getManagedConfig().getConfigUrl()); + } +} diff --git a/sdks/java/managed/src/test/java/org/apache/beam/sdk/managed/ManagedTest.java b/sdks/java/managed/src/test/java/org/apache/beam/sdk/managed/ManagedTest.java index ceb71a06f33c..7ed364d0e174 100644 --- a/sdks/java/managed/src/test/java/org/apache/beam/sdk/managed/ManagedTest.java +++ b/sdks/java/managed/src/test/java/org/apache/beam/sdk/managed/ManagedTest.java @@ -21,6 +21,7 @@ import java.util.Arrays; import java.util.List; import java.util.stream.Collectors; +import org.apache.beam.sdk.managed.testing.TestSchemaTransformProvider; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; diff --git a/sdks/python/apache_beam/io/gcp/bigquery.py b/sdks/python/apache_beam/io/gcp/bigquery.py index 43bd17022180..d89ce712d8f6 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery.py +++ b/sdks/python/apache_beam/io/gcp/bigquery.py @@ -2574,13 +2574,13 @@ def expand(self, input): expansion_service=self._expansion_service, rearrange_based_on_discovery=True, table=table, - createDisposition=self._create_disposition, - writeDisposition=self._write_disposition, - triggeringFrequencySeconds=self._triggering_frequency, - autoSharding=self._with_auto_sharding, - numStreams=self._num_storage_api_streams, - useAtLeastOnceSemantics=self._use_at_least_once, - errorHandling={ + create_disposition=self._create_disposition, + write_disposition=self._write_disposition, + triggering_frequency_seconds=self._triggering_frequency, + auto_sharding=self._with_auto_sharding, + num_streams=self._num_storage_api_streams, + use_at_least_once_semantics=self._use_at_least_once, + error_handling={ 'output': StorageWriteToBigQuery.FAILED_ROWS_WITH_ERRORS })) diff --git a/sdks/python/apache_beam/io/gcp/bigtableio.py b/sdks/python/apache_beam/io/gcp/bigtableio.py index f8534f38ddfc..0f3944a791bd 100644 --- a/sdks/python/apache_beam/io/gcp/bigtableio.py +++ b/sdks/python/apache_beam/io/gcp/bigtableio.py @@ -225,9 +225,9 @@ def expand(self, input): identifier=self.schematransform_config.identifier, expansion_service=self._expansion_service, rearrange_based_on_discovery=True, - tableId=self._table_id, - instanceId=self._instance_id, - projectId=self._project_id) + table_id=self._table_id, + instance_id=self._instance_id, + project_id=self._project_id) return ( input @@ -323,9 +323,9 @@ def expand(self, input): identifier=self.schematransform_config.identifier, expansion_service=self._expansion_service, rearrange_based_on_discovery=True, - tableId=self._table_id, - instanceId=self._instance_id, - projectId=self._project_id) + table_id=self._table_id, + instance_id=self._instance_id, + project_id=self._project_id) return ( input.pipeline diff --git a/sdks/python/apache_beam/transforms/external_transform_provider.py b/sdks/python/apache_beam/transforms/external_transform_provider.py index 2799bd1b9e93..67adda5aec03 100644 --- a/sdks/python/apache_beam/transforms/external_transform_provider.py +++ b/sdks/python/apache_beam/transforms/external_transform_provider.py @@ -39,32 +39,6 @@ def snake_case_to_upper_camel_case(string): return output -def snake_case_to_lower_camel_case(string): - """Convert snake_case to lowerCamelCase""" - if len(string) <= 1: - return string.lower() - upper = snake_case_to_upper_camel_case(string) - return upper[0].lower() + upper[1:] - - -def camel_case_to_snake_case(string): - """Convert camelCase to snake_case""" - arr = [] - word = [] - for i, n in enumerate(string): - # If seeing an upper letter after a lower letter, we just witnessed a word - # If seeing an upper letter and the next letter is lower, we may have just - # witnessed an all caps word - if n.isupper() and ((i > 0 and string[i - 1].islower()) or - (i + 1 < len(string) and string[i + 1].islower())): - arr.append(''.join(word)) - word = [n.lower()] - else: - word.append(n.lower()) - arr.append(''.join(word)) - return '_'.join(arr).strip('_') - - # Information regarding a Wrapper parameter. ParamInfo = namedtuple('ParamInfo', ['type', 'description', 'original_name']) @@ -76,7 +50,7 @@ def get_config_with_descriptions( descriptions = schematransform.configuration_schema._field_descriptions fields_with_descriptions = {} for field in schema.fields: - fields_with_descriptions[camel_case_to_snake_case(field.name)] = ParamInfo( + fields_with_descriptions[field.name] = ParamInfo( typing_from_runner_api(field.type), descriptions[field.name], field.name) @@ -105,16 +79,11 @@ def __init__(self, expansion_service=None, **kwargs): expansion_service or self.default_expansion_service def expand(self, input): - camel_case_kwargs = { - snake_case_to_lower_camel_case(k): v - for k, v in self._kwargs.items() - } - external_schematransform = SchemaAwareExternalTransform( identifier=self.identifier, expansion_service=self._expansion_service, rearrange_based_on_discovery=True, - **camel_case_kwargs) + **self._kwargs) return input | external_schematransform diff --git a/sdks/python/apache_beam/transforms/external_transform_provider_it_test.py b/sdks/python/apache_beam/transforms/external_transform_provider_it_test.py index a53001c85fd3..95720cee7eee 100644 --- a/sdks/python/apache_beam/transforms/external_transform_provider_it_test.py +++ b/sdks/python/apache_beam/transforms/external_transform_provider_it_test.py @@ -37,9 +37,7 @@ from apache_beam.transforms.external_transform_provider import STANDARD_URN_PATTERN from apache_beam.transforms.external_transform_provider import ExternalTransform from apache_beam.transforms.external_transform_provider import ExternalTransformProvider -from apache_beam.transforms.external_transform_provider import camel_case_to_snake_case from apache_beam.transforms.external_transform_provider import infer_name_from_identifier -from apache_beam.transforms.external_transform_provider import snake_case_to_lower_camel_case from apache_beam.transforms.external_transform_provider import snake_case_to_upper_camel_case from apache_beam.transforms.xlang.io import GenerateSequence @@ -54,26 +52,6 @@ def test_snake_case_to_upper_camel_case(self): for case in test_cases: self.assertEqual(case[1], snake_case_to_upper_camel_case(case[0])) - def test_snake_case_to_lower_camel_case(self): - test_cases = [("", ""), ("test", "test"), ("test_name", "testName"), - ("test_double_underscore", "testDoubleUnderscore"), - ("TEST_CAPITALIZED", "testCapitalized"), - ("_prepended_underscore", "prependedUnderscore"), - ("appended_underscore_", "appendedUnderscore")] - for case in test_cases: - self.assertEqual(case[1], snake_case_to_lower_camel_case(case[0])) - - def test_camel_case_to_snake_case(self): - test_cases = [("", ""), ("Test", "test"), ("TestName", "test_name"), - ("TestDoubleUnderscore", - "test_double_underscore"), ("MyToLoFo", "my_to_lo_fo"), - ("BEGINNINGAllCaps", - "beginning_all_caps"), ("AllCapsENDING", "all_caps_ending"), - ("AllCapsMIDDLEWord", "all_caps_middle_word"), - ("lowerCamelCase", "lower_camel_case")] - for case in test_cases: - self.assertEqual(case[1], camel_case_to_snake_case(case[0])) - def test_infer_name_from_identifier(self): standard_test_cases = [ ("beam:schematransform:org.apache.beam:transform:v1", "Transform"), diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index 5f53302028c8..1e9c7c605460 100755 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -889,7 +889,7 @@ def java_window_into(java_provider, windowing): return java_provider.create_transform( 'WindowIntoStrategy', { - 'serializedWindowingStrategy': windowing_strategy.to_runner_api( + 'serialized_windowing_strategy': windowing_strategy.to_runner_api( empty_context).SerializeToString() }, None) diff --git a/sdks/python/build.gradle b/sdks/python/build.gradle index 33c339dbd50b..e21f5db75ad2 100644 --- a/sdks/python/build.gradle +++ b/sdks/python/build.gradle @@ -80,7 +80,7 @@ tasks.register("generateExternalTransformsConfig") { exec { executable 'sh' args '-c', "pip install $PyYaml && " + - "python -m apache_beam.yaml.generate_yaml_docs.py --cleanup --generate-config-only" + "python gen_xlang_wrappers.py --cleanup --generate-config-only" } } } diff --git a/sdks/python/gen_xlang_wrappers.py b/sdks/python/gen_xlang_wrappers.py index a75fc05cba73..ea4f496c2d04 100644 --- a/sdks/python/gen_xlang_wrappers.py +++ b/sdks/python/gen_xlang_wrappers.py @@ -233,24 +233,6 @@ def pretty_type(tp): return (tp, nullable) -def camel_case_to_snake_case(string): - """Convert camelCase to snake_case""" - arr = [] - word = [] - for i, n in enumerate(string): - # If seeing an upper letter after a lower letter, we just witnessed a word - # If seeing an upper letter and the next letter is lower, we may have just - # witnessed an all caps word - if n.isupper() and ((i > 0 and string[i - 1].islower()) or - (i + 1 < len(string) and string[i + 1].islower())): - arr.append(''.join(word)) - word = [n.lower()] - else: - word.append(n.lower()) - arr.append(''.join(word)) - return '_'.join(arr).strip('_') - - def get_wrappers_from_transform_configs(config_file) -> Dict[str, List[str]]: """ Generates code for external transform wrapper classes (subclasses of @@ -287,9 +269,8 @@ def get_wrappers_from_transform_configs(config_file) -> Dict[str, List[str]]: parameters = [] for param, info in fields.items(): - pythonic_name = camel_case_to_snake_case(param) param_details = { - "name": pythonic_name, + "name": param, "type": info['type'], "description": info['description'], } diff --git a/sdks/standard_expansion_services.yaml b/sdks/standard_expansion_services.yaml index e9e6871be82f..31a1a6343aed 100644 --- a/sdks/standard_expansion_services.yaml +++ b/sdks/standard_expansion_services.yaml @@ -44,6 +44,9 @@ # Handwritten Kafka wrappers already exist in apache_beam/io/kafka.py - 'beam:schematransform:org.apache.beam:kafka_write:v1' - 'beam:schematransform:org.apache.beam:kafka_read:v1' + # Not ready to generate + - 'beam:schematransform:org.apache.beam:iceberg_write:v1' + - 'beam:schematransform:org.apache.beam:iceberg_read:v1' # TODO(ahmedabu98): Enable this service in a future PR #- gradle_target: 'sdks:java:io:google-cloud-platform:expansion-service:shadowJar' diff --git a/sdks/standard_external_transforms.yaml b/sdks/standard_external_transforms.yaml index b43e93ab4919..05aa3c9b9de5 100644 --- a/sdks/standard_external_transforms.yaml +++ b/sdks/standard_external_transforms.yaml @@ -19,7 +19,7 @@ # configuration in /sdks/standard_expansion_services.yaml. # Refer to gen_xlang_wrappers.py for more info. # -# Last updated on: 2024-02-22 +# Last updated on: 2024-04-18 - default_service: sdks:java:io:expansion-service:shadowJar description: 'Outputs a PCollection of Beam Rows, each containing a single INT64 @@ -43,7 +43,8 @@ description: Specifies the rate to generate a given number of elements per a given number of seconds. Applicable only to unbounded sequences. nullable: true - type: Row(seconds=typing.Union[numpy.int64, NoneType], elements=) + type: Row(elements=, seconds=typing.Union[numpy.int64, + NoneType]) start: description: The minimum number to generate (inclusive). nullable: false