kennknowles commented on code in PR #30332: URL: https://github.com/apache/beam/pull/30332#discussion_r1491906800
########## runners/flink/src/main/java/org/apache/beam/runners/flink/adapter/BeamFlinkAbstractAdapter.java: ########## @@ -0,0 +1,278 @@ +/* + * 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.runners.flink.adapter; + +import com.google.auto.service.AutoService; +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.function.BiFunction; +import java.util.function.Function; +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.runners.flink.FlinkPortablePipelineTranslator; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PortablePipelineOptions; +import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.NativeTransforms; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.PipelineTranslation; +import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.util.construction.TransformPayloadTranslatorRegistrar; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.PDone; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.WindowingStrategy; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.ExecutionEnvironment; + +public abstract class BeamFlinkAbstractAdapter<DataSetOrStream> { + protected final PipelineOptions pipelineOptions; + protected final ExecutionEnvironment executionEnvironment; + protected final CoderRegistry coderRegistry = CoderRegistry.createDefault(); + + protected BeamFlinkAbstractAdapter( + PipelineOptions pipelineOptions, ExecutionEnvironment executionEnvironment) { + this.pipelineOptions = pipelineOptions; + this.executionEnvironment = executionEnvironment; + } + + protected abstract TypeInformation<?> getTypeInformation(DataSetOrStream dataSetOrStream); + + @SuppressWarnings({"nullness", "rawtypes"}) + protected <BeamInputType extends PInput, BeamOutputType extends POutput> + Map<String, DataSetOrStream> applyBeamPTransformInternal( + Map<String, ? extends DataSetOrStream> inputs, + BiFunction<Pipeline, Map<String, PCollection<?>>, BeamInputType> toBeamInput, + Function<BeamOutputType, Map<String, PCollection<?>>> fromBeamOutput, + PTransform<? super BeamInputType, BeamOutputType> transform) { + Pipeline pipeline = Pipeline.create(); + + // Construct beam inputs corresponding to each Flink input. + Map<String, PCollection<?>> beamInputs = + // Copy as transformEntries lazy recomputes entries. + ImmutableMap.copyOf( + Maps.transformEntries( + inputs, + (key, flinkInput) -> + pipeline.apply( + new FlinkInput<>( + key, + BeamAdapterUtils.typeInformationToCoder( + getTypeInformation(flinkInput), coderRegistry))))); + + // Actually apply the transform to create Beam outputs. + Map<String, PCollection<?>> beamOutputs = + fromBeamOutput.apply(applyTransform(toBeamInput.apply(pipeline, beamInputs), transform)); + + // This attaches PTransforms to each output which will be used to populate the Flink outputs + // during translation. + beamOutputs.entrySet().stream() + .forEach( + e -> { + ((PCollection<Object>) e.getValue()).apply(new FlinkOutput<Object>(e.getKey())); + }); + + // This "environment" executes the SDK harness in the parent worker process. + // TODO(robertwb): Support other modes. + // TODO(robertwb): In embedded mode, consider an optimized data (and state) channel rather than + // serializing everything over grpc protos. + pipelineOptions + .as(PortablePipelineOptions.class) + .setDefaultEnvironmentType(Environments.ENVIRONMENT_EMBEDDED); + + // Extract the pipeline definition so that we can apply or Flink translation logic. + SdkComponents components = SdkComponents.create(pipelineOptions); + RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(pipeline, components); + + Map<String, DataSetOrStream> outputs = new HashMap<>(); + FlinkTranslatorAndContext<?> translatorAndContext = createTranslatorAndContext(inputs, outputs); + applyFlinkTranslator(pipelineProto, translatorAndContext); + return outputs; + } + + protected abstract FlinkTranslatorAndContext<?> createTranslatorAndContext( + Map<String, ? extends DataSetOrStream> inputs, Map<String, DataSetOrStream> outputs); + + static class FlinkTranslatorAndContext< + T extends FlinkPortablePipelineTranslator.TranslationContext> { + public final FlinkPortablePipelineTranslator<T> translator; + public final T translationContext; + + FlinkTranslatorAndContext(FlinkPortablePipelineTranslator<T> translator, T translationContext) { + this.translator = translator; + this.translationContext = translationContext; + } + } + + private static <T extends FlinkPortablePipelineTranslator.TranslationContext> + void applyFlinkTranslator( + RunnerApi.Pipeline pipelineProto, FlinkTranslatorAndContext<T> translatorAndContext) { + applyFlinkTranslator( + pipelineProto, translatorAndContext.translator, translatorAndContext.translationContext); + } + + private static <T extends FlinkPortablePipelineTranslator.TranslationContext> + void applyFlinkTranslator( + RunnerApi.Pipeline pipelineProto, + FlinkPortablePipelineTranslator<T> translator, + T translationContext) { + translator.translate(translationContext, translator.prepareForTranslation(pipelineProto)); + } + + static class FlinkInput<T> extends PTransform<PBegin, PCollection<T>> { Review Comment: I don't recall how the visibility of this resolves. Perhaps a minor thing but it would be nice for this to be package-private. In fact ideally everything is package private except just the call to `applyBeamTransform(DataSet, PTransform)` -- 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]
