wenlong88 commented on a change in pull request #18707: URL: https://github.com/apache/flink/pull/18707#discussion_r804436560
########## File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/ForwardHashExchangeProcessor.java ########## @@ -0,0 +1,171 @@ +/* + * 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.processor; + +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 org.apache.flink.table.planner.plan.nodes.exec.ExecNodeGraph; +import org.apache.flink.table.planner.plan.nodes.exec.InputProperty; +import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecExchange; +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.stream.StreamExecNode; +import org.apache.flink.table.planner.plan.nodes.exec.visitor.AbstractExecNodeExactlyOnceVisitor; +import org.apache.flink.table.planner.plan.nodes.exec.visitor.ExecNodeVisitor; +import org.apache.flink.table.types.logical.RowType; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * A {@link ExecNodeGraphProcessor} which finds all {@link ExecNode}s that require hash + * distribution, but its input has no hash Exchange node. Currently, the hash distribution is + * satisfied by its none exchange inputs with FORWARD partitioner (the node and its input has the + * same parallelism). Once the parallelism is changed, the FORWARD behavior will be broken, and the + * result will be wrong. + * + * <p>In order to meet the needs of flexible parallelism changing, a special {@link + * BatchExecExchange} (with KEEP_INPUT_AS_IS distribution flag) will be added for the {@link + * ExecNode} as its input. And then the runtime will decide which partitioner can be used when + * adaptive scheduler is enabled: FORWARD partitioner if the parallelism is the same, or HASH + * partitioner if the parallelism is different. + * + * <p>Its works only for batch job when dynamic-graph is enabled. + */ +public class ForwardHashExchangeProcessor implements ExecNodeGraphProcessor { + + @Override + public ExecNodeGraph process(ExecNodeGraph execGraph, ProcessorContext context) { + if (execGraph.getRootNodes().get(0) instanceof StreamExecNode) { + throw new TableException("StreamExecNode is not supported yet"); + } + if (!context.getPlanner().getExecEnv().getConfig().isDynamicGraph()) { + return execGraph; + } + ExecNodeVisitor visitor = + new AbstractExecNodeExactlyOnceVisitor() { + @Override + protected void visitNode(ExecNode<?> node) { + visitInputs(node); + if (node instanceof CommonExecExchange) { + return; + } + boolean changed = false; + List<ExecEdge> newEdges = new ArrayList<>(node.getInputEdges()); + for (int i = 0; i < node.getInputProperties().size(); ++i) { + InputProperty inputProperty = node.getInputProperties().get(i); + InputProperty.RequiredDistribution requiredDistribution = + inputProperty.getRequiredDistribution(); + if (requiredDistribution.getType() + != InputProperty.DistributionType.HASH) { + continue; + } + ExecEdge edge = node.getInputEdges().get(i); + if (!hasExchangeInput(edge)) { + InputProperty newInputProperty = + InputProperty.builder() + .requiredDistribution( + InputProperty.keepInputAsIsDistribution( + requiredDistribution)) + .damBehavior(inputProperty.getDamBehavior()) + .priority(inputProperty.getPriority()) + .build(); + BatchExecExchange newExchange = + new BatchExecExchange( + newInputProperty, + (RowType) edge.getOutputType(), + newInputProperty.toString()); + + ExecEdge newEdge1 = + new ExecEdge( + edge.getSource(), + newExchange, + edge.getShuffle(), + edge.getExchangeMode()); + newExchange.setInputEdges(Collections.singletonList(newEdge1)); + + ExecEdge newEdge2 = + new ExecEdge( + newExchange, + edge.getTarget(), + edge.getShuffle(), + edge.getExchangeMode()); + + // update the edge + newEdges.set(i, newEdge2); + updateOriginalEdgeInMultipleInput(node, i, newExchange); + changed = true; + } + } + if (changed) { + node.setInputEdges(newEdges); + } + } + }; + execGraph.getRootNodes().forEach(s -> s.accept(visitor)); + return execGraph; + } + + private boolean hasExchangeInput(ExecEdge edge) { Review comment: rename to isExchangeInput ########## File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecMultipleInput.java ########## @@ -135,4 +143,12 @@ public BatchExecMultipleInput( return multipleInputTransform; } + + public ExecNode<?> getRootNode() { Review comment: useless? ########## File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/ForwardHashExchangeProcessor.java ########## @@ -0,0 +1,171 @@ +/* + * 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.processor; + +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 org.apache.flink.table.planner.plan.nodes.exec.ExecNodeGraph; +import org.apache.flink.table.planner.plan.nodes.exec.InputProperty; +import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecExchange; +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.stream.StreamExecNode; +import org.apache.flink.table.planner.plan.nodes.exec.visitor.AbstractExecNodeExactlyOnceVisitor; +import org.apache.flink.table.planner.plan.nodes.exec.visitor.ExecNodeVisitor; +import org.apache.flink.table.types.logical.RowType; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * A {@link ExecNodeGraphProcessor} which finds all {@link ExecNode}s that require hash + * distribution, but its input has no hash Exchange node. Currently, the hash distribution is + * satisfied by its none exchange inputs with FORWARD partitioner (the node and its input has the + * same parallelism). Once the parallelism is changed, the FORWARD behavior will be broken, and the + * result will be wrong. + * + * <p>In order to meet the needs of flexible parallelism changing, a special {@link + * BatchExecExchange} (with KEEP_INPUT_AS_IS distribution flag) will be added for the {@link + * ExecNode} as its input. And then the runtime will decide which partitioner can be used when + * adaptive scheduler is enabled: FORWARD partitioner if the parallelism is the same, or HASH + * partitioner if the parallelism is different. + * + * <p>Its works only for batch job when dynamic-graph is enabled. + */ +public class ForwardHashExchangeProcessor implements ExecNodeGraphProcessor { + + @Override + public ExecNodeGraph process(ExecNodeGraph execGraph, ProcessorContext context) { + if (execGraph.getRootNodes().get(0) instanceof StreamExecNode) { + throw new TableException("StreamExecNode is not supported yet"); + } + if (!context.getPlanner().getExecEnv().getConfig().isDynamicGraph()) { + return execGraph; + } + ExecNodeVisitor visitor = + new AbstractExecNodeExactlyOnceVisitor() { + @Override + protected void visitNode(ExecNode<?> node) { + visitInputs(node); + if (node instanceof CommonExecExchange) { + return; + } + boolean changed = false; + List<ExecEdge> newEdges = new ArrayList<>(node.getInputEdges()); + for (int i = 0; i < node.getInputProperties().size(); ++i) { + InputProperty inputProperty = node.getInputProperties().get(i); + InputProperty.RequiredDistribution requiredDistribution = + inputProperty.getRequiredDistribution(); + if (requiredDistribution.getType() + != InputProperty.DistributionType.HASH) { + continue; + } + ExecEdge edge = node.getInputEdges().get(i); + if (!hasExchangeInput(edge)) { + InputProperty newInputProperty = + InputProperty.builder() + .requiredDistribution( + InputProperty.keepInputAsIsDistribution( + requiredDistribution)) + .damBehavior(inputProperty.getDamBehavior()) + .priority(inputProperty.getPriority()) + .build(); + BatchExecExchange newExchange = + new BatchExecExchange( + newInputProperty, + (RowType) edge.getOutputType(), + newInputProperty.toString()); + + ExecEdge newEdge1 = Review comment: a suggestion: how about add an updateInput(ExecNodeBase newInput, int inputIndex) to ExecNodeBase, and BatchExecMultiInput rewrite the method, and update the input of the original node. we could make the magic of MultiInput internal in BatchMultiInput and make it easier to understand the logic here. ########## File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/ForwardHashExchangeProcessor.java ########## @@ -0,0 +1,171 @@ +/* + * 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.processor; + +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 org.apache.flink.table.planner.plan.nodes.exec.ExecNodeGraph; +import org.apache.flink.table.planner.plan.nodes.exec.InputProperty; +import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecExchange; +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.stream.StreamExecNode; +import org.apache.flink.table.planner.plan.nodes.exec.visitor.AbstractExecNodeExactlyOnceVisitor; +import org.apache.flink.table.planner.plan.nodes.exec.visitor.ExecNodeVisitor; +import org.apache.flink.table.types.logical.RowType; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * A {@link ExecNodeGraphProcessor} which finds all {@link ExecNode}s that require hash + * distribution, but its input has no hash Exchange node. Currently, the hash distribution is + * satisfied by its none exchange inputs with FORWARD partitioner (the node and its input has the + * same parallelism). Once the parallelism is changed, the FORWARD behavior will be broken, and the + * result will be wrong. + * + * <p>In order to meet the needs of flexible parallelism changing, a special {@link + * BatchExecExchange} (with KEEP_INPUT_AS_IS distribution flag) will be added for the {@link + * ExecNode} as its input. And then the runtime will decide which partitioner can be used when + * adaptive scheduler is enabled: FORWARD partitioner if the parallelism is the same, or HASH Review comment: in currently implementation of runtime, the partitioner is chosen at compile time. when it is not chained, the partitioner would be hash ignoring the parallelism. ########## File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecExchange.java ########## @@ -126,22 +130,25 @@ public String getDescription() { parallelism = 1; break; case HASH: - int[] keys = ((HashDistribution) inputProperty.getRequiredDistribution()).getKeys(); - RowType inputType = (RowType) inputEdge.getOutputType(); - String[] fieldNames = - Arrays.stream(keys) - .mapToObj(i -> inputType.getFieldNames().get(i)) - .toArray(String[]::new); partitioner = - new BinaryHashPartitioner( - HashCodeGenerator.generateRowHash( - new CodeGeneratorContext(planner.getTableConfig()), - inputEdge.getOutputType(), - "HashPartitioner", - keys), - fieldNames); + createHashPartitioner( + ((HashDistribution) requiredDistribution), inputType, planner); parallelism = ExecutionConfig.PARALLELISM_DEFAULT; break; + case KEEP_INPUT_AS_IS: + RequiredDistribution inputDistribution = + ((KeepInputAsIsDistribution) requiredDistribution).getInputDistribution(); + checkArgument( Review comment: could it be a singleton distribution? -- 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. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org