-
Notifications
You must be signed in to change notification settings - Fork 4.6k
Revert #37631 and #38497 on HEAD #38516
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||||||||
|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -1333,20 +1333,19 @@ public DataflowPipelineJob run(Pipeline pipeline) { | |||||||||
| // with the SDK harness image (which implements Fn API). | ||||||||||
| // | ||||||||||
| // The same Environment is used in different and contradictory ways, depending on whether | ||||||||||
| // it is a portable or non-portable job submission. | ||||||||||
| // it is a v1 or v2 job submission. | ||||||||||
| RunnerApi.Environment defaultEnvironmentForDataflow = | ||||||||||
| Environments.createDockerEnvironment(workerHarnessContainerImageURL); | ||||||||||
|
|
||||||||||
| // The SdkComponents for portable and non-portable job submission must be kept distinct. Both | ||||||||||
| // The SdkComponents for portable an non-portable job submission must be kept distinct. Both | ||||||||||
|
Contributor
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. |
||||||||||
| // need the default environment. | ||||||||||
| SdkComponents portableComponents = | ||||||||||
| SdkComponents.create( | ||||||||||
| options, | ||||||||||
| defaultEnvironmentForDataflow | ||||||||||
| .toBuilder() | ||||||||||
| .addAllDependencies(getDefaultArtifacts()) | ||||||||||
| .addAllCapabilities(Environments.getJavaCapabilities()) | ||||||||||
| .build()); | ||||||||||
| SdkComponents portableComponents = SdkComponents.create(); | ||||||||||
| portableComponents.registerEnvironment( | ||||||||||
| defaultEnvironmentForDataflow | ||||||||||
| .toBuilder() | ||||||||||
| .addAllDependencies(getDefaultArtifacts()) | ||||||||||
| .addAllCapabilities(Environments.getJavaCapabilities()) | ||||||||||
| .build()); | ||||||||||
|
|
||||||||||
| RunnerApi.Pipeline portablePipelineProto = | ||||||||||
| PipelineTranslation.toProto(pipeline, portableComponents, false); | ||||||||||
|
|
@@ -1375,30 +1374,28 @@ public DataflowPipelineJob run(Pipeline pipeline) { | |||||||||
| options.as(SdkHarnessOptions.class).setPipelineProtoHash(pipelineProtoHash); | ||||||||||
|
|
||||||||||
| if (useUnifiedWorker(options)) { | ||||||||||
| LOG.info( | ||||||||||
| "Skipping non-portable transform replacements since job will run on portable worker."); | ||||||||||
| LOG.info("Skipping v1 transform replacements since job will run on v2."); | ||||||||||
| } else { | ||||||||||
| // Now rewrite things to be as needed for non-portable (mutates the pipeline). | ||||||||||
| // This way the job submitted is valid for portable and non-portable, simultaneously. | ||||||||||
| // Now rewrite things to be as needed for v1 (mutates the pipeline) | ||||||||||
| // This way the job submitted is valid for v1 and v2, simultaneously | ||||||||||
|
Comment on lines
+1379
to
+1380
Contributor
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. Comments should end with a period for better readability and consistency with the rest of the codebase.
Suggested change
|
||||||||||
| replaceV1Transforms(pipeline); | ||||||||||
| } | ||||||||||
| // Capture the SdkComponents for look up during step translations. | ||||||||||
| SdkComponents dataflowNonPortableComponents = | ||||||||||
| SdkComponents.create( | ||||||||||
| options, | ||||||||||
| defaultEnvironmentForDataflow | ||||||||||
| .toBuilder() | ||||||||||
| .addAllDependencies(getDefaultArtifacts()) | ||||||||||
| .addAllCapabilities(Environments.getJavaCapabilities()) | ||||||||||
| .build()); | ||||||||||
| // No need to perform transform upgrading for the non-portable runner proto. | ||||||||||
| RunnerApi.Pipeline dataflowNonPortablePipelineProto = | ||||||||||
| PipelineTranslation.toProto(pipeline, dataflowNonPortableComponents, true, false); | ||||||||||
| // Capture the SdkComponents for look up during step translations | ||||||||||
| SdkComponents dataflowV1Components = SdkComponents.create(); | ||||||||||
| dataflowV1Components.registerEnvironment( | ||||||||||
| defaultEnvironmentForDataflow | ||||||||||
| .toBuilder() | ||||||||||
| .addAllDependencies(getDefaultArtifacts()) | ||||||||||
| .addAllCapabilities(Environments.getJavaCapabilities()) | ||||||||||
| .build()); | ||||||||||
| // No need to perform transform upgrading for the Runner v1 proto. | ||||||||||
| RunnerApi.Pipeline dataflowV1PipelineProto = | ||||||||||
| PipelineTranslation.toProto(pipeline, dataflowV1Components, true, false); | ||||||||||
|
|
||||||||||
| if (LOG.isDebugEnabled()) { | ||||||||||
| LOG.debug( | ||||||||||
| "Dataflow non-portable worker pipeline proto:\n{}", | ||||||||||
| TextFormat.printer().printToString(dataflowNonPortablePipelineProto)); | ||||||||||
| "Dataflow v1 pipeline proto:\n{}", | ||||||||||
| TextFormat.printer().printToString(dataflowV1PipelineProto)); | ||||||||||
| } | ||||||||||
|
|
||||||||||
| // Set a unique client_request_id in the CreateJob request. | ||||||||||
|
|
@@ -1418,11 +1415,7 @@ public DataflowPipelineJob run(Pipeline pipeline) { | |||||||||
|
|
||||||||||
| JobSpecification jobSpecification = | ||||||||||
| translator.translate( | ||||||||||
| pipeline, | ||||||||||
| dataflowNonPortablePipelineProto, | ||||||||||
| dataflowNonPortableComponents, | ||||||||||
| this, | ||||||||||
| packages); | ||||||||||
| pipeline, dataflowV1PipelineProto, dataflowV1Components, this, packages); | ||||||||||
|
|
||||||||||
| if (!isNullOrEmpty(dataflowOptions.getDataflowWorkerJar()) && !useUnifiedWorker(options)) { | ||||||||||
| List<String> experiments = | ||||||||||
|
|
||||||||||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -25,13 +25,11 @@ | |
| import org.apache.beam.model.pipeline.v1.RunnerApi; | ||
| import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; | ||
| import org.apache.beam.sdk.coders.Coder; | ||
| import org.apache.beam.sdk.options.PipelineOptions; | ||
| import org.apache.beam.sdk.util.SerializableUtils; | ||
| import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; | ||
| import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; | ||
| import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BiMap; | ||
| import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableBiMap; | ||
| import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; | ||
| import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; | ||
| import org.checkerframework.checker.nullness.qual.MonotonicNonNull; | ||
| import org.checkerframework.dataflow.qual.Deterministic; | ||
|
|
@@ -64,8 +62,6 @@ private static class DefaultTranslationContext implements TranslationContext {} | |
|
|
||
| private static @MonotonicNonNull BiMap<Class<? extends Coder>, String> knownCoderUrns; | ||
|
|
||
| private static @MonotonicNonNull List<CoderTranslatorRegistrar> coderTranslatorRegistrars; | ||
|
|
||
| private static @MonotonicNonNull Map<Class<? extends Coder>, CoderTranslator<? extends Coder>> | ||
| knownTranslators; | ||
|
|
||
|
|
@@ -84,53 +80,6 @@ static BiMap<Class<? extends Coder>, String> getKnownCoderUrns() { | |
| return knownCoderUrns; | ||
| } | ||
|
|
||
| private static void initializeCoderTranslatorRegistrars() { | ||
| ImmutableList.Builder<CoderTranslatorRegistrar> registrars = ImmutableList.builder(); | ||
| for (CoderTranslatorRegistrar coderTranslatorRegistrar : | ||
| ServiceLoader.load(CoderTranslatorRegistrar.class)) { | ||
| registrars.add(coderTranslatorRegistrar); | ||
| } | ||
| coderTranslatorRegistrars = registrars.build(); | ||
| } | ||
|
|
||
| static boolean isKnownCoder(Coder<?> coder, PipelineOptions options) { | ||
| if (coderTranslatorRegistrars == null) { | ||
| initializeCoderTranslatorRegistrars(); | ||
| } | ||
| for (CoderTranslatorRegistrar registrar : coderTranslatorRegistrars) { | ||
| if (registrar.isKnownCoder(coder, options)) { | ||
| return true; | ||
| } | ||
| } | ||
| return false; | ||
| } | ||
|
|
||
| static CoderTranslator<? extends Coder> getCoderTranslator(Class<? extends Coder> coderClass) { | ||
| if (coderTranslatorRegistrars == null) { | ||
| initializeCoderTranslatorRegistrars(); | ||
| } | ||
| for (CoderTranslatorRegistrar registrar : coderTranslatorRegistrars) { | ||
| CoderTranslator translator = registrar.getCoderTranslator(coderClass); | ||
| if (translator != null) { | ||
| return translator; | ||
| } | ||
| } | ||
| return null; | ||
| } | ||
|
|
||
| static Class<? extends Coder> getCoderForUrn(String coderUrn) { | ||
| if (coderTranslatorRegistrars == null) { | ||
| initializeCoderTranslatorRegistrars(); | ||
| } | ||
| for (CoderTranslatorRegistrar registrar : coderTranslatorRegistrars) { | ||
| Class<? extends Coder> coder = registrar.getCoderForUrn(coderUrn); | ||
| if (coder != null) { | ||
| return coder; | ||
| } | ||
| } | ||
| return null; | ||
| } | ||
|
|
||
| @VisibleForTesting | ||
| @Deterministic | ||
| static Map<Class<? extends Coder>, CoderTranslator<? extends Coder>> getKnownTranslators() { | ||
|
|
@@ -158,7 +107,7 @@ public static RunnerApi.MessageWithComponents toProto(Coder<?> coder) throws IOE | |
|
|
||
| public static RunnerApi.Coder toProto(Coder<?> coder, SdkComponents components) | ||
| throws IOException { | ||
| if (isKnownCoder(coder, components.getPipelineOptions())) { | ||
| if (getKnownCoderUrns().containsKey(coder.getClass())) { | ||
| return toKnownCoder(coder, components); | ||
| } | ||
|
|
||
|
|
@@ -180,10 +129,7 @@ private static RunnerApi.Coder toUnknownCoderWrapper(UnknownCoderWrapper coder) | |
|
|
||
| private static RunnerApi.Coder toKnownCoder(Coder<?> coder, SdkComponents components) | ||
| throws IOException { | ||
| CoderTranslator translator = getCoderTranslator(coder.getClass()); | ||
| if (translator == null) { | ||
| throw new IOException("Unable to find CoderTranslator for known Coder"); | ||
| } | ||
| CoderTranslator translator = getKnownTranslators().get(coder.getClass()); | ||
|
Contributor
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. The explicit null check and descriptive CoderTranslator translator = getKnownTranslators().get(coder.getClass());
if (translator == null) {
throw new IOException("Unable to find CoderTranslator for known Coder: " + coder.getClass());
} |
||
| List<String> componentIds = registerComponents(coder, translator, components); | ||
| return RunnerApi.Coder.newBuilder() | ||
| .addAllComponentCoderIds(componentIds) | ||
|
|
@@ -240,8 +186,8 @@ private static Coder<?> fromKnownCoder( | |
| components.getComponents().getCodersOrThrow(componentId), components, context); | ||
| coderComponents.add(innerCoder); | ||
| } | ||
| Class<? extends Coder> coderType = getCoderForUrn(coderUrn); | ||
| CoderTranslator<?> translator = getCoderTranslator(coderType); | ||
| Class<? extends Coder> coderType = getKnownCoderUrns().inverse().get(coderUrn); | ||
| CoderTranslator<?> translator = getKnownTranslators().get(coderType); | ||
| if (translator != null) { | ||
| return translator.fromComponents( | ||
| coderComponents, coder.getSpec().getPayload().toByteArray(), context); | ||
|
|
||
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.
The file is missing a newline at the end. It is a best practice to end text files with a newline character to ensure compatibility with various Unix tools and to avoid 'No newline at end of file' warnings in diffs.