[ 
https://issues.apache.org/jira/browse/BEAM-3972?focusedWorklogId=97704&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-97704
 ]

ASF GitHub Bot logged work on BEAM-3972:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 02/May/18 22:04
            Start Date: 02/May/18 22:04
    Worklog Time Spent: 10m 
      Work Description: tgroh commented on a change in pull request #5226: 
[BEAM-3972] Translate portable batch pipelines by proto
URL: https://github.com/apache/beam/pull/5226#discussion_r185632348
 
 

 ##########
 File path: 
runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPortablePipelineTranslator.java
 ##########
 @@ -0,0 +1,607 @@
+/*
+ * 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;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.google.common.collect.BiMap;
+import com.google.common.collect.ImmutableBiMap;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.protobuf.InvalidProtocolBufferException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
+import 
org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload.SideInputId;
+import org.apache.beam.runners.core.construction.CoderTranslation;
+import org.apache.beam.runners.core.construction.PTransformTranslation;
+import org.apache.beam.runners.core.construction.PipelineOptionsTranslation;
+import org.apache.beam.runners.core.construction.RehydratedComponents;
+import org.apache.beam.runners.core.construction.WindowingStrategyTranslation;
+import org.apache.beam.runners.core.construction.graph.ExecutableStage;
+import org.apache.beam.runners.core.construction.graph.PipelineNode;
+import 
org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
+import org.apache.beam.runners.core.construction.graph.QueryablePipeline;
+import org.apache.beam.runners.flink.translation.functions.FlinkAssignWindows;
+import 
org.apache.beam.runners.flink.translation.functions.FlinkExecutableStageFunction;
+import 
org.apache.beam.runners.flink.translation.functions.FlinkExecutableStagePruningFunction;
+import 
org.apache.beam.runners.flink.translation.functions.FlinkPartialReduceFunction;
+import org.apache.beam.runners.flink.translation.functions.FlinkReduceFunction;
+import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
+import org.apache.beam.runners.flink.translation.types.KvKeySelector;
+import org.apache.beam.runners.flink.translation.wrappers.ImpulseInputFormat;
+import org.apache.beam.runners.fnexecution.wire.WireCoders;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.join.RawUnionValue;
+import org.apache.beam.sdk.transforms.join.UnionCoder;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.WindowingStrategy;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.DiscardingOutputFormat;
+import org.apache.flink.api.java.operators.DataSource;
+import org.apache.flink.api.java.operators.FlatMapOperator;
+import org.apache.flink.api.java.operators.GroupCombineOperator;
+import org.apache.flink.api.java.operators.GroupReduceOperator;
+import org.apache.flink.api.java.operators.Grouping;
+import org.apache.flink.api.java.operators.MapPartitionOperator;
+
+/**
+ * A translator that translates bounded portable pipelines into executable 
Flink pipelines.
+ *
+ * <p>Example usage:
+ *
+ * <pre>
+ *   FlinkBatchPortablePipelineTranslator translator =
+ *       FlinkBatchPortablePipelineTranslator.createTranslator();
+ *   BatchTranslationContext context =
+ *       
FlinkBatchPortablePipelineTranslator.createTranslationContext(options);
+ *   translator.translate(context, pipeline);
+ *   ExecutionEnvironment executionEnvironment = 
context.getExecutionEnvironment();
+ *   // Do something with executionEnvironment...
+ * </pre>
+ *
+ * <p>After translation the {@link ExecutionEnvironment} in the translation 
context will contain the
+ * full not-yet-executed pipeline DAG corresponding to the input pipeline.
+ */
+public class FlinkBatchPortablePipelineTranslator
+    implements FlinkPortablePipelineTranslator<
+        FlinkBatchPortablePipelineTranslator.BatchTranslationContext> {
+
+  /**
+   * Creates a batch translation context. The resulting Flink execution dag 
will live in a new
+   * {@link ExecutionEnvironment}.
+   */
+  public static BatchTranslationContext 
createTranslationContext(FlinkPipelineOptions options) {
+    ExecutionEnvironment executionEnvironment =
+        FlinkExecutionEnvironments.createBatchExecutionEnvironment(options);
+    return new BatchTranslationContext(options, executionEnvironment);
+  }
+
+  /** Creates a batch translator. */
+  public static FlinkBatchPortablePipelineTranslator createTranslator() {
+    ImmutableMap.Builder<String, PTransformTranslator> translatorMap =
+        ImmutableMap.builder();
+    translatorMap.put(
+        PTransformTranslation.FLATTEN_TRANSFORM_URN,
+        FlinkBatchPortablePipelineTranslator::translateFlatten);
+    translatorMap.put(
+        PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN,
+        FlinkBatchPortablePipelineTranslator::translateGroupByKey);
+    translatorMap.put(
+        PTransformTranslation.IMPULSE_TRANSFORM_URN,
+        FlinkBatchPortablePipelineTranslator::translateImpulse);
+    translatorMap.put(
+        PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN,
+        FlinkBatchPortablePipelineTranslator::translateAssignWindows);
+    translatorMap.put(
+        ExecutableStage.URN, 
FlinkBatchPortablePipelineTranslator::translateExecutableStage);
+    translatorMap.put(
+        PTransformTranslation.RESHUFFLE_URN,
+        FlinkBatchPortablePipelineTranslator::translateReshuffle);
+    translatorMap.put(
+        PTransformTranslation.CREATE_VIEW_TRANSFORM_URN,
+        FlinkBatchPortablePipelineTranslator::translateView);
+    return new FlinkBatchPortablePipelineTranslator(translatorMap.build());
+  }
+
+  /**
+   * Batch translation context. Stores metadata about known 
PCollections/DataSets and holds the
+   * flink {@link ExecutionEnvironment} that the execution plan will be 
applied to.
+   */
+  public static class BatchTranslationContext
+      implements FlinkPortablePipelineTranslator.TranslationContext {
+
+    private final FlinkPipelineOptions options;
+    private final ExecutionEnvironment executionEnvironment;
+    private final Map<String, DataSet<?>> dataSets;
+    private final Set<String> danglingDataSets;
+
+    private BatchTranslationContext(
+        FlinkPipelineOptions options, ExecutionEnvironment 
executionEnvironment) {
+      this.options = options;
+      this.executionEnvironment = executionEnvironment;
+      dataSets = new HashMap<>();
+      danglingDataSets = new HashSet<>();
+    }
+
+    @Override
+    public FlinkPipelineOptions getPipelineOptions() {
+      return options;
+    }
+
+    public ExecutionEnvironment getExecutionEnvironment() {
+      return executionEnvironment;
+    }
+
+    public <T> void addDataSet(String pCollectionId, DataSet<T> dataSet) {
+      checkArgument(!dataSets.containsKey(pCollectionId));
+      dataSets.put(pCollectionId, dataSet);
+      danglingDataSets.add(pCollectionId);
+    }
+
+    public <T> DataSet<T> getDataSetOrThrow(String pCollectionId) {
+      DataSet<T> dataSet = (DataSet<T>) dataSets.get(pCollectionId);
+      if (dataSet == null) {
+        throw new IllegalArgumentException(
+            String.format("Unknown dataset for id %s.", pCollectionId));
+      }
+
+      // Assume that the DataSet is consumed if requested. We use this as a 
proxy for consumption
+      // because Flink does not expose its internal execution plan.
+      danglingDataSets.remove(pCollectionId);
+      return dataSet;
+    }
+
+    public Collection<DataSet<?>> getDanglingDataSets() {
+      return danglingDataSets.stream().map(id -> 
dataSets.get(id)).collect(Collectors.toList());
+    }
+  }
+
+  /** Transform translation interface. */
+  @FunctionalInterface
+  private interface PTransformTranslator {
+    /** Translate the PTransform of the given id into the given translation 
context. */
+    void translate(String id, RunnerApi.Pipeline pipeline, 
BatchTranslationContext t);
+  }
+
+  private final Map<String, PTransformTranslator> urnToTransformTranslator;
+
+  private FlinkBatchPortablePipelineTranslator(
+      Map<String, PTransformTranslator> urnToTransformTranslator) {
+    this.urnToTransformTranslator = urnToTransformTranslator;
+  }
+
+  @Override
+  public void translate(BatchTranslationContext context, RunnerApi.Pipeline 
pipeline) {
+    // Use a QueryablePipeline to traverse transforms topologically.
+    QueryablePipeline p =
+        QueryablePipeline.forTransforms(
+            pipeline.getRootTransformIdsList(), pipeline.getComponents());
+    for (PipelineNode.PTransformNode transform : 
p.getTopologicallyOrderedTransforms()) {
+      urnToTransformTranslator
+          .getOrDefault(
+              transform.getTransform().getSpec().getUrn(),
+              FlinkBatchPortablePipelineTranslator::urnNotFound)
+          .translate(transform.getId(), pipeline, context);
+    }
+
+    // Ensure that side effects are performed for unconsumed DataSets.
+    for (DataSet<?> dataSet : context.getDanglingDataSets()) {
+      dataSet.output(new DiscardingOutputFormat<>());
+    }
+  }
+
+  private static <InputT> void translateView(
+      String id, RunnerApi.Pipeline pipeline, BatchTranslationContext context) 
{
+    RunnerApi.PTransform transform = 
pipeline.getComponents().getTransformsOrThrow(id);
 
 Review comment:
   Do we ever not make this call?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 97704)
    Time Spent: 7h 50m  (was: 7h 40m)

> Flink runner translates batch pipelines directly by proto
> ---------------------------------------------------------
>
>                 Key: BEAM-3972
>                 URL: https://issues.apache.org/jira/browse/BEAM-3972
>             Project: Beam
>          Issue Type: Bug
>          Components: runner-flink
>            Reporter: Ben Sidhom
>            Assignee: Ben Sidhom
>            Priority: Major
>          Time Spent: 7h 50m
>  Remaining Estimate: 0h
>
> The non-portable runner uses reydrated pipelines which lack necessary 
> information. The portable Flink runner needs to translate pipelines directly 
> by proto in order to wire components into individual executable stages 
> correctly.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to