robertwb commented on code in PR #30332:
URL: https://github.com/apache/beam/pull/30332#discussion_r1503489865


##########
runners/flink/src/main/java/org/apache/beam/runners/flink/adapter/BeamAdapterUtils.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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 java.io.IOException;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.MapCoder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.util.construction.CoderTranslation;
+import org.apache.beam.sdk.util.construction.RehydratedComponents;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+
+public class BeamAdapterUtils {
+  private BeamAdapterUtils() {}
+
+  @SuppressWarnings("nullness")

Review Comment:
   I tried to suppress them as locally as possible. 
   
   I've changed this to explicitly assert non-nullness here, but I'm not sure 
it's an improvement. 



##########
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:
   These are package private. applyBeamTransform is the only thing I want to be 
public.



##########
runners/flink/src/main/java/org/apache/beam/runners/flink/adapter/BeamAdapterUtils.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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 java.io.IOException;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.MapCoder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.util.construction.CoderTranslation;
+import org.apache.beam.sdk.util.construction.RehydratedComponents;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+
+public class BeamAdapterUtils {
+  private BeamAdapterUtils() {}
+
+  @SuppressWarnings("nullness")
+  static <T> Coder<T> typeInformationToCoder(
+      TypeInformation<T> typeInfo, CoderRegistry coderRegistry) {
+    Class<T> clazz = typeInfo.getTypeClass();
+    if (typeInfo instanceof CoderTypeInformation) {
+      return ((CoderTypeInformation) typeInfo).getCoder();
+    } else if (clazz.getTypeParameters().length == 0) {
+      try {
+        return coderRegistry.getCoder(clazz);
+      } catch (CannotProvideCoderException exn) {
+        throw new RuntimeException(exn);
+      }
+    } else if (Iterable.class.isAssignableFrom(clazz)) {
+      return (Coder)
+          IterableCoder.of(
+              typeInformationToCoder(typeInfo.getGenericParameters().get("T"), 
coderRegistry));
+    } else if (Map.class.isAssignableFrom(clazz)) {
+      return (Coder)
+          MapCoder.of(
+              typeInformationToCoder(typeInfo.getGenericParameters().get("K"), 
coderRegistry),
+              typeInformationToCoder(typeInfo.getGenericParameters().get("V"), 
coderRegistry));
+    } else {
+      throw new RuntimeException("Coder translation for " + typeInfo + " not 
yet supported.");
+    }
+  }
+
+  static <T> TypeInformation<T> coderTotoTypeInformation(Coder<T> coder, 
PipelineOptions options) {

Review Comment:
   Done.



##########
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;

Review Comment:
   I was able to extract it out. (This was an artifact of splitting the class 
up on the generic vs. type-specific bits.) I think this is better now, though 
it does give one method that has a ton of parameters...
   
   I also pulled out FlinkInput/FlinkOutput, but we could make them inner 
classes again if you'd prefer.



##########
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;

Review Comment:
   Yeah. Means there's possibly issues linking in more than one runner though...



##########
runners/flink/src/test/java/org/apache/beam/runners/flink/adapter/BeamFlinkDataSetAdapterTest.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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 static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+
+import java.util.Map;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PCollectionTuple;
+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.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.junit.Test;
+
+public class BeamFlinkDataSetAdapterTest {
+
+  private static PTransform<PCollection<? extends String>, 
PCollection<String>> withPrefix(
+      String prefix) {
+    return ParDo.of(
+        new DoFn<String, String>() {
+          @ProcessElement
+          public void processElement(@Element String word, 
OutputReceiver<String> out) {
+            out.output(prefix + word);
+          }
+        });
+  }
+
+  @Test
+  public void testApplySimpleTransform() throws Exception {
+    ExecutionEnvironment env = 
ExecutionEnvironment.createCollectionsEnvironment();
+
+    DataSet<String> input = env.fromCollection(ImmutableList.of("a", "b", 
"c"));
+    DataSet<String> result =
+        new BeamFlinkDataSetAdapter(PipelineOptionsFactory.create(), env)

Review Comment:
   +1
   
   PipelineOptions are now optional. I had initially thought of extracting the 
env from the input, but this is not possible transforms with no inputs. I 
actually don't think that's so bad for the `Reads` (it's similar to what one 
needs to do for Flink anyway) but this now precludes empty tuples. I think 
these are rare, we could add an overload taking the environment explicitly if 
need be. 
   
   I had considered going all the way and making these static methods, but I 
want to reserve the right to store any other state here (including possibly 
across invocations) and this'll allow for people to choose shorter variable 
names too. 



-- 
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]

Reply via email to