提交 4f323bac 编写于 作者: G godfreyhe

[FLINK-20706][table-planner-blink] Introduce StreamPhysicalUnion, and make...

[FLINK-20706][table-planner-blink] Introduce StreamPhysicalUnion, and make StreamExecUnion only extended from ExecNode

This closes #14455
上级 febce359
/*
* 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.common;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.streaming.api.transformations.UnionTransformation;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.delegation.PlannerBase;
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.ExecNodeBase;
import org.apache.flink.table.types.logical.RowType;
import java.util.ArrayList;
import java.util.List;
/**
* Base {@link ExecNode} that is not a physical node and just union the inputs' records into one node.
*/
public abstract class CommonExecUnion extends ExecNodeBase<RowData> {
public CommonExecUnion(
List<ExecEdge> inputEdges,
RowType outputType,
String description) {
super(inputEdges, outputType, description);
}
@SuppressWarnings({ "unchecked", "rawtypes" })
@Override
protected Transformation<RowData> translateToPlanInternal(PlannerBase planner) {
final List<Transformation<RowData>> inputTransforms = new ArrayList<>();
for (ExecNode<?> input : getInputNodes()) {
inputTransforms.add((Transformation<RowData>) input.translateToPlan(planner));
}
return new UnionTransformation(inputTransforms);
}
}
......@@ -29,6 +29,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecBoundedStre
import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecMultipleInput;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecExchange;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecTableSourceScan;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecUnion;
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.StreamExecDataStreamScan;
......@@ -37,8 +38,6 @@ 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.util.Preconditions;
import org.apache.calcite.rel.core.Union;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
......@@ -232,7 +231,7 @@ public class MultipleInputNodeCreationProcessor implements DAGProcessor {
}
boolean shouldRemove = false;
if (wrapper.execNode instanceof Union) {
if (wrapper.execNode instanceof CommonExecUnion) {
// optimization 1. we do not allow union to be the tail of a multiple input
// as we're paying extra function calls for this, unless one of the united
// input is a FLIP-27 source
......@@ -278,7 +277,7 @@ public class MultipleInputNodeCreationProcessor implements DAGProcessor {
continue;
}
boolean isUnion = wrapper.execNode instanceof Union;
boolean isUnion = wrapper.execNode instanceof CommonExecUnion;
if (group.members.size() == 1) {
// optimization 4. we clean up multiple input groups with only 1 member,
......@@ -320,7 +319,7 @@ public class MultipleInputNodeCreationProcessor implements DAGProcessor {
List<ExecNodeWrapper> sameGroupWrappers = getInputWrappersInSameGroup(inputWrapper, wrapper.group);
sameGroupWrappersList.add(sameGroupWrappers);
long numberOfValuableNodes = sameGroupWrappers.stream()
.filter(w -> w.inputs.size() >= 2 && !(w.execNode instanceof Union))
.filter(w -> w.inputs.size() >= 2 && !(w.execNode instanceof CommonExecUnion))
.count();
if (numberOfValuableNodes > 0) {
numberOfUsefulInputs++;
......
/*
* 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.table.data.RowData;
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.common.CommonExecUnion;
import org.apache.flink.table.types.logical.RowType;
import java.util.List;
/**
* Stream {@link ExecNode} that is not a physical node and just union the inputs' records into one node.
*/
public class StreamExecUnion extends CommonExecUnion implements StreamExecNode<RowData> {
public StreamExecUnion(
List<ExecEdge> inputEdges,
RowType outputType,
String description) {
super(inputEdges, outputType, description);
}
}
......@@ -23,7 +23,8 @@ import org.apache.flink.streaming.api.transformations.UnionTransformation
import org.apache.flink.table.data.RowData
import org.apache.flink.table.planner.delegation.BatchPlanner
import org.apache.flink.table.planner.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef}
import org.apache.flink.table.planner.plan.nodes.exec.{LegacyBatchExecNode, ExecEdge}
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecUnion
import org.apache.flink.table.planner.plan.nodes.exec.{ExecEdge, LegacyBatchExecNode}
import org.apache.calcite.plan.{RelOptCluster, RelOptRule, RelTraitSet}
import org.apache.calcite.rel.RelDistribution.Type.{ANY, BROADCAST_DISTRIBUTED, HASH_DISTRIBUTED, RANDOM_DISTRIBUTED, RANGE_DISTRIBUTED, ROUND_ROBIN_DISTRIBUTED, SINGLETON}
......@@ -46,7 +47,8 @@ class BatchExecUnion(
outputRowType: RelDataType)
extends Union(cluster, traitSet, inputRels, all)
with BatchPhysicalRel
with LegacyBatchExecNode[RowData] {
with LegacyBatchExecNode[RowData]
with CommonExecUnion {
require(all, "Only support union all now")
......
......@@ -18,11 +18,9 @@
package org.apache.flink.table.planner.plan.nodes.physical.stream
import org.apache.flink.api.dag.Transformation
import org.apache.flink.streaming.api.transformations.UnionTransformation
import org.apache.flink.table.data.RowData
import org.apache.flink.table.planner.delegation.StreamPlanner
import org.apache.flink.table.planner.plan.nodes.exec.LegacyStreamExecNode
import org.apache.flink.table.planner.calcite.FlinkTypeFactory
import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecUnion
import org.apache.flink.table.planner.plan.nodes.exec.{ExecEdge, ExecNode}
import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
import org.apache.calcite.rel.`type`.RelDataType
......@@ -34,17 +32,16 @@ import java.util
import scala.collection.JavaConversions._
/**
* Stream physical RelNode for [[Union]].
*/
class StreamExecUnion(
* Stream physical RelNode for [[Union]].
*/
class StreamPhysicalUnion(
cluster: RelOptCluster,
traitSet: RelTraitSet,
inputRels: util.List[RelNode],
all: Boolean,
outputRowType: RelDataType)
extends Union(cluster, traitSet, inputRels, all)
with StreamPhysicalRel
with LegacyStreamExecNode[RowData] {
with StreamPhysicalRel {
require(all, "Only support union all")
......@@ -53,20 +50,18 @@ class StreamExecUnion(
override def deriveRowType(): RelDataType = outputRowType
override def copy(traitSet: RelTraitSet, inputs: util.List[RelNode], all: Boolean): SetOp = {
new StreamExecUnion(cluster, traitSet, inputs, all, outputRowType)
new StreamPhysicalUnion(cluster, traitSet, inputs, all, outputRowType)
}
override def explainTerms(pw: RelWriter): RelWriter = {
super.explainTerms(pw).item("union", outputRowType.getFieldNames.mkString(", "))
}
//~ ExecNode methods -----------------------------------------------------------
override protected def translateToPlanInternal(
planner: StreamPlanner): Transformation[RowData] = {
val transformations = getInputNodes.map {
input => input.translateToPlan(planner).asInstanceOf[Transformation[RowData]]
}
new UnionTransformation(transformations)
override def translateToExecNode(): ExecNode[_] = {
new StreamExecUnion(
getInputs.map(_ => ExecEdge.DEFAULT),
FlinkTypeFactory.toLogicalRowType(getRowType),
getRelDetailedDescription
)
}
}
......@@ -281,7 +281,7 @@ class FlinkChangelogModeInferenceProgram extends FlinkOptimizeProgram[StreamOpti
// forward children mode
createNewNode(rel, children, childrenTrait, requiredTrait, requester)
case union: StreamExecUnion =>
case union: StreamPhysicalUnion =>
// transparent forward requiredTrait to children
val children = visitChildren(rel, requiredTrait, requester)
// union provides all possible kinds of children have
......@@ -582,7 +582,7 @@ class FlinkChangelogModeInferenceProgram extends FlinkOptimizeProgram[StreamOpti
createNewNode(rel, Some(children), childTrait)
}
case union: StreamExecUnion =>
case union: StreamPhysicalUnion =>
val children = union.getInputs.map {
case child: StreamPhysicalRel =>
val childModifyKindSet = getModifyKindSet(child)
......
......@@ -406,7 +406,7 @@ object FlinkStreamRuleSets {
StreamPhysicalCalcRule.INSTANCE,
StreamPhysicalPythonCalcRule.INSTANCE,
// union
StreamExecUnionRule.INSTANCE,
StreamPhysicalUnionRule.INSTANCE,
// sort
StreamExecSortRule.INSTANCE,
StreamExecLimitRule.INSTANCE,
......
......@@ -20,7 +20,7 @@ package org.apache.flink.table.planner.plan.rules.physical.stream
import org.apache.flink.table.planner.plan.nodes.FlinkConventions
import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalUnion
import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamExecUnion
import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion
import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet}
import org.apache.calcite.rel.RelNode
......@@ -29,14 +29,14 @@ import org.apache.calcite.rel.convert.ConverterRule
import scala.collection.JavaConversions._
/**
* Rule that converts [[FlinkLogicalUnion]] to [[StreamExecUnion]].
* Rule that converts [[FlinkLogicalUnion]] to [[StreamPhysicalUnion]].
*/
class StreamExecUnionRule
class StreamPhysicalUnionRule
extends ConverterRule(
classOf[FlinkLogicalUnion],
FlinkConventions.LOGICAL,
FlinkConventions.STREAM_PHYSICAL,
"StreamExecUnionRule") {
"StreamPhysicalUnionRule") {
override def matches(call: RelOptRuleCall): Boolean = {
call.rel(0).asInstanceOf[FlinkLogicalUnion].all
......@@ -47,7 +47,7 @@ class StreamExecUnionRule
val traitSet: RelTraitSet = rel.getTraitSet.replace(FlinkConventions.STREAM_PHYSICAL)
val newInputs = union.getInputs.map(RelOptRule.convert(_, FlinkConventions.STREAM_PHYSICAL))
new StreamExecUnion(
new StreamPhysicalUnion(
rel.getCluster,
traitSet,
newInputs,
......@@ -56,6 +56,6 @@ class StreamExecUnionRule
}
}
object StreamExecUnionRule {
val INSTANCE: RelOptRule = new StreamExecUnionRule
object StreamPhysicalUnionRule {
val INSTANCE: RelOptRule = new StreamPhysicalUnionRule
}
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册