godfreyhe commented on a change in pull request #13707:
URL: https://github.com/apache/flink/pull/13707#discussion_r509142238



##########
File path: 
flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/multipleinput/TableOperatorWrapperGenerator.java
##########
@@ -0,0 +1,342 @@
+/*
+ * 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.flink.table.runtime.operators.multipleinput;
+
+import org.apache.flink.api.common.operators.ResourceSpec;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.core.memory.ManagedMemoryUseCase;
+import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator;
+import org.apache.flink.streaming.api.operators.SimpleOperatorFactory;
+import org.apache.flink.streaming.api.transformations.OneInputTransformation;
+import org.apache.flink.streaming.api.transformations.TwoInputTransformation;
+import org.apache.flink.streaming.api.transformations.UnionTransformation;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.operators.multipleinput.input.InputSpec;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A generator that generates a {@link TableOperatorWrapper} graph from a 
graph of {@link Transformation}.
+ */
+public class TableOperatorWrapperGenerator {
+
+       /**
+        * Original input transformations for {@link 
MultipleInputStreamOperator}.
+        */
+       private final List<Transformation<?>> inputTransforms;
+
+       /**
+        * The tail (root) transformation of the transformation-graph in {@link 
MultipleInputStreamOperator}.
+        */
+       private final Transformation<?> tailTransform;
+
+       /**
+        * The read order corresponding to each transformation in {@link 
#inputTransforms}.
+        */
+       private final int[] readOrders;
+
+       /**
+        * Reordered input transformations which order corresponds to the order 
of {@link #inputSpecs}.
+        */
+       private final List<Transformation<?>> orderedInputTransforms;
+
+       /**
+        * The input specs which order corresponds to the order of {@link 
#orderedInputTransforms}.
+        */
+       private final List<InputSpec> inputSpecs;
+
+       /**
+        * The head (leaf) operator wrappers of the operator-graph in {@link 
MultipleInputStreamOperator}.
+        */
+       private final List<TableOperatorWrapper<?>> headWrappers;
+
+       /**
+        * The tail (root) operator wrapper of the operator-graph in {@link 
MultipleInputStreamOperator}.
+        */
+       private TableOperatorWrapper<?> tailWrapper;
+
+       /**
+        * Map the visited transformation to its generated TableOperatorWrapper.
+        */
+       private final Map<Transformation<?>, TableOperatorWrapper<?>> 
visitedTransforms;
+       /**
+        * The identifier for each sub operator in {@link 
MultipleInputStreamOperator}.
+        */
+       private int identifierOfSubOp = 0;
+
+       private int parallelism;
+       private int maxParallelism;
+       private ResourceSpec minResources;
+       private ResourceSpec preferredResources;
+       /**
+        * managed memory weight for batch operator.
+        */
+       private int managedMemoryWeight;
+
+       public TableOperatorWrapperGenerator(
+                       List<Transformation<?>> inputTransforms,
+                       Transformation<?> tailTransform) {
+               this(inputTransforms, tailTransform, new 
int[inputTransforms.size()]);
+       }
+
+       public TableOperatorWrapperGenerator(
+                       List<Transformation<?>> inputTransforms,
+                       Transformation<?> tailTransform,
+                       int[] readOrders) {
+               this.inputTransforms = inputTransforms;
+               this.tailTransform = tailTransform;
+               this.readOrders = readOrders;
+               this.inputSpecs = new ArrayList<>();
+               this.headWrappers = new ArrayList<>();
+               this.orderedInputTransforms = new ArrayList<>();
+               this.visitedTransforms = new IdentityHashMap<>();
+
+               this.parallelism = -1;
+               this.maxParallelism = -1;
+       }
+
+       public void generate() {
+               tailWrapper = visit(tailTransform);
+               checkState(orderedInputTransforms.size() == 
inputTransforms.size());
+               checkState(orderedInputTransforms.size() == inputSpecs.size());
+               calculateManagedMemoryFraction();
+       }
+
+       public List<Transformation<?>> getOrderedInputTransforms() {
+               return orderedInputTransforms;
+       }
+
+       public List<InputSpec> getInputSpecs() {
+               return inputSpecs;
+       }
+
+       public List<TableOperatorWrapper<?>> getHeadWrappers() {
+               return headWrappers;
+       }
+
+       public TableOperatorWrapper<?> getTailWrapper() {
+               return tailWrapper;
+       }
+
+       public int getParallelism() {
+               return parallelism;
+       }
+
+       public int getMaxParallelism() {
+               return maxParallelism;
+       }
+
+       public ResourceSpec getMinResources() {
+               return minResources;
+       }
+
+       public ResourceSpec getPreferredResources() {
+               return preferredResources;
+       }
+
+       public int getManagedMemoryWeight() {
+               return managedMemoryWeight;
+       }
+
+       private TableOperatorWrapper<?> visit(Transformation<?> transform) {
+               // ignore UnionTransformation because it's not a really operator
+               if (!(transform instanceof UnionTransformation)) {
+                       calcParallelismAndResource(transform);
+               }
+
+               final TableOperatorWrapper<?> wrapper;
+               if (visitedTransforms.containsKey(transform)) {
+                       wrapper = visitedTransforms.get(transform);
+               } else {
+                       wrapper = visitTransformation(transform);
+                       visitedTransforms.put(transform, wrapper);
+               }
+               return wrapper;
+       }
+
+       private void calcParallelismAndResource(Transformation<?> transform) {
+               int currentParallelism = transform.getParallelism();
+               if (parallelism < 0) {
+                       parallelism = currentParallelism;
+               } else {
+                       checkState(
+                                       currentParallelism < 0 || parallelism 
== currentParallelism,
+                                       "Parallelism of a transformation in 
MultipleInputNode is different from others. This is a bug.");

Review comment:
       good catch, `set the parallelism to the maximum parallelism of the 
members` looks good to me. but we should add more strict validation for the 
plan when constructing multiple input node.




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

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to