-
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
[YAML] Implement basic java mapping operations. #28657
Merged
Merged
Changes from 7 commits
Commits
Show all changes
10 commits
Select commit
Hold shift + click to select a range
b6b8ebe
[YAML] Implement basic java mapping operations.
robertwb 9aa3bb8
Workaround javac classpath issue.
robertwb 99d0425
Add java map providers to standard providers listing.
robertwb 1e727ad
checkStyle
robertwb 43fdc57
checkstyle
robertwb f5aede2
package info
robertwb 461db06
fix some warnings
robertwb 443b400
fix naming conventions
robertwb d57ea81
Merge branch 'master' into yaml-java-map
robertwb 5fd6532
Allow fetching jars from non-local filesystems.
robertwb 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
253 changes: 253 additions & 0 deletions
253
...va/org/apache/beam/sdk/schemas/transforms/providers/JavaMapToFieldsTransformProvider.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,253 @@ | ||
/* | ||
* 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.providers; | ||
|
||
import com.google.auto.service.AutoService; | ||
import com.google.auto.value.AutoValue; | ||
import java.net.MalformedURLException; | ||
import java.util.ArrayList; | ||
import java.util.Collections; | ||
import java.util.List; | ||
import java.util.Map; | ||
import javax.annotation.Nullable; | ||
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; | ||
import org.apache.beam.sdk.transforms.DoFn; | ||
import org.apache.beam.sdk.transforms.ParDo; | ||
import org.apache.beam.sdk.values.PCollectionRowTuple; | ||
import org.apache.beam.sdk.values.PCollectionTuple; | ||
import org.apache.beam.sdk.values.Row; | ||
import org.apache.beam.sdk.values.TupleTag; | ||
import org.apache.beam.sdk.values.TupleTagList; | ||
|
||
/** | ||
* An implementation of {@link TypedSchemaTransformProvider} for MapToFields for the java language. | ||
* | ||
* <p><b>Internal only:</b> This class is actively being worked on, and it will likely change. We | ||
* provide no backwards compatibility guarantees, and it should not be implemented outside the Beam | ||
* repository. | ||
*/ | ||
@SuppressWarnings({ | ||
"nullness" // TODO(https://github.com/apache/beam/issues/20497) | ||
}) | ||
@AutoService(SchemaTransformProvider.class) | ||
public class JavaMapToFieldsTransformProvider | ||
extends TypedSchemaTransformProvider<JavaMapToFieldsTransformProvider.Configuration> { | ||
protected static final String INPUT_ROWS_TAG = "input"; | ||
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); | ||
} | ||
|
||
@Override | ||
public String identifier() { | ||
return String.format("beam:schematransform:org.apache.beam:yaml:map_to_fields-java:v1"); | ||
} | ||
|
||
@Override | ||
public List<String> inputCollectionNames() { | ||
return Collections.singletonList(INPUT_ROWS_TAG); | ||
} | ||
|
||
@Override | ||
public List<String> outputCollectionNames() { | ||
return Collections.singletonList(OUTPUT_ROWS_TAG); | ||
} | ||
|
||
@DefaultSchema(AutoValueSchema.class) | ||
@AutoValue | ||
public abstract static class Configuration { | ||
@Nullable | ||
public abstract String getLanguage(); | ||
|
||
@Nullable | ||
public abstract Boolean getAppend(); | ||
|
||
@Nullable | ||
public abstract List<String> getDrop(); | ||
|
||
public abstract Map<String, JavaRowUdf.Configuration> getFields(); | ||
|
||
@Nullable | ||
@SuppressWarnings("all") | ||
public abstract ErrorHandling getError_handling(); | ||
|
||
public static Builder builder() { | ||
return new AutoValue_JavaMapToFieldsTransformProvider_Configuration.Builder(); | ||
} | ||
|
||
@AutoValue.Builder | ||
public abstract static class Builder { | ||
|
||
public abstract Builder setLanguage(String language); | ||
|
||
public abstract Builder setAppend(Boolean append); | ||
|
||
public abstract Builder setDrop(List<String> drop); | ||
|
||
public abstract Builder setFields(Map<String, JavaRowUdf.Configuration> fields); | ||
|
||
@SuppressWarnings("all") | ||
public abstract Builder setError_handling(ErrorHandling error_handling); | ||
|
||
public abstract Configuration build(); | ||
} | ||
|
||
@AutoValue | ||
public abstract static class ErrorHandling { | ||
@SchemaFieldDescription("The name of the output PCollection containing failed writes.") | ||
public abstract String getOutput(); | ||
|
||
public static Builder builder() { | ||
return new AutoValue_JavaMapToFieldsTransformProvider_Configuration_ErrorHandling.Builder(); | ||
} | ||
|
||
@AutoValue.Builder | ||
public abstract static class Builder { | ||
public abstract Builder setOutput(String output); | ||
|
||
public abstract ErrorHandling build(); | ||
} | ||
} | ||
} | ||
|
||
/** A {@link SchemaTransform} for MapToFields-java. */ | ||
protected static class JavaMapToFieldsTransform extends SchemaTransform { | ||
|
||
private final Configuration configuration; | ||
|
||
JavaMapToFieldsTransform(Configuration configuration) { | ||
this.configuration = configuration; | ||
} | ||
|
||
@Override | ||
public PCollectionRowTuple expand(PCollectionRowTuple input) { | ||
Schema inputSchema = input.get(INPUT_ROWS_TAG).getSchema(); | ||
Schema.Builder outputSchemaBuilder = new Schema.Builder(); | ||
// TODO(yaml): Consider allowing the full java schema naming syntax | ||
// (perhaps as a different dialect/language). | ||
boolean append = configuration.getAppend() != null && configuration.getAppend(); | ||
List<String> toDrop = | ||
configuration.getDrop() == null ? Collections.emptyList() : configuration.getDrop(); | ||
List<JavaRowUdf> udfs = new ArrayList<>(); | ||
if (append) { | ||
for (Schema.Field field : inputSchema.getFields()) { | ||
if (!toDrop.contains(field.getName())) { | ||
try { | ||
udfs.add( | ||
new JavaRowUdf( | ||
JavaRowUdf.Configuration.builder().setExpression(field.getName()).build(), | ||
inputSchema)); | ||
} catch (MalformedURLException | ||
| ReflectiveOperationException | ||
| StringCompiler.CompileException exn) { | ||
throw new RuntimeException(exn); | ||
} | ||
outputSchemaBuilder = outputSchemaBuilder.addField(field); | ||
} | ||
} | ||
} | ||
for (Map.Entry<String, JavaRowUdf.Configuration> entry : | ||
configuration.getFields().entrySet()) { | ||
if (!"java".equals(configuration.getLanguage())) { | ||
String expr = entry.getValue().getExpression(); | ||
if (expr == null || !inputSchema.hasField(expr)) { | ||
throw new IllegalArgumentException( | ||
"Unknown field or missing language specification for '" + entry.getKey() + "'"); | ||
} | ||
} | ||
try { | ||
JavaRowUdf udf = new JavaRowUdf(entry.getValue(), inputSchema); | ||
udfs.add(udf); | ||
outputSchemaBuilder = outputSchemaBuilder.addField(entry.getKey(), udf.getOutputType()); | ||
} catch (MalformedURLException | ||
| ReflectiveOperationException | ||
| StringCompiler.CompileException exn) { | ||
throw new RuntimeException(exn); | ||
} | ||
} | ||
Schema outputSchema = outputSchemaBuilder.build(); | ||
boolean handleErrors = | ||
configuration.getError_handling() != null | ||
&& configuration.getError_handling().getOutput() != null; | ||
Schema errorSchema = | ||
Schema.of( | ||
Schema.Field.of("failed_row", Schema.FieldType.row(inputSchema)), | ||
Schema.Field.of("error_message", Schema.FieldType.STRING)); | ||
|
||
PCollectionTuple pcolls = | ||
input | ||
.get(INPUT_ROWS_TAG) | ||
.apply( | ||
"MapToFields", | ||
ParDo.of(createDoFn(udfs, outputSchema, errorSchema, handleErrors)) | ||
.withOutputTags(mappedValues, TupleTagList.of(errorValues))); | ||
pcolls.get(mappedValues).setRowSchema(outputSchema); | ||
pcolls.get(errorValues).setRowSchema(errorSchema); | ||
|
||
PCollectionRowTuple result = | ||
PCollectionRowTuple.of(OUTPUT_ROWS_TAG, pcolls.get(mappedValues)); | ||
if (handleErrors) { | ||
result = result.and(configuration.getError_handling().getOutput(), pcolls.get(errorValues)); | ||
} | ||
return result; | ||
} | ||
|
||
private static final TupleTag<Row> mappedValues = new TupleTag<Row>() {}; | ||
private static final TupleTag<Row> errorValues = new TupleTag<Row>() {}; | ||
|
||
private static DoFn<Row, Row> createDoFn( | ||
List<JavaRowUdf> udfs, Schema outputSchema, Schema errorSchema, boolean handleErrors) { | ||
return new DoFn<Row, Row>() { | ||
@ProcessElement | ||
public void processElement(@Element Row inputRow, MultiOutputReceiver out) { | ||
try { | ||
Row.Builder outputRow = Row.withSchema(outputSchema); | ||
for (JavaRowUdf udf : udfs) { | ||
outputRow.addValue(udf.getFunction().apply(inputRow)); | ||
} | ||
out.get(mappedValues).output(outputRow.build()); | ||
} catch (Exception exn) { | ||
if (handleErrors) { | ||
out.get(errorValues) | ||
.output( | ||
Row.withSchema(errorSchema) | ||
.withFieldValue("failed_row", inputRow) | ||
.withFieldValue("error_message", exn.getMessage()) | ||
.build()); | ||
} else { | ||
throw new RuntimeException(exn); | ||
} | ||
} | ||
} | ||
}; | ||
} | ||
} | ||
} |
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.
Is there a reason for naming these with underscores when they can be remapped by the YAML provider?
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.
No, I can go ahead and fix this.
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.
Done.