zhuzhurk commented on code in PR #25593:
URL: https://github.com/apache/flink/pull/25593#discussion_r1894581435


##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/adaptive/AdaptiveJoinOperatorGenerator.java:
##########
@@ -135,50 +148,60 @@ public StreamOperatorFactory<?> genOperatorFactory(
     }
 
     @Override
-    public Tuple2<Boolean, Boolean> enrichAndCheckBroadcast(
-            long leftInputSize, long rightInputSize, long threshold) {
-        Tuple2<Boolean, Boolean> isBroadcastAndLeftBuild;
+    public Tuple2<Boolean, Boolean> tryBroadcastOptimization(
+            Long leftInputSize,
+            Long rightInputSize,
+            Long threshold,
+            Function<Boolean, Boolean> tryTransformEdges) {
         boolean leftSizeSmallerThanThreshold = leftInputSize <= threshold;
         boolean rightSizeSmallerThanThreshold = rightInputSize <= threshold;
         boolean leftSmallerThanRight = leftInputSize < rightInputSize;
+        boolean isBroadcastJoinTemp;
+        boolean leftIsBuildTemp;
         switch (joinType) {
             case RIGHT:
                 // For a right outer join, if the left side can be broadcast, 
then the left side is
                 // always the build side; otherwise, the smaller side is the 
build side.
-                isBroadcastAndLeftBuild =
-                        new Tuple2<>(
-                                leftSizeSmallerThanThreshold,
-                                leftSizeSmallerThanThreshold ? true : 
leftSmallerThanRight);
+                isBroadcastJoinTemp = leftSizeSmallerThanThreshold;
+                leftIsBuildTemp = isBroadcastJoinTemp ? true : 
leftSmallerThanRight;
                 break;
             case INNER:
-                isBroadcastAndLeftBuild =
-                        new Tuple2<>(
-                                leftSizeSmallerThanThreshold || 
rightSizeSmallerThanThreshold,
-                                leftSmallerThanRight);
+                isBroadcastJoinTemp = leftSizeSmallerThanThreshold || 
rightSizeSmallerThanThreshold;
+                leftIsBuildTemp = leftSmallerThanRight;
                 break;
             case LEFT:
             case SEMI:
             case ANTI:
                 // For left outer / semi / anti join, if the right side can be 
broadcast, then the
                 // right side is always the build side; otherwise, the smaller 
side is the build
                 // side.
-                isBroadcastAndLeftBuild =
-                        new Tuple2<>(
-                                rightSizeSmallerThanThreshold,
-                                rightSizeSmallerThanThreshold ? false : 
leftSmallerThanRight);
+                isBroadcastJoinTemp = rightSizeSmallerThanThreshold;
+                leftIsBuildTemp = isBroadcastJoinTemp ? false : 
leftSmallerThanRight;
                 break;
             case FULL:
             default:
                 throw new RuntimeException(String.format("Unexpected join type 
%s.", joinType));
         }
 
-        isBroadcastJoin = isBroadcastAndLeftBuild.f0;
-        leftIsBuild = isBroadcastAndLeftBuild.f1;
+        if (isBroadcastJoinTemp && tryTransformEdges.apply(leftIsBuildTemp)) {

Review Comment:
   Comments are needed to explain this logic which is not straight forward for 
others.



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/adaptive/AdaptiveJoin.java:
##########
@@ -51,6 +53,18 @@ public interface AdaptiveJoin extends Serializable {
      *     broadcast hash join, false else. The second element of tuple is 
true if left side is
      *     smaller, false else.
      */
-    Tuple2<Boolean, Boolean> enrichAndCheckBroadcast(
-            long leftInputBytes, long rightInputBytes, long threshold);
+    Tuple2<Boolean, Boolean> tryBroadcastOptimization(
+            Long leftInputBytes,
+            Long rightInputBytes,
+            Long threshold,
+            Function<Boolean, Boolean> tryTransformEdges);

Review Comment:
   It's hard to understand what `tryTransformEdges` means.



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/adaptive/AdaptiveJoinOperatorGenerator.java:
##########
@@ -135,50 +148,60 @@ public StreamOperatorFactory<?> genOperatorFactory(
     }
 
     @Override
-    public Tuple2<Boolean, Boolean> enrichAndCheckBroadcast(
-            long leftInputSize, long rightInputSize, long threshold) {
-        Tuple2<Boolean, Boolean> isBroadcastAndLeftBuild;
+    public Tuple2<Boolean, Boolean> tryBroadcastOptimization(
+            Long leftInputSize,
+            Long rightInputSize,
+            Long threshold,
+            Function<Boolean, Boolean> tryTransformEdges) {
         boolean leftSizeSmallerThanThreshold = leftInputSize <= threshold;
         boolean rightSizeSmallerThanThreshold = rightInputSize <= threshold;
         boolean leftSmallerThanRight = leftInputSize < rightInputSize;
+        boolean isBroadcastJoinTemp;
+        boolean leftIsBuildTemp;
         switch (joinType) {
             case RIGHT:
                 // For a right outer join, if the left side can be broadcast, 
then the left side is
                 // always the build side; otherwise, the smaller side is the 
build side.
-                isBroadcastAndLeftBuild =
-                        new Tuple2<>(
-                                leftSizeSmallerThanThreshold,
-                                leftSizeSmallerThanThreshold ? true : 
leftSmallerThanRight);
+                isBroadcastJoinTemp = leftSizeSmallerThanThreshold;
+                leftIsBuildTemp = isBroadcastJoinTemp ? true : 
leftSmallerThanRight;
                 break;
             case INNER:
-                isBroadcastAndLeftBuild =
-                        new Tuple2<>(
-                                leftSizeSmallerThanThreshold || 
rightSizeSmallerThanThreshold,
-                                leftSmallerThanRight);
+                isBroadcastJoinTemp = leftSizeSmallerThanThreshold || 
rightSizeSmallerThanThreshold;
+                leftIsBuildTemp = leftSmallerThanRight;
                 break;
             case LEFT:
             case SEMI:
             case ANTI:
                 // For left outer / semi / anti join, if the right side can be 
broadcast, then the
                 // right side is always the build side; otherwise, the smaller 
side is the build
                 // side.
-                isBroadcastAndLeftBuild =
-                        new Tuple2<>(
-                                rightSizeSmallerThanThreshold,
-                                rightSizeSmallerThanThreshold ? false : 
leftSmallerThanRight);
+                isBroadcastJoinTemp = rightSizeSmallerThanThreshold;
+                leftIsBuildTemp = isBroadcastJoinTemp ? false : 
leftSmallerThanRight;
                 break;
             case FULL:
             default:
                 throw new RuntimeException(String.format("Unexpected join type 
%s.", joinType));
         }
 
-        isBroadcastJoin = isBroadcastAndLeftBuild.f0;
-        leftIsBuild = isBroadcastAndLeftBuild.f1;
+        if (isBroadcastJoinTemp && tryTransformEdges.apply(leftIsBuildTemp)) {
+            isBroadcastJoin = true;
+            leftIsBuild = leftIsBuildTemp;
+        } else {
+            isBroadcastJoin = false;
+            leftIsBuild = leftSmallerThanRight;
+        }
+
+        return new Tuple2<>(isBroadcastJoin, leftIsBuild);
+    }
+
+    @Override
+    public boolean isLeftBuild() {
         // Sort merge join requires the left side to be read first if the 
broadcast threshold is not
         // met.

Review Comment:
   -> Sort merge join requires the left side to be read first. Therefore, if it 
cannot be converted into a BroadcastHashJoin, we should mark its left side as 
the build side.



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/adaptive/AdaptiveJoinOperatorGenerator.java:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.adaptive;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.streaming.api.operators.AdaptiveJoin;
+import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
+import org.apache.flink.table.planner.plan.utils.HashJoinOperatorUtil;
+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.types.logical.RowType;
+
+/**
+ * Implementation class for {@link AdaptiveJoin}. It can selectively generate 
broadcast hash join,
+ * shuffle hash join or shuffle merge join operator based on actual conditions.
+ */
+public class AdaptiveJoinOperatorGenerator implements AdaptiveJoin {
+
+    private final int[] leftKeys;
+
+    private final int[] rightKeys;
+
+    private final FlinkJoinType joinType;
+
+    private final boolean[] filterNulls;
+
+    private final RowType leftType;
+
+    private final RowType rightType;
+
+    private final GeneratedJoinCondition condFunc;
+
+    private final int leftRowSize;
+
+    private final long leftRowCount;
+
+    private final int rightRowSize;
+
+    private final long rightRowCount;
+
+    private final boolean tryDistinctBuildRow;
+
+    private final long managedMemory;
+
+    private final OperatorType originalJoin;
+
+    private boolean leftIsBuild;
+
+    private boolean isBroadcastJoin;
+
+    public AdaptiveJoinOperatorGenerator(
+            int[] leftKeys,
+            int[] rightKeys,
+            FlinkJoinType joinType,
+            boolean[] filterNulls,
+            RowType leftType,
+            RowType rightType,
+            GeneratedJoinCondition condFunc,
+            int leftRowSize,
+            int rightRowSize,
+            long leftRowCount,
+            long rightRowCount,
+            boolean tryDistinctBuildRow,
+            long managedMemory,
+            OperatorType originalJoin) {
+        this.leftKeys = leftKeys;
+        this.rightKeys = rightKeys;
+        this.joinType = joinType;
+        this.filterNulls = filterNulls;
+        this.leftType = leftType;
+        this.rightType = rightType;
+        this.condFunc = condFunc;
+        this.leftRowSize = leftRowSize;
+        this.rightRowSize = rightRowSize;
+        this.leftRowCount = leftRowCount;
+        this.rightRowCount = rightRowCount;
+        this.tryDistinctBuildRow = tryDistinctBuildRow;
+        this.managedMemory = managedMemory;
+        this.originalJoin = originalJoin;
+    }
+
+    @Override
+    public StreamOperatorFactory<?> genOperatorFactory(
+            ClassLoader classLoader, ReadableConfig config) {
+        if (isBroadcastJoin || originalJoin == OperatorType.ShuffleHashJoin) {
+            return HashJoinOperatorUtil.generateOperatorFactory(
+                    leftKeys,
+                    rightKeys,
+                    joinType,
+                    filterNulls,
+                    leftType,
+                    rightType,
+                    condFunc,
+                    leftIsBuild,
+                    leftRowSize,
+                    rightRowSize,
+                    leftRowCount,
+                    rightRowCount,
+                    tryDistinctBuildRow,
+                    managedMemory,
+                    config,
+                    classLoader);
+        } else {
+            return SorMergeJoinOperatorUtil.generateOperatorFactory(

Review Comment:
   Yes this logic is straight forward. I guess I overlooked something, but I 
cannot remember now.



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/adaptive/AdaptiveJoinOperatorGenerator.java:
##########
@@ -94,6 +98,15 @@ public AdaptiveJoinOperatorGenerator(
         this.rightRowCount = rightRowCount;
         this.tryDistinctBuildRow = tryDistinctBuildRow;
         this.managedMemory = managedMemory;
+        Preconditions.checkState(

Review Comment:
   ` Preconditions.checkState` should be imported in ahead.



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/adaptive/AdaptiveJoinOperatorFactory.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.runtime.operators.join.adaptive;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.AdaptiveJoin;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.table.planner.loader.PlannerModule;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+
+/**
+ * Adaptive join factory.
+ *
+ * <p>Note: This class will hold an {@link AdaptiveJoin} and serve as a proxy 
class to provide an
+ * interface externally. Due to runtime access visibility constraints with the 
table-planner module,
+ * the {@link AdaptiveJoin} object will be serialized during the Table Planner 
phase and will be
+ * lazily deserialized before the dynamic generation of the JobGraph.
+ *
+ * @param <OUT> The output type of the operator
+ */
+@Internal
+public class AdaptiveJoinOperatorFactory<OUT> extends 
AbstractStreamOperatorFactory<OUT>
+        implements AdaptiveJoin {
+    private static final long serialVersionUID = 1L;
+
+    private final byte[] adaptiveJoinSerialized;
+
+    private transient AdaptiveJoin adaptiveJoin;
+
+    private StreamOperatorFactory<OUT> finalFactory;
+
+    public AdaptiveJoinOperatorFactory(byte[] adaptiveJoinSerialized) {
+        this.adaptiveJoinSerialized = adaptiveJoinSerialized;

Review Comment:
   this.adaptiveJoinSerialized = checkNotNull(adaptiveJoinSerialized);
   
   It's better that `Preconditions.checkNotNull` is imported in ahead.



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/adaptive/AdaptiveJoinOperatorGeneratorTest.java:
##########
@@ -0,0 +1,424 @@
+/*
+ * 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
+ * imitations under the License.
+ */
+
+package org.apache.flink.table.planner.adaptive;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.operators.SimpleOperatorFactory;
+import org.apache.flink.table.data.binary.BinaryRowData;
+import org.apache.flink.table.planner.plan.utils.OperatorType;
+import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
+import org.apache.flink.table.runtime.generated.JoinCondition;
+import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import 
org.apache.flink.table.runtime.operators.join.Int2HashJoinOperatorTestBase;
+import org.apache.flink.table.runtime.operators.join.adaptive.AdaptiveJoin;
+import org.apache.flink.table.runtime.util.UniformBinaryRowGenerator;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.MutableObjectIterator;
+
+import org.junit.jupiter.api.Test;
+
+import java.io.Serializable;
+import java.util.function.Function;
+
+import static 
org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_RESOURCE_HASH_JOIN_MEMORY;
+import static org.apache.flink.table.runtime.util.JoinUtil.getJoinType;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Test for {@link AdaptiveJoinOperatorGenerator}. */
+class AdaptiveJoinOperatorGeneratorTest extends Int2HashJoinOperatorTestBase {
+
+    private final Function<Boolean, Boolean> areEdgesCanBeTransformed =
+            new StreamEdgesTransformationChecker();
+
+    // --------- Test if the join operator can be converted to a broadcast 
hash join  -------------
+    @Test
+    void testInnerJoinCheckBroadcast() {
+        AdaptiveJoin adaptiveJoin =
+                genAdaptiveJoin(FlinkJoinType.INNER, 
OperatorType.ShuffleHashJoin);
+        assertThat(adaptiveJoin.tryBroadcastOptimization(2L, 10L, 5L, 
areEdgesCanBeTransformed))
+                .isEqualTo(new Tuple2<>(true, true));
+        assertThat(adaptiveJoin.tryBroadcastOptimization(10L, 2L, 5L, 
areEdgesCanBeTransformed))
+                .isEqualTo(new Tuple2<>(true, false));
+        assertThat(adaptiveJoin.tryBroadcastOptimization(2L, 10L, 1L, 
areEdgesCanBeTransformed))
+                .isEqualTo(new Tuple2<>(false, true));
+        assertThat(adaptiveJoin.tryBroadcastOptimization(10L, 2L, 1L, 
areEdgesCanBeTransformed))
+                .isEqualTo(new Tuple2<>(false, false));
+
+        adaptiveJoin = genAdaptiveJoin(FlinkJoinType.INNER, 
OperatorType.SortMergeJoin);
+        assertThat(adaptiveJoin.tryBroadcastOptimization(2L, 10L, 5L, 
areEdgesCanBeTransformed))
+                .isEqualTo(new Tuple2<>(true, true));
+        assertThat(adaptiveJoin.tryBroadcastOptimization(10L, 2L, 5L, 
areEdgesCanBeTransformed))
+                .isEqualTo(new Tuple2<>(true, false));
+        assertThat(adaptiveJoin.tryBroadcastOptimization(2L, 10L, 1L, 
areEdgesCanBeTransformed))
+                .isEqualTo(new Tuple2<>(false, true));
+        assertThat(adaptiveJoin.tryBroadcastOptimization(10L, 2L, 1L, 
areEdgesCanBeTransformed))
+                .isEqualTo(new Tuple2<>(false, false));
+    }
+
+    @Test
+    void testRightJoinCheckBroadcast() {
+        AdaptiveJoin adaptiveJoin =
+                genAdaptiveJoin(FlinkJoinType.RIGHT, 
OperatorType.ShuffleHashJoin);
+        assertThat(adaptiveJoin.tryBroadcastOptimization(2L, 10L, 5L, 
areEdgesCanBeTransformed))
+                .isEqualTo(new Tuple2<>(true, true));
+        assertThat(adaptiveJoin.tryBroadcastOptimization(10L, 2L, 5L, 
areEdgesCanBeTransformed))
+                .isEqualTo(new Tuple2<>(false, false));
+        assertThat(adaptiveJoin.tryBroadcastOptimization(2L, 10L, 1L, 
areEdgesCanBeTransformed))
+                .isEqualTo(new Tuple2<>(false, true));
+        assertThat(adaptiveJoin.tryBroadcastOptimization(10L, 2L, 1L, 
areEdgesCanBeTransformed))
+                .isEqualTo(new Tuple2<>(false, false));
+
+        adaptiveJoin = genAdaptiveJoin(FlinkJoinType.RIGHT, 
OperatorType.SortMergeJoin);
+        assertThat(adaptiveJoin.tryBroadcastOptimization(2L, 10L, 5L, 
areEdgesCanBeTransformed))
+                .isEqualTo(new Tuple2<>(true, true));
+        assertThat(adaptiveJoin.tryBroadcastOptimization(10L, 2L, 5L, 
areEdgesCanBeTransformed))
+                .isEqualTo(new Tuple2<>(false, false));
+        assertThat(adaptiveJoin.tryBroadcastOptimization(2L, 10L, 1L, 
areEdgesCanBeTransformed))
+                .isEqualTo(new Tuple2<>(false, true));
+        assertThat(adaptiveJoin.tryBroadcastOptimization(10L, 2L, 1L, 
areEdgesCanBeTransformed))
+                .isEqualTo(new Tuple2<>(false, false));
+    }
+
+    @Test
+    void testLeftJoinCheckBroadcast() {
+        testBuildRightCheckBroadcast(FlinkJoinType.LEFT);
+    }
+
+    @Test
+    void testSemiJoinCheckBroadcast() {
+        testBuildRightCheckBroadcast(FlinkJoinType.SEMI);
+    }
+
+    @Test
+    void testAntiJoinCheckBroadcast() {
+        testBuildRightCheckBroadcast(FlinkJoinType.ANTI);
+    }
+
+    @Test
+    void testFullJoinCheckBroadcast() {
+        AdaptiveJoin adaptiveJoin =
+                genAdaptiveJoin(FlinkJoinType.FULL, 
OperatorType.ShuffleHashJoin);
+        assertThatThrownBy(
+                        () ->
+                                adaptiveJoin.tryBroadcastOptimization(
+                                        2L, 10L, 5L, areEdgesCanBeTransformed))
+                .hasMessageContaining("Unexpected join type");
+    }
+
+    void testBuildRightCheckBroadcast(FlinkJoinType joinType) {
+        AdaptiveJoin adaptiveJoin = genAdaptiveJoin(joinType, 
OperatorType.ShuffleHashJoin);
+        assertThat(adaptiveJoin.tryBroadcastOptimization(2L, 10L, 5L, 
areEdgesCanBeTransformed))
+                .isEqualTo(new Tuple2<>(false, true));
+        assertThat(adaptiveJoin.tryBroadcastOptimization(10L, 2L, 5L, 
areEdgesCanBeTransformed))
+                .isEqualTo(new Tuple2<>(true, false));
+        assertThat(adaptiveJoin.tryBroadcastOptimization(2L, 10L, 1L, 
areEdgesCanBeTransformed))
+                .isEqualTo(new Tuple2<>(false, true));
+        assertThat(adaptiveJoin.tryBroadcastOptimization(10L, 2L, 1L, 
areEdgesCanBeTransformed))
+                .isEqualTo(new Tuple2<>(false, false));
+
+        adaptiveJoin = genAdaptiveJoin(joinType, OperatorType.SortMergeJoin);
+        assertThat(adaptiveJoin.tryBroadcastOptimization(2L, 10L, 5L, 
areEdgesCanBeTransformed))
+                .isEqualTo(new Tuple2<>(false, true));
+        assertThat(adaptiveJoin.tryBroadcastOptimization(10L, 2L, 5L, 
areEdgesCanBeTransformed))
+                .isEqualTo(new Tuple2<>(true, false));
+        assertThat(adaptiveJoin.tryBroadcastOptimization(2L, 10L, 1L, 
areEdgesCanBeTransformed))
+                .isEqualTo(new Tuple2<>(false, true));
+        assertThat(adaptiveJoin.tryBroadcastOptimization(10L, 2L, 1L, 
areEdgesCanBeTransformed))
+                .isEqualTo(new Tuple2<>(false, false));
+    }
+
+    // ---------------------- Test the correctness of the generated join 
operator -----------------
+    @Test
+    void testGenerateOperatorCorrectness() {
+        Object sortMergeJoinOperator =
+                newOperator(FlinkJoinType.INNER, true, false, 
OperatorType.SortMergeJoin);
+        
assertThat(sortMergeJoinOperator).isInstanceOf(SimpleOperatorFactory.class);
+
+        Object broadcastHashJoinOperator =
+                newOperator(FlinkJoinType.INNER, true, true, 
OperatorType.SortMergeJoin);
+        
assertThat(broadcastHashJoinOperator).isInstanceOf(CodeGenOperatorFactory.class);
+
+        Object hashJoinOperator =
+                newOperator(FlinkJoinType.INNER, true, false, 
OperatorType.ShuffleHashJoin);
+        
assertThat(hashJoinOperator).isInstanceOf(CodeGenOperatorFactory.class);
+
+        Object broadcastHashJoinOperator2 =
+                newOperator(FlinkJoinType.INNER, true, true, 
OperatorType.ShuffleHashJoin);
+        
assertThat(broadcastHashJoinOperator2).isInstanceOf(CodeGenOperatorFactory.class);
+    }
+
+    @Test
+    void testShuffleHashJoinTransformationCorrectness() throws Exception {
+        OperatorType joinType = OperatorType.ShuffleHashJoin;
+
+        // all cases to ShuffleHashJoin
+        testInnerJoin(true, joinType, false);
+        testInnerJoin(false, joinType, false);
+
+        testLeftOutJoin(true, joinType, false);
+        testLeftOutJoin(false, joinType, false);
+
+        testRightOutJoin(true, joinType, false);
+        testRightOutJoin(false, joinType, false);
+
+        testSemiJoin(joinType, false);
+
+        testAntiJoin(joinType, false);
+
+        // all cases to BroadcastHashJoin
+        testInnerJoin(true, joinType, true);
+        testInnerJoin(false, joinType, true);
+
+        testLeftOutJoin(false, joinType, true);
+
+        testRightOutJoin(true, joinType, true);
+
+        testSemiJoin(joinType, true);
+
+        testAntiJoin(joinType, true);
+    }
+
+    @Test
+    void testSortMergeJoinTransformationCorrectness() throws Exception {
+        OperatorType joinType = OperatorType.SortMergeJoin;
+
+        // all cases to SortMergeJoin
+        testInnerJoin(true, joinType, false);
+        testInnerJoin(true, joinType, true);
+
+        testLeftOutJoin(true, joinType, false);
+
+        testRightOutJoin(true, joinType, false);
+
+        testAntiJoin(joinType, false);
+
+        testAntiJoin(joinType, false);
+
+        // all cases to BroadcastHashJoin
+        testInnerJoin(true, joinType, true);
+        testInnerJoin(false, joinType, true);
+
+        testLeftOutJoin(false, joinType, true);
+
+        testRightOutJoin(true, joinType, true);
+
+        testSemiJoin(joinType, true);
+
+        testAntiJoin(joinType, true);
+    }
+
+    private void testInnerJoin(
+            boolean isBuildLeft, OperatorType originalJoinType, boolean 
isBroadcast)
+            throws Exception {
+        int numKeys = 100;
+        int buildValsPerKey = 3;
+        int probeValsPerKey = 10;
+        MutableObjectIterator<BinaryRowData> buildInput =
+                new UniformBinaryRowGenerator(numKeys, buildValsPerKey, false);
+        MutableObjectIterator<BinaryRowData> probeInput =
+                new UniformBinaryRowGenerator(numKeys, probeValsPerKey, true);
+
+        buildJoin(
+                buildInput,
+                probeInput,
+                originalJoinType,
+                false,
+                false,
+                isBuildLeft,
+                isBroadcast,
+                numKeys * buildValsPerKey * probeValsPerKey,
+                numKeys,
+                165);
+    }
+
+    private void testLeftOutJoin(
+            boolean isBuildLeft, OperatorType originalJoinType, boolean 
isBroadcast)
+            throws Exception {
+        int numKeys1 = 9;
+        int numKeys2 = 10;
+        int buildValsPerKey = 3;
+        int probeValsPerKey = 10;
+        MutableObjectIterator<BinaryRowData> buildInput =
+                new UniformBinaryRowGenerator(
+                        isBuildLeft ? numKeys1 : numKeys2, buildValsPerKey, 
true);
+        MutableObjectIterator<BinaryRowData> probeInput =
+                new UniformBinaryRowGenerator(
+                        isBuildLeft ? numKeys2 : numKeys1, probeValsPerKey, 
true);
+
+        buildJoin(
+                buildInput,
+                probeInput,
+                originalJoinType,
+                true,
+                false,
+                isBuildLeft,
+                isBroadcast,
+                numKeys1 * buildValsPerKey * probeValsPerKey,
+                numKeys1,
+                165);
+    }
+
+    private void testRightOutJoin(
+            boolean isBuildLeft, OperatorType originalJoinType, boolean 
isBroadcast)
+            throws Exception {
+        int numKeys1 = 9;
+        int numKeys2 = 10;
+        int buildValsPerKey = 3;
+        int probeValsPerKey = 10;
+        MutableObjectIterator<BinaryRowData> buildInput =
+                new UniformBinaryRowGenerator(numKeys1, buildValsPerKey, true);
+        MutableObjectIterator<BinaryRowData> probeInput =
+                new UniformBinaryRowGenerator(numKeys2, probeValsPerKey, true);
+
+        buildJoin(
+                buildInput,
+                probeInput,
+                originalJoinType,
+                false,
+                true,
+                isBuildLeft,
+                isBroadcast,
+                isBuildLeft ? 280 : 270,
+                numKeys2,
+                -1);
+    }
+
+    private void testSemiJoin(OperatorType originalJoinType, boolean 
isBroadcast) throws Exception {
+        int numKeys1 = 9;
+        int numKeys2 = 10;
+        int buildValsPerKey = 3;
+        int probeValsPerKey = 10;
+        if (originalJoinType == OperatorType.SortMergeJoin && !isBroadcast) {
+            numKeys1 = 10;
+            numKeys2 = 9;
+            buildValsPerKey = 10;
+            probeValsPerKey = 3;
+        }
+        MutableObjectIterator<BinaryRowData> buildInput =
+                new UniformBinaryRowGenerator(numKeys1, buildValsPerKey, true);
+        MutableObjectIterator<BinaryRowData> probeInput =
+                new UniformBinaryRowGenerator(numKeys2, probeValsPerKey, true);
+
+        Object operator = newOperator(FlinkJoinType.SEMI, false, isBroadcast, 
originalJoinType);
+        joinAndAssert(operator, buildInput, probeInput, 90, 9, 45, true);
+    }
+
+    private void testAntiJoin(OperatorType originalJoinType, boolean 
isBroadcast) throws Exception {
+        int numKeys1 = 9;
+        int numKeys2 = 10;
+        int buildValsPerKey = 3;
+        int probeValsPerKey = 10;
+        if (originalJoinType == OperatorType.SortMergeJoin && !isBroadcast) {
+            numKeys1 = 10;
+            numKeys2 = 9;
+            buildValsPerKey = 10;
+            probeValsPerKey = 3;
+        }
+        MutableObjectIterator<BinaryRowData> buildInput =
+                new UniformBinaryRowGenerator(numKeys1, buildValsPerKey, true);
+        MutableObjectIterator<BinaryRowData> probeInput =
+                new UniformBinaryRowGenerator(numKeys2, probeValsPerKey, true);
+
+        Object operator = newOperator(FlinkJoinType.ANTI, false, isBroadcast, 
originalJoinType);
+        joinAndAssert(operator, buildInput, probeInput, 10, 1, 45, true);
+    }
+
+    public void buildJoin(
+            MutableObjectIterator<BinaryRowData> buildInput,
+            MutableObjectIterator<BinaryRowData> probeInput,
+            OperatorType originalJoinType,
+            boolean leftOut,
+            boolean rightOut,
+            boolean buildLeft,
+            boolean isBroadcast,
+            int expectOutSize,
+            int expectOutKeySize,
+            int expectOutVal)
+            throws Exception {
+        FlinkJoinType flinkJoinType = getJoinType(leftOut, rightOut);
+        Object operator = newOperator(flinkJoinType, buildLeft, isBroadcast, 
originalJoinType);

Review Comment:
   Can we add assertion on the type of this operator?
   From the current test one can hardly tell wether the operator is converted 
into a `BroadcastHashJoin` or not.



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