JunRuiLee commented on code in PR #25578:
URL: https://github.com/apache/flink/pull/25578#discussion_r1820633873


##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/JoinUtil.scala:
##########
@@ -275,4 +281,32 @@ object JoinUtil {
       rowCount * FlinkRelMdUtil.binaryRowAverageSize(relNode)
     }
   }
+
+  def getLargeManagedMemory(joinType: FlinkJoinType, config: ExecNodeConfig): 
Long = {
+    val hashJoinManagedMemory =
+      
config.get(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_HASH_JOIN_MEMORY).getBytes
+    // The memory used by SortMergeJoinIterator that buffer the matched rows, 
each side needs
+    // this memory if it is full outer join
+    val externalBufferMemory =
+      
config.get(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_EXTERNAL_BUFFER_MEMORY).getBytes
+    // The memory used by BinaryExternalSorter for sort, the left and right 
side both need it
+    val sortMemory = 
config.get(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_SORT_MEMORY).getBytes
+    var externalBufferNum = 1
+    if (joinType eq FlinkJoinType.FULL) externalBufferNum = 2
+    val sortMergeJoinManagedMemory = externalBufferMemory * externalBufferNum 
+ sortMemory * 2
+    // Due to hash join maybe fallback to sort merge join, so here managed 
memory choose the
+    // large one
+    Math.max(hashJoinManagedMemory, sortMergeJoinManagedMemory)
+  }
+
+  def getJoinStrategyHint(relHints: ImmutableList[RelHint], joinStrategy: 
JoinStrategy): Boolean = {

Review Comment:
   I prefer to rename this method to isJoinStrategyHintMatched.



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecAdaptiveBroadcastJoin.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.ReadableConfig;
+import org.apache.flink.streaming.api.operators.AdaptiveBroadcastJoin;
+import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
+import org.apache.flink.table.data.RowData;
+import 
org.apache.flink.table.planner.adaptive.AdaptiveBroadcastJoinOperatorGenerator;
+import org.apache.flink.table.planner.codegen.CodeGeneratorContext;
+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.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeConfig;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
+import 
org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
+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.OperatorType;
+import org.apache.flink.table.planner.plan.utils.SorMergeJoinOperatorUtil;
+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.SortMergeJoinFunction;
+import 
org.apache.flink.table.runtime.operators.join.adaptive.AdaptiveBroadcastJoinOperatorFactory;
+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.InstantiationUtil;
+
+import org.apache.calcite.rex.RexNode;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.stream.IntStream;
+
+/** {@link BatchExecNode} for Adaptive Broadcast Join. */
+public class BatchExecAdaptiveBroadcastJoin extends ExecNodeBase<RowData>
+        implements BatchExecNode<RowData>, 
SingleTransformationTranslator<RowData> {
+
+    private final FlinkJoinType joinType;
+    private final int[] leftKeys;
+    private final int[] rightKeys;
+    private final boolean[] filterNulls;
+    private final boolean leftIsBuild;
+    private final int estimatedLeftAvgRowSize;
+    private final int estimatedRightAvgRowSize;
+    private final long estimatedLeftRowCount;
+    private final long estimatedRightRowCount;
+    private final boolean tryDistinctBuildRow;
+    private final RexNode condition;
+    private final String description;
+    private final OperatorType originalJoin;
+
+    public BatchExecAdaptiveBroadcastJoin(
+            ReadableConfig tableConfig,
+            FlinkJoinType joinType,
+            int[] leftKeys,
+            int[] rightKeys,
+            boolean[] filterNulls,
+            int estimatedLeftAvgRowSize,
+            int estimatedRightAvgRowSize,
+            long estimatedLeftRowCount,
+            long estimatedRightRowCount,
+            boolean leftIsBuild,
+            boolean tryDistinctBuildRow,
+            List<InputProperty> inputProperties,
+            RowType outputType,
+            String description,
+            RexNode condition,
+            OperatorType originalJoin) {
+        super(
+                ExecNodeContext.newNodeId(),
+                
ExecNodeContext.newContext(BatchExecAdaptiveBroadcastJoin.class),
+                ExecNodeContext.newPersistedConfig(
+                        BatchExecAdaptiveBroadcastJoin.class, tableConfig),
+                inputProperties,
+                outputType,
+                description);
+        this.joinType = joinType;
+        this.leftKeys = leftKeys;
+        this.rightKeys = rightKeys;
+        this.filterNulls = filterNulls;
+        this.leftIsBuild = leftIsBuild;
+        this.estimatedLeftAvgRowSize = estimatedLeftAvgRowSize;
+        this.estimatedRightAvgRowSize = estimatedRightAvgRowSize;
+        this.estimatedLeftRowCount = estimatedLeftRowCount;
+        this.estimatedRightRowCount = estimatedRightRowCount;
+        this.tryDistinctBuildRow = tryDistinctBuildRow;
+        this.condition = condition;
+        this.description = description;
+        this.originalJoin = originalJoin;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    protected Transformation<RowData> translateToPlanInternal(
+            PlannerBase planner, ExecNodeConfig config) {
+        ExecEdge leftInputEdge = getInputEdges().get(0);
+        ExecEdge rightInputEdge = getInputEdges().get(1);
+
+        Transformation<RowData> leftInputTransform =
+                (Transformation<RowData>) 
leftInputEdge.translateToPlan(planner);
+        Transformation<RowData> rightInputTransform =
+                (Transformation<RowData>) 
rightInputEdge.translateToPlan(planner);
+        // get input types
+        RowType leftType = (RowType) leftInputEdge.getOutputType();
+        RowType rightType = (RowType) rightInputEdge.getOutputType();
+
+        LogicalType[] keyFieldTypes =
+                
IntStream.of(leftKeys).mapToObj(leftType::getTypeAt).toArray(LogicalType[]::new);
+        RowType keyType = RowType.of(keyFieldTypes);
+
+        GeneratedJoinCondition condFunc =
+                JoinUtil.generateConditionFunction(
+                        config,
+                        planner.getFlinkContext().getClassLoader(),
+                        condition,
+                        leftType,
+                        rightType);
+
+        // operator
+

Review Comment:
   Unnecessary blank line



##########
flink-table/flink-table-planner/src/test/resources/restore-tests/batch-exec-join_1/join-inner-join-with-duplicate-key/plan/join-inner-join-with-duplicate-key.json:
##########
@@ -289,4 +290,4 @@
     },
     "shuffleMode" : "PIPELINED"
   } ]
-}
\ No newline at end of file
+}

Review Comment:
   Could we remove this unnecessary change? The others are similar as well.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to