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

SchemaTransformProviderTranslation #30943

Closed
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
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 @@ -75,13 +75,9 @@ public String description() {
}

@Override
public Class<GenerateSequenceConfiguration> configurationClass() {
return GenerateSequenceConfiguration.class;
}

@Override
public SchemaTransform from(GenerateSequenceConfiguration configuration) {
return new GenerateSequenceSchemaTransform(configuration);
public SchemaTransform<GenerateSequenceConfiguration> from(
GenerateSequenceConfiguration configuration) {
return new GenerateSequenceSchemaTransform(configuration, identifier());
}

@DefaultSchema(AutoValueSchema.class)
Expand Down Expand Up @@ -163,10 +159,13 @@ public void validate() {
}
}

protected static class GenerateSequenceSchemaTransform extends SchemaTransform {
protected static class GenerateSequenceSchemaTransform
extends SchemaTransform<GenerateSequenceConfiguration> {
private final GenerateSequenceConfiguration configuration;

GenerateSequenceSchemaTransform(GenerateSequenceConfiguration configuration) {
GenerateSequenceSchemaTransform(
GenerateSequenceConfiguration configuration, String identifier) {
super(configuration, identifier);
configuration.validate();
this.configuration = configuration;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,9 +17,20 @@
*/
package org.apache.beam.sdk.schemas.transforms;

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 java.lang.reflect.ParameterizedType;

import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.apache.beam.sdk.annotations.Internal;
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.transforms.PTransform;
import org.apache.beam.sdk.values.PCollectionRowTuple;
import org.apache.beam.sdk.values.Row;

/**
* An abstraction representing schema capable and aware transforms. The interface is intended to be
Expand All @@ -34,4 +45,45 @@
*/
@Internal
public abstract class SchemaTransform
extends PTransform<PCollectionRowTuple, PCollectionRowTuple> {}
extends PTransform<PCollectionRowTuple, PCollectionRowTuple> {
private @Nullable Row configurationRow;
private @Nullable String identifier;
private boolean registered = false;

public SchemaTransform register(Row configurationRow, String identifier) {
this.configurationRow = configurationRow;
this.identifier = identifier;
registered = true;

return this;
}

public <ConfigT> SchemaTransform register(ConfigT configuration, Class<ConfigT> configClass, String identifier) {
SchemaRegistry registry = SchemaRegistry.createDefault();
try {
// Get initial row with values
// sort lexicographically and convert field names to snake_case
Row configRow = registry
.getToRowFunction(configClass)
.apply(configuration)
.sorted()
.toSnakeCase();
return register(configRow, identifier);
} catch (NoSuchSchemaException e) {
throw new RuntimeException(
String.format("Unable to find schema for this SchemaTransform's config type: %s", configClass), e);
}
}

public Row getConfigurationRow() {
return Preconditions.checkNotNull(configurationRow, "Could not fetch SchemaTransform's configuration. " +
"Please store it using SchemaTransform::register.");
}
public String getIdentifier() {
return Preconditions.checkNotNull(identifier, "Could not fetch SchemaTransform's identifier. " +
"Please store it using SchemaTransform::register.");
}
public boolean isRegistered() {
return registered;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,102 @@
/*
* 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.util.construction.PTransformTranslation.TransformPayloadTranslator;

import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.ServiceLoader;

import org.apache.beam.model.pipeline.v1.ExternalTransforms;
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.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;

public class SchemaTransformProviderTranslation {
public static class SchemaTransformTranslator
implements TransformPayloadTranslator<SchemaTransform> {
private final SchemaTransformProvider provider;

public SchemaTransformTranslator(SchemaTransformProvider provider) {
this.provider = provider;
}

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

@Override
@SuppressWarnings("argument")
public @Nullable FunctionSpec translate(
AppliedPTransform<?, ?, SchemaTransform> 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 Row toConfigRow(SchemaTransform transform) {
return transform.getConfigurationRow();
}

@Override
public SchemaTransform fromConfigRow(Row configRow, PipelineOptions options) {
return provider.from(configRow);
}
}


private static Map<String, SchemaTransformTranslator> cachedTranslators;
public static Map<String, SchemaTransformTranslator> getDefaultTranslators() {
if (cachedTranslators != null) {
return cachedTranslators;
}
cachedTranslators = new HashMap<>();
for (SchemaTransformProvider provider : ServiceLoader.load(SchemaTransformProvider.class)) {
cachedTranslators.put(provider.identifier(), new SchemaTransformTranslator(provider));
}
return cachedTranslators;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -49,13 +49,8 @@ public class FlattenTransformProvider
protected static final String OUTPUT_ROWS_TAG = "output";

@Override
protected Class<Configuration> configurationClass() {
return Configuration.class;
}

@Override
protected SchemaTransform from(Configuration configuration) {
return new SchemaTransform() {
protected SchemaTransform<Configuration> from(Configuration configuration) {
return new SchemaTransform<Configuration>(configuration, identifier()) {
@Override
public PCollectionRowTuple expand(PCollectionRowTuple input) {
return PCollectionRowTuple.of(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,13 +56,8 @@ public class JavaExplodeTransformProvider
protected static final String OUTPUT_ROWS_TAG = "output";

@Override
protected Class<Configuration> configurationClass() {
return Configuration.class;
}

@Override
protected SchemaTransform from(Configuration configuration) {
return new ExplodeTransform(configuration);
protected SchemaTransform<Configuration> from(Configuration configuration) {
return new ExplodeTransform(configuration, identifier());
}

@Override
Expand Down Expand Up @@ -105,11 +100,11 @@ public abstract static class Builder {
}

/** A {@link SchemaTransform} for Explode. */
protected static class ExplodeTransform extends SchemaTransform {

protected static class ExplodeTransform extends SchemaTransform<Configuration> {
private final Configuration configuration;

ExplodeTransform(Configuration configuration) {
ExplodeTransform(Configuration configuration, String identifier) {
super(configuration, identifier);
this.configuration = configuration;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,13 +54,8 @@ public class JavaFilterTransformProvider
protected static final String OUTPUT_ROWS_TAG = "output";

@Override
protected Class<Configuration> configurationClass() {
return Configuration.class;
}

@Override
protected SchemaTransform from(Configuration configuration) {
return new JavaFilterTransform(configuration);
protected SchemaTransform<Configuration> from(Configuration configuration) {
return new JavaFilterTransform(configuration, identifier());
}

@Override
Expand Down Expand Up @@ -107,11 +102,12 @@ public abstract static class Builder {
}

/** A {@link SchemaTransform} for Filter-java. */
protected static class JavaFilterTransform extends SchemaTransform {
protected static class JavaFilterTransform extends SchemaTransform<Configuration> {

private final Configuration configuration;

JavaFilterTransform(Configuration configuration) {
JavaFilterTransform(Configuration configuration, String identifier) {
super(configuration, identifier);
this.configuration = configuration;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,13 +56,8 @@ public class JavaMapToFieldsTransformProvider
protected static final String OUTPUT_ROWS_TAG = "output";

@Override
protected Class<Configuration> configurationClass() {
return Configuration.class;
}

@Override
protected SchemaTransform from(Configuration configuration) {
return new JavaMapToFieldsTransform(configuration);
protected SchemaTransform<Configuration> from(Configuration configuration) {
return new JavaMapToFieldsTransform(configuration, identifier());
}

@Override
Expand Down Expand Up @@ -119,11 +114,12 @@ public abstract static class Builder {
}

/** A {@link SchemaTransform} for MapToFields-java. */
protected static class JavaMapToFieldsTransform extends SchemaTransform {
protected static class JavaMapToFieldsTransform extends SchemaTransform<Configuration> {

private final Configuration configuration;

JavaMapToFieldsTransform(Configuration configuration) {
JavaMapToFieldsTransform(Configuration configuration, String identifier) {
super(configuration, identifier);
this.configuration = configuration;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,13 +63,8 @@ public class LoggingTransformProvider
protected static final String OUTPUT_ROWS_TAG = "output";

@Override
protected Class<Configuration> configurationClass() {
return Configuration.class;
}

@Override
protected SchemaTransform from(Configuration configuration) {
return new LoggingTransform(configuration);
protected SchemaTransform<Configuration> from(Configuration configuration) {
return new LoggingTransform(configuration, identifier());
}

@Override
Expand Down Expand Up @@ -134,13 +129,14 @@ public abstract static class Builder {
}

/** A {@link SchemaTransform} for logging. */
protected static class LoggingTransform extends SchemaTransform {
protected static class LoggingTransform extends SchemaTransform<Configuration> {

private static final Logger LOG = LoggerFactory.getLogger(LoggingTransform.class);

private final Configuration configuration;

LoggingTransform(Configuration configuration) {
LoggingTransform(Configuration configuration, String identifier) {
super(configuration, identifier);
this.configuration = configuration;
}

Expand Down
Loading
Loading