Skip to content

Commit

Permalink
[FLINK-20515][table-planner-blink] Make StreamExecMultipleInput exten…
Browse files Browse the repository at this point in the history
…ded only from StreamExecNode and port it to Java

This closes apache#14380
  • Loading branch information
godfreyhe committed Dec 15, 2020
1 parent ddd4e38 commit 6c886a4
Show file tree
Hide file tree
Showing 3 changed files with 86 additions and 77 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -29,15 +29,14 @@
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.stream.StreamExecMultipleInput;
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.stream.StreamExecDataStreamScan;
import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamExecMultipleInput;
import org.apache.flink.util.Preconditions;

import org.apache.calcite.rel.RelDistribution;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.core.Exchange;
import org.apache.calcite.rel.core.Union;

Expand Down Expand Up @@ -472,22 +471,17 @@ private ExecNode<?> createMultipleInputNode(
private StreamExecMultipleInput createStreamMultipleInputNode(
MultipleInputGroup group,
List<Tuple2<ExecNode<?>, ExecEdge>> inputs) {
RelNode outputRel = (RelNode) group.root.execNode;
RelNode[] inputRels = new RelNode[inputs.size()];
ExecNode<?> rootNode = group.root.execNode;
List<ExecNode<?>> inputNodes = new ArrayList<>();
for (int i = 0; i < inputs.size(); i++) {
inputRels[i] = (RelNode) inputs.get(i).f0;
inputNodes.add(inputs.get(i).f0);
for (Tuple2<ExecNode<?>, ExecEdge> tuple2 : inputs) {
inputNodes.add(tuple2.f0);
}

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

private BatchExecMultipleInput createBatchMultipleInputNode(
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
/*
* 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.stream;

import org.apache.flink.api.dag.Transformation;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.delegation.StreamPlanner;
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.stream.Collectors;

/**
* Stream 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 |
* | / \ | StreamExecMultipleInput
* | 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 StreamExecMultipleInput extends StreamExecNode<RowData> {

private final ExecNode<?> rootNode;

public StreamExecMultipleInput(
List<ExecNode<?>> inputNodes,
ExecNode<?> rootNode,
String description) {
super(
inputNodes,
inputNodes.stream().map(i -> ExecEdge.DEFAULT).collect(Collectors.toList()),
rootNode.getOutputType(),
description);
this.rootNode = rootNode;
}

@Override
protected Transformation<RowData> translateToPlanInternal(StreamPlanner planner) {
throw new UnsupportedOperationException("This method is not implemented yet.");
}
}

This file was deleted.

0 comments on commit 6c886a4

Please sign in to comment.