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/04 12:49:35 UTC

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

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



##########
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;

Review comment:
       add prefix `estimated` ?

##########
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")

Review comment:
       typo: `Unchecked` -> `unchecked`

##########
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,

Review comment:
       RowType

##########
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,

Review comment:
       add `@Nullable` annotation

##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashJoin.scala
##########
@@ -174,7 +163,37 @@ class BatchExecHashJoin(
 
   //~ ExecNode methods -----------------------------------------------------------
 
-  override def getInputEdges: util.List[ExecEdge] = {
+  override def translateToExecNode(): ExecNode[_] = {
+
+    val nonEquiPredicates = if (!joinInfo.isEqui) {
+      joinInfo.getRemaining(getCluster.getRexBuilder)
+    } else {
+      null
+    }
+    val mq = getCluster.getMetadataQuery
+    val leftRowSize = Util.first(mq.getAverageRowSize(left), 24).toInt
+    val leftRowCount = Util.first(mq.getRowCount(left), 200000).toLong
+    val rightRowSize = Util.first(mq.getAverageRowSize(right), 24).toInt
+    val rightRowCount = Util.first(mq.getRowCount(right), 200000).toLong
+    new BatchExecHashJoin(
+        JoinTypeUtil.getFlinkJoinType(joinType),
+        leftKeys,
+        rightKeys,
+        filterNulls,
+        nonEquiPredicates,
+        leftRowSize,
+        rightRowSize,
+        leftRowCount,
+        rightRowCount,
+        leftIsBuild,
+        tryDistinctBuildRow,
+        getInputEdges,
+        FlinkTypeFactory.toLogicalRowType(getRowType),
+        getRelDetailedDescription
+    )
+  }
+
+  def getInputEdges: util.List[ExecEdge] = {

Review comment:
       add `private`

##########
File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecNestedLoopJoin.java
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.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.NestedLoopJoinCodeGenerator;
+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.runtime.operators.CodeGenOperatorFactory;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+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.calcite.rex.RexNode;
+
+import java.util.List;
+
+/** Batch {@link ExecNode} for Nested-loop Join. */
+public class BatchExecNestedLoopJoin extends ExecNodeBase<RowData>
+        implements BatchExecNode<RowData> {
+
+    private final FlinkJoinType joinType;
+    private final RexNode condition;
+    private final boolean leftIsBuild;
+    private final boolean singleRowJoin;
+
+    public BatchExecNestedLoopJoin(
+            FlinkJoinType joinType,
+            RexNode condition,
+            boolean leftIsBuild,
+            boolean singleRowJoin,
+            List<ExecEdge> inputEdges,
+            LogicalType outputType,

Review comment:
       RowType

##########
File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortMergeJoin.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.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.ProjectionCodeGenerator;
+import org.apache.flink.table.planner.codegen.sort.SortCodeGenerator;
+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.planner.plan.utils.SortUtil;
+import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.SortMergeJoinOperator;
+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.Arrays;
+import java.util.List;
+import java.util.stream.IntStream;
+
+import scala.Tuple3;
+
+/** Batch {@link ExecNode} for Sort Merge Join. */
+public class BatchExecSortMergeJoin 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 leftIsSmaller;
+
+    public BatchExecSortMergeJoin(
+            FlinkJoinType joinType,
+            int[] leftKeys,
+            int[] rightKeys,
+            boolean[] filterNulls,
+            RexNode nonEquiConditions,
+            boolean leftIsSmaller,
+            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.leftIsSmaller = leftIsSmaller;
+    }
+
+    @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);
+
+        long externalBufferMemory =
+                MemorySize.parse(
+                                config.getConfiguration()
+                                        .getString(
+                                                ExecutionConfigOptions
+                                                        .TABLE_EXEC_RESOURCE_EXTERNAL_BUFFER_MEMORY))
+                        .getBytes();
+        long sortMemory =
+                MemorySize.parse(
+                                config.getConfiguration()
+                                        .getString(
+                                                ExecutionConfigOptions
+                                                        .TABLE_EXEC_RESOURCE_SORT_MEMORY))
+                        .getBytes();
+        int externalBufferNum = 1;
+        if (joinType == FlinkJoinType.FULL) {
+            externalBufferNum = 2;
+        }
+
+        long managedMemory = externalBufferMemory * externalBufferNum + sortMemory * 2;
+
+        SortCodeGenerator leftSortGen = newSortGen(config, leftKeys, lType);
+        SortCodeGenerator rightSortGen = newSortGen(config, rightKeys, rType);
+
+        int[] keyPositions = IntStream.range(0, leftKeys.length).toArray();
+        SortMergeJoinOperator operator =
+                new SortMergeJoinOperator(
+                        1.0 * externalBufferMemory / managedMemory,
+                        joinType,
+                        leftIsSmaller,
+                        condFunc,
+                        ProjectionCodeGenerator.generateProjection(
+                                new CodeGeneratorContext(config),
+                                "SMJProjection",
+                                lType,
+                                keyType,
+                                leftKeys),
+                        ProjectionCodeGenerator.generateProjection(
+                                new CodeGeneratorContext(config),
+                                "SMJProjection",
+                                rType,
+                                keyType,
+                                rightKeys),
+                        leftSortGen.generateNormalizedKeyComputer("LeftComputer"),
+                        leftSortGen.generateRecordComparator("LeftComparator"),
+                        rightSortGen.generateNormalizedKeyComputer("RightComputer"),
+                        rightSortGen.generateRecordComparator("RightComparator"),
+                        newSortGen(config, keyPositions, keyType)
+                                .generateRecordComparator("KeyComparator"),
+                        filterNulls);
+
+        TwoInputTransformation<RowData, RowData, RowData> ret =
+                ExecNodeUtil.createTwoInputTransformation(
+                        lInput,
+                        rInput,
+                        getDesc(),
+                        SimpleOperatorFactory.of(operator),
+                        InternalTypeInfo.of(getOutputType()),
+                        rInput.getParallelism(),
+                        managedMemory);
+
+        if (inputsContainSingleton()) {
+            ret.setParallelism(1);
+            ret.setMaxParallelism(1);
+        }
+        return ret;
+    }
+
+    private SortCodeGenerator newSortGen(TableConfig config, int[] originalKeys, RowType type) {
+        boolean[] originalOrders = new boolean[originalKeys.length];
+        Arrays.fill(originalOrders, true);
+        boolean[] nullsIsLast = SortUtil.getNullDefaultOrders(originalOrders);
+        Tuple3<int[], boolean[], boolean[]> ret =
+                SortUtil.deduplicateSortKeys(originalKeys, originalOrders, nullsIsLast);

Review comment:
       add some comments to explain what's each element in Tuple3?

##########
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);

Review comment:
       `leftKeys` should be non-empty.

##########
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();

Review comment:
       add left/right input node local field ?

##########
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:
       give build/probe fields directly in the constructor ?

##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashJoin.scala
##########
@@ -174,7 +163,37 @@ class BatchExecHashJoin(
 
   //~ ExecNode methods -----------------------------------------------------------

Review comment:
       remove this line

##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashJoin.scala
##########
@@ -174,7 +163,37 @@ class BatchExecHashJoin(
 
   //~ ExecNode methods -----------------------------------------------------------
 
-  override def getInputEdges: util.List[ExecEdge] = {
+  override def translateToExecNode(): ExecNode[_] = {
+

Review comment:
       remove empty line

##########
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;

Review comment:
       add `@Nullable` annotation

##########
File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortMergeJoin.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.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.ProjectionCodeGenerator;
+import org.apache.flink.table.planner.codegen.sort.SortCodeGenerator;
+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.planner.plan.utils.SortUtil;
+import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.SortMergeJoinOperator;
+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.Arrays;
+import java.util.List;
+import java.util.stream.IntStream;
+
+import scala.Tuple3;
+
+/** Batch {@link ExecNode} for Sort Merge Join. */
+public class BatchExecSortMergeJoin 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 leftIsSmaller;
+
+    public BatchExecSortMergeJoin(
+            FlinkJoinType joinType,
+            int[] leftKeys,
+            int[] rightKeys,
+            boolean[] filterNulls,
+            RexNode nonEquiConditions,
+            boolean leftIsSmaller,
+            List<ExecEdge> inputEdges,
+            LogicalType outputType,

Review comment:
       RowType

##########
File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortMergeJoin.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.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.ProjectionCodeGenerator;
+import org.apache.flink.table.planner.codegen.sort.SortCodeGenerator;
+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.planner.plan.utils.SortUtil;
+import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.SortMergeJoinOperator;
+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.Arrays;
+import java.util.List;
+import java.util.stream.IntStream;
+
+import scala.Tuple3;
+
+/** Batch {@link ExecNode} for Sort Merge Join. */
+public class BatchExecSortMergeJoin 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;

Review comment:
       add @Nullable annotation

##########
File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortMergeJoin.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.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.ProjectionCodeGenerator;
+import org.apache.flink.table.planner.codegen.sort.SortCodeGenerator;
+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.planner.plan.utils.SortUtil;
+import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.SortMergeJoinOperator;
+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.Arrays;
+import java.util.List;
+import java.util.stream.IntStream;
+
+import scala.Tuple3;
+
+/** Batch {@link ExecNode} for Sort Merge Join. */
+public class BatchExecSortMergeJoin 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 leftIsSmaller;
+
+    public BatchExecSortMergeJoin(
+            FlinkJoinType joinType,
+            int[] leftKeys,
+            int[] rightKeys,
+            boolean[] filterNulls,
+            RexNode nonEquiConditions,
+            boolean leftIsSmaller,
+            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.leftIsSmaller = leftIsSmaller;
+    }
+
+    @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);
+
+        long externalBufferMemory =
+                MemorySize.parse(
+                                config.getConfiguration()
+                                        .getString(
+                                                ExecutionConfigOptions
+                                                        .TABLE_EXEC_RESOURCE_EXTERNAL_BUFFER_MEMORY))
+                        .getBytes();
+        long sortMemory =
+                MemorySize.parse(
+                                config.getConfiguration()
+                                        .getString(
+                                                ExecutionConfigOptions
+                                                        .TABLE_EXEC_RESOURCE_SORT_MEMORY))
+                        .getBytes();

Review comment:
       extract an utility method ?

##########
File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortMergeJoin.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.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.ProjectionCodeGenerator;
+import org.apache.flink.table.planner.codegen.sort.SortCodeGenerator;
+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.planner.plan.utils.SortUtil;
+import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.SortMergeJoinOperator;
+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.Arrays;
+import java.util.List;
+import java.util.stream.IntStream;
+
+import scala.Tuple3;
+
+/** Batch {@link ExecNode} for Sort Merge Join. */
+public class BatchExecSortMergeJoin 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 leftIsSmaller;
+
+    public BatchExecSortMergeJoin(
+            FlinkJoinType joinType,
+            int[] leftKeys,
+            int[] rightKeys,
+            boolean[] filterNulls,
+            RexNode nonEquiConditions,

Review comment:
       add @Nullable annotation

##########
File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortMergeJoin.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.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.ProjectionCodeGenerator;
+import org.apache.flink.table.planner.codegen.sort.SortCodeGenerator;
+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.planner.plan.utils.SortUtil;
+import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.SortMergeJoinOperator;
+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.Arrays;
+import java.util.List;
+import java.util.stream.IntStream;
+
+import scala.Tuple3;
+
+/** Batch {@link ExecNode} for Sort Merge Join. */
+public class BatchExecSortMergeJoin 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 leftIsSmaller;
+
+    public BatchExecSortMergeJoin(
+            FlinkJoinType joinType,
+            int[] leftKeys,
+            int[] rightKeys,
+            boolean[] filterNulls,
+            RexNode nonEquiConditions,
+            boolean leftIsSmaller,
+            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.leftIsSmaller = leftIsSmaller;
+    }
+
+    @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();
+

Review comment:
       add left/right input node local variable ?

##########
File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortMergeJoin.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.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.ProjectionCodeGenerator;
+import org.apache.flink.table.planner.codegen.sort.SortCodeGenerator;
+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.planner.plan.utils.SortUtil;
+import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.SortMergeJoinOperator;
+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.Arrays;
+import java.util.List;
+import java.util.stream.IntStream;
+
+import scala.Tuple3;
+
+/** Batch {@link ExecNode} for Sort Merge Join. */
+public class BatchExecSortMergeJoin 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 leftIsSmaller;
+
+    public BatchExecSortMergeJoin(
+            FlinkJoinType joinType,
+            int[] leftKeys,
+            int[] rightKeys,
+            boolean[] filterNulls,
+            RexNode nonEquiConditions,
+            boolean leftIsSmaller,
+            List<ExecEdge> inputEdges,
+            LogicalType outputType,
+            String description) {
+        super(inputEdges, outputType, description);
+        this.joinType = joinType;
+
+        Preconditions.checkArgument(leftKeys.length == rightKeys.length);

Review comment:
       `leftKeys` should not be empty




----------------------------------------------------------------
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