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


##########
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:
   Toto



##########
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:
   I'd say we should try to find a way to polish this into an extremely minimal 
API as we go:
   
    - make the pipeline options optional and default them internal to the 
adapter
    - extract the env from the inputs so you don't have to pass it here
   
   Ideally we could get very close to this signature:
   
   ```
      DataSet<OutputT> applyBeamTransform(DataSet<InputT>, PTransform<InputT, 
OutputT>)
   ```
   
   (which of course is begging to be upstreamed into `DataSet` API)



##########
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:
   I had forgotten about this or missed it. That's fun...



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