-
Notifications
You must be signed in to change notification settings - Fork 4.3k
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
Default translation for SchemaTransforms #31558
Closed
Closed
Changes from all commits
Commits
Show all changes
11 commits
Select commit
Hold shift + click to select a range
c4a8b58
SchemaTransformProviderTranslation
ahmedabu98 e770eac
tests
ahmedabu98 3184255
fix debezium test
ahmedabu98 ef54903
Merge branch 'master' of https://github.com/ahmedabu98/beam into sche…
ahmedabu98 90df729
checkpoint
ahmedabu98 2727e2d
SchemaTransformTranslation
ahmedabu98 3c51cea
cleanup
ahmedabu98 0ca3f4a
add documentation and improvements
ahmedabu98 0ebc306
cleaner API; remove concept of ConfigT from SchemaTransform
ahmedabu98 0e74801
automate tests as well
ahmedabu98 549dad0
spotless
ahmedabu98 File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -20,6 +20,9 @@ | |
import org.apache.beam.sdk.annotations.Internal; | ||
import org.apache.beam.sdk.transforms.PTransform; | ||
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.Preconditions; | ||
import org.checkerframework.checker.nullness.qual.Nullable; | ||
|
||
/** | ||
* An abstraction representing schema capable and aware transforms. The interface is intended to be | ||
|
@@ -33,5 +36,39 @@ | |
* compatibility guarantees and it should not be implemented outside of the Beam repository. | ||
*/ | ||
@Internal | ||
public abstract class SchemaTransform | ||
extends PTransform<PCollectionRowTuple, PCollectionRowTuple> {} | ||
public abstract class SchemaTransform extends PTransform<PCollectionRowTuple, PCollectionRowTuple> { | ||
private @Nullable Row configurationRow; | ||
private @Nullable String identifier; | ||
private boolean registered = false; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I use "registered" throughout, but I'm not sure if it's the best terminology here. Open to other suggestions! |
||
|
||
/** | ||
* Stores the transform's identifier and configuration {@link Row} used to build this instance. | ||
* Doing so allows this transform to be translated from/to proto using {@link | ||
* org.apache.beam.sdk.util.construction.PTransformTranslation.SchemaTransformTranslator}. | ||
*/ | ||
public SchemaTransform register(Row configurationRow, String identifier) { | ||
this.configurationRow = configurationRow; | ||
this.identifier = identifier; | ||
registered = true; | ||
|
||
return this; | ||
} | ||
|
||
public Row getConfigurationRow() { | ||
return Preconditions.checkNotNull( | ||
configurationRow, | ||
"Could not fetch Row configuration for %s. Please store it using .register().", | ||
getClass()); | ||
} | ||
|
||
public String getIdentifier() { | ||
return Preconditions.checkNotNull( | ||
identifier, | ||
"Could not fetch identifier for %s. Please store it using .register().", | ||
getClass()); | ||
} | ||
|
||
public boolean isRegistered() { | ||
return registered; | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
112 changes: 112 additions & 0 deletions
112
.../src/test/java/org/apache/beam/sdk/schemas/transforms/SchemaTransformTranslationTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,112 @@ | ||
/* | ||
* 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.sdk.schemas.transforms.SchemaTransformTranslation.SchemaTransformPayloadTranslator; | ||
import static org.junit.Assert.assertEquals; | ||
|
||
import java.io.IOException; | ||
import java.util.List; | ||
import java.util.stream.Collectors; | ||
import org.apache.beam.model.pipeline.v1.ExternalTransforms; | ||
import org.apache.beam.model.pipeline.v1.RunnerApi; | ||
import org.apache.beam.sdk.Pipeline; | ||
import org.apache.beam.sdk.coders.RowCoder; | ||
import org.apache.beam.sdk.options.PipelineOptionsFactory; | ||
import org.apache.beam.sdk.schemas.Schema; | ||
import org.apache.beam.sdk.schemas.SchemaTranslation; | ||
import org.apache.beam.sdk.util.construction.BeamUrns; | ||
import org.apache.beam.sdk.util.construction.PipelineTranslation; | ||
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.junit.Test; | ||
|
||
/** Base class for standard {@link SchemaTransform} translation tests. */ | ||
public abstract class SchemaTransformTranslationTest { | ||
protected abstract SchemaTransformProvider provider(); | ||
|
||
protected abstract Row configurationRow(); | ||
|
||
/** Input used for this SchemaTransform. Used to build a pipeline to test proto translation. */ | ||
protected PCollectionRowTuple input(Pipeline p) { | ||
return PCollectionRowTuple.empty(p); | ||
}; | ||
|
||
@Test | ||
public void testRecreateTransformFromRow() { | ||
SchemaTransformProvider provider = provider(); | ||
SchemaTransformPayloadTranslator translator = new SchemaTransformPayloadTranslator(provider); | ||
SchemaTransform originalTransform = provider.from(configurationRow()); | ||
|
||
Row translatedConfigRow = translator.toConfigRow(originalTransform); | ||
SchemaTransform translatedTransform = | ||
translator.fromConfigRow(translatedConfigRow, PipelineOptionsFactory.create()); | ||
|
||
assertEquals(configurationRow(), translatedTransform.getConfigurationRow()); | ||
} | ||
|
||
@Test | ||
public void testTransformProtoTranslation() throws InvalidProtocolBufferException, IOException { | ||
SchemaTransformProvider provider = provider(); | ||
Row configurationRow = configurationRow(); | ||
|
||
// Infer if it's a read or write SchemaTransform and build pipeline accordingly | ||
Pipeline p = Pipeline.create(); | ||
SchemaTransform schemaTransform = provider.from(configurationRow); | ||
input(p).apply(schemaTransform); | ||
|
||
// Then translate the pipeline to a proto and extract the SchemaTransform's proto | ||
RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p); | ||
List<RunnerApi.PTransform> schemaTransformProto = | ||
pipelineProto.getComponents().getTransformsMap().values().stream() | ||
.filter( | ||
tr -> { | ||
RunnerApi.FunctionSpec spec = tr.getSpec(); | ||
try { | ||
return spec.getUrn().equals(BeamUrns.getUrn(SCHEMA_TRANSFORM)) | ||
&& ExternalTransforms.SchemaTransformPayload.parseFrom(spec.getPayload()) | ||
.getIdentifier() | ||
.equals(provider.identifier()); | ||
} catch (InvalidProtocolBufferException e) { | ||
throw new RuntimeException(e); | ||
} | ||
}) | ||
.collect(Collectors.toList()); | ||
assertEquals(1, schemaTransformProto.size()); | ||
RunnerApi.FunctionSpec spec = schemaTransformProto.get(0).getSpec(); | ||
|
||
// Check that the proto contains correct values | ||
ExternalTransforms.SchemaTransformPayload payload = | ||
ExternalTransforms.SchemaTransformPayload.parseFrom(spec.getPayload()); | ||
Schema translatedSchema = SchemaTranslation.schemaFromProto(payload.getConfigurationSchema()); | ||
assertEquals(provider.configurationSchema(), translatedSchema); | ||
Row translatedConfigRow = | ||
RowCoder.of(translatedSchema).decode(payload.getConfigurationRow().newInput()); | ||
|
||
assertEquals(configurationRow, translatedConfigRow); | ||
|
||
// Use the information in the proto to recreate the transform | ||
SchemaTransform translatedTransform = | ||
new SchemaTransformPayloadTranslator(provider) | ||
.fromConfigRow(translatedConfigRow, PipelineOptionsFactory.create()); | ||
|
||
assertEquals(configurationRow, translatedTransform.getConfigurationRow()); | ||
} | ||
} |
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It feels like putting these (optional) private variables as part of the base class violates separation of concerns.