Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Managed Transform protos & translation; Iceberg SchemaTransforms & translation #30910

Merged
merged 66 commits into from
Apr 22, 2024
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
66 commits
Select commit Hold shift + click to select a range
42611e0
iceberg write schematransform and test
ahmedabu98 Apr 9, 2024
16e6235
cleanup
ahmedabu98 Apr 9, 2024
ed72898
IcebergIO translation and tests
ahmedabu98 Apr 11, 2024
1738345
add sanity check for building with Row; add documentation about outpu…
ahmedabu98 Apr 11, 2024
364ebbe
spotless
ahmedabu98 Apr 11, 2024
79d2c94
spotless
ahmedabu98 Apr 11, 2024
30de265
permitUnusedDeclared iceberg
ahmedabu98 Apr 11, 2024
905d590
Change ManagedSchemaTransformProvider to take a Row config instead of…
ahmedabu98 Apr 11, 2024
553281f
Merge branch 'master' of https://github.com/ahmedabu98/beam into iceb…
ahmedabu98 Apr 11, 2024
2c733ec
Merge branch 'master' of https://github.com/ahmedabu98/beam into iceb…
ahmedabu98 Apr 11, 2024
1067d84
don't auto generate external wrapper for this just yet
ahmedabu98 Apr 11, 2024
6db699a
spotless
ahmedabu98 Apr 11, 2024
301e388
spotless
ahmedabu98 Apr 11, 2024
f1576e3
Merge branch 'master' of https://github.com/ahmedabu98/beam into iceb…
ahmedabu98 Apr 14, 2024
04cc2db
Merge branch 'master' of https://github.com/ahmedabu98/beam into iceb…
ahmedabu98 Apr 15, 2024
27e5fb0
Read schematransform and tests
ahmedabu98 Apr 15, 2024
e2cb93b
Merge branch 'iceberg_translation' of https://github.com/ahmedabu98/b…
ahmedabu98 Apr 15, 2024
aa8b1ed
pulling in IcebergIO changes; spotless
ahmedabu98 Apr 15, 2024
6823524
Merge branch 'managed_row_config' of https://github.com/ahmedabu98/be…
ahmedabu98 Apr 15, 2024
0674069
icebergio translation; managed translation; protos
ahmedabu98 Apr 16, 2024
9034cee
spotless
ahmedabu98 Apr 16, 2024
0bc3779
spotless; use underscore instead of camel case field names when trans…
ahmedabu98 Apr 16, 2024
27b5e6a
add grpc dependency
ahmedabu98 Apr 16, 2024
05e3a1a
updated proto description; fix gen xlang command
ahmedabu98 Apr 17, 2024
2b49c37
ManagedTransform explicit input/output types; move iceberg package to…
ahmedabu98 Apr 17, 2024
7de01bb
externalizable IcebergCatalogConfig
ahmedabu98 Apr 17, 2024
6f739f4
externalizable IcebergCatalogConfig supports all properties; address …
ahmedabu98 Apr 17, 2024
1df6aa7
unify iceberg urns and identifiers; update some comments
ahmedabu98 Apr 18, 2024
6d7a353
one source for all supported managed transform identifiers
ahmedabu98 Apr 18, 2024
ea5a34d
add documentation
ahmedabu98 Apr 18, 2024
642da92
custom serialization for OneTableDynamicDestinations
ahmedabu98 Apr 18, 2024
5a2fd87
add iceberg via managed API tests; update proto doc
ahmedabu98 Apr 18, 2024
804fdac
rename config; change test schematransform location
ahmedabu98 Apr 18, 2024
c301cfc
spotless
ahmedabu98 Apr 18, 2024
841fd43
add missing package-info file
ahmedabu98 Apr 18, 2024
928eead
spotless
ahmedabu98 Apr 18, 2024
4f12cbd
replace icebergIO translation with iceberg schematransform translatio…
ahmedabu98 Apr 18, 2024
9b26b18
remove ExternalizableIcebergCatalogConfig (no longer needed)
ahmedabu98 Apr 18, 2024
d912a22
pull identifiers from generated proto
ahmedabu98 Apr 18, 2024
523fc69
remove unused hadoop dependency
ahmedabu98 Apr 18, 2024
cb89d16
update generate sequence wrapper after Schema sorting
ahmedabu98 Apr 18, 2024
17ecec6
managed transform translation uses default schema
ahmedabu98 Apr 19, 2024
108a12c
yaml returns null row; cleanup
ahmedabu98 Apr 19, 2024
5923a97
spotless
ahmedabu98 Apr 19, 2024
8ab4c0c
remove SchemaAwareTransformPayload and use SchemaTransformPayload ins…
ahmedabu98 Apr 19, 2024
3426360
create a beam-schema-compatible class for Snapshot info
ahmedabu98 Apr 19, 2024
49a1596
removed new proto file and moved Managed URNs to beam_runner_api.prot…
ahmedabu98 Apr 19, 2024
52a308c
Row and Schema snake_case <-> camelCase conversion logic
ahmedabu98 Apr 19, 2024
d9fcb06
Row sorted() util
ahmedabu98 Apr 19, 2024
11354c8
use Row::sorted to fetch Managed & Iceberg row configs
ahmedabu98 Apr 19, 2024
2992192
use snake_case convention when translating transforms to spec; remove…
ahmedabu98 Apr 19, 2024
b311068
spotless
ahmedabu98 Apr 19, 2024
2461b44
cleanup
ahmedabu98 Apr 20, 2024
ecb4dbb
DefaultSchemaProvider can now provide the underlying SchemaProvider
ahmedabu98 Apr 20, 2024
68895a7
perform snake_case <-> camelCase conversions directly in TypedSchemaT…
ahmedabu98 Apr 20, 2024
d2135b8
update icebergIO and managed translations to reflect field name conve…
ahmedabu98 Apr 20, 2024
64863ce
sorted SnapshotInfo
ahmedabu98 Apr 20, 2024
5afb633
update manual Python wrappers to use snake_case convention; remove ca…
ahmedabu98 Apr 20, 2024
2ddd5bb
Row utils allow nullable
ahmedabu98 Apr 20, 2024
d5a4d66
add FileWriteResult test for version number; fix existing Java and YA…
ahmedabu98 Apr 22, 2024
3b74f77
add schema-aware transform urn to transform annotations during transl…
ahmedabu98 Apr 22, 2024
af65032
add comments why we sort and snake_case configuration schemas
ahmedabu98 Apr 22, 2024
7130e56
add SchemaTransformTranslation abstraction. when encountering a SCHEM…
ahmedabu98 Apr 22, 2024
de81e60
add documentation
ahmedabu98 Apr 22, 2024
34dc371
prioritize registered providers; remove snake_case <-> camelCase conv…
ahmedabu98 Apr 22, 2024
82b481d
cleanup
ahmedabu98 Apr 22, 2024
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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<String> RUNNER_IMPLEMENTED_TRANSFORMS =
Expand Down Expand Up @@ -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)));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;

Expand All @@ -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;
Expand All @@ -51,38 +53,47 @@ static class IcebergReadSchemaTransformTranslator

@Override
public String getUrn() {
return READ_PROVIDER.identifier();
return BeamUrns.getUrn(SCHEMA_TRANSFORM);
ahmedabu98 marked this conversation as resolved.
Show resolved Hide resolved
}

@Override
public @Nullable FunctionSpec translate(
AppliedPTransform<?, ?, IcebergReadSchemaTransform> 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()
chamikaramj marked this conversation as resolved.
Show resolved Hide resolved
.setUrn(getUrn())
.setPayload(
ahmedabu98 marked this conversation as resolved.
Show resolved Hide resolved
SchemaTransformPayload.newBuilder()
.setIdentifier(READ_PROVIDER.identifier())
.setConfigurationSchema(expansionSchema)
.setConfigurationRow(ByteString.copyFrom(os.toByteArray()))
.setIdentifier(getUrn())
.build()
.toByteString())
.build();
}

@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) {
ahmedabu98 marked this conversation as resolved.
Show resolved Hide resolved
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());
}
}

Expand All @@ -109,38 +120,48 @@ static class IcebergWriteSchemaTransformTranslator

@Override
public String getUrn() {
return WRITE_PROVIDER.identifier();
return BeamUrns.getUrn(SCHEMA_TRANSFORM);
}

@Override
public @Nullable FunctionSpec translate(
AppliedPTransform<?, ?, IcebergWriteSchemaTransform> 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();
}

@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());
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,13 +17,15 @@
*/
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;
import static org.apache.beam.sdk.io.iceberg.IcebergWriteSchemaTransformProvider.INPUT_TAG;
import static org.apache.beam.sdk.io.iceberg.IcebergWriteSchemaTransformProvider.IcebergWriteSchemaTransform;
import static org.junit.Assert.assertEquals;

import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.UUID;
Expand All @@ -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;
Expand All @@ -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")
Expand All @@ -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();
Expand Down Expand Up @@ -122,18 +125,31 @@ public void testWriteTransformProtoTranslation() throws Exception {
RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p);
List<RunnerApi.PTransform> 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();

// 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 =
Expand All @@ -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)
Expand Down Expand Up @@ -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 =
Expand All @@ -201,17 +218,30 @@ public void testReadTransformProtoTranslation() throws Exception {
RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p);
List<RunnerApi.PTransform> 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();

// 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 =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;

Expand All @@ -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;
Expand All @@ -51,25 +50,27 @@ static class ManagedSchemaTransformTranslator

@Override
public String getUrn() {
return PROVIDER.identifier();
return BeamUrns.getUrn(SCHEMA_TRANSFORM);
}

@Override
@SuppressWarnings("argument")
public @Nullable FunctionSpec translate(
AppliedPTransform<?, ?, ManagedSchemaTransform> 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(
ahmedabu98 marked this conversation as resolved.
Show resolved Hide resolved
SchemaTransformPayload.newBuilder()
.setIdentifier(managedConfig.getTransformIdentifier())
.setIdentifier(PROVIDER.identifier())
.setConfigurationSchema(expansionSchema)
.setConfigurationRow(ByteString.copyFrom(os.toByteArray()))
.build()
Expand All @@ -79,25 +80,19 @@ public String getUrn() {

@Override
public Row toConfigRow(ManagedSchemaTransform transform) {
ahmedabu98 marked this conversation as resolved.
Show resolved Hide resolved
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());
}
}

Expand Down
Loading
Loading