You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2021/01/06 11:32:08 UTC

[GitHub] [flink] wenlong88 commented on a change in pull request #14516: [FLINK-20783][table-planner-blink] Separate implementation of ExecNode and PhysicalNode in batch join

wenlong88 commented on a change in pull request #14516:
URL: https://github.com/apache/flink/pull/14516#discussion_r552521445



##########
File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashJoin.java
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.configuration.MemorySize;
+import org.apache.flink.streaming.api.operators.SimpleOperatorFactory;
+import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
+import org.apache.flink.streaming.api.transformations.TwoInputTransformation;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.codegen.CodeGeneratorContext;
+import org.apache.flink.table.planner.codegen.LongHashJoinGenerator;
+import org.apache.flink.table.planner.codegen.ProjectionCodeGenerator;
+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.planner.plan.nodes.exec.utils.ExecNodeUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
+import org.apache.flink.table.runtime.generated.GeneratedProjection;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.HashJoinOperator;
+import org.apache.flink.table.runtime.operators.join.HashJoinType;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.calcite.rex.RexNode;
+
+import java.util.List;
+import java.util.stream.IntStream;
+
+/** Batch {@link ExecNode} for Hash Join. */
+public class BatchExecHashJoin extends ExecNodeBase<RowData> implements BatchExecNode<RowData> {
+
+    private final FlinkJoinType joinType;
+    private final int[] leftKeys;
+    private final int[] rightKeys;
+    private final boolean[] filterNulls;
+    private final RexNode nonEquiConditions;
+    private final boolean leftIsBuild;
+    private final int leftAvgRowSize;
+    private final int rightAvgRowSize;
+    private final long leftRowCount;
+    private final long rightRowCount;
+    private final boolean tryDistinctBuildRow;
+
+    public BatchExecHashJoin(
+            FlinkJoinType joinType,
+            int[] leftKeys,
+            int[] rightKeys,
+            boolean[] filterNulls,
+            RexNode nonEquiConditions,
+            int leftAvgRowSize,
+            int rightAvgRowSize,
+            long leftRowCount,
+            long rightRowCount,
+            boolean leftIsBuild,
+            boolean tryDistinctBuildRow,
+            List<ExecEdge> inputEdges,
+            LogicalType outputType,
+            String description) {
+        super(inputEdges, outputType, description);
+        this.joinType = joinType;
+
+        Preconditions.checkArgument(leftKeys.length == rightKeys.length);
+        Preconditions.checkArgument(leftKeys.length == filterNulls.length);
+        this.leftKeys = leftKeys;
+        this.rightKeys = rightKeys;
+        this.filterNulls = filterNulls;
+        this.nonEquiConditions = nonEquiConditions;
+        this.leftIsBuild = leftIsBuild;
+        this.leftAvgRowSize = leftAvgRowSize;
+        this.rightAvgRowSize = rightAvgRowSize;
+        this.leftRowCount = leftRowCount;
+        this.rightRowCount = rightRowCount;
+        this.tryDistinctBuildRow = tryDistinctBuildRow;
+    }
+
+    @Override
+    @SuppressWarnings("Unchecked")
+    protected Transformation<RowData> translateToPlanInternal(PlannerBase planner) {
+        TableConfig config = planner.getTableConfig();
+
+        Transformation<RowData> lInput =
+                (Transformation<RowData>) getInputNodes().get(0).translateToPlan(planner);
+        Transformation<RowData> rInput =
+                (Transformation<RowData>) getInputNodes().get(1).translateToPlan(planner);
+
+        // get type
+        RowType lType = (RowType) getInputNodes().get(0).getOutputType();
+        RowType rType = (RowType) getInputNodes().get(1).getOutputType();
+
+        LogicalType[] keyFieldTypes =
+                IntStream.of(leftKeys).mapToObj(lType::getTypeAt).toArray(LogicalType[]::new);
+        RowType keyType = RowType.of(keyFieldTypes);
+
+        GeneratedJoinCondition condFunc =
+                JoinUtil.generateConditionFunction(config, nonEquiConditions, lType, rType);
+
+        // projection for equals
+        GeneratedProjection lProj =
+                ProjectionCodeGenerator.generateProjection(
+                        new CodeGeneratorContext(config),
+                        "HashJoinLeftProjection",
+                        lType,
+                        keyType,
+                        leftKeys);
+        GeneratedProjection rProj =
+                ProjectionCodeGenerator.generateProjection(
+                        new CodeGeneratorContext(config),
+                        "HashJoinRightProjection",
+                        rType,
+                        keyType,
+                        rightKeys);
+
+        Transformation<RowData> build;
+        Transformation<RowData> probe;
+        GeneratedProjection bProj;
+        GeneratedProjection pProj;
+        int[] bKeys;
+        int[] pKeys;
+        RowType bType;
+        RowType pType;
+        int bRowSize;
+        long bRowCount;
+        long pRowCount;
+        boolean reverseJoin = !leftIsBuild;
+        if (leftIsBuild) {
+            build = lInput;
+            bProj = lProj;
+            bType = lType;
+            bRowSize = leftAvgRowSize;
+            bRowCount = leftRowCount;
+            bKeys = leftKeys;
+
+            probe = rInput;
+            pProj = rProj;
+            pType = rType;
+            pRowCount = leftRowCount;
+            pKeys = rightKeys;
+        } else {
+            build = rInput;
+            bProj = rProj;
+            bType = rType;
+            bRowSize = rightAvgRowSize;
+            bRowCount = rightRowCount;
+            bKeys = rightKeys;
+
+            probe = lInput;
+            pProj = lProj;
+            pType = lType;
+            pRowCount = leftRowCount;
+            pKeys = leftKeys;
+        }

Review comment:
       the flag leftIsBuild cannot be reduced, so there is no need to specified build/probe side in the constructor, it is better to left them inside the ExecNode.




----------------------------------------------------------------
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:
users@infra.apache.org