zhuzhurk commented on code in PR #25593: URL: https://github.com/apache/flink/pull/25593#discussion_r1871331676
########## 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: checkNotNull ########## 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; + } + + @Override + public StreamOperatorFactory<?> genOperatorFactory( + ClassLoader classLoader, ReadableConfig config) { + checkAndLazyInitialize(); + this.finalFactory = + (StreamOperatorFactory<OUT>) adaptiveJoin.genOperatorFactory(classLoader, config); + return this.finalFactory; + } + + @Override + public Tuple2<Boolean, Boolean> enrichAndCheckBroadcast( + long leftInputSize, long rightInputSize, long threshold) { + checkAndLazyInitialize(); + return adaptiveJoin.enrichAndCheckBroadcast(leftInputSize, rightInputSize, threshold); + } + + private void checkAndLazyInitialize() { + if (this.adaptiveJoin == null) { + lazyInitialize(); + } + } + + @Override + public <T extends StreamOperator<OUT>> T createStreamOperator( + StreamOperatorParameters<OUT> parameters) { + Preconditions.checkNotNull( + finalFactory, + String.format( + "The OperatorFactory of task [%s] have not been initialized.", + parameters.getContainingTask())); + if (finalFactory instanceof AbstractStreamOperatorFactory) { + ((AbstractStreamOperatorFactory<OUT>) finalFactory) + .setProcessingTimeService(processingTimeService); + } + StreamOperator<OUT> operator = finalFactory.createStreamOperator(parameters); + return (T) operator; + } + + @Override + public Class<? extends StreamOperator> getStreamOperatorClass(ClassLoader classLoader) { + return finalFactory.getStreamOperatorClass(classLoader); + } + + private void lazyInitialize() { + if (!tryInitializeAdaptiveJoin(Thread.currentThread().getContextClassLoader())) { Review Comment: In which case do we expect this invocation(using the thread contextClassLoader) to return true? ########## 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; Review Comment: It's better to annotate it with `@Nullable` ########## 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; Review Comment: Should it check that the join type must be either `ShuffleHashJoin` or `SortMergeJoin`? ########## flink-runtime/src/main/java/org/apache/flink/streaming/api/operators/AdaptiveJoin.java: ########## @@ -0,0 +1,56 @@ +/* + * 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.streaming.api.operators; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.ReadableConfig; + +import java.io.Serializable; + +/** + * Interface for implementing an adaptive broadcast join operator. This interface allows a join + * operator to be dynamically optimized to a broadcast join during runtime if a specific input side + * meets the conditions. If not, the join operator can revert to its original implementation. + */ +@Internal +public interface AdaptiveJoin extends Serializable { + + /** + * Generates a StreamOperatorFactory for the join operator using the provided ClassLoader and + * config. + * + * @param classLoader the ClassLoader to be used for loading classes. + * @param config the configuration to be applied for creating the operator factory. + * @return a StreamOperatorFactory instance. + */ + StreamOperatorFactory<?> genOperatorFactory(ClassLoader classLoader, ReadableConfig config); + + /** + * Enrich the input data sizes and checks for broadcast support. + * + * @param leftInputBytes The size of the left input in bytes. + * @param rightInputBytes The size of the right input in bytes. + * @param threshold The threshold for enabling broadcast hash join. + * @return A Tuple2 instance. The first element of tuple is true if join can convert to + * broadcast hash join, false else. The second element of tuple is true if left side is + * smaller, false else. + */ + Tuple2<Boolean, Boolean> enrichAndCheckBroadcast( Review Comment: Will it be invoked multiple times? And each time the input bytes may be recorded in the `AdaptiveJoin`? ########## flink-runtime/src/main/java/org/apache/flink/streaming/api/operators/AdaptiveJoin.java: ########## @@ -0,0 +1,56 @@ +/* + * 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.streaming.api.operators; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.ReadableConfig; + +import java.io.Serializable; + +/** + * Interface for implementing an adaptive broadcast join operator. This interface allows a join + * operator to be dynamically optimized to a broadcast join during runtime if a specific input side + * meets the conditions. If not, the join operator can revert to its original implementation. + */ +@Internal +public interface AdaptiveJoin extends Serializable { Review Comment: This class should be added in a table module. ########## 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; + } + + @Override + public StreamOperatorFactory<?> genOperatorFactory( + ClassLoader classLoader, ReadableConfig config) { + checkAndLazyInitialize(); + this.finalFactory = + (StreamOperatorFactory<OUT>) adaptiveJoin.genOperatorFactory(classLoader, config); + return this.finalFactory; + } + + @Override + public Tuple2<Boolean, Boolean> enrichAndCheckBroadcast( + long leftInputSize, long rightInputSize, long threshold) { + checkAndLazyInitialize(); + return adaptiveJoin.enrichAndCheckBroadcast(leftInputSize, rightInputSize, threshold); + } + + private void checkAndLazyInitialize() { + if (this.adaptiveJoin == null) { + lazyInitialize(); + } + } + + @Override + public <T extends StreamOperator<OUT>> T createStreamOperator( + StreamOperatorParameters<OUT> parameters) { + Preconditions.checkNotNull( + finalFactory, + String.format( + "The OperatorFactory of task [%s] have not been initialized.", + parameters.getContainingTask())); + if (finalFactory instanceof AbstractStreamOperatorFactory) { + ((AbstractStreamOperatorFactory<OUT>) finalFactory) + .setProcessingTimeService(processingTimeService); + } + StreamOperator<OUT> operator = finalFactory.createStreamOperator(parameters); + return (T) operator; + } + + @Override + public Class<? extends StreamOperator> getStreamOperatorClass(ClassLoader classLoader) { + return finalFactory.getStreamOperatorClass(classLoader); Review Comment: Is it possible that this method is invoked before the `finalFactory` is generated? ########## 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( + condFunc, + leftType, + rightType, + leftKeys, + rightKeys, + joinType, + config, + leftIsBuild, + filterNulls, + managedMemory, + classLoader); + } + } + + @Override + public Tuple2<Boolean, Boolean> enrichAndCheckBroadcast( + long leftInputSize, long rightInputSize, long threshold) { + Tuple2<Boolean, Boolean> isBroadcastAndLeftBuild; + boolean leftSizeSmallerThanThreshold = leftInputSize <= threshold; + boolean rightSizeSmallerThanThreshold = rightInputSize <= threshold; + boolean leftSmallerThanRight = leftInputSize < rightInputSize; + 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); + break; + case INNER: + isBroadcastAndLeftBuild = + new Tuple2<>( + leftSizeSmallerThanThreshold || rightSizeSmallerThanThreshold, + 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); + break; + case FULL: + default: + throw new RuntimeException(String.format("Unexpected join type %s.", joinType)); + } + + isBroadcastJoin = isBroadcastAndLeftBuild.f0; + leftIsBuild = isBroadcastAndLeftBuild.f1; + // Sort merge join requires the left side to be read first if the broadcast threshold is not + // met. + if (!isBroadcastJoin && originalJoin == OperatorType.SortMergeJoin) { + return new Tuple2<>(false, true); Review Comment: What would happen if it does not adjust the value here? Looks to me the result reflects which side should be the build side, instead of which side is smaller as explained in the comments? ########## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/adaptive/AdaptiveJoinOperatorGeneratorTest.java: ########## @@ -0,0 +1,383 @@ +/* + * 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.AdaptiveJoin; +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.join.FlinkJoinType; +import org.apache.flink.table.runtime.operators.join.Int2HashJoinOperatorTestBase; +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 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 { + + // --------- 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.enrichAndCheckBroadcast(2, 10, 5)) + .isEqualTo(new Tuple2<>(true, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 5)) + .isEqualTo(new Tuple2<>(true, false)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 1)) + .isEqualTo(new Tuple2<>(false, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 1)) + .isEqualTo(new Tuple2<>(false, false)); + + adaptiveJoin = genAdaptiveJoin(FlinkJoinType.INNER, OperatorType.SortMergeJoin); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 5)) + .isEqualTo(new Tuple2<>(true, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 5)) + .isEqualTo(new Tuple2<>(true, false)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 1)) + .isEqualTo(new Tuple2<>(false, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 1)) + .isEqualTo(new Tuple2<>(false, true)); + } + + @Test + void testRightJoinCheckBroadcast() { + AdaptiveJoin adaptiveJoin = + genAdaptiveJoin(FlinkJoinType.RIGHT, OperatorType.ShuffleHashJoin); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 5)) + .isEqualTo(new Tuple2<>(true, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 5)) + .isEqualTo(new Tuple2<>(false, false)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 1)) + .isEqualTo(new Tuple2<>(false, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 1)) + .isEqualTo(new Tuple2<>(false, false)); + + adaptiveJoin = genAdaptiveJoin(FlinkJoinType.RIGHT, OperatorType.SortMergeJoin); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 5)) + .isEqualTo(new Tuple2<>(true, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 5)) + .isEqualTo(new Tuple2<>(false, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 1)) + .isEqualTo(new Tuple2<>(false, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 1)) + .isEqualTo(new Tuple2<>(false, true)); + } + + @Test + void testLeftJoinCheckBroadcast() { + testBuildRightCheckBroadcast(FlinkJoinType.LEFT); + } + + @Test + void testSemiJoinCheckBroadcast() { + testBuildRightCheckBroadcast(FlinkJoinType.SEMI); + } + + @Test + void testAntiJoinCheckBroadcast() { + testBuildRightCheckBroadcast(FlinkJoinType.ANTI); + } + + @Test + void testFULLJoinCheckBroadcast() { Review Comment: FULL -> Full ########## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/adaptive/AdaptiveJoinOperatorGeneratorTest.java: ########## @@ -0,0 +1,383 @@ +/* + * 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.AdaptiveJoin; +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.join.FlinkJoinType; +import org.apache.flink.table.runtime.operators.join.Int2HashJoinOperatorTestBase; +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 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 { + + // --------- 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.enrichAndCheckBroadcast(2, 10, 5)) + .isEqualTo(new Tuple2<>(true, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 5)) + .isEqualTo(new Tuple2<>(true, false)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 1)) + .isEqualTo(new Tuple2<>(false, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 1)) + .isEqualTo(new Tuple2<>(false, false)); + + adaptiveJoin = genAdaptiveJoin(FlinkJoinType.INNER, OperatorType.SortMergeJoin); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 5)) + .isEqualTo(new Tuple2<>(true, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 5)) + .isEqualTo(new Tuple2<>(true, false)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 1)) + .isEqualTo(new Tuple2<>(false, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 1)) + .isEqualTo(new Tuple2<>(false, true)); + } + + @Test + void testRightJoinCheckBroadcast() { + AdaptiveJoin adaptiveJoin = + genAdaptiveJoin(FlinkJoinType.RIGHT, OperatorType.ShuffleHashJoin); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 5)) + .isEqualTo(new Tuple2<>(true, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 5)) + .isEqualTo(new Tuple2<>(false, false)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 1)) + .isEqualTo(new Tuple2<>(false, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 1)) + .isEqualTo(new Tuple2<>(false, false)); + + adaptiveJoin = genAdaptiveJoin(FlinkJoinType.RIGHT, OperatorType.SortMergeJoin); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 5)) + .isEqualTo(new Tuple2<>(true, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 5)) + .isEqualTo(new Tuple2<>(false, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 1)) + .isEqualTo(new Tuple2<>(false, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 1)) + .isEqualTo(new Tuple2<>(false, true)); + } + + @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.enrichAndCheckBroadcast(2, 10, 5)) + .hasMessageContaining("Unexpected join type"); + } + + void testBuildRightCheckBroadcast(FlinkJoinType joinType) { + AdaptiveJoin adaptiveJoin = genAdaptiveJoin(joinType, OperatorType.ShuffleHashJoin); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 5)) + .isEqualTo(new Tuple2<>(false, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 5)) + .isEqualTo(new Tuple2<>(true, false)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 1)) + .isEqualTo(new Tuple2<>(false, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 1)) + .isEqualTo(new Tuple2<>(false, false)); + + adaptiveJoin = genAdaptiveJoin(joinType, OperatorType.SortMergeJoin); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 5)) + .isEqualTo(new Tuple2<>(false, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 5)) + .isEqualTo(new Tuple2<>(true, false)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 1)) + .isEqualTo(new Tuple2<>(false, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 1)) + .isEqualTo(new Tuple2<>(false, true)); + } + + // ---------------------- Test the correctness of the generated join operator ----------------- + @Test + void testInnerJoinCorrectness() throws Exception { + // Shuffle Hash join + testInnerJoin(true, OperatorType.ShuffleHashJoin, false); + testInnerJoin(false, OperatorType.ShuffleHashJoin, false); + // Sort merge join + testInnerJoin(true, OperatorType.SortMergeJoin, false); + testInnerJoin(true, OperatorType.SortMergeJoin, true); + // Broadcast Hash join + testInnerJoin(true, OperatorType.ShuffleHashJoin, true); + testInnerJoin(false, OperatorType.ShuffleHashJoin, true); Review Comment: Maybe we can re-organize the assertions to make it a bit more clear to understand what joins will be generated in different cases. - For ShuffleHashJoin - all cases to ShuffleHashJoin - all cases to BroadcastHashJoin - ForSortMergeJoin - all cases to SortMergeJoin - all cases to BroadcastHashJoin ########## 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: Why do we convert it to a sort-merge-join if it was originally a shuffled-hash-join which cannot be converted into a broadcast-hash-join? ########## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/adaptive/AdaptiveJoinOperatorGeneratorTest.java: ########## @@ -0,0 +1,383 @@ +/* + * 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.AdaptiveJoin; +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.join.FlinkJoinType; +import org.apache.flink.table.runtime.operators.join.Int2HashJoinOperatorTestBase; +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 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 { + + // --------- 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.enrichAndCheckBroadcast(2, 10, 5)) + .isEqualTo(new Tuple2<>(true, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 5)) + .isEqualTo(new Tuple2<>(true, false)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 1)) + .isEqualTo(new Tuple2<>(false, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 1)) + .isEqualTo(new Tuple2<>(false, false)); + + adaptiveJoin = genAdaptiveJoin(FlinkJoinType.INNER, OperatorType.SortMergeJoin); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 5)) + .isEqualTo(new Tuple2<>(true, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 5)) + .isEqualTo(new Tuple2<>(true, false)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 1)) + .isEqualTo(new Tuple2<>(false, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 1)) + .isEqualTo(new Tuple2<>(false, true)); + } + + @Test + void testRightJoinCheckBroadcast() { + AdaptiveJoin adaptiveJoin = + genAdaptiveJoin(FlinkJoinType.RIGHT, OperatorType.ShuffleHashJoin); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 5)) + .isEqualTo(new Tuple2<>(true, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 5)) + .isEqualTo(new Tuple2<>(false, false)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 1)) + .isEqualTo(new Tuple2<>(false, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 1)) + .isEqualTo(new Tuple2<>(false, false)); + + adaptiveJoin = genAdaptiveJoin(FlinkJoinType.RIGHT, OperatorType.SortMergeJoin); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 5)) + .isEqualTo(new Tuple2<>(true, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 5)) + .isEqualTo(new Tuple2<>(false, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 1)) + .isEqualTo(new Tuple2<>(false, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 1)) + .isEqualTo(new Tuple2<>(false, true)); + } + + @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.enrichAndCheckBroadcast(2, 10, 5)) + .hasMessageContaining("Unexpected join type"); + } + + void testBuildRightCheckBroadcast(FlinkJoinType joinType) { + AdaptiveJoin adaptiveJoin = genAdaptiveJoin(joinType, OperatorType.ShuffleHashJoin); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 5)) + .isEqualTo(new Tuple2<>(false, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 5)) + .isEqualTo(new Tuple2<>(true, false)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 1)) + .isEqualTo(new Tuple2<>(false, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 1)) + .isEqualTo(new Tuple2<>(false, false)); + + adaptiveJoin = genAdaptiveJoin(joinType, OperatorType.SortMergeJoin); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 5)) + .isEqualTo(new Tuple2<>(false, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 5)) + .isEqualTo(new Tuple2<>(true, false)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(2, 10, 1)) + .isEqualTo(new Tuple2<>(false, true)); + assertThat(adaptiveJoin.enrichAndCheckBroadcast(10, 2, 1)) + .isEqualTo(new Tuple2<>(false, true)); + } + + // ---------------------- Test the correctness of the generated join operator ----------------- + @Test + void testInnerJoinCorrectness() throws Exception { + // Shuffle Hash join + testInnerJoin(true, OperatorType.ShuffleHashJoin, false); + testInnerJoin(false, OperatorType.ShuffleHashJoin, false); + // Sort merge join + testInnerJoin(true, OperatorType.SortMergeJoin, false); + testInnerJoin(true, OperatorType.SortMergeJoin, true); + // Broadcast Hash join + testInnerJoin(true, OperatorType.ShuffleHashJoin, true); + testInnerJoin(false, OperatorType.ShuffleHashJoin, true); Review Comment: And maybe also verify the generated operator type? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org