From 42611e0491e9ed624afb1faf367510abe9546436 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 9 Apr 2024 16:50:41 -0400 Subject: [PATCH 01/60] iceberg write schematransform and test --- .../IcebergWriteSchemaTransformProvider.java | 216 ++++++++++++++++++ ...ebergWriteSchemaTransformProviderTest.java | 128 +++++++++++ 2 files changed, 344 insertions(+) create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProvider.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProviderTest.java diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProvider.java new file mode 100644 index 000000000000..51dd17092b98 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProvider.java @@ -0,0 +1,216 @@ +/* + * 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.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; + +import com.google.auto.service.AutoService; +import com.google.auto.value.AutoValue; +import java.util.*; +import org.apache.beam.io.iceberg.IcebergWriteSchemaTransformProvider.Config; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.util.Preconditions; +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.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.catalog.TableIdentifier; +import org.checkerframework.checker.nullness.qual.Nullable; + +@AutoService(SchemaTransformProvider.class) +public class IcebergWriteSchemaTransformProvider extends TypedSchemaTransformProvider { + + static final String INPUT_TAG = "input"; + static final String OUTPUT_TAG = "output"; + + @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 "beam:schematransform:org.apache.beam:iceberg_write:v1"; + } + + @DefaultSchema(AutoValueSchema.class) + @AutoValue + public abstract static class Config { + public static Builder builder() { + return new AutoValue_IcebergWriteSchemaTransformProvider_Config.Builder(); + } + + public abstract String getTable(); + + public abstract CatalogConfig getCatalogConfig(); + + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setTable(String tables); + + public abstract Builder setCatalogConfig(CatalogConfig catalogConfig); + + public abstract Config build(); + } + + public void validate() { + getCatalogConfig().validate(); + } + } + + @DefaultSchema(AutoValueSchema.class) + @AutoValue + public abstract static class CatalogConfig { + public static Builder builder() { + return new AutoValue_IcebergWriteSchemaTransformProvider_CatalogConfig.Builder(); + } + + public abstract String getCatalogName(); + + 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 CatalogConfig build(); + } + + Set validTypes = + 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( + validTypes.contains(Preconditions.checkArgumentNotNull(getCatalogType())), + "Invalid catalog type. Please pick one of %s", + validTypes); + } + } + } + + @VisibleForTesting + static class IcebergWriteSchemaTransform extends SchemaTransform { + private final Config configuration; + + IcebergWriteSchemaTransform(Config configuration) { + this.configuration = configuration; + } + + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + + PCollection rows = input.get(INPUT_TAG); + + CatalogConfig catalogConfig = configuration.getCatalogConfig(); + + IcebergCatalogConfig.Builder catalogBuilder = + IcebergCatalogConfig.builder() + .setName(catalogConfig.getCatalogName()) + .setIcebergCatalogType(catalogConfig.getCatalogType()) + .setWarehouseLocation(catalogConfig.getWarehouseLocation()); + + if (!Strings.isNullOrEmpty(catalogConfig.getCatalogType())) { + catalogBuilder = catalogBuilder.setIcebergCatalogType(catalogConfig.getCatalogType()); + } + if (!Strings.isNullOrEmpty(catalogConfig.getWarehouseLocation())) { + catalogBuilder = catalogBuilder.setWarehouseLocation(catalogConfig.getWarehouseLocation()); + } + + // TODO: support dynamic destinations + DynamicDestinations dynamicDestinations = + DynamicDestinations.singleTable(TableIdentifier.parse(configuration.getTable())); + + IcebergWriteResult result = + rows.apply( + IcebergIO.writeToDynamicDestinations(catalogBuilder.build(), dynamicDestinations)); + + PCollection snapshots = + result + .getSnapshots() + .apply(MapElements.via(new SnapshotToRow())) + .setRowSchema(SnapshotToRow.SNAPSHOT_SCHEMA); + + return PCollectionRowTuple.of(OUTPUT_TAG, snapshots); + } + + @VisibleForTesting + static class SnapshotToRow extends SimpleFunction, Row> { + static final Schema SNAPSHOT_SCHEMA = + Schema.builder() + .addStringField("table") + .addStringField("operation") + .addMapField("summary", Schema.FieldType.STRING, Schema.FieldType.STRING) + .addStringField("manifestListLocation") + .build(); + + @Override + public Row apply(KV input) { + Snapshot snapshot = input.getValue(); + Row row = + Row.withSchema(SNAPSHOT_SCHEMA) + .addValues( + input.getKey(), + snapshot.operation(), + snapshot.summary(), + snapshot.manifestListLocation()) + .build(); + System.out.println("SNAPSHOT: " + snapshot); + System.out.println("ROW: " + row); + return row; + } + } + } +} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProviderTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProviderTest.java new file mode 100644 index 000000000000..ee8a03e719cd --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProviderTest.java @@ -0,0 +1,128 @@ +/* + * 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.io.iceberg.IcebergWriteSchemaTransformProvider.CatalogConfig; +import static org.apache.beam.io.iceberg.IcebergWriteSchemaTransformProvider.Config; +import static org.apache.beam.io.iceberg.IcebergWriteSchemaTransformProvider.INPUT_TAG; +import static org.apache.beam.io.iceberg.IcebergWriteSchemaTransformProvider.OUTPUT_TAG; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertEquals; + +import java.util.List; +import java.util.UUID; +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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@RunWith(JUnit4.class) +public class IcebergWriteSchemaTransformProviderTest { + + private static final Logger LOG = + LoggerFactory.getLogger(IcebergWriteSchemaTransformProviderTest.class); + + @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 testSimpleAppend() { + String identifier = "default.table_" + Long.toString(UUID.randomUUID().hashCode(), 16); + + TableIdentifier tableId = TableIdentifier.parse(identifier); + System.out.println(tableId); + + // Create a table and add records to it. + Table table = warehouse.createTable(tableId, TestFixtures.SCHEMA); + + Config config = + Config.builder() + .setTable(identifier) + .setCatalogConfig( + CatalogConfig.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(SchemaHelper.convert(TestFixtures.SCHEMA))); + + PCollection result = + input + .apply("Append To Table", new IcebergWriteSchemaTransformProvider().from(config)) + .get(OUTPUT_TAG); + + PAssert.that(result).satisfies(new VerifyOutputs(identifier, "append")); + + LOG.info("Executing pipeline"); + testPipeline.run().waitUntilFinish(); + LOG.info("Done running pipeline"); + + 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; + } + } +} From 16e62354072a456895af1feb5ffec57744589677 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 9 Apr 2024 16:56:55 -0400 Subject: [PATCH 02/60] cleanup --- .../iceberg/IcebergWriteSchemaTransformProviderTest.java | 7 ------- 1 file changed, 7 deletions(-) diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProviderTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProviderTest.java index ee8a03e719cd..65aa317b7ec5 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProviderTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProviderTest.java @@ -46,15 +46,10 @@ import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; @RunWith(JUnit4.class) public class IcebergWriteSchemaTransformProviderTest { - private static final Logger LOG = - LoggerFactory.getLogger(IcebergWriteSchemaTransformProviderTest.class); - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); @Rule @@ -98,9 +93,7 @@ public void testSimpleAppend() { PAssert.that(result).satisfies(new VerifyOutputs(identifier, "append")); - LOG.info("Executing pipeline"); testPipeline.run().waitUntilFinish(); - LOG.info("Done running pipeline"); List writtenRecords = ImmutableList.copyOf(IcebergGenerics.read(table).build()); From ed728986b430e606f6439ee89e6f7b2a5fe54160 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 11 Apr 2024 12:04:19 -0400 Subject: [PATCH 03/60] IcebergIO translation and tests --- .../bigquery/BigQueryIOTranslationTest.java | 2 +- .../org/apache/beam/io/iceberg/IcebergIO.java | 62 ++++++-- .../beam/io/iceberg/IcebergIOTranslation.java | 136 ++++++++++++++++++ .../beam/io/iceberg/IcebergIOWriteTest.java | 4 +- .../io/iceberg/IcebergIoTranslationTest.java | 120 ++++++++++++++++ 5 files changed, 311 insertions(+), 13 deletions(-) create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIOTranslation.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergIoTranslationTest.java 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 7f2ff8945482..5be0c4a57c2c 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 @@ -42,7 +42,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/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 index 4b9e3102a02e..84d2c2b468ec 100644 --- 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 @@ -17,34 +17,78 @@ */ package org.apache.beam.io.iceberg; +import com.google.auto.value.AutoValue; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.transforms.PTransform; 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.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(); + } + public static WriteRows writeToDynamicDestinations( IcebergCatalogConfig catalog, DynamicDestinations dynamicDestinations) { - return new WriteRows(catalog, dynamicDestinations); + return new AutoValue_IcebergIO_WriteRows.Builder() + .setCatalogConfig(catalog) + .setDynamicDestinations(dynamicDestinations) + .build(); } - static class WriteRows extends PTransform, IcebergWriteResult> { + @AutoValue + public abstract static class WriteRows extends PTransform, IcebergWriteResult> { + + abstract IcebergCatalogConfig getCatalogConfig(); + + abstract @Nullable TableIdentifier getTableIdentifier(); + + abstract @Nullable DynamicDestinations getDynamicDestinations(); + + abstract Builder toBuilder(); - private final IcebergCatalogConfig catalog; - private final DynamicDestinations dynamicDestinations; + @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(); + } - private WriteRows(IcebergCatalogConfig catalog, DynamicDestinations dynamicDestinations) { - this.catalog = catalog; - this.dynamicDestinations = dynamicDestinations; + 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(dynamicDestinations)) + .apply("Set Destination Metadata", new AssignDestinations(destinations)) .apply( - "Write Rows to Destinations", new WriteToDestinations(catalog, dynamicDestinations)); + "Write Rows to Destinations", + new WriteToDestinations(getCatalogConfig(), destinations)); } } } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIOTranslation.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIOTranslation.java new file mode 100644 index 000000000000..04898d4ced25 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIOTranslation.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.io.iceberg; + +import static org.apache.beam.io.iceberg.IcebergIO.WriteRows; +import static org.apache.beam.sdk.util.construction.TransformUpgrader.fromByteArray; +import static org.apache.beam.sdk.util.construction.TransformUpgrader.toByteArray; + +import com.google.auto.service.AutoService; +import java.io.IOException; +import java.io.InvalidClassException; +import java.util.*; +import java.util.stream.Collectors; +import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.Schema.FieldType; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.PTransformTranslation.TransformPayloadTranslator; +import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.util.construction.TransformPayloadTranslatorRegistrar; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.catalog.TableIdentifier; +import org.checkerframework.checker.nullness.qual.Nullable; + +@SuppressWarnings({"rawtypes", "nullness"}) +public class IcebergIOTranslation { + static class IcebergIOWriteTranslator implements TransformPayloadTranslator { + + static Schema schema = + Schema.builder() + .addByteArrayField("catalog_config") + .addNullableArrayField("table_identifier", FieldType.STRING) + .addNullableByteArrayField("dynamic_destinations") + .build(); + + public static final String ICEBERG_WRITE_TRANSFORM_URN = + "beam:transform:org.apache.beam:iceberg_write:v1"; + + @Override + public String getUrn() { + return ICEBERG_WRITE_TRANSFORM_URN; + } + + @Override + public @Nullable FunctionSpec translate( + AppliedPTransform application, SdkComponents components) + throws IOException { + // Setting an empty payload since Iceberg transform payload is not actually used by runners + // currently. + return FunctionSpec.newBuilder().setUrn(getUrn()).setPayload(ByteString.empty()).build(); + } + + @Override + public Row toConfigRow(WriteRows transform) { + + Map fieldValues = new HashMap<>(); + + if (transform.getCatalogConfig() != null) { + fieldValues.put("catalog_config", toByteArray(transform.getCatalogConfig())); + } + if (transform.getTableIdentifier() != null) { + TableIdentifier identifier = transform.getTableIdentifier(); + List identifierParts = + Arrays.stream(identifier.namespace().levels()).collect(Collectors.toList()); + identifierParts.add(identifier.name()); + fieldValues.put("table_identifier", identifierParts); + } + if (transform.getDynamicDestinations() != null) { + fieldValues.put("dynamic_destinations", toByteArray(transform.getDynamicDestinations())); + } + + return Row.withSchema(schema).withFieldValues(fieldValues).build(); + } + + @Override + public WriteRows fromConfigRow(Row configRow, PipelineOptions options) { + try { + IcebergIO.WriteRows.Builder builder = new AutoValue_IcebergIO_WriteRows.Builder(); + + byte[] catalogBytes = configRow.getBytes("catalog_config"); + if (catalogBytes != null) { + builder = builder.setCatalogConfig((IcebergCatalogConfig) fromByteArray(catalogBytes)); + } + Collection tableIdentifierParts = configRow.getArray("table_identifier"); + if (tableIdentifierParts != null) { + builder = + builder.setTableIdentifier( + TableIdentifier.parse(String.join(".", tableIdentifierParts))); + } + byte[] dynamicDestinationsBytes = configRow.getBytes("dynamic_destinations"); + if (dynamicDestinationsBytes != null) { + builder = + builder.setDynamicDestinations( + (DynamicDestinations) fromByteArray(dynamicDestinationsBytes)); + } + return builder.build(); + } catch (InvalidClassException e) { + throw new RuntimeException(e); + } + } + } + + @AutoService(TransformPayloadTranslatorRegistrar.class) + public static class WriteRegistrar implements TransformPayloadTranslatorRegistrar { + + @Override + @SuppressWarnings({ + "rawtypes", + }) + public Map, ? extends TransformPayloadTranslator> + getTransformPayloadTranslators() { + return ImmutableMap., TransformPayloadTranslator>builder() + .put(AutoValue_IcebergIO_WriteRows.class, new IcebergIOWriteTranslator()) + .build(); + } + } +} 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/io/iceberg/IcebergIOWriteTest.java index c77d162aafd4..15b72e7f7b20 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/io/iceberg/IcebergIOWriteTest.java @@ -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(SchemaHelper.convert(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(); diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergIoTranslationTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergIoTranslationTest.java new file mode 100644 index 000000000000..018717bd480e --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergIoTranslationTest.java @@ -0,0 +1,120 @@ +/* + * 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.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +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.TemporaryFolder; + +public class IcebergIoTranslationTest { + // A mapping from Read transform builder methods to the corresponding schema fields in + // BigQueryIOTranslation. + static final Map WRITE_TRANSFORM_SCHEMA_MAPPING = + ImmutableMap.builder() + .put("getCatalogConfig", "catalog_config") + .put("getTableIdentifier", "table_identifier") + .put("getDynamicDestinations", "dynamic_destinations") + .build(); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public transient TestDataWarehouse warehouse = new TestDataWarehouse(TEMPORARY_FOLDER, "default"); + + @Test + public void testReCreateWriteTransformFromRowTable() { + // setting a subset of fields here. + IcebergCatalogConfig config = + IcebergCatalogConfig.builder() + .setName("test_catalog") + .setIcebergCatalogType(CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .setWarehouseLocation(warehouse.location) + .build(); + IcebergIO.WriteRows writeTransform = + IcebergIO.writeRows(config).to(TableIdentifier.of("test_namespace", "test_table")); + + IcebergIOTranslation.IcebergIOWriteTranslator translator = + new IcebergIOTranslation.IcebergIOWriteTranslator(); + Row row = translator.toConfigRow(writeTransform); + + IcebergIO.WriteRows writeTransformFromRow = + translator.fromConfigRow(row, PipelineOptionsFactory.create()); + assertNotNull(writeTransformFromRow.getTableIdentifier()); + assertEquals( + "test_namespace", writeTransformFromRow.getTableIdentifier().namespace().levels()[0]); + assertEquals("test_table", writeTransformFromRow.getTableIdentifier().name()); + assertEquals("test_catalog", writeTransformFromRow.getCatalogConfig().getName()); + assertEquals( + CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP, + writeTransformFromRow.getCatalogConfig().getIcebergCatalogType()); + assertEquals( + warehouse.location, writeTransformFromRow.getCatalogConfig().getWarehouseLocation()); + } + + @Test + public void testWriteTransformRowIncludesAllFields() { + // These fields do not represent properties of the transform. + List getMethodNames = + Arrays.stream(IcebergIO.WriteRows.class.getDeclaredMethods()) + .map(method -> method.getName()) + .filter(methodName -> methodName.startsWith("get")) + .collect(Collectors.toList()); + + // Just to make sure that this does not pass trivially. + assertTrue(getMethodNames.size() > 0); + + for (String getMethodName : getMethodNames) { + assertTrue( + "Method " + + getMethodName + + " will not be tracked when upgrading the 'IcebergIO.WriteRows' transform. Please update" + + "'IcebergIOTranslation.IcebergIOWriteTranslator' to track the new method " + + "and update this test.", + WRITE_TRANSFORM_SCHEMA_MAPPING.keySet().contains(getMethodName)); + } + + // Confirming that all fields mentioned in `WRITE_TRANSFORM_SCHEMA_MAPPING` are + // actually available in the schema. + WRITE_TRANSFORM_SCHEMA_MAPPING.values().stream() + .forEach( + fieldName -> { + assertTrue( + "Field name " + + fieldName + + " was not found in the transform schema defined in " + + "IcebergIOTranslation.IcebergIOWriteTranslator.", + IcebergIOTranslation.IcebergIOWriteTranslator.schema + .getFieldNames() + .contains(fieldName)); + }); + } +} From 1738345ed8154a2991d214b422564add30c49754 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 11 Apr 2024 12:28:11 -0400 Subject: [PATCH 04/60] add sanity check for building with Row; add documentation about output schema; add iceberg to IO expansion service --- sdks/java/io/expansion-service/build.gradle | 1 + .../IcebergWriteSchemaTransformProvider.java | 31 +++++++++++++------ ...ebergWriteSchemaTransformProviderTest.java | 25 +++++++++++++++ 3 files changed, 47 insertions(+), 10 deletions(-) diff --git a/sdks/java/io/expansion-service/build.gradle b/sdks/java/io/expansion-service/build.gradle index 9ab71ff16d3d..acee66d55811 100644 --- a/sdks/java/io/expansion-service/build.gradle +++ b/sdks/java/io/expansion-service/build.gradle @@ -33,6 +33,7 @@ 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") 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/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProvider.java index 51dd17092b98..cb65886970e7 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProvider.java @@ -44,12 +44,31 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.checkerframework.checker.nullness.qual.Nullable; +/** + * SchemaTransform implementation for {@link IcebergIO#writeToDynamicDestinations}. 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") + .addStringField("operation") + .addMapField("summary", Schema.FieldType.STRING, Schema.FieldType.STRING) + .addStringField("manifestListLocation") + .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(); @@ -181,26 +200,18 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { result .getSnapshots() .apply(MapElements.via(new SnapshotToRow())) - .setRowSchema(SnapshotToRow.SNAPSHOT_SCHEMA); + .setRowSchema(OUTPUT_SCHEMA); return PCollectionRowTuple.of(OUTPUT_TAG, snapshots); } @VisibleForTesting static class SnapshotToRow extends SimpleFunction, Row> { - static final Schema SNAPSHOT_SCHEMA = - Schema.builder() - .addStringField("table") - .addStringField("operation") - .addMapField("summary", Schema.FieldType.STRING, Schema.FieldType.STRING) - .addStringField("manifestListLocation") - .build(); - @Override public Row apply(KV input) { Snapshot snapshot = input.getValue(); Row row = - Row.withSchema(SNAPSHOT_SCHEMA) + Row.withSchema(OUTPUT_SCHEMA) .addValues( input.getKey(), snapshot.operation(), diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProviderTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProviderTest.java index 65aa317b7ec5..9ae38ecfd843 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProviderTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProviderTest.java @@ -26,6 +26,9 @@ import java.util.List; import java.util.UUID; +import org.apache.beam.sdk.schemas.NoSuchSchemaException; +import org.apache.beam.sdk.schemas.SchemaRegistry; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; @@ -57,6 +60,28 @@ public class IcebergWriteSchemaTransformProviderTest { @Rule public transient TestPipeline testPipeline = TestPipeline.create(); + @Test + public void testBuildTransformWithRow() throws NoSuchSchemaException { + Row catalogConfigRow = + Row.withSchema( + SchemaRegistry.createDefault() + .getSchema(IcebergWriteSchemaTransformProvider.CatalogConfig.class)) + .withFieldValue("catalogName", "test_name") + .withFieldValue("catalogType", "test_type") + .withFieldValue("catalogImplementation", "testImplementation") + .withFieldValue("warehouseLocation", "test_location") + .build(); + Row transformConfigRow = + Row.withSchema(new IcebergWriteSchemaTransformProvider().configurationSchema()) + .withFieldValue("table", "test_table_identifier") + .withFieldValue("catalogConfig", catalogConfigRow) + .build(); + + SchemaTransform transform = new IcebergWriteSchemaTransformProvider().from(transformConfigRow); + + System.out.println(transform.getName()); + } + @Test public void testSimpleAppend() { String identifier = "default.table_" + Long.toString(UUID.randomUUID().hashCode(), 16); From 364ebbe90c6360e39a44059ec9bfd7d94dd22f6f Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 11 Apr 2024 12:31:04 -0400 Subject: [PATCH 05/60] spotless --- .../beam/io/iceberg/IcebergWriteSchemaTransformProvider.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProvider.java index cb65886970e7..ec4a946f22d4 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProvider.java @@ -21,7 +21,9 @@ import com.google.auto.service.AutoService; import com.google.auto.value.AutoValue; -import java.util.*; +import java.util.Collections; +import java.util.List; +import java.util.Set; import org.apache.beam.io.iceberg.IcebergWriteSchemaTransformProvider.Config; import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.Schema; From 79d2c9486d9d51019b2429cdb055bf5f03ae2b82 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 11 Apr 2024 12:36:15 -0400 Subject: [PATCH 06/60] spotless --- .../org/apache/beam/io/iceberg/IcebergIOTranslation.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIOTranslation.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIOTranslation.java index 04898d4ced25..29d9b3790830 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIOTranslation.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIOTranslation.java @@ -24,7 +24,11 @@ import com.google.auto.service.AutoService; import java.io.IOException; import java.io.InvalidClassException; -import java.util.*; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.stream.Collectors; import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; import org.apache.beam.sdk.options.PipelineOptions; From 30de265e2881dfee63be0a0217b16fcf5c66df1c Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 11 Apr 2024 13:35:30 -0400 Subject: [PATCH 07/60] permitUnusedDeclared iceberg --- sdks/java/io/expansion-service/build.gradle | 1 + 1 file changed, 1 insertion(+) diff --git a/sdks/java/io/expansion-service/build.gradle b/sdks/java/io/expansion-service/build.gradle index acee66d55811..15e0baaa7318 100644 --- a/sdks/java/io/expansion-service/build.gradle +++ b/sdks/java/io/expansion-service/build.gradle @@ -34,6 +34,7 @@ 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") From 905d59045a22df9167aa479dbf5d41a50c0fe479 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 11 Apr 2024 18:33:02 -0400 Subject: [PATCH 08/60] Change ManagedSchemaTransformProvider to take a Row config instead of a Yaml string --- .../beam/sdk/schemas/utils/YamlUtils.java | 27 ++++++++--- .../apache/beam/sdk/util/YamlUtilsTest.java | 36 ++++++++++++++- .../org/apache/beam/sdk/managed/Managed.java | 45 +++++++++++++------ .../ManagedSchemaTransformProvider.java | 33 +++++++++----- .../ManagedSchemaTransformProviderTest.java | 27 ++++++----- 5 files changed, 120 insertions(+), 48 deletions(-) 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..5571c9e4fed6 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 @@ -53,11 +53,11 @@ public class YamlUtils { .put(Schema.TypeName.BYTES, str -> BaseEncoding.base64().decode(str)) .build(); - public static Row toBeamRow(@Nullable String yamlString, Schema schema) { + public static @Nullable Row toBeamRow(@Nullable String yamlString, Schema schema) { return toBeamRow(yamlString, schema, false); } - public static Row toBeamRow( + public static @Nullable Row toBeamRow( @Nullable String yamlString, Schema schema, boolean convertNamesToCamelCase) { if (yamlString == null || yamlString.isEmpty()) { List requiredFields = @@ -65,7 +65,7 @@ public static Row toBeamRow( .filter(field -> !field.getType().getNullable()) .collect(Collectors.toList()); if (requiredFields.isEmpty()) { - return Row.nullRow(schema); + return null; } else { throw new IllegalArgumentException( String.format( @@ -147,14 +147,27 @@ public static Row toBeamRow( } @SuppressWarnings("nullness") - public static Row toBeamRow(Map yamlMap, Schema rowSchema, boolean toCamelCase) { + public static @Nullable 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 null; + } 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/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..01cf784f6298 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 @@ -18,14 +18,17 @@ package org.apache.beam.sdk.util; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; 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; @@ -47,7 +50,7 @@ public String makeNested(String input) { public void testEmptyYamlString() { Schema schema = Schema.builder().build(); - assertEquals(Row.nullRow(schema), YamlUtils.toBeamRow("", schema)); + assertNull(YamlUtils.toBeamRow("", schema)); } @Test @@ -225,4 +228,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/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..d746de306f55 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 @@ -26,6 +26,7 @@ import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; import org.apache.beam.sdk.schemas.utils.YamlUtils; 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.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; @@ -36,8 +37,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 +49,7 @@ *
{@code
  * PCollectionRowTuple output = PCollectionRowTuple.empty(pipeline).apply(
  *       Managed.read(ICEBERG)
- *           .withConfig(ImmutableMap..builder()
+ *           .withConfig(ImmutableMap..builder()
  *               .put("foo", "abc")
  *               .put("bar", 123)
  *               .build()));
@@ -87,15 +88,14 @@ public class Managed {
           .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 @@ -131,7 +131,7 @@ public static ManagedTransform write(String sink) { public abstract static class ManagedTransform extends SchemaTransform { abstract String getIdentifier(); - abstract @Nullable String getConfig(); + abstract @Nullable Map getConfig(); abstract @Nullable String getConfigUrl(); @@ -144,7 +144,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 +161,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(); } /** @@ -179,15 +179,32 @@ ManagedTransform withSupportedIdentifiers(List supportedIdentifiers) { @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { + ManagedSchemaTransformProvider provider = + new ManagedSchemaTransformProvider(getSupportedIdentifiers()); + + SchemaTransformProvider underlyingTransformProvider = + provider.getAllProviders().get(getIdentifier()); + if (underlyingTransformProvider == null) { + throw new RuntimeException( + String.format( + "Could not find transform with identifier %s, or it may not be supported.", + getIdentifier())); + } + + Row transformConfigRow = + getConfig() != null + ? YamlUtils.toBeamRow( + getConfig(), underlyingTransformProvider.configurationSchema(), true) + : null; + ManagedSchemaTransformProvider.ManagedConfig managedConfig = ManagedSchemaTransformProvider.ManagedConfig.builder() .setTransformIdentifier(getIdentifier()) - .setConfig(getConfig()) + .setConfig(transformConfigRow) .setConfigUrl(getConfigUrl()) .build(); - SchemaTransform underlyingTransform = - new ManagedSchemaTransformProvider(getSupportedIdentifiers()).from(managedConfig); + SchemaTransform underlyingTransform = provider.from(managedConfig); return input.apply(underlyingTransform); } 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..f64f71a7259b 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,8 +24,10 @@ 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; @@ -41,6 +43,7 @@ import org.apache.beam.sdk.schemas.utils.YamlUtils; import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.common.base.Predicates; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; @@ -90,8 +93,8 @@ public static Builder builder() { @SchemaFieldDescription("URL path to the YAML config file used to build the underlying IO.") public abstract @Nullable String getConfigUrl(); - @SchemaFieldDescription("YAML string config used to build the underlying IO.") - public abstract @Nullable String getConfig(); + @SchemaFieldDescription("Row config used to build the underlying IO.") + public abstract @Nullable Row getConfig(); @AutoValue.Builder public abstract static class Builder { @@ -99,16 +102,17 @@ public abstract static class Builder { public abstract Builder setConfigUrl(@Nullable String configUrl); - public abstract Builder setConfig(@Nullable String config); + public abstract Builder setConfig(@Nullable Row config); public abstract ManagedConfig build(); } protected void validate() { - boolean configExists = !Strings.isNullOrEmpty(getConfig()); + boolean configExists = getConfig() != null; 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."); } } @@ -158,25 +162,30 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { @VisibleForTesting static Row getRowConfig(ManagedConfig config, Schema transformSchema) { - String transformYamlConfig; - if (!Strings.isNullOrEmpty(config.getConfigUrl())) { + Row transformRowConfig = config.getConfig(); + // Attempt to construct a Row config from a YAML file + if (transformRowConfig == null) { 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); + String yamlConfig = new String(buffer.array(), StandardCharsets.UTF_8); + transformRowConfig = YamlUtils.toBeamRow(yamlConfig, transformSchema, true); } catch (IOException e) { throw new RuntimeException(e); } - } else { - transformYamlConfig = config.getConfig(); } - return YamlUtils.toBeamRow(transformYamlConfig, transformSchema, true); + // If our config is still null (perhaps the underlying transform doesn't have any parameters), + // default to an empty row. + if (transformRowConfig == null) { + transformRowConfig = Row.nullRow(transformSchema); + } + + return transformRowConfig; } - @VisibleForTesting Map getAllProviders() { return schemaTransformProviders; } 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..32c858e6497e 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 @@ -49,28 +49,27 @@ public void testFailWhenNoConfigSpecified() { } @Test - public void testGetRowFromYamlConfig() { - String yamlString = "extra_string: abc\n" + "extra_integer: 123"; + public void testGetConfigRow() { + Schema underlyingTransformSchema = new TestSchemaTransformProvider().configurationSchema(); + Row configRow = + Row.withSchema(underlyingTransformSchema) + .withFieldValue("extraString", "abc") + .withFieldValue("extraInteger", 123) + .build(); ManagedConfig config = ManagedConfig.builder() .setTransformIdentifier(TestSchemaTransformProvider.IDENTIFIER) - .setConfig(yamlString) + .setConfig(configRow) .build(); - Schema configSchema = new TestSchemaTransformProvider().configurationSchema(); - Row expectedRow = - Row.withSchema(configSchema) - .withFieldValue("extraString", "abc") - .withFieldValue("extraInteger", 123) - .build(); - Row configRow = - ManagedSchemaTransformProvider.getRowConfig( - config, new TestSchemaTransformProvider().configurationSchema()); - assertEquals(expectedRow, configRow); + Row returnedRow = + ManagedSchemaTransformProvider.getRowConfig(config, underlyingTransformSchema); + + assertEquals(configRow, returnedRow); } @Test - public void testGetRowFromConfigUrl() throws URISyntaxException { + public void testGetConfigRowFromYamlFile() throws URISyntaxException { String yamlConfigPath = Paths.get(getClass().getClassLoader().getResource("test_config.yaml").toURI()) .toFile() From 1067d84af992d13a02251f07ac2d797a92400196 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 11 Apr 2024 18:50:13 -0400 Subject: [PATCH 09/60] don't auto generate external wrapper for this just yet --- .github/workflows/beam_PreCommit_Xlang_Generated_Transforms.yml | 2 +- sdks/standard_expansion_services.yaml | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/.github/workflows/beam_PreCommit_Xlang_Generated_Transforms.yml b/.github/workflows/beam_PreCommit_Xlang_Generated_Transforms.yml index f8d64eb5d4a6..fed883412fc6 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 \ No newline at end of file diff --git a/sdks/standard_expansion_services.yaml b/sdks/standard_expansion_services.yaml index e9e6871be82f..f9e1a55c57b2 100644 --- a/sdks/standard_expansion_services.yaml +++ b/sdks/standard_expansion_services.yaml @@ -44,6 +44,8 @@ # 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 publish yet + - 'beam:schematransform:org.apache.beam:iceberg_write:v1' # TODO(ahmedabu98): Enable this service in a future PR #- gradle_target: 'sdks:java:io:google-cloud-platform:expansion-service:shadowJar' From 6db699a41b1ab5aeafdc4784d674164dc19b2c6a Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 11 Apr 2024 18:53:21 -0400 Subject: [PATCH 10/60] spotless --- .../apache/beam/sdk/managed/ManagedSchemaTransformProvider.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 f64f71a7259b..ddbc10b5e694 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 @@ -43,7 +43,7 @@ import org.apache.beam.sdk.schemas.utils.YamlUtils; import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.grpc.v1p60p1.com.google.common.base.Predicates; +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.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; From 301e388ea6a229bd91997727f6926746268b2314 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 11 Apr 2024 19:50:09 -0400 Subject: [PATCH 11/60] spotless --- .../apache/beam/sdk/managed/ManagedSchemaTransformProvider.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 ddbc10b5e694..3ea6a4bd8470 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 @@ -43,9 +43,9 @@ import org.apache.beam.sdk.schemas.utils.YamlUtils; 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.Predicates; 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) From 27e5fb0beb33b153e328e97aa8e5277f94cd5fbc Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Mon, 15 Apr 2024 12:49:12 -0400 Subject: [PATCH 12/60] Read schematransform and tests --- .../IcebergReadSchemaTransformProvider.java | 116 ++++++++++++++++++ .../IcebergWriteSchemaTransformProvider.java | 74 ++--------- .../iceberg/SchemaTransformCatalogConfig.java | 59 +++++++++ ...cebergReadSchemaTransformProviderTest.java | 113 +++++++++++++++++ ...ebergWriteSchemaTransformProviderTest.java | 8 +- 5 files changed, 303 insertions(+), 67 deletions(-) create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergReadSchemaTransformProvider.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/SchemaTransformCatalogConfig.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergReadSchemaTransformProviderTest.java diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergReadSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergReadSchemaTransformProvider.java new file mode 100644 index 000000000000..465e55129414 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergReadSchemaTransformProvider.java @@ -0,0 +1,116 @@ +/* + * 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 com.google.auto.service.AutoService; +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.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.io.iceberg.IcebergReadSchemaTransformProvider.Config; + +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; + +import java.util.Collections; +import java.util.List; + +/** + * SchemaTransform implementation for {@link IcebergIO#readTable}. 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 "beam:schematransform:org.apache.beam:iceberg_read:v1"; + } + + @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 SchemaTransformCatalogConfig getCatalogConfig(); + + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setTable(String tables); + + public abstract Builder setCatalogConfig(SchemaTransformCatalogConfig catalogConfig); + + public abstract Config build(); + } + + public void validate() { + getCatalogConfig().validate(); + } + } + + + + private static class IcebergReadSchemaTransform extends SchemaTransform { + private final Config configuration; + + IcebergReadSchemaTransform(Config configuration) { + this.configuration = configuration; + } + + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + SchemaTransformCatalogConfig 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.readTable(catalogBuilder.build(), 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/IcebergWriteSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProvider.java index ec4a946f22d4..a65351c639d5 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProvider.java @@ -101,13 +101,13 @@ public static Builder builder() { public abstract String getTable(); - public abstract CatalogConfig getCatalogConfig(); + public abstract SchemaTransformCatalogConfig getCatalogConfig(); @AutoValue.Builder public abstract static class Builder { public abstract Builder setTable(String tables); - public abstract Builder setCatalogConfig(CatalogConfig catalogConfig); + public abstract Builder setCatalogConfig(SchemaTransformCatalogConfig catalogConfig); public abstract Config build(); } @@ -117,53 +117,9 @@ public void validate() { } } - @DefaultSchema(AutoValueSchema.class) - @AutoValue - public abstract static class CatalogConfig { - public static Builder builder() { - return new AutoValue_IcebergWriteSchemaTransformProvider_CatalogConfig.Builder(); - } - - public abstract String getCatalogName(); - - 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 CatalogConfig build(); - } - - Set validTypes = - 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( - validTypes.contains(Preconditions.checkArgumentNotNull(getCatalogType())), - "Invalid catalog type. Please pick one of %s", - validTypes); - } - } - } @VisibleForTesting - static class IcebergWriteSchemaTransform extends SchemaTransform { + private static class IcebergWriteSchemaTransform extends SchemaTransform { private final Config configuration; IcebergWriteSchemaTransform(Config configuration) { @@ -175,13 +131,11 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { PCollection rows = input.get(INPUT_TAG); - CatalogConfig catalogConfig = configuration.getCatalogConfig(); + SchemaTransformCatalogConfig catalogConfig = configuration.getCatalogConfig(); IcebergCatalogConfig.Builder catalogBuilder = IcebergCatalogConfig.builder() - .setName(catalogConfig.getCatalogName()) - .setIcebergCatalogType(catalogConfig.getCatalogType()) - .setWarehouseLocation(catalogConfig.getWarehouseLocation()); + .setName(catalogConfig.getCatalogName()); if (!Strings.isNullOrEmpty(catalogConfig.getCatalogType())) { catalogBuilder = catalogBuilder.setIcebergCatalogType(catalogConfig.getCatalogType()); @@ -212,17 +166,13 @@ static class SnapshotToRow extends SimpleFunction, Row> { @Override public Row apply(KV input) { Snapshot snapshot = input.getValue(); - Row row = - Row.withSchema(OUTPUT_SCHEMA) - .addValues( - input.getKey(), - snapshot.operation(), - snapshot.summary(), - snapshot.manifestListLocation()) - .build(); - System.out.println("SNAPSHOT: " + snapshot); - System.out.println("ROW: " + row); - return row; + return Row.withSchema(OUTPUT_SCHEMA) + .addValues( + input.getKey(), + snapshot.operation(), + snapshot.summary(), + snapshot.manifestListLocation()) + .build(); } } } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/SchemaTransformCatalogConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/SchemaTransformCatalogConfig.java new file mode 100644 index 000000000000..5d687f0409de --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/SchemaTransformCatalogConfig.java @@ -0,0 +1,59 @@ +package org.apache.beam.io.iceberg; + +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.util.Preconditions; +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; + +import java.util.Set; + +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; + +@DefaultSchema(AutoValueSchema.class) +@AutoValue +public abstract class SchemaTransformCatalogConfig { + public static Builder builder() { + return new AutoValue_SchemaTransformCatalogConfig.Builder(); + } + + public abstract String getCatalogName(); + + 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 SchemaTransformCatalogConfig build(); + } + + Set validTypes = + 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( + validTypes.contains(Preconditions.checkArgumentNotNull(getCatalogType())), + "Invalid catalog type. Please pick one of %s", + validTypes); + } + } +} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergReadSchemaTransformProviderTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergReadSchemaTransformProviderTest.java new file mode 100644 index 000000000000..71b55234b321 --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergReadSchemaTransformProviderTest.java @@ -0,0 +1,113 @@ +package org.apache.beam.io.iceberg; + +import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.schemas.NoSuchSchemaException; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.SchemaRegistry; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.ParDo; +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.TypeDescriptors; +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 java.util.List; +import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; + +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() throws NoSuchSchemaException { + Row catalogConfigRow = + Row.withSchema( + SchemaRegistry.createDefault() + .getSchema(SchemaTransformCatalogConfig.class)) + .withFieldValue("catalogName", "test_name") + .withFieldValue("catalogType", "test_type") + .withFieldValue("catalogImplementation", "testImplementation") + .withFieldValue("warehouseLocation", "test_location") + .build(); + Row transformConfigRow = + Row.withSchema(new IcebergReadSchemaTransformProvider().configurationSchema()) + .withFieldValue("table", "test_table_identifier") + .withFieldValue("catalogConfig", catalogConfigRow) + .build(); + + SchemaTransform transform = new IcebergReadSchemaTransformProvider().from(transformConfigRow); + + System.out.println(transform.getName()); + } + + @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()); + + SchemaTransformCatalogConfig catalogConfig = SchemaTransformCatalogConfig.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(IcebergReadSchemaTransformProvider.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/io/iceberg/IcebergWriteSchemaTransformProviderTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProviderTest.java index 9ae38ecfd843..ac113b6317d3 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProviderTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProviderTest.java @@ -17,7 +17,6 @@ */ package org.apache.beam.io.iceberg; -import static org.apache.beam.io.iceberg.IcebergWriteSchemaTransformProvider.CatalogConfig; import static org.apache.beam.io.iceberg.IcebergWriteSchemaTransformProvider.Config; import static org.apache.beam.io.iceberg.IcebergWriteSchemaTransformProvider.INPUT_TAG; import static org.apache.beam.io.iceberg.IcebergWriteSchemaTransformProvider.OUTPUT_TAG; @@ -65,7 +64,7 @@ public void testBuildTransformWithRow() throws NoSuchSchemaException { Row catalogConfigRow = Row.withSchema( SchemaRegistry.createDefault() - .getSchema(IcebergWriteSchemaTransformProvider.CatalogConfig.class)) + .getSchema(SchemaTransformCatalogConfig.class)) .withFieldValue("catalogName", "test_name") .withFieldValue("catalogType", "test_type") .withFieldValue("catalogImplementation", "testImplementation") @@ -87,7 +86,6 @@ public void testSimpleAppend() { String identifier = "default.table_" + Long.toString(UUID.randomUUID().hashCode(), 16); TableIdentifier tableId = TableIdentifier.parse(identifier); - System.out.println(tableId); // Create a table and add records to it. Table table = warehouse.createTable(tableId, TestFixtures.SCHEMA); @@ -96,7 +94,7 @@ public void testSimpleAppend() { Config.builder() .setTable(identifier) .setCatalogConfig( - CatalogConfig.builder() + SchemaTransformCatalogConfig.builder() .setCatalogName("hadoop") .setCatalogType(CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) .setWarehouseLocation(warehouse.location) @@ -109,7 +107,7 @@ public void testSimpleAppend() { testPipeline .apply( "Records To Add", Create.of(TestFixtures.asRows(TestFixtures.FILE1SNAPSHOT1))) - .setRowSchema(SchemaHelper.convert(TestFixtures.SCHEMA))); + .setRowSchema(SchemaAndRowConversions.icebergSchemaToBeamSchema(TestFixtures.SCHEMA))); PCollection result = input From aa8b1ed7e354203629320ada135454256b0eeb27 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Mon, 15 Apr 2024 12:53:59 -0400 Subject: [PATCH 13/60] pulling in IcebergIO changes; spotless --- .../IcebergReadSchemaTransformProvider.java | 135 ++++++------ .../IcebergWriteSchemaTransformProvider.java | 18 +- .../iceberg/SchemaTransformCatalogConfig.java | 80 ++++--- ...cebergReadSchemaTransformProviderTest.java | 196 ++++++++++-------- ...ebergWriteSchemaTransformProviderTest.java | 11 +- 5 files changed, 229 insertions(+), 211 deletions(-) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergReadSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergReadSchemaTransformProvider.java index 465e55129414..b5d7539ce491 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergReadSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergReadSchemaTransformProvider.java @@ -19,98 +19,99 @@ import com.google.auto.service.AutoService; import com.google.auto.value.AutoValue; +import java.util.Collections; +import java.util.List; +import org.apache.beam.io.iceberg.IcebergReadSchemaTransformProvider.Config; import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; -import org.apache.beam.io.iceberg.IcebergReadSchemaTransformProvider.Config; - 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; -import java.util.Collections; -import java.util.List; - /** - * SchemaTransform implementation for {@link IcebergIO#readTable}. Reads records from Iceberg and outputs a - * {@link org.apache.beam.sdk.values.PCollection} of Beam {@link org.apache.beam.sdk.values.Row}s. + * 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); + 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 "beam:schematransform:org.apache.beam:iceberg_read:v1"; + } + + @DefaultSchema(AutoValueSchema.class) + @AutoValue + public abstract static class Config { + public static Builder builder() { + return new AutoValue_IcebergReadSchemaTransformProvider_Config.Builder(); } - @Override - public String identifier() { - return "beam:schematransform:org.apache.beam:iceberg_read:v1"; - } - - @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 String getTable(); - public abstract SchemaTransformCatalogConfig getCatalogConfig(); + public abstract SchemaTransformCatalogConfig getCatalogConfig(); - @AutoValue.Builder - public abstract static class Builder { - public abstract Builder setTable(String tables); + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setTable(String tables); - public abstract Builder setCatalogConfig(SchemaTransformCatalogConfig catalogConfig); + public abstract Builder setCatalogConfig(SchemaTransformCatalogConfig catalogConfig); - public abstract Config build(); - } - - public void validate() { - getCatalogConfig().validate(); - } + public abstract Config build(); } + public void validate() { + getCatalogConfig().validate(); + } + } + private static class IcebergReadSchemaTransform extends SchemaTransform { + private final Config configuration; - private static class IcebergReadSchemaTransform extends SchemaTransform { - private final Config configuration; - - IcebergReadSchemaTransform(Config configuration) { - this.configuration = configuration; - } - - @Override - public PCollectionRowTuple expand(PCollectionRowTuple input) { - SchemaTransformCatalogConfig 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.readTable(catalogBuilder.build(), TableIdentifier.parse(configuration.getTable()))); + IcebergReadSchemaTransform(Config configuration) { + this.configuration = configuration; + } - return PCollectionRowTuple.of(OUTPUT_TAG, output); - } + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + SchemaTransformCatalogConfig 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/IcebergWriteSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProvider.java index a65351c639d5..e1e51d564d12 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProvider.java @@ -17,13 +17,10 @@ */ package org.apache.beam.io.iceberg; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; - import com.google.auto.service.AutoService; import com.google.auto.value.AutoValue; import java.util.Collections; import java.util.List; -import java.util.Set; import org.apache.beam.io.iceberg.IcebergWriteSchemaTransformProvider.Config; import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.Schema; @@ -33,22 +30,18 @@ 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.util.Preconditions; 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.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; -import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.Snapshot; import org.apache.iceberg.catalog.TableIdentifier; -import org.checkerframework.checker.nullness.qual.Nullable; /** - * SchemaTransform implementation for {@link IcebergIO#writeToDynamicDestinations}. Writes Beam Rows - * to Iceberg and outputs a {@code PCollection} representing snapshots created in the process. + * 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 { @@ -117,7 +110,6 @@ public void validate() { } } - @VisibleForTesting private static class IcebergWriteSchemaTransform extends SchemaTransform { private final Config configuration; @@ -134,8 +126,7 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { SchemaTransformCatalogConfig catalogConfig = configuration.getCatalogConfig(); IcebergCatalogConfig.Builder catalogBuilder = - IcebergCatalogConfig.builder() - .setName(catalogConfig.getCatalogName()); + IcebergCatalogConfig.builder().setName(catalogConfig.getCatalogName()); if (!Strings.isNullOrEmpty(catalogConfig.getCatalogType())) { catalogBuilder = catalogBuilder.setIcebergCatalogType(catalogConfig.getCatalogType()); @@ -149,8 +140,7 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { DynamicDestinations.singleTable(TableIdentifier.parse(configuration.getTable())); IcebergWriteResult result = - rows.apply( - IcebergIO.writeToDynamicDestinations(catalogBuilder.build(), dynamicDestinations)); + rows.apply(IcebergIO.writeRows(catalogBuilder.build()).to(dynamicDestinations)); PCollection snapshots = result diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/SchemaTransformCatalogConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/SchemaTransformCatalogConfig.java index 5d687f0409de..9dd7b0f71554 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/SchemaTransformCatalogConfig.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/SchemaTransformCatalogConfig.java @@ -1,6 +1,26 @@ +/* + * 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.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.annotations.DefaultSchema; import org.apache.beam.sdk.util.Preconditions; @@ -9,51 +29,47 @@ import org.apache.iceberg.CatalogUtil; import org.checkerframework.checker.nullness.qual.Nullable; -import java.util.Set; - -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; - @DefaultSchema(AutoValueSchema.class) @AutoValue public abstract class SchemaTransformCatalogConfig { - public static Builder builder() { - return new AutoValue_SchemaTransformCatalogConfig.Builder(); - } + public static Builder builder() { + return new AutoValue_SchemaTransformCatalogConfig.Builder(); + } - public abstract String getCatalogName(); + public abstract String getCatalogName(); - public abstract @Nullable String getCatalogType(); + public abstract @Nullable String getCatalogType(); - public abstract @Nullable String getCatalogImplementation(); + public abstract @Nullable String getCatalogImplementation(); - public abstract @Nullable String getWarehouseLocation(); + public abstract @Nullable String getWarehouseLocation(); - @AutoValue.Builder - public abstract static class Builder { + @AutoValue.Builder + public abstract static class Builder { - public abstract Builder setCatalogName(String catalogName); + public abstract Builder setCatalogName(String catalogName); - public abstract Builder setCatalogType(String catalogType); + public abstract Builder setCatalogType(String catalogType); - public abstract Builder setCatalogImplementation(String catalogImplementation); + public abstract Builder setCatalogImplementation(String catalogImplementation); - public abstract Builder setWarehouseLocation(String warehouseLocation); + public abstract Builder setWarehouseLocation(String warehouseLocation); - public abstract SchemaTransformCatalogConfig build(); - } + public abstract SchemaTransformCatalogConfig build(); + } + + Set validTypes = + Sets.newHashSet( + CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP, + CatalogUtil.ICEBERG_CATALOG_TYPE_HIVE, + CatalogUtil.ICEBERG_CATALOG_TYPE_REST); - Set validTypes = - 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( - validTypes.contains(Preconditions.checkArgumentNotNull(getCatalogType())), - "Invalid catalog type. Please pick one of %s", - validTypes); - } + public void validate() { + if (Strings.isNullOrEmpty(getCatalogType())) { + checkArgument( + validTypes.contains(Preconditions.checkArgumentNotNull(getCatalogType())), + "Invalid catalog type. Please pick one of %s", + validTypes); } + } } diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergReadSchemaTransformProviderTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergReadSchemaTransformProviderTest.java index 71b55234b321..9b73743fa3e7 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergReadSchemaTransformProviderTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergReadSchemaTransformProviderTest.java @@ -1,18 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.beam.io.iceberg; -import org.apache.beam.sdk.coders.RowCoder; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; + +import java.util.List; +import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.Stream; 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.transforms.SchemaTransform; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.MapElements; -import org.apache.beam.sdk.transforms.ParDo; 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.TypeDescriptors; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; @@ -21,93 +41,87 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.util.List; -import java.util.UUID; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.containsInAnyOrder; - 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() throws NoSuchSchemaException { - Row catalogConfigRow = - Row.withSchema( - SchemaRegistry.createDefault() - .getSchema(SchemaTransformCatalogConfig.class)) - .withFieldValue("catalogName", "test_name") - .withFieldValue("catalogType", "test_type") - .withFieldValue("catalogImplementation", "testImplementation") - .withFieldValue("warehouseLocation", "test_location") - .build(); - Row transformConfigRow = - Row.withSchema(new IcebergReadSchemaTransformProvider().configurationSchema()) - .withFieldValue("table", "test_table_identifier") - .withFieldValue("catalogConfig", catalogConfigRow) - .build(); - - SchemaTransform transform = new IcebergReadSchemaTransformProvider().from(transformConfigRow); - - System.out.println(transform.getName()); - } - - @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()); - - SchemaTransformCatalogConfig catalogConfig = SchemaTransformCatalogConfig.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(IcebergReadSchemaTransformProvider.OUTPUT_TAG); - - PAssert.that(output) - .satisfies( - (Iterable rows) -> { - assertThat(rows, containsInAnyOrder(expectedRows.toArray())); - return null; - }); - - testPipeline.run(); - } + @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() throws NoSuchSchemaException { + Row catalogConfigRow = + Row.withSchema(SchemaRegistry.createDefault().getSchema(SchemaTransformCatalogConfig.class)) + .withFieldValue("catalogName", "test_name") + .withFieldValue("catalogType", "test_type") + .withFieldValue("catalogImplementation", "testImplementation") + .withFieldValue("warehouseLocation", "test_location") + .build(); + Row transformConfigRow = + Row.withSchema(new IcebergReadSchemaTransformProvider().configurationSchema()) + .withFieldValue("table", "test_table_identifier") + .withFieldValue("catalogConfig", 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()); + + SchemaTransformCatalogConfig catalogConfig = + SchemaTransformCatalogConfig.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(IcebergReadSchemaTransformProvider.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/io/iceberg/IcebergWriteSchemaTransformProviderTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProviderTest.java index ac113b6317d3..639fbdd331de 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProviderTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProviderTest.java @@ -62,9 +62,7 @@ public class IcebergWriteSchemaTransformProviderTest { @Test public void testBuildTransformWithRow() throws NoSuchSchemaException { Row catalogConfigRow = - Row.withSchema( - SchemaRegistry.createDefault() - .getSchema(SchemaTransformCatalogConfig.class)) + Row.withSchema(SchemaRegistry.createDefault().getSchema(SchemaTransformCatalogConfig.class)) .withFieldValue("catalogName", "test_name") .withFieldValue("catalogType", "test_type") .withFieldValue("catalogImplementation", "testImplementation") @@ -76,9 +74,7 @@ public void testBuildTransformWithRow() throws NoSuchSchemaException { .withFieldValue("catalogConfig", catalogConfigRow) .build(); - SchemaTransform transform = new IcebergWriteSchemaTransformProvider().from(transformConfigRow); - - System.out.println(transform.getName()); + new IcebergWriteSchemaTransformProvider().from(transformConfigRow); } @Test @@ -107,7 +103,8 @@ public void testSimpleAppend() { testPipeline .apply( "Records To Add", Create.of(TestFixtures.asRows(TestFixtures.FILE1SNAPSHOT1))) - .setRowSchema(SchemaAndRowConversions.icebergSchemaToBeamSchema(TestFixtures.SCHEMA))); + .setRowSchema( + SchemaAndRowConversions.icebergSchemaToBeamSchema(TestFixtures.SCHEMA))); PCollection result = input From 067406961db9d86cda5a3adbd215ba7902bac711 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 16 Apr 2024 18:42:43 -0400 Subject: [PATCH 14/60] icebergio translation; managed translation; protos --- .../pipeline/v1/schema_aware_transforms.proto | 73 ++++++ .../construction/PTransformTranslation.java | 5 +- sdks/java/io/iceberg/build.gradle | 1 + .../beam/io/iceberg/IcebergIOTranslation.java | 73 +++--- .../io/iceberg/IcebergIOTranslationTest.java | 152 +++++++++++- ...cebergReadSchemaTransformProviderTest.java | 1 - ...ebergWriteSchemaTransformProviderTest.java | 1 - sdks/java/managed/build.gradle | 3 +- .../org/apache/beam/sdk/managed/Managed.java | 24 +- .../ManagedSchemaTransformProvider.java | 100 ++++---- .../ManagedSchemaTransformTranslation.java | 123 ++++++++++ .../managed/TestSchemaTransformProvider.java | 11 + .../ManagedSchemaTransformProviderTest.java | 50 +++- ...ManagedSchemaTransformTranslationTest.java | 217 ++++++++++++++++++ 14 files changed, 716 insertions(+), 118 deletions(-) create mode 100644 model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema_aware_transforms.proto create mode 100644 sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedSchemaTransformTranslation.java rename sdks/java/managed/src/{test => main}/java/org/apache/beam/sdk/managed/TestSchemaTransformProvider.java (92%) create mode 100644 sdks/java/managed/src/test/java/org/apache/beam/sdk/managed/ManagedSchemaTransformTranslationTest.java diff --git a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema_aware_transforms.proto b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema_aware_transforms.proto new file mode 100644 index 000000000000..b553fa25b37f --- /dev/null +++ b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema_aware_transforms.proto @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +// Protocol Buffers describing available Schema-Aware transforms. +// These are transforms that can be configured using Beam Schema-compatible parameters. +// Runners can override these transforms with a native implementation. + +syntax = "proto3"; + +package org.apache.beam.model.pipeline.v1; + +option go_package = "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1;pipeline_v1"; +option java_package = "org.apache.beam.model.pipeline.v1"; +option java_outer_classname = "SchemaAwareTransforms"; + +import "org/apache/beam/model/pipeline/v1/beam_runner_api.proto"; +import "org/apache/beam/model/pipeline/v1/schema.proto"; + + +message StandardSchemaAwareTransforms { + // Payload for all of these: SchemaAwareTransformPayload + enum IOs { + ICEBERG_READ = 0 [(beam_urn) = "beam:transform:iceberg_read:v1"]; + ICEBERG_WRITE = 1 [(beam_urn) = "beam:transform:iceberg_write:v1"]; + } + enum Managed { + // Payload: ManagedTransformPayload + MANAGED = 0 [(beam_urn) = "beam:transform:managed:v1"]; + } +} + + +// 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. +message SchemaAwareTransformPayload { + // The transform's configuration schema + Schema expansion_schema = 1; + // The configuration used to build this transform. + // Must be compatible with the configuration schema, and decodable via beam:coder:row:v1. + bytes expansion_payload = 2; +} + +// Payload for a Managed transform +// This can be used by runners that wish to override an underlying transform +// with a different implementation. +message ManagedTransformPayload { + // The underlying transform's URN. + string underlying_transform_urn = 1; + // The underlying transform's configuration Schema. + Schema expansion_schema = 2; + // The configuration used to build the underlying transform. + // Must be compatible with the configuration schema, and decodable via beam:coder:row:v1. + bytes expansion_payload = 3; + // The underlying transform's configuration, represented as a YAML string. + string yaml_config = 4; +} 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..1d321162b91e 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 @@ -35,6 +35,7 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.StandardPTransforms; import org.apache.beam.model.pipeline.v1.RunnerApi.StandardPTransforms.CombineComponents; import org.apache.beam.model.pipeline.v1.RunnerApi.StandardPTransforms.SplittableParDoComponents; +import org.apache.beam.model.pipeline.v1.SchemaAwareTransforms.StandardSchemaAwareTransforms; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.coders.RowCoder; @@ -92,6 +93,7 @@ public class PTransformTranslation { public static final String MAP_WINDOWS_TRANSFORM_URN = "beam:transform:map_windows:v1"; public static final String MERGE_WINDOWS_TRANSFORM_URN = "beam:transform:merge_windows:v1"; public static final String TO_STRING_TRANSFORM_URN = "beam:transform:to_string:v1"; + public static final String MANAGED_TRANSFORM_URN = "beam:transform:managed:v1"; // Required runner implemented transforms. These transforms should never specify an environment. public static final ImmutableSet RUNNER_IMPLEMENTED_TRANSFORMS = @@ -182,6 +184,7 @@ public class PTransformTranslation { MAP_WINDOWS_TRANSFORM_URN.equals(getUrn(StandardPTransforms.Primitives.MAP_WINDOWS))); checkState( MERGE_WINDOWS_TRANSFORM_URN.equals(getUrn(StandardPTransforms.Primitives.MERGE_WINDOWS))); + checkState(MANAGED_TRANSFORM_URN.equals(getUrn(StandardSchemaAwareTransforms.Managed.MANAGED))); // DeprecatedPrimitives checkState(READ_TRANSFORM_URN.equals(getUrn(StandardPTransforms.DeprecatedPrimitives.READ))); @@ -276,7 +279,7 @@ static RunnerApi.PTransform toProto( * *

    Does not register the {@code appliedPTransform} within the provided {@link SdkComponents}. */ - static RunnerApi.PTransform toProto( + public static RunnerApi.PTransform toProto( AppliedPTransform appliedPTransform, SdkComponents components) throws IOException { return toProto(appliedPTransform, Collections.emptyList(), components); } diff --git a/sdks/java/io/iceberg/build.gradle b/sdks/java/io/iceberg/build.gradle index caa72bef7536..7feaeb8c4790 100644 --- a/sdks/java/io/iceberg/build.gradle +++ b/sdks/java/io/iceberg/build.gradle @@ -42,6 +42,7 @@ def hive_version = "3.1.3" dependencies { implementation library.java.vendored_guava_32_1_2_jre + implementation project(path: ":model:pipeline", configuration: "shadow") implementation project(path: ":sdks:java:core", configuration: "shadow") implementation library.java.slf4j_api implementation library.java.joda_time diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIOTranslation.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIOTranslation.java index 5dbc2a08f23b..48b7ee244099 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIOTranslation.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIOTranslation.java @@ -23,19 +23,22 @@ import static org.apache.beam.sdk.util.construction.TransformUpgrader.toByteArray; import com.google.auto.service.AutoService; +import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InvalidClassException; import java.util.Arrays; -import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.stream.Collectors; import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; +import org.apache.beam.model.pipeline.v1.SchemaApi; +import org.apache.beam.model.pipeline.v1.SchemaAwareTransforms.SchemaAwareTransformPayload; +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.Schema; -import org.apache.beam.sdk.schemas.Schema.FieldType; +import org.apache.beam.sdk.schemas.SchemaTranslation; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.construction.PTransformTranslation.TransformPayloadTranslator; import org.apache.beam.sdk.util.construction.SdkComponents; @@ -50,10 +53,10 @@ public class IcebergIOTranslation { static class IcebergIOReadTranslator implements TransformPayloadTranslator { - static Schema schema = + static Schema READ_SCHEMA = Schema.builder() .addByteArrayField("catalog_config") - .addNullableArrayField("table_identifier", FieldType.STRING) + .addNullableStringField("table_identifier") .build(); public static final String ICEBERG_READ_TRANSFORM_URN = @@ -68,9 +71,20 @@ public String getUrn() { public @Nullable FunctionSpec translate( AppliedPTransform application, SdkComponents components) throws IOException { - // Setting an empty payload since Iceberg transform payload is not actually used by runners - // currently. - return FunctionSpec.newBuilder().setUrn(getUrn()).setPayload(ByteString.empty()).build(); + SchemaApi.Schema expansionSchema = SchemaTranslation.schemaToProto(READ_SCHEMA, true); + Row configRow = toConfigRow(application.getTransform()); + ByteArrayOutputStream os = new ByteArrayOutputStream(); + RowCoder.of(READ_SCHEMA).encode(configRow, os); + + return FunctionSpec.newBuilder() + .setUrn(getUrn()) + .setPayload( + SchemaAwareTransformPayload.newBuilder() + .setExpansionSchema(expansionSchema) + .setExpansionPayload(ByteString.copyFrom(os.toByteArray())) + .build() + .toByteString()) + .build(); } @Override @@ -86,10 +100,10 @@ public Row toConfigRow(ReadRows transform) { List identifierParts = Arrays.stream(identifier.namespace().levels()).collect(Collectors.toList()); identifierParts.add(identifier.name()); - fieldValues.put("table_identifier", identifierParts); + fieldValues.put("table_identifier", String.join(".", identifierParts)); } - return Row.withSchema(schema).withFieldValues(fieldValues).build(); + return Row.withSchema(READ_SCHEMA).withFieldValues(fieldValues).build(); } @Override @@ -101,11 +115,9 @@ public ReadRows fromConfigRow(Row configRow, PipelineOptions options) { if (catalogBytes != null) { builder = builder.setCatalogConfig((IcebergCatalogConfig) fromByteArray(catalogBytes)); } - Collection tableIdentifierParts = configRow.getArray("table_identifier"); - if (tableIdentifierParts != null) { - builder = - builder.setTableIdentifier( - TableIdentifier.parse(String.join(".", tableIdentifierParts))); + String tableIdentifier = configRow.getString("table_identifier"); + if (tableIdentifier != null) { + builder = builder.setTableIdentifier(TableIdentifier.parse(tableIdentifier)); } return builder.build(); } catch (InvalidClassException e) { @@ -130,10 +142,10 @@ public static class ReadRegistrar implements TransformPayloadTranslatorRegistrar static class IcebergIOWriteTranslator implements TransformPayloadTranslator { - static Schema schema = + static final Schema WRITE_SCHEMA = Schema.builder() .addByteArrayField("catalog_config") - .addNullableArrayField("table_identifier", FieldType.STRING) + .addNullableStringField("table_identifier") .addNullableByteArrayField("dynamic_destinations") .build(); @@ -149,9 +161,20 @@ public String getUrn() { public @Nullable FunctionSpec translate( AppliedPTransform application, SdkComponents components) throws IOException { - // Setting an empty payload since Iceberg transform payload is not actually used by runners - // currently. - return FunctionSpec.newBuilder().setUrn(getUrn()).setPayload(ByteString.empty()).build(); + SchemaApi.Schema expansionSchema = SchemaTranslation.schemaToProto(WRITE_SCHEMA, true); + Row configRow = toConfigRow(application.getTransform()); + ByteArrayOutputStream os = new ByteArrayOutputStream(); + RowCoder.of(WRITE_SCHEMA).encode(configRow, os); + + return FunctionSpec.newBuilder() + .setUrn(getUrn()) + .setPayload( + SchemaAwareTransformPayload.newBuilder() + .setExpansionSchema(expansionSchema) + .setExpansionPayload(ByteString.copyFrom(os.toByteArray())) + .build() + .toByteString()) + .build(); } @Override @@ -167,13 +190,13 @@ public Row toConfigRow(WriteRows transform) { List identifierParts = Arrays.stream(identifier.namespace().levels()).collect(Collectors.toList()); identifierParts.add(identifier.name()); - fieldValues.put("table_identifier", identifierParts); + fieldValues.put("table_identifier", String.join(".", identifierParts)); } if (transform.getDynamicDestinations() != null) { fieldValues.put("dynamic_destinations", toByteArray(transform.getDynamicDestinations())); } - return Row.withSchema(schema).withFieldValues(fieldValues).build(); + return Row.withSchema(WRITE_SCHEMA).withFieldValues(fieldValues).build(); } @Override @@ -185,11 +208,9 @@ public WriteRows fromConfigRow(Row configRow, PipelineOptions options) { if (catalogBytes != null) { builder = builder.setCatalogConfig((IcebergCatalogConfig) fromByteArray(catalogBytes)); } - Collection tableIdentifierParts = configRow.getArray("table_identifier"); - if (tableIdentifierParts != null) { - builder = - builder.setTableIdentifier( - TableIdentifier.parse(String.join(".", tableIdentifierParts))); + String tableIdentifier = configRow.getString("table_identifier"); + if (tableIdentifier != null) { + builder = builder.setTableIdentifier(TableIdentifier.parse(tableIdentifier)); } byte[] dynamicDestinationsBytes = configRow.getBytes("dynamic_destinations"); if (dynamicDestinationsBytes != null) { diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergIOTranslationTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergIOTranslationTest.java index 94692a78804f..930fc00f466d 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergIOTranslationTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergIOTranslationTest.java @@ -17,15 +17,26 @@ */ package org.apache.beam.io.iceberg; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; +import static org.apache.beam.io.iceberg.IcebergIOTranslation.IcebergIOReadTranslator.READ_SCHEMA; +import static org.apache.beam.io.iceberg.IcebergIOTranslation.IcebergIOWriteTranslator.WRITE_SCHEMA; +import static org.junit.Assert.*; import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.UUID; import java.util.stream.Collectors; +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.model.pipeline.v1.SchemaAwareTransforms; +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.PipelineTranslation; +import org.apache.beam.sdk.util.construction.TransformUpgrader; +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.collect.ImmutableMap; import org.apache.iceberg.CatalogUtil; @@ -59,7 +70,7 @@ public class IcebergIOTranslationTest { public transient TestDataWarehouse warehouse = new TestDataWarehouse(TEMPORARY_FOLDER, "default"); @Test - public void testReCreateWriteTransformFromRowTable() { + public void testReCreateWriteTransformFromRow() { // setting a subset of fields here. IcebergCatalogConfig config = IcebergCatalogConfig.builder() @@ -88,6 +99,73 @@ public void testReCreateWriteTransformFromRowTable() { warehouse.location, writeTransformFromRow.getCatalogConfig().getWarehouseLocation()); } + @Test + public void testWriteTransformProtoTranslation() 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); + + IcebergCatalogConfig catalogConfig = + IcebergCatalogConfig.builder() + .setName("test_catalog") + .setIcebergCatalogType(CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .setWarehouseLocation(warehouse.location) + .build(); + String identifier = "default.table_" + Long.toString(UUID.randomUUID().hashCode(), 16); + + IcebergIO.WriteRows writeTransform = + IcebergIO.writeRows(catalogConfig).to(TableIdentifier.parse(identifier)); + + input.apply(writeTransform); + + // Then translate the pipeline to a proto and extract IcebergIO.WriteRows proto + RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p); + List writeTransformProto = + pipelineProto.getComponents().getTransformsMap().values().stream() + .filter( + tr -> + tr.getSpec() + .getUrn() + .equals( + IcebergIOTranslation.IcebergIOWriteTranslator + .ICEBERG_WRITE_TRANSFORM_URN)) + .collect(Collectors.toList()); + assertEquals(1, writeTransformProto.size()); + RunnerApi.FunctionSpec spec = writeTransformProto.get(0).getSpec(); + + // Check that the proto contains correct values + SchemaAwareTransforms.SchemaAwareTransformPayload payload = + SchemaAwareTransforms.SchemaAwareTransformPayload.parseFrom(spec.getPayload()); + Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getExpansionSchema()); + assertEquals(WRITE_SCHEMA, schemaFromSpec); + Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getExpansionPayload().newInput()); + Row expectedRow = + Row.withSchema(WRITE_SCHEMA) + .withFieldValue("table_identifier", identifier) + .withFieldValue("catalog_config", TransformUpgrader.toByteArray(catalogConfig)) + .withFieldValue("dynamic_destinations", null) + .build(); + assertEquals(expectedRow, rowFromSpec); + + // Use the information in the proto to recreate the IcebergIO.WriteRows transform + IcebergIOTranslation.IcebergIOWriteTranslator translator = + new IcebergIOTranslation.IcebergIOWriteTranslator(); + IcebergIO.WriteRows writeTransformFromSpec = + translator.fromConfigRow(rowFromSpec, PipelineOptionsFactory.create()); + + assertEquals(TableIdentifier.parse(identifier), writeTransformFromSpec.getTableIdentifier()); + assertEquals(catalogConfig, writeTransformFromSpec.getCatalogConfig()); + assertNull(writeTransformFromSpec.getDynamicDestinations()); + } + @Test public void testWriteTransformRowIncludesAllFields() { List getMethodNames = @@ -119,14 +197,12 @@ public void testWriteTransformRowIncludesAllFields() { + fieldName + " was not found in the transform schema defined in " + "IcebergIOTranslation.IcebergIOWriteTranslator.", - IcebergIOTranslation.IcebergIOWriteTranslator.schema - .getFieldNames() - .contains(fieldName)); + WRITE_SCHEMA.getFieldNames().contains(fieldName)); }); } @Test - public void testReCreateReadTransformFromRowTable() { + public void testReCreateReadTransformFromRow() { // setting a subset of fields here. IcebergCatalogConfig config = IcebergCatalogConfig.builder() @@ -155,6 +231,64 @@ public void testReCreateReadTransformFromRowTable() { warehouse.location, readTransformFromRow.getCatalogConfig().getWarehouseLocation()); } + @Test + public void testReadTransformProtoTranslation() throws Exception { + // First build a pipeline + Pipeline p = Pipeline.create(); + + IcebergCatalogConfig catalogConfig = + IcebergCatalogConfig.builder() + .setName("test_catalog") + .setIcebergCatalogType(CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .setWarehouseLocation(warehouse.location) + .build(); + String identifier = "default.table_" + Long.toString(UUID.randomUUID().hashCode(), 16); + TableIdentifier tableId = TableIdentifier.parse(identifier); + + warehouse.createTable(tableId, TestFixtures.SCHEMA); + + IcebergIO.ReadRows readTransform = IcebergIO.readRows(catalogConfig).from(tableId); + + p.apply(readTransform); + + // Then translate the pipeline to a proto and extract IcebergIO.ReadRows proto + RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p); + List readTransformProto = + pipelineProto.getComponents().getTransformsMap().values().stream() + .filter( + tr -> + tr.getSpec() + .getUrn() + .equals( + IcebergIOTranslation.IcebergIOReadTranslator + .ICEBERG_READ_TRANSFORM_URN)) + .collect(Collectors.toList()); + assertEquals(1, readTransformProto.size()); + RunnerApi.FunctionSpec spec = readTransformProto.get(0).getSpec(); + + // Check that the proto contains correct values + SchemaAwareTransforms.SchemaAwareTransformPayload payload = + SchemaAwareTransforms.SchemaAwareTransformPayload.parseFrom(spec.getPayload()); + Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getExpansionSchema()); + assertEquals(READ_SCHEMA, schemaFromSpec); + Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getExpansionPayload().newInput()); + Row expectedRow = + Row.withSchema(READ_SCHEMA) + .withFieldValue("table_identifier", identifier) + .withFieldValue("catalog_config", TransformUpgrader.toByteArray(catalogConfig)) + .build(); + assertEquals(expectedRow, rowFromSpec); + + // Use the information in the proto to recreate the IcebergIO.ReadRows transform + IcebergIOTranslation.IcebergIOReadTranslator translator = + new IcebergIOTranslation.IcebergIOReadTranslator(); + IcebergIO.ReadRows readTransformFromSpec = + translator.fromConfigRow(rowFromSpec, PipelineOptionsFactory.create()); + + assertEquals(tableId, readTransformFromSpec.getTableIdentifier()); + assertEquals(catalogConfig, readTransformFromSpec.getCatalogConfig()); + } + @Test public void testReadTransformRowIncludesAllFields() { List getMethodNames = @@ -186,7 +320,7 @@ public void testReadTransformRowIncludesAllFields() { + fieldName + " was not found in the transform schema defined in " + "IcebergIOTranslation.IcebergIOReadTranslator.", - IcebergIOTranslation.IcebergIOReadTranslator.schema + IcebergIOTranslation.IcebergIOReadTranslator.READ_SCHEMA .getFieldNames() .contains(fieldName)); }); diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergReadSchemaTransformProviderTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergReadSchemaTransformProviderTest.java index 9b73743fa3e7..9be7f1797cd9 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergReadSchemaTransformProviderTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergReadSchemaTransformProviderTest.java @@ -27,7 +27,6 @@ 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.transforms.SchemaTransform; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollection; diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProviderTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProviderTest.java index 639fbdd331de..3eff5cae402d 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProviderTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProviderTest.java @@ -27,7 +27,6 @@ import java.util.UUID; import org.apache.beam.sdk.schemas.NoSuchSchemaException; import org.apache.beam.sdk.schemas.SchemaRegistry; -import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; diff --git a/sdks/java/managed/build.gradle b/sdks/java/managed/build.gradle index 88e537d66f8c..4b1da0599838 100644 --- a/sdks/java/managed/build.gradle +++ b/sdks/java/managed/build.gradle @@ -28,8 +28,9 @@ ext.summary = """Library that provides managed IOs.""" dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation project(path: ":model:pipeline", configuration: "shadow") + implementation library.java.vendored_grpc_1_60_1 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 d746de306f55..b29aba68443e 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 @@ -26,7 +26,6 @@ import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; import org.apache.beam.sdk.schemas.utils.YamlUtils; 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.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; @@ -179,32 +178,15 @@ ManagedTransform withSupportedIdentifiers(List supportedIdentifiers) { @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { - ManagedSchemaTransformProvider provider = - new ManagedSchemaTransformProvider(getSupportedIdentifiers()); - - SchemaTransformProvider underlyingTransformProvider = - provider.getAllProviders().get(getIdentifier()); - if (underlyingTransformProvider == null) { - throw new RuntimeException( - String.format( - "Could not find transform with identifier %s, or it may not be supported.", - getIdentifier())); - } - - Row transformConfigRow = - getConfig() != null - ? YamlUtils.toBeamRow( - getConfig(), underlyingTransformProvider.configurationSchema(), true) - : null; - ManagedSchemaTransformProvider.ManagedConfig managedConfig = ManagedSchemaTransformProvider.ManagedConfig.builder() .setTransformIdentifier(getIdentifier()) - .setConfig(transformConfigRow) + .setConfig(YamlUtils.yamlStringFromMap(getConfig())) .setConfigUrl(getConfigUrl()) .build(); - SchemaTransform underlyingTransform = provider.from(managedConfig); + SchemaTransform underlyingTransform = + new ManagedSchemaTransformProvider(getSupportedIdentifiers()).from(managedConfig); return input.apply(underlyingTransform); } 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 3ea6a4bd8470..c188b4fb42d3 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 @@ -54,14 +54,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)) { @@ -70,19 +70,21 @@ 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) @AutoValue @VisibleForTesting - abstract static class ManagedConfig { + public abstract static class ManagedConfig { public static Builder builder() { return new AutoValue_ManagedSchemaTransformProvider_ManagedConfig.Builder(); } @@ -93,8 +95,8 @@ public static Builder builder() { @SchemaFieldDescription("URL path to the YAML config file used to build the underlying IO.") public abstract @Nullable String getConfigUrl(); - @SchemaFieldDescription("Row config used to build the underlying IO.") - public abstract @Nullable Row getConfig(); + @SchemaFieldDescription("YAML string config used to build the underlying IO.") + public abstract @Nullable String getConfig(); @AutoValue.Builder public abstract static class Builder { @@ -102,19 +104,36 @@ public abstract static class Builder { public abstract Builder setConfigUrl(@Nullable String configUrl); - public abstract Builder setConfig(@Nullable Row config); + public abstract Builder setConfig(@Nullable String yamlConfig); public abstract ManagedConfig build(); } protected void validate() { - boolean configExists = getConfig() != null; + boolean configExists = !Strings.isNullOrEmpty(getConfig()); boolean configUrlExists = !Strings.isNullOrEmpty(getConfigUrl()); List configs = Arrays.asList(configExists, configUrlExists); checkArgument( 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 @@ -126,29 +145,28 @@ 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 { + public static class ManagedSchemaTransform extends SchemaTransform { private final Row transformConfig; + private final ManagedConfig managedConfig; private final SchemaTransformProvider underlyingTransformProvider; ManagedSchemaTransform( - Row transformConfig, SchemaTransformProvider underlyingTransformProvider) { + ManagedConfig managedConfig, SchemaTransformProvider underlyingTransformProvider) { + // parse config before expansion to check if it matches underlying transform's config schema + Schema transformConfigSchema = underlyingTransformProvider.configurationSchema(); + Row transformConfig; + try { + transformConfig = getRowConfig(managedConfig, transformConfigSchema); + } catch (Exception e) { + throw new IllegalArgumentException( + "Encountered an error when retrieving a Row configuration", e); + } + this.transformConfig = transformConfig; + this.managedConfig = managedConfig; this.underlyingTransformProvider = underlyingTransformProvider; } @@ -158,32 +176,18 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { return input.apply(underlyingTransform); } + + public ManagedConfig getManagedConfig() { + return this.managedConfig; + } } @VisibleForTesting static Row getRowConfig(ManagedConfig config, Schema transformSchema) { - Row transformRowConfig = config.getConfig(); - // Attempt to construct a Row config from a YAML file - if (transformRowConfig == null) { - try { - MatchResult.Metadata fileMetaData = - FileSystems.matchSingleFileSpec(Preconditions.checkNotNull(config.getConfigUrl())); - ByteBuffer buffer = ByteBuffer.allocate((int) fileMetaData.sizeBytes()); - FileSystems.open(fileMetaData.resourceId()).read(buffer); - String yamlConfig = new String(buffer.array(), StandardCharsets.UTF_8); - transformRowConfig = YamlUtils.toBeamRow(yamlConfig, transformSchema, true); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - // If our config is still null (perhaps the underlying transform doesn't have any parameters), - // default to an empty row. - if (transformRowConfig == null) { - transformRowConfig = Row.nullRow(transformSchema); - } - - return transformRowConfig; + Row configRow = YamlUtils.toBeamRow(config.resolveUnderlyingConfig(), transformSchema, true); + // If our config is still null (perhaps the underlying transform doesn't have any required + // parameters), then return an empty row. + return configRow != null ? configRow : Row.nullRow(transformSchema); } Map getAllProviders() { 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..c9d63a9d435e --- /dev/null +++ b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedSchemaTransformTranslation.java @@ -0,0 +1,123 @@ +/* + * 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.RunnerApi.FunctionSpec; +import static org.apache.beam.sdk.managed.ManagedSchemaTransformProvider.ManagedConfig; +import static org.apache.beam.sdk.managed.ManagedSchemaTransformProvider.ManagedSchemaTransform; +import static org.apache.beam.sdk.util.construction.PTransformTranslation.TransformPayloadTranslator; + +import com.google.auto.service.AutoService; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.Map; +import org.apache.beam.model.pipeline.v1.SchemaApi; +import org.apache.beam.model.pipeline.v1.SchemaAwareTransforms.ManagedTransformPayload; +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.NoSuchSchemaException; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.SchemaRegistry; +import org.apache.beam.sdk.schemas.SchemaTranslation; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.util.construction.TransformPayloadTranslatorRegistrar; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.checkerframework.checker.nullness.qual.Nullable; + +public class ManagedSchemaTransformTranslation { + static class ManagedSchemaTransformTranslator + implements TransformPayloadTranslator { + private final ManagedSchemaTransformProvider provider; + static final Schema SCHEMA; + + static { + try { + SCHEMA = SchemaRegistry.createDefault().getSchema(ManagedConfig.class); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e); + } + } + + public ManagedSchemaTransformTranslator() { + provider = new ManagedSchemaTransformProvider(null); + } + + @Override + public String getUrn() { + return provider.identifier(); + } + + @Override + @SuppressWarnings("argument") + public @Nullable FunctionSpec translate( + AppliedPTransform application, SdkComponents components) + throws IOException { + SchemaApi.Schema expansionSchema = SchemaTranslation.schemaToProto(SCHEMA, true); + ManagedConfig managedConfig = application.getTransform().getManagedConfig(); + Row configRow = toConfigRow(application.getTransform()); + ByteArrayOutputStream os = new ByteArrayOutputStream(); + RowCoder.of(SCHEMA).encode(configRow, os); + + return FunctionSpec.newBuilder() + .setUrn(getUrn()) + .setPayload( + ManagedTransformPayload.newBuilder() + .setUnderlyingTransformUrn(managedConfig.getTransformIdentifier()) + .setYamlConfig(managedConfig.resolveUnderlyingConfig()) + .setExpansionSchema(expansionSchema) + .setExpansionPayload(ByteString.copyFrom(os.toByteArray())) + .build() + .toByteString()) + .build(); + } + + @Override + @SuppressWarnings("nullable") + public Row toConfigRow(ManagedSchemaTransform transform) { + ManagedConfig config = transform.getManagedConfig(); + try { + return SchemaRegistry.createDefault().getToRowFunction(ManagedConfig.class).apply(config); + } catch (NoSuchSchemaException e) { + throw new RuntimeException("Encountered error when finding schema for ManagedConfig"); + } + } + + @Override + public ManagedSchemaTransform fromConfigRow(Row configRow, PipelineOptions options) { + return (ManagedSchemaTransform) provider.from(configRow); + } + } + + @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/test/java/org/apache/beam/sdk/managed/TestSchemaTransformProvider.java b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/TestSchemaTransformProvider.java similarity index 92% 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/TestSchemaTransformProvider.java index 136d98d468d0..bc47fb9be763 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/TestSchemaTransformProvider.java @@ -20,7 +20,9 @@ import com.google.auto.service.AutoService; import com.google.auto.value.AutoValue; 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; @@ -36,6 +38,15 @@ public class TestSchemaTransformProvider extends TypedSchemaTransformProvider { static final String IDENTIFIER = "beam:schematransform:org.apache.beam:test_transform:v1"; + static final Schema SCHEMA; + + static { + try { + SCHEMA = SchemaRegistry.createDefault().getSchema(Config.class); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e); + } + } @DefaultSchema(AutoValueSchema.class) @AutoValue 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 32c858e6497e..7c0b6b8a4667 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 @@ -49,23 +49,24 @@ public void testFailWhenNoConfigSpecified() { } @Test - public void testGetConfigRow() { - Schema underlyingTransformSchema = new TestSchemaTransformProvider().configurationSchema(); - Row configRow = - Row.withSchema(underlyingTransformSchema) - .withFieldValue("extraString", "abc") - .withFieldValue("extraInteger", 123) - .build(); + public void testGetConfigRowFromYamlString() { + String yamlString = "extra_string: abc\n" + "extra_integer: 123"; ManagedConfig config = ManagedConfig.builder() .setTransformIdentifier(TestSchemaTransformProvider.IDENTIFIER) - .setConfig(configRow) + .setConfig(yamlString) + .build(); + + Row expectedRow = + Row.withSchema(TestSchemaTransformProvider.SCHEMA) + .withFieldValue("extraString", "abc") + .withFieldValue("extraInteger", 123) .build(); Row returnedRow = - ManagedSchemaTransformProvider.getRowConfig(config, underlyingTransformSchema); + ManagedSchemaTransformProvider.getRowConfig(config, TestSchemaTransformProvider.SCHEMA); - assertEquals(configRow, returnedRow); + assertEquals(expectedRow, returnedRow); } @Test @@ -92,6 +93,35 @@ public void testGetConfigRowFromYamlFile() 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..5dea2f8a4e05 --- /dev/null +++ b/sdks/java/managed/src/test/java/org/apache/beam/sdk/managed/ManagedSchemaTransformTranslationTest.java @@ -0,0 +1,217 @@ +/* + * 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.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 static org.junit.Assert.assertTrue; + +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.model.pipeline.v1.SchemaAwareTransforms; +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.schemas.utils.YamlUtils; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +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.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.junit.Test; + +public class ManagedSchemaTransformTranslationTest { + // A mapping from ManagedConfig builder methods to the corresponding schema fields in + // ManagedSchemaTransformTranslation. + static final Map MANAGED_TRANSFORM_SCHEMA_MAPPING = + ImmutableMap.builder() + .put("getTransformIdentifier", "transformIdentifier") + .put("getConfigUrl", "configUrl") + .put("getConfig", "config") + .build(); + + @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(); + + ManagedSchemaTransformProvider provider = new ManagedSchemaTransformProvider(null); + 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(); + + ManagedSchemaTransformProvider provider = new ManagedSchemaTransformProvider(null); + 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 testManagedTransformRowIncludesAllFields() { + List getMethodNames = + Arrays.stream(ManagedConfig.class.getDeclaredMethods()) + .map(method -> method.getName()) + .filter(methodName -> methodName.startsWith("get")) + .collect(Collectors.toList()); + + // Just to make sure that this does not pass trivially. + assertTrue(getMethodNames.size() > 0); + + for (String getMethodName : getMethodNames) { + assertTrue( + "Method " + + getMethodName + + " will not be tracked when upgrading the 'ManagedSchemaTransform'. Please update" + + "'ManagedSchemaTransformTranslation.ManagedSchemaTransformTranslator' to track the " + + "new method and update this test.", + MANAGED_TRANSFORM_SCHEMA_MAPPING.keySet().contains(getMethodName)); + } + + // Confirming that all fields mentioned in `WRITE_TRANSFORM_SCHEMA_MAPPING` are + // actually available in the schema. + MANAGED_TRANSFORM_SCHEMA_MAPPING.values().stream() + .forEach( + fieldName -> { + assertTrue( + "Field name " + + fieldName + + " was not found in the transform schema defined in " + + "ManagedSchemaTransformTranslation.ManagedSchemaTransformTranslator.", + ManagedSchemaTransformTranslator.SCHEMA.getFieldNames().contains(fieldName)); + }); + } + + @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 -> tr.getSpec().getUrn().equals(PTransformTranslation.MANAGED_TRANSFORM_URN)) + .collect(Collectors.toList()); + assertEquals(1, managedTransformProto.size()); + RunnerApi.FunctionSpec spec = managedTransformProto.get(0).getSpec(); + + // Check that the proto contains correct values + SchemaAwareTransforms.ManagedTransformPayload payload = + SchemaAwareTransforms.ManagedTransformPayload.parseFrom(spec.getPayload()); + assertEquals(TestSchemaTransformProvider.IDENTIFIER, payload.getUnderlyingTransformUrn()); + assertEquals(yamlStringConfig, payload.getYamlConfig()); + Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getExpansionSchema()); + assertEquals(ManagedSchemaTransformTranslator.SCHEMA, schemaFromSpec); + Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getExpansionPayload().newInput()); + Row expectedRow = + Row.withSchema(ManagedSchemaTransformTranslator.SCHEMA) + .withFieldValue("transformIdentifier", TestSchemaTransformProvider.IDENTIFIER) + .withFieldValue("configUrl", 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()); + } +} From 9034cee2c069445752cf728bd5707e952ad7611a Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 16 Apr 2024 19:27:23 -0400 Subject: [PATCH 15/60] spotless --- .../org/apache/beam/io/iceberg/IcebergIOTranslationTest.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergIOTranslationTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergIOTranslationTest.java index 930fc00f466d..07abb87e29fc 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergIOTranslationTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergIOTranslationTest.java @@ -19,7 +19,10 @@ import static org.apache.beam.io.iceberg.IcebergIOTranslation.IcebergIOReadTranslator.READ_SCHEMA; import static org.apache.beam.io.iceberg.IcebergIOTranslation.IcebergIOWriteTranslator.WRITE_SCHEMA; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; import java.util.Arrays; import java.util.List; From 0bc37797ed9ab0f4e01af67aec3b74f1b1a57ab6 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 16 Apr 2024 19:43:22 -0400 Subject: [PATCH 16/60] spotless; use underscore instead of camel case field names when translating managed transform config --- .../beam/io/iceberg/IcebergIOTranslation.java | 2 +- .../ManagedSchemaTransformTranslation.java | 56 +++++++++++++------ ...ManagedSchemaTransformTranslationTest.java | 8 +-- 3 files changed, 43 insertions(+), 23 deletions(-) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIOTranslation.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIOTranslation.java index 48b7ee244099..15742a9f9efe 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIOTranslation.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIOTranslation.java @@ -53,7 +53,7 @@ public class IcebergIOTranslation { static class IcebergIOReadTranslator implements TransformPayloadTranslator { - static Schema READ_SCHEMA = + static final Schema READ_SCHEMA = Schema.builder() .addByteArrayField("catalog_config") .addNullableStringField("table_identifier") 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 index c9d63a9d435e..0add6a96df73 100644 --- 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 @@ -25,15 +25,14 @@ import com.google.auto.service.AutoService; import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.util.HashMap; import java.util.Map; import org.apache.beam.model.pipeline.v1.SchemaApi; import org.apache.beam.model.pipeline.v1.SchemaAwareTransforms.ManagedTransformPayload; 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.NoSuchSchemaException; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.SchemaRegistry; import org.apache.beam.sdk.schemas.SchemaTranslation; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.construction.SdkComponents; @@ -47,15 +46,12 @@ public class ManagedSchemaTransformTranslation { static class ManagedSchemaTransformTranslator implements TransformPayloadTranslator { private final ManagedSchemaTransformProvider provider; - static final Schema SCHEMA; - - static { - try { - SCHEMA = SchemaRegistry.createDefault().getSchema(ManagedConfig.class); - } catch (NoSuchSchemaException e) { - throw new RuntimeException(e); - } - } + static final Schema SCHEMA = + Schema.builder() + .addStringField("transform_identifier") + .addNullableStringField("config") + .addNullableStringField("config_url") + .build(); public ManagedSchemaTransformTranslator() { provider = new ManagedSchemaTransformProvider(null); @@ -91,19 +87,43 @@ public String getUrn() { } @Override - @SuppressWarnings("nullable") public Row toConfigRow(ManagedSchemaTransform transform) { - ManagedConfig config = transform.getManagedConfig(); - try { - return SchemaRegistry.createDefault().getToRowFunction(ManagedConfig.class).apply(config); - } catch (NoSuchSchemaException e) { - throw new RuntimeException("Encountered error when finding schema for ManagedConfig"); + ManagedConfig managedConfig = transform.getManagedConfig(); + Map fieldValues = new HashMap<>(); + + if (managedConfig.getTransformIdentifier() != null) { + fieldValues.put("transform_identifier", managedConfig.getTransformIdentifier()); + } + String config = managedConfig.getConfig(); + if (config != null) { + fieldValues.put("config", config); + } + String configUrl = managedConfig.getConfigUrl(); + if (configUrl != null) { + fieldValues.put("config_url", configUrl); } + + return Row.withSchema(SCHEMA).withFieldValues(fieldValues).build(); } @Override public ManagedSchemaTransform fromConfigRow(Row configRow, PipelineOptions options) { - return (ManagedSchemaTransform) provider.from(configRow); + ManagedConfig.Builder configBuilder = ManagedConfig.builder(); + + String transformIdentifier = configRow.getValue("transform_identifier"); + if (transformIdentifier != null) { + configBuilder = configBuilder.setTransformIdentifier(transformIdentifier); + } + String config = configRow.getValue("config"); + if (config != null) { + configBuilder = configBuilder.setConfig(config); + } + String configUrl = configRow.getValue("config_url"); + if (configUrl != null) { + configBuilder = configBuilder.setConfigUrl(configUrl); + } + + return (ManagedSchemaTransform) provider.from(configBuilder.build()); } } 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 index 5dea2f8a4e05..88b925e8b13d 100644 --- 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 @@ -53,8 +53,8 @@ public class ManagedSchemaTransformTranslationTest { // ManagedSchemaTransformTranslation. static final Map MANAGED_TRANSFORM_SCHEMA_MAPPING = ImmutableMap.builder() - .put("getTransformIdentifier", "transformIdentifier") - .put("getConfigUrl", "configUrl") + .put("getTransformIdentifier", "transform_identifier") + .put("getConfigUrl", "config_url") .put("getConfig", "config") .build(); @@ -197,8 +197,8 @@ public void testProtoTranslation() throws Exception { Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getExpansionPayload().newInput()); Row expectedRow = Row.withSchema(ManagedSchemaTransformTranslator.SCHEMA) - .withFieldValue("transformIdentifier", TestSchemaTransformProvider.IDENTIFIER) - .withFieldValue("configUrl", null) + .withFieldValue("transform_identifier", TestSchemaTransformProvider.IDENTIFIER) + .withFieldValue("config_url", null) .withFieldValue("config", yamlStringConfig) .build(); assertEquals(expectedRow, rowFromSpec); From 27b5e6add76cbe0e32bfa3d8a45bb0c8fe55161e Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 16 Apr 2024 19:49:25 -0400 Subject: [PATCH 17/60] add grpc dependency --- sdks/java/io/iceberg/build.gradle | 1 + 1 file changed, 1 insertion(+) diff --git a/sdks/java/io/iceberg/build.gradle b/sdks/java/io/iceberg/build.gradle index 7feaeb8c4790..c7d469c88da0 100644 --- a/sdks/java/io/iceberg/build.gradle +++ b/sdks/java/io/iceberg/build.gradle @@ -42,6 +42,7 @@ def hive_version = "3.1.3" dependencies { implementation library.java.vendored_guava_32_1_2_jre + implementation library.java.vendored_grpc_1_60_1 implementation project(path: ":model:pipeline", configuration: "shadow") implementation project(path: ":sdks:java:core", configuration: "shadow") implementation library.java.slf4j_api From 05e3a1a7bb65a1ca606682d4bfafa30bb152619b Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Wed, 17 Apr 2024 00:19:41 -0400 Subject: [PATCH 18/60] updated proto description; fix gen xlang command --- .../beam/model/pipeline/v1/schema_aware_transforms.proto | 8 ++++---- .../beam/sdk/util/construction/PTransformTranslation.java | 2 +- sdks/python/build.gradle | 2 +- sdks/standard_expansion_services.yaml | 1 + 4 files changed, 7 insertions(+), 6 deletions(-) diff --git a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema_aware_transforms.proto b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema_aware_transforms.proto index b553fa25b37f..ceeee15a2c7c 100644 --- a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema_aware_transforms.proto +++ b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema_aware_transforms.proto @@ -50,7 +50,7 @@ message StandardSchemaAwareTransforms { // This is a transform that is aware of its input and output PCollection schemas // and is configured using Beam Schema-compatible parameters. message SchemaAwareTransformPayload { - // The transform's configuration schema + // The schema of the configuration row used to upgrade the transform Schema expansion_schema = 1; // The configuration used to build this transform. // Must be compatible with the configuration schema, and decodable via beam:coder:row:v1. @@ -63,10 +63,10 @@ message SchemaAwareTransformPayload { message ManagedTransformPayload { // The underlying transform's URN. string underlying_transform_urn = 1; - // The underlying transform's configuration Schema. + // The managed transform configuration Schema. Schema expansion_schema = 2; - // The configuration used to build the underlying transform. - // Must be compatible with the configuration schema, and decodable via beam:coder:row:v1. + // The configuration used to build the managed transform. + // Must be compatible with the expansion schema, and decodable via beam:coder:row:v1. bytes expansion_payload = 3; // The underlying transform's configuration, represented as a YAML string. string yaml_config = 4; 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 1d321162b91e..9adf1d72a28d 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 @@ -279,7 +279,7 @@ static RunnerApi.PTransform toProto( * *

    Does not register the {@code appliedPTransform} within the provided {@link SdkComponents}. */ - public static RunnerApi.PTransform toProto( + static RunnerApi.PTransform toProto( AppliedPTransform appliedPTransform, SdkComponents components) throws IOException { return toProto(appliedPTransform, Collections.emptyList(), components); } 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/standard_expansion_services.yaml b/sdks/standard_expansion_services.yaml index f9e1a55c57b2..8c9f969efab8 100644 --- a/sdks/standard_expansion_services.yaml +++ b/sdks/standard_expansion_services.yaml @@ -46,6 +46,7 @@ - 'beam:schematransform:org.apache.beam:kafka_read:v1' # Not ready to publish yet - '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' From 2b49c37aadbd4a1fd68bfb72533de563159406be Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Wed, 17 Apr 2024 12:18:12 -0400 Subject: [PATCH 19/60] ManagedTransform explicit input/output types; move iceberg package to org.apache.beam.sdk.io.iceberg --- .../beam/{ => sdk}/io/iceberg/AppendFilesToTables.java | 2 +- .../beam/{ => sdk}/io/iceberg/AssignDestinations.java | 2 +- .../beam/{ => sdk}/io/iceberg/DynamicDestinations.java | 2 +- .../apache/beam/{ => sdk}/io/iceberg/FileWriteResult.java | 2 +- .../beam/{ => sdk}/io/iceberg/IcebergCatalogConfig.java | 2 +- .../beam/{ => sdk}/io/iceberg/IcebergDestination.java | 2 +- .../org/apache/beam/{ => sdk}/io/iceberg/IcebergIO.java | 2 +- .../beam/{ => sdk}/io/iceberg/IcebergIOTranslation.java | 6 +++--- .../io/iceberg/IcebergReadSchemaTransformProvider.java | 4 ++-- .../beam/{ => sdk}/io/iceberg/IcebergScanConfig.java | 2 +- .../{ => sdk}/io/iceberg/IcebergTableCreateConfig.java | 2 +- .../beam/{ => sdk}/io/iceberg/IcebergWriteResult.java | 2 +- .../io/iceberg/IcebergWriteSchemaTransformProvider.java | 4 ++-- .../{ => sdk}/io/iceberg/OneTableDynamicDestinations.java | 2 +- .../apache/beam/{ => sdk}/io/iceberg/PropertyBuilder.java | 2 +- .../apache/beam/{ => sdk}/io/iceberg/RecordWriter.java | 4 ++-- .../org/apache/beam/{ => sdk}/io/iceberg/ScanSource.java | 2 +- .../apache/beam/{ => sdk}/io/iceberg/ScanTaskReader.java | 2 +- .../apache/beam/{ => sdk}/io/iceberg/ScanTaskSource.java | 2 +- .../{ => sdk}/io/iceberg/SchemaAndRowConversions.java | 2 +- .../io/iceberg/SchemaTransformCatalogConfig.java | 2 +- .../{ => sdk}/io/iceberg/WriteGroupedRowsToFiles.java | 2 +- .../beam/{ => sdk}/io/iceberg/WriteToDestinations.java | 2 +- .../{ => sdk}/io/iceberg/WriteUngroupedRowsToFiles.java | 2 +- .../apache/beam/{ => sdk}/io/iceberg/package-info.java | 2 +- .../beam/{ => sdk}/io/iceberg/FileWriteResultTest.java | 2 +- .../beam/{ => sdk}/io/iceberg/IcebergIOReadTest.java | 2 +- .../{ => sdk}/io/iceberg/IcebergIOTranslationTest.java | 6 +++--- .../beam/{ => sdk}/io/iceberg/IcebergIOWriteTest.java | 4 ++-- .../iceberg/IcebergReadSchemaTransformProviderTest.java | 2 +- .../iceberg/IcebergWriteSchemaTransformProviderTest.java | 8 ++++---- .../apache/beam/{ => sdk}/io/iceberg/ScanSourceTest.java | 2 +- .../{ => sdk}/io/iceberg/SchemaAndRowConversionsTest.java | 2 +- .../beam/{ => sdk}/io/iceberg/TestDataWarehouse.java | 2 +- .../apache/beam/{ => sdk}/io/iceberg/TestFixtures.java | 2 +- .../main/java/org/apache/beam/sdk/managed/Managed.java | 4 +++- 36 files changed, 49 insertions(+), 47 deletions(-) rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/AppendFilesToTables.java (98%) 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 (99%) 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%) rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/IcebergIO.java (99%) rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/IcebergIOTranslation.java (98%) rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/IcebergReadSchemaTransformProvider.java (97%) rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/IcebergScanConfig.java (99%) 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 (98%) rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/IcebergWriteSchemaTransformProvider.java (98%) rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/OneTableDynamicDestinations.java (98%) 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%) rename sdks/java/io/iceberg/src/main/java/org/apache/beam/{ => sdk}/io/iceberg/SchemaTransformCatalogConfig.java (98%) 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 (99%) 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 (99%) rename sdks/java/io/iceberg/src/test/java/org/apache/beam/{ => sdk}/io/iceberg/IcebergIOReadTest.java (99%) rename sdks/java/io/iceberg/src/test/java/org/apache/beam/{ => sdk}/io/iceberg/IcebergIOTranslationTest.java (98%) rename sdks/java/io/iceberg/src/test/java/org/apache/beam/{ => sdk}/io/iceberg/IcebergIOWriteTest.java (98%) rename sdks/java/io/iceberg/src/test/java/org/apache/beam/{ => sdk}/io/iceberg/IcebergReadSchemaTransformProviderTest.java (99%) rename sdks/java/io/iceberg/src/test/java/org/apache/beam/{ => sdk}/io/iceberg/IcebergWriteSchemaTransformProviderTest.java (94%) 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%) 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 98% 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..2795d3392d04 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,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.coders.KvCoder; import org.apache.beam.sdk.coders.SerializableCoder; 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 99% 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..210b39b6a1de 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; 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/io/iceberg/IcebergIO.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java similarity index 99% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIO.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java index 542f134942ec..75a35e6f8a30 100644 --- 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/sdk/io/iceberg/IcebergIO.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/IcebergIOTranslation.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslation.java similarity index 98% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIOTranslation.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslation.java index 15742a9f9efe..abccd3d3cbfc 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIOTranslation.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslation.java @@ -15,10 +15,10 @@ * 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.IcebergIO.ReadRows; -import static org.apache.beam.io.iceberg.IcebergIO.WriteRows; +import static org.apache.beam.sdk.io.iceberg.IcebergIO.ReadRows; +import static org.apache.beam.sdk.io.iceberg.IcebergIO.WriteRows; import static org.apache.beam.sdk.util.construction.TransformUpgrader.fromByteArray; import static org.apache.beam.sdk.util.construction.TransformUpgrader.toByteArray; diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergReadSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java similarity index 97% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergReadSchemaTransformProvider.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java index b5d7539ce491..0268c6402727 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergReadSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java @@ -15,13 +15,13 @@ * 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.service.AutoService; import com.google.auto.value.AutoValue; import java.util.Collections; import java.util.List; -import org.apache.beam.io.iceberg.IcebergReadSchemaTransformProvider.Config; +import org.apache.beam.sdk.io.iceberg.IcebergReadSchemaTransformProvider.Config; import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.schemas.transforms.SchemaTransform; 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/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 98% 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..428fa0103004 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; diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java similarity index 98% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProvider.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java index e1e51d564d12..b19ca1a5cfc4 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java @@ -15,13 +15,13 @@ * 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.service.AutoService; import com.google.auto.value.AutoValue; import java.util.Collections; import java.util.List; -import org.apache.beam.io.iceberg.IcebergWriteSchemaTransformProvider.Config; +import org.apache.beam.sdk.io.iceberg.IcebergWriteSchemaTransformProvider.Config; import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; 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 98% 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..f0931cb326a3 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,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.values.Row; 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/io/iceberg/SchemaTransformCatalogConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SchemaTransformCatalogConfig.java similarity index 98% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/SchemaTransformCatalogConfig.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SchemaTransformCatalogConfig.java index 9dd7b0f71554..5c7422bbded2 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/SchemaTransformCatalogConfig.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SchemaTransformCatalogConfig.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.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; 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 99% 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..649ad3d6d5c3 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; 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 99% 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..f38485e58e16 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,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/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 99% 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 e7b75b9e57d7..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; diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergIOTranslationTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslationTest.java similarity index 98% rename from sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergIOTranslationTest.java rename to sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslationTest.java index 07abb87e29fc..bbb62eb5868c 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergIOTranslationTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslationTest.java @@ -15,10 +15,10 @@ * 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.IcebergIOTranslation.IcebergIOReadTranslator.READ_SCHEMA; -import static org.apache.beam.io.iceberg.IcebergIOTranslation.IcebergIOWriteTranslator.WRITE_SCHEMA; +import static org.apache.beam.sdk.io.iceberg.IcebergIOTranslation.IcebergIOReadTranslator.READ_SCHEMA; +import static org.apache.beam.sdk.io.iceberg.IcebergIOTranslation.IcebergIOWriteTranslator.WRITE_SCHEMA; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; 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 98% 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 cd2303716198..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; diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergReadSchemaTransformProviderTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProviderTest.java similarity index 99% rename from sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergReadSchemaTransformProviderTest.java rename to sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProviderTest.java index 9be7f1797cd9..e712615270ac 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergReadSchemaTransformProviderTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProviderTest.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; diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProviderTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProviderTest.java similarity index 94% rename from sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProviderTest.java rename to sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProviderTest.java index 3eff5cae402d..981b28ee7c60 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergWriteSchemaTransformProviderTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProviderTest.java @@ -15,11 +15,11 @@ * 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.IcebergWriteSchemaTransformProvider.Config; -import static org.apache.beam.io.iceberg.IcebergWriteSchemaTransformProvider.INPUT_TAG; -import static org.apache.beam.io.iceberg.IcebergWriteSchemaTransformProvider.OUTPUT_TAG; +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; 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/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 b29aba68443e..35e015e2194c 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 @@ -25,6 +25,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; @@ -127,7 +128,8 @@ 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 Map getConfig(); From 7de01bbd009c4fc40271d5b607088ae8b6861e76 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Wed, 17 Apr 2024 15:38:35 -0400 Subject: [PATCH 20/60] externalizable IcebergCatalogConfig --- .../ExternalizableIcebergCatalogConfig.java | 128 ++++++++++++++++++ .../sdk/io/iceberg/IcebergIOTranslation.java | 16 ++- ...xternalizableIcebergCatalogConfigTest.java | 62 +++++++++ .../io/iceberg/IcebergIOTranslationTest.java | 10 +- 4 files changed, 210 insertions(+), 6 deletions(-) create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ExternalizableIcebergCatalogConfig.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ExternalizableIcebergCatalogConfigTest.java diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ExternalizableIcebergCatalogConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ExternalizableIcebergCatalogConfig.java new file mode 100644 index 000000000000..d33998a30042 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ExternalizableIcebergCatalogConfig.java @@ -0,0 +1,128 @@ +/* + * 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.common.base.Preconditions; +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * A wrapper to more precisely serialize a {@link IcebergCatalogConfig}. + * + *

    Only includes properties used in {@link IcebergCatalogConfig#catalog()} to generate a {@link + * org.apache.iceberg.catalog.Catalog}: + * + *

      + *
    • {@link IcebergCatalogConfig#getName()} + *
    • {@link IcebergCatalogConfig#getIcebergCatalogType()} + *
    • {@link IcebergCatalogConfig#getCatalogImplementation()} + *
    • {@link IcebergCatalogConfig#getFileIOImplementation()} + *
    • {@link IcebergCatalogConfig#getWarehouseLocation()} + *
    • {@link IcebergCatalogConfig#getMetricsReporterImplementation()} + *
    • {@link IcebergCatalogConfig#getCacheEnabled()} + *
    • {@link IcebergCatalogConfig#getCacheCaseSensitive()} ()} + *
    • {@link IcebergCatalogConfig#getCacheExpirationIntervalMillis()} + *
    • {@link IcebergCatalogConfig#getConfiguration()} + *
    + * + * * + */ +public class ExternalizableIcebergCatalogConfig implements Externalizable { + private static final long serialVersionUID = 0L; + + private @Nullable IcebergCatalogConfig catalogConfig; + + // Keep this in sync with IcebergCatalogConfig properties map + static List PROPERTY_KEYS = + ImmutableList.builder() + .add("name") + .add(CatalogUtil.ICEBERG_CATALOG_TYPE) + .add(CatalogProperties.CATALOG_IMPL) + .add(CatalogProperties.FILE_IO_IMPL) + .add(CatalogProperties.WAREHOUSE_LOCATION) + .add(CatalogProperties.METRICS_REPORTER_IMPL) + .add(CatalogProperties.CACHE_ENABLED) + .add(CatalogProperties.CACHE_CASE_SENSITIVE) + .add(CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS) + .build(); + + public ExternalizableIcebergCatalogConfig() {} + + public ExternalizableIcebergCatalogConfig(@Nullable IcebergCatalogConfig catalogConfig) { + if (catalogConfig == null) { + throw new NullPointerException("Configuration must not be null."); + } + this.catalogConfig = catalogConfig; + } + + public @Nullable IcebergCatalogConfig get() { + return catalogConfig; + } + + @Override + public void writeExternal(ObjectOutput out) throws IOException { + if (catalogConfig == null) { + return; + } + Map properties = new HashMap<>(PROPERTY_KEYS.size()); + properties.put("name", catalogConfig.getName()); + properties.putAll(Preconditions.checkNotNull(catalogConfig).properties()); + for (String prop : PROPERTY_KEYS) { + out.writeUTF(properties.getOrDefault(prop, "")); + } + if (catalogConfig != null && catalogConfig.getConfiguration() != null) { + catalogConfig.getConfiguration().write(out); + } + } + + private @Nullable String orNull(String value) { + return Strings.isNullOrEmpty(value) ? null : value; + } + + @Override + public void readExternal(ObjectInput in) throws IOException { + IcebergCatalogConfig.Builder builder = + IcebergCatalogConfig.builder() + .setName(in.readUTF()) + .setIcebergCatalogType(orNull(in.readUTF())) + .setCatalogImplementation(orNull(in.readUTF())) + .setFileIOImplementation(orNull(in.readUTF())) + .setWarehouseLocation(orNull(in.readUTF())) + .setMetricsReporterImplementation(orNull(in.readUTF())) + .setCacheEnabled(Boolean.parseBoolean(in.readUTF())) + .setCacheCaseSensitive(Boolean.parseBoolean(in.readUTF())) + .setCacheExpirationIntervalMillis(Long.parseLong(in.readUTF())); + if (in.available() > 0) { + Configuration hadoopConf = new Configuration(); + hadoopConf.readFields(in); + builder = builder.setConfiguration(hadoopConf); + } + catalogConfig = builder.build(); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslation.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslation.java index abccd3d3cbfc..3426a908fc4a 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslation.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslation.java @@ -93,7 +93,9 @@ public Row toConfigRow(ReadRows transform) { Map fieldValues = new HashMap<>(); if (transform.getCatalogConfig() != null) { - fieldValues.put("catalog_config", toByteArray(transform.getCatalogConfig())); + fieldValues.put( + "catalog_config", + toByteArray(new ExternalizableIcebergCatalogConfig(transform.getCatalogConfig()))); } if (transform.getTableIdentifier() != null) { TableIdentifier identifier = transform.getTableIdentifier(); @@ -113,7 +115,9 @@ public ReadRows fromConfigRow(Row configRow, PipelineOptions options) { byte[] catalogBytes = configRow.getBytes("catalog_config"); if (catalogBytes != null) { - builder = builder.setCatalogConfig((IcebergCatalogConfig) fromByteArray(catalogBytes)); + builder = + builder.setCatalogConfig( + ((ExternalizableIcebergCatalogConfig) fromByteArray(catalogBytes)).get()); } String tableIdentifier = configRow.getString("table_identifier"); if (tableIdentifier != null) { @@ -183,7 +187,9 @@ public Row toConfigRow(WriteRows transform) { Map fieldValues = new HashMap<>(); if (transform.getCatalogConfig() != null) { - fieldValues.put("catalog_config", toByteArray(transform.getCatalogConfig())); + fieldValues.put( + "catalog_config", + toByteArray(new ExternalizableIcebergCatalogConfig(transform.getCatalogConfig()))); } if (transform.getTableIdentifier() != null) { TableIdentifier identifier = transform.getTableIdentifier(); @@ -206,7 +212,9 @@ public WriteRows fromConfigRow(Row configRow, PipelineOptions options) { byte[] catalogBytes = configRow.getBytes("catalog_config"); if (catalogBytes != null) { - builder = builder.setCatalogConfig((IcebergCatalogConfig) fromByteArray(catalogBytes)); + builder = + builder.setCatalogConfig( + ((ExternalizableIcebergCatalogConfig) fromByteArray(catalogBytes)).get()); } String tableIdentifier = configRow.getString("table_identifier"); if (tableIdentifier != null) { diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ExternalizableIcebergCatalogConfigTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ExternalizableIcebergCatalogConfigTest.java new file mode 100644 index 000000000000..2b5c61c97f3b --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ExternalizableIcebergCatalogConfigTest.java @@ -0,0 +1,62 @@ +/* + * 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.junit.Assert.assertEquals; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class ExternalizableIcebergCatalogConfigTest { + @Test + public void testSerializeDeserialize() throws IOException, ClassNotFoundException { + IcebergCatalogConfig config = + IcebergCatalogConfig.builder() + .setName("test-name") + .setIcebergCatalogType("test-type") + .setCatalogImplementation("test-implementation") + .setFileIOImplementation("test-fileio") + .setWarehouseLocation("test-location") + .setMetricsReporterImplementation("test-metrics") + .setCacheEnabled(true) + .setCacheCaseSensitive(true) + .setCacheExpirationIntervalMillis(100) + .build(); + + ExternalizableIcebergCatalogConfig externalizableConfig = + new ExternalizableIcebergCatalogConfig(config); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + ObjectOutputStream oos = new ObjectOutputStream(baos); + + oos.writeObject(externalizableConfig); + + ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray()); + ObjectInputStream ois = new ObjectInputStream(bais); + ExternalizableIcebergCatalogConfig roundtripConfig = + (ExternalizableIcebergCatalogConfig) ois.readObject(); + + assertEquals(config, roundtripConfig.get()); + } +} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslationTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslationTest.java index bbb62eb5868c..139652ff26bf 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslationTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslationTest.java @@ -153,7 +153,10 @@ public void testWriteTransformProtoTranslation() throws Exception { Row expectedRow = Row.withSchema(WRITE_SCHEMA) .withFieldValue("table_identifier", identifier) - .withFieldValue("catalog_config", TransformUpgrader.toByteArray(catalogConfig)) + .withFieldValue( + "catalog_config", + TransformUpgrader.toByteArray( + new ExternalizableIcebergCatalogConfig(catalogConfig))) .withFieldValue("dynamic_destinations", null) .build(); assertEquals(expectedRow, rowFromSpec); @@ -278,7 +281,10 @@ public void testReadTransformProtoTranslation() throws Exception { Row expectedRow = Row.withSchema(READ_SCHEMA) .withFieldValue("table_identifier", identifier) - .withFieldValue("catalog_config", TransformUpgrader.toByteArray(catalogConfig)) + .withFieldValue( + "catalog_config", + TransformUpgrader.toByteArray( + new ExternalizableIcebergCatalogConfig(catalogConfig))) .build(); assertEquals(expectedRow, rowFromSpec); From 6f739f4aea71fa4fee15470a4932f98cd2c0d3e1 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Wed, 17 Apr 2024 18:55:30 -0400 Subject: [PATCH 21/60] externalizable IcebergCatalogConfig supports all properties; address some comments --- ...m_PreCommit_Xlang_Generated_Transforms.yml | 2 +- .../pipeline/v1/schema_aware_transforms.proto | 12 +- sdks/java/io/iceberg/build.gradle | 1 + .../ExternalizableIcebergCatalogConfig.java | 153 ++++++++++-------- .../sdk/io/iceberg/IcebergIOTranslation.java | 6 +- .../IcebergReadSchemaTransformProvider.java | 2 +- .../IcebergWriteSchemaTransformProvider.java | 2 +- .../org/apache/beam/sdk/managed/Managed.java | 7 +- .../ManagedSchemaTransformProvider.java | 4 +- .../ManagedSchemaTransformTranslation.java | 4 +- ...ManagedSchemaTransformTranslationTest.java | 4 +- sdks/standard_expansion_services.yaml | 3 - 12 files changed, 103 insertions(+), 97 deletions(-) diff --git a/.github/workflows/beam_PreCommit_Xlang_Generated_Transforms.yml b/.github/workflows/beam_PreCommit_Xlang_Generated_Transforms.yml index fed883412fc6..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 --info \ 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/schema_aware_transforms.proto b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema_aware_transforms.proto index ceeee15a2c7c..2a3e5d693e7a 100644 --- a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema_aware_transforms.proto +++ b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema_aware_transforms.proto @@ -40,7 +40,7 @@ message StandardSchemaAwareTransforms { ICEBERG_WRITE = 1 [(beam_urn) = "beam:transform:iceberg_write:v1"]; } enum Managed { - // Payload: ManagedTransformPayload + // Payload: ManagedSchemaTransformPayload MANAGED = 0 [(beam_urn) = "beam:transform:managed:v1"]; } } @@ -49,6 +49,7 @@ message StandardSchemaAwareTransforms { // 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 by runners to override the schema-aware transform. message SchemaAwareTransformPayload { // The schema of the configuration row used to upgrade the transform Schema expansion_schema = 1; @@ -57,15 +58,14 @@ message SchemaAwareTransformPayload { bytes expansion_payload = 2; } -// Payload for a Managed transform -// This can be used by runners that wish to override an underlying transform -// with a different implementation. -message ManagedTransformPayload { +// Payload for a ManagedSchemaTransform. +// The information available in the payload can be used by runners to override the ManagedSchemaTransform. +message ManagedSchemaTransformPayload { // The underlying transform's URN. string underlying_transform_urn = 1; // The managed transform configuration Schema. Schema expansion_schema = 2; - // The configuration used to build the managed transform. + // The configuration used to upgrade the managed transform. // Must be compatible with the expansion schema, and decodable via beam:coder:row:v1. bytes expansion_payload = 3; // The underlying transform's configuration, represented as a YAML string. diff --git a/sdks/java/io/iceberg/build.gradle b/sdks/java/io/iceberg/build.gradle index c7d469c88da0..54dd0f23fba5 100644 --- a/sdks/java/io/iceberg/build.gradle +++ b/sdks/java/io/iceberg/build.gradle @@ -45,6 +45,7 @@ dependencies { implementation library.java.vendored_grpc_1_60_1 implementation project(path: ":model:pipeline", configuration: "shadow") implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation project(":sdks:java:io:hadoop-common") 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/sdk/io/iceberg/ExternalizableIcebergCatalogConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ExternalizableIcebergCatalogConfig.java index d33998a30042..318198b41964 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ExternalizableIcebergCatalogConfig.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ExternalizableIcebergCatalogConfig.java @@ -17,61 +17,26 @@ */ package org.apache.beam.sdk.io.iceberg; -import com.google.common.base.Preconditions; import java.io.Externalizable; import java.io.IOException; import java.io.ObjectInput; import java.io.ObjectOutput; import java.util.HashMap; -import java.util.List; import java.util.Map; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.CatalogUtil; +import org.apache.beam.sdk.io.hadoop.SerializableConfiguration; import org.checkerframework.checker.nullness.qual.Nullable; /** - * A wrapper to more precisely serialize a {@link IcebergCatalogConfig}. - * - *

    Only includes properties used in {@link IcebergCatalogConfig#catalog()} to generate a {@link - * org.apache.iceberg.catalog.Catalog}: - * - *

      - *
    • {@link IcebergCatalogConfig#getName()} - *
    • {@link IcebergCatalogConfig#getIcebergCatalogType()} - *
    • {@link IcebergCatalogConfig#getCatalogImplementation()} - *
    • {@link IcebergCatalogConfig#getFileIOImplementation()} - *
    • {@link IcebergCatalogConfig#getWarehouseLocation()} - *
    • {@link IcebergCatalogConfig#getMetricsReporterImplementation()} - *
    • {@link IcebergCatalogConfig#getCacheEnabled()} - *
    • {@link IcebergCatalogConfig#getCacheCaseSensitive()} ()} - *
    • {@link IcebergCatalogConfig#getCacheExpirationIntervalMillis()} - *
    • {@link IcebergCatalogConfig#getConfiguration()} - *
    - * - * * + * A wrapper to more precisely serialize a {@link IcebergCatalogConfig} object. {@link + * IcebergCatalogConfig} is an AutoValue class, which raises some complications when trying to have + * it directly implement {@link Externalizable}. Hence, this class is used to wrap around the {@link + * IcebergCatalogConfig} object when serializing and deserializing. * */ public class ExternalizableIcebergCatalogConfig implements Externalizable { private static final long serialVersionUID = 0L; private @Nullable IcebergCatalogConfig catalogConfig; - // Keep this in sync with IcebergCatalogConfig properties map - static List PROPERTY_KEYS = - ImmutableList.builder() - .add("name") - .add(CatalogUtil.ICEBERG_CATALOG_TYPE) - .add(CatalogProperties.CATALOG_IMPL) - .add(CatalogProperties.FILE_IO_IMPL) - .add(CatalogProperties.WAREHOUSE_LOCATION) - .add(CatalogProperties.METRICS_REPORTER_IMPL) - .add(CatalogProperties.CACHE_ENABLED) - .add(CatalogProperties.CACHE_CASE_SENSITIVE) - .add(CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS) - .build(); - public ExternalizableIcebergCatalogConfig() {} public ExternalizableIcebergCatalogConfig(@Nullable IcebergCatalogConfig catalogConfig) { @@ -86,43 +51,89 @@ public ExternalizableIcebergCatalogConfig(@Nullable IcebergCatalogConfig catalog } @Override + @SuppressWarnings("nullness") public void writeExternal(ObjectOutput out) throws IOException { - if (catalogConfig == null) { - return; - } - Map properties = new HashMap<>(PROPERTY_KEYS.size()); + Map properties = new HashMap<>(); properties.put("name", catalogConfig.getName()); - properties.putAll(Preconditions.checkNotNull(catalogConfig).properties()); - for (String prop : PROPERTY_KEYS) { - out.writeUTF(properties.getOrDefault(prop, "")); - } - if (catalogConfig != null && catalogConfig.getConfiguration() != null) { - catalogConfig.getConfiguration().write(out); - } - } + properties.put("icebergCatalogType", catalogConfig.getIcebergCatalogType()); + properties.put("catalogImplementation", catalogConfig.getCatalogImplementation()); + properties.put("fileIOImplementation", catalogConfig.getFileIOImplementation()); + properties.put("warehouseLocation", catalogConfig.getWarehouseLocation()); + properties.put( + "metricsReporterImplementation", catalogConfig.getMetricsReporterImplementation()); + properties.put("cacheEnabled", catalogConfig.getCacheEnabled()); + properties.put("cacheCaseSensitive", catalogConfig.getCacheCaseSensitive()); + properties.put( + "cacheExpirationIntervalMillis", catalogConfig.getCacheExpirationIntervalMillis()); + properties.put("ioManifestCacheEnabled", catalogConfig.getIOManifestCacheEnabled()); + properties.put( + "ioManifestCacheExpirationIntervalMillis", + catalogConfig.getIOManifestCacheExpirationIntervalMillis()); + properties.put("ioManifestCacheMaxTotalBytes", catalogConfig.getIOManifestCacheMaxTotalBytes()); + properties.put( + "ioManifestCacheMaxContentLength", catalogConfig.getIOManifestCacheMaxContentLength()); + properties.put("uri", catalogConfig.getUri()); + properties.put("clientPoolSize", catalogConfig.getClientPoolSize()); + properties.put("clientPoolEvictionIntervalMs", catalogConfig.getClientPoolEvictionIntervalMs()); + properties.put("clientPoolCacheKeys", catalogConfig.getClientPoolCacheKeys()); + properties.put("lockImplementation", catalogConfig.getLockImplementation()); + properties.put("lockHeartbeatIntervalMillis", catalogConfig.getLockHeartbeatIntervalMillis()); + properties.put("lockHeartbeatTimeoutMillis", catalogConfig.getLockHeartbeatTimeoutMillis()); + properties.put("lockHeartbeatThreads", catalogConfig.getLockHeartbeatThreads()); + properties.put("lockAcquireIntervalMillis", catalogConfig.getLockAcquireIntervalMillis()); + properties.put("lockAcquireTimeoutMillis", catalogConfig.getLockAcquireTimeoutMillis()); + properties.put("appIdentifier", catalogConfig.getAppIdentifier()); + properties.put("user", catalogConfig.getUser()); + properties.put("authSessionTimeoutMillis", catalogConfig.getAuthSessionTimeoutMillis()); + properties.put( + "configuration", + catalogConfig.getConfiguration() == null + ? null + : new SerializableConfiguration(catalogConfig.getConfiguration())); - private @Nullable String orNull(String value) { - return Strings.isNullOrEmpty(value) ? null : value; + out.writeObject(properties); } @Override - public void readExternal(ObjectInput in) throws IOException { - IcebergCatalogConfig.Builder builder = + @SuppressWarnings("nullness") + public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + Map properties = (Map) in.readObject(); + catalogConfig = IcebergCatalogConfig.builder() - .setName(in.readUTF()) - .setIcebergCatalogType(orNull(in.readUTF())) - .setCatalogImplementation(orNull(in.readUTF())) - .setFileIOImplementation(orNull(in.readUTF())) - .setWarehouseLocation(orNull(in.readUTF())) - .setMetricsReporterImplementation(orNull(in.readUTF())) - .setCacheEnabled(Boolean.parseBoolean(in.readUTF())) - .setCacheCaseSensitive(Boolean.parseBoolean(in.readUTF())) - .setCacheExpirationIntervalMillis(Long.parseLong(in.readUTF())); - if (in.available() > 0) { - Configuration hadoopConf = new Configuration(); - hadoopConf.readFields(in); - builder = builder.setConfiguration(hadoopConf); - } - catalogConfig = builder.build(); + .setName((String) properties.get("name")) + .setIcebergCatalogType((String) properties.get("icebergCatalogType")) + .setCatalogImplementation((String) properties.get("catalogImplementation")) + .setFileIOImplementation((String) properties.get("fileIOImplementation")) + .setWarehouseLocation((String) properties.get("warehouseLocation")) + .setMetricsReporterImplementation( + (String) properties.get("metricsReporterImplementation")) + .setCacheEnabled((Boolean) properties.get("cacheEnabled")) + .setCacheCaseSensitive((Boolean) properties.get("cacheCaseSensitive")) + .setCacheExpirationIntervalMillis( + (Long) properties.get("cacheExpirationIntervalMillis")) + .setIOManifestCacheEnabled((Boolean) properties.get("ioManifestCacheEnabled")) + .setIOManifestCacheExpirationIntervalMillis( + (Long) properties.get("ioManifestCacheExpirationIntervalMillis")) + .setIOManifestCacheMaxTotalBytes((Long) properties.get("ioManifestCacheMaxTotalBytes")) + .setIOManifestCacheMaxContentLength( + (Long) properties.get("ioManifestCacheMaxContentLength")) + .setUri((String) properties.get("uri")) + .setClientPoolSize((Integer) properties.get("clientPoolSize")) + .setClientPoolEvictionIntervalMs((Long) properties.get("clientPoolEvictionIntervalMs")) + .setClientPoolCacheKeys((String) properties.get("clientPoolCacheKeys")) + .setLockImplementation((String) properties.get("lockImplementation")) + .setLockHeartbeatIntervalMillis((Long) properties.get("lockHeartbeatIntervalMillis")) + .setLockHeartbeatTimeoutMillis((Long) properties.get("lockHeartbeatTimeoutMillis")) + .setLockHeartbeatThreads((Integer) properties.get("lockHeartbeatThreads")) + .setLockAcquireIntervalMillis((Long) properties.get("lockAcquireIntervalMillis")) + .setLockAcquireTimeoutMillis((Long) properties.get("lockAcquireTimeoutMillis")) + .setAppIdentifier((String) properties.get("appIdentifier")) + .setUser((String) properties.get("user")) + .setAuthSessionTimeoutMillis((Long) properties.get("authSessionTimeoutMillis")) + .setConfiguration( + properties.get("configuration") == null + ? null + : ((SerializableConfiguration) properties.get("configuration")).get()) + .build(); } } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslation.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslation.java index 3426a908fc4a..e2c7ac163333 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslation.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslation.java @@ -59,8 +59,7 @@ static class IcebergIOReadTranslator implements TransformPayloadTranslator outputCollectionNames() { @Override public String identifier() { - return "beam:schematransform:org.apache.beam:iceberg_read:v1"; + return "beam:transform:iceberg_read:v1"; } @DefaultSchema(AutoValueSchema.class) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java index b19ca1a5cfc4..73902ffa7de2 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java @@ -82,7 +82,7 @@ public List outputCollectionNames() { @Override public String identifier() { - return "beam:schematransform:org.apache.beam:iceberg_write:v1"; + return "beam:transform:iceberg_write:v1"; } @DefaultSchema(AutoValueSchema.class) 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 35e015e2194c..4bafe967e7d2 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 @@ -78,13 +78,12 @@ 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, "beam:transform:iceberg_read:v1").build(); public static final Map WRITE_TRANSFORMS = ImmutableMap.builder() - .put(ICEBERG, "beam:schematransform:org.apache.beam:iceberg_write:v1") + .put(ICEBERG, "beam:transform:iceberg_write:v1") .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 c188b4fb42d3..7c3b9ec95b07 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 @@ -84,7 +84,7 @@ public ManagedSchemaTransformProvider() {} @DefaultSchema(AutoValueSchema.class) @AutoValue @VisibleForTesting - public abstract static class ManagedConfig { + abstract static class ManagedConfig { public static Builder builder() { return new AutoValue_ManagedSchemaTransformProvider_ManagedConfig.Builder(); } @@ -148,7 +148,7 @@ protected SchemaTransform from(ManagedConfig managedConfig) { return new ManagedSchemaTransform(managedConfig, schemaTransformProvider); } - public static class ManagedSchemaTransform extends SchemaTransform { + static class ManagedSchemaTransform extends SchemaTransform { private final Row transformConfig; private final ManagedConfig managedConfig; private final SchemaTransformProvider underlyingTransformProvider; 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 index 0add6a96df73..9539df85690c 100644 --- 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 @@ -28,7 +28,7 @@ import java.util.HashMap; import java.util.Map; import org.apache.beam.model.pipeline.v1.SchemaApi; -import org.apache.beam.model.pipeline.v1.SchemaAwareTransforms.ManagedTransformPayload; +import org.apache.beam.model.pipeline.v1.SchemaAwareTransforms.ManagedSchemaTransformPayload; import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.AppliedPTransform; @@ -76,7 +76,7 @@ public String getUrn() { return FunctionSpec.newBuilder() .setUrn(getUrn()) .setPayload( - ManagedTransformPayload.newBuilder() + ManagedSchemaTransformPayload.newBuilder() .setUnderlyingTransformUrn(managedConfig.getTransformIdentifier()) .setYamlConfig(managedConfig.resolveUnderlyingConfig()) .setExpansionSchema(expansionSchema) 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 index 88b925e8b13d..7cdff3625cb2 100644 --- 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 @@ -188,8 +188,8 @@ public void testProtoTranslation() throws Exception { RunnerApi.FunctionSpec spec = managedTransformProto.get(0).getSpec(); // Check that the proto contains correct values - SchemaAwareTransforms.ManagedTransformPayload payload = - SchemaAwareTransforms.ManagedTransformPayload.parseFrom(spec.getPayload()); + SchemaAwareTransforms.ManagedSchemaTransformPayload payload = + SchemaAwareTransforms.ManagedSchemaTransformPayload.parseFrom(spec.getPayload()); assertEquals(TestSchemaTransformProvider.IDENTIFIER, payload.getUnderlyingTransformUrn()); assertEquals(yamlStringConfig, payload.getYamlConfig()); Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getExpansionSchema()); diff --git a/sdks/standard_expansion_services.yaml b/sdks/standard_expansion_services.yaml index 8c9f969efab8..e9e6871be82f 100644 --- a/sdks/standard_expansion_services.yaml +++ b/sdks/standard_expansion_services.yaml @@ -44,9 +44,6 @@ # 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 publish yet - - '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' From 1df6aa70b7564c1a1262bfaf24b39f4e53a95a15 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 18 Apr 2024 12:03:36 -0400 Subject: [PATCH 22/60] unify iceberg urns and identifiers; update some comments --- .../model/pipeline/v1/schema_aware_transforms.proto | 10 +++++----- .../beam/sdk/io/iceberg/IcebergIOTranslation.java | 6 ++++-- .../io/iceberg/IcebergReadSchemaTransformProvider.java | 2 +- .../iceberg/IcebergWriteSchemaTransformProvider.java | 2 +- .../main/java/org/apache/beam/sdk/managed/Managed.java | 6 ++++-- .../sdk/managed/ManagedSchemaTransformProvider.java | 8 +++++--- .../sdk/managed/ManagedSchemaTransformTranslation.java | 2 +- .../managed/ManagedSchemaTransformTranslationTest.java | 3 ++- 8 files changed, 23 insertions(+), 16 deletions(-) diff --git a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema_aware_transforms.proto b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema_aware_transforms.proto index 2a3e5d693e7a..62c5f2da71bb 100644 --- a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema_aware_transforms.proto +++ b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema_aware_transforms.proto @@ -36,8 +36,8 @@ import "org/apache/beam/model/pipeline/v1/schema.proto"; message StandardSchemaAwareTransforms { // Payload for all of these: SchemaAwareTransformPayload enum IOs { - ICEBERG_READ = 0 [(beam_urn) = "beam:transform:iceberg_read:v1"]; - ICEBERG_WRITE = 1 [(beam_urn) = "beam:transform:iceberg_write:v1"]; + ICEBERG_READ = 0 [(beam_urn) = "beam:schematransform:org.apache.beam:iceberg_read:v1"]; + ICEBERG_WRITE = 1 [(beam_urn) = "beam:schematransform:org.apache.beam:iceberg_write:v1"]; } enum Managed { // Payload: ManagedSchemaTransformPayload @@ -61,9 +61,9 @@ message SchemaAwareTransformPayload { // Payload for a ManagedSchemaTransform. // The information available in the payload can be used by runners to override the ManagedSchemaTransform. message ManagedSchemaTransformPayload { - // The underlying transform's URN. - string underlying_transform_urn = 1; - // The managed transform configuration Schema. + // The underlying schema-aware transform's identifier. + string underlying_transform_identifier = 1; + // The managed transform's configuration Schema. Schema expansion_schema = 2; // The configuration used to upgrade the managed transform. // Must be compatible with the expansion schema, and decodable via beam:coder:row:v1. diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslation.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslation.java index e2c7ac163333..e5b06a021a26 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslation.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslation.java @@ -59,7 +59,8 @@ static class IcebergIOReadTranslator implements TransformPayloadTranslator outputCollectionNames() { @Override public String identifier() { - return "beam:transform:iceberg_read:v1"; + return "beam:schematransform:org.apache.beam:iceberg_read:v1"; } @DefaultSchema(AutoValueSchema.class) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java index 73902ffa7de2..b19ca1a5cfc4 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java @@ -82,7 +82,7 @@ public List outputCollectionNames() { @Override public String identifier() { - return "beam:transform:iceberg_write:v1"; + return "beam:schematransform:org.apache.beam:iceberg_write:v1"; } @DefaultSchema(AutoValueSchema.class) 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 4bafe967e7d2..d5eef3c6e0b6 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 @@ -80,10 +80,12 @@ public class Managed { // Supported SchemaTransforms public static final Map READ_TRANSFORMS = - ImmutableMap.builder().put(ICEBERG, "beam:transform:iceberg_read:v1").build(); + ImmutableMap.builder() + .put(ICEBERG, "beam:schematransform:org.apache.beam:iceberg_read:v1") + .build(); public static final Map WRITE_TRANSFORMS = ImmutableMap.builder() - .put(ICEBERG, "beam:transform:iceberg_write:v1") + .put(ICEBERG, "beam:schematransform:org.apache.beam:iceberg_write:v1") .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 7c3b9ec95b07..15e2912cc524 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 @@ -89,13 +89,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 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 index 9539df85690c..8a1e5f12c2b1 100644 --- 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 @@ -77,7 +77,7 @@ public String getUrn() { .setUrn(getUrn()) .setPayload( ManagedSchemaTransformPayload.newBuilder() - .setUnderlyingTransformUrn(managedConfig.getTransformIdentifier()) + .setUnderlyingTransformIdentifier(managedConfig.getTransformIdentifier()) .setYamlConfig(managedConfig.resolveUnderlyingConfig()) .setExpansionSchema(expansionSchema) .setExpansionPayload(ByteString.copyFrom(os.toByteArray())) 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 index 7cdff3625cb2..d530e5b841be 100644 --- 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 @@ -190,7 +190,8 @@ public void testProtoTranslation() throws Exception { // Check that the proto contains correct values SchemaAwareTransforms.ManagedSchemaTransformPayload payload = SchemaAwareTransforms.ManagedSchemaTransformPayload.parseFrom(spec.getPayload()); - assertEquals(TestSchemaTransformProvider.IDENTIFIER, payload.getUnderlyingTransformUrn()); + assertEquals( + TestSchemaTransformProvider.IDENTIFIER, payload.getUnderlyingTransformIdentifier()); assertEquals(yamlStringConfig, payload.getYamlConfig()); Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getExpansionSchema()); assertEquals(ManagedSchemaTransformTranslator.SCHEMA, schemaFromSpec); From 6d7a353bcd2fcb76d5ce7ab2a660f22489f63a4f Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 18 Apr 2024 12:37:47 -0400 Subject: [PATCH 23/60] one source for all supported managed transform identifiers --- sdks/java/io/iceberg/build.gradle | 1 + .../sdk/io/iceberg/IcebergIOTranslation.java | 11 +++------ .../IcebergReadSchemaTransformProvider.java | 3 ++- .../IcebergWriteSchemaTransformProvider.java | 3 ++- .../io/iceberg/IcebergIOTranslationTest.java | 17 +++---------- .../org/apache/beam/sdk/managed/Managed.java | 11 ++++----- .../managed/ManagedTransformConstants.java | 24 +++++++++++++++++++ .../apache/beam/sdk/managed/ManagedTest.java | 3 +++ 8 files changed, 43 insertions(+), 30 deletions(-) create mode 100644 sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java diff --git a/sdks/java/io/iceberg/build.gradle b/sdks/java/io/iceberg/build.gradle index 54dd0f23fba5..a1adb42ac26d 100644 --- a/sdks/java/io/iceberg/build.gradle +++ b/sdks/java/io/iceberg/build.gradle @@ -45,6 +45,7 @@ dependencies { implementation library.java.vendored_grpc_1_60_1 implementation project(path: ":model:pipeline", configuration: "shadow") implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation project(":sdks:java:managed") implementation project(":sdks:java:io:hadoop-common") implementation library.java.slf4j_api implementation library.java.joda_time diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslation.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslation.java index e5b06a021a26..8b24061f49ee 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslation.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslation.java @@ -35,6 +35,7 @@ import org.apache.beam.model.pipeline.v1.SchemaApi; import org.apache.beam.model.pipeline.v1.SchemaAwareTransforms.SchemaAwareTransformPayload; import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.managed.ManagedTransformConstants; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.schemas.Schema; @@ -59,12 +60,9 @@ static class IcebergIOReadTranslator implements TransformPayloadTranslator outputCollectionNames() { @Override public String identifier() { - return "beam:schematransform:org.apache.beam:iceberg_read:v1"; + return ManagedTransformConstants.ICEBERG_READ; } @DefaultSchema(AutoValueSchema.class) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java index b19ca1a5cfc4..e9ce6e653c81 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java @@ -22,6 +22,7 @@ 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.Schema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; @@ -82,7 +83,7 @@ public List outputCollectionNames() { @Override public String identifier() { - return "beam:schematransform:org.apache.beam:iceberg_write:v1"; + return ManagedTransformConstants.ICEBERG_WRITE; } @DefaultSchema(AutoValueSchema.class) diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslationTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslationTest.java index 139652ff26bf..00052808b24f 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslationTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslationTest.java @@ -33,6 +33,7 @@ import org.apache.beam.model.pipeline.v1.SchemaAwareTransforms; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.managed.ManagedTransformConstants; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.SchemaTranslation; @@ -133,13 +134,7 @@ public void testWriteTransformProtoTranslation() throws Exception { RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p); List writeTransformProto = pipelineProto.getComponents().getTransformsMap().values().stream() - .filter( - tr -> - tr.getSpec() - .getUrn() - .equals( - IcebergIOTranslation.IcebergIOWriteTranslator - .ICEBERG_WRITE_TRANSFORM_URN)) + .filter(tr -> tr.getSpec().getUrn().equals(ManagedTransformConstants.ICEBERG_WRITE)) .collect(Collectors.toList()); assertEquals(1, writeTransformProto.size()); RunnerApi.FunctionSpec spec = writeTransformProto.get(0).getSpec(); @@ -261,13 +256,7 @@ public void testReadTransformProtoTranslation() throws Exception { RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p); List readTransformProto = pipelineProto.getComponents().getTransformsMap().values().stream() - .filter( - tr -> - tr.getSpec() - .getUrn() - .equals( - IcebergIOTranslation.IcebergIOReadTranslator - .ICEBERG_READ_TRANSFORM_URN)) + .filter(tr -> tr.getSpec().getUrn().equals(ManagedTransformConstants.ICEBERG_READ)) .collect(Collectors.toList()); assertEquals(1, readTransformProto.size()); RunnerApi.FunctionSpec spec = readTransformProto.get(0).getSpec(); 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 d5eef3c6e0b6..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; @@ -80,13 +83,9 @@ public class Managed { // 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.ManagedTransform} transform for the specified source. The 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..eef0740fbca8 --- /dev/null +++ b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java @@ -0,0 +1,24 @@ +/* + * 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; + +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/ManagedTest.java b/sdks/java/managed/src/test/java/org/apache/beam/sdk/managed/ManagedTest.java index ceb71a06f33c..0f8fd5ce33c8 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 @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.managed; +import static org.apache.beam.sdk.managed.Managed.READ_TRANSFORMS; + import java.nio.file.Paths; import java.util.Arrays; import java.util.List; @@ -41,6 +43,7 @@ public class ManagedTest { @Test public void testInvalidTransform() { + System.out.println(READ_TRANSFORMS); thrown.expect(NullPointerException.class); thrown.expectMessage("An unsupported source was specified"); Managed.read("nonexistent-source"); From ea5a34da805d934bf91ee6e4588d2b867f154d94 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 18 Apr 2024 12:39:14 -0400 Subject: [PATCH 24/60] add documentation --- .../org/apache/beam/sdk/managed/ManagedTransformConstants.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java index eef0740fbca8..7631ffc8a18c 100644 --- a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java +++ b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java @@ -17,6 +17,9 @@ */ 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 = From 642da9262decedbdf86e3531f08241e6584bdf4e Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 18 Apr 2024 13:42:48 -0400 Subject: [PATCH 25/60] custom serialization for OneTableDynamicDestinations --- .../sdk/io/iceberg/IcebergIOTranslation.java | 27 +++--- .../IcebergWriteSchemaTransformProvider.java | 3 + .../iceberg/OneTableDynamicDestinations.java | 31 ++++++- .../iceberg/SchemaTransformCatalogConfig.java | 8 +- .../io/iceberg/IcebergIOTranslationTest.java | 93 +++++++++++++++++-- .../managed/ManagedTransformConstants.java | 4 +- sdks/standard_expansion_services.yaml | 3 + 7 files changed, 135 insertions(+), 34 deletions(-) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslation.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslation.java index 8b24061f49ee..74956843d51b 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslation.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslation.java @@ -26,11 +26,8 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InvalidClassException; -import java.util.Arrays; import java.util.HashMap; -import java.util.List; import java.util.Map; -import java.util.stream.Collectors; import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; import org.apache.beam.model.pipeline.v1.SchemaApi; import org.apache.beam.model.pipeline.v1.SchemaAwareTransforms.SchemaAwareTransformPayload; @@ -96,11 +93,7 @@ public Row toConfigRow(ReadRows transform) { toByteArray(new ExternalizableIcebergCatalogConfig(transform.getCatalogConfig()))); } if (transform.getTableIdentifier() != null) { - TableIdentifier identifier = transform.getTableIdentifier(); - List identifierParts = - Arrays.stream(identifier.namespace().levels()).collect(Collectors.toList()); - identifierParts.add(identifier.name()); - fieldValues.put("table_identifier", String.join(".", identifierParts)); + fieldValues.put("table_identifier", transform.getTableIdentifier().toString()); } return Row.withSchema(READ_SCHEMA).withFieldValues(fieldValues).build(); @@ -187,14 +180,18 @@ public Row toConfigRow(WriteRows transform) { toByteArray(new ExternalizableIcebergCatalogConfig(transform.getCatalogConfig()))); } if (transform.getTableIdentifier() != null) { - TableIdentifier identifier = transform.getTableIdentifier(); - List identifierParts = - Arrays.stream(identifier.namespace().levels()).collect(Collectors.toList()); - identifierParts.add(identifier.name()); - fieldValues.put("table_identifier", String.join(".", identifierParts)); + fieldValues.put("table_identifier", transform.getTableIdentifier().toString()); } - if (transform.getDynamicDestinations() != null) { - fieldValues.put("dynamic_destinations", toByteArray(transform.getDynamicDestinations())); + DynamicDestinations dynamicDestinations = transform.getDynamicDestinations(); + if (dynamicDestinations != null) { + System.out.println(dynamicDestinations.getClass()); + if (!dynamicDestinations.getClass().equals(OneTableDynamicDestinations.class)) { + throw new IllegalArgumentException( + String.format( + "Unsupported dynamic destinations class was found: %s. Translation is currently only supported for %s.", + dynamicDestinations.getClass(), OneTableDynamicDestinations.class)); + } + fieldValues.put("dynamic_destinations", toByteArray(dynamicDestinations)); } return Row.withSchema(WRITE_SCHEMA).withFieldValues(fieldValues).build(); diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java index e9ce6e653c81..95dff190bf6b 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java @@ -26,6 +26,7 @@ import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.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; @@ -93,8 +94,10 @@ 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 SchemaTransformCatalogConfig getCatalogConfig(); @AutoValue.Builder diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/OneTableDynamicDestinations.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/OneTableDynamicDestinations.java index f0931cb326a3..27d86ae41fb3 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/OneTableDynamicDestinations.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/OneTableDynamicDestinations.java @@ -17,25 +17,34 @@ */ 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.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; 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() { + Preconditions.checkState(!Strings.isNullOrEmpty(tableIdString)); if (tableId == null) { - tableId = TableIdentifier.parse(tableIdString); + tableId = TableIdentifier.parse(Preconditions.checkNotNull(tableIdString)); } return tableId; } @@ -62,4 +71,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/sdk/io/iceberg/SchemaTransformCatalogConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SchemaTransformCatalogConfig.java index 5c7422bbded2..719efb038d96 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SchemaTransformCatalogConfig.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SchemaTransformCatalogConfig.java @@ -23,6 +23,7 @@ import java.util.Set; import org.apache.beam.sdk.schemas.AutoValueSchema; 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.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; @@ -38,6 +39,7 @@ public static Builder builder() { public abstract String getCatalogName(); + @SchemaFieldDescription("Valid types are: {hadoop, hive, rest}") public abstract @Nullable String getCatalogType(); public abstract @Nullable String getCatalogImplementation(); @@ -58,7 +60,7 @@ public abstract static class Builder { public abstract SchemaTransformCatalogConfig build(); } - Set validTypes = + public static final Set VALID_CATALOG_TYPES = Sets.newHashSet( CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP, CatalogUtil.ICEBERG_CATALOG_TYPE_HIVE, @@ -67,9 +69,9 @@ public abstract static class Builder { public void validate() { if (Strings.isNullOrEmpty(getCatalogType())) { checkArgument( - validTypes.contains(Preconditions.checkArgumentNotNull(getCatalogType())), + VALID_CATALOG_TYPES.contains(Preconditions.checkArgumentNotNull(getCatalogType())), "Invalid catalog type. Please pick one of %s", - validTypes); + VALID_CATALOG_TYPES); } } } diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslationTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslationTest.java index 00052808b24f..d6d0624233ff 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslationTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslationTest.java @@ -44,10 +44,12 @@ import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.FileFormat; 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 IcebergIOTranslationTest { @@ -73,6 +75,8 @@ public class IcebergIOTranslationTest { @Rule public transient TestDataWarehouse warehouse = new TestDataWarehouse(TEMPORARY_FOLDER, "default"); + @Rule public transient ExpectedException thrown = ExpectedException.none(); + @Test public void testReCreateWriteTransformFromRow() { // setting a subset of fields here. @@ -82,8 +86,9 @@ public void testReCreateWriteTransformFromRow() { .setIcebergCatalogType(CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) .setWarehouseLocation(warehouse.location) .build(); + String tableId = "test_namespace.test_table"; IcebergIO.WriteRows writeTransform = - IcebergIO.writeRows(config).to(TableIdentifier.of("test_namespace", "test_table")); + IcebergIO.writeRows(config).to(TableIdentifier.parse(tableId)); IcebergIOTranslation.IcebergIOWriteTranslator translator = new IcebergIOTranslation.IcebergIOWriteTranslator(); @@ -92,9 +97,42 @@ public void testReCreateWriteTransformFromRow() { IcebergIO.WriteRows writeTransformFromRow = translator.fromConfigRow(row, PipelineOptionsFactory.create()); assertNotNull(writeTransformFromRow.getTableIdentifier()); + assertEquals(tableId, writeTransformFromRow.getTableIdentifier().toString()); + assertEquals("test_catalog", writeTransformFromRow.getCatalogConfig().getName()); + assertEquals( + CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP, + writeTransformFromRow.getCatalogConfig().getIcebergCatalogType()); assertEquals( - "test_namespace", writeTransformFromRow.getTableIdentifier().namespace().levels()[0]); - assertEquals("test_table", writeTransformFromRow.getTableIdentifier().name()); + warehouse.location, writeTransformFromRow.getCatalogConfig().getWarehouseLocation()); + } + + @Test + public void testReCreateWriteTransformWithOneTableDynamicDestinationsFromRow() { + // setting a subset of fields here. + IcebergCatalogConfig config = + IcebergCatalogConfig.builder() + .setName("test_catalog") + .setIcebergCatalogType(CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .setWarehouseLocation(warehouse.location) + .build(); + TableIdentifier tableIdentifier = TableIdentifier.of("test_namespace", "test_table"); + IcebergIO.WriteRows writeTransform = + IcebergIO.writeRows(config).to(DynamicDestinations.singleTable(tableIdentifier)); + + IcebergIOTranslation.IcebergIOWriteTranslator translator = + new IcebergIOTranslation.IcebergIOWriteTranslator(); + Row row = translator.toConfigRow(writeTransform); + + IcebergIO.WriteRows writeTransformFromRow = + translator.fromConfigRow(row, PipelineOptionsFactory.create()); + assertNull(writeTransformFromRow.getTableIdentifier()); + DynamicDestinations dynamicDestinations = writeTransformFromRow.getDynamicDestinations(); + + assertNotNull(dynamicDestinations); + assertEquals(OneTableDynamicDestinations.class, dynamicDestinations.getClass()); + assertEquals(Schema.builder().build(), dynamicDestinations.getMetadataSchema()); + assertEquals( + tableIdentifier, ((OneTableDynamicDestinations) dynamicDestinations).getTableIdentifier()); assertEquals("test_catalog", writeTransformFromRow.getCatalogConfig().getName()); assertEquals( CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP, @@ -103,6 +141,43 @@ public void testReCreateWriteTransformFromRow() { warehouse.location, writeTransformFromRow.getCatalogConfig().getWarehouseLocation()); } + @Test + public void testReCreateWriteTransformFromRowFailsWithUnsupportedDynamicDestinations() { + // setting a subset of fields here. + IcebergCatalogConfig config = IcebergCatalogConfig.builder().setName("test_catalog").build(); + TableIdentifier tableIdentifier = TableIdentifier.of("test_namespace", "test_table"); + IcebergIO.WriteRows writeTransform = + IcebergIO.writeRows(config) + .to( + new DynamicDestinations() { + @Override + public Schema getMetadataSchema() { + return Schema.builder().build(); + } + + @Override + public Row assignDestinationMetadata(Row data) { + return Row.nullRow(getMetadataSchema()); + } + + @Override + public IcebergDestination instantiateDestination(Row dest) { + return IcebergDestination.builder() + .setTableIdentifier(tableIdentifier) + .setTableCreateConfig(null) + .setFileFormat(FileFormat.PARQUET) + .build(); + } + }); + + IcebergIOTranslation.IcebergIOWriteTranslator translator = + new IcebergIOTranslation.IcebergIOWriteTranslator(); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("Unsupported dynamic destinations class was found"); + translator.toConfigRow(writeTransform); + } + @Test public void testWriteTransformProtoTranslation() throws Exception { // First build a pipeline @@ -162,7 +237,8 @@ public void testWriteTransformProtoTranslation() throws Exception { IcebergIO.WriteRows writeTransformFromSpec = translator.fromConfigRow(rowFromSpec, PipelineOptionsFactory.create()); - assertEquals(TableIdentifier.parse(identifier), writeTransformFromSpec.getTableIdentifier()); + assertNotNull(writeTransformFromSpec.getTableIdentifier()); + assertEquals(identifier, writeTransformFromSpec.getTableIdentifier().toString()); assertEquals(catalogConfig, writeTransformFromSpec.getCatalogConfig()); assertNull(writeTransformFromSpec.getDynamicDestinations()); } @@ -211,8 +287,9 @@ public void testReCreateReadTransformFromRow() { .setIcebergCatalogType(CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) .setWarehouseLocation(warehouse.location) .build(); - IcebergIO.ReadRows readTransform = - IcebergIO.readRows(config).from(TableIdentifier.of("test_namespace", "test_table")); + + String tableId = "test_namespace.test_table"; + IcebergIO.ReadRows readTransform = IcebergIO.readRows(config).from(TableIdentifier.of(tableId)); IcebergIOTranslation.IcebergIOReadTranslator translator = new IcebergIOTranslation.IcebergIOReadTranslator(); @@ -221,9 +298,7 @@ public void testReCreateReadTransformFromRow() { IcebergIO.ReadRows readTransformFromRow = translator.fromConfigRow(row, PipelineOptionsFactory.create()); assertNotNull(readTransformFromRow.getTableIdentifier()); - assertEquals( - "test_namespace", readTransformFromRow.getTableIdentifier().namespace().levels()[0]); - assertEquals("test_table", readTransformFromRow.getTableIdentifier().name()); + assertEquals(tableId, readTransformFromRow.getTableIdentifier().toString()); assertEquals("test_catalog", readTransformFromRow.getCatalogConfig().getName()); assertEquals( CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP, diff --git a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java index 7631ffc8a18c..48735d8c33a3 100644 --- a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java +++ b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java @@ -17,9 +17,7 @@ */ package org.apache.beam.sdk.managed; -/** - * This class contains constants for supported managed transform identifiers. - */ +/** 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 = 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' From 5a2fd879ad6687a315ece2ced6fda17c42f61473 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 18 Apr 2024 14:18:00 -0400 Subject: [PATCH 26/60] add iceberg via managed API tests; update proto doc --- .../pipeline/v1/schema_aware_transforms.proto | 6 +- ...cebergReadSchemaTransformProviderTest.java | 61 ++++++++++++++++++- ...ebergWriteSchemaTransformProviderTest.java | 38 ++++++++++++ .../ManagedSchemaTransformProvider.java | 4 +- 4 files changed, 102 insertions(+), 7 deletions(-) diff --git a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema_aware_transforms.proto b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema_aware_transforms.proto index 62c5f2da71bb..4c6c90e1b24f 100644 --- a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema_aware_transforms.proto +++ b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema_aware_transforms.proto @@ -51,9 +51,9 @@ message StandardSchemaAwareTransforms { // and is configured using Beam Schema-compatible parameters. // The information available in the payload can be used by runners to override the schema-aware transform. message SchemaAwareTransformPayload { - // The schema of the configuration row used to upgrade the transform + // The schema of the configuration row used to override the transform Schema expansion_schema = 1; - // The configuration used to build this transform. + // The configuration used to override this transform. // Must be compatible with the configuration schema, and decodable via beam:coder:row:v1. bytes expansion_payload = 2; } @@ -65,7 +65,7 @@ message ManagedSchemaTransformPayload { string underlying_transform_identifier = 1; // The managed transform's configuration Schema. Schema expansion_schema = 2; - // The configuration used to upgrade the managed transform. + // The configuration used to override the managed transform. // Must be compatible with the expansion schema, and decodable via beam:coder:row:v1. bytes expansion_payload = 3; // The underlying transform's configuration, represented as a YAML string. 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 index e712615270ac..3848eed99a18 100644 --- 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 @@ -17,13 +17,16 @@ */ 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.NoSuchSchemaException; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.SchemaRegistry; @@ -39,6 +42,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.yaml.snakeyaml.Yaml; public class IcebergReadSchemaTransformProviderTest { @@ -112,7 +116,62 @@ public void testSimpleScan() throws Exception { PCollection output = PCollectionRowTuple.empty(testPipeline) .apply(new IcebergReadSchemaTransformProvider().from(readConfig)) - .get(IcebergReadSchemaTransformProvider.OUTPUT_TAG); + .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( 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 index 981b28ee7c60..cdb91d2074c5 100644 --- 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 @@ -24,7 +24,9 @@ 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.schemas.NoSuchSchemaException; import org.apache.beam.sdk.schemas.SchemaRegistry; import org.apache.beam.sdk.testing.PAssert; @@ -47,6 +49,7 @@ 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 { @@ -119,6 +122,40 @@ public void testSimpleAppend() { 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; @@ -130,6 +167,7 @@ public VerifyOutputs(String identifier, String operation) { @Override public Void apply(Iterable input) { + System.out.println(input); Row row = input.iterator().next(); assertEquals(tableId, row.getString("table")); 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 15e2912cc524..d1ea52b59d5e 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 @@ -174,9 +174,7 @@ static class ManagedSchemaTransform extends SchemaTransform { @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { - SchemaTransform underlyingTransform = underlyingTransformProvider.from(transformConfig); - - return input.apply(underlyingTransform); + return input.apply(underlyingTransformProvider.from(transformConfig)); } public ManagedConfig getManagedConfig() { From 804fdacad2c4c40206c144d8d63a261320509a5f Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 18 Apr 2024 15:03:20 -0400 Subject: [PATCH 27/60] rename config; change test schematransform location --- .../sdk/io/iceberg/IcebergReadSchemaTransformProvider.java | 6 +++--- ...onfig.java => IcebergSchemaTransformCatalogConfig.java} | 6 +++--- .../io/iceberg/IcebergWriteSchemaTransformProvider.java | 6 +++--- .../beam/sdk/io/iceberg/OneTableDynamicDestinations.java | 1 - .../io/iceberg/IcebergReadSchemaTransformProviderTest.java | 6 +++--- .../iceberg/IcebergWriteSchemaTransformProviderTest.java | 4 ++-- .../managed/{ => testing}/TestSchemaTransformProvider.java | 7 ++++--- .../sdk/managed/ManagedSchemaTransformProviderTest.java | 2 ++ .../sdk/managed/ManagedSchemaTransformTranslationTest.java | 1 + .../test/java/org/apache/beam/sdk/managed/ManagedTest.java | 2 ++ 10 files changed, 23 insertions(+), 18 deletions(-) rename sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/{SchemaTransformCatalogConfig.java => IcebergSchemaTransformCatalogConfig.java} (93%) rename sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/{ => testing}/TestSchemaTransformProvider.java (93%) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java index 45833801f125..956109afc37b 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java @@ -68,13 +68,13 @@ public static Builder builder() { public abstract String getTable(); - public abstract SchemaTransformCatalogConfig getCatalogConfig(); + public abstract IcebergSchemaTransformCatalogConfig getCatalogConfig(); @AutoValue.Builder public abstract static class Builder { public abstract Builder setTable(String tables); - public abstract Builder setCatalogConfig(SchemaTransformCatalogConfig catalogConfig); + public abstract Builder setCatalogConfig(IcebergSchemaTransformCatalogConfig catalogConfig); public abstract Config build(); } @@ -93,7 +93,7 @@ private static class IcebergReadSchemaTransform extends SchemaTransform { @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { - SchemaTransformCatalogConfig catalogConfig = configuration.getCatalogConfig(); + IcebergSchemaTransformCatalogConfig catalogConfig = configuration.getCatalogConfig(); IcebergCatalogConfig.Builder catalogBuilder = IcebergCatalogConfig.builder().setName(catalogConfig.getCatalogName()); diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SchemaTransformCatalogConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformCatalogConfig.java similarity index 93% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SchemaTransformCatalogConfig.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformCatalogConfig.java index 719efb038d96..6c2af4df9a8e 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SchemaTransformCatalogConfig.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformCatalogConfig.java @@ -32,9 +32,9 @@ @DefaultSchema(AutoValueSchema.class) @AutoValue -public abstract class SchemaTransformCatalogConfig { +public abstract class IcebergSchemaTransformCatalogConfig { public static Builder builder() { - return new AutoValue_SchemaTransformCatalogConfig.Builder(); + return new AutoValue_IcebergSchemaTransformCatalogConfig.Builder(); } public abstract String getCatalogName(); @@ -57,7 +57,7 @@ public abstract static class Builder { public abstract Builder setWarehouseLocation(String warehouseLocation); - public abstract SchemaTransformCatalogConfig build(); + public abstract IcebergSchemaTransformCatalogConfig build(); } public static final Set VALID_CATALOG_TYPES = diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java index 95dff190bf6b..64eeb0844cc8 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java @@ -98,13 +98,13 @@ public static Builder builder() { public abstract String getTable(); @SchemaFieldDescription("Configuration parameters used to set up the Iceberg catalog.") - public abstract SchemaTransformCatalogConfig getCatalogConfig(); + public abstract IcebergSchemaTransformCatalogConfig getCatalogConfig(); @AutoValue.Builder public abstract static class Builder { public abstract Builder setTable(String tables); - public abstract Builder setCatalogConfig(SchemaTransformCatalogConfig catalogConfig); + public abstract Builder setCatalogConfig(IcebergSchemaTransformCatalogConfig catalogConfig); public abstract Config build(); } @@ -127,7 +127,7 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { PCollection rows = input.get(INPUT_TAG); - SchemaTransformCatalogConfig catalogConfig = configuration.getCatalogConfig(); + IcebergSchemaTransformCatalogConfig catalogConfig = configuration.getCatalogConfig(); IcebergCatalogConfig.Builder catalogBuilder = IcebergCatalogConfig.builder().setName(catalogConfig.getCatalogName()); diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/OneTableDynamicDestinations.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/OneTableDynamicDestinations.java index 27d86ae41fb3..4ca6db1f3755 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/OneTableDynamicDestinations.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/OneTableDynamicDestinations.java @@ -42,7 +42,6 @@ class OneTableDynamicDestinations implements DynamicDestinations, Externalizable @VisibleForTesting TableIdentifier getTableIdentifier() { - Preconditions.checkState(!Strings.isNullOrEmpty(tableIdString)); if (tableId == null) { tableId = TableIdentifier.parse(Preconditions.checkNotNull(tableIdString)); } 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 index 3848eed99a18..ed7756b09128 100644 --- 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 @@ -55,7 +55,7 @@ public class IcebergReadSchemaTransformProviderTest { @Test public void testBuildTransformWithRow() throws NoSuchSchemaException { Row catalogConfigRow = - Row.withSchema(SchemaRegistry.createDefault().getSchema(SchemaTransformCatalogConfig.class)) + Row.withSchema(SchemaRegistry.createDefault().getSchema(IcebergSchemaTransformCatalogConfig.class)) .withFieldValue("catalogName", "test_name") .withFieldValue("catalogType", "test_type") .withFieldValue("catalogImplementation", "testImplementation") @@ -100,8 +100,8 @@ public void testSimpleScan() throws Exception { .map(record -> SchemaAndRowConversions.recordToRow(schema, record)) .collect(Collectors.toList()); - SchemaTransformCatalogConfig catalogConfig = - SchemaTransformCatalogConfig.builder() + IcebergSchemaTransformCatalogConfig catalogConfig = + IcebergSchemaTransformCatalogConfig.builder() .setCatalogName("hadoop") .setCatalogType(CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) .setWarehouseLocation(warehouse.location) 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 index cdb91d2074c5..48aaf95b63a4 100644 --- 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 @@ -64,7 +64,7 @@ public class IcebergWriteSchemaTransformProviderTest { @Test public void testBuildTransformWithRow() throws NoSuchSchemaException { Row catalogConfigRow = - Row.withSchema(SchemaRegistry.createDefault().getSchema(SchemaTransformCatalogConfig.class)) + Row.withSchema(SchemaRegistry.createDefault().getSchema(IcebergSchemaTransformCatalogConfig.class)) .withFieldValue("catalogName", "test_name") .withFieldValue("catalogType", "test_type") .withFieldValue("catalogImplementation", "testImplementation") @@ -92,7 +92,7 @@ public void testSimpleAppend() { Config.builder() .setTable(identifier) .setCatalogConfig( - SchemaTransformCatalogConfig.builder() + IcebergSchemaTransformCatalogConfig.builder() .setCatalogName("hadoop") .setCatalogType(CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) .setWarehouseLocation(warehouse.location) diff --git a/sdks/java/managed/src/main/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 93% rename from sdks/java/managed/src/main/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 bc47fb9be763..0c072f5bf152 100644 --- a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/TestSchemaTransformProvider.java +++ b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/testing/TestSchemaTransformProvider.java @@ -15,10 +15,11 @@ * 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; +import org.apache.beam.sdk.managed.testing.AutoValue_TestSchemaTransformProvider_Config; import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.NoSuchSchemaException; import org.apache.beam.sdk.schemas.Schema; @@ -37,8 +38,8 @@ @AutoService(SchemaTransformProvider.class) public class TestSchemaTransformProvider extends TypedSchemaTransformProvider { - static final String IDENTIFIER = "beam:schematransform:org.apache.beam:test_transform:v1"; - static final Schema SCHEMA; + public static final String IDENTIFIER = "beam:schematransform:org.apache.beam:test_transform:v1"; + public static final Schema SCHEMA; static { try { 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 7c0b6b8a4667..c888000011e7 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,8 @@ 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; 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 index d530e5b841be..61340b8ff01a 100644 --- 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 @@ -35,6 +35,7 @@ import org.apache.beam.model.pipeline.v1.SchemaAwareTransforms; 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; 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 0f8fd5ce33c8..c80de978d4cd 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 @@ -23,6 +23,8 @@ 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; From c301cfcfd1e42be0469da9cbf9d42c7aeb7d220d Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 18 Apr 2024 15:09:38 -0400 Subject: [PATCH 28/60] spotless --- .../beam/sdk/io/iceberg/OneTableDynamicDestinations.java | 1 - .../sdk/io/iceberg/IcebergReadSchemaTransformProviderTest.java | 3 ++- .../io/iceberg/IcebergWriteSchemaTransformProviderTest.java | 3 ++- .../beam/sdk/managed/testing/TestSchemaTransformProvider.java | 1 - .../beam/sdk/managed/ManagedSchemaTransformProviderTest.java | 1 - .../src/test/java/org/apache/beam/sdk/managed/ManagedTest.java | 1 - sdks/standard_expansion_services.yaml | 2 ++ 7 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/OneTableDynamicDestinations.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/OneTableDynamicDestinations.java index 4ca6db1f3755..e09fdf171fd6 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/OneTableDynamicDestinations.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/OneTableDynamicDestinations.java @@ -25,7 +25,6 @@ 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.Strings; import org.apache.iceberg.FileFormat; import org.apache.iceberg.catalog.TableIdentifier; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; 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 index ed7756b09128..4d3794138a66 100644 --- 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 @@ -55,7 +55,8 @@ public class IcebergReadSchemaTransformProviderTest { @Test public void testBuildTransformWithRow() throws NoSuchSchemaException { Row catalogConfigRow = - Row.withSchema(SchemaRegistry.createDefault().getSchema(IcebergSchemaTransformCatalogConfig.class)) + Row.withSchema( + SchemaRegistry.createDefault().getSchema(IcebergSchemaTransformCatalogConfig.class)) .withFieldValue("catalogName", "test_name") .withFieldValue("catalogType", "test_type") .withFieldValue("catalogImplementation", "testImplementation") 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 index 48aaf95b63a4..095c3a76637c 100644 --- 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 @@ -64,7 +64,8 @@ public class IcebergWriteSchemaTransformProviderTest { @Test public void testBuildTransformWithRow() throws NoSuchSchemaException { Row catalogConfigRow = - Row.withSchema(SchemaRegistry.createDefault().getSchema(IcebergSchemaTransformCatalogConfig.class)) + Row.withSchema( + SchemaRegistry.createDefault().getSchema(IcebergSchemaTransformCatalogConfig.class)) .withFieldValue("catalogName", "test_name") .withFieldValue("catalogType", "test_type") .withFieldValue("catalogImplementation", "testImplementation") diff --git a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/testing/TestSchemaTransformProvider.java b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/testing/TestSchemaTransformProvider.java index 0c072f5bf152..f36b30d638be 100644 --- a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/testing/TestSchemaTransformProvider.java +++ b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/testing/TestSchemaTransformProvider.java @@ -19,7 +19,6 @@ import com.google.auto.service.AutoService; import com.google.auto.value.AutoValue; -import org.apache.beam.sdk.managed.testing.AutoValue_TestSchemaTransformProvider_Config; import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.NoSuchSchemaException; import org.apache.beam.sdk.schemas.Schema; 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 c888000011e7..0ce349e53285 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,7 +24,6 @@ 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; 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 c80de978d4cd..41dec0ddd86f 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 @@ -23,7 +23,6 @@ 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; diff --git a/sdks/standard_expansion_services.yaml b/sdks/standard_expansion_services.yaml index 31a1a6343aed..02f5044b2d1a 100644 --- a/sdks/standard_expansion_services.yaml +++ b/sdks/standard_expansion_services.yaml @@ -47,6 +47,8 @@ # Not ready to generate - 'beam:schematransform:org.apache.beam:iceberg_write:v1' - 'beam:schematransform:org.apache.beam:iceberg_read:v1' + # Not relevant + - 'beam:schematransform:org.apache.beam:test_transform:v1' # TODO(ahmedabu98): Enable this service in a future PR #- gradle_target: 'sdks:java:io:google-cloud-platform:expansion-service:shadowJar' From 841fd4393f7b47480c565a097952b576071ff720 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 18 Apr 2024 15:15:34 -0400 Subject: [PATCH 29/60] add missing package-info file --- .../sdk/managed/testing/package-info.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) create mode 100644 sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/testing/package-info.java 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..ab2cc5d8f399 --- /dev/null +++ b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/testing/package-info.java @@ -0,0 +1,19 @@ +/* + * 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.testing; From 928eeada13b46811f3191a199cf58795e3b93e2d Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 18 Apr 2024 15:23:54 -0400 Subject: [PATCH 30/60] spotless --- .../java/org/apache/beam/sdk/managed/testing/package-info.java | 1 + 1 file changed, 1 insertion(+) 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 index ab2cc5d8f399..e90e3ce8a94e 100644 --- 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 @@ -16,4 +16,5 @@ * limitations under the License. */ +/** Test transform for Managed API. */ package org.apache.beam.sdk.managed.testing; From 4f12cbdceb268e41cab095971764b7f5544772ad Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 18 Apr 2024 17:50:02 -0400 Subject: [PATCH 31/60] replace icebergIO translation with iceberg schematransform translation; fix Schema::sorted to do recursive sorting --- .../org/apache/beam/sdk/schemas/Schema.java | 12 +- .../apache/beam/sdk/schemas/SchemaTest.java | 35 ++ .../sdk/io/iceberg/IcebergIOTranslation.java | 241 ----------- .../IcebergReadSchemaTransformProvider.java | 17 +- .../IcebergSchemaTransformCatalogConfig.java | 27 ++ .../IcebergSchemaTransformTranslation.java | 157 +++++++ .../IcebergWriteSchemaTransformProvider.java | 18 +- .../io/iceberg/IcebergIOTranslationTest.java | 401 ------------------ ...IcebergSchemaTransformTranslationTest.java | 226 ++++++++++ 9 files changed, 484 insertions(+), 650 deletions(-) delete mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslation.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslation.java delete mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslationTest.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslationTest.java 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..451bc8831c4c 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 @@ -326,7 +326,7 @@ public static Schema of(Field... fields) { return Schema.builder().addFields(fields).build(); } - /** Returns an identical Schema with sorted fields. */ + /** Returns an identical Schema with 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 +336,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()); 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..e43527b819a8 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 @@ -232,6 +232,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/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslation.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslation.java deleted file mode 100644 index 74956843d51b..000000000000 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslation.java +++ /dev/null @@ -1,241 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.iceberg; - -import static org.apache.beam.sdk.io.iceberg.IcebergIO.ReadRows; -import static org.apache.beam.sdk.io.iceberg.IcebergIO.WriteRows; -import static org.apache.beam.sdk.util.construction.TransformUpgrader.fromByteArray; -import static org.apache.beam.sdk.util.construction.TransformUpgrader.toByteArray; - -import com.google.auto.service.AutoService; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.InvalidClassException; -import java.util.HashMap; -import java.util.Map; -import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; -import org.apache.beam.model.pipeline.v1.SchemaApi; -import org.apache.beam.model.pipeline.v1.SchemaAwareTransforms.SchemaAwareTransformPayload; -import org.apache.beam.sdk.coders.RowCoder; -import org.apache.beam.sdk.managed.ManagedTransformConstants; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.runners.AppliedPTransform; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.SchemaTranslation; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.construction.PTransformTranslation.TransformPayloadTranslator; -import org.apache.beam.sdk.util.construction.SdkComponents; -import org.apache.beam.sdk.util.construction.TransformPayloadTranslatorRegistrar; -import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.catalog.TableIdentifier; -import org.checkerframework.checker.nullness.qual.Nullable; - -@SuppressWarnings({"rawtypes", "nullness"}) -public class IcebergIOTranslation { - static class IcebergIOReadTranslator implements TransformPayloadTranslator { - - static final Schema READ_SCHEMA = - Schema.builder() - .addByteArrayField("catalog_config") - .addNullableStringField("table_identifier") - .build(); - - @Override - public String getUrn() { - return ManagedTransformConstants.ICEBERG_READ; - } - - @Override - public @Nullable FunctionSpec translate( - AppliedPTransform application, SdkComponents components) - throws IOException { - SchemaApi.Schema expansionSchema = SchemaTranslation.schemaToProto(READ_SCHEMA, true); - Row configRow = toConfigRow(application.getTransform()); - ByteArrayOutputStream os = new ByteArrayOutputStream(); - RowCoder.of(READ_SCHEMA).encode(configRow, os); - - return FunctionSpec.newBuilder() - .setUrn(getUrn()) - .setPayload( - SchemaAwareTransformPayload.newBuilder() - .setExpansionSchema(expansionSchema) - .setExpansionPayload(ByteString.copyFrom(os.toByteArray())) - .build() - .toByteString()) - .build(); - } - - @Override - public Row toConfigRow(ReadRows transform) { - - Map fieldValues = new HashMap<>(); - - if (transform.getCatalogConfig() != null) { - fieldValues.put( - "catalog_config", - toByteArray(new ExternalizableIcebergCatalogConfig(transform.getCatalogConfig()))); - } - if (transform.getTableIdentifier() != null) { - fieldValues.put("table_identifier", transform.getTableIdentifier().toString()); - } - - return Row.withSchema(READ_SCHEMA).withFieldValues(fieldValues).build(); - } - - @Override - public ReadRows fromConfigRow(Row configRow, PipelineOptions options) { - try { - ReadRows.Builder builder = new AutoValue_IcebergIO_ReadRows.Builder(); - - byte[] catalogBytes = configRow.getBytes("catalog_config"); - if (catalogBytes != null) { - builder = - builder.setCatalogConfig( - ((ExternalizableIcebergCatalogConfig) fromByteArray(catalogBytes)).get()); - } - String tableIdentifier = configRow.getString("table_identifier"); - if (tableIdentifier != null) { - builder = builder.setTableIdentifier(TableIdentifier.parse(tableIdentifier)); - } - return builder.build(); - } catch (InvalidClassException e) { - throw new RuntimeException(e); - } - } - } - - @AutoService(TransformPayloadTranslatorRegistrar.class) - public static class ReadRegistrar implements TransformPayloadTranslatorRegistrar { - @Override - @SuppressWarnings({ - "rawtypes", - }) - public Map, ? extends TransformPayloadTranslator> - getTransformPayloadTranslators() { - return ImmutableMap., TransformPayloadTranslator>builder() - .put(AutoValue_IcebergIO_ReadRows.class, new IcebergIOReadTranslator()) - .build(); - } - } - - static class IcebergIOWriteTranslator implements TransformPayloadTranslator { - - static final Schema WRITE_SCHEMA = - Schema.builder() - .addByteArrayField("catalog_config") - .addNullableStringField("table_identifier") - .addNullableByteArrayField("dynamic_destinations") - .build(); - - @Override - public String getUrn() { - return ManagedTransformConstants.ICEBERG_WRITE; - } - - @Override - public @Nullable FunctionSpec translate( - AppliedPTransform application, SdkComponents components) - throws IOException { - SchemaApi.Schema expansionSchema = SchemaTranslation.schemaToProto(WRITE_SCHEMA, true); - Row configRow = toConfigRow(application.getTransform()); - ByteArrayOutputStream os = new ByteArrayOutputStream(); - RowCoder.of(WRITE_SCHEMA).encode(configRow, os); - - return FunctionSpec.newBuilder() - .setUrn(getUrn()) - .setPayload( - SchemaAwareTransformPayload.newBuilder() - .setExpansionSchema(expansionSchema) - .setExpansionPayload(ByteString.copyFrom(os.toByteArray())) - .build() - .toByteString()) - .build(); - } - - @Override - public Row toConfigRow(WriteRows transform) { - - Map fieldValues = new HashMap<>(); - - if (transform.getCatalogConfig() != null) { - fieldValues.put( - "catalog_config", - toByteArray(new ExternalizableIcebergCatalogConfig(transform.getCatalogConfig()))); - } - if (transform.getTableIdentifier() != null) { - fieldValues.put("table_identifier", transform.getTableIdentifier().toString()); - } - DynamicDestinations dynamicDestinations = transform.getDynamicDestinations(); - if (dynamicDestinations != null) { - System.out.println(dynamicDestinations.getClass()); - if (!dynamicDestinations.getClass().equals(OneTableDynamicDestinations.class)) { - throw new IllegalArgumentException( - String.format( - "Unsupported dynamic destinations class was found: %s. Translation is currently only supported for %s.", - dynamicDestinations.getClass(), OneTableDynamicDestinations.class)); - } - fieldValues.put("dynamic_destinations", toByteArray(dynamicDestinations)); - } - - return Row.withSchema(WRITE_SCHEMA).withFieldValues(fieldValues).build(); - } - - @Override - public WriteRows fromConfigRow(Row configRow, PipelineOptions options) { - try { - IcebergIO.WriteRows.Builder builder = new AutoValue_IcebergIO_WriteRows.Builder(); - - byte[] catalogBytes = configRow.getBytes("catalog_config"); - if (catalogBytes != null) { - builder = - builder.setCatalogConfig( - ((ExternalizableIcebergCatalogConfig) fromByteArray(catalogBytes)).get()); - } - String tableIdentifier = configRow.getString("table_identifier"); - if (tableIdentifier != null) { - builder = builder.setTableIdentifier(TableIdentifier.parse(tableIdentifier)); - } - byte[] dynamicDestinationsBytes = configRow.getBytes("dynamic_destinations"); - if (dynamicDestinationsBytes != null) { - builder = - builder.setDynamicDestinations( - (DynamicDestinations) fromByteArray(dynamicDestinationsBytes)); - } - return builder.build(); - } catch (InvalidClassException e) { - throw new RuntimeException(e); - } - } - } - - @AutoService(TransformPayloadTranslatorRegistrar.class) - public static class WriteRegistrar implements TransformPayloadTranslatorRegistrar { - @Override - @SuppressWarnings({ - "rawtypes", - }) - public Map, ? extends TransformPayloadTranslator> - getTransformPayloadTranslators() { - return ImmutableMap., TransformPayloadTranslator>builder() - .put(AutoValue_IcebergIO_WriteRows.class, new IcebergIOWriteTranslator()) - .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 index 956109afc37b..32c3d39d0950 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java @@ -24,6 +24,7 @@ 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.Schema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; @@ -46,7 +47,7 @@ public class IcebergReadSchemaTransformProvider extends TypedSchemaTransformProv @Override protected SchemaTransform from(Config configuration) { configuration.validate(); - return new IcebergReadSchemaTransform(configuration); + return new IcebergReadSchemaTransform(configuration, configurationSchema()); } @Override @@ -84,11 +85,21 @@ public void validate() { } } - private static class IcebergReadSchemaTransform extends SchemaTransform { + static class IcebergReadSchemaTransform extends SchemaTransform { private final Config configuration; + private final Row configurationRow; - IcebergReadSchemaTransform(Config configuration) { + IcebergReadSchemaTransform(Config configuration, Schema configSchema) { this.configuration = configuration; + configurationRow = + Row.withSchema(configSchema) + .withFieldValue("table", configuration.getTable()) + .withFieldValue("catalogConfig", configuration.getCatalogConfig().toRow()) + .build(); + } + + Row getConfigurationRow() { + return configurationRow; } @Override 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 index 6c2af4df9a8e..0acdecc6866d 100644 --- 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 @@ -22,9 +22,13 @@ 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; @@ -60,6 +64,29 @@ public abstract static class Builder { public abstract IcebergSchemaTransformCatalogConfig build(); } + public static final Schema SCHEMA; + + static { + try { + SCHEMA = + SchemaRegistry.createDefault() + .getSchema(IcebergSchemaTransformCatalogConfig.class) + .sorted(); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e); + } + } + + @SuppressWarnings("argument") + public Row toRow() { + return Row.withSchema(SCHEMA) + .withFieldValue("catalogName", getCatalogName()) + .withFieldValue("catalogType", getCatalogType()) + .withFieldValue("catalogImplementation", getCatalogImplementation()) + .withFieldValue("warehouseLocation", getWarehouseLocation()) + .build(); + } + public static final Set VALID_CATALOG_TYPES = Sets.newHashSet( CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP, 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..6f23c664d910 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslation.java @@ -0,0 +1,157 @@ +/* + * 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 com.google.auto.service.AutoService; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.Map; +import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; +import org.apache.beam.model.pipeline.v1.SchemaApi; +import org.apache.beam.model.pipeline.v1.SchemaAwareTransforms.SchemaAwareTransformPayload; +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.transforms.PTransform; +import org.apache.beam.sdk.util.construction.PTransformTranslation.TransformPayloadTranslator; +import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.util.construction.TransformPayloadTranslatorRegistrar; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.checkerframework.checker.nullness.qual.Nullable; + +@SuppressWarnings({"rawtypes", "nullness"}) +public class IcebergSchemaTransformTranslation { + static class IcebergReadSchemaTransformTranslator + implements TransformPayloadTranslator { + static final IcebergReadSchemaTransformProvider READ_PROVIDER = + new IcebergReadSchemaTransformProvider(); + + @Override + public String getUrn() { + return READ_PROVIDER.identifier(); + } + + @Override + public @Nullable FunctionSpec translate( + AppliedPTransform application, SdkComponents components) + throws IOException { + SchemaApi.Schema expansionSchema = + SchemaTranslation.schemaToProto(READ_PROVIDER.configurationSchema(), true); + Row configRow = toConfigRow(application.getTransform()); + ByteArrayOutputStream os = new ByteArrayOutputStream(); + RowCoder.of(READ_PROVIDER.configurationSchema()).encode(configRow, os); + + return FunctionSpec.newBuilder() + .setUrn(getUrn()) + .setPayload( + SchemaAwareTransformPayload.newBuilder() + .setExpansionSchema(expansionSchema) + .setExpansionPayload(ByteString.copyFrom(os.toByteArray())) + .build() + .toByteString()) + .build(); + } + + @Override + public Row toConfigRow(IcebergReadSchemaTransform transform) { + return transform.getConfigurationRow(); + } + + @Override + public IcebergReadSchemaTransform fromConfigRow(Row configRow, PipelineOptions options) { + return (IcebergReadSchemaTransform) READ_PROVIDER.from(configRow); + } + } + + @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 + implements TransformPayloadTranslator { + + static final IcebergWriteSchemaTransformProvider WRITE_PROVIDER = + new IcebergWriteSchemaTransformProvider(); + + @Override + public String getUrn() { + return WRITE_PROVIDER.identifier(); + } + + @Override + public @Nullable FunctionSpec translate( + AppliedPTransform application, SdkComponents components) + throws IOException { + SchemaApi.Schema expansionSchema = + SchemaTranslation.schemaToProto(WRITE_PROVIDER.configurationSchema(), true); + Row configRow = toConfigRow(application.getTransform()); + ByteArrayOutputStream os = new ByteArrayOutputStream(); + RowCoder.of(WRITE_PROVIDER.configurationSchema()).encode(configRow, os); + + return FunctionSpec.newBuilder() + .setUrn(getUrn()) + .setPayload( + SchemaAwareTransformPayload.newBuilder() + .setExpansionSchema(expansionSchema) + .setExpansionPayload(ByteString.copyFrom(os.toByteArray())) + .build() + .toByteString()) + .build(); + } + + @Override + public Row toConfigRow(IcebergWriteSchemaTransform transform) { + return transform.getConfigurationRow(); + } + + @Override + public IcebergWriteSchemaTransform fromConfigRow(Row configRow, PipelineOptions options) { + return (IcebergWriteSchemaTransform) WRITE_PROVIDER.from(configRow); + } + } + + @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/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java index 64eeb0844cc8..fc257e964259 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java @@ -69,7 +69,7 @@ public String description() { @Override protected SchemaTransform from(Config configuration) { configuration.validate(); - return new IcebergWriteSchemaTransform(configuration); + return new IcebergWriteSchemaTransform(configuration, configurationSchema()); } @Override @@ -114,12 +114,22 @@ public void validate() { } } - @VisibleForTesting - private static class IcebergWriteSchemaTransform extends SchemaTransform { + static class IcebergWriteSchemaTransform extends SchemaTransform { private final Config configuration; + private final Row configurationRow; - IcebergWriteSchemaTransform(Config configuration) { + IcebergWriteSchemaTransform(Config configuration, Schema configSchema) { this.configuration = configuration; + + configurationRow = + Row.withSchema(configSchema) + .withFieldValue("table", configuration.getTable()) + .withFieldValue("catalogConfig", configuration.getCatalogConfig().toRow()) + .build(); + } + + Row getConfigurationRow() { + return configurationRow; } @Override diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslationTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslationTest.java deleted file mode 100644 index d6d0624233ff..000000000000 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOTranslationTest.java +++ /dev/null @@ -1,401 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.iceberg; - -import static org.apache.beam.sdk.io.iceberg.IcebergIOTranslation.IcebergIOReadTranslator.READ_SCHEMA; -import static org.apache.beam.sdk.io.iceberg.IcebergIOTranslation.IcebergIOWriteTranslator.WRITE_SCHEMA; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; - -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.stream.Collectors; -import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.model.pipeline.v1.SchemaAwareTransforms; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.RowCoder; -import org.apache.beam.sdk.managed.ManagedTransformConstants; -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.PipelineTranslation; -import org.apache.beam.sdk.util.construction.TransformUpgrader; -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.collect.ImmutableMap; -import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.FileFormat; -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 IcebergIOTranslationTest { - // A mapping from WriteRows transform builder methods to the corresponding schema fields in - // IcebergIOTranslation. - static final Map WRITE_TRANSFORM_SCHEMA_MAPPING = - ImmutableMap.builder() - .put("getCatalogConfig", "catalog_config") - .put("getTableIdentifier", "table_identifier") - .put("getDynamicDestinations", "dynamic_destinations") - .build(); - - // A mapping from ReadRows transform builder methods to the corresponding schema fields in - // IcebergIOTranslation. - static final Map READ_TRANSFORM_SCHEMA_MAPPING = - ImmutableMap.builder() - .put("getCatalogConfig", "catalog_config") - .put("getTableIdentifier", "table_identifier") - .build(); - - @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(); - - @Test - public void testReCreateWriteTransformFromRow() { - // setting a subset of fields here. - IcebergCatalogConfig config = - IcebergCatalogConfig.builder() - .setName("test_catalog") - .setIcebergCatalogType(CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) - .setWarehouseLocation(warehouse.location) - .build(); - String tableId = "test_namespace.test_table"; - IcebergIO.WriteRows writeTransform = - IcebergIO.writeRows(config).to(TableIdentifier.parse(tableId)); - - IcebergIOTranslation.IcebergIOWriteTranslator translator = - new IcebergIOTranslation.IcebergIOWriteTranslator(); - Row row = translator.toConfigRow(writeTransform); - - IcebergIO.WriteRows writeTransformFromRow = - translator.fromConfigRow(row, PipelineOptionsFactory.create()); - assertNotNull(writeTransformFromRow.getTableIdentifier()); - assertEquals(tableId, writeTransformFromRow.getTableIdentifier().toString()); - assertEquals("test_catalog", writeTransformFromRow.getCatalogConfig().getName()); - assertEquals( - CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP, - writeTransformFromRow.getCatalogConfig().getIcebergCatalogType()); - assertEquals( - warehouse.location, writeTransformFromRow.getCatalogConfig().getWarehouseLocation()); - } - - @Test - public void testReCreateWriteTransformWithOneTableDynamicDestinationsFromRow() { - // setting a subset of fields here. - IcebergCatalogConfig config = - IcebergCatalogConfig.builder() - .setName("test_catalog") - .setIcebergCatalogType(CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) - .setWarehouseLocation(warehouse.location) - .build(); - TableIdentifier tableIdentifier = TableIdentifier.of("test_namespace", "test_table"); - IcebergIO.WriteRows writeTransform = - IcebergIO.writeRows(config).to(DynamicDestinations.singleTable(tableIdentifier)); - - IcebergIOTranslation.IcebergIOWriteTranslator translator = - new IcebergIOTranslation.IcebergIOWriteTranslator(); - Row row = translator.toConfigRow(writeTransform); - - IcebergIO.WriteRows writeTransformFromRow = - translator.fromConfigRow(row, PipelineOptionsFactory.create()); - assertNull(writeTransformFromRow.getTableIdentifier()); - DynamicDestinations dynamicDestinations = writeTransformFromRow.getDynamicDestinations(); - - assertNotNull(dynamicDestinations); - assertEquals(OneTableDynamicDestinations.class, dynamicDestinations.getClass()); - assertEquals(Schema.builder().build(), dynamicDestinations.getMetadataSchema()); - assertEquals( - tableIdentifier, ((OneTableDynamicDestinations) dynamicDestinations).getTableIdentifier()); - assertEquals("test_catalog", writeTransformFromRow.getCatalogConfig().getName()); - assertEquals( - CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP, - writeTransformFromRow.getCatalogConfig().getIcebergCatalogType()); - assertEquals( - warehouse.location, writeTransformFromRow.getCatalogConfig().getWarehouseLocation()); - } - - @Test - public void testReCreateWriteTransformFromRowFailsWithUnsupportedDynamicDestinations() { - // setting a subset of fields here. - IcebergCatalogConfig config = IcebergCatalogConfig.builder().setName("test_catalog").build(); - TableIdentifier tableIdentifier = TableIdentifier.of("test_namespace", "test_table"); - IcebergIO.WriteRows writeTransform = - IcebergIO.writeRows(config) - .to( - new DynamicDestinations() { - @Override - public Schema getMetadataSchema() { - return Schema.builder().build(); - } - - @Override - public Row assignDestinationMetadata(Row data) { - return Row.nullRow(getMetadataSchema()); - } - - @Override - public IcebergDestination instantiateDestination(Row dest) { - return IcebergDestination.builder() - .setTableIdentifier(tableIdentifier) - .setTableCreateConfig(null) - .setFileFormat(FileFormat.PARQUET) - .build(); - } - }); - - IcebergIOTranslation.IcebergIOWriteTranslator translator = - new IcebergIOTranslation.IcebergIOWriteTranslator(); - - thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("Unsupported dynamic destinations class was found"); - translator.toConfigRow(writeTransform); - } - - @Test - public void testWriteTransformProtoTranslation() 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); - - IcebergCatalogConfig catalogConfig = - IcebergCatalogConfig.builder() - .setName("test_catalog") - .setIcebergCatalogType(CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) - .setWarehouseLocation(warehouse.location) - .build(); - String identifier = "default.table_" + Long.toString(UUID.randomUUID().hashCode(), 16); - - IcebergIO.WriteRows writeTransform = - IcebergIO.writeRows(catalogConfig).to(TableIdentifier.parse(identifier)); - - input.apply(writeTransform); - - // Then translate the pipeline to a proto and extract IcebergIO.WriteRows proto - RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p); - List writeTransformProto = - pipelineProto.getComponents().getTransformsMap().values().stream() - .filter(tr -> tr.getSpec().getUrn().equals(ManagedTransformConstants.ICEBERG_WRITE)) - .collect(Collectors.toList()); - assertEquals(1, writeTransformProto.size()); - RunnerApi.FunctionSpec spec = writeTransformProto.get(0).getSpec(); - - // Check that the proto contains correct values - SchemaAwareTransforms.SchemaAwareTransformPayload payload = - SchemaAwareTransforms.SchemaAwareTransformPayload.parseFrom(spec.getPayload()); - Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getExpansionSchema()); - assertEquals(WRITE_SCHEMA, schemaFromSpec); - Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getExpansionPayload().newInput()); - Row expectedRow = - Row.withSchema(WRITE_SCHEMA) - .withFieldValue("table_identifier", identifier) - .withFieldValue( - "catalog_config", - TransformUpgrader.toByteArray( - new ExternalizableIcebergCatalogConfig(catalogConfig))) - .withFieldValue("dynamic_destinations", null) - .build(); - assertEquals(expectedRow, rowFromSpec); - - // Use the information in the proto to recreate the IcebergIO.WriteRows transform - IcebergIOTranslation.IcebergIOWriteTranslator translator = - new IcebergIOTranslation.IcebergIOWriteTranslator(); - IcebergIO.WriteRows writeTransformFromSpec = - translator.fromConfigRow(rowFromSpec, PipelineOptionsFactory.create()); - - assertNotNull(writeTransformFromSpec.getTableIdentifier()); - assertEquals(identifier, writeTransformFromSpec.getTableIdentifier().toString()); - assertEquals(catalogConfig, writeTransformFromSpec.getCatalogConfig()); - assertNull(writeTransformFromSpec.getDynamicDestinations()); - } - - @Test - public void testWriteTransformRowIncludesAllFields() { - List getMethodNames = - Arrays.stream(IcebergIO.WriteRows.class.getDeclaredMethods()) - .map(method -> method.getName()) - .filter(methodName -> methodName.startsWith("get")) - .collect(Collectors.toList()); - - // Just to make sure that this does not pass trivially. - assertTrue(getMethodNames.size() > 0); - - for (String getMethodName : getMethodNames) { - assertTrue( - "Method " - + getMethodName - + " will not be tracked when upgrading the 'IcebergIO.WriteRows' transform. Please update" - + "'IcebergIOTranslation.IcebergIOWriteTranslator' to track the new method " - + "and update this test.", - WRITE_TRANSFORM_SCHEMA_MAPPING.keySet().contains(getMethodName)); - } - - // Confirming that all fields mentioned in `WRITE_TRANSFORM_SCHEMA_MAPPING` are - // actually available in the schema. - WRITE_TRANSFORM_SCHEMA_MAPPING.values().stream() - .forEach( - fieldName -> { - assertTrue( - "Field name " - + fieldName - + " was not found in the transform schema defined in " - + "IcebergIOTranslation.IcebergIOWriteTranslator.", - WRITE_SCHEMA.getFieldNames().contains(fieldName)); - }); - } - - @Test - public void testReCreateReadTransformFromRow() { - // setting a subset of fields here. - IcebergCatalogConfig config = - IcebergCatalogConfig.builder() - .setName("test_catalog") - .setIcebergCatalogType(CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) - .setWarehouseLocation(warehouse.location) - .build(); - - String tableId = "test_namespace.test_table"; - IcebergIO.ReadRows readTransform = IcebergIO.readRows(config).from(TableIdentifier.of(tableId)); - - IcebergIOTranslation.IcebergIOReadTranslator translator = - new IcebergIOTranslation.IcebergIOReadTranslator(); - Row row = translator.toConfigRow(readTransform); - - IcebergIO.ReadRows readTransformFromRow = - translator.fromConfigRow(row, PipelineOptionsFactory.create()); - assertNotNull(readTransformFromRow.getTableIdentifier()); - assertEquals(tableId, readTransformFromRow.getTableIdentifier().toString()); - assertEquals("test_catalog", readTransformFromRow.getCatalogConfig().getName()); - assertEquals( - CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP, - readTransformFromRow.getCatalogConfig().getIcebergCatalogType()); - assertEquals( - warehouse.location, readTransformFromRow.getCatalogConfig().getWarehouseLocation()); - } - - @Test - public void testReadTransformProtoTranslation() throws Exception { - // First build a pipeline - Pipeline p = Pipeline.create(); - - IcebergCatalogConfig catalogConfig = - IcebergCatalogConfig.builder() - .setName("test_catalog") - .setIcebergCatalogType(CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) - .setWarehouseLocation(warehouse.location) - .build(); - String identifier = "default.table_" + Long.toString(UUID.randomUUID().hashCode(), 16); - TableIdentifier tableId = TableIdentifier.parse(identifier); - - warehouse.createTable(tableId, TestFixtures.SCHEMA); - - IcebergIO.ReadRows readTransform = IcebergIO.readRows(catalogConfig).from(tableId); - - p.apply(readTransform); - - // Then translate the pipeline to a proto and extract IcebergIO.ReadRows proto - RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p); - List readTransformProto = - pipelineProto.getComponents().getTransformsMap().values().stream() - .filter(tr -> tr.getSpec().getUrn().equals(ManagedTransformConstants.ICEBERG_READ)) - .collect(Collectors.toList()); - assertEquals(1, readTransformProto.size()); - RunnerApi.FunctionSpec spec = readTransformProto.get(0).getSpec(); - - // Check that the proto contains correct values - SchemaAwareTransforms.SchemaAwareTransformPayload payload = - SchemaAwareTransforms.SchemaAwareTransformPayload.parseFrom(spec.getPayload()); - Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getExpansionSchema()); - assertEquals(READ_SCHEMA, schemaFromSpec); - Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getExpansionPayload().newInput()); - Row expectedRow = - Row.withSchema(READ_SCHEMA) - .withFieldValue("table_identifier", identifier) - .withFieldValue( - "catalog_config", - TransformUpgrader.toByteArray( - new ExternalizableIcebergCatalogConfig(catalogConfig))) - .build(); - assertEquals(expectedRow, rowFromSpec); - - // Use the information in the proto to recreate the IcebergIO.ReadRows transform - IcebergIOTranslation.IcebergIOReadTranslator translator = - new IcebergIOTranslation.IcebergIOReadTranslator(); - IcebergIO.ReadRows readTransformFromSpec = - translator.fromConfigRow(rowFromSpec, PipelineOptionsFactory.create()); - - assertEquals(tableId, readTransformFromSpec.getTableIdentifier()); - assertEquals(catalogConfig, readTransformFromSpec.getCatalogConfig()); - } - - @Test - public void testReadTransformRowIncludesAllFields() { - List getMethodNames = - Arrays.stream(IcebergIO.ReadRows.class.getDeclaredMethods()) - .map(method -> method.getName()) - .filter(methodName -> methodName.startsWith("get")) - .collect(Collectors.toList()); - - // Just to make sure that this does not pass trivially. - assertTrue(getMethodNames.size() > 0); - - for (String getMethodName : getMethodNames) { - assertTrue( - "Method " - + getMethodName - + " will not be tracked when upgrading the 'IcebergIO.ReadRows' transform. Please update" - + "'IcebergIOTranslation.IcebergIOReadTranslator' to track the new method " - + "and update this test.", - READ_TRANSFORM_SCHEMA_MAPPING.keySet().contains(getMethodName)); - } - - // Confirming that all fields mentioned in `WRITE_TRANSFORM_SCHEMA_MAPPING` are - // actually available in the schema. - READ_TRANSFORM_SCHEMA_MAPPING.values().stream() - .forEach( - fieldName -> { - assertTrue( - "Field name " - + fieldName - + " was not found in the transform schema defined in " - + "IcebergIOTranslation.IcebergIOReadTranslator.", - IcebergIOTranslation.IcebergIOReadTranslator.READ_SCHEMA - .getFieldNames() - .contains(fieldName)); - }); - } -} 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..b05ee9288218 --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslationTest.java @@ -0,0 +1,226 @@ +/* + * 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.INPUT_TAG; +import static org.apache.beam.sdk.io.iceberg.IcebergWriteSchemaTransformProvider.IcebergWriteSchemaTransform; +import static org.junit.Assert.assertEquals; + +import java.util.Collections; +import java.util.List; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.model.pipeline.v1.SchemaAwareTransforms; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.managed.ManagedTransformConstants; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.schemas.NoSuchSchemaException; +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.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.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() throws NoSuchSchemaException { + Row catalogConfigRow = + Row.withSchema(IcebergSchemaTransformCatalogConfig.SCHEMA) + .withFieldValue("catalogName", "test_name") + .withFieldValue("catalogType", "test_type") + .withFieldValue("catalogImplementation", "testImplementation") + .withFieldValue("warehouseLocation", "test_location") + .build(); + Row transformConfigRow = + Row.withSchema(WRITE_PROVIDER.configurationSchema()) + .withFieldValue("table", "test_table_identifier") + .withFieldValue("catalogConfig", 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 Exception { + // 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("catalogName", "test_catalog") + .withFieldValue("catalogType", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .withFieldValue("catalogImplementation", "test_implementation") + .withFieldValue("warehouseLocation", warehouse.location) + .build(); + Row transformConfigRow = + Row.withSchema(WRITE_PROVIDER.configurationSchema()) + .withFieldValue("table", "test_identifier") + .withFieldValue("catalogConfig", 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 -> tr.getSpec().getUrn().equals(ManagedTransformConstants.ICEBERG_WRITE)) + .collect(Collectors.toList()); + assertEquals(1, writeTransformProto.size()); + RunnerApi.FunctionSpec spec = writeTransformProto.get(0).getSpec(); + + // Check that the proto contains correct values + SchemaAwareTransforms.SchemaAwareTransformPayload payload = + SchemaAwareTransforms.SchemaAwareTransformPayload.parseFrom(spec.getPayload()); + Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getExpansionSchema()); + assertEquals(WRITE_PROVIDER.configurationSchema(), schemaFromSpec); + System.out.println( + "spec: " + schemaFromSpec.getField("catalogConfig").getType().getRowSchema()); + Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getExpansionPayload().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() throws NoSuchSchemaException { + // setting a subset of fields here. + Row catalogConfigRow = + Row.withSchema(IcebergSchemaTransformCatalogConfig.SCHEMA) + .withFieldValue("catalogName", "test_name") + .withFieldValue("catalogType", "test_type") + .withFieldValue("catalogImplementation", "testImplementation") + .withFieldValue("warehouseLocation", "test_location") + .build(); + Row transformConfigRow = + Row.withSchema(READ_PROVIDER.configurationSchema()) + .withFieldValue("table", "test_table_identifier") + .withFieldValue("catalogConfig", 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 Exception { + // First build a pipeline + Pipeline p = Pipeline.create(); + Row catalogConfigRow = + Row.withSchema(IcebergSchemaTransformCatalogConfig.SCHEMA) + .withFieldValue("catalogName", "test_catalog") + .withFieldValue("catalogType", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .withFieldValue("warehouseLocation", 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("catalogConfig", 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 -> tr.getSpec().getUrn().equals(ManagedTransformConstants.ICEBERG_READ)) + .collect(Collectors.toList()); + assertEquals(1, readTransformProto.size()); + RunnerApi.FunctionSpec spec = readTransformProto.get(0).getSpec(); + + // Check that the proto contains correct values + SchemaAwareTransforms.SchemaAwareTransformPayload payload = + SchemaAwareTransforms.SchemaAwareTransformPayload.parseFrom(spec.getPayload()); + Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getExpansionSchema()); + assertEquals(READ_PROVIDER.configurationSchema(), schemaFromSpec); + Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getExpansionPayload().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()); + } +} From 9b26b18909baf4d7a9be7164337de30ae4fb4343 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 18 Apr 2024 17:51:27 -0400 Subject: [PATCH 32/60] remove ExternalizableIcebergCatalogConfig (no longer needed) --- .../ExternalizableIcebergCatalogConfig.java | 139 ------------------ ...xternalizableIcebergCatalogConfigTest.java | 62 -------- 2 files changed, 201 deletions(-) delete mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ExternalizableIcebergCatalogConfig.java delete mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ExternalizableIcebergCatalogConfigTest.java diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ExternalizableIcebergCatalogConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ExternalizableIcebergCatalogConfig.java deleted file mode 100644 index 318198b41964..000000000000 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ExternalizableIcebergCatalogConfig.java +++ /dev/null @@ -1,139 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.iceberg; - -import java.io.Externalizable; -import java.io.IOException; -import java.io.ObjectInput; -import java.io.ObjectOutput; -import java.util.HashMap; -import java.util.Map; -import org.apache.beam.sdk.io.hadoop.SerializableConfiguration; -import org.checkerframework.checker.nullness.qual.Nullable; - -/** - * A wrapper to more precisely serialize a {@link IcebergCatalogConfig} object. {@link - * IcebergCatalogConfig} is an AutoValue class, which raises some complications when trying to have - * it directly implement {@link Externalizable}. Hence, this class is used to wrap around the {@link - * IcebergCatalogConfig} object when serializing and deserializing. * - */ -public class ExternalizableIcebergCatalogConfig implements Externalizable { - private static final long serialVersionUID = 0L; - - private @Nullable IcebergCatalogConfig catalogConfig; - - public ExternalizableIcebergCatalogConfig() {} - - public ExternalizableIcebergCatalogConfig(@Nullable IcebergCatalogConfig catalogConfig) { - if (catalogConfig == null) { - throw new NullPointerException("Configuration must not be null."); - } - this.catalogConfig = catalogConfig; - } - - public @Nullable IcebergCatalogConfig get() { - return catalogConfig; - } - - @Override - @SuppressWarnings("nullness") - public void writeExternal(ObjectOutput out) throws IOException { - Map properties = new HashMap<>(); - properties.put("name", catalogConfig.getName()); - properties.put("icebergCatalogType", catalogConfig.getIcebergCatalogType()); - properties.put("catalogImplementation", catalogConfig.getCatalogImplementation()); - properties.put("fileIOImplementation", catalogConfig.getFileIOImplementation()); - properties.put("warehouseLocation", catalogConfig.getWarehouseLocation()); - properties.put( - "metricsReporterImplementation", catalogConfig.getMetricsReporterImplementation()); - properties.put("cacheEnabled", catalogConfig.getCacheEnabled()); - properties.put("cacheCaseSensitive", catalogConfig.getCacheCaseSensitive()); - properties.put( - "cacheExpirationIntervalMillis", catalogConfig.getCacheExpirationIntervalMillis()); - properties.put("ioManifestCacheEnabled", catalogConfig.getIOManifestCacheEnabled()); - properties.put( - "ioManifestCacheExpirationIntervalMillis", - catalogConfig.getIOManifestCacheExpirationIntervalMillis()); - properties.put("ioManifestCacheMaxTotalBytes", catalogConfig.getIOManifestCacheMaxTotalBytes()); - properties.put( - "ioManifestCacheMaxContentLength", catalogConfig.getIOManifestCacheMaxContentLength()); - properties.put("uri", catalogConfig.getUri()); - properties.put("clientPoolSize", catalogConfig.getClientPoolSize()); - properties.put("clientPoolEvictionIntervalMs", catalogConfig.getClientPoolEvictionIntervalMs()); - properties.put("clientPoolCacheKeys", catalogConfig.getClientPoolCacheKeys()); - properties.put("lockImplementation", catalogConfig.getLockImplementation()); - properties.put("lockHeartbeatIntervalMillis", catalogConfig.getLockHeartbeatIntervalMillis()); - properties.put("lockHeartbeatTimeoutMillis", catalogConfig.getLockHeartbeatTimeoutMillis()); - properties.put("lockHeartbeatThreads", catalogConfig.getLockHeartbeatThreads()); - properties.put("lockAcquireIntervalMillis", catalogConfig.getLockAcquireIntervalMillis()); - properties.put("lockAcquireTimeoutMillis", catalogConfig.getLockAcquireTimeoutMillis()); - properties.put("appIdentifier", catalogConfig.getAppIdentifier()); - properties.put("user", catalogConfig.getUser()); - properties.put("authSessionTimeoutMillis", catalogConfig.getAuthSessionTimeoutMillis()); - properties.put( - "configuration", - catalogConfig.getConfiguration() == null - ? null - : new SerializableConfiguration(catalogConfig.getConfiguration())); - - out.writeObject(properties); - } - - @Override - @SuppressWarnings("nullness") - public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - Map properties = (Map) in.readObject(); - catalogConfig = - IcebergCatalogConfig.builder() - .setName((String) properties.get("name")) - .setIcebergCatalogType((String) properties.get("icebergCatalogType")) - .setCatalogImplementation((String) properties.get("catalogImplementation")) - .setFileIOImplementation((String) properties.get("fileIOImplementation")) - .setWarehouseLocation((String) properties.get("warehouseLocation")) - .setMetricsReporterImplementation( - (String) properties.get("metricsReporterImplementation")) - .setCacheEnabled((Boolean) properties.get("cacheEnabled")) - .setCacheCaseSensitive((Boolean) properties.get("cacheCaseSensitive")) - .setCacheExpirationIntervalMillis( - (Long) properties.get("cacheExpirationIntervalMillis")) - .setIOManifestCacheEnabled((Boolean) properties.get("ioManifestCacheEnabled")) - .setIOManifestCacheExpirationIntervalMillis( - (Long) properties.get("ioManifestCacheExpirationIntervalMillis")) - .setIOManifestCacheMaxTotalBytes((Long) properties.get("ioManifestCacheMaxTotalBytes")) - .setIOManifestCacheMaxContentLength( - (Long) properties.get("ioManifestCacheMaxContentLength")) - .setUri((String) properties.get("uri")) - .setClientPoolSize((Integer) properties.get("clientPoolSize")) - .setClientPoolEvictionIntervalMs((Long) properties.get("clientPoolEvictionIntervalMs")) - .setClientPoolCacheKeys((String) properties.get("clientPoolCacheKeys")) - .setLockImplementation((String) properties.get("lockImplementation")) - .setLockHeartbeatIntervalMillis((Long) properties.get("lockHeartbeatIntervalMillis")) - .setLockHeartbeatTimeoutMillis((Long) properties.get("lockHeartbeatTimeoutMillis")) - .setLockHeartbeatThreads((Integer) properties.get("lockHeartbeatThreads")) - .setLockAcquireIntervalMillis((Long) properties.get("lockAcquireIntervalMillis")) - .setLockAcquireTimeoutMillis((Long) properties.get("lockAcquireTimeoutMillis")) - .setAppIdentifier((String) properties.get("appIdentifier")) - .setUser((String) properties.get("user")) - .setAuthSessionTimeoutMillis((Long) properties.get("authSessionTimeoutMillis")) - .setConfiguration( - properties.get("configuration") == null - ? null - : ((SerializableConfiguration) properties.get("configuration")).get()) - .build(); - } -} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ExternalizableIcebergCatalogConfigTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ExternalizableIcebergCatalogConfigTest.java deleted file mode 100644 index 2b5c61c97f3b..000000000000 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ExternalizableIcebergCatalogConfigTest.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.iceberg; - -import static org.junit.Assert.assertEquals; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -@RunWith(JUnit4.class) -public class ExternalizableIcebergCatalogConfigTest { - @Test - public void testSerializeDeserialize() throws IOException, ClassNotFoundException { - IcebergCatalogConfig config = - IcebergCatalogConfig.builder() - .setName("test-name") - .setIcebergCatalogType("test-type") - .setCatalogImplementation("test-implementation") - .setFileIOImplementation("test-fileio") - .setWarehouseLocation("test-location") - .setMetricsReporterImplementation("test-metrics") - .setCacheEnabled(true) - .setCacheCaseSensitive(true) - .setCacheExpirationIntervalMillis(100) - .build(); - - ExternalizableIcebergCatalogConfig externalizableConfig = - new ExternalizableIcebergCatalogConfig(config); - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - ObjectOutputStream oos = new ObjectOutputStream(baos); - - oos.writeObject(externalizableConfig); - - ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray()); - ObjectInputStream ois = new ObjectInputStream(bais); - ExternalizableIcebergCatalogConfig roundtripConfig = - (ExternalizableIcebergCatalogConfig) ois.readObject(); - - assertEquals(config, roundtripConfig.get()); - } -} From d912a225429f1bc986793e79f4b7127d45d58f9e Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 18 Apr 2024 18:08:12 -0400 Subject: [PATCH 33/60] pull identifiers from generated proto --- .../beam/sdk/managed/ManagedTransformConstants.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java index 48735d8c33a3..5971673a7050 100644 --- a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java +++ b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java @@ -17,9 +17,13 @@ */ package org.apache.beam.sdk.managed; +import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn; + +import org.apache.beam.model.pipeline.v1.SchemaAwareTransforms.StandardSchemaAwareTransforms; + /** 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_READ = getUrn(StandardSchemaAwareTransforms.IOs.ICEBERG_READ); public static final String ICEBERG_WRITE = - "beam:schematransform:org.apache.beam:iceberg_write:v1"; + getUrn(StandardSchemaAwareTransforms.IOs.ICEBERG_WRITE); } From 523fc69d896fb3fb967149fd446bf80b58ff7997 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 18 Apr 2024 18:18:12 -0400 Subject: [PATCH 34/60] remove unused hadoop dependency --- sdks/java/io/iceberg/build.gradle | 1 - 1 file changed, 1 deletion(-) diff --git a/sdks/java/io/iceberg/build.gradle b/sdks/java/io/iceberg/build.gradle index a1adb42ac26d..397bd099b853 100644 --- a/sdks/java/io/iceberg/build.gradle +++ b/sdks/java/io/iceberg/build.gradle @@ -46,7 +46,6 @@ dependencies { implementation project(path: ":model:pipeline", configuration: "shadow") implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(":sdks:java:managed") - implementation project(":sdks:java:io:hadoop-common") implementation library.java.slf4j_api implementation library.java.joda_time implementation "org.apache.parquet:parquet-column:$parquet_version" From cb89d1626691f29f8ae89ed345a4c307c2adab77 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 18 Apr 2024 18:34:24 -0400 Subject: [PATCH 35/60] update generate sequence wrapper after Schema sorting --- sdks/standard_external_transforms.yaml | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) 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 From 17ecec6c1e9149517dcd676e129106329b04520d Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 18 Apr 2024 21:27:50 -0400 Subject: [PATCH 36/60] managed transform translation uses default schema --- .../ManagedSchemaTransformTranslation.java | 60 ++++++------------- .../testing/TestSchemaTransformProvider.java | 2 +- ...ManagedSchemaTransformTranslationTest.java | 40 +------------ sdks/standard_expansion_services.yaml | 2 - 4 files changed, 22 insertions(+), 82 deletions(-) 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 index 8a1e5f12c2b1..7eafe969ccba 100644 --- 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 @@ -25,14 +25,15 @@ import com.google.auto.service.AutoService; import java.io.ByteArrayOutputStream; import java.io.IOException; -import java.util.HashMap; import java.util.Map; import org.apache.beam.model.pipeline.v1.SchemaApi; import org.apache.beam.model.pipeline.v1.SchemaAwareTransforms.ManagedSchemaTransformPayload; 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.NoSuchSchemaException; import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.SchemaRegistry; import org.apache.beam.sdk.schemas.SchemaTranslation; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.construction.SdkComponents; @@ -45,17 +46,8 @@ public class ManagedSchemaTransformTranslation { static class ManagedSchemaTransformTranslator implements TransformPayloadTranslator { - private final ManagedSchemaTransformProvider provider; - static final Schema SCHEMA = - Schema.builder() - .addStringField("transform_identifier") - .addNullableStringField("config") - .addNullableStringField("config_url") - .build(); - - public ManagedSchemaTransformTranslator() { - provider = new ManagedSchemaTransformProvider(null); - } + static final ManagedSchemaTransformProvider provider = new ManagedSchemaTransformProvider(null); + static final Schema SCHEMA = provider.configurationSchema(); @Override public String getUrn() { @@ -89,41 +81,27 @@ public String getUrn() { @Override public Row toConfigRow(ManagedSchemaTransform transform) { ManagedConfig managedConfig = transform.getManagedConfig(); - Map fieldValues = new HashMap<>(); + System.out.println("MANAGED config: \n" + managedConfig); - if (managedConfig.getTransformIdentifier() != null) { - fieldValues.put("transform_identifier", managedConfig.getTransformIdentifier()); - } - String config = managedConfig.getConfig(); - if (config != null) { - fieldValues.put("config", config); + Row configRow; + try { + configRow = + SchemaRegistry.createDefault() + .getToRowFunction(ManagedConfig.class) + .apply(managedConfig); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e); } - String configUrl = managedConfig.getConfigUrl(); - if (configUrl != null) { - fieldValues.put("config_url", configUrl); - } - - return Row.withSchema(SCHEMA).withFieldValues(fieldValues).build(); + System.out.println("config row: \n" + configRow); + // Sanity check: sort fields according to the configuration schema + return SCHEMA.getFields().stream() + .map(field -> configRow.getValue(field.getName())) + .collect(Row.toRow(SCHEMA)); } @Override public ManagedSchemaTransform fromConfigRow(Row configRow, PipelineOptions options) { - ManagedConfig.Builder configBuilder = ManagedConfig.builder(); - - String transformIdentifier = configRow.getValue("transform_identifier"); - if (transformIdentifier != null) { - configBuilder = configBuilder.setTransformIdentifier(transformIdentifier); - } - String config = configRow.getValue("config"); - if (config != null) { - configBuilder = configBuilder.setConfig(config); - } - String configUrl = configRow.getValue("config_url"); - if (configUrl != null) { - configBuilder = configBuilder.setConfigUrl(configUrl); - } - - return (ManagedSchemaTransform) provider.from(configBuilder.build()); + return (ManagedSchemaTransform) provider.from(configRow); } } diff --git a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/testing/TestSchemaTransformProvider.java b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/testing/TestSchemaTransformProvider.java index f36b30d638be..92c2de80109d 100644 --- a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/testing/TestSchemaTransformProvider.java +++ b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/testing/TestSchemaTransformProvider.java @@ -37,7 +37,7 @@ @AutoService(SchemaTransformProvider.class) public class TestSchemaTransformProvider extends TypedSchemaTransformProvider { - public static final String IDENTIFIER = "beam:schematransform:org.apache.beam:test_transform:v1"; + public static final String IDENTIFIER = "beam:test_schematransform:v1"; public static final Schema SCHEMA; static { 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 index 61340b8ff01a..3cf9833fce30 100644 --- 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 @@ -23,7 +23,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; import java.net.URISyntaxException; import java.nio.file.Paths; @@ -116,41 +115,6 @@ public void testReCreateTransformFromRowWithConfig() { assertNull(originalConfig.getConfigUrl()); } - @Test - public void testManagedTransformRowIncludesAllFields() { - List getMethodNames = - Arrays.stream(ManagedConfig.class.getDeclaredMethods()) - .map(method -> method.getName()) - .filter(methodName -> methodName.startsWith("get")) - .collect(Collectors.toList()); - - // Just to make sure that this does not pass trivially. - assertTrue(getMethodNames.size() > 0); - - for (String getMethodName : getMethodNames) { - assertTrue( - "Method " - + getMethodName - + " will not be tracked when upgrading the 'ManagedSchemaTransform'. Please update" - + "'ManagedSchemaTransformTranslation.ManagedSchemaTransformTranslator' to track the " - + "new method and update this test.", - MANAGED_TRANSFORM_SCHEMA_MAPPING.keySet().contains(getMethodName)); - } - - // Confirming that all fields mentioned in `WRITE_TRANSFORM_SCHEMA_MAPPING` are - // actually available in the schema. - MANAGED_TRANSFORM_SCHEMA_MAPPING.values().stream() - .forEach( - fieldName -> { - assertTrue( - "Field name " - + fieldName - + " was not found in the transform schema defined in " - + "ManagedSchemaTransformTranslation.ManagedSchemaTransformTranslator.", - ManagedSchemaTransformTranslator.SCHEMA.getFieldNames().contains(fieldName)); - }); - } - @Test public void testProtoTranslation() throws Exception { // First build a pipeline @@ -199,8 +163,8 @@ public void testProtoTranslation() throws Exception { Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getExpansionPayload().newInput()); Row expectedRow = Row.withSchema(ManagedSchemaTransformTranslator.SCHEMA) - .withFieldValue("transform_identifier", TestSchemaTransformProvider.IDENTIFIER) - .withFieldValue("config_url", null) + .withFieldValue("transformIdentifier", TestSchemaTransformProvider.IDENTIFIER) + .withFieldValue("configUrl", null) .withFieldValue("config", yamlStringConfig) .build(); assertEquals(expectedRow, rowFromSpec); diff --git a/sdks/standard_expansion_services.yaml b/sdks/standard_expansion_services.yaml index 02f5044b2d1a..31a1a6343aed 100644 --- a/sdks/standard_expansion_services.yaml +++ b/sdks/standard_expansion_services.yaml @@ -47,8 +47,6 @@ # Not ready to generate - 'beam:schematransform:org.apache.beam:iceberg_write:v1' - 'beam:schematransform:org.apache.beam:iceberg_read:v1' - # Not relevant - - 'beam:schematransform:org.apache.beam:test_transform:v1' # TODO(ahmedabu98): Enable this service in a future PR #- gradle_target: 'sdks:java:io:google-cloud-platform:expansion-service:shadowJar' From 108a12c33ea04a162c891b084a2bc317c2fdb423 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 18 Apr 2024 21:36:01 -0400 Subject: [PATCH 37/60] yaml returns null row; cleanup --- .../java/org/apache/beam/sdk/schemas/utils/YamlUtils.java | 8 ++++---- .../test/java/org/apache/beam/sdk/util/YamlUtilsTest.java | 3 +-- .../io/iceberg/IcebergSchemaTransformTranslationTest.java | 2 -- .../iceberg/IcebergWriteSchemaTransformProviderTest.java | 1 - .../beam/sdk/managed/ManagedSchemaTransformProvider.java | 8 ++++---- .../sdk/managed/ManagedSchemaTransformTranslation.java | 3 --- .../java/org/apache/beam/sdk/managed/ManagedTest.java | 3 --- 7 files changed, 9 insertions(+), 19 deletions(-) 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 5571c9e4fed6..049da266c45d 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 @@ -57,7 +57,7 @@ public class YamlUtils { return toBeamRow(yamlString, schema, false); } - public static @Nullable Row toBeamRow( + public static Row toBeamRow( @Nullable String yamlString, Schema schema, boolean convertNamesToCamelCase) { if (yamlString == null || yamlString.isEmpty()) { List requiredFields = @@ -65,7 +65,7 @@ public class YamlUtils { .filter(field -> !field.getType().getNullable()) .collect(Collectors.toList()); if (requiredFields.isEmpty()) { - return null; + return Row.nullRow(schema); } else { throw new IllegalArgumentException( String.format( @@ -147,7 +147,7 @@ public class YamlUtils { } @SuppressWarnings("nullness") - public static @Nullable Row toBeamRow( + public static Row toBeamRow( @Nullable Map map, Schema rowSchema, boolean toCamelCase) { if (map == null || map.isEmpty()) { List requiredFields = @@ -155,7 +155,7 @@ public class YamlUtils { .filter(field -> !field.getType().getNullable()) .collect(Collectors.toList()); if (requiredFields.isEmpty()) { - return null; + return Row.nullRow(rowSchema); } else { throw new IllegalArgumentException( String.format( 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 01cf784f6298..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 @@ -18,7 +18,6 @@ package org.apache.beam.sdk.util; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; import java.math.BigDecimal; import java.util.Arrays; @@ -50,7 +49,7 @@ public String makeNested(String input) { public void testEmptyYamlString() { Schema schema = Schema.builder().build(); - assertNull(YamlUtils.toBeamRow("", schema)); + assertEquals(Row.nullRow(schema), YamlUtils.toBeamRow("", schema)); } @Test 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 index b05ee9288218..aa6979c0b439 100644 --- 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 @@ -130,8 +130,6 @@ public void testWriteTransformProtoTranslation() throws Exception { SchemaAwareTransforms.SchemaAwareTransformPayload.parseFrom(spec.getPayload()); Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getExpansionSchema()); assertEquals(WRITE_PROVIDER.configurationSchema(), schemaFromSpec); - System.out.println( - "spec: " + schemaFromSpec.getField("catalogConfig").getType().getRowSchema()); Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getExpansionPayload().newInput()); assertEquals(transformConfigRow, rowFromSpec); 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 index 095c3a76637c..26bc4efbf0b1 100644 --- 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 @@ -168,7 +168,6 @@ public VerifyOutputs(String identifier, String operation) { @Override public Void apply(Iterable input) { - System.out.println(input); Row row = input.iterator().next(); assertEquals(tableId, row.getString("table")); 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 d1ea52b59d5e..9115255b4b71 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 @@ -182,12 +182,12 @@ public ManagedConfig getManagedConfig() { } } + /** */ @VisibleForTesting static Row getRowConfig(ManagedConfig config, Schema transformSchema) { - Row configRow = YamlUtils.toBeamRow(config.resolveUnderlyingConfig(), transformSchema, true); - // If our config is still null (perhaps the underlying transform doesn't have any required - // parameters), then return an empty row. - return configRow != null ? configRow : Row.nullRow(transformSchema); + // May return an empty row (perhaps the underlying transform doesn't have any required + // parameters) + return YamlUtils.toBeamRow(config.resolveUnderlyingConfig(), transformSchema, true); } Map getAllProviders() { 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 index 7eafe969ccba..dd08b42afcdf 100644 --- 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 @@ -81,8 +81,6 @@ public String getUrn() { @Override public Row toConfigRow(ManagedSchemaTransform transform) { ManagedConfig managedConfig = transform.getManagedConfig(); - System.out.println("MANAGED config: \n" + managedConfig); - Row configRow; try { configRow = @@ -92,7 +90,6 @@ public Row toConfigRow(ManagedSchemaTransform transform) { } catch (NoSuchSchemaException e) { throw new RuntimeException(e); } - System.out.println("config row: \n" + configRow); // Sanity check: sort fields according to the configuration schema return SCHEMA.getFields().stream() .map(field -> configRow.getValue(field.getName())) 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 41dec0ddd86f..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 @@ -17,8 +17,6 @@ */ package org.apache.beam.sdk.managed; -import static org.apache.beam.sdk.managed.Managed.READ_TRANSFORMS; - import java.nio.file.Paths; import java.util.Arrays; import java.util.List; @@ -44,7 +42,6 @@ public class ManagedTest { @Test public void testInvalidTransform() { - System.out.println(READ_TRANSFORMS); thrown.expect(NullPointerException.class); thrown.expectMessage("An unsupported source was specified"); Managed.read("nonexistent-source"); From 5923a971965882c36b4c0ee5871d2006a472d06f Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 18 Apr 2024 22:07:26 -0400 Subject: [PATCH 38/60] spotless --- .../iceberg/IcebergSchemaTransformTranslation.java | 13 +++++++------ .../IcebergSchemaTransformTranslationTest.java | 14 ++++++++------ .../managed/ManagedSchemaTransformTranslation.java | 8 ++++---- 3 files changed, 19 insertions(+), 16 deletions(-) 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 index 6f23c664d910..2d8544102f08 100644 --- 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 @@ -30,6 +30,7 @@ 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.Schema; import org.apache.beam.sdk.schemas.SchemaTranslation; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.construction.PTransformTranslation.TransformPayloadTranslator; @@ -46,6 +47,7 @@ static class IcebergReadSchemaTransformTranslator implements TransformPayloadTranslator { static final IcebergReadSchemaTransformProvider READ_PROVIDER = new IcebergReadSchemaTransformProvider(); + static final Schema READ_SCHEMA = READ_PROVIDER.configurationSchema(); @Override public String getUrn() { @@ -56,11 +58,10 @@ public String getUrn() { public @Nullable FunctionSpec translate( AppliedPTransform application, SdkComponents components) throws IOException { - SchemaApi.Schema expansionSchema = - SchemaTranslation.schemaToProto(READ_PROVIDER.configurationSchema(), true); + SchemaApi.Schema expansionSchema = SchemaTranslation.schemaToProto(READ_SCHEMA, true); Row configRow = toConfigRow(application.getTransform()); ByteArrayOutputStream os = new ByteArrayOutputStream(); - RowCoder.of(READ_PROVIDER.configurationSchema()).encode(configRow, os); + RowCoder.of(READ_SCHEMA).encode(configRow, os); return FunctionSpec.newBuilder() .setUrn(getUrn()) @@ -103,6 +104,7 @@ static class IcebergWriteSchemaTransformTranslator static final IcebergWriteSchemaTransformProvider WRITE_PROVIDER = new IcebergWriteSchemaTransformProvider(); + static final Schema WRITE_SCHEMA = WRITE_PROVIDER.configurationSchema(); @Override public String getUrn() { @@ -113,11 +115,10 @@ public String getUrn() { public @Nullable FunctionSpec translate( AppliedPTransform application, SdkComponents components) throws IOException { - SchemaApi.Schema expansionSchema = - SchemaTranslation.schemaToProto(WRITE_PROVIDER.configurationSchema(), true); + SchemaApi.Schema expansionSchema = SchemaTranslation.schemaToProto(WRITE_SCHEMA, true); Row configRow = toConfigRow(application.getTransform()); ByteArrayOutputStream os = new ByteArrayOutputStream(); - RowCoder.of(WRITE_PROVIDER.configurationSchema()).encode(configRow, os); + RowCoder.of(WRITE_SCHEMA).encode(configRow, os); return FunctionSpec.newBuilder() .setUrn(getUrn()) 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 index aa6979c0b439..1c943500632a 100644 --- 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 @@ -18,6 +18,8 @@ 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.IcebergSchemaTransformTranslation.IcebergReadSchemaTransformTranslator.READ_SCHEMA; +import static org.apache.beam.sdk.io.iceberg.IcebergSchemaTransformTranslation.IcebergWriteSchemaTransformTranslator.WRITE_SCHEMA; 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; @@ -71,7 +73,7 @@ public void testReCreateWriteTransformFromRow() throws NoSuchSchemaException { .withFieldValue("warehouseLocation", "test_location") .build(); Row transformConfigRow = - Row.withSchema(WRITE_PROVIDER.configurationSchema()) + Row.withSchema(WRITE_SCHEMA) .withFieldValue("table", "test_table_identifier") .withFieldValue("catalogConfig", catalogConfigRow) .build(); @@ -107,7 +109,7 @@ public void testWriteTransformProtoTranslation() throws Exception { .withFieldValue("warehouseLocation", warehouse.location) .build(); Row transformConfigRow = - Row.withSchema(WRITE_PROVIDER.configurationSchema()) + Row.withSchema(WRITE_SCHEMA) .withFieldValue("table", "test_identifier") .withFieldValue("catalogConfig", catalogConfigRow) .build(); @@ -129,7 +131,7 @@ public void testWriteTransformProtoTranslation() throws Exception { SchemaAwareTransforms.SchemaAwareTransformPayload payload = SchemaAwareTransforms.SchemaAwareTransformPayload.parseFrom(spec.getPayload()); Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getExpansionSchema()); - assertEquals(WRITE_PROVIDER.configurationSchema(), schemaFromSpec); + assertEquals(WRITE_SCHEMA, schemaFromSpec); Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getExpansionPayload().newInput()); assertEquals(transformConfigRow, rowFromSpec); @@ -154,7 +156,7 @@ public void testReCreateReadTransformFromRow() throws NoSuchSchemaException { .withFieldValue("warehouseLocation", "test_location") .build(); Row transformConfigRow = - Row.withSchema(READ_PROVIDER.configurationSchema()) + Row.withSchema(READ_SCHEMA) .withFieldValue("table", "test_table_identifier") .withFieldValue("catalogConfig", catalogConfigRow) .build(); @@ -186,7 +188,7 @@ public void testReadTransformProtoTranslation() throws Exception { warehouse.createTable(TableIdentifier.parse(identifier), TestFixtures.SCHEMA); Row transformConfigRow = - Row.withSchema(READ_PROVIDER.configurationSchema()) + Row.withSchema(READ_SCHEMA) .withFieldValue("table", identifier) .withFieldValue("catalogConfig", catalogConfigRow) .build(); @@ -209,7 +211,7 @@ public void testReadTransformProtoTranslation() throws Exception { SchemaAwareTransforms.SchemaAwareTransformPayload payload = SchemaAwareTransforms.SchemaAwareTransformPayload.parseFrom(spec.getPayload()); Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getExpansionSchema()); - assertEquals(READ_PROVIDER.configurationSchema(), schemaFromSpec); + assertEquals(READ_SCHEMA, schemaFromSpec); Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getExpansionPayload().newInput()); assertEquals(transformConfigRow, rowFromSpec); 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 index dd08b42afcdf..974b13a4ac6a 100644 --- 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 @@ -46,12 +46,12 @@ public class ManagedSchemaTransformTranslation { static class ManagedSchemaTransformTranslator implements TransformPayloadTranslator { - static final ManagedSchemaTransformProvider provider = new ManagedSchemaTransformProvider(null); - static final Schema SCHEMA = provider.configurationSchema(); + static final ManagedSchemaTransformProvider PROVIDER = new ManagedSchemaTransformProvider(null); + static final Schema SCHEMA = PROVIDER.configurationSchema(); @Override public String getUrn() { - return provider.identifier(); + return PROVIDER.identifier(); } @Override @@ -98,7 +98,7 @@ public Row toConfigRow(ManagedSchemaTransform transform) { @Override public ManagedSchemaTransform fromConfigRow(Row configRow, PipelineOptions options) { - return (ManagedSchemaTransform) provider.from(configRow); + return (ManagedSchemaTransform) PROVIDER.from(configRow); } } From 8ab4c0cab2c625741046dc1750e4a5e574967c45 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Fri, 19 Apr 2024 10:24:19 -0400 Subject: [PATCH 39/60] remove SchemaAwareTransformPayload and use SchemaTransformPayload instead; rename StandardSchemaAwareTransforms -> ManagedSchemaAwareTransforms --- .../model/pipeline/v1/external_transforms.proto | 4 ++++ .../pipeline/v1/schema_aware_transforms.proto | 16 ++-------------- .../util/construction/PTransformTranslation.java | 4 ++-- .../IcebergSchemaTransformTranslation.java | 16 +++++++++------- .../IcebergSchemaTransformTranslationTest.java | 16 +++++++--------- .../sdk/managed/ManagedTransformConstants.java | 7 +++---- 6 files changed, 27 insertions(+), 36 deletions(-) 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/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema_aware_transforms.proto b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema_aware_transforms.proto index 4c6c90e1b24f..ab86b1fc0a1f 100644 --- a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema_aware_transforms.proto +++ b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema_aware_transforms.proto @@ -33,8 +33,8 @@ import "org/apache/beam/model/pipeline/v1/beam_runner_api.proto"; import "org/apache/beam/model/pipeline/v1/schema.proto"; -message StandardSchemaAwareTransforms { - // Payload for all of these: SchemaAwareTransformPayload +message ManagedSchemaAwareTransforms { + // Payload for all of these: SchemaTransformPayload (in external_transforms.proto) enum IOs { ICEBERG_READ = 0 [(beam_urn) = "beam:schematransform:org.apache.beam:iceberg_read:v1"]; ICEBERG_WRITE = 1 [(beam_urn) = "beam:schematransform:org.apache.beam:iceberg_write:v1"]; @@ -46,18 +46,6 @@ message StandardSchemaAwareTransforms { } -// 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 by runners to override the schema-aware transform. -message SchemaAwareTransformPayload { - // The schema of the configuration row used to override the transform - Schema expansion_schema = 1; - // The configuration used to override this transform. - // Must be compatible with the configuration schema, and decodable via beam:coder:row:v1. - bytes expansion_payload = 2; -} - // Payload for a ManagedSchemaTransform. // The information available in the payload can be used by runners to override the ManagedSchemaTransform. message ManagedSchemaTransformPayload { 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 9adf1d72a28d..b6c193674dcb 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 @@ -35,7 +35,7 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.StandardPTransforms; import org.apache.beam.model.pipeline.v1.RunnerApi.StandardPTransforms.CombineComponents; import org.apache.beam.model.pipeline.v1.RunnerApi.StandardPTransforms.SplittableParDoComponents; -import org.apache.beam.model.pipeline.v1.SchemaAwareTransforms.StandardSchemaAwareTransforms; +import org.apache.beam.model.pipeline.v1.SchemaAwareTransforms.ManagedSchemaAwareTransforms; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.coders.RowCoder; @@ -184,7 +184,7 @@ public class PTransformTranslation { MAP_WINDOWS_TRANSFORM_URN.equals(getUrn(StandardPTransforms.Primitives.MAP_WINDOWS))); checkState( MERGE_WINDOWS_TRANSFORM_URN.equals(getUrn(StandardPTransforms.Primitives.MERGE_WINDOWS))); - checkState(MANAGED_TRANSFORM_URN.equals(getUrn(StandardSchemaAwareTransforms.Managed.MANAGED))); + checkState(MANAGED_TRANSFORM_URN.equals(getUrn(ManagedSchemaAwareTransforms.Managed.MANAGED))); // DeprecatedPrimitives checkState(READ_TRANSFORM_URN.equals(getUrn(StandardPTransforms.DeprecatedPrimitives.READ))); 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 index 2d8544102f08..967315d4dfe1 100644 --- 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 @@ -24,9 +24,9 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.util.Map; +import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload; import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; import org.apache.beam.model.pipeline.v1.SchemaApi; -import org.apache.beam.model.pipeline.v1.SchemaAwareTransforms.SchemaAwareTransformPayload; import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.AppliedPTransform; @@ -66,9 +66,10 @@ public String getUrn() { return FunctionSpec.newBuilder() .setUrn(getUrn()) .setPayload( - SchemaAwareTransformPayload.newBuilder() - .setExpansionSchema(expansionSchema) - .setExpansionPayload(ByteString.copyFrom(os.toByteArray())) + SchemaTransformPayload.newBuilder() + .setConfigurationSchema(expansionSchema) + .setConfigurationRow(ByteString.copyFrom(os.toByteArray())) + .setIdentifier(getUrn()) .build() .toByteString()) .build(); @@ -123,9 +124,10 @@ public String getUrn() { return FunctionSpec.newBuilder() .setUrn(getUrn()) .setPayload( - SchemaAwareTransformPayload.newBuilder() - .setExpansionSchema(expansionSchema) - .setExpansionPayload(ByteString.copyFrom(os.toByteArray())) + SchemaTransformPayload.newBuilder() + .setConfigurationSchema(expansionSchema) + .setConfigurationRow(ByteString.copyFrom(os.toByteArray())) + .setIdentifier(getUrn()) .build() .toByteString()) .build(); 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 index 1c943500632a..1ae1209880b7 100644 --- 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 @@ -28,8 +28,8 @@ 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.model.pipeline.v1.SchemaAwareTransforms; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.managed.ManagedTransformConstants; @@ -128,11 +128,10 @@ public void testWriteTransformProtoTranslation() throws Exception { RunnerApi.FunctionSpec spec = writeTransformProto.get(0).getSpec(); // Check that the proto contains correct values - SchemaAwareTransforms.SchemaAwareTransformPayload payload = - SchemaAwareTransforms.SchemaAwareTransformPayload.parseFrom(spec.getPayload()); - Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getExpansionSchema()); + SchemaTransformPayload payload = SchemaTransformPayload.parseFrom(spec.getPayload()); + Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getConfigurationSchema()); assertEquals(WRITE_SCHEMA, schemaFromSpec); - Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getExpansionPayload().newInput()); + Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getConfigurationRow().newInput()); assertEquals(transformConfigRow, rowFromSpec); @@ -208,11 +207,10 @@ public void testReadTransformProtoTranslation() throws Exception { RunnerApi.FunctionSpec spec = readTransformProto.get(0).getSpec(); // Check that the proto contains correct values - SchemaAwareTransforms.SchemaAwareTransformPayload payload = - SchemaAwareTransforms.SchemaAwareTransformPayload.parseFrom(spec.getPayload()); - Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getExpansionSchema()); + SchemaTransformPayload payload = SchemaTransformPayload.parseFrom(spec.getPayload()); + Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getConfigurationSchema()); assertEquals(READ_SCHEMA, schemaFromSpec); - Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getExpansionPayload().newInput()); + Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getConfigurationRow().newInput()); assertEquals(transformConfigRow, rowFromSpec); // Use the information in the proto to recreate the IcebergReadSchemaTransform diff --git a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java index 5971673a7050..9a0cb3f102fa 100644 --- a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java +++ b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java @@ -19,11 +19,10 @@ import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn; -import org.apache.beam.model.pipeline.v1.SchemaAwareTransforms.StandardSchemaAwareTransforms; +import org.apache.beam.model.pipeline.v1.SchemaAwareTransforms.ManagedSchemaAwareTransforms; /** This class contains constants for supported managed transform identifiers. */ public class ManagedTransformConstants { - public static final String ICEBERG_READ = getUrn(StandardSchemaAwareTransforms.IOs.ICEBERG_READ); - public static final String ICEBERG_WRITE = - getUrn(StandardSchemaAwareTransforms.IOs.ICEBERG_WRITE); + public static final String ICEBERG_READ = getUrn(ManagedSchemaAwareTransforms.IOs.ICEBERG_READ); + public static final String ICEBERG_WRITE = getUrn(ManagedSchemaAwareTransforms.IOs.ICEBERG_WRITE); } From 34263607e6a390d6c79b8a73ff33f37c07b0b8e4 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Fri, 19 Apr 2024 12:17:02 -0400 Subject: [PATCH 40/60] create a beam-schema-compatible class for Snapshot info --- .../sdk/io/iceberg/AppendFilesToTables.java | 15 ++- .../sdk/io/iceberg/IcebergWriteResult.java | 11 +- .../IcebergWriteSchemaTransformProvider.java | 22 ++-- .../beam/sdk/io/iceberg/SnapshotInfo.java | 109 ++++++++++++++++++ .../sdk/io/iceberg/WriteToDestinations.java | 3 +- 5 files changed, 129 insertions(+), 31 deletions(-) create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotInfo.java diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/AppendFilesToTables.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/AppendFilesToTables.java index 2795d3392d04..e4ba60001824 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/AppendFilesToTables.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/AppendFilesToTables.java @@ -18,7 +18,6 @@ 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/sdk/io/iceberg/IcebergWriteResult.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteResult.java index 428fa0103004..8e2549b5dadb 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteResult.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteResult.java @@ -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 index fc257e964259..624c57889cc6 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java @@ -38,7 +38,6 @@ 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.Snapshot; import org.apache.iceberg.catalog.TableIdentifier; /** @@ -52,12 +51,7 @@ public class IcebergWriteSchemaTransformProvider extends TypedSchemaTransformPro static final String OUTPUT_TAG = "output"; static final Schema OUTPUT_SCHEMA = - Schema.builder() - .addStringField("table") - .addStringField("operation") - .addMapField("summary", Schema.FieldType.STRING, Schema.FieldType.STRING) - .addStringField("manifestListLocation") - .build(); + Schema.builder().addStringField("table").addFields(SnapshotInfo.SCHEMA.getFields()).build(); @Override public String description() { @@ -166,16 +160,14 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { } @VisibleForTesting - static class SnapshotToRow extends SimpleFunction, Row> { + static class SnapshotToRow extends SimpleFunction, Row> { @Override - public Row apply(KV input) { - Snapshot snapshot = input.getValue(); + public Row apply(KV input) { + SnapshotInfo snapshot = input.getValue(); + return Row.withSchema(OUTPUT_SCHEMA) - .addValues( - input.getKey(), - snapshot.operation(), - snapshot.summary(), - snapshot.manifestListLocation()) + .addValue(input.getKey()) + .addValues(snapshot.toRow().getValues()) .build(); } } 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..e64fdf6100ec --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotInfo.java @@ -0,0 +1,109 @@ +/* + * 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.*; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.values.Row; +import org.apache.iceberg.Snapshot; + +/** + * This is a Beam Schema-compatible representation of an Iceberg {@link + * org.apache.iceberg.Snapshot}. + */ +@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); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e); + } + } + + public static SchemaCoder CODER; + public static Schema SCHEMA; + + static { + try { + SchemaRegistry registry = SchemaRegistry.createDefault(); + CODER = registry.getSchemaCoder(SnapshotInfo.class); + SCHEMA = registry.getSchema(SnapshotInfo.class); + } 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/sdk/io/iceberg/WriteToDestinations.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteToDestinations.java index 649ad3d6d5c3..65fd551c782a 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteToDestinations.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteToDestinations.java @@ -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); From 49a15960a2d9162ed6db57390576c3aa93bd4a73 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Fri, 19 Apr 2024 15:55:39 -0400 Subject: [PATCH 41/60] 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 --- .../model/pipeline/v1/beam_runner_api.proto | 6 ++ .../pipeline/v1/schema_aware_transforms.proto | 61 ------------------- .../beam/sdk/io/iceberg/FileWriteResult.java | 6 ++ .../ManagedSchemaTransformTranslation.java | 11 ++-- ...ManagedSchemaTransformTranslationTest.java | 13 ++-- 5 files changed, 22 insertions(+), 75 deletions(-) delete mode 100644 model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema_aware_transforms.proto diff --git a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto index bbd326f91deb..d2445925d0e2 100644 --- a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto +++ b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto @@ -443,6 +443,12 @@ message StandardPTransforms { enum GroupIntoBatchesComponents { GROUP_INTO_BATCHES = 0 [(beam_urn) = "beam:transform:group_into_batches:v1"]; } + // Payload for all of these: SchemaTransformPayload (in external_transforms.proto) + enum ManagedSchemaTransforms { + MANAGED = 0 [(beam_urn) = "beam:transform:managed:v1"]; + ICEBERG_READ = 1 [(beam_urn) = "beam:schematransform:org.apache.beam:iceberg_read:v1"]; + ICEBERG_WRITE = 2 [(beam_urn) = "beam:schematransform:org.apache.beam:iceberg_write:v1"]; + } } message StandardSideInputTypes { diff --git a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema_aware_transforms.proto b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema_aware_transforms.proto deleted file mode 100644 index ab86b1fc0a1f..000000000000 --- a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/schema_aware_transforms.proto +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -// Protocol Buffers describing available Schema-Aware transforms. -// These are transforms that can be configured using Beam Schema-compatible parameters. -// Runners can override these transforms with a native implementation. - -syntax = "proto3"; - -package org.apache.beam.model.pipeline.v1; - -option go_package = "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1;pipeline_v1"; -option java_package = "org.apache.beam.model.pipeline.v1"; -option java_outer_classname = "SchemaAwareTransforms"; - -import "org/apache/beam/model/pipeline/v1/beam_runner_api.proto"; -import "org/apache/beam/model/pipeline/v1/schema.proto"; - - -message ManagedSchemaAwareTransforms { - // Payload for all of these: SchemaTransformPayload (in external_transforms.proto) - enum IOs { - ICEBERG_READ = 0 [(beam_urn) = "beam:schematransform:org.apache.beam:iceberg_read:v1"]; - ICEBERG_WRITE = 1 [(beam_urn) = "beam:schematransform:org.apache.beam:iceberg_write:v1"]; - } - enum Managed { - // Payload: ManagedSchemaTransformPayload - MANAGED = 0 [(beam_urn) = "beam:transform:managed:v1"]; - } -} - - -// Payload for a ManagedSchemaTransform. -// The information available in the payload can be used by runners to override the ManagedSchemaTransform. -message ManagedSchemaTransformPayload { - // The underlying schema-aware transform's identifier. - string underlying_transform_identifier = 1; - // The managed transform's configuration Schema. - Schema expansion_schema = 2; - // The configuration used to override the managed transform. - // Must be compatible with the expansion schema, and decodable via beam:coder:row:v1. - bytes expansion_payload = 3; - // The underlying transform's configuration, represented as a YAML string. - string yaml_config = 4; -} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/FileWriteResult.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/FileWriteResult.java index 210b39b6a1de..d058af7d7d3e 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/FileWriteResult.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/FileWriteResult.java @@ -92,6 +92,10 @@ 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 (whether with 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(0); tableIdentifierCoder.encode(value.getTableIdentifier().toString(), outStream); partitionSpecCoder.encode(value.getPartitionSpec(), outStream); dataFileBytesCoder.encode( @@ -100,6 +104,8 @@ 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/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 index 974b13a4ac6a..b4412f9d5dab 100644 --- 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 @@ -26,8 +26,8 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.util.Map; +import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload; import org.apache.beam.model.pipeline.v1.SchemaApi; -import org.apache.beam.model.pipeline.v1.SchemaAwareTransforms.ManagedSchemaTransformPayload; import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.AppliedPTransform; @@ -68,11 +68,10 @@ public String getUrn() { return FunctionSpec.newBuilder() .setUrn(getUrn()) .setPayload( - ManagedSchemaTransformPayload.newBuilder() - .setUnderlyingTransformIdentifier(managedConfig.getTransformIdentifier()) - .setYamlConfig(managedConfig.resolveUnderlyingConfig()) - .setExpansionSchema(expansionSchema) - .setExpansionPayload(ByteString.copyFrom(os.toByteArray())) + SchemaTransformPayload.newBuilder() + .setIdentifier(managedConfig.getTransformIdentifier()) + .setConfigurationSchema(expansionSchema) + .setConfigurationRow(ByteString.copyFrom(os.toByteArray())) .build() .toByteString()) .build(); 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 index 3cf9833fce30..d60bf8df4621 100644 --- 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 @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.managed; +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; @@ -31,7 +32,6 @@ import java.util.Map; import java.util.stream.Collectors; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.model.pipeline.v1.SchemaAwareTransforms; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.managed.testing.TestSchemaTransformProvider; @@ -153,14 +153,11 @@ public void testProtoTranslation() throws Exception { RunnerApi.FunctionSpec spec = managedTransformProto.get(0).getSpec(); // Check that the proto contains correct values - SchemaAwareTransforms.ManagedSchemaTransformPayload payload = - SchemaAwareTransforms.ManagedSchemaTransformPayload.parseFrom(spec.getPayload()); - assertEquals( - TestSchemaTransformProvider.IDENTIFIER, payload.getUnderlyingTransformIdentifier()); - assertEquals(yamlStringConfig, payload.getYamlConfig()); - Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getExpansionSchema()); + SchemaTransformPayload payload = SchemaTransformPayload.parseFrom(spec.getPayload()); + assertEquals(TestSchemaTransformProvider.IDENTIFIER, payload.getIdentifier()); + Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getConfigurationSchema()); assertEquals(ManagedSchemaTransformTranslator.SCHEMA, schemaFromSpec); - Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getExpansionPayload().newInput()); + Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getConfigurationRow().newInput()); Row expectedRow = Row.withSchema(ManagedSchemaTransformTranslator.SCHEMA) .withFieldValue("transformIdentifier", TestSchemaTransformProvider.IDENTIFIER) From 52a308c051d3ffd2c84932884ebfae7ba087f728 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Fri, 19 Apr 2024 17:24:23 -0400 Subject: [PATCH 42/60] Row and Schema snake_case <-> camelCase conversion logic --- .../org/apache/beam/sdk/schemas/Schema.java | 39 +++++++++++ .../java/org/apache/beam/sdk/values/Row.java | 26 ++++++++ .../apache/beam/sdk/schemas/SchemaTest.java | 65 +++++++++++++++++++ .../org/apache/beam/sdk/values/RowTest.java | 56 ++++++++++++++++ .../managed/ManagedTransformConstants.java | 28 -------- 5 files changed, 186 insertions(+), 28 deletions(-) delete mode 100644 sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java 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 451bc8831c4c..4a94f823901c 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; @@ -1461,4 +1462,42 @@ public int getFieldCount() { public Options getOptions() { return this.options; } + + /** Recursively converts all field names to `lower_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/values/Row.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java index f5c6c7fcf34a..f1eaf87bd001 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,30 @@ public static Row nullRow(Schema schema) { .addValues(Collections.nCopies(schema.getFieldCount(), null)) .build(); } + + /** Returns an equivalent {@link Row} with `lower_snake_case` field names. */ + public Row toSnakeCase() { + return getSchema().getFields().stream() + .map( + field -> { + if (field.getType().getRowSchema() != null) { + return ((Row) getValue(field.getName())).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) { + return ((Row) getValue(field.getName())).toCamelCase(); + } + return (Object) getValue(field.getName()); + }) + .collect(toRow(getSchema().toCamelCase())); + } } 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 e43527b819a8..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 = 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..c9ce1eac2609 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,60 @@ public void testWithFieldValues() { assertEquals( enumerationType.valueOf("zero"), row.getLogicalTypeValue(0, EnumerationType.Value.class)); } + + @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/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 deleted file mode 100644 index 9a0cb3f102fa..000000000000 --- a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.managed; - -import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn; - -import org.apache.beam.model.pipeline.v1.SchemaAwareTransforms.ManagedSchemaAwareTransforms; - -/** This class contains constants for supported managed transform identifiers. */ -public class ManagedTransformConstants { - public static final String ICEBERG_READ = getUrn(ManagedSchemaAwareTransforms.IOs.ICEBERG_READ); - public static final String ICEBERG_WRITE = getUrn(ManagedSchemaAwareTransforms.IOs.ICEBERG_WRITE); -} From d9fcb064727c7a81ce6d2fe912658448af16bd1a Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Fri, 19 Apr 2024 19:47:22 -0400 Subject: [PATCH 43/60] Row sorted() util --- .../java/org/apache/beam/sdk/values/Row.java | 14 ++++++++ .../org/apache/beam/sdk/values/RowTest.java | 34 +++++++++++++++++++ .../managed/ManagedTransformConstants.java | 2 ++ 3 files changed, 50 insertions(+) create mode 100644 sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java 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 f1eaf87bd001..9abaa61f019f 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 @@ -894,6 +894,20 @@ public static Row nullRow(Schema schema) { .build(); } + /** Returns an identical {@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) { + return ((Row) getValue(field.getName())).sorted(); + } + return (Object) getValue(field.getName()); + }) + .collect(Row.toRow(sortedSchema)); + } + /** Returns an equivalent {@link Row} with `lower_snake_case` field names. */ public Row toSnakeCase() { return getSchema().getFields().stream() 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 c9ce1eac2609..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 @@ -807,6 +807,40 @@ public void testWithFieldValues() { 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 = 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..2d643b98abec --- /dev/null +++ b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java @@ -0,0 +1,2 @@ +package org.apache.beam.sdk.managed;public class ManagedTransformConstants { +} From 11354c891f96ae99932f7cd1587c681fe3e621c3 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Fri, 19 Apr 2024 19:52:58 -0400 Subject: [PATCH 44/60] use Row::sorted to fetch Managed & Iceberg row configs --- .../org/apache/beam/sdk/schemas/Schema.java | 5 +++- .../IcebergReadSchemaTransformProvider.java | 22 ++++++++++-------- .../IcebergWriteSchemaTransformProvider.java | 22 ++++++++++-------- .../ManagedSchemaTransformProvider.java | 23 +++++++++++++++---- 4 files changed, 46 insertions(+), 26 deletions(-) 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 4a94f823901c..da3cca1434b0 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 @@ -327,7 +327,10 @@ public static Schema of(Field... fields) { return Schema.builder().addFields(fields).build(); } - /** Returns an identical Schema with sorted fields. Recursively sorts nested 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} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java index 32c3d39d0950..578be14c1e07 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java @@ -24,7 +24,8 @@ 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.Schema; +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; @@ -47,7 +48,7 @@ public class IcebergReadSchemaTransformProvider extends TypedSchemaTransformProv @Override protected SchemaTransform from(Config configuration) { configuration.validate(); - return new IcebergReadSchemaTransform(configuration, configurationSchema()); + return new IcebergReadSchemaTransform(configuration); } @Override @@ -87,19 +88,20 @@ public void validate() { static class IcebergReadSchemaTransform extends SchemaTransform { private final Config configuration; - private final Row configurationRow; - IcebergReadSchemaTransform(Config configuration, Schema configSchema) { + IcebergReadSchemaTransform(Config configuration) { this.configuration = configuration; - configurationRow = - Row.withSchema(configSchema) - .withFieldValue("table", configuration.getTable()) - .withFieldValue("catalogConfig", configuration.getCatalogConfig().toRow()) - .build(); } Row getConfigurationRow() { - return configurationRow; + try { + return SchemaRegistry.createDefault() + .getToRowFunction(Config.class) + .apply(configuration) + .sorted(); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e); + } } @Override diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java index 624c57889cc6..05c5590a7331 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java @@ -24,7 +24,9 @@ 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; @@ -63,7 +65,7 @@ public String description() { @Override protected SchemaTransform from(Config configuration) { configuration.validate(); - return new IcebergWriteSchemaTransform(configuration, configurationSchema()); + return new IcebergWriteSchemaTransform(configuration); } @Override @@ -110,20 +112,20 @@ public void validate() { static class IcebergWriteSchemaTransform extends SchemaTransform { private final Config configuration; - private final Row configurationRow; - IcebergWriteSchemaTransform(Config configuration, Schema configSchema) { + IcebergWriteSchemaTransform(Config configuration) { this.configuration = configuration; - - configurationRow = - Row.withSchema(configSchema) - .withFieldValue("table", configuration.getTable()) - .withFieldValue("catalogConfig", configuration.getCatalogConfig().toRow()) - .build(); } Row getConfigurationRow() { - return configurationRow; + try { + return SchemaRegistry.createDefault() + .getToRowFunction(Config.class) + .apply(configuration) + .sorted(); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e); + } } @Override 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 9115255b4b71..f16aa031c611 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 @@ -34,7 +34,9 @@ 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; @@ -151,35 +153,46 @@ protected SchemaTransform from(ManagedConfig managedConfig) { } static class ManagedSchemaTransform extends SchemaTransform { - private final Row transformConfig; private final ManagedConfig managedConfig; + private final Row underlyingTransformConfig; private final SchemaTransformProvider underlyingTransformProvider; ManagedSchemaTransform( ManagedConfig managedConfig, SchemaTransformProvider underlyingTransformProvider) { // parse config before expansion to check if it matches underlying transform's config schema Schema transformConfigSchema = underlyingTransformProvider.configurationSchema(); - Row transformConfig; + Row underlyingTransformConfig; try { - transformConfig = getRowConfig(managedConfig, transformConfigSchema); + underlyingTransformConfig = getRowConfig(managedConfig, transformConfigSchema); } catch (Exception e) { throw new IllegalArgumentException( "Encountered an error when retrieving a Row configuration", e); } - this.transformConfig = transformConfig; this.managedConfig = managedConfig; + this.underlyingTransformConfig = underlyingTransformConfig; this.underlyingTransformProvider = underlyingTransformProvider; } @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { - return input.apply(underlyingTransformProvider.from(transformConfig)); + return input.apply(underlyingTransformProvider.from(underlyingTransformConfig)); } public ManagedConfig getManagedConfig() { return this.managedConfig; } + + Row getConfigurationRow() { + try { + return SchemaRegistry.createDefault() + .getToRowFunction(ManagedConfig.class) + .apply(managedConfig) + .sorted(); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e); + } + } } /** */ From 2992192b6377d4da5cca8eee3c850f8b63e859f0 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Fri, 19 Apr 2024 19:57:14 -0400 Subject: [PATCH 45/60] use snake_case convention when translating transforms to spec; remove Managed and Iceberg urns from proto and use SCHEMA_TRANSFORM URN --- .../model/pipeline/v1/beam_runner_api.proto | 6 --- .../construction/PTransformTranslation.java | 3 -- .../IcebergSchemaTransformTranslation.java | 45 +++++++++++----- ...IcebergSchemaTransformTranslationTest.java | 54 ++++++++++++++----- .../ManagedSchemaTransformTranslation.java | 41 +++++++------- .../managed/ManagedTransformConstants.java | 25 ++++++++- ...ManagedSchemaTransformTranslationTest.java | 45 +++++++++------- 7 files changed, 143 insertions(+), 76 deletions(-) diff --git a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto index d2445925d0e2..bbd326f91deb 100644 --- a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto +++ b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto @@ -443,12 +443,6 @@ message StandardPTransforms { enum GroupIntoBatchesComponents { GROUP_INTO_BATCHES = 0 [(beam_urn) = "beam:transform:group_into_batches:v1"]; } - // Payload for all of these: SchemaTransformPayload (in external_transforms.proto) - enum ManagedSchemaTransforms { - MANAGED = 0 [(beam_urn) = "beam:transform:managed:v1"]; - ICEBERG_READ = 1 [(beam_urn) = "beam:schematransform:org.apache.beam:iceberg_read:v1"]; - ICEBERG_WRITE = 2 [(beam_urn) = "beam:schematransform:org.apache.beam:iceberg_write:v1"]; - } } message StandardSideInputTypes { 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 b6c193674dcb..3167c2f22b0d 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 @@ -35,7 +35,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.StandardPTransforms; import org.apache.beam.model.pipeline.v1.RunnerApi.StandardPTransforms.CombineComponents; import org.apache.beam.model.pipeline.v1.RunnerApi.StandardPTransforms.SplittableParDoComponents; -import org.apache.beam.model.pipeline.v1.SchemaAwareTransforms.ManagedSchemaAwareTransforms; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.coders.RowCoder; @@ -93,7 +92,6 @@ public class PTransformTranslation { public static final String MAP_WINDOWS_TRANSFORM_URN = "beam:transform:map_windows:v1"; public static final String MERGE_WINDOWS_TRANSFORM_URN = "beam:transform:merge_windows:v1"; public static final String TO_STRING_TRANSFORM_URN = "beam:transform:to_string:v1"; - public static final String MANAGED_TRANSFORM_URN = "beam:transform:managed:v1"; // Required runner implemented transforms. These transforms should never specify an environment. public static final ImmutableSet RUNNER_IMPLEMENTED_TRANSFORMS = @@ -184,7 +182,6 @@ public class PTransformTranslation { MAP_WINDOWS_TRANSFORM_URN.equals(getUrn(StandardPTransforms.Primitives.MAP_WINDOWS))); checkState( MERGE_WINDOWS_TRANSFORM_URN.equals(getUrn(StandardPTransforms.Primitives.MERGE_WINDOWS))); - checkState(MANAGED_TRANSFORM_URN.equals(getUrn(ManagedSchemaAwareTransforms.Managed.MANAGED))); // DeprecatedPrimitives checkState(READ_TRANSFORM_URN.equals(getUrn(StandardPTransforms.DeprecatedPrimitives.READ))); 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 index 967315d4dfe1..30e914f091ae 100644 --- 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 @@ -17,6 +17,7 @@ */ 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.IcebergWriteSchemaTransform; @@ -33,6 +34,7 @@ import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.SchemaTranslation; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.BeamUrns; import org.apache.beam.sdk.util.construction.PTransformTranslation.TransformPayloadTranslator; import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.util.construction.TransformPayloadTranslatorRegistrar; @@ -51,25 +53,26 @@ static class IcebergReadSchemaTransformTranslator @Override public String getUrn() { - return READ_PROVIDER.identifier(); + return BeamUrns.getUrn(SCHEMA_TRANSFORM); } @Override public @Nullable FunctionSpec translate( AppliedPTransform application, SdkComponents components) throws IOException { - SchemaApi.Schema expansionSchema = SchemaTranslation.schemaToProto(READ_SCHEMA, true); + Schema snakeCaseSchema = READ_SCHEMA.toSnakeCase(); + SchemaApi.Schema expansionSchema = SchemaTranslation.schemaToProto(snakeCaseSchema, true); Row configRow = toConfigRow(application.getTransform()); ByteArrayOutputStream os = new ByteArrayOutputStream(); - RowCoder.of(READ_SCHEMA).encode(configRow, os); + RowCoder.of(snakeCaseSchema).encode(configRow, os); return FunctionSpec.newBuilder() .setUrn(getUrn()) .setPayload( SchemaTransformPayload.newBuilder() + .setIdentifier(READ_PROVIDER.identifier()) .setConfigurationSchema(expansionSchema) .setConfigurationRow(ByteString.copyFrom(os.toByteArray())) - .setIdentifier(getUrn()) .build() .toByteString()) .build(); @@ -77,12 +80,20 @@ public String getUrn() { @Override public Row toConfigRow(IcebergReadSchemaTransform transform) { - return transform.getConfigurationRow(); + // Will retrieve a Row with snake_case naming convention. + // Transform expects camelCase convention, so convert back + // TODO(https://github.com/apache/beam/issues/31061): Remove conversion when + // TypedSchemaTransformProvider starts generating with snake_case convention + return transform.getConfigurationRow().toSnakeCase(); } @Override public IcebergReadSchemaTransform fromConfigRow(Row configRow, PipelineOptions options) { - return (IcebergReadSchemaTransform) READ_PROVIDER.from(configRow); + // Will retrieve a Row with snake_case naming convention. + // Transform expects camelCase convention, so convert back + // TODO(https://github.com/apache/beam/issues/31061): Remove conversion when + // TypedSchemaTransformProvider starts generating with snake_case convention + return (IcebergReadSchemaTransform) READ_PROVIDER.from(configRow.toCamelCase()); } } @@ -109,25 +120,28 @@ static class IcebergWriteSchemaTransformTranslator @Override public String getUrn() { - return WRITE_PROVIDER.identifier(); + return BeamUrns.getUrn(SCHEMA_TRANSFORM); } @Override public @Nullable FunctionSpec translate( AppliedPTransform application, SdkComponents components) throws IOException { - SchemaApi.Schema expansionSchema = SchemaTranslation.schemaToProto(WRITE_SCHEMA, true); + // TODO(https://github.com/apache/beam/issues/31061): Remove conversion when + // TypedSchemaTransformProvider starts generating with snake_case convention + Schema snakeCaseSchema = WRITE_SCHEMA.toSnakeCase(); + SchemaApi.Schema expansionSchema = SchemaTranslation.schemaToProto(snakeCaseSchema, true); Row configRow = toConfigRow(application.getTransform()); ByteArrayOutputStream os = new ByteArrayOutputStream(); - RowCoder.of(WRITE_SCHEMA).encode(configRow, os); + RowCoder.of(snakeCaseSchema).encode(configRow, os); return FunctionSpec.newBuilder() .setUrn(getUrn()) .setPayload( SchemaTransformPayload.newBuilder() + .setIdentifier(WRITE_PROVIDER.identifier()) .setConfigurationSchema(expansionSchema) .setConfigurationRow(ByteString.copyFrom(os.toByteArray())) - .setIdentifier(getUrn()) .build() .toByteString()) .build(); @@ -135,12 +149,19 @@ public String getUrn() { @Override public Row toConfigRow(IcebergWriteSchemaTransform transform) { - return transform.getConfigurationRow(); + // Return with snake_case naming convention! + // TODO(https://github.com/apache/beam/issues/31061): Remove conversion when + // TypedSchemaTransformProvider starts generating with snake_case convention + return transform.getConfigurationRow().toSnakeCase(); } @Override public IcebergWriteSchemaTransform fromConfigRow(Row configRow, PipelineOptions options) { - return (IcebergWriteSchemaTransform) WRITE_PROVIDER.from(configRow); + // Will retrieve a Row with snake_case naming convention. + // Transform expects camelCase convention, so convert back + // TODO(https://github.com/apache/beam/issues/31061): Remove conversion when + // TypedSchemaTransformProvider starts generating with snake_case convention + return (IcebergWriteSchemaTransform) WRITE_PROVIDER.from(configRow.toCamelCase()); } } 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 index 1ae1209880b7..3d3b558a015e 100644 --- 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 @@ -17,6 +17,7 @@ */ 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.IcebergSchemaTransformTranslation.IcebergReadSchemaTransformTranslator.READ_SCHEMA; import static org.apache.beam.sdk.io.iceberg.IcebergSchemaTransformTranslation.IcebergWriteSchemaTransformTranslator.WRITE_SCHEMA; @@ -24,6 +25,7 @@ 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; @@ -32,16 +34,16 @@ 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.ManagedTransformConstants; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.schemas.NoSuchSchemaException; 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; @@ -64,7 +66,7 @@ public class IcebergSchemaTransformTranslationTest { new IcebergReadSchemaTransformProvider(); @Test - public void testReCreateWriteTransformFromRow() throws NoSuchSchemaException { + public void testReCreateWriteTransformFromRow() { Row catalogConfigRow = Row.withSchema(IcebergSchemaTransformCatalogConfig.SCHEMA) .withFieldValue("catalogName", "test_name") @@ -91,7 +93,8 @@ public void testReCreateWriteTransformFromRow() throws NoSuchSchemaException { } @Test - public void testWriteTransformProtoTranslation() throws Exception { + public void testWriteTransformProtoTranslation() + throws InvalidProtocolBufferException, IOException { // First build a pipeline Pipeline p = Pipeline.create(); Schema inputSchema = Schema.builder().addStringField("str").build(); @@ -122,7 +125,18 @@ public void testWriteTransformProtoTranslation() throws Exception { RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p); List writeTransformProto = pipelineProto.getComponents().getTransformsMap().values().stream() - .filter(tr -> tr.getSpec().getUrn().equals(ManagedTransformConstants.ICEBERG_WRITE)) + .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(); @@ -130,10 +144,12 @@ public void testWriteTransformProtoTranslation() throws Exception { // Check that the proto contains correct values SchemaTransformPayload payload = SchemaTransformPayload.parseFrom(spec.getPayload()); Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getConfigurationSchema()); - assertEquals(WRITE_SCHEMA, schemaFromSpec); + // TODO(https://github.com/apache/beam/issues/31061): Remove conversion when + // TypedSchemaTransformProvider starts generating with snake_case convention + assertEquals(WRITE_SCHEMA.toSnakeCase(), schemaFromSpec); Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getConfigurationRow().newInput()); - assertEquals(transformConfigRow, rowFromSpec); + assertEquals(transformConfigRow, rowFromSpec.toCamelCase()); // Use the information in the proto to recreate the IcebergWriteSchemaTransform IcebergSchemaTransformTranslation.IcebergWriteSchemaTransformTranslator translator = @@ -145,7 +161,7 @@ public void testWriteTransformProtoTranslation() throws Exception { } @Test - public void testReCreateReadTransformFromRow() throws NoSuchSchemaException { + public void testReCreateReadTransformFromRow() { // setting a subset of fields here. Row catalogConfigRow = Row.withSchema(IcebergSchemaTransformCatalogConfig.SCHEMA) @@ -174,7 +190,8 @@ public void testReCreateReadTransformFromRow() throws NoSuchSchemaException { } @Test - public void testReadTransformProtoTranslation() throws Exception { + public void testReadTransformProtoTranslation() + throws InvalidProtocolBufferException, IOException { // First build a pipeline Pipeline p = Pipeline.create(); Row catalogConfigRow = @@ -201,7 +218,18 @@ public void testReadTransformProtoTranslation() throws Exception { RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p); List readTransformProto = pipelineProto.getComponents().getTransformsMap().values().stream() - .filter(tr -> tr.getSpec().getUrn().equals(ManagedTransformConstants.ICEBERG_READ)) + .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(); @@ -209,9 +237,11 @@ public void testReadTransformProtoTranslation() throws Exception { // Check that the proto contains correct values SchemaTransformPayload payload = SchemaTransformPayload.parseFrom(spec.getPayload()); Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getConfigurationSchema()); - assertEquals(READ_SCHEMA, schemaFromSpec); + // TODO(https://github.com/apache/beam/issues/31061): Remove conversion when + // TypedSchemaTransformProvider starts generating with snake_case convention + assertEquals(READ_SCHEMA.toSnakeCase(), schemaFromSpec); Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getConfigurationRow().newInput()); - assertEquals(transformConfigRow, rowFromSpec); + assertEquals(transformConfigRow, rowFromSpec.toCamelCase()); // Use the information in the proto to recreate the IcebergReadSchemaTransform IcebergSchemaTransformTranslation.IcebergReadSchemaTransformTranslator translator = 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 index b4412f9d5dab..bc3ec621805e 100644 --- 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 @@ -17,8 +17,8 @@ */ 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.RunnerApi.FunctionSpec; -import static org.apache.beam.sdk.managed.ManagedSchemaTransformProvider.ManagedConfig; import static org.apache.beam.sdk.managed.ManagedSchemaTransformProvider.ManagedSchemaTransform; import static org.apache.beam.sdk.util.construction.PTransformTranslation.TransformPayloadTranslator; @@ -31,11 +31,10 @@ 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.NoSuchSchemaException; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.SchemaRegistry; import org.apache.beam.sdk.schemas.SchemaTranslation; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.BeamUrns; import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.util.construction.TransformPayloadTranslatorRegistrar; import org.apache.beam.sdk.values.Row; @@ -51,7 +50,7 @@ static class ManagedSchemaTransformTranslator @Override public String getUrn() { - return PROVIDER.identifier(); + return BeamUrns.getUrn(SCHEMA_TRANSFORM); } @Override @@ -59,17 +58,19 @@ public String getUrn() { public @Nullable FunctionSpec translate( AppliedPTransform application, SdkComponents components) throws IOException { - SchemaApi.Schema expansionSchema = SchemaTranslation.schemaToProto(SCHEMA, true); - ManagedConfig managedConfig = application.getTransform().getManagedConfig(); + // TODO(https://github.com/apache/beam/issues/31061): Remove conversion when + // TypedSchemaTransformProvider starts generating with snake_case convention + Schema snakeCaseSchema = SCHEMA.toSnakeCase(); + SchemaApi.Schema expansionSchema = SchemaTranslation.schemaToProto(snakeCaseSchema, true); Row configRow = toConfigRow(application.getTransform()); ByteArrayOutputStream os = new ByteArrayOutputStream(); - RowCoder.of(SCHEMA).encode(configRow, os); + RowCoder.of(snakeCaseSchema).encode(configRow, os); return FunctionSpec.newBuilder() .setUrn(getUrn()) .setPayload( SchemaTransformPayload.newBuilder() - .setIdentifier(managedConfig.getTransformIdentifier()) + .setIdentifier(PROVIDER.identifier()) .setConfigurationSchema(expansionSchema) .setConfigurationRow(ByteString.copyFrom(os.toByteArray())) .build() @@ -79,25 +80,19 @@ public String getUrn() { @Override public Row toConfigRow(ManagedSchemaTransform transform) { - ManagedConfig managedConfig = transform.getManagedConfig(); - Row configRow; - try { - configRow = - SchemaRegistry.createDefault() - .getToRowFunction(ManagedConfig.class) - .apply(managedConfig); - } catch (NoSuchSchemaException e) { - throw new RuntimeException(e); - } - // Sanity check: sort fields according to the configuration schema - return SCHEMA.getFields().stream() - .map(field -> configRow.getValue(field.getName())) - .collect(Row.toRow(SCHEMA)); + // Return with snake_case naming convention! + // TODO(https://github.com/apache/beam/issues/31061): Remove conversion when + // TypedSchemaTransformProvider starts generating with snake_case convention + return transform.getConfigurationRow().toSnakeCase(); } @Override public ManagedSchemaTransform fromConfigRow(Row configRow, PipelineOptions options) { - return (ManagedSchemaTransform) PROVIDER.from(configRow); + // Will retrieve a Row with snake_case naming convention. + // Transform expects camelCase convention, so convert back + // TODO(https://github.com/apache/beam/issues/31061): Remove conversion when + // TypedSchemaTransformProvider starts generating with snake_case convention + return (ManagedSchemaTransform) PROVIDER.from(configRow.toCamelCase()); } } diff --git a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java index 2d643b98abec..48735d8c33a3 100644 --- a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java +++ b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java @@ -1,2 +1,25 @@ -package org.apache.beam.sdk.managed;public class ManagedTransformConstants { +/* + * 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/ManagedSchemaTransformTranslationTest.java b/sdks/java/managed/src/test/java/org/apache/beam/sdk/managed/ManagedSchemaTransformTranslationTest.java index d60bf8df4621..553d1dbb8469 100644 --- 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 @@ -17,6 +17,7 @@ */ 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; @@ -40,23 +41,17 @@ 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.PTransformTranslation; +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 { - // A mapping from ManagedConfig builder methods to the corresponding schema fields in - // ManagedSchemaTransformTranslation. - static final Map MANAGED_TRANSFORM_SCHEMA_MAPPING = - ImmutableMap.builder() - .put("getTransformIdentifier", "transform_identifier") - .put("getConfigUrl", "config_url") - .put("getConfig", "config") - .build(); + static ManagedSchemaTransformProvider PROVIDER = new ManagedSchemaTransformProvider(null); @Test public void testReCreateTransformFromRowWithConfigUrl() throws URISyntaxException { @@ -71,9 +66,8 @@ public void testReCreateTransformFromRowWithConfigUrl() throws URISyntaxExceptio .setConfigUrl(yamlConfigPath) .build(); - ManagedSchemaTransformProvider provider = new ManagedSchemaTransformProvider(null); ManagedSchemaTransform originalTransform = - (ManagedSchemaTransform) provider.from(originalConfig); + (ManagedSchemaTransform) PROVIDER.from(originalConfig); ManagedSchemaTransformTranslator translator = new ManagedSchemaTransformTranslator(); Row configRow = translator.toConfigRow(originalTransform); @@ -98,9 +92,8 @@ public void testReCreateTransformFromRowWithConfig() { .setConfig(yamlString) .build(); - ManagedSchemaTransformProvider provider = new ManagedSchemaTransformProvider(null); ManagedSchemaTransform originalTransform = - (ManagedSchemaTransform) provider.from(originalConfig); + (ManagedSchemaTransform) PROVIDER.from(originalConfig); ManagedSchemaTransformTranslator translator = new ManagedSchemaTransformTranslator(); Row configRow = translator.toConfigRow(originalTransform); @@ -147,21 +140,35 @@ public void testProtoTranslation() throws Exception { RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p); List managedTransformProto = pipelineProto.getComponents().getTransformsMap().values().stream() - .filter(tr -> tr.getSpec().getUrn().equals(PTransformTranslation.MANAGED_TRANSFORM_URN)) + .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(TestSchemaTransformProvider.IDENTIFIER, payload.getIdentifier()); + assertEquals(PROVIDER.identifier(), payload.getIdentifier()); Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getConfigurationSchema()); - assertEquals(ManagedSchemaTransformTranslator.SCHEMA, schemaFromSpec); + // TODO(https://github.com/apache/beam/issues/31061): Remove conversion when + // TypedSchemaTransformProvider starts generating with snake_case convention + assertEquals(ManagedSchemaTransformTranslator.SCHEMA, schemaFromSpec.toCamelCase()); Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getConfigurationRow().newInput()); + // Translation logic outputs a Row with snake_case naming convention Row expectedRow = - Row.withSchema(ManagedSchemaTransformTranslator.SCHEMA) - .withFieldValue("transformIdentifier", TestSchemaTransformProvider.IDENTIFIER) - .withFieldValue("configUrl", null) + Row.withSchema(ManagedSchemaTransformTranslator.SCHEMA.toSnakeCase()) + .withFieldValue("transform_identifier", TestSchemaTransformProvider.IDENTIFIER) + .withFieldValue("config_url", null) .withFieldValue("config", yamlStringConfig) .build(); assertEquals(expectedRow, rowFromSpec); From b311068a63f3746e89e49c5ae7c60d533bca78de Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Fri, 19 Apr 2024 19:57:38 -0400 Subject: [PATCH 46/60] spotless --- .../java/org/apache/beam/sdk/io/iceberg/FileWriteResult.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/FileWriteResult.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/FileWriteResult.java index d058af7d7d3e..ec5310b9f3b3 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/FileWriteResult.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/FileWriteResult.java @@ -92,10 +92,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 (whether with Beam, Iceberg, Avro, etc..), + // "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(0); + tableIdentifierCoder.encode(value.getTableIdentifier().toString(), outStream); partitionSpecCoder.encode(value.getPartitionSpec(), outStream); dataFileBytesCoder.encode( @@ -106,6 +108,7 @@ public void encode(FileWriteResult value, OutputStream outStream) 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 = From 2461b446dd3ee74adee2919dd7f2fbd94c4e007e Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Fri, 19 Apr 2024 23:17:22 -0400 Subject: [PATCH 47/60] cleanup --- .../src/main/java/org/apache/beam/sdk/schemas/Schema.java | 2 +- .../java/org/apache/beam/sdk/schemas/utils/YamlUtils.java | 2 +- .../core/src/main/java/org/apache/beam/sdk/values/Row.java | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) 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 da3cca1434b0..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 @@ -1466,7 +1466,7 @@ public Options getOptions() { return this.options; } - /** Recursively converts all field names to `lower_snake_case`. */ + /** Recursively converts all field names to `snake_case`. */ public Schema toSnakeCase() { return this.getFields().stream() .map( 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 049da266c45d..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 @@ -53,7 +53,7 @@ public class YamlUtils { .put(Schema.TypeName.BYTES, str -> BaseEncoding.base64().decode(str)) .build(); - public static @Nullable Row toBeamRow(@Nullable String yamlString, Schema schema) { + public static Row toBeamRow(@Nullable String yamlString, Schema schema) { return toBeamRow(yamlString, schema, false); } 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 9abaa61f019f..451c804c1993 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 @@ -894,7 +894,7 @@ public static Row nullRow(Schema schema) { .build(); } - /** Returns an identical {@link Row} with fields lexicographically sorted by their name. */ + /** Returns an equivalent {@link Row} with fields lexicographically sorted by their name. */ public Row sorted() { Schema sortedSchema = getSchema().sorted(); return sortedSchema.getFields().stream() @@ -908,7 +908,7 @@ public Row sorted() { .collect(Row.toRow(sortedSchema)); } - /** Returns an equivalent {@link Row} with `lower_snake_case` field names. */ + /** Returns an equivalent {@link Row} with `snake_case` field names. */ public Row toSnakeCase() { return getSchema().getFields().stream() .map( From ecb4dbb1f7aca39c5ab1944dbda2a96ba42b73d4 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Fri, 19 Apr 2024 23:18:21 -0400 Subject: [PATCH 48/60] DefaultSchemaProvider can now provide the underlying SchemaProvider --- .../beam/sdk/schemas/SchemaRegistry.java | 23 +++++++++ .../schemas/annotations/DefaultSchema.java | 18 +++++++ .../beam/sdk/schemas/SchemaRegistryTest.java | 49 +++++++++++++++++++ 3 files changed, 90 insertions(+) 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/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(); + } + } } From 68895a79529535342d8ed1056dab99a05ea23950 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Fri, 19 Apr 2024 23:26:43 -0400 Subject: [PATCH 49/60] perform snake_case <-> camelCase conversions directly in TypedSchemaTransformProvider --- .../TypedSchemaTransformProvider.java | 28 ++++++++++++++++--- .../TypedSchemaTransformProviderTest.java | 24 ++++++++-------- 2 files changed, 37 insertions(+), 15 deletions(-) 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..b59da955526a 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,6 +17,7 @@ */ 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; @@ -26,10 +27,13 @@ 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; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.common.base.Preconditions; /** * Like {@link SchemaTransformProvider} except uses a configuration object instead of Schema and @@ -77,7 +81,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 +91,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 +107,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) + && Preconditions.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/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()); From d2135b8b06d2ed4963fd9342bf0b77c1bc4b77e8 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Fri, 19 Apr 2024 23:30:04 -0400 Subject: [PATCH 50/60] update icebergIO and managed translations to reflect field name convention changes --- .../IcebergReadSchemaTransformProvider.java | 3 +- .../IcebergSchemaTransformCatalogConfig.java | 11 ++-- .../IcebergSchemaTransformTranslation.java | 35 +++---------- .../IcebergWriteSchemaTransformProvider.java | 3 +- ...cebergReadSchemaTransformProviderTest.java | 17 +++---- ...IcebergSchemaTransformTranslationTest.java | 50 +++++++++---------- ...ebergWriteSchemaTransformProviderTest.java | 17 +++---- .../ManagedSchemaTransformProvider.java | 5 +- .../ManagedSchemaTransformTranslation.java | 18 ++----- .../testing/TestSchemaTransformProvider.java | 17 ++----- .../ManagedSchemaTransformProviderTest.java | 8 +-- ...ManagedSchemaTransformTranslationTest.java | 6 +-- 12 files changed, 72 insertions(+), 118 deletions(-) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java index 578be14c1e07..46b1129bf0f4 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java @@ -98,7 +98,8 @@ Row getConfigurationRow() { return SchemaRegistry.createDefault() .getToRowFunction(Config.class) .apply(configuration) - .sorted(); + .sorted() + .toSnakeCase(); } catch (NoSuchSchemaException e) { throw new RuntimeException(e); } 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 index 0acdecc6866d..605592c98fce 100644 --- 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 @@ -71,7 +71,8 @@ public abstract static class Builder { SCHEMA = SchemaRegistry.createDefault() .getSchema(IcebergSchemaTransformCatalogConfig.class) - .sorted(); + .sorted() + .toSnakeCase(); } catch (NoSuchSchemaException e) { throw new RuntimeException(e); } @@ -80,10 +81,10 @@ public abstract static class Builder { @SuppressWarnings("argument") public Row toRow() { return Row.withSchema(SCHEMA) - .withFieldValue("catalogName", getCatalogName()) - .withFieldValue("catalogType", getCatalogType()) - .withFieldValue("catalogImplementation", getCatalogImplementation()) - .withFieldValue("warehouseLocation", getWarehouseLocation()) + .withFieldValue("catalog_name", getCatalogName()) + .withFieldValue("catalog_type", getCatalogType()) + .withFieldValue("catalog_implementation", getCatalogImplementation()) + .withFieldValue("warehouse_location", getWarehouseLocation()) .build(); } 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 index 30e914f091ae..e3ed91c932a1 100644 --- 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 @@ -60,11 +60,10 @@ public String getUrn() { public @Nullable FunctionSpec translate( AppliedPTransform application, SdkComponents components) throws IOException { - Schema snakeCaseSchema = READ_SCHEMA.toSnakeCase(); - SchemaApi.Schema expansionSchema = SchemaTranslation.schemaToProto(snakeCaseSchema, true); + SchemaApi.Schema expansionSchema = SchemaTranslation.schemaToProto(READ_SCHEMA, true); Row configRow = toConfigRow(application.getTransform()); ByteArrayOutputStream os = new ByteArrayOutputStream(); - RowCoder.of(snakeCaseSchema).encode(configRow, os); + RowCoder.of(READ_SCHEMA).encode(configRow, os); return FunctionSpec.newBuilder() .setUrn(getUrn()) @@ -80,20 +79,12 @@ public String getUrn() { @Override public Row toConfigRow(IcebergReadSchemaTransform transform) { - // Will retrieve a Row with snake_case naming convention. - // Transform expects camelCase convention, so convert back - // TODO(https://github.com/apache/beam/issues/31061): Remove conversion when - // TypedSchemaTransformProvider starts generating with snake_case convention - return transform.getConfigurationRow().toSnakeCase(); + return transform.getConfigurationRow(); } @Override public IcebergReadSchemaTransform fromConfigRow(Row configRow, PipelineOptions options) { - // Will retrieve a Row with snake_case naming convention. - // Transform expects camelCase convention, so convert back - // TODO(https://github.com/apache/beam/issues/31061): Remove conversion when - // TypedSchemaTransformProvider starts generating with snake_case convention - return (IcebergReadSchemaTransform) READ_PROVIDER.from(configRow.toCamelCase()); + return (IcebergReadSchemaTransform) READ_PROVIDER.from(configRow); } } @@ -127,13 +118,10 @@ public String getUrn() { public @Nullable FunctionSpec translate( AppliedPTransform application, SdkComponents components) throws IOException { - // TODO(https://github.com/apache/beam/issues/31061): Remove conversion when - // TypedSchemaTransformProvider starts generating with snake_case convention - Schema snakeCaseSchema = WRITE_SCHEMA.toSnakeCase(); - SchemaApi.Schema expansionSchema = SchemaTranslation.schemaToProto(snakeCaseSchema, true); + SchemaApi.Schema expansionSchema = SchemaTranslation.schemaToProto(WRITE_SCHEMA, true); Row configRow = toConfigRow(application.getTransform()); ByteArrayOutputStream os = new ByteArrayOutputStream(); - RowCoder.of(snakeCaseSchema).encode(configRow, os); + RowCoder.of(WRITE_SCHEMA).encode(configRow, os); return FunctionSpec.newBuilder() .setUrn(getUrn()) @@ -149,19 +137,12 @@ public String getUrn() { @Override public Row toConfigRow(IcebergWriteSchemaTransform transform) { - // Return with snake_case naming convention! - // TODO(https://github.com/apache/beam/issues/31061): Remove conversion when - // TypedSchemaTransformProvider starts generating with snake_case convention - return transform.getConfigurationRow().toSnakeCase(); + return transform.getConfigurationRow(); } @Override public IcebergWriteSchemaTransform fromConfigRow(Row configRow, PipelineOptions options) { - // Will retrieve a Row with snake_case naming convention. - // Transform expects camelCase convention, so convert back - // TODO(https://github.com/apache/beam/issues/31061): Remove conversion when - // TypedSchemaTransformProvider starts generating with snake_case convention - return (IcebergWriteSchemaTransform) WRITE_PROVIDER.from(configRow.toCamelCase()); + return (IcebergWriteSchemaTransform) WRITE_PROVIDER.from(configRow); } } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java index 05c5590a7331..8f90b650dbcd 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java @@ -122,7 +122,8 @@ Row getConfigurationRow() { return SchemaRegistry.createDefault() .getToRowFunction(Config.class) .apply(configuration) - .sorted(); + .sorted() + .toSnakeCase(); } catch (NoSuchSchemaException e) { throw new RuntimeException(e); } 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 index 4d3794138a66..22514b035450 100644 --- 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 @@ -27,9 +27,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.beam.sdk.managed.Managed; -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.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollection; @@ -53,19 +51,18 @@ public class IcebergReadSchemaTransformProviderTest { @Rule public TestPipeline testPipeline = TestPipeline.create(); @Test - public void testBuildTransformWithRow() throws NoSuchSchemaException { + public void testBuildTransformWithRow() { Row catalogConfigRow = - Row.withSchema( - SchemaRegistry.createDefault().getSchema(IcebergSchemaTransformCatalogConfig.class)) - .withFieldValue("catalogName", "test_name") - .withFieldValue("catalogType", "test_type") - .withFieldValue("catalogImplementation", "testImplementation") - .withFieldValue("warehouseLocation", "test_location") + 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("catalogConfig", catalogConfigRow) + .withFieldValue("catalog_config", catalogConfigRow) .build(); new IcebergReadSchemaTransformProvider().from(transformConfigRow); 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 index 3d3b558a015e..5fc02af833f5 100644 --- 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 @@ -69,15 +69,15 @@ public class IcebergSchemaTransformTranslationTest { public void testReCreateWriteTransformFromRow() { Row catalogConfigRow = Row.withSchema(IcebergSchemaTransformCatalogConfig.SCHEMA) - .withFieldValue("catalogName", "test_name") - .withFieldValue("catalogType", "test_type") - .withFieldValue("catalogImplementation", "testImplementation") - .withFieldValue("warehouseLocation", "test_location") + .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_SCHEMA) .withFieldValue("table", "test_table_identifier") - .withFieldValue("catalogConfig", catalogConfigRow) + .withFieldValue("catalog_config", catalogConfigRow) .build(); IcebergWriteSchemaTransform writeTransform = (IcebergWriteSchemaTransform) WRITE_PROVIDER.from(transformConfigRow); @@ -106,15 +106,15 @@ public void testWriteTransformProtoTranslation() Row catalogConfigRow = Row.withSchema(IcebergSchemaTransformCatalogConfig.SCHEMA) - .withFieldValue("catalogName", "test_catalog") - .withFieldValue("catalogType", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) - .withFieldValue("catalogImplementation", "test_implementation") - .withFieldValue("warehouseLocation", warehouse.location) + .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_SCHEMA) .withFieldValue("table", "test_identifier") - .withFieldValue("catalogConfig", catalogConfigRow) + .withFieldValue("catalog_config", catalogConfigRow) .build(); IcebergWriteSchemaTransform writeTransform = @@ -144,12 +144,10 @@ public void testWriteTransformProtoTranslation() // Check that the proto contains correct values SchemaTransformPayload payload = SchemaTransformPayload.parseFrom(spec.getPayload()); Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getConfigurationSchema()); - // TODO(https://github.com/apache/beam/issues/31061): Remove conversion when - // TypedSchemaTransformProvider starts generating with snake_case convention - assertEquals(WRITE_SCHEMA.toSnakeCase(), schemaFromSpec); + assertEquals(WRITE_SCHEMA, schemaFromSpec); Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getConfigurationRow().newInput()); - assertEquals(transformConfigRow, rowFromSpec.toCamelCase()); + assertEquals(transformConfigRow, rowFromSpec); // Use the information in the proto to recreate the IcebergWriteSchemaTransform IcebergSchemaTransformTranslation.IcebergWriteSchemaTransformTranslator translator = @@ -165,15 +163,15 @@ public void testReCreateReadTransformFromRow() { // setting a subset of fields here. Row catalogConfigRow = Row.withSchema(IcebergSchemaTransformCatalogConfig.SCHEMA) - .withFieldValue("catalogName", "test_name") - .withFieldValue("catalogType", "test_type") - .withFieldValue("catalogImplementation", "testImplementation") - .withFieldValue("warehouseLocation", "test_location") + .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_SCHEMA) .withFieldValue("table", "test_table_identifier") - .withFieldValue("catalogConfig", catalogConfigRow) + .withFieldValue("catalog_config", catalogConfigRow) .build(); IcebergReadSchemaTransform readTransform = @@ -196,9 +194,9 @@ public void testReadTransformProtoTranslation() Pipeline p = Pipeline.create(); Row catalogConfigRow = Row.withSchema(IcebergSchemaTransformCatalogConfig.SCHEMA) - .withFieldValue("catalogName", "test_catalog") - .withFieldValue("catalogType", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) - .withFieldValue("warehouseLocation", warehouse.location) + .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); @@ -206,7 +204,7 @@ public void testReadTransformProtoTranslation() Row transformConfigRow = Row.withSchema(READ_SCHEMA) .withFieldValue("table", identifier) - .withFieldValue("catalogConfig", catalogConfigRow) + .withFieldValue("catalog_config", catalogConfigRow) .build(); IcebergReadSchemaTransform readTransform = @@ -237,11 +235,9 @@ public void testReadTransformProtoTranslation() // Check that the proto contains correct values SchemaTransformPayload payload = SchemaTransformPayload.parseFrom(spec.getPayload()); Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getConfigurationSchema()); - // TODO(https://github.com/apache/beam/issues/31061): Remove conversion when - // TypedSchemaTransformProvider starts generating with snake_case convention - assertEquals(READ_SCHEMA.toSnakeCase(), schemaFromSpec); + assertEquals(READ_SCHEMA, schemaFromSpec); Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getConfigurationRow().newInput()); - assertEquals(transformConfigRow, rowFromSpec.toCamelCase()); + assertEquals(transformConfigRow, rowFromSpec); // Use the information in the proto to recreate the IcebergReadSchemaTransform IcebergSchemaTransformTranslation.IcebergReadSchemaTransformTranslator translator = 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 index 26bc4efbf0b1..5089a58f0a97 100644 --- 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 @@ -27,8 +27,6 @@ import java.util.Map; import java.util.UUID; import org.apache.beam.sdk.managed.Managed; -import org.apache.beam.sdk.schemas.NoSuchSchemaException; -import org.apache.beam.sdk.schemas.SchemaRegistry; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; @@ -62,19 +60,18 @@ public class IcebergWriteSchemaTransformProviderTest { @Rule public transient TestPipeline testPipeline = TestPipeline.create(); @Test - public void testBuildTransformWithRow() throws NoSuchSchemaException { + public void testBuildTransformWithRow() { Row catalogConfigRow = - Row.withSchema( - SchemaRegistry.createDefault().getSchema(IcebergSchemaTransformCatalogConfig.class)) - .withFieldValue("catalogName", "test_name") - .withFieldValue("catalogType", "test_type") - .withFieldValue("catalogImplementation", "testImplementation") - .withFieldValue("warehouseLocation", "test_location") + 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("catalogConfig", catalogConfigRow) + .withFieldValue("catalog_config", catalogConfigRow) .build(); new IcebergWriteSchemaTransformProvider().from(transformConfigRow); 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 f16aa031c611..eb4c9444af28 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 @@ -188,7 +188,8 @@ Row getConfigurationRow() { return SchemaRegistry.createDefault() .getToRowFunction(ManagedConfig.class) .apply(managedConfig) - .sorted(); + .sorted() + .toSnakeCase(); } catch (NoSuchSchemaException e) { throw new RuntimeException(e); } @@ -200,7 +201,7 @@ Row getConfigurationRow() { 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, true); + return YamlUtils.toBeamRow(config.resolveUnderlyingConfig(), transformSchema, false); } Map getAllProviders() { 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 index bc3ec621805e..ef0cdfa7105d 100644 --- 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 @@ -58,13 +58,10 @@ public String getUrn() { public @Nullable FunctionSpec translate( AppliedPTransform application, SdkComponents components) throws IOException { - // TODO(https://github.com/apache/beam/issues/31061): Remove conversion when - // TypedSchemaTransformProvider starts generating with snake_case convention - Schema snakeCaseSchema = SCHEMA.toSnakeCase(); - SchemaApi.Schema expansionSchema = SchemaTranslation.schemaToProto(snakeCaseSchema, true); + SchemaApi.Schema expansionSchema = SchemaTranslation.schemaToProto(SCHEMA, true); Row configRow = toConfigRow(application.getTransform()); ByteArrayOutputStream os = new ByteArrayOutputStream(); - RowCoder.of(snakeCaseSchema).encode(configRow, os); + RowCoder.of(SCHEMA).encode(configRow, os); return FunctionSpec.newBuilder() .setUrn(getUrn()) @@ -80,19 +77,12 @@ public String getUrn() { @Override public Row toConfigRow(ManagedSchemaTransform transform) { - // Return with snake_case naming convention! - // TODO(https://github.com/apache/beam/issues/31061): Remove conversion when - // TypedSchemaTransformProvider starts generating with snake_case convention - return transform.getConfigurationRow().toSnakeCase(); + return transform.getConfigurationRow(); } @Override public ManagedSchemaTransform fromConfigRow(Row configRow, PipelineOptions options) { - // Will retrieve a Row with snake_case naming convention. - // Transform expects camelCase convention, so convert back - // TODO(https://github.com/apache/beam/issues/31061): Remove conversion when - // TypedSchemaTransformProvider starts generating with snake_case convention - return (ManagedSchemaTransform) PROVIDER.from(configRow.toCamelCase()); + return (ManagedSchemaTransform) PROVIDER.from(configRow); } } diff --git a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/testing/TestSchemaTransformProvider.java b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/testing/TestSchemaTransformProvider.java index 92c2de80109d..2771ecd01643 100644 --- a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/testing/TestSchemaTransformProvider.java +++ b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/testing/TestSchemaTransformProvider.java @@ -20,9 +20,7 @@ import com.google.auto.service.AutoService; import com.google.auto.value.AutoValue; 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; @@ -37,16 +35,9 @@ @AutoService(SchemaTransformProvider.class) public class TestSchemaTransformProvider extends TypedSchemaTransformProvider { - public static final String IDENTIFIER = "beam:test_schematransform:v1"; - public static final Schema SCHEMA; - - static { - try { - SCHEMA = SchemaRegistry.createDefault().getSchema(Config.class); - } catch (NoSuchSchemaException e) { - throw new RuntimeException(e); - } - } + 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 @@ -104,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/test/java/org/apache/beam/sdk/managed/ManagedSchemaTransformProviderTest.java b/sdks/java/managed/src/test/java/org/apache/beam/sdk/managed/ManagedSchemaTransformProviderTest.java index 0ce349e53285..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 @@ -60,8 +60,8 @@ public void testGetConfigRowFromYamlString() { Row expectedRow = Row.withSchema(TestSchemaTransformProvider.SCHEMA) - .withFieldValue("extraString", "abc") - .withFieldValue("extraInteger", 123) + .withFieldValue("extra_string", "abc") + .withFieldValue("extra_integer", 123) .build(); Row returnedRow = @@ -84,8 +84,8 @@ public void testGetConfigRowFromYamlFile() 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( 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 index 553d1dbb8469..24b9c3cd291a 100644 --- 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 @@ -160,13 +160,11 @@ public void testProtoTranslation() throws Exception { SchemaTransformPayload payload = SchemaTransformPayload.parseFrom(spec.getPayload()); assertEquals(PROVIDER.identifier(), payload.getIdentifier()); Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getConfigurationSchema()); - // TODO(https://github.com/apache/beam/issues/31061): Remove conversion when - // TypedSchemaTransformProvider starts generating with snake_case convention - assertEquals(ManagedSchemaTransformTranslator.SCHEMA, schemaFromSpec.toCamelCase()); + assertEquals(ManagedSchemaTransformTranslator.SCHEMA, schemaFromSpec); Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getConfigurationRow().newInput()); // Translation logic outputs a Row with snake_case naming convention Row expectedRow = - Row.withSchema(ManagedSchemaTransformTranslator.SCHEMA.toSnakeCase()) + Row.withSchema(ManagedSchemaTransformTranslator.SCHEMA) .withFieldValue("transform_identifier", TestSchemaTransformProvider.IDENTIFIER) .withFieldValue("config_url", null) .withFieldValue("config", yamlStringConfig) From 64863ce25304ce823a6b3ccad2ced6e24c068a71 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Fri, 19 Apr 2024 23:30:48 -0400 Subject: [PATCH 51/60] sorted SnapshotInfo --- .../beam/sdk/io/iceberg/SnapshotInfo.java | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) 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 index e64fdf6100ec..06b898fc68ca 100644 --- 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 @@ -20,14 +20,20 @@ import com.google.auto.value.AutoValue; import java.util.Map; import javax.annotation.Nullable; -import org.apache.beam.sdk.schemas.*; +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 a Beam Schema-compatible representation of an Iceberg {@link - * 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 @@ -47,7 +53,10 @@ public static SnapshotInfo fromSnapshot(Snapshot snapshot) { public Row toRow() { try { - return SchemaRegistry.createDefault().getToRowFunction(SnapshotInfo.class).apply(this); + return SchemaRegistry.createDefault() + .getToRowFunction(SnapshotInfo.class) + .apply(this) + .sorted(); } catch (NoSuchSchemaException e) { throw new RuntimeException(e); } @@ -60,7 +69,7 @@ public Row toRow() { try { SchemaRegistry registry = SchemaRegistry.createDefault(); CODER = registry.getSchemaCoder(SnapshotInfo.class); - SCHEMA = registry.getSchema(SnapshotInfo.class); + SCHEMA = registry.getSchema(SnapshotInfo.class).sorted(); } catch (NoSuchSchemaException e) { throw new RuntimeException(e); } From 5afb63386268d3556b26eb96331a678111a89ef3 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Fri, 19 Apr 2024 23:52:54 -0400 Subject: [PATCH 52/60] update manual Python wrappers to use snake_case convention; remove case conversion step from Python auto-xlang; spotless --- ...am_PostCommit_Python_Xlang_Gcp_Direct.json | 3 ++- ...m_PostCommit_Python_Xlang_IO_Dataflow.json | 3 ++- .../TypedSchemaTransformProvider.java | 4 ++-- .../beam/sdk/io/iceberg/SnapshotInfo.java | 4 ++-- ...ManagedSchemaTransformTranslationTest.java | 2 +- sdks/python/apache_beam/io/gcp/bigquery.py | 14 ++++++------- sdks/python/apache_beam/io/gcp/bigtableio.py | 12 +++++------ sdks/python/gen_xlang_wrappers.py | 21 +------------------ 8 files changed, 23 insertions(+), 40 deletions(-) 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/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 b59da955526a..1e5984aabaaa 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 @@ -20,6 +20,7 @@ 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; @@ -33,7 +34,6 @@ import org.apache.beam.sdk.schemas.SchemaProvider; import org.apache.beam.sdk.schemas.SchemaRegistry; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.grpc.v1p60p1.com.google.common.base.Preconditions; /** * Like {@link SchemaTransformProvider} except uses a configuration object instead of Schema and @@ -113,7 +113,7 @@ private ConfigT configFromRow(Row configuration) { // camelCase naming convention SchemaProvider schemaProvider = registry.getSchemaProvider(configurationClass()); if (schemaProvider.getClass().equals(DefaultSchemaProvider.class) - && Preconditions.checkNotNull( + && checkNotNull( ((DefaultSchemaProvider) schemaProvider) .getUnderlyingSchemaProvider(configurationClass())) .getClass() 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 index 06b898fc68ca..03240eff9f7b 100644 --- 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 @@ -62,8 +62,8 @@ public Row toRow() { } } - public static SchemaCoder CODER; - public static Schema SCHEMA; + public static final SchemaCoder CODER; + public static final Schema SCHEMA; static { try { 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 index 24b9c3cd291a..33e023718823 100644 --- 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 @@ -51,7 +51,7 @@ import org.junit.Test; public class ManagedSchemaTransformTranslationTest { - static ManagedSchemaTransformProvider PROVIDER = new ManagedSchemaTransformProvider(null); + static final ManagedSchemaTransformProvider PROVIDER = new ManagedSchemaTransformProvider(null); @Test public void testReCreateTransformFromRowWithConfigUrl() throws URISyntaxException { diff --git a/sdks/python/apache_beam/io/gcp/bigquery.py b/sdks/python/apache_beam/io/gcp/bigquery.py index 08698b273b1e..1d8183b2f44f 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/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'], } From 2ddd5bbfb0ca04d9f1ee744cc7c93a2a02c7270b Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Sat, 20 Apr 2024 00:26:38 -0400 Subject: [PATCH 53/60] Row utils allow nullable --- .../main/java/org/apache/beam/sdk/values/Row.java | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) 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 451c804c1993..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 @@ -901,7 +901,10 @@ public Row sorted() { .map( field -> { if (field.getType().getRowSchema() != null) { - return ((Row) getValue(field.getName())).sorted(); + Row innerRow = getValue(field.getName()); + if (innerRow != null) { + return innerRow.sorted(); + } } return (Object) getValue(field.getName()); }) @@ -914,7 +917,10 @@ public Row toSnakeCase() { .map( field -> { if (field.getType().getRowSchema() != null) { - return ((Row) getValue(field.getName())).toSnakeCase(); + Row innerRow = getValue(field.getName()); + if (innerRow != null) { + return innerRow.toSnakeCase(); + } } return (Object) getValue(field.getName()); }) @@ -927,7 +933,10 @@ public Row toCamelCase() { .map( field -> { if (field.getType().getRowSchema() != null) { - return ((Row) getValue(field.getName())).toCamelCase(); + Row innerRow = getValue(field.getName()); + if (innerRow != null) { + return innerRow.toCamelCase(); + } } return (Object) getValue(field.getName()); }) From d5a4d669425dace8870ee25c3f7e3c01387b42a2 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Mon, 22 Apr 2024 09:16:00 -0400 Subject: [PATCH 54/60] add FileWriteResult test for version number; fix existing Java and YAML tests --- .../FileWriteSchemaTransformProviderTest.java | 32 ++++++------------- ...xportReadSchemaTransformConfiguration.java | 14 -------- ...oadsWriteSchemaTransformConfiguration.java | 14 -------- ...ExportReadSchemaTransformProviderTest.java | 25 +++++++-------- ...LoadsWriteSchemaTransformProviderTest.java | 12 +++---- .../beam/sdk/io/iceberg/FileWriteResult.java | 3 +- .../sdk/io/iceberg/FileWriteResultTest.java | 15 +++++++++ .../KafkaReadSchemaTransformProviderTest.java | 16 +++++----- sdks/python/apache_beam/yaml/yaml_provider.py | 2 +- 9 files changed, 54 insertions(+), 79 deletions(-) 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/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/FileWriteResult.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/FileWriteResult.java index ec5310b9f3b3..c12febc03f48 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/FileWriteResult.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/FileWriteResult.java @@ -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(); @@ -96,7 +97,7 @@ public void encode(FileWriteResult value, OutputStream outStream) // 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(0); + outStream.write(VERSION); tableIdentifierCoder.encode(value.getTableIdentifier().toString(), outStream); partitionSpecCoder.encode(value.getPartitionSpec(), outStream); diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/FileWriteResultTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/FileWriteResultTest.java index f38485e58e16..644130593152 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/FileWriteResultTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/FileWriteResultTest.java @@ -21,7 +21,10 @@ 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/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/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index dcf7ffaa6af3..c5ae7b583afe 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) From 3b74f7743d46aea4d6e6080e290d568fadd6470e Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Mon, 22 Apr 2024 09:16:45 -0400 Subject: [PATCH 55/60] add schema-aware transform urn to transform annotations during translation --- .../sdk/util/construction/PTransformTranslation.java | 11 +++++++++++ 1 file changed, 11 insertions(+) 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; From af650323402d46cd80f3de91ecf3e19cde5203ba Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Mon, 22 Apr 2024 09:28:07 -0400 Subject: [PATCH 56/60] add comments why we sort and snake_case configuration schemas --- .../io/iceberg/IcebergReadSchemaTransformProvider.java | 2 ++ .../io/iceberg/IcebergSchemaTransformCatalogConfig.java | 2 ++ .../io/iceberg/IcebergWriteSchemaTransformProvider.java | 9 +++++---- .../beam/sdk/managed/ManagedSchemaTransformProvider.java | 2 ++ 4 files changed, 11 insertions(+), 4 deletions(-) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java index 46b1129bf0f4..fb32e18d9374 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java @@ -95,6 +95,8 @@ static class IcebergReadSchemaTransform extends SchemaTransform { 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) 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 index 605592c98fce..473bcaa3cfd9 100644 --- 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 @@ -68,6 +68,8 @@ public abstract static class Builder { 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) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java index 8f90b650dbcd..b490693a9adb 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java @@ -119,6 +119,8 @@ static class IcebergWriteSchemaTransform extends SchemaTransform { 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) @@ -147,11 +149,10 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { } // TODO: support dynamic destinations - DynamicDestinations dynamicDestinations = - DynamicDestinations.singleTable(TableIdentifier.parse(configuration.getTable())); - IcebergWriteResult result = - rows.apply(IcebergIO.writeRows(catalogBuilder.build()).to(dynamicDestinations)); + rows.apply( + IcebergIO.writeRows(catalogBuilder.build()) + .to(TableIdentifier.parse(configuration.getTable()))); PCollection snapshots = result 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 eb4c9444af28..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 @@ -185,6 +185,8 @@ public ManagedConfig getManagedConfig() { 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(ManagedConfig.class) .apply(managedConfig) From 7130e569d5962b9cc029aa4a5f2995e51c9c529d Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Mon, 22 Apr 2024 11:04:27 -0400 Subject: [PATCH 57/60] add SchemaTransformTranslation abstraction. when encountering a SCHEMA_TRANSFORM urn, fetch underlying identifier --- .../SchemaTransformTranslation.java | 79 ++++++++++++++++ .../expansion/service/ExpansionService.java | 8 +- .../IcebergSchemaTransformTranslation.java | 90 ++----------------- ...IcebergSchemaTransformTranslationTest.java | 14 ++- .../ManagedSchemaTransformTranslation.java | 53 ++--------- ...ManagedSchemaTransformTranslationTest.java | 4 +- 6 files changed, 107 insertions(+), 141 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/SchemaTransformTranslation.java 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/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..17fbd598361d 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; @@ -68,6 +70,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; @@ -170,6 +173,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( @@ -590,7 +596,7 @@ private Map loadRegisteredTransforms() { pipelineOptions.as(ExpansionServiceOptions.class).getJavaClassLookupAllowlist(); assert allowList != null; transformProvider = new JavaClassLookupTransformProvider(allowList); - } else if (getUrn(ExpansionMethods.Enum.SCHEMA_TRANSFORM).equals(urn)) { + } else if (getUrn(SCHEMA_TRANSFORM).equals(urn)) { transformProvider = ExpansionServiceSchemaTransformProvider.of(); } else { transformProvider = getRegisteredTransforms().get(urn); 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 index e3ed91c932a1..c33f7d6261e8 100644 --- 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 @@ -17,75 +17,32 @@ */ 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.IcebergWriteSchemaTransform; +import static org.apache.beam.sdk.schemas.transforms.SchemaTransformTranslation.SchemaTransformPayloadTranslator; import com.google.auto.service.AutoService; -import java.io.ByteArrayOutputStream; -import java.io.IOException; import java.util.Map; -import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload; -import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; -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.Schema; -import org.apache.beam.sdk.schemas.SchemaTranslation; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.construction.BeamUrns; import org.apache.beam.sdk.util.construction.PTransformTranslation.TransformPayloadTranslator; -import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.util.construction.TransformPayloadTranslatorRegistrar; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.checkerframework.checker.nullness.qual.Nullable; @SuppressWarnings({"rawtypes", "nullness"}) public class IcebergSchemaTransformTranslation { static class IcebergReadSchemaTransformTranslator - implements TransformPayloadTranslator { - static final IcebergReadSchemaTransformProvider READ_PROVIDER = - new IcebergReadSchemaTransformProvider(); - static final Schema READ_SCHEMA = READ_PROVIDER.configurationSchema(); - + extends SchemaTransformPayloadTranslator { @Override - public String getUrn() { - return BeamUrns.getUrn(SCHEMA_TRANSFORM); - } - - @Override - public @Nullable FunctionSpec translate( - AppliedPTransform application, SdkComponents components) - throws IOException { - SchemaApi.Schema expansionSchema = SchemaTranslation.schemaToProto(READ_SCHEMA, true); - Row configRow = toConfigRow(application.getTransform()); - ByteArrayOutputStream os = new ByteArrayOutputStream(); - RowCoder.of(READ_SCHEMA).encode(configRow, os); - - return FunctionSpec.newBuilder() - .setUrn(getUrn()) - .setPayload( - SchemaTransformPayload.newBuilder() - .setIdentifier(READ_PROVIDER.identifier()) - .setConfigurationSchema(expansionSchema) - .setConfigurationRow(ByteString.copyFrom(os.toByteArray())) - .build() - .toByteString()) - .build(); + public SchemaTransformProvider provider() { + return new IcebergReadSchemaTransformProvider(); } @Override public Row toConfigRow(IcebergReadSchemaTransform transform) { return transform.getConfigurationRow(); } - - @Override - public IcebergReadSchemaTransform fromConfigRow(Row configRow, PipelineOptions options) { - return (IcebergReadSchemaTransform) READ_PROVIDER.from(configRow); - } } @AutoService(TransformPayloadTranslatorRegistrar.class) @@ -103,47 +60,16 @@ public static class ReadRegistrar implements TransformPayloadTranslatorRegistrar } static class IcebergWriteSchemaTransformTranslator - implements TransformPayloadTranslator { - - static final IcebergWriteSchemaTransformProvider WRITE_PROVIDER = - new IcebergWriteSchemaTransformProvider(); - static final Schema WRITE_SCHEMA = WRITE_PROVIDER.configurationSchema(); - + extends SchemaTransformPayloadTranslator { @Override - public String getUrn() { - return BeamUrns.getUrn(SCHEMA_TRANSFORM); - } - - @Override - public @Nullable FunctionSpec translate( - AppliedPTransform application, SdkComponents components) - throws IOException { - SchemaApi.Schema expansionSchema = SchemaTranslation.schemaToProto(WRITE_SCHEMA, true); - Row configRow = toConfigRow(application.getTransform()); - ByteArrayOutputStream os = new ByteArrayOutputStream(); - RowCoder.of(WRITE_SCHEMA).encode(configRow, os); - - return FunctionSpec.newBuilder() - .setUrn(getUrn()) - .setPayload( - SchemaTransformPayload.newBuilder() - .setIdentifier(WRITE_PROVIDER.identifier()) - .setConfigurationSchema(expansionSchema) - .setConfigurationRow(ByteString.copyFrom(os.toByteArray())) - .build() - .toByteString()) - .build(); + public SchemaTransformProvider provider() { + return new IcebergWriteSchemaTransformProvider(); } @Override public Row toConfigRow(IcebergWriteSchemaTransform transform) { return transform.getConfigurationRow(); } - - @Override - public IcebergWriteSchemaTransform fromConfigRow(Row configRow, PipelineOptions options) { - return (IcebergWriteSchemaTransform) WRITE_PROVIDER.from(configRow); - } } @AutoService(TransformPayloadTranslatorRegistrar.class) 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 index 5fc02af833f5..d643e194dcba 100644 --- 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 @@ -19,8 +19,6 @@ 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.IcebergSchemaTransformTranslation.IcebergReadSchemaTransformTranslator.READ_SCHEMA; -import static org.apache.beam.sdk.io.iceberg.IcebergSchemaTransformTranslation.IcebergWriteSchemaTransformTranslator.WRITE_SCHEMA; 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; @@ -75,7 +73,7 @@ public void testReCreateWriteTransformFromRow() { .withFieldValue("warehouse_location", "test_location") .build(); Row transformConfigRow = - Row.withSchema(WRITE_SCHEMA) + Row.withSchema(WRITE_PROVIDER.configurationSchema()) .withFieldValue("table", "test_table_identifier") .withFieldValue("catalog_config", catalogConfigRow) .build(); @@ -112,7 +110,7 @@ public void testWriteTransformProtoTranslation() .withFieldValue("warehouse_location", warehouse.location) .build(); Row transformConfigRow = - Row.withSchema(WRITE_SCHEMA) + Row.withSchema(WRITE_PROVIDER.configurationSchema()) .withFieldValue("table", "test_identifier") .withFieldValue("catalog_config", catalogConfigRow) .build(); @@ -144,7 +142,7 @@ public void testWriteTransformProtoTranslation() // Check that the proto contains correct values SchemaTransformPayload payload = SchemaTransformPayload.parseFrom(spec.getPayload()); Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getConfigurationSchema()); - assertEquals(WRITE_SCHEMA, schemaFromSpec); + assertEquals(WRITE_PROVIDER.configurationSchema(), schemaFromSpec); Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getConfigurationRow().newInput()); assertEquals(transformConfigRow, rowFromSpec); @@ -169,7 +167,7 @@ public void testReCreateReadTransformFromRow() { .withFieldValue("warehouse_location", "test_location") .build(); Row transformConfigRow = - Row.withSchema(READ_SCHEMA) + Row.withSchema(READ_PROVIDER.configurationSchema()) .withFieldValue("table", "test_table_identifier") .withFieldValue("catalog_config", catalogConfigRow) .build(); @@ -202,7 +200,7 @@ public void testReadTransformProtoTranslation() warehouse.createTable(TableIdentifier.parse(identifier), TestFixtures.SCHEMA); Row transformConfigRow = - Row.withSchema(READ_SCHEMA) + Row.withSchema(READ_PROVIDER.configurationSchema()) .withFieldValue("table", identifier) .withFieldValue("catalog_config", catalogConfigRow) .build(); @@ -235,7 +233,7 @@ public void testReadTransformProtoTranslation() // Check that the proto contains correct values SchemaTransformPayload payload = SchemaTransformPayload.parseFrom(spec.getPayload()); Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getConfigurationSchema()); - assertEquals(READ_SCHEMA, schemaFromSpec); + assertEquals(READ_PROVIDER.configurationSchema(), schemaFromSpec); Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getConfigurationRow().newInput()); assertEquals(transformConfigRow, rowFromSpec); 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 index ef0cdfa7105d..2b1e6544ef8b 100644 --- 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 @@ -17,73 +17,30 @@ */ 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.RunnerApi.FunctionSpec; 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.io.ByteArrayOutputStream; -import java.io.IOException; import java.util.Map; -import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload; -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.Schema; -import org.apache.beam.sdk.schemas.SchemaTranslation; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.construction.BeamUrns; -import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.util.construction.TransformPayloadTranslatorRegistrar; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.checkerframework.checker.nullness.qual.Nullable; public class ManagedSchemaTransformTranslation { static class ManagedSchemaTransformTranslator - implements TransformPayloadTranslator { - static final ManagedSchemaTransformProvider PROVIDER = new ManagedSchemaTransformProvider(null); - static final Schema SCHEMA = PROVIDER.configurationSchema(); - + extends SchemaTransformPayloadTranslator { @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(SCHEMA, true); - Row configRow = toConfigRow(application.getTransform()); - ByteArrayOutputStream os = new ByteArrayOutputStream(); - RowCoder.of(SCHEMA).encode(configRow, os); - - return FunctionSpec.newBuilder() - .setUrn(getUrn()) - .setPayload( - SchemaTransformPayload.newBuilder() - .setIdentifier(PROVIDER.identifier()) - .setConfigurationSchema(expansionSchema) - .setConfigurationRow(ByteString.copyFrom(os.toByteArray())) - .build() - .toByteString()) - .build(); + public SchemaTransformProvider provider() { + return new ManagedSchemaTransformProvider(null); } @Override public Row toConfigRow(ManagedSchemaTransform transform) { return transform.getConfigurationRow(); } - - @Override - public ManagedSchemaTransform fromConfigRow(Row configRow, PipelineOptions options) { - return (ManagedSchemaTransform) PROVIDER.from(configRow); - } } @AutoService(TransformPayloadTranslatorRegistrar.class) 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 index 33e023718823..b4b41ded841c 100644 --- 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 @@ -160,11 +160,11 @@ public void testProtoTranslation() throws Exception { SchemaTransformPayload payload = SchemaTransformPayload.parseFrom(spec.getPayload()); assertEquals(PROVIDER.identifier(), payload.getIdentifier()); Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getConfigurationSchema()); - assertEquals(ManagedSchemaTransformTranslator.SCHEMA, schemaFromSpec); + 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(ManagedSchemaTransformTranslator.SCHEMA) + Row.withSchema(PROVIDER.configurationSchema()) .withFieldValue("transform_identifier", TestSchemaTransformProvider.IDENTIFIER) .withFieldValue("config_url", null) .withFieldValue("config", yamlStringConfig) From de81e6057d561549469f114ad341c4f3fc7eebb2 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Mon, 22 Apr 2024 14:42:43 -0400 Subject: [PATCH 58/60] add documentation --- .../schemas/transforms/TypedSchemaTransformProvider.java | 7 +++++++ sdks/java/io/iceberg/build.gradle | 2 -- sdks/java/managed/build.gradle | 2 -- 3 files changed, 7 insertions(+), 4 deletions(-) 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 1e5984aabaaa..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 @@ -41,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. diff --git a/sdks/java/io/iceberg/build.gradle b/sdks/java/io/iceberg/build.gradle index 397bd099b853..e721b98f1029 100644 --- a/sdks/java/io/iceberg/build.gradle +++ b/sdks/java/io/iceberg/build.gradle @@ -42,8 +42,6 @@ def hive_version = "3.1.3" dependencies { implementation library.java.vendored_guava_32_1_2_jre - implementation library.java.vendored_grpc_1_60_1 - implementation project(path: ":model:pipeline", configuration: "shadow") implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(":sdks:java:managed") implementation library.java.slf4j_api diff --git a/sdks/java/managed/build.gradle b/sdks/java/managed/build.gradle index 4b1da0599838..f06df27429b1 100644 --- a/sdks/java/managed/build.gradle +++ b/sdks/java/managed/build.gradle @@ -28,8 +28,6 @@ ext.summary = """Library that provides managed IOs.""" dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") - implementation project(path: ":model:pipeline", configuration: "shadow") - implementation library.java.vendored_grpc_1_60_1 implementation library.java.vendored_guava_32_1_2_jre testImplementation library.java.junit From 34dc371e210a37700092e6e6e456701f3c7a6235 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Mon, 22 Apr 2024 15:57:34 -0400 Subject: [PATCH 59/60] prioritize registered providers; remove snake_case <-> camelCase conversions from python side --- .../expansion/service/ExpansionService.java | 35 +++++++++++++------ .../transforms/external_transform_provider.py | 35 ++----------------- 2 files changed, 26 insertions(+), 44 deletions(-) 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 17fbd598361d..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 @@ -43,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; @@ -84,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; @@ -590,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(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/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 From 82b481d4264d6ec2f8ef040081e6bdbaf21eb7fa Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Mon, 22 Apr 2024 17:15:26 -0400 Subject: [PATCH 60/60] cleanup --- .../external_transform_provider_it_test.py | 22 ------------------- 1 file changed, 22 deletions(-) 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"),