Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
26 changes: 26 additions & 0 deletions flink-end-to-end-tests/flink-end-to-end-tests-table-api/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -104,4 +104,30 @@ under the License.
</dependency>

</dependencies>

<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-shade-plugin</artifactId>
<executions>
<execution>
<id>JoinWithCustomTypeExample</id>
<phase>package</phase>
<goals>
<goal>shade</goal>
</goals>
<configuration>
<finalName>JoinWithCustomTypeExample</finalName>
<transformers>
<transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
<mainClass>org.apache.flink.table.test.join.JoinWithCustomTypeExample</mainClass>
</transformer>
</transformers>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,76 @@
/*
* 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.test.join;

import org.apache.flink.api.common.RuntimeExecutionMode;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeinfo.Types;
import org.apache.flink.api.java.typeutils.PojoTypeInfo;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink;
import org.apache.flink.table.api.Table;
import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
import org.apache.flink.types.Row;

import java.util.ArrayList;

import static org.apache.flink.table.api.Expressions.$;

/** Example application that tests JoinWithCustomType with Table API. */
public class JoinWithCustomTypeExample {

private static final TypeInformation<Integer> INT = Types.INT;

public static void main(String[] args) throws Exception {
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.setRuntimeMode(RuntimeExecutionMode.BATCH);

StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);

runTest(env, tEnv);
}

private static void runTest(StreamExecutionEnvironment env, StreamTableEnvironment tEnv)
throws Exception {
System.out.println("Running join with custom type test...");

TestClass value = new TestClass();
TypeInformation<TestClass> valueTypeInfo =
new PojoTypeInfo<>(TestClass.class, new ArrayList<>());

Table table1 =
tEnv.fromDataStream(
env.fromData(Row.of(1)).returns(Types.ROW_NAMED(new String[] {"id"}, INT)));

Table table2 =
tEnv.fromDataStream(
env.fromData(Row.of(1, value))
.returns(
Types.ROW_NAMED(
new String[] {"id2", "value"},
INT,
valueTypeInfo)));

tEnv.toDataStream(table1.leftOuterJoin(table2, $("id").isEqual($("id2"))))
.sinkTo(new DiscardingSink<>());

env.execute("joinWithCustomType");
System.out.println("Job joinWithCustomType completed successfully!");
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,22 @@
/*
* 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.test.join;

/** Custom class for testing join with custom type. */
public class TestClass {}
2 changes: 2 additions & 0 deletions flink-end-to-end-tests/run-nightly-tests.sh
Original file line number Diff line number Diff line change
Expand Up @@ -223,6 +223,8 @@ function run_group_2 {
# run_test "PyFlink YARN application on Docker test" "$END_TO_END_DIR/test-scripts/test_pyflink_yarn.sh" "skip_check_exceptions"
# fi

run_test "Flink Table API end-to-end test" "$END_TO_END_DIR/test-scripts/test_table_api.sh"

################################################################################
# Sticky Scheduling
################################################################################
Expand Down
37 changes: 37 additions & 0 deletions flink-end-to-end-tests/test-scripts/test_table_api.sh
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
#!/usr/bin/env bash
################################################################################
# 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.
################################################################################

# End to end test for join with custom type examples. It only verifies that the job can be submitted
# and run correctly.
#
# Usage:
# FLINK_DIR=<flink dir> flink-end-to-end-tests/test-scripts/test_table_api.sh

source "$(dirname "$0")"/common.sh

start_cluster

TEST_PROGRAM_JAR=${END_TO_END_DIR}/flink-end-to-end-tests-table-api/target/JoinWithCustomTypeExample.jar

$FLINK_DIR/bin/flink run -p 1 $TEST_PROGRAM_JAR
EXIT_CODE=$?

stop_cluster

exit $EXIT_CODE
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import java.io.IOException;
import java.io.InputStream;
import java.net.URL;
import java.net.URLClassLoader;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
Expand Down Expand Up @@ -135,7 +136,7 @@ private PlannerModule() {
}
}

public ClassLoader getSubmoduleClassLoader() {
public URLClassLoader getSubmoduleClassLoader() {
return this.submoduleClassLoader;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,30 +20,21 @@
import org.apache.flink.configuration.ReadableConfig;
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.runtime.operators.join.adaptive.AdaptiveJoin;
import org.apache.flink.table.runtime.operators.join.adaptive.AdaptiveJoinGenerator;
import org.apache.flink.table.types.logical.RowType;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import static org.apache.flink.util.Preconditions.checkState;

/**
* Implementation class for {@link AdaptiveJoin}. It can selectively generate broadcast hash join,
* shuffle hash join or shuffle merge join operator based on actual conditions.
* Implementation class for {@link AdaptiveJoinGenerator}. 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 static final Logger LOG = LoggerFactory.getLogger(AdaptiveJoinOperatorGenerator.class);
public class AdaptiveJoinOperatorGenerator implements AdaptiveJoinGenerator {
private final int[] leftKeys;

private final int[] rightKeys;

private final FlinkJoinType joinType;

private final boolean[] filterNulls;

private final RowType leftType;
Expand All @@ -64,18 +55,9 @@ public class AdaptiveJoinOperatorGenerator implements AdaptiveJoin {

private final long managedMemory;

private final OperatorType originalJoin;

private boolean leftIsBuild;

private boolean originalLeftIsBuild;

private boolean isBroadcastJoin;

public AdaptiveJoinOperatorGenerator(
int[] leftKeys,
int[] rightKeys,
FlinkJoinType joinType,
boolean[] filterNulls,
RowType leftType,
RowType rightType,
Expand All @@ -85,12 +67,9 @@ public AdaptiveJoinOperatorGenerator(
long leftRowCount,
long rightRowCount,
boolean tryDistinctBuildRow,
long managedMemory,
boolean leftIsBuild,
OperatorType originalJoin) {
long managedMemory) {
this.leftKeys = leftKeys;
this.rightKeys = rightKeys;
this.joinType = joinType;
this.filterNulls = filterNulls;
this.leftType = leftType;
this.rightType = rightType;
Expand All @@ -101,23 +80,17 @@ public AdaptiveJoinOperatorGenerator(
this.rightRowCount = rightRowCount;
this.tryDistinctBuildRow = tryDistinctBuildRow;
this.managedMemory = managedMemory;
checkState(
originalJoin == OperatorType.ShuffleHashJoin
|| originalJoin == OperatorType.SortMergeJoin,
String.format(
"Adaptive join "
+ "currently only supports adaptive optimization for ShuffleHashJoin and "
+ "SortMergeJoin, not including %s.",
originalJoin.toString()));
this.leftIsBuild = leftIsBuild;
this.originalLeftIsBuild = leftIsBuild;
this.originalJoin = originalJoin;
}

@Override
public StreamOperatorFactory<?> genOperatorFactory(
ClassLoader classLoader, ReadableConfig config) {
if (isBroadcastJoin || originalJoin == OperatorType.ShuffleHashJoin) {
ClassLoader classLoader,
ReadableConfig config,
FlinkJoinType joinType,
boolean originIsSortMergeJoin,
boolean isBroadcastJoin,
boolean leftIsBuild) {
if (isBroadcastJoin || !originIsSortMergeJoin) {
return HashJoinOperatorUtil.generateOperatorFactory(
leftKeys,
rightKeys,
Expand Down Expand Up @@ -150,32 +123,4 @@ public StreamOperatorFactory<?> genOperatorFactory(
classLoader);
}
}

@Override
public FlinkJoinType getJoinType() {
return joinType;
}

@Override
public void markAsBroadcastJoin(boolean canBroadcast, boolean leftIsBuild) {
this.isBroadcastJoin = canBroadcast;
this.leftIsBuild = leftIsBuild;
}

@Override
public boolean shouldReorderInputs() {
// Sort merge join requires the left side to be read first if the broadcast threshold is not
// met.
if (!isBroadcastJoin && originalJoin == OperatorType.SortMergeJoin) {
return false;
}

if (leftIsBuild != originalLeftIsBuild) {
LOG.info(
"The build side of the adaptive join has been updated. Compile phase build side: {}, Runtime build side: {}.",
originalLeftIsBuild ? "left" : "right",
leftIsBuild ? "left" : "right");
}
return !leftIsBuild;
}
}
Loading