-
Notifications
You must be signed in to change notification settings - Fork 4.3k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Managed Transform protos & translation; Iceberg SchemaTransforms & tr…
…anslation (#30910) * iceberg write schematransform and test * IcebergIO translation and tests * unify iceberg urns and identifiers; update some comments * replace icebergIO translation with iceberg schematransform translation; fix Schema::sorted to do recursive sorting * removed new proto file and moved Managed URNs to beam_runner_api.proto; we now use SchemaTransformPayload for all schematransforms, including Managed; adding a version number to FileWriteResult encoding so that we can use it to fork in the future whhen needed * Row and Schema snake_case <-> camelCase conversion logic * Row sorted() util * use Row::sorted to fetch Managed & Iceberg row configs * use snake_case convention when translating transforms to spec; remove Managed and Iceberg urns from proto and use SCHEMA_TRANSFORM URN * perform snake_case <-> camelCase conversions directly in TypedSchemaTransformProvider * add SchemaTransformTranslation abstraction. when encountering a SCHEMA_TRANSFORM urn, fetch underlying identifier * prioritize registered providers; remove snake_case <-> camelCase conversions from python side
- Loading branch information
1 parent
a0dad08
commit 37609ba
Showing
83 changed files
with
2,566 additions
and
434 deletions.
There are no files selected for viewing
3 changes: 2 additions & 1 deletion
3
.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Direct.json
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 |
---|---|---|
@@ -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 | ||
} |
3 changes: 2 additions & 1 deletion
3
.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Dataflow.json
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 |
---|---|---|
@@ -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 | ||
} |
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
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
79 changes: 79 additions & 0 deletions
79
...core/src/main/java/org/apache/beam/sdk/schemas/transforms/SchemaTransformTranslation.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,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<T extends SchemaTransform> | ||
implements TransformPayloadTranslator<T> { | ||
public abstract SchemaTransformProvider provider(); | ||
|
||
@Override | ||
public String getUrn() { | ||
return BeamUrns.getUrn(SCHEMA_TRANSFORM); | ||
} | ||
|
||
@Override | ||
@SuppressWarnings("argument") | ||
public @Nullable FunctionSpec translate( | ||
AppliedPTransform<?, ?, T> 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); | ||
} | ||
} | ||
} |
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
Oops, something went wrong.