Skip to content

Commit

Permalink
[FLINK-20515][table-planner-blink] Make BatchExecMultipleInput extend…
Browse files Browse the repository at this point in the history
…ed only from BatchExecNode and port it to Java

This closes apache#14380
  • Loading branch information
godfreyhe committed Dec 15, 2020
1 parent 67b488d commit ddd4e38
Show file tree
Hide file tree
Showing 6 changed files with 163 additions and 177 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,119 @@
/*
* 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.planner.plan.nodes.exec.batch;

import org.apache.flink.api.dag.Transformation;
import org.apache.flink.streaming.api.operators.ChainingStrategy;
import org.apache.flink.streaming.api.transformations.MultipleInputTransformation;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.delegation.BatchPlanner;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
import org.apache.flink.table.runtime.operators.multipleinput.BatchMultipleInputStreamOperatorFactory;
import org.apache.flink.table.runtime.operators.multipleinput.TableOperatorWrapperGenerator;
import org.apache.flink.table.runtime.operators.multipleinput.input.InputSpec;
import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;

import org.apache.commons.lang3.tuple.Pair;

import java.util.ArrayList;
import java.util.List;
import java.util.stream.Collectors;

/**
* Batch exec node for multiple input which contains a sub-graph of {@link ExecNode}s.
* The root node of the sub-graph is {@link #rootNode}, and the leaf nodes of the sub-graph are
* the output nodes of the {@link #getInputNodes()}.
*
* <p>The following example shows a graph of {@code ExecNode}s with multiple input node:
* <pre>{@code
* Sink
* |
* +---------+--------+
* | | |
* | Join |
* | / \ | BatchExecMultipleInput
* | Agg1 Agg2 |
* | | | |
* +----+-------+-----+
* | |
* Exchange1 Exchange2
* | |
* Scan1 Scan2
* }</pre>
*
* <p>The multiple input node contains three nodes: `Join`, `Agg1` and `Agg2`.
* `Join` is the root node ({@link #rootNode}) of the sub-graph,
* `Agg1` and `Agg2` are the leaf nodes of the sub-graph,
* `Exchange1` and `Exchange2` are the input nodes of the multiple input node.
*/
public class BatchExecMultipleInput extends BatchExecNode<RowData> {

private final ExecNode<?> rootNode;

public BatchExecMultipleInput(
List<ExecNode<?>> inputNodes,
List<ExecEdge> inputEdges,
ExecNode<?> rootNode,
String description) {
super(inputNodes, inputEdges, rootNode.getOutputType(), description);
this.rootNode = rootNode;
}

@Override
protected Transformation<RowData> translateToPlanInternal(BatchPlanner planner) {
final List<Transformation<?>> inputTransforms = new ArrayList<>();
for (ExecNode<?> input : getInputNodes()) {
inputTransforms.add(input.translateToPlan(planner));
}
final Transformation<?> outputTransform = rootNode.translateToPlan(planner);
final int[] readOrders = getInputEdges().stream().map(ExecEdge::getPriority).mapToInt(i -> i).toArray();

final TableOperatorWrapperGenerator generator = new TableOperatorWrapperGenerator(
inputTransforms, outputTransform, readOrders);
generator.generate();

final List<Pair<Transformation<?>, InputSpec>> inputTransformAndInputSpecPairs =
generator.getInputTransformAndInputSpecPairs();

final MultipleInputTransformation<RowData> multipleInputTransform = new MultipleInputTransformation<>(
getDesc(),
new BatchMultipleInputStreamOperatorFactory(
inputTransformAndInputSpecPairs.stream().map(Pair::getValue).collect(Collectors.toList()),
generator.getHeadWrappers(),
generator.getTailWrapper()),
InternalTypeInfo.of(getOutputType()),
generator.getParallelism());
inputTransformAndInputSpecPairs.forEach(input -> multipleInputTransform.addInput(input.getKey()));

if (generator.getMaxParallelism() > 0) {
multipleInputTransform.setMaxParallelism(generator.getMaxParallelism());
}
// set resources
multipleInputTransform.setResources(generator.getMinResources(), generator.getPreferredResources());
long memoryKB = generator.getManagedMemoryWeight();
ExecNodeUtil.setManagedMemoryWeight(multipleInputTransform, memoryKB * 1024L);

// set chaining strategy for source chaining
multipleInputTransform.setChainingStrategy(ChainingStrategy.HEAD_WITH_SOURCES);

return multipleInputTransform;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,11 +26,12 @@
import org.apache.flink.table.planner.plan.nodes.common.CommonPhysicalTableSourceScan;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecMultipleInput;
import org.apache.flink.table.planner.plan.nodes.exec.processor.utils.InputOrderCalculator;
import org.apache.flink.table.planner.plan.nodes.exec.processor.utils.InputPriorityConflictResolver;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
import org.apache.flink.table.planner.plan.nodes.exec.visitor.AbstractExecNodeExactlyOnceVisitor;
import org.apache.flink.table.planner.plan.nodes.physical.batch.BatchExecBoundedStreamScan;
import org.apache.flink.table.planner.plan.nodes.physical.batch.BatchExecMultipleInput;
import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamExecDataStreamScan;
import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamExecMultipleInput;
import org.apache.flink.util.Preconditions;
Expand Down Expand Up @@ -494,8 +495,8 @@ private BatchExecMultipleInput createBatchMultipleInputNode(
List<Tuple2<ExecNode<?>, ExecEdge>> inputs) {
// first calculate the input orders using InputPriorityConflictResolver
Set<ExecNode<?>> inputSet = new HashSet<>();
for (Tuple2<ExecNode<?>, ExecEdge> t : inputs) {
inputSet.add(t.f0);
for (Tuple2<ExecNode<?>, ExecEdge> tuple2 : inputs) {
inputSet.add(tuple2.f0);
}
InputOrderCalculator calculator = new InputOrderCalculator(
group.root.execNode,
Expand All @@ -504,31 +505,26 @@ private BatchExecMultipleInput createBatchMultipleInputNode(
Map<ExecNode<?>, Integer> inputOrderMap = calculator.calculate();

// then create input rels and edges with the input orders
RelNode outputRel = (RelNode) group.root.execNode;
RelNode[] inputRels = new RelNode[inputs.size()];
ExecNode<?> rootNode = group.root.execNode;
List<ExecNode<?>> inputNodes = new ArrayList<>();
ExecEdge[] inputEdges = new ExecEdge[inputs.size()];
for (int i = 0; i < inputs.size(); i++) {
ExecNode<?> inputNode = inputs.get(i).f0;
ExecEdge originalInputEdge = inputs.get(i).f1;
inputRels[i] = (RelNode) inputNode;
List<ExecEdge> inputEdges = new ArrayList<>();
for (Tuple2<ExecNode<?>, ExecEdge> tuple2 : inputs) {
ExecNode<?> inputNode = tuple2.f0;
ExecEdge originalInputEdge = tuple2.f1;
inputNodes.add(inputNode);
inputEdges[i] = ExecEdge.builder()
inputEdges.add(ExecEdge.builder()
.requiredShuffle(originalInputEdge.getRequiredShuffle())
.damBehavior(originalInputEdge.getDamBehavior())
.priority(inputOrderMap.get(inputNode))
.build();
.build());
}

BatchExecMultipleInput multipleInput = new BatchExecMultipleInput(
outputRel.getCluster(),
outputRel.getTraitSet(),
inputRels,
outputRel,
inputEdges);
// TODO remove this later
multipleInput.setInputNodes(inputNodes);
return multipleInput;
String description = ExecNodeUtil.getMultipleInputDescription(rootNode, inputNodes, inputEdges);
return new BatchExecMultipleInput(
inputNodes,
inputEdges,
rootNode,
description);
}

// --------------------------------------------------------------------------------
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,9 +25,12 @@
import org.apache.flink.streaming.api.transformations.OneInputTransformation;
import org.apache.flink.streaming.api.transformations.TwoInputTransformation;
import org.apache.flink.table.api.TableException;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;

import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;

/**
* An Utility class that helps translating {@link ExecNode} to {@link Transformation}.
Expand All @@ -37,7 +40,7 @@ public class ExecNodeUtil {
/**
* Set memoryBytes to {@link Transformation#declareManagedMemoryUseCaseAtOperatorScope(ManagedMemoryUseCase, int)}.
*/
public static <T> Transformation<T> setManagedMemoryWeight(
public static <T> void setManagedMemoryWeight(
Transformation<T> transformation,
long memoryBytes) {
// Using Bytes can easily overflow
Expand All @@ -51,7 +54,6 @@ public static <T> Transformation<T> setManagedMemoryWeight(
throw new TableException("Managed memory weight has been set, this should not happen.");
}
}
return transformation;
}

/**
Expand Down Expand Up @@ -86,4 +88,27 @@ public static <T> TwoInputTransformation<T, T, T> createTwoInputTransformation(
setManagedMemoryWeight(transformation, memoryBytes);
return transformation;
}

/**
* Return description for multiple input node.
*/
public static String getMultipleInputDescription(
ExecNode<?> rootNode,
List<ExecNode<?>> inputNodes,
List<ExecEdge> inputEdges) {
String members = ExecNodePlanDumper.treeToString(rootNode, inputNodes, true).replace("\n", "\\n");
StringBuilder sb = new StringBuilder();
sb.append("MultipleInput(");
List<String> readOrders = inputEdges.stream()
.map(ExecEdge::getPriority)
.map(Object::toString)
.collect(Collectors.toList());
boolean hasDiffReadOrder = readOrders.stream().distinct().count() > 1;
if (hasDiffReadOrder) {
sb.append("readOrder=[").append(String.join(",", readOrders)).append("], ");
}
sb.append("members=[\\n").append(members).append("]");
sb.append(")");
return sb.toString();
}
}

This file was deleted.

Loading

0 comments on commit ddd4e38

Please sign in to comment.