scwhittle commented on code in PR #37631:
URL: https://github.com/apache/beam/pull/37631#discussion_r2822240558
##########
runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java:
##########
@@ -1372,7 +1372,7 @@ public DataflowPipelineJob run(Pipeline pipeline) {
replaceV1Transforms(pipeline);
}
// Capture the SdkComponents for look up during step translations
- SdkComponents dataflowV1Components = SdkComponents.create();
+ SdkComponents dataflowV1Components = SdkComponents.create(options);
Review Comment:
rename v1?
##########
sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CoderTranslators.java:
##########
@@ -177,6 +194,97 @@ public RowCoder fromComponents(
};
}
+ static <T> CoderTranslator<SchemaCoder<T>> schema() {
+ return new CoderTranslator<SchemaCoder<T>>() {
+ private static final String TO_ROW_FUNCTION_URN =
"beam:torowfn:javasdk:v1";
+ private static final String FROM_ROW_FUNCTION_URN =
"beam:fromrowfn:javasdk:v1";
+ private static final String TYPE_DESCRIPTOR_URN =
"beam:typedescriptor:javasdk:v1";
+
+ @Override
+ public ImmutableList<? extends Coder<?>> getComponents(SchemaCoder<T>
from) {
+ return ImmutableList.of();
+ }
+
+ @Override
+ public String getUrn(SchemaCoder<T> from, TranslationContext context) {
+ if (context instanceof TranslationContextWithOptions) {
+ PipelineOptions options =
+ ((TranslationContextWithOptions)
context).pipelineOptions().get();
+ if (StreamingOptions.updateCompatibilityVersionLessThan(options,
"2.72")) {
+ return CoderTranslation.JAVA_SERIALIZED_CODER_URN;
+ }
+ }
+ return CoderTranslation.getKnownCoderUrns()
+ .getOrDefault(from.getClass(),
CoderTranslation.JAVA_SERIALIZED_CODER_URN);
+ }
+
+ @Override
+ public byte[] getPayload(SchemaCoder<T> from, TranslationContext
context) {
+ if (context instanceof TranslationContextWithOptions) {
+ PipelineOptions options =
+ ((TranslationContextWithOptions)
context).pipelineOptions().get();
+ if (StreamingOptions.updateCompatibilityVersionLessThan(options,
"2.72")) {
+ return SerializableUtils.serializeToByteArray(from);
+ }
+ }
+ SchemaApi.SchemaCoderPayload.Builder payload =
SchemaApi.SchemaCoderPayload.newBuilder();
+ payload.setSchema(SchemaTranslation.schemaToProto(from.getSchema(),
true));
+ payload
+ .getToRowFnBuilder()
+ .setUrn(TO_ROW_FUNCTION_URN)
+ .setPayload(
+ ByteString.copyFrom(
+
SerializableUtils.serializeToByteArray(from.getToRowFunction())));
+ payload
+ .getFromRowFnBuilder()
+ .setUrn(FROM_ROW_FUNCTION_URN)
+ .setPayload(
+ ByteString.copyFrom(
+
SerializableUtils.serializeToByteArray(from.getFromRowFunction())));
+ payload
+ .addAdditionalCoderInfosBuilder()
+ .setUrn(TYPE_DESCRIPTOR_URN)
+ .setPayload(
+ ByteString.copyFrom(
+
SerializableUtils.serializeToByteArray(from.getEncodedTypeDescriptor())));
+ return payload.build().toByteArray();
+ }
+
+ @Override
+ public SchemaCoder<T> fromComponents(
+ List<Coder<?>> components, byte[] payload,
CoderTranslation.TranslationContext context) {
+ checkArgument(
Review Comment:
enforce that this isn't called if !isStructuredCoderEnabled
##########
sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CoderTranslators.java:
##########
@@ -177,6 +194,97 @@ public RowCoder fromComponents(
};
}
+ static <T> CoderTranslator<SchemaCoder<T>> schema() {
+ return new CoderTranslator<SchemaCoder<T>>() {
+ private static final String TO_ROW_FUNCTION_URN =
"beam:torowfn:javasdk:v1";
+ private static final String FROM_ROW_FUNCTION_URN =
"beam:fromrowfn:javasdk:v1";
+ private static final String TYPE_DESCRIPTOR_URN =
"beam:typedescriptor:javasdk:v1";
+
+ @Override
+ public ImmutableList<? extends Coder<?>> getComponents(SchemaCoder<T>
from) {
+ return ImmutableList.of();
+ }
+
+ @Override
+ public String getUrn(SchemaCoder<T> from, TranslationContext context) {
+ if (context instanceof TranslationContextWithOptions) {
Review Comment:
how about a private helper method to call from both getUrn and getPayload?
think it would help readability and reduce duplication
isStructuredCoderEnabled()
##########
sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java:
##########
@@ -675,6 +675,8 @@ private Map<String, TransformProvider>
loadRegisteredTransforms() {
inputs);
// Needed to find which transform was new...
+ // This SdkComponents comes from rehydratedComponents, but doesn't take
into account any
+ // additional translation options specified in PipelineOptions.
Review Comment:
should it? Does this mean that the new format doesn't work with xlang?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]