-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/SecurityOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/SecurityOptions.java
index 24ecf2f6eaf1e..97fa7a1717400 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/SecurityOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/SecurityOptions.java
@@ -498,19 +498,20 @@ public static Configuration forProvider(Configuration configuration, String prov
* The standard SSL algorithms to be supported.
*
* More options here -
- * http://docs.oracle.com/javase/8/docs/technotes/guides/security/StandardNames.html#ciphersuites
+ * https://docs.oracle.com/en/java/javase/11/docs/specs/security/standard-names.html#jsse-cipher-suite-names
*/
@Documentation.Section(Documentation.Sections.SECURITY_SSL)
public static final ConfigOption SSL_ALGORITHMS =
key("security.ssl.algorithms")
.stringType()
- .defaultValue("TLS_RSA_WITH_AES_128_CBC_SHA")
+ .defaultValue(
+ "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384")
.withDescription(
Description.builder()
.text(
"The comma separated list of standard SSL algorithms to be supported. Read more %s",
link(
- "http://docs.oracle.com/javase/8/docs/technotes/guides/security/StandardNames.html#ciphersuites",
+ "https://docs.oracle.com/en/java/javase/11/docs/specs/security/standard-names.html#jsse-cipher-suite-names",
"here"))
.build());
diff --git a/flink-core/src/main/java/org/apache/flink/util/SerializedThrowable.java b/flink-core/src/main/java/org/apache/flink/util/SerializedThrowable.java
index ae1ea5e14e159..7937e2de8a68a 100644
--- a/flink-core/src/main/java/org/apache/flink/util/SerializedThrowable.java
+++ b/flink-core/src/main/java/org/apache/flink/util/SerializedThrowable.java
@@ -68,10 +68,14 @@ private SerializedThrowable(Throwable exception, Set alreadySeen) {
if (!(exception instanceof SerializedThrowable)) {
// serialize and memoize the original message
byte[] serialized;
- try {
- serialized = InstantiationUtil.serializeObject(exception);
- } catch (Throwable t) {
- serialized = null;
+ // introduce the synchronization here to avoid deadlock of multi thread serializing
+ // exceptions
+ synchronized (SerializedThrowable.class) {
+ try {
+ serialized = InstantiationUtil.serializeObject(exception);
+ } catch (Throwable t) {
+ serialized = null;
+ }
}
this.serializedException = serialized;
this.cachedException = new WeakReference<>(exception);
@@ -94,7 +98,7 @@ private SerializedThrowable(Throwable exception, Set alreadySeen) {
}
}
// mimic suppressed exceptions
- addAllSuppressed(exception.getSuppressed());
+ this.addAllSuppressed(exception.getSuppressed(), alreadySeen);
} else {
// copy from that serialized throwable
SerializedThrowable other = (SerializedThrowable) exception;
@@ -104,7 +108,7 @@ private SerializedThrowable(Throwable exception, Set alreadySeen) {
this.cachedException = other.cachedException;
this.setStackTrace(other.getStackTrace());
this.initCause(other.getCause());
- this.addAllSuppressed(other.getSuppressed());
+ this.addAllSuppressed(other.getSuppressed(), alreadySeen);
}
}
@@ -141,15 +145,23 @@ public String getFullStringifiedStackTrace() {
return fullStringifiedStackTrace;
}
- private void addAllSuppressed(Throwable[] suppressed) {
+ /**
+ * Add all suppressed exceptions to this exception.
+ *
+ * @param suppressed The suppressed exceptions to add.
+ * @param alreadySeen The set of exceptions that have already been seen.
+ */
+ private void addAllSuppressed(Throwable[] suppressed, Set alreadySeen) {
for (Throwable s : suppressed) {
- SerializedThrowable serializedThrowable;
- if (s instanceof SerializedThrowable) {
- serializedThrowable = (SerializedThrowable) s;
- } else {
- serializedThrowable = new SerializedThrowable(s);
+ if (alreadySeen.add(s)) {
+ SerializedThrowable serializedThrowable;
+ if (s instanceof SerializedThrowable) {
+ serializedThrowable = (SerializedThrowable) s;
+ } else {
+ serializedThrowable = new SerializedThrowable(s);
+ }
+ this.addSuppressed(serializedThrowable);
}
- this.addSuppressed(serializedThrowable);
}
}
diff --git a/flink-core/src/test/java/org/apache/flink/configuration/SecurityOptionsTest.java b/flink-core/src/test/java/org/apache/flink/configuration/SecurityOptionsTest.java
index 7edf67ab8eaef..34ac0160bf63b 100644
--- a/flink-core/src/test/java/org/apache/flink/configuration/SecurityOptionsTest.java
+++ b/flink-core/src/test/java/org/apache/flink/configuration/SecurityOptionsTest.java
@@ -58,4 +58,11 @@ void checkEnableRestSSLAuthentication() {
options.set(SecurityOptions.SSL_REST_AUTHENTICATION_ENABLED, true);
assertThat(SecurityOptions.isRestSSLAuthenticationEnabled(options)).isTrue();
}
+
+ @Test
+ void checkDefaultCipherSuite() {
+ assertThat(SecurityOptions.SSL_ALGORITHMS.defaultValue())
+ .isEqualTo(
+ "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384");
+ }
}
diff --git a/flink-core/src/test/java/org/apache/flink/core/fs/local/LocalFileSystemTest.java b/flink-core/src/test/java/org/apache/flink/core/fs/local/LocalFileSystemTest.java
index dadd7ada77dd4..ffe5bf71c894d 100644
--- a/flink-core/src/test/java/org/apache/flink/core/fs/local/LocalFileSystemTest.java
+++ b/flink-core/src/test/java/org/apache/flink/core/fs/local/LocalFileSystemTest.java
@@ -30,7 +30,6 @@
import org.apache.commons.lang3.RandomStringUtils;
import org.junit.jupiter.api.Disabled;
-import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
@@ -234,7 +233,6 @@ void testRenameNonExistingFile() throws IOException {
}
@Test
- @Tag("FailsInGHAContainerWithRootUser")
@Disabled
void testRenameFileWithNoAccess() throws IOException {
final FileSystem fs = FileSystem.getLocalFileSystem();
diff --git a/flink-core/src/test/java/org/apache/flink/util/FileUtilsTest.java b/flink-core/src/test/java/org/apache/flink/util/FileUtilsTest.java
index 25411483d46c9..0628ace8e0ad7 100644
--- a/flink-core/src/test/java/org/apache/flink/util/FileUtilsTest.java
+++ b/flink-core/src/test/java/org/apache/flink/util/FileUtilsTest.java
@@ -24,7 +24,6 @@
import org.apache.flink.core.testutils.CheckedThread;
import org.apache.flink.testutils.junit.utils.TempDirUtils;
-import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
@@ -128,7 +127,6 @@ void testDeleteNonExistentDirectory() throws Exception {
FileUtils.deleteDirectory(doesNotExist);
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testDeleteProtectedDirectory() throws Exception {
// deleting a write protected file should throw an error
diff --git a/flink-dist/src/main/resources/META-INF/NOTICE b/flink-dist/src/main/resources/META-INF/NOTICE
index 6854033457692..e45b4ccbacdfd 100644
--- a/flink-dist/src/main/resources/META-INF/NOTICE
+++ b/flink-dist/src/main/resources/META-INF/NOTICE
@@ -17,7 +17,7 @@ This project bundles the following dependencies under the Apache Software Licens
- org.apache.commons:commons-math3:3.6.1
- org.apache.commons:commons-text:1.10.0
- org.javassist:javassist:3.24.0-GA
-- org.lz4:lz4-java:1.8.0
+- at.yawk.lz4:lz4-java:1.10.3
- org.objenesis:objenesis:3.4
- org.xerial.snappy:snappy-java:1.1.10.7
- tools.profiler:async-profiler:2.9
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-table-api/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-table-api/pom.xml
index d8a6fbc8cc10d..9bc236942cfea 100644
--- a/flink-end-to-end-tests/flink-end-to-end-tests-table-api/pom.xml
+++ b/flink-end-to-end-tests/flink-end-to-end-tests-table-api/pom.xml
@@ -39,6 +39,13 @@ under the License.
provided
+
+
+ org.scala-lang
+ scala-library
+ 2.13.18
+
+
org.apache.flink
@@ -104,4 +111,30 @@ under the License.
+
+
+
+
+ org.apache.maven.plugins
+ maven-shade-plugin
+
+
+ JoinWithCustomTypeExample
+ package
+
+ shade
+
+
+ JoinWithCustomTypeExample
+
+
+ org.apache.flink.table.test.join.JoinWithCustomTypeExample
+
+
+
+
+
+
+
+
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-table-api/src/main/java/org/apache/flink/table/test/join/JoinWithCustomTypeExample.java b/flink-end-to-end-tests/flink-end-to-end-tests-table-api/src/main/java/org/apache/flink/table/test/join/JoinWithCustomTypeExample.java
new file mode 100644
index 0000000000000..a598a72f91533
--- /dev/null
+++ b/flink-end-to-end-tests/flink-end-to-end-tests-table-api/src/main/java/org/apache/flink/table/test/join/JoinWithCustomTypeExample.java
@@ -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 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 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!");
+ }
+}
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-table-api/src/main/java/org/apache/flink/table/test/join/TestClass.java b/flink-end-to-end-tests/flink-end-to-end-tests-table-api/src/main/java/org/apache/flink/table/test/join/TestClass.java
new file mode 100644
index 0000000000000..dad5872c0fd24
--- /dev/null
+++ b/flink-end-to-end-tests/flink-end-to-end-tests-table-api/src/main/java/org/apache/flink/table/test/join/TestClass.java
@@ -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 {}
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-table-api/src/test/java/org/apache/flink/table/test/join/JoinWithCustomTypeExampleTest.java b/flink-end-to-end-tests/flink-end-to-end-tests-table-api/src/test/java/org/apache/flink/table/test/join/JoinWithCustomTypeExampleTest.java
new file mode 100644
index 0000000000000..0e51c0dbc5f95
--- /dev/null
+++ b/flink-end-to-end-tests/flink-end-to-end-tests-table-api/src/test/java/org/apache/flink/table/test/join/JoinWithCustomTypeExampleTest.java
@@ -0,0 +1,29 @@
+/*
+ * 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.junit.jupiter.api.Test;
+
+public class JoinWithCustomTypeExampleTest {
+
+ @Test
+ void testJoinWithCustomTypeExample() throws Exception {
+ JoinWithCustomTypeExample.main(new String[0]);
+ }
+}
diff --git a/flink-end-to-end-tests/flink-local-recovery-and-allocation-test/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java b/flink-end-to-end-tests/flink-local-recovery-and-allocation-test/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java
index 15bb8c2e7f62c..a0b267612319d 100644
--- a/flink-end-to-end-tests/flink-local-recovery-and-allocation-test/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java
+++ b/flink-end-to-end-tests/flink-local-recovery-and-allocation-test/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java
@@ -419,13 +419,34 @@ private boolean isScheduledToCorrectAllocation(
private static int getJvmPid() throws Exception {
java.lang.management.RuntimeMXBean runtime =
java.lang.management.ManagementFactory.getRuntimeMXBean();
- java.lang.reflect.Field jvm = runtime.getClass().getDeclaredField("jvm");
- jvm.setAccessible(true);
- sun.management.VMManagement mgmt = (sun.management.VMManagement) jvm.get(runtime);
- java.lang.reflect.Method pidMethod = mgmt.getClass().getDeclaredMethod("getProcessId");
- pidMethod.setAccessible(true);
- return (int) (Integer) pidMethod.invoke(mgmt);
+ // Try to use reflection to access sun.management.VMManagement
+ // This avoids compile-time dependency on internal JDK classes
+ try {
+ java.lang.reflect.Field jvm = runtime.getClass().getDeclaredField("jvm");
+ jvm.setAccessible(true);
+ Object mgmt = jvm.get(runtime);
+
+ // Check if the class exists (it may not in some JDK distributions like IBM Semeru)
+ if (mgmt != null) {
+ java.lang.reflect.Method pidMethod =
+ mgmt.getClass().getDeclaredMethod("getProcessId");
+ pidMethod.setAccessible(true);
+ return (int) (Integer) pidMethod.invoke(mgmt);
+ }
+ } catch (NoSuchFieldException | NoSuchMethodException | ClassCastException e) {
+ // Fall through to alternative method
+ }
+
+ // Fallback: parse PID from RuntimeMXBean name (format: "pid@hostname")
+ String jvmName = runtime.getName();
+ int atIndex = jvmName.indexOf('@');
+ if (atIndex > 0) {
+ return Integer.parseInt(jvmName.substring(0, atIndex));
+ }
+
+ throw new UnsupportedOperationException(
+ "Unable to determine JVM PID. This JDK distribution may not support the required internal APIs.");
}
/** Records the information required to check sticky scheduling after a restart. */
diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh
index 92304ef616a0a..fe47e8e0635a7 100755
--- a/flink-end-to-end-tests/run-nightly-tests.sh
+++ b/flink-end-to-end-tests/run-nightly-tests.sh
@@ -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 with custom type end-to-end test" "$END_TO_END_DIR/test-scripts/test_table_api.sh"
+
################################################################################
# Sticky Scheduling
################################################################################
diff --git a/flink-end-to-end-tests/test-scripts/common_docker.sh b/flink-end-to-end-tests/test-scripts/common_docker.sh
index d4ddbec669b4d..d711b228a7d83 100644
--- a/flink-end-to-end-tests/test-scripts/common_docker.sh
+++ b/flink-end-to-end-tests/test-scripts/common_docker.sh
@@ -47,7 +47,7 @@ function build_image() {
local server_pid=$!
echo "Preparing Dockerfiles"
- retry_times_with_exponential_backoff 5 git clone https://github.com/apache/flink-docker.git --branch dev-master --single-branch
+ retry_times_with_exponential_backoff 5 git clone https://github.com/apache/flink-docker.git --branch dev-2.2 --single-branch
local java_version=17
if [[ ${PROFILE} == *"jdk17"* ]]; then
diff --git a/flink-end-to-end-tests/test-scripts/common_kubernetes.sh b/flink-end-to-end-tests/test-scripts/common_kubernetes.sh
index d5599d9ee3c04..190cb46826093 100755
--- a/flink-end-to-end-tests/test-scripts/common_kubernetes.sh
+++ b/flink-end-to-end-tests/test-scripts/common_kubernetes.sh
@@ -24,9 +24,10 @@ CONTAINER_SCRIPTS=${END_TO_END_DIR}/test-scripts/container-scripts
RETRY_COUNT=3
RETRY_BACKOFF_TIME=5
RESULT_HASH="d41d8cd98f00b204e9800998ecf8427e"
-MINIKUBE_VERSION="v1.28.0"
-CRICTL_VERSION="v1.24.2"
-CRI_DOCKERD_VERSION="0.2.3"
+MINIKUBE_VERSION="v1.38.0"
+CRICTL_VERSION="v1.35.0"
+CRI_DOCKERD_VERSION="0.3.24"
+CNI_PLUGINS_VERSION="v1.6.2"
NON_LINUX_ENV_NOTE="****** Please start/stop minikube manually in non-linux environment. ******"
@@ -60,6 +61,12 @@ function setup_kubernetes_for_linux {
# conntrack is required for minikube 1.9 and later
sudo apt-get install conntrack
+ # CNI plugins are required for minikube with the none driver and Kubernetes v1.24+
+ local cni_archive="cni-plugins-linux-${arch}-${CNI_PLUGINS_VERSION}.tgz"
+ retry_download "https://github.com/containernetworking/plugins/releases/download/${CNI_PLUGINS_VERSION}/${cni_archive}"
+ sudo mkdir -p /opt/cni/bin
+ sudo tar zxvf ${cni_archive} -C /opt/cni/bin
+ rm -f ${cni_archive}
# crictl is required for cri-dockerd
local crictl_archive
crictl_archive="crictl-$CRICTL_VERSION-linux-${arch}.tar.gz"
@@ -170,7 +177,7 @@ function stop_kubernetes {
kill $minikube_mount_pid 2> /dev/null
else
echo "Stopping minikube ..."
- stop_command="minikube stop"
+ stop_command="sudo -E minikube stop"
if ! retry_times ${RETRY_COUNT} ${RETRY_BACKOFF_TIME} "${stop_command}"; then
echo "Could not stop minikube. Aborting..."
exit 1
diff --git a/flink-end-to-end-tests/test-scripts/test_confluent_schema_registry.sh b/flink-end-to-end-tests/test-scripts/test_confluent_schema_registry.sh
index 3719266aff8ec..324e3a34c5526 100755
--- a/flink-end-to-end-tests/test-scripts/test_confluent_schema_registry.sh
+++ b/flink-end-to-end-tests/test-scripts/test_confluent_schema_registry.sh
@@ -20,7 +20,7 @@
set -Eeuo pipefail
KAFKA_VERSION="3.2.3"
-CONFLUENT_VERSION="7.2.2"
+CONFLUENT_VERSION="7.2.9"
CONFLUENT_MAJOR_VERSION="7.2"
# Check the Confluent Platform <> Apache Kafka compatibility matrix when updating KAFKA_VERSION
KAFKA_SQL_VERSION="universal"
diff --git a/flink-end-to-end-tests/test-scripts/test_kubernetes_pyflink_application.sh b/flink-end-to-end-tests/test-scripts/test_kubernetes_pyflink_application.sh
index 389ee33b79277..da93a77c71fbc 100755
--- a/flink-end-to-end-tests/test-scripts/test_kubernetes_pyflink_application.sh
+++ b/flink-end-to-end-tests/test-scripts/test_kubernetes_pyflink_application.sh
@@ -80,7 +80,9 @@ cp "${FLINK_PYTHON_DIR}/pyproject.toml" $PYFLINK_DOCKER_DIR/
if [[ -d "dist" ]]; then rm -Rf dist; fi
cd ${PYFLINK_DOCKER_DIR}
echo "FROM ${PURE_FLINK_IMAGE_NAME}" >> Dockerfile
+echo "USER root" >> Dockerfile
echo "RUN apt-get update && apt-get install build-essential -y" >> Dockerfile
+echo "USER flink" >> Dockerfile
echo "COPY lint-python.sh /tmp/lint-python.sh" >> Dockerfile
echo "COPY pyproject.toml /tmp/pyproject.toml" >> Dockerfile
echo "RUN bash /tmp/lint-python.sh -s basic" >> Dockerfile
diff --git a/flink-end-to-end-tests/test-scripts/test_pyflink.sh b/flink-end-to-end-tests/test-scripts/test_pyflink.sh
index 20fd2640553c0..9e98a30bf279f 100755
--- a/flink-end-to-end-tests/test-scripts/test_pyflink.sh
+++ b/flink-end-to-end-tests/test-scripts/test_pyflink.sh
@@ -20,7 +20,7 @@
set -Eeuo pipefail
KAFKA_VERSION="3.2.3"
-CONFLUENT_VERSION="7.2.2"
+CONFLUENT_VERSION="7.2.9"
CONFLUENT_MAJOR_VERSION="7.2"
# Check the Confluent Platform <> Apache Kafka compatibility matrix when updating KAFKA_VERSION
KAFKA_SQL_VERSION="universal"
diff --git a/flink-end-to-end-tests/test-scripts/test_table_api.sh b/flink-end-to-end-tests/test-scripts/test_table_api.sh
new file mode 100755
index 0000000000000..450eaa17e301f
--- /dev/null
+++ b/flink-end-to-end-tests/test-scripts/test_table_api.sh
@@ -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-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
diff --git a/flink-formats/flink-avro-confluent-registry/pom.xml b/flink-formats/flink-avro-confluent-registry/pom.xml
index 010d90fb48ef3..15037ffc824a2 100644
--- a/flink-formats/flink-avro-confluent-registry/pom.xml
+++ b/flink-formats/flink-avro-confluent-registry/pom.xml
@@ -60,6 +60,10 @@ under the License.
org.lz4
lz4-java
+
+ at.yawk.lz4
+ lz4-java
+
io.swagger
swagger-core
diff --git a/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/MetricConfig.java b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/MetricConfig.java
index 2ff0426ad5ae8..17e77a0b1d19e 100644
--- a/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/MetricConfig.java
+++ b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/MetricConfig.java
@@ -22,81 +22,137 @@
import java.util.Properties;
-/** A properties class with added utility method to extract primitives. */
+/**
+ * A properties class with added utility methods to extract primitives.
+ *
+ * Values may be stored as strings via {@link #setProperty(String, String)} or as native Java
+ * types via {@link #put(Object, Object)} (e.g., when Flink's YAML configuration parser stores
+ * Integer, Long, or Boolean values directly). The getter methods handle both representations
+ * transparently.
+ */
@Public
public class MetricConfig extends Properties {
+ /**
+ * Returns the value associated with the given key as a {@code String}.
+ *
+ *
If the value is not a {@link String}, its {@code toString()} representation is returned.
+ *
+ * @param key the hashtable key.
+ * @param defaultValue a default value.
+ * @return the value in this property list with the specified key value as a String.
+ */
public String getString(String key, String defaultValue) {
- return getProperty(key, defaultValue);
+ final Object value = get(key);
+ if (value == null) {
+ return defaultValue;
+ }
+ return value.toString();
}
/**
- * Searches for the property with the specified key in this property list. If the key is not
- * found in this property list, the default property list, and its defaults, recursively, are
- * then checked. The method returns the default value argument if the property is not found.
+ * Returns the value associated with the given key as an {@code int}.
+ *
+ *
If the value is a {@link Number}, its {@code intValue()} is returned directly. Otherwise,
+ * the value's string representation is parsed via {@link Integer#parseInt(String)}.
*
* @param key the hashtable key.
* @param defaultValue a default value.
- * @return the value in this property list with the specified key value parsed as an int.
+ * @return the value in this property list with the specified key value as an int.
*/
public int getInteger(String key, int defaultValue) {
- String argument = getProperty(key, null);
- return argument == null ? defaultValue : Integer.parseInt(argument);
+ final Object value = get(key);
+ if (value == null) {
+ return defaultValue;
+ }
+ if (value instanceof Number) {
+ return ((Number) value).intValue();
+ }
+ return Integer.parseInt(value.toString());
}
/**
- * Searches for the property with the specified key in this property list. If the key is not
- * found in this property list, the default property list, and its defaults, recursively, are
- * then checked. The method returns the default value argument if the property is not found.
+ * Returns the value associated with the given key as a {@code long}.
+ *
+ *
If the value is a {@link Number}, its {@code longValue()} is returned directly. Otherwise,
+ * the value's string representation is parsed via {@link Long#parseLong(String)}.
*
* @param key the hashtable key.
* @param defaultValue a default value.
- * @return the value in this property list with the specified key value parsed as a long.
+ * @return the value in this property list with the specified key value as a long.
*/
public long getLong(String key, long defaultValue) {
- String argument = getProperty(key, null);
- return argument == null ? defaultValue : Long.parseLong(argument);
+ final Object value = get(key);
+ if (value == null) {
+ return defaultValue;
+ }
+ if (value instanceof Number) {
+ return ((Number) value).longValue();
+ }
+ return Long.parseLong(value.toString());
}
/**
- * Searches for the property with the specified key in this property list. If the key is not
- * found in this property list, the default property list, and its defaults, recursively, are
- * then checked. The method returns the default value argument if the property is not found.
+ * Returns the value associated with the given key as a {@code float}.
+ *
+ *
If the value is a {@link Number}, its {@code floatValue()} is returned directly.
+ * Otherwise, the value's string representation is parsed via {@link Float#parseFloat(String)}.
*
* @param key the hashtable key.
* @param defaultValue a default value.
- * @return the value in this property list with the specified key value parsed as a float.
+ * @return the value in this property list with the specified key value as a float.
*/
public float getFloat(String key, float defaultValue) {
- String argument = getProperty(key, null);
- return argument == null ? defaultValue : Float.parseFloat(argument);
+ final Object value = get(key);
+ if (value == null) {
+ return defaultValue;
+ }
+ if (value instanceof Number) {
+ return ((Number) value).floatValue();
+ }
+ return Float.parseFloat(value.toString());
}
/**
- * Searches for the property with the specified key in this property list. If the key is not
- * found in this property list, the default property list, and its defaults, recursively, are
- * then checked. The method returns the default value argument if the property is not found.
+ * Returns the value associated with the given key as a {@code double}.
+ *
+ *
If the value is a {@link Number}, its {@code doubleValue()} is returned directly.
+ * Otherwise, the value's string representation is parsed via {@link
+ * Double#parseDouble(String)}.
*
* @param key the hashtable key.
* @param defaultValue a default value.
- * @return the value in this property list with the specified key value parsed as a double.
+ * @return the value in this property list with the specified key value as a double.
*/
public double getDouble(String key, double defaultValue) {
- String argument = getProperty(key, null);
- return argument == null ? defaultValue : Double.parseDouble(argument);
+ final Object value = get(key);
+ if (value == null) {
+ return defaultValue;
+ }
+ if (value instanceof Number) {
+ return ((Number) value).doubleValue();
+ }
+ return Double.parseDouble(value.toString());
}
/**
- * Searches for the property with the specified key in this property list. If the key is not
- * found in this property list, the default property list, and its defaults, recursively, are
- * then checked. The method returns the default value argument if the property is not found.
+ * Returns the value associated with the given key as a {@code boolean}.
+ *
+ *
If the value is a {@link Boolean}, it is returned directly. Otherwise, the value's string
+ * representation is parsed via {@link Boolean#parseBoolean(String)}.
*
* @param key the hashtable key.
* @param defaultValue a default value.
- * @return the value in this property list with the specified key value parsed as a boolean.
+ * @return the value in this property list with the specified key value as a boolean.
*/
public boolean getBoolean(String key, boolean defaultValue) {
- String argument = getProperty(key, null);
- return argument == null ? defaultValue : Boolean.parseBoolean(argument);
+ final Object value = get(key);
+ if (value == null) {
+ return defaultValue;
+ }
+ if (value instanceof Boolean) {
+ return (Boolean) value;
+ }
+ return Boolean.parseBoolean(value.toString());
}
}
diff --git a/flink-metrics/flink-metrics-core/src/test/java/org/apache/flink/metrics/MetricConfigTest.java b/flink-metrics/flink-metrics-core/src/test/java/org/apache/flink/metrics/MetricConfigTest.java
new file mode 100644
index 0000000000000..2824328c6a5cd
--- /dev/null
+++ b/flink-metrics/flink-metrics-core/src/test/java/org/apache/flink/metrics/MetricConfigTest.java
@@ -0,0 +1,121 @@
+/*
+ * 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.metrics;
+
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for {@link MetricConfig}. */
+class MetricConfigTest {
+
+ @ParameterizedTest
+ @MethodSource("fromStringCases")
+ void testGetFromString(
+ final String storedValue, final TypedGetter getter, final Object expected) {
+ final MetricConfig config = new MetricConfig();
+ config.setProperty("key", storedValue);
+ assertThat(getter.get(config, "key")).isEqualTo(expected);
+ }
+
+ @ParameterizedTest
+ @MethodSource("nativeTypeCases")
+ void testGetFromNativeType(
+ final Object storedValue, final TypedGetter getter, final Object expected) {
+ final MetricConfig config = new MetricConfig();
+ config.put("key", storedValue);
+ assertThat(getter.get(config, "key")).isEqualTo(expected);
+ }
+
+ @ParameterizedTest
+ @MethodSource("crossTypeCases")
+ void testGetCrossType(
+ final Object storedValue, final TypedGetter getter, final Object expected) {
+ final MetricConfig config = new MetricConfig();
+ config.put("key", storedValue);
+ assertThat(getter.get(config, "key")).isEqualTo(expected);
+ }
+
+ @ParameterizedTest
+ @MethodSource("defaultValueCases")
+ void testGetDefault(final TypedGetter getter, final Object expected) {
+ final MetricConfig config = new MetricConfig();
+ assertThat(getter.get(config, "missing")).isEqualTo(expected);
+ }
+
+ @FunctionalInterface
+ private interface TypedGetter {
+ Object get(MetricConfig config, String key);
+ }
+
+ private static Stream fromStringCases() {
+ return Stream.of(
+ Arguments.of("42", (TypedGetter) (c, k) -> c.getInteger(k, 0), 42),
+ Arguments.of(
+ "123456789012345",
+ (TypedGetter) (c, k) -> c.getLong(k, 0L),
+ 123456789012345L),
+ Arguments.of("3.14", (TypedGetter) (c, k) -> c.getFloat(k, 0.0f), 3.14f),
+ Arguments.of(
+ "2.718281828", (TypedGetter) (c, k) -> c.getDouble(k, 0.0), 2.718281828),
+ Arguments.of("true", (TypedGetter) (c, k) -> c.getBoolean(k, false), true));
+ }
+
+ private static Stream nativeTypeCases() {
+ return Stream.of(
+ Arguments.of(42, (TypedGetter) (c, k) -> c.getInteger(k, 0), 42),
+ Arguments.of(
+ 123456789012345L,
+ (TypedGetter) (c, k) -> c.getLong(k, 0L),
+ 123456789012345L),
+ Arguments.of(3.14f, (TypedGetter) (c, k) -> c.getFloat(k, 0.0f), 3.14f),
+ Arguments.of(2.718281828, (TypedGetter) (c, k) -> c.getDouble(k, 0.0), 2.718281828),
+ Arguments.of(true, (TypedGetter) (c, k) -> c.getBoolean(k, false), true),
+ Arguments.of(42, (TypedGetter) (c, k) -> c.getString(k, "default"), "42"),
+ Arguments.of(
+ 123456789012345L,
+ (TypedGetter) (c, k) -> c.getString(k, "default"),
+ "123456789012345"),
+ Arguments.of(3.14f, (TypedGetter) (c, k) -> c.getString(k, "default"), "3.14"),
+ Arguments.of(true, (TypedGetter) (c, k) -> c.getString(k, "default"), "true"));
+ }
+
+ private static Stream crossTypeCases() {
+ return Stream.of(
+ Arguments.of(42, (TypedGetter) (c, k) -> c.getLong(k, 0L), 42L),
+ Arguments.of(100L, (TypedGetter) (c, k) -> c.getInteger(k, 0), 100),
+ Arguments.of(42, (TypedGetter) (c, k) -> c.getDouble(k, 0.0), 42.0),
+ Arguments.of(42, (TypedGetter) (c, k) -> c.getFloat(k, 0.0f), 42.0f));
+ }
+
+ private static Stream defaultValueCases() {
+ return Stream.of(
+ Arguments.of((TypedGetter) (c, k) -> c.getInteger(k, 99), 99),
+ Arguments.of((TypedGetter) (c, k) -> c.getLong(k, 99L), 99L),
+ Arguments.of((TypedGetter) (c, k) -> c.getFloat(k, 1.5f), 1.5f),
+ Arguments.of((TypedGetter) (c, k) -> c.getDouble(k, 1.5), 1.5),
+ Arguments.of((TypedGetter) (c, k) -> c.getBoolean(k, true), true),
+ Arguments.of((TypedGetter) (c, k) -> c.getBoolean(k, false), false),
+ Arguments.of((TypedGetter) (c, k) -> c.getString(k, "default"), "default"));
+ }
+}
diff --git a/flink-models/flink-model-openai/pom.xml b/flink-models/flink-model-openai/pom.xml
index f251b2e0e266f..f68d82db40d71 100644
--- a/flink-models/flink-model-openai/pom.xml
+++ b/flink-models/flink-model-openai/pom.xml
@@ -76,6 +76,7 @@ under the License.
com.knuddels
jtokkit
1.1.0
+ ${flink.markBundledAsOptional}
diff --git a/flink-models/flink-model-openai/src/main/resources/META-INF/NOTICE b/flink-models/flink-model-openai/src/main/resources/META-INF/NOTICE
index c8908682f8865..6273d7f94f1e1 100644
--- a/flink-models/flink-model-openai/src/main/resources/META-INF/NOTICE
+++ b/flink-models/flink-model-openai/src/main/resources/META-INF/NOTICE
@@ -29,3 +29,7 @@ This project bundles the following dependencies under the Apache Software Licens
- org.jetbrains.kotlin:kotlin-stdlib-common:1.9.10
- org.jetbrains.kotlin:kotlin-stdlib-jdk7:1.9.10
- org.jetbrains.kotlin:kotlin-stdlib-jdk8:1.9.10
+
+This project bundles the following dependencies under the MIT (https://opensource.org/licenses/MIT)
+
+- com.knuddels:jtokkit:1.1.0
diff --git a/flink-python/dev/build-wheels.sh b/flink-python/dev/build-wheels.sh
index 39180452450cd..1b9eb38cab3f4 100755
--- a/flink-python/dev/build-wheels.sh
+++ b/flink-python/dev/build-wheels.sh
@@ -43,7 +43,7 @@ if [[ "$(uname)" != "Darwin" ]]; then
echo "Converting linux_x86_64 wheel to manylinux1"
source `pwd`/dev/.uv/bin/activate
# 4.1 install patchelf and auditwheel
- uv pip install patchelf==0.17.2.1 auditwheel==3.2.0
+ uv pip install patchelf==0.17.2.1 auditwheel==6.6.0
# 4.2 convert Linux wheel
for wheel_file in dist/*.whl; do
auditwheel repair ${wheel_file} -w dist
diff --git a/flink-python/docs/_templates/layout.html b/flink-python/docs/_templates/layout.html
index 796d06174e938..6648a73588146 100644
--- a/flink-python/docs/_templates/layout.html
+++ b/flink-python/docs/_templates/layout.html
@@ -21,4 +21,23 @@
{{ super() }}
+
+
+
{% endblock %}
diff --git a/flink-python/docs/conf.py b/flink-python/docs/conf.py
index c467c0319bbdf..26ed6541a41e4 100644
--- a/flink-python/docs/conf.py
+++ b/flink-python/docs/conf.py
@@ -124,6 +124,9 @@
"navigation_depth": 0
}
+# Base URL for canonical links, pointing to stable docs version
+html_baseurl = "https://nightlies.apache.org/flink/flink-docs-stable/api/python/"
+
# The name of an image file (relative to this directory) to place at the top
# of the sidebar.
html_logo = "../../docs/static/navbar-brand-logo.jpg"
diff --git a/flink-python/pom.xml b/flink-python/pom.xml
index 6ab12545090de..e6d224579678a 100644
--- a/flink-python/pom.xml
+++ b/flink-python/pom.xml
@@ -133,7 +133,7 @@ under the License.
com.alibaba
pemja
- 0.5.5
+ 0.5.7
${flink.markBundledAsOptional}
diff --git a/flink-python/pyflink/datastream/formats/csv.py b/flink-python/pyflink/datastream/formats/csv.py
index 53f85a15441b4..d7da4a03afd76 100644
--- a/flink-python/pyflink/datastream/formats/csv.py
+++ b/flink-python/pyflink/datastream/formats/csv.py
@@ -237,6 +237,7 @@ def set_null_value(self, null_value: str):
Set literal for null value, default to empty sequence.
"""
self._j_schema_builder.setNullValue(null_value)
+ return self
def disable_quote_char(self):
"""
diff --git a/flink-python/pyflink/datastream/formats/tests/test_csv.py b/flink-python/pyflink/datastream/formats/tests/test_csv.py
index 97d32826020c2..a088cfb502ca9 100644
--- a/flink-python/pyflink/datastream/formats/tests/test_csv.py
+++ b/flink-python/pyflink/datastream/formats/tests/test_csv.py
@@ -77,6 +77,12 @@ def test_csv_strict_headers(self):
self.env.execute('test_csv_strict_headers')
_check_csv_strict_headers_results(self, self.test_sink.get_results(True, False))
+ def test_csv_default_null_value(self):
+ schema, lines = _create_csv_default_null_value_schema_and_lines()
+ self._build_csv_job(schema, lines)
+ self.env.execute('test_csv_default_null_value')
+ _check_csv_default_null_value_results(self, self.test_sink.get_results(True, False))
+
def test_csv_default_quote_char(self):
schema, lines = _create_csv_default_quote_char_schema_and_lines()
self._build_csv_job(schema, lines)
@@ -344,6 +350,24 @@ def _check_csv_use_header_results(test, results):
test.assertEqual(row['number'], 123)
+def _create_csv_default_null_value_schema_and_lines() -> Tuple[CsvSchema, List[str]]:
+ schema = CsvSchema.builder() \
+ .add_string_column('string') \
+ .add_number_column('number') \
+ .set_null_value('') \
+ .build()
+ lines = [
+ ',123\n'
+ ]
+ return schema, lines
+
+
+def _check_csv_default_null_value_results(test, results):
+ row = results[0]
+ test.assertEqual(row['string'], None)
+ test.assertEqual(row['number'], 123)
+
+
def _create_csv_strict_headers_schema_and_lines() -> Tuple[CsvSchema, List[str]]:
schema = CsvSchema.builder() \
.add_string_column('string') \
diff --git a/flink-python/pyflink/examples/datastream/connectors/kafka_avro_format.py b/flink-python/pyflink/examples/datastream/connectors/kafka_avro_format.py
index 2a66bc4e8b1d7..e6256d85493eb 100644
--- a/flink-python/pyflink/examples/datastream/connectors/kafka_avro_format.py
+++ b/flink-python/pyflink/examples/datastream/connectors/kafka_avro_format.py
@@ -18,9 +18,10 @@
import logging
import sys
-from pyflink.common import Types
+from pyflink.common import Types, WatermarkStrategy
from pyflink.datastream import StreamExecutionEnvironment
-from pyflink.datastream.connectors.kafka import FlinkKafkaProducer, FlinkKafkaConsumer
+from pyflink.datastream.connectors.kafka import (KafkaRecordSerializationSchema, KafkaSink,
+ KafkaSource, KafkaOffsetsInitializer)
from pyflink.datastream.formats.avro import AvroRowSerializationSchema, AvroRowDeserializationSchema
@@ -43,14 +44,20 @@ def write_to_kafka(env):
}"""
)
- kafka_producer = FlinkKafkaProducer(
- topic='test_avro_topic',
- serialization_schema=serialization_schema,
- producer_config={'bootstrap.servers': 'localhost:9092', 'group.id': 'test_group'}
+ record_serializer = KafkaRecordSerializationSchema.builder() \
+ .set_topic('test_avro_topic') \
+ .set_value_serialization_schema(serialization_schema) \
+ .build()
+ kafka_sink = (
+ KafkaSink.builder()
+ .set_record_serializer(record_serializer)
+ .set_bootstrap_servers('localhost:9092')
+ .set_property("group.id", "test_group")
+ .build()
)
# note that the output type of ds must be RowTypeInfo
- ds.add_sink(kafka_producer)
+ ds.sink_to(kafka_sink)
env.execute()
@@ -67,14 +74,22 @@ def read_from_kafka(env):
}"""
)
- kafka_consumer = FlinkKafkaConsumer(
- topics='test_avro_topic',
- deserialization_schema=deserialization_schema,
- properties={'bootstrap.servers': 'localhost:9092', 'group.id': 'test_group_1'}
+ kafka_source = (
+ KafkaSource.builder()
+ .set_topics('test_avro_topic')
+ .set_value_only_deserializer(deserialization_schema)
+ .set_properties({'bootstrap.servers': 'localhost:9092', 'group.id': 'test_group_1'})
+ .set_starting_offsets(KafkaOffsetsInitializer.earliest())
+ .build()
)
- kafka_consumer.set_start_from_earliest()
- env.add_source(kafka_consumer).print()
+ ds = env.from_source(
+ kafka_source,
+ watermark_strategy=WatermarkStrategy.no_watermarks(),
+ source_name="kafka source"
+ )
+
+ ds.print()
env.execute()
diff --git a/flink-python/pyflink/examples/datastream/connectors/kafka_bytearray_format.py b/flink-python/pyflink/examples/datastream/connectors/kafka_bytearray_format.py
new file mode 100644
index 0000000000000..d2e311cd2e658
--- /dev/null
+++ b/flink-python/pyflink/examples/datastream/connectors/kafka_bytearray_format.py
@@ -0,0 +1,102 @@
+################################################################################
+# 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.
+################################################################################
+import json
+import logging
+import sys
+
+from pyflink.common import Types, ByteArraySchema, WatermarkStrategy
+from pyflink.datastream import StreamExecutionEnvironment
+from pyflink.datastream.connectors.kafka import KafkaSource, \
+ KafkaOffsetsInitializer, KafkaSink, KafkaRecordSerializationSchema
+
+
+# This example works since Flink 2.0 since ByteArraySchema was introduced in Flink 2.0
+
+# Make sure that the Kafka cluster is started and the topic 'test_json_topic' is
+# created before executing this job.
+def write_to_kafka(env):
+ data = [
+ (json.dumps({
+ "id": 1,
+ "country": "USA"
+ }).encode("utf-8"),),
+ (json.dumps({
+ "id": 2,
+ "country": "Canada"
+ }).encode("utf-8"),),
+ (json.dumps({
+ "id": 3,
+ "country": "Germany"
+ }).encode("utf-8"),)
+ ]
+ type_info = Types.ROW([Types.PRIMITIVE_ARRAY(Types.BYTE())])
+ ds = env.from_collection(data, type_info=type_info)
+
+ # declare the output type as Types.PRIMITIVE_ARRAY(Types.BYTE()),
+ # otherwise, Types.PICKLED_BYTE_ARRAY() will be used by default, it will
+ # use pickler to serialize the result byte array which is unnecessary
+ ds = ds.map(lambda x: x[0], output_type=Types.PRIMITIVE_ARRAY(Types.BYTE()))
+
+ record_serializer = KafkaRecordSerializationSchema.builder() \
+ .set_topic('test_bytearray_topic') \
+ .set_value_serialization_schema(ByteArraySchema()) \
+ .build()
+ kafka_sink = (
+ KafkaSink.builder()
+ .set_record_serializer(record_serializer)
+ .set_bootstrap_servers('localhost:9092')
+ .set_property("group.id", "test_group")
+ .build()
+ )
+
+ ds.sink_to(kafka_sink)
+ env.execute()
+
+
+def read_from_kafka(env):
+ kafka_source = (
+ KafkaSource.builder()
+ .set_topics('test_bytearray_topic')
+ .set_value_only_deserializer(ByteArraySchema())
+ .set_properties({'bootstrap.servers': 'localhost:9092', 'group.id': 'test_group_1'})
+ .set_starting_offsets(KafkaOffsetsInitializer.earliest())
+ .build()
+ )
+
+ ds = env.from_source(
+ kafka_source,
+ watermark_strategy=WatermarkStrategy.no_watermarks(),
+ source_name="kafka source"
+ )
+
+ # the data read out from the source is byte array, decode it as a string
+ ds.map(lambda data: data.decode("utf-8")).print()
+ env.execute()
+
+
+if __name__ == '__main__':
+ logging.basicConfig(stream=sys.stdout, level=logging.INFO, format="%(message)s")
+
+ env = StreamExecutionEnvironment.get_execution_environment()
+ env.add_jars("file:///path/to/flink-sql-connector-kafka-1.15.0.jar")
+
+ print("start writing data to kafka")
+ write_to_kafka(env)
+
+ print("start reading data from kafka")
+ read_from_kafka(env)
diff --git a/flink-python/pyflink/examples/datastream/connectors/kafka_csv_format.py b/flink-python/pyflink/examples/datastream/connectors/kafka_csv_format.py
index 39c134a8ed336..3fca6240e6660 100644
--- a/flink-python/pyflink/examples/datastream/connectors/kafka_csv_format.py
+++ b/flink-python/pyflink/examples/datastream/connectors/kafka_csv_format.py
@@ -18,9 +18,10 @@
import logging
import sys
-from pyflink.common import Types
+from pyflink.common import Types, WatermarkStrategy
from pyflink.datastream import StreamExecutionEnvironment
-from pyflink.datastream.connectors.kafka import FlinkKafkaProducer, FlinkKafkaConsumer
+from pyflink.datastream.connectors.kafka import (KafkaRecordSerializationSchema, KafkaSink,
+ KafkaSource, KafkaOffsetsInitializer)
from pyflink.datastream.formats.csv import CsvRowSerializationSchema, CsvRowDeserializationSchema
@@ -33,14 +34,20 @@ def write_to_kafka(env):
type_info=type_info)
serialization_schema = CsvRowSerializationSchema.Builder(type_info).build()
- kafka_producer = FlinkKafkaProducer(
- topic='test_csv_topic',
- serialization_schema=serialization_schema,
- producer_config={'bootstrap.servers': 'localhost:9092', 'group.id': 'test_group'}
+ record_serializer = KafkaRecordSerializationSchema.builder() \
+ .set_topic('test_csv_topic') \
+ .set_value_serialization_schema(serialization_schema) \
+ .build()
+ kafka_sink = (
+ KafkaSink.builder()
+ .set_record_serializer(record_serializer)
+ .set_bootstrap_servers('localhost:9092')
+ .set_property("group.id", "test_group")
+ .build()
)
# note that the output type of ds must be RowTypeInfo
- ds.add_sink(kafka_producer)
+ ds.sink_to(kafka_sink)
env.execute()
@@ -48,14 +55,22 @@ def read_from_kafka(env):
type_info = Types.ROW([Types.INT(), Types.STRING()])
deserialization_schema = CsvRowDeserializationSchema.Builder(type_info).build()
- kafka_consumer = FlinkKafkaConsumer(
- topics='test_csv_topic',
- deserialization_schema=deserialization_schema,
- properties={'bootstrap.servers': 'localhost:9092', 'group.id': 'test_group_1'}
+ kafka_source = (
+ KafkaSource.builder()
+ .set_topics('test_csv_topic')
+ .set_value_only_deserializer(deserialization_schema)
+ .set_properties({'bootstrap.servers': 'localhost:9092', 'group.id': 'test_group_1'})
+ .set_starting_offsets(KafkaOffsetsInitializer.earliest())
+ .build()
)
- kafka_consumer.set_start_from_earliest()
- env.add_source(kafka_consumer).print()
+ ds = env.from_source(
+ kafka_source,
+ watermark_strategy=WatermarkStrategy.no_watermarks(),
+ source_name="kafka source"
+ )
+
+ ds.print()
env.execute()
diff --git a/flink-python/pyflink/examples/datastream/connectors/kafka_json_format.py b/flink-python/pyflink/examples/datastream/connectors/kafka_json_format.py
index 3cae241ba4316..25480b95cef9c 100644
--- a/flink-python/pyflink/examples/datastream/connectors/kafka_json_format.py
+++ b/flink-python/pyflink/examples/datastream/connectors/kafka_json_format.py
@@ -18,9 +18,10 @@
import logging
import sys
-from pyflink.common import Types
+from pyflink.common import Types, WatermarkStrategy
from pyflink.datastream import StreamExecutionEnvironment
-from pyflink.datastream.connectors.kafka import FlinkKafkaProducer, FlinkKafkaConsumer
+from pyflink.datastream.connectors.kafka import (KafkaRecordSerializationSchema, KafkaSink,
+ KafkaSource, KafkaOffsetsInitializer)
from pyflink.datastream.formats.json import JsonRowSerializationSchema, JsonRowDeserializationSchema
@@ -35,14 +36,20 @@ def write_to_kafka(env):
serialization_schema = JsonRowSerializationSchema.Builder() \
.with_type_info(type_info) \
.build()
- kafka_producer = FlinkKafkaProducer(
- topic='test_json_topic',
- serialization_schema=serialization_schema,
- producer_config={'bootstrap.servers': 'localhost:9092', 'group.id': 'test_group'}
+ record_serializer = KafkaRecordSerializationSchema.builder() \
+ .set_topic('test_json_topic') \
+ .set_value_serialization_schema(serialization_schema) \
+ .build()
+ kafka_sink = (
+ KafkaSink.builder()
+ .set_record_serializer(record_serializer)
+ .set_bootstrap_servers('localhost:9092')
+ .set_property("group.id", "test_group")
+ .build()
)
# note that the output type of ds must be RowTypeInfo
- ds.add_sink(kafka_producer)
+ ds.sink_to(kafka_sink)
env.execute()
@@ -50,14 +57,22 @@ def read_from_kafka(env):
deserialization_schema = JsonRowDeserializationSchema.Builder() \
.type_info(Types.ROW([Types.INT(), Types.STRING()])) \
.build()
- kafka_consumer = FlinkKafkaConsumer(
- topics='test_json_topic',
- deserialization_schema=deserialization_schema,
- properties={'bootstrap.servers': 'localhost:9092', 'group.id': 'test_group_1'}
+ kafka_source = (
+ KafkaSource.builder()
+ .set_topics('test_json_topic')
+ .set_value_only_deserializer(deserialization_schema)
+ .set_properties({'bootstrap.servers': 'localhost:9092', 'group.id': 'test_group_1'})
+ .set_starting_offsets(KafkaOffsetsInitializer.earliest())
+ .build()
+ )
+
+ ds = env.from_source(
+ kafka_source,
+ watermark_strategy=WatermarkStrategy.no_watermarks(),
+ source_name="kafka source"
)
- kafka_consumer.set_start_from_earliest()
- env.add_source(kafka_consumer).print()
+ ds.print()
env.execute()
diff --git a/flink-python/pyflink/gen_protos.py b/flink-python/pyflink/gen_protos.py
index 021023616cd3e..6bee23ee713f7 100644
--- a/flink-python/pyflink/gen_protos.py
+++ b/flink-python/pyflink/gen_protos.py
@@ -30,7 +30,9 @@
import time
import warnings
-import pkg_resources
+from importlib import metadata as importlib_metadata
+from importlib import resources as importlib_resources
+from packaging.version import parse as parse_version
GRPC_TOOLS = 'grpcio-tools>=1.29.0,<=1.71.0'
PROTO_PATHS = ['proto']
@@ -87,7 +89,9 @@ def generate_proto_files(force=True, output_dir=DEFAULT_PYTHON_OUTPUT_PATH):
else:
_check_grpcio_tools_version()
logging.info('Regenerating out-of-date Python proto definitions.')
- builtin_protos = pkg_resources.resource_filename('grpc_tools', '_proto')
+ # Get the grpc_tools _proto directory path
+ grpc_tools_files = importlib_resources.files('grpc_tools')
+ builtin_protos = str(grpc_tools_files.joinpath('_proto'))
args = (
[sys.executable] + # expecting to be called from command line
['--proto_path=%s' % builtin_protos] +
@@ -122,8 +126,7 @@ def _install_grpcio_tools_and_generate_proto_files(force, output_dir):
start = time.time()
# since '--prefix' option is only supported for pip 8.0+, so here we fallback to
# use '--install-option' when the pip version is lower than 8.0.0.
- pip_version = pkg_resources.get_distribution("pip").version
- from pkg_resources import parse_version
+ pip_version = importlib_metadata.version("pip")
if parse_version(pip_version) >= parse_version('8.0.0'):
subprocess.check_call(
[sys.executable, '-m', 'pip', 'install',
@@ -144,10 +147,8 @@ def _install_grpcio_tools_and_generate_proto_files(force, output_dir):
sys.stderr.flush()
shutil.rmtree(build_path, ignore_errors=True)
sys.path.append(install_obj.install_purelib)
- pkg_resources.working_set.add_entry(install_obj.install_purelib)
if install_obj.install_purelib != install_obj.install_platlib:
sys.path.append(install_obj.install_platlib)
- pkg_resources.working_set.add_entry(install_obj.install_platlib)
try:
generate_proto_files(force, output_dir)
finally:
@@ -185,11 +186,11 @@ def _add_license_header(dir, file_name):
def _check_grpcio_tools_version():
- version = pkg_resources.get_distribution("grpcio-tools").parsed_version
- from pkg_resources import parse_version
+ version_str = importlib_metadata.version("grpcio-tools")
+ version = parse_version(version_str)
if version < parse_version('1.29.0') or version > parse_version('1.71.0'):
raise RuntimeError(
- "Version of grpcio-tools must be between 1.29.0 and 1.71.0, got %s" % version)
+ "Version of grpcio-tools must be between 1.29.0 and 1.71.0, got %s" % version_str)
if __name__ == '__main__':
diff --git a/flink-python/pyflink/table/catalog.py b/flink-python/pyflink/table/catalog.py
index 40bb06c8eb4e8..d72eca4293ccd 100644
--- a/flink-python/pyflink/table/catalog.py
+++ b/flink-python/pyflink/table/catalog.py
@@ -1559,11 +1559,11 @@ def _from_j_column(j_column) -> Optional["Column"]:
raise TypeError("The input %s is not an instance of Column." % j_column)
if get_java_class(JPhysicalColumn).isAssignableFrom(j_column.getClass()):
- return PhysicalColumn(j_physical_column=j_column.getClass())
+ return PhysicalColumn(j_physical_column=j_column)
elif get_java_class(JComputedColumn).isAssignableFrom(j_column.getClass()):
- return ComputedColumn(j_computed_column=j_column.getClass())
+ return ComputedColumn(j_computed_column=j_column)
elif get_java_class(JMetadataColumn).isAssignableFrom(j_column.getClass()):
- return MetadataColumn(j_metadata_column=j_column.getClass())
+ return MetadataColumn(j_metadata_column=j_column)
else:
return None
diff --git a/flink-python/pyflink/table/expression.py b/flink-python/pyflink/table/expression.py
index 7ed3ff77f6e7a..64367287a7b93 100644
--- a/flink-python/pyflink/table/expression.py
+++ b/flink-python/pyflink/table/expression.py
@@ -1219,10 +1219,10 @@ def like(self,
pattern: Union[str, 'Expression[str]'] = None,
escape=None) -> 'Expression[bool]':
"""
- Returns true, if a string matches the specified LIKE pattern
+ Returns true, if a string matches the specified LIKE pattern.
e.g. 'Jo_n%' matches all strings that start with 'Jo(arbitrary letter)n'.
- An escape character consisting of a single char can be defined if necessary,
- '\\' by default.
+ An escape character consisting of a single char can be defined if necessary.
+ There is no default escape character.
"""
if escape is None:
return _binary_op("like")(self, pattern)
diff --git a/flink-python/pyflink/table/tests/test_schema_operation.py b/flink-python/pyflink/table/tests/test_schema_operation.py
index c293e783d4815..d8ce568aa2edb 100644
--- a/flink-python/pyflink/table/tests/test_schema_operation.py
+++ b/flink-python/pyflink/table/tests/test_schema_operation.py
@@ -15,7 +15,7 @@
# See the License for the specific language governing permissions and
# limitations under the License.
################################################################################
-from pyflink.table.catalog import ResolvedSchema
+from pyflink.table.catalog import ResolvedSchema, PhysicalColumn
from pyflink.table.table_schema import TableSchema
from pyflink.table.types import DataTypes
from pyflink.testing.test_case_utils import PyFlinkStreamTableTestCase
@@ -33,7 +33,7 @@ def test_get_schema(self):
result = t.group_by(t.c).select(t.a.sum.alias('a'), t.c.alias('b'))
schema = result.get_schema()
- assert schema == TableSchema(["a", "b"], [DataTypes.BIGINT(), DataTypes.STRING()])
+ self.assertEqual(schema, TableSchema(["a", "b"], [DataTypes.BIGINT(), DataTypes.STRING()]))
def test_get_resolved_schema(self):
t = self.t_env.from_elements([(1, 'Hi', 'Hello')], ['a', 'b', 'c'])
@@ -42,7 +42,23 @@ def test_get_resolved_schema(self):
['a', 'b', 'c'],
[DataTypes.BIGINT(), DataTypes.STRING(), DataTypes.STRING()],
)
- assert resolved_schema == expected_schema
+ self.assertEqual(resolved_schema, expected_schema)
+
+ def test_resolved_schema_get_columns(self):
+ physical_schema = ResolvedSchema.physical(
+ ['a', 'b', 'c'],
+ [DataTypes.BIGINT(), DataTypes.STRING(), DataTypes.STRING()],
+ )
+
+ columns = physical_schema.get_columns()
+ self.assertEqual(len(columns), 3)
+ for column in columns:
+ self.assertEqual(type(column), PhysicalColumn)
+
+ for idx in range(3):
+ column = physical_schema.get_column(idx)
+ self.assertEqual(type(column), PhysicalColumn)
+
if __name__ == '__main__':
import unittest
diff --git a/flink-python/pyproject.toml b/flink-python/pyproject.toml
index 836b5f771271b..04cac48efa83b 100644
--- a/flink-python/pyproject.toml
+++ b/flink-python/pyproject.toml
@@ -19,7 +19,7 @@
# Minimum requirements for the build system to execute.
requires = [
"packaging>=20.5; platform_machine=='arm64'", # macos M1
- "setuptools>=75.3",
+ "setuptools>=75.3,<82",
"wheel",
"cython>=0.29.24,<3; sys_platform == 'darwin' and python_version == '3.8'",
"fastavro==1.7.4; sys_platform == 'darwin' and python_version == '3.8'",
@@ -30,7 +30,7 @@ requires = [
[dependency-groups]
dev = [
"pip>=20.3",
- "setuptools>=75.3",
+ "setuptools>=75.3,<82",
"wheel",
"apache-beam>=2.54.0,<=2.61.0",
"cython>=0.29.24",
@@ -45,7 +45,7 @@ dev = [
"fastavro>=1.1.0,!=1.8.0",
"grpcio>=1.29.0,<=1.71.0",
"grpcio-tools>=1.29.0,<=1.71.0",
- "pemja>=0.5.5,<0.5.6; platform_system != 'Windows'",
+ "pemja>=0.5.7,<0.5.8; platform_system != 'Windows'",
"httplib2>=0.19.0",
"protobuf~=4.25",
"pytest~=8.0",
diff --git a/flink-python/setup.py b/flink-python/setup.py
index 9d021fc9f357f..a48a0e954f36e 100644
--- a/flink-python/setup.py
+++ b/flink-python/setup.py
@@ -325,7 +325,7 @@ def extracted_output_files(base_dir, file_path, output_directory):
'numpy>=1.22.4',
'pandas>=1.3.0,<2.3', # FLINK-38513: 2.3+ drops cp39 wheels
'pyarrow>=5.0.0,<21.0.0',
- 'pemja>=0.5.5,<0.5.6;platform_system != "Windows"',
+ 'pemja>=0.5.7,<0.5.8;platform_system != "Windows"',
'httplib2>=0.19.0',
'ruamel.yaml>=0.18.4',
apache_flink_libraries_dependency]
diff --git a/flink-python/src/main/java/org/apache/flink/python/env/PythonDependencyInfo.java b/flink-python/src/main/java/org/apache/flink/python/env/PythonDependencyInfo.java
index 2f65e5f7be759..1d3665c2ab1bd 100644
--- a/flink-python/src/main/java/org/apache/flink/python/env/PythonDependencyInfo.java
+++ b/flink-python/src/main/java/org/apache/flink/python/env/PythonDependencyInfo.java
@@ -39,6 +39,7 @@
import static org.apache.flink.python.PythonOptions.PYTHON_FILES_DISTRIBUTED_CACHE_INFO;
import static org.apache.flink.python.PythonOptions.PYTHON_PATH;
import static org.apache.flink.python.PythonOptions.PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO;
+import static org.apache.flink.python.util.PythonDependencyUtils.getArchiveTargetDirName;
/** PythonDependencyInfo contains the information of third-party dependencies. */
@Internal
@@ -203,4 +204,32 @@ public static PythonDependencyInfo create(
config.get(PYTHON_EXECUTION_MODE),
config.get(PYTHON_PATH));
}
+
+ /**
+ * Checks whether the configured python executable path is located within one of the shipped
+ * archives.
+ *
+ * This is determined by comparing the first path component of the python executable (the
+ * base directory) against the target directory names of all registered archives.
+ *
+ * @return {@code true} if the python executable's base directory matches an archive target
+ * directory
+ */
+ public boolean isPythonExecFromArchives() {
+ if (archives.isEmpty()) {
+ return false;
+ }
+ int index = pythonExec.indexOf(File.separator);
+ if (index == -1) {
+ index = pythonExec.length();
+ }
+ String pythonExecBaseDir = pythonExec.substring(0, index);
+ for (Map.Entry entry : archives.entrySet()) {
+ String targetDirName = getArchiveTargetDirName(entry.getValue());
+ if (targetDirName.equals(pythonExecBaseDir)) {
+ return true;
+ }
+ }
+ return false;
+ }
}
diff --git a/flink-python/src/main/java/org/apache/flink/python/env/embedded/EmbeddedPythonEnvironmentManager.java b/flink-python/src/main/java/org/apache/flink/python/env/embedded/EmbeddedPythonEnvironmentManager.java
index 896c3a2c4ffa6..7d8fe49d5b484 100644
--- a/flink-python/src/main/java/org/apache/flink/python/env/embedded/EmbeddedPythonEnvironmentManager.java
+++ b/flink-python/src/main/java/org/apache/flink/python/env/embedded/EmbeddedPythonEnvironmentManager.java
@@ -24,6 +24,8 @@
import org.apache.flink.python.env.PythonDependencyInfo;
import org.apache.flink.python.env.PythonEnvironment;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
import pemja.core.PythonInterpreterConfig;
import java.io.File;
@@ -36,6 +38,8 @@
*/
@Internal
public class EmbeddedPythonEnvironmentManager extends AbstractPythonEnvironmentManager {
+ private static final Logger LOG =
+ LoggerFactory.getLogger(EmbeddedPythonEnvironmentManager.class);
public EmbeddedPythonEnvironmentManager(
PythonDependencyInfo dependencyInfo,
@@ -70,13 +74,41 @@ public PythonEnvironment createEnvironment() throws Exception {
flinkPython + File.pathSeparator + env.getOrDefault("PYTHONPATH", ""));
}
- PythonInterpreterConfig interpreterConfig =
+ PythonInterpreterConfig.PythonInterpreterConfigBuilder interpreterConfigBuilder =
PythonInterpreterConfig.newBuilder()
- .setPythonExec(dependencyInfo.getPythonExec())
.setExcType(execType)
- .addPythonPaths(env.getOrDefault("PYTHONPATH", ""))
- .build();
-
- return new EmbeddedPythonEnvironment(interpreterConfig, env);
+ .addPythonPaths(env.getOrDefault("PYTHONPATH", ""));
+ if (dependencyInfo.isPythonExecFromArchives()) {
+ String pythonExecPath = dependencyInfo.getPythonExec();
+ String binSeparator = File.separator + "bin" + File.separator;
+ int binIndex = pythonExecPath.lastIndexOf(binSeparator);
+ if (binIndex == -1) {
+ throw new RuntimeException(
+ String.format(
+ "Python executable '%s' is from archives but does not contain "
+ + "a '%s' path component. Expected a path like "
+ + "'venv/bin/python'.",
+ pythonExecPath, binSeparator));
+ }
+ String pythonHome =
+ String.join(
+ File.separator,
+ env.get(PYTHON_WORKING_DIR),
+ pythonExecPath.substring(0, binIndex));
+ String pythonExec =
+ String.join(File.separator, env.get(PYTHON_WORKING_DIR), pythonExecPath);
+ LOG.info(
+ "Use python home and python exec from archives. Python home: {}, Python exec: {}",
+ pythonHome,
+ pythonExec);
+ interpreterConfigBuilder.setPythonHome(pythonHome).setPythonExec(pythonExec);
+ } else {
+ LOG.info(
+ "Python interpreter path is not from archives, use python exec from "
+ + "config {}.",
+ dependencyInfo.getPythonExec());
+ interpreterConfigBuilder.setPythonExec(dependencyInfo.getPythonExec());
+ }
+ return new EmbeddedPythonEnvironment(interpreterConfigBuilder.build(), env);
}
}
diff --git a/flink-python/src/main/java/org/apache/flink/python/util/PythonDependencyUtils.java b/flink-python/src/main/java/org/apache/flink/python/util/PythonDependencyUtils.java
index 15d10e42bb97d..a1b54188029e6 100644
--- a/flink-python/src/main/java/org/apache/flink/python/util/PythonDependencyUtils.java
+++ b/flink-python/src/main/java/org/apache/flink/python/util/PythonDependencyUtils.java
@@ -69,6 +69,23 @@ public class PythonDependencyUtils {
public static final String PARAM_DELIMITER = "#";
private static final String HASH_ALGORITHM = "SHA-256";
+ /**
+ * Extracts the target directory name from an archive value string.
+ *
+ * The archive value may contain a {@link #PARAM_DELIMITER} separator. If present, the format
+ * is "originalFileName#targetDirName" and the target directory name is the part after the
+ * delimiter. Otherwise, the entire value is used as the target directory name.
+ *
+ * @param archiveValue the archive value string, e.g. "venv.zip#venv" or "venv"
+ * @return the target directory name
+ */
+ public static String getArchiveTargetDirName(String archiveValue) {
+ if (archiveValue.contains(PARAM_DELIMITER)) {
+ return archiveValue.split(PARAM_DELIMITER, 2)[1];
+ }
+ return archiveValue;
+ }
+
/**
* Adds python dependencies to registered cache file list according to given configuration and
* returns a new configuration which contains the metadata of the registered python
diff --git a/flink-python/src/main/resources/META-INF/NOTICE b/flink-python/src/main/resources/META-INF/NOTICE
index 6a7f3ade23196..2bca2176728a8 100644
--- a/flink-python/src/main/resources/META-INF/NOTICE
+++ b/flink-python/src/main/resources/META-INF/NOTICE
@@ -28,7 +28,7 @@ This project bundles the following dependencies under the Apache Software Licens
- org.apache.beam:beam-sdks-java-transform-service-launcher:2.54.0
- org.apache.beam:beam-vendor-guava-32_1_2-jre:0.1
- org.apache.beam:beam-vendor-grpc-1_60_1:0.1
-- com.alibaba:pemja:0.5.5
+- com.alibaba:pemja:0.5.7
This project bundles the following dependencies under the BSD license.
See bundled license files for details
diff --git a/flink-python/src/test/java/org/apache/flink/python/env/PythonDependencyInfoTest.java b/flink-python/src/test/java/org/apache/flink/python/env/PythonDependencyInfoTest.java
index f3b4c4370774a..dfc2341d0ec6a 100644
--- a/flink-python/src/test/java/org/apache/flink/python/env/PythonDependencyInfoTest.java
+++ b/flink-python/src/test/java/org/apache/flink/python/env/PythonDependencyInfoTest.java
@@ -27,6 +27,7 @@
import org.junit.jupiter.api.Test;
+import java.io.File;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
@@ -136,4 +137,64 @@ void testParsePythonExec() {
assertThat(dependencyInfo.getPythonExec()).isEqualTo("/usr/bin/python3");
}
+
+ @Test
+ void testIsPythonExecFromArchives() {
+ // exec matches archive target dir
+ Map archives = new HashMap<>();
+ archives.put("/tmp/venv.zip", "venv");
+ assertThat(
+ new PythonDependencyInfo(
+ new HashMap<>(),
+ null,
+ null,
+ archives,
+ "venv" + File.separator + "bin" + File.separator + "python")
+ .isPythonExecFromArchives())
+ .isTrue();
+
+ // exec does not match any archive
+ assertThat(
+ new PythonDependencyInfo(
+ new HashMap<>(), null, null, archives, "/usr/bin/python3")
+ .isPythonExecFromArchives())
+ .isFalse();
+
+ // archive value with param delimiter ("originalFileName#targetDirName")
+ Map archivesWithDelimiter = new HashMap<>();
+ archivesWithDelimiter.put("/tmp/venv.zip", "venv.zip#myenv");
+ assertThat(
+ new PythonDependencyInfo(
+ new HashMap<>(),
+ null,
+ null,
+ archivesWithDelimiter,
+ "myenv"
+ + File.separator
+ + "bin"
+ + File.separator
+ + "python")
+ .isPythonExecFromArchives())
+ .isTrue();
+
+ // empty archives
+ assertThat(
+ new PythonDependencyInfo(
+ new HashMap<>(),
+ null,
+ null,
+ new HashMap<>(),
+ "venv" + File.separator + "bin" + File.separator + "python")
+ .isPythonExecFromArchives())
+ .isFalse();
+
+ // exec without separator — entire string is the base dir
+ Map archivesForBareExec = new HashMap<>();
+ archivesForBareExec.put("/tmp/python.zip", "python");
+ assertThat(
+ new PythonDependencyInfo(
+ new HashMap<>(), null, null, archivesForBareExec, "python")
+ .isPythonExecFromArchives())
+ .isTrue();
+ }
}
diff --git a/flink-python/src/test/java/org/apache/flink/python/util/PythonDependencyUtilsTest.java b/flink-python/src/test/java/org/apache/flink/python/util/PythonDependencyUtilsTest.java
index e75c5c6c5515c..3240e9a4f6bef 100644
--- a/flink-python/src/test/java/org/apache/flink/python/util/PythonDependencyUtilsTest.java
+++ b/flink-python/src/test/java/org/apache/flink/python/util/PythonDependencyUtilsTest.java
@@ -39,6 +39,7 @@
import static org.apache.flink.python.util.PythonDependencyUtils.CACHE;
import static org.apache.flink.python.util.PythonDependencyUtils.FILE;
import static org.apache.flink.python.util.PythonDependencyUtils.configurePythonDependencies;
+import static org.apache.flink.python.util.PythonDependencyUtils.getArchiveTargetDirName;
import static org.apache.flink.python.util.PythonDependencyUtils.merge;
import static org.assertj.core.api.Assertions.assertThat;
@@ -237,6 +238,12 @@ void testPythonPath() {
verifyConfiguration(expectedConfiguration, actual);
}
+ @Test
+ void testGetArchiveTargetDirName() {
+ assertThat(getArchiveTargetDirName("venv.zip#venv")).isEqualTo("venv");
+ assertThat(getArchiveTargetDirName("venv")).isEqualTo("venv");
+ }
+
private void verifyCachedFiles(Map expected, Configuration config) {
Map actual =
config.getOptional(PipelineOptions.CACHED_FILES).orElse(new ArrayList<>()).stream()
diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/ActorSystemBootstrapTools.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/ActorSystemBootstrapTools.java
index 517a48669e688..a206e4ef08bef 100644
--- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/ActorSystemBootstrapTools.java
+++ b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/ActorSystemBootstrapTools.java
@@ -19,6 +19,7 @@
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ConfigurationUtils;
import org.apache.flink.configuration.RpcOptions;
import org.apache.flink.runtime.rpc.RpcSystem;
import org.apache.flink.util.NetUtils;
@@ -32,7 +33,9 @@
import java.io.IOException;
import java.net.BindException;
import java.util.Iterator;
+import java.util.Map;
import java.util.Optional;
+import java.util.stream.Collectors;
/** Tools for starting the Actor Systems used to run the JobManager and TaskManager actors. */
public class ActorSystemBootstrapTools {
@@ -241,6 +244,22 @@ public static ActorSystem startLocalActorSystem(
}
}
+ /**
+ * Converts the given Pekko {@link Config} into a flattened {@link Map}.
+ *
+ * @param config The Pekko configuration
+ * @return A map of configuration keys to string values
+ */
+ @VisibleForTesting
+ static Map toMaskedMap(Config config) {
+ return ConfigurationUtils.hideSensitiveValues(
+ config.entrySet().stream()
+ .collect(
+ Collectors.toMap(
+ Map.Entry::getKey,
+ entry -> String.valueOf(entry.getValue().unwrapped()))));
+ }
+
/**
* Starts an Actor System with given Pekko config.
*
@@ -251,7 +270,9 @@ public static ActorSystem startLocalActorSystem(
*/
private static ActorSystem startActorSystem(
Config config, String actorSystemName, Logger logger) {
- logger.debug("Using pekko configuration\n {}", config);
+ if (logger.isDebugEnabled()) {
+ logger.debug("Using pekko configuration\n {}", toMaskedMap(config));
+ }
ActorSystem actorSystem = PekkoUtils.createActorSystem(actorSystemName, config);
logger.info("Actor system started at {}", PekkoUtils.getAddress(actorSystem));
diff --git a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/ActorSystemBootstrapToolsTest.java b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/ActorSystemBootstrapToolsTest.java
index 5343c4011cf1c..1ebecb27fb925 100644
--- a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/ActorSystemBootstrapToolsTest.java
+++ b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/ActorSystemBootstrapToolsTest.java
@@ -18,11 +18,14 @@
package org.apache.flink.runtime.rpc.pekko;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
import org.apache.flink.core.testutils.FlinkAssertions;
import org.apache.flink.util.ExecutorUtils;
import org.apache.flink.util.concurrent.FutureUtils;
import org.apache.flink.util.function.CheckedSupplier;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
import org.apache.pekko.actor.ActorSystem;
import org.junit.jupiter.api.Test;
import org.slf4j.Logger;
@@ -32,6 +35,7 @@
import java.net.InetAddress;
import java.net.ServerSocket;
import java.util.List;
+import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CyclicBarrier;
import java.util.concurrent.ExecutorService;
@@ -40,6 +44,7 @@
import java.util.stream.Collectors;
import java.util.stream.IntStream;
+import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatThrownBy;
/** Tests for the {@link ActorSystemBootstrapTools}. */
@@ -111,4 +116,35 @@ void testActorSystemInstantiationFailureWhenPortOccupied() throws Exception {
portOccupier.close();
}
}
+
+ @Test
+ void testToMaskedMapMasksOnlySensitiveKeys() {
+ Config config =
+ ConfigFactory.parseMap(
+ Map.of(
+ "pekko.loglevel", "OFF",
+ "pekko.remote.artery.enabled", "false",
+ "pekko.remote.classic.netty.ssl.security.key-password", "secret",
+ "pekko.remote.classic.netty.ssl.security.key-store-password",
+ "secret2",
+ "pekko.remote.classic.netty.ssl.security.trust-store-password",
+ "secret3"));
+
+ Map result = ActorSystemBootstrapTools.toMaskedMap(config);
+
+ // Non-sensitive values should remain the same
+ assertThat(result.get("pekko.loglevel")).isEqualTo("OFF");
+ assertThat(result.get("pekko.remote.artery.enabled")).isEqualTo("false");
+
+ // Sensitive values should be masked
+ assertThat(result.get("pekko.remote.classic.netty.ssl.security.key-password"))
+ .isNotEqualTo("secret")
+ .isEqualTo(GlobalConfiguration.HIDDEN_CONTENT);
+ assertThat(result.get("pekko.remote.classic.netty.ssl.security.key-store-password"))
+ .isNotEqualTo("secret2")
+ .isEqualTo(GlobalConfiguration.HIDDEN_CONTENT);
+ assertThat(result.get("pekko.remote.classic.netty.ssl.security.trust-store-password"))
+ .isNotEqualTo("secret3")
+ .isEqualTo(GlobalConfiguration.HIDDEN_CONTENT);
+ }
}
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandlerTest.java
index 9d909f2e46665..593559da3d294 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandlerTest.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandlerTest.java
@@ -31,7 +31,6 @@
import org.junit.jupiter.api.Assumptions;
import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
@@ -104,7 +103,6 @@ void testDeleteUnknownJar() throws Exception {
});
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testFailedDelete() throws Exception {
makeJarDirReadOnly();
diff --git a/flink-runtime-web/web-dashboard/src/app/components/dagre/dagre.component.ts b/flink-runtime-web/web-dashboard/src/app/components/dagre/dagre.component.ts
index 7cc46320aa5d3..b25698e8a737b 100644
--- a/flink-runtime-web/web-dashboard/src/app/components/dagre/dagre.component.ts
+++ b/flink-runtime-web/web-dashboard/src/app/components/dagre/dagre.component.ts
@@ -359,7 +359,6 @@ export class DagreComponent extends NzGraph {
* @param $event
*/
onNodeMouseEnter($event: MouseEvent): void {
- this.graphElement.nativeElement.appendChild($event.target);
this.layoutLinks.forEach(l => {
if (l.id.split('-').indexOf(($event.target as HTMLElement).id) !== -1) {
l.options.focused = true;
@@ -376,20 +375,6 @@ export class DagreComponent extends NzGraph {
l.options.focused = this.focusedLinkIds.indexOf(l.id) !== -1;
});
- this.graphElement.nativeElement.appendChild(this.overlayElement.nativeElement);
-
- this.graphElement.nativeElement.querySelectorAll(`.link-group`).forEach((e: Element) => {
- if (this.focusedLinkIds.indexOf(e.id) !== -1) {
- this.graphElement.nativeElement.appendChild(e);
- }
- });
-
- this.graphElement.nativeElement.querySelectorAll(`.node-group`).forEach((e: Element) => {
- if ([this.selectedNodeId, ...this.circleNodeIds].indexOf(e.id) !== -1) {
- this.graphElement.nativeElement.appendChild(e);
- }
- });
-
this.cd.detectChanges();
}
diff --git a/flink-runtime-web/web-dashboard/src/app/pages/job/overview/job-overview.component.ts b/flink-runtime-web/web-dashboard/src/app/pages/job/overview/job-overview.component.ts
index c03e4ad908c39..8b46f079f1ea7 100644
--- a/flink-runtime-web/web-dashboard/src/app/pages/job/overview/job-overview.component.ts
+++ b/flink-runtime-web/web-dashboard/src/app/pages/job/overview/job-overview.component.ts
@@ -169,11 +169,16 @@ export class JobOverviewComponent implements OnInit, OnDestroy {
map(result => {
return {
...node,
- backPressuredPercentage: Math.min(Math.round(result.backPressuredTimeMsPerSecond.max / 10), 100),
- busyPercentage: Math.min(Math.round(result.busyTimeMsPerSecond.max / 10), 100),
- dataSkewPercentage: result.numRecordsInPerSecond.skew
+ backPressuredPercentage: result.backPressuredTimeMsPerSecond
+ ? Math.min(Math.round(result.backPressuredTimeMsPerSecond.max / 10), 100)
+ : NaN,
+ busyPercentage: result.busyTimeMsPerSecond
+ ? Math.min(Math.round(result.busyTimeMsPerSecond.max / 10), 100)
+ : NaN,
+ dataSkewPercentage: result.numRecordsInPerSecond?.skew ?? NaN
};
- })
+ }),
+ catchError(() => of(node))
);
})
).pipe(catchError(() => of(nodes)));
@@ -185,7 +190,8 @@ export class JobOverviewComponent implements OnInit, OnDestroy {
return this.metricService.loadWatermarks(this.jobId, node.id).pipe(
map(result => {
return { ...node, lowWatermark: result.lowWatermark };
- })
+ }),
+ catchError(() => of(node))
);
})
).pipe(catchError(() => of(nodes)));
diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml
index 61a2a5beac4ce..3e86087ffb221 100644
--- a/flink-runtime/pom.xml
+++ b/flink-runtime/pom.xml
@@ -238,7 +238,7 @@ under the License.
- org.lz4
+ at.yawk.lz4
lz4-java
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/ResourceRequestPreMappings.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/ResourceRequestPreMappings.java
new file mode 100644
index 0000000000000..7aec0effe2317
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/ResourceRequestPreMappings.java
@@ -0,0 +1,412 @@
+/*
+ * 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.runtime.jobmaster.slotpool;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.util.CollectionUtil;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/**
+ * This class is designed to handle the pre-matching of resource requests in the context of balanced
+ * task scheduling for streaming jobs. During the batch allocation of resources, where resource
+ * requests are allocated in a single, non-interleaved operation, it is impossible to make immediate
+ * individual adjustments to unmatched resource requests. This may lead to situations where not all
+ * resource requests can be successfully fulfilled. For example:
+ *
+ *
+ * resource requests:
+ * - resource request-1: ResourceProfile-1(UNKNOWN)
+ * - resource request-2: ResourceProfile-2(cpu=2 core, memory=2G)
+ *
+ * available slots:
+ * - slot-a: ResourceProfile-a(cpu=1 core, memory=1G)
+ * - slot-b: ResourceProfile-b(cpu=2 core, memory=2G)
+ *
+ *
+ * When the strategy {@link TasksBalancedRequestSlotMatchingStrategy} performs resource allocation,
+ * the following matching mapping might occur, preventing all slot requests from being successfully
+ * assigned in a consistent manner and thus hindering the scheduling of the entire job:
+ *
+ *
+ * the unexpected mapping case:
+ * - resource request-1: ResourceProfile-1(UNKNOWN) was matched with slot-b: ResourceProfile-b(cpu=2 core, memory=2G)
+ * - resource request-2: ResourceProfile-2(cpu=2 core, memory=2G) was not matched
+ *
+ *
+ * Therefore, it is crucial to determine how ResourceProfiles should match before the batch
+ * allocation of resource requests, aiming to assure the allocation successfully at least. An ideal
+ * matching relationship would be:
+ *
+ *
+ * - ResourceProfile-1(UNKNOWN) -> ResourceProfile-a(cpu=1 core, memory=1G)
+ * - ResourceProfile-2(cpu=2 core, memory=2G) -> ResourceProfile-b(cpu=2 core, memory=2G)
+ *
+ *
+ * This is the motivation for introducing the current class.
+ */
+final class ResourceRequestPreMappings {
+
+ private final boolean matchingFulfilled;
+ // The variable to keep base mappings result related information, which can assure that
+ // the allocation for all requests could be run successfully at least.
+ private final Map>
+ baseRequiredResourcePreMappings;
+ // The variable to keep the remaining available flexible resources besides the
+ // baseRequiredResourcePreMappings.
+ private final Map remainingFlexibleResources;
+
+ private ResourceRequestPreMappings(
+ boolean matchingFulfilled,
+ final Map>
+ baseRequiredResourcePreMappings,
+ final Map remainingFlexibleResources) {
+ this.matchingFulfilled = matchingFulfilled;
+
+ this.baseRequiredResourcePreMappings =
+ CollectionUtil.newHashMapWithExpectedSize(baseRequiredResourcePreMappings.size());
+ this.baseRequiredResourcePreMappings.putAll(baseRequiredResourcePreMappings);
+
+ this.remainingFlexibleResources =
+ CollectionUtil.newHashMapWithExpectedSize(remainingFlexibleResources.size());
+ this.remainingFlexibleResources.putAll(remainingFlexibleResources);
+ }
+
+ static ResourceRequestPreMappings createFrom(
+ Collection pendingRequests, Collection extends PhysicalSlot> slots) {
+ return new ResourceRequestPreMappingsBuilder(pendingRequests, slots).build();
+ }
+
+ boolean isMatchingFulfilled() {
+ return matchingFulfilled;
+ }
+
+ boolean hasAvailableProfile(
+ ResourceProfile requiredResourceProfile, ResourceProfile acquirableResourceProfile) {
+ // Check for base mappings first
+ Map basePreMapping =
+ baseRequiredResourcePreMappings.getOrDefault(
+ requiredResourceProfile, new HashMap<>());
+ Integer remainingCnt = basePreMapping.getOrDefault(acquirableResourceProfile, 0);
+
+ if (remainingCnt > 0) {
+ return true;
+ } else {
+ return remainingFlexibleResources.getOrDefault(acquirableResourceProfile, 0) > 0;
+ }
+ }
+
+ void decrease(
+ ResourceProfile requiredResourceProfile, ResourceProfile acquiredResourceProfile) {
+ Map basePreMapping =
+ baseRequiredResourcePreMappings.getOrDefault(
+ requiredResourceProfile, new HashMap<>());
+ Integer remainingCntOfBaseMappings =
+ basePreMapping.getOrDefault(acquiredResourceProfile, 0);
+ Integer remainingCntOfFlexibleResources =
+ remainingFlexibleResources.getOrDefault(acquiredResourceProfile, 0);
+
+ Preconditions.checkState(
+ remainingCntOfBaseMappings > 0 || remainingCntOfFlexibleResources > 0,
+ "Remaining acquired resource profile %s to match %s is not enough.",
+ acquiredResourceProfile,
+ requiredResourceProfile);
+
+ if (remainingCntOfBaseMappings > 0) {
+ basePreMapping.put(acquiredResourceProfile, remainingCntOfBaseMappings - 1);
+ return;
+ }
+
+ if (remainingCntOfFlexibleResources > 0) {
+ remainingFlexibleResources.put(
+ acquiredResourceProfile, remainingCntOfFlexibleResources - 1);
+ // release a resource back to remainingFlexibleResources.
+ adjustBaseToRemainingFlexibleResources(basePreMapping);
+ }
+ }
+
+ private void adjustBaseToRemainingFlexibleResources(
+ Map basePreMapping) {
+ Optional> releasableOptOfBaseMappings =
+ basePreMapping.entrySet().stream()
+ .filter(entry -> entry.getValue() > 0)
+ .findFirst();
+ Preconditions.checkState(
+ releasableOptOfBaseMappings.isPresent(),
+ "No releasable mapping found in the base mappings between resources and requests.");
+ Map.Entry releasable = releasableOptOfBaseMappings.get();
+ ResourceProfile releasableResourceProfile = releasable.getKey();
+
+ basePreMapping.put(releasableResourceProfile, releasable.getValue() - 1);
+
+ remainingFlexibleResources.compute(
+ releasableResourceProfile,
+ (resourceProfile, oldValue) -> oldValue == null ? 1 : oldValue + 1);
+ }
+
+ @VisibleForTesting
+ static ResourceRequestPreMappings createFrom(
+ boolean allMatchable,
+ final Map>
+ baseRequiredResourcePreMappings,
+ final Map remainingFlexibleResources) {
+ return new ResourceRequestPreMappings(
+ allMatchable, baseRequiredResourcePreMappings, remainingFlexibleResources);
+ }
+
+ @VisibleForTesting
+ Map> getBaseRequiredResourcePreMappings() {
+ return Collections.unmodifiableMap(baseRequiredResourcePreMappings);
+ }
+
+ @VisibleForTesting
+ int getAvailableResourceCntOfBasePreMappings(
+ ResourceProfile requiredResourceProfile, ResourceProfile acquirableResourceProfile) {
+ return baseRequiredResourcePreMappings
+ .getOrDefault(requiredResourceProfile, new HashMap<>())
+ .getOrDefault(acquirableResourceProfile, 0);
+ }
+
+ @VisibleForTesting
+ Map getRemainingFlexibleResources() {
+ return Collections.unmodifiableMap(remainingFlexibleResources);
+ }
+
+ @VisibleForTesting
+ int getAvailableResourceCntOfRemainingFlexibleMapping(
+ ResourceProfile availableResourceProfile) {
+ return remainingFlexibleResources.getOrDefault(availableResourceProfile, 0);
+ }
+
+ private static final class ResourceRequestPreMappingsBuilder {
+
+ private final Map unfulfilledRequired;
+ private final Map availableResources;
+
+ // The variable to maintain the base mappings result related information, which can
+ // assure that the allocation for all requests could be run successfully at least.
+ private final Map>
+ baseRequiredResourcePreMappings;
+
+ private ResourceRequestPreMappingsBuilder(
+ Collection pendingRequests,
+ Collection extends PhysicalSlot> slots) {
+ this.unfulfilledRequired =
+ pendingRequests.stream()
+ .collect(
+ Collectors.groupingBy(
+ PendingRequest::getResourceProfile,
+ Collectors.summingInt(ignored -> 1)));
+ this.unfulfilledRequired
+ .keySet()
+ .forEach(
+ rp ->
+ Preconditions.checkState(
+ !rp.equals(ResourceProfile.ZERO)
+ && !rp.equals(ResourceProfile.ANY),
+ "The required resource must not be ResourceProfile.ZERO and ResourceProfile.ANY."));
+ this.availableResources =
+ slots.stream()
+ .collect(
+ Collectors.groupingBy(
+ PhysicalSlot::getResourceProfile,
+ Collectors.summingInt(ignored -> 1)));
+ this.availableResources
+ .keySet()
+ .forEach(
+ rp ->
+ Preconditions.checkState(
+ !rp.equals(ResourceProfile.UNKNOWN)
+ && !rp.equals(ResourceProfile.ZERO),
+ "The resource profile of a slot must not be ResourceProfile.UNKNOWN and ResourceProfile.ZERO."));
+ this.baseRequiredResourcePreMappings =
+ CollectionUtil.newHashMapWithExpectedSize(slots.size());
+ }
+
+ private ResourceRequestPreMappings build() {
+ if (unfulfilledRequired.isEmpty()
+ || availableResources.isEmpty()
+ || !canFulfillDesiredResources()) {
+ return currentPreMappings(false);
+ }
+
+ buildFineGrainedRequestFulfilledExactMapping();
+ if (isMatchingFulfilled()) {
+ return currentPreMappings(true);
+ }
+
+ buildRemainingFineGrainedRequestFulfilledAnyMapping();
+ if (isMatchingFulfilled()) {
+ return currentPreMappings(true);
+ }
+
+ buildUnknownRequestFulfilledMapping();
+ return currentPreMappings(isMatchingFulfilled());
+ }
+
+ private void buildFineGrainedRequestFulfilledExactMapping() {
+ for (Map.Entry unfulfilledEntry :
+ new HashMap<>(unfulfilledRequired).entrySet()) {
+ ResourceProfile requiredFineGrainedResourceProfile = unfulfilledEntry.getKey();
+ if (ResourceProfile.UNKNOWN.equals(requiredFineGrainedResourceProfile)) {
+ continue;
+ }
+ // checking fine-grained
+ int unfulfilledFineGrainedRequiredCnt = unfulfilledEntry.getValue();
+ int availableFineGrainedResourceCnt =
+ availableResources.getOrDefault(requiredFineGrainedResourceProfile, 0);
+ if (unfulfilledFineGrainedRequiredCnt <= 0
+ || availableFineGrainedResourceCnt <= 0) {
+ continue;
+ }
+
+ int diff = unfulfilledFineGrainedRequiredCnt - availableFineGrainedResourceCnt;
+
+ Map fulfilledProfileCount =
+ baseRequiredResourcePreMappings.computeIfAbsent(
+ requiredFineGrainedResourceProfile, ignored -> new HashMap<>());
+ fulfilledProfileCount.put(
+ requiredFineGrainedResourceProfile,
+ diff > 0
+ ? availableFineGrainedResourceCnt
+ : unfulfilledFineGrainedRequiredCnt);
+
+ int newUnfulfilledFineGrainedRequiredCnt = Math.max(diff, 0);
+ int unAvailableFineGrainedResourceCnt = Math.max(-diff, 0);
+ availableResources.put(
+ requiredFineGrainedResourceProfile, unAvailableFineGrainedResourceCnt);
+ unfulfilledRequired.put(
+ requiredFineGrainedResourceProfile, newUnfulfilledFineGrainedRequiredCnt);
+ }
+ }
+
+ private void buildRemainingFineGrainedRequestFulfilledAnyMapping() {
+ Integer availableResourceProfileANYCount =
+ availableResources.getOrDefault(ResourceProfile.ANY, 0);
+ if (availableResourceProfileANYCount <= 0) {
+ return;
+ }
+
+ for (Map.Entry unfulfilledEntry :
+ new HashMap<>(unfulfilledRequired).entrySet()) {
+ availableResourceProfileANYCount =
+ availableResources.getOrDefault(ResourceProfile.ANY, 0);
+
+ if (availableResourceProfileANYCount <= 0) {
+ return;
+ }
+ ResourceProfile fineGrainedRequestResourceProfile = unfulfilledEntry.getKey();
+ if (ResourceProfile.UNKNOWN.equals(fineGrainedRequestResourceProfile)) {
+ continue;
+ }
+ // checking fine-grained
+ int unfulfilledFineGrainedRequiredCnt =
+ unfulfilledRequired.getOrDefault(fineGrainedRequestResourceProfile, 0);
+ if (unfulfilledFineGrainedRequiredCnt <= 0) {
+ continue;
+ }
+
+ int diff = unfulfilledFineGrainedRequiredCnt - availableResourceProfileANYCount;
+
+ Map fulfilledProfileCount =
+ baseRequiredResourcePreMappings.computeIfAbsent(
+ fineGrainedRequestResourceProfile, ignored -> new HashMap<>());
+ fulfilledProfileCount.put(
+ ResourceProfile.ANY,
+ diff > 0
+ ? availableResourceProfileANYCount
+ : unfulfilledFineGrainedRequiredCnt);
+
+ int newUnfulfilledFineGrainedRequiredCnt = Math.max(diff, 0);
+ int newAvailableResourceProfileANYCount = Math.max(-diff, 0);
+ availableResources.put(ResourceProfile.ANY, newAvailableResourceProfileANYCount);
+ unfulfilledRequired.put(
+ fineGrainedRequestResourceProfile, newUnfulfilledFineGrainedRequiredCnt);
+ }
+ }
+
+ private void buildUnknownRequestFulfilledMapping() {
+ if (unfulfilledRequired.getOrDefault(ResourceProfile.UNKNOWN, 0) <= 0) {
+ return;
+ }
+
+ for (Map.Entry availableResourceEntry :
+ new HashMap<>(availableResources).entrySet()) {
+ Integer unfulfilledUnknownRequiredCnt =
+ unfulfilledRequired.getOrDefault(ResourceProfile.UNKNOWN, 0);
+ ResourceProfile availableResourceProfile = availableResourceEntry.getKey();
+ int availableResourceCnt =
+ availableResources.getOrDefault(availableResourceProfile, 0);
+ if (availableResourceCnt <= 0) {
+ continue;
+ }
+ if (unfulfilledUnknownRequiredCnt <= 0) {
+ return;
+ }
+ int diff = unfulfilledUnknownRequiredCnt - availableResourceCnt;
+ Map fulfilledProfileCount =
+ baseRequiredResourcePreMappings.computeIfAbsent(
+ ResourceProfile.UNKNOWN, ignored -> new HashMap<>());
+ fulfilledProfileCount.put(
+ availableResourceProfile,
+ diff > 0 ? availableResourceCnt : unfulfilledUnknownRequiredCnt);
+
+ int newUnfulfilledUnknownRequiredCnt = Math.max(diff, 0);
+ int newAvailableResourceCnt = Math.max(-diff, 0);
+ availableResources.put(availableResourceProfile, newAvailableResourceCnt);
+ unfulfilledRequired.put(ResourceProfile.UNKNOWN, newUnfulfilledUnknownRequiredCnt);
+ }
+ }
+
+ private ResourceRequestPreMappings currentPreMappings(boolean matchingFulfilled) {
+ if (!matchingFulfilled) {
+ return new ResourceRequestPreMappings(false, new HashMap<>(), new HashMap<>());
+ }
+ return new ResourceRequestPreMappings(
+ true,
+ Collections.unmodifiableMap(baseRequiredResourcePreMappings),
+ Collections.unmodifiableMap(availableResources));
+ }
+
+ private boolean isMatchingFulfilled() {
+ for (ResourceProfile unfulfilledProfile : unfulfilledRequired.keySet()) {
+ Integer unfulfilled = unfulfilledRequired.getOrDefault(unfulfilledProfile, 0);
+ if (unfulfilled > 0) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ private boolean canFulfillDesiredResources() {
+ Integer totalUnfulfilledCnt =
+ unfulfilledRequired.values().stream().reduce(0, Integer::sum);
+ Integer totalAvailableCnt =
+ availableResources.values().stream().reduce(0, Integer::sum);
+ return totalAvailableCnt >= totalUnfulfilledCnt;
+ }
+ }
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/TasksBalancedRequestSlotMatchingStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/TasksBalancedRequestSlotMatchingStrategy.java
index 8b2fb88c6374d..c70f15ad17e7e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/TasksBalancedRequestSlotMatchingStrategy.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/TasksBalancedRequestSlotMatchingStrategy.java
@@ -67,12 +67,10 @@ static final class PhysicalSlotElementComparator implements Comparator matchRequestsAndSlots(
Collection extends PhysicalSlot> slots,
Collection pendingRequests,
Map taskExecutorsLoad) {
- if (pendingRequests.isEmpty()) {
+ ResourceRequestPreMappings resourceRequestPreMappings =
+ ResourceRequestPreMappings.createFrom(pendingRequests, slots);
+ if (!resourceRequestPreMappings.isMatchingFulfilled()) {
return Collections.emptyList();
}
final Collection resultingMatches = new ArrayList<>();
final List sortedRequests = sortByLoadingDescend(pendingRequests);
- LOG.debug(
- "Available slots: {}, sortedRequests: {}, taskExecutorsLoad: {}",
- slots,
- sortedRequests,
- taskExecutorsLoad);
+
+ logDebugInfo(slots, taskExecutorsLoad, sortedRequests);
+
Collection slotElements =
slots.stream().map(PhysicalSlotElement::new).collect(Collectors.toList());
final Map> profileSlots =
@@ -143,33 +149,51 @@ public Collection matchRequestsAndSlots(
final Map> taskExecutorSlots =
groupSlotsByTaskExecutor(slotElements);
for (PendingRequest request : sortedRequests) {
- Optional bestSlotEle =
- tryMatchPhysicalSlot(request, profileSlots, taskExecutorsLoad);
- if (bestSlotEle.isPresent()) {
- PhysicalSlotElement slotElement = bestSlotEle.get();
- updateReferenceAfterMatching(
- profileSlots,
- taskExecutorsLoad,
- taskExecutorSlots,
- slotElement,
- request.getLoading());
+ ResourceProfile requestProfile = request.getResourceProfile();
+ Optional bestSlotEleOpt =
+ tryMatchPhysicalSlot(
+ request, profileSlots, taskExecutorsLoad, resourceRequestPreMappings);
+ if (bestSlotEleOpt.isPresent()) {
+ PhysicalSlotElement slotElement = bestSlotEleOpt.get();
+ updateTaskExecutorsLoad(taskExecutorsLoad, request, slotElement);
+ updateReferenceRemainingSlots(profileSlots, taskExecutorSlots, slotElement);
+ resourceRequestPreMappings.decrease(
+ requestProfile, slotElement.getResourceProfile());
resultingMatches.add(RequestSlotMatch.createFor(request, slotElement.physicalSlot));
}
}
return resultingMatches;
}
+ private static void updateTaskExecutorsLoad(
+ Map taskExecutorsLoad,
+ PendingRequest request,
+ PhysicalSlotElement slotElement) {
+ taskExecutorsLoad.compute(
+ slotElement.getResourceID(),
+ (ignoredId, oldLoading) ->
+ Objects.isNull(oldLoading)
+ ? request.getLoading()
+ : oldLoading.merge(request.getLoading()));
+ }
+
+ private static void logDebugInfo(
+ Collection extends PhysicalSlot> slots,
+ Map taskExecutorsLoad,
+ List sortedRequests) {
+ LOG.debug(
+ "Available slots: {}, sortedRequests: {}, taskExecutorsLoad: {}",
+ slots,
+ sortedRequests,
+ taskExecutorsLoad);
+ }
+
private Map> groupSlotsByTaskExecutor(
Collection slotElements) {
return slotElements.stream()
.collect(
Collectors.groupingBy(
- physicalSlot ->
- physicalSlot
- .physicalSlot
- .getTaskManagerLocation()
- .getResourceID(),
- Collectors.toSet()));
+ PhysicalSlotElement::getResourceID, Collectors.toSet()));
}
private Map> getSlotCandidatesByProfile(
@@ -180,7 +204,7 @@ private Map> getSlotCand
new PhysicalSlotElementPriorityComparator(taskExecutorsLoad);
for (PhysicalSlotElement slotEle : slotElements) {
result.compute(
- slotEle.physicalSlot.getResourceProfile(),
+ slotEle.getResourceProfile(),
(resourceProfile, oldSlots) -> {
HeapPriorityQueue values =
Objects.isNull(oldSlots)
@@ -197,12 +221,17 @@ private Map> getSlotCand
private Optional tryMatchPhysicalSlot(
PendingRequest request,
Map> profileToSlotMap,
- Map taskExecutorsLoad) {
+ Map taskExecutorsLoad,
+ ResourceRequestPreMappings resourceRequestPreMappings) {
final ResourceProfile requestProfile = request.getResourceProfile();
final Set candidateProfiles =
profileToSlotMap.keySet().stream()
- .filter(slotProfile -> slotProfile.isMatching(requestProfile))
+ .filter(
+ slotProfile ->
+ slotProfile.isMatching(requestProfile)
+ && resourceRequestPreMappings.hasAvailableProfile(
+ requestProfile, slotProfile))
.collect(Collectors.toSet());
return candidateProfiles.stream()
@@ -216,25 +245,17 @@ private Optional tryMatchPhysicalSlot(
.min(new PhysicalSlotElementComparator(taskExecutorsLoad));
}
- private void updateReferenceAfterMatching(
+ private void updateReferenceRemainingSlots(
Map> profileSlots,
- Map taskExecutorsLoad,
Map> taskExecutorSlots,
- PhysicalSlotElement targetSlotElement,
- LoadingWeight loading) {
- final ResourceID tmID =
- targetSlotElement.physicalSlot.getTaskManagerLocation().getResourceID();
- // Update the loading for the target task executor.
- taskExecutorsLoad.compute(
- tmID,
- (ignoredId, oldLoading) ->
- Objects.isNull(oldLoading) ? loading : oldLoading.merge(loading));
+ PhysicalSlotElement targetSlotElement) {
+ final ResourceID tmID = targetSlotElement.getResourceID();
// Update the sorted set for slots that is located on the same task executor as targetSlot.
// Use Map#remove to avoid the ConcurrentModifyException.
final Set slotToReSort = taskExecutorSlots.remove(tmID);
for (PhysicalSlotElement slotEle : slotToReSort) {
HeapPriorityQueue slotsOfProfile =
- profileSlots.get(slotEle.physicalSlot.getResourceProfile());
+ profileSlots.get(slotEle.getResourceProfile());
// Re-add for the latest order.
slotsOfProfile.remove(slotEle);
if (!slotEle.equals(targetSlotElement)) {
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/InternalSourceSplitMetricGroup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/InternalSourceSplitMetricGroup.java
index 706606a159811..bc522d228b3f5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/InternalSourceSplitMetricGroup.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/InternalSourceSplitMetricGroup.java
@@ -48,6 +48,7 @@ public class InternalSourceSplitMetricGroup extends ProxyMetricGroup currentWatermark) {
super(parentMetricGroup);
this.clock = clock;
+ this.splitId = splitId;
splitWatermarkMetricGroup = parentMetricGroup.addGroup(SPLIT, splitId).addGroup(WATERMARK);
pausedTimePerSecond =
splitWatermarkMetricGroup.gauge(
@@ -75,12 +77,6 @@ private InternalSourceSplitMetricGroup(
MetricNames.SPLIT_CURRENT_WATERMARK, currentWatermark);
}
- public static InternalSourceSplitMetricGroup wrap(
- OperatorMetricGroup operatorMetricGroup, String splitId, Gauge currentWatermark) {
- return new InternalSourceSplitMetricGroup(
- operatorMetricGroup, SystemClock.getInstance(), splitId, currentWatermark);
- }
-
@VisibleForTesting
public static InternalSourceSplitMetricGroup mock(
MetricGroup metricGroup, String splitId, Gauge currentWatermark) {
@@ -88,7 +84,6 @@ public static InternalSourceSplitMetricGroup mock(
metricGroup, SystemClock.getInstance(), splitId, currentWatermark);
}
- @VisibleForTesting
public static InternalSourceSplitMetricGroup wrap(
OperatorMetricGroup operatorMetricGroup,
Clock clock,
@@ -118,7 +113,7 @@ public void markPaused() {
// If a split got paused it means it emitted records,
// hence it shouldn't be considered idle anymore
markNotIdle();
- LOG.warn("Split marked paused while still idle");
+ LOG.warn("[{}] Split marked paused while still idle", splitId);
}
this.pausedTimePerSecond.markStart();
}
@@ -129,7 +124,7 @@ public void markIdle() {
// If a split is marked idle, it has no records to emit.
// hence it shouldn't be considered paused anymore
markNotPaused();
- LOG.warn("Split marked idle while still paused");
+ LOG.warn("[{}] Split marked idle while still paused", splitId);
}
this.idleTimePerSecond.markStart();
}
@@ -208,4 +203,10 @@ public void onSplitFinished() {
public MetricGroup getSplitWatermarkMetricGroup() {
return splitWatermarkMetricGroup;
}
+
+ @VisibleForTesting
+ public void updateTimers() {
+ this.idleTimePerSecond.update();
+ this.pausedTimePerSecond.update();
+ }
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManager.java
index c19943cdaa1cd..f0045fe221c40 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManager.java
@@ -56,6 +56,7 @@
import java.util.HashSet;
import java.util.List;
import java.util.Map;
+import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.StringJoiner;
@@ -69,6 +70,8 @@
/** Implementation of {@link SlotManager} supporting fine-grained resource management. */
public class FineGrainedSlotManager implements SlotManager {
+ public static final Duration METRICS_UPDATE_INTERVAL = Duration.ofSeconds(1);
+
private static final Logger LOG = LoggerFactory.getLogger(FineGrainedSlotManager.class);
private final TaskManagerTracker taskManagerTracker;
@@ -114,6 +117,8 @@ public class FineGrainedSlotManager implements SlotManager {
@Nullable private ScheduledFuture> clusterReconciliationCheck;
+ @Nullable private ScheduledFuture> metricsUpdateFuture;
+
@Nullable private CompletableFuture requirementsCheckFuture;
@Nullable private CompletableFuture declareNeededResourceFuture;
@@ -124,6 +129,11 @@ public class FineGrainedSlotManager implements SlotManager {
/** True iff the component has been started. */
private boolean started;
+ /** Metrics. */
+ private long lastNumberFreeSlots;
+
+ private long lastNumberRegisteredSlots;
+
public FineGrainedSlotManager(
ScheduledExecutor scheduledExecutor,
SlotManagerConfiguration slotManagerConfiguration,
@@ -159,6 +169,7 @@ public FineGrainedSlotManager(
mainThreadExecutor = null;
clusterReconciliationCheck = null;
requirementsCheckFuture = null;
+ metricsUpdateFuture = null;
started = false;
}
@@ -227,10 +238,26 @@ public void start(
}
private void registerSlotManagerMetrics() {
- slotManagerMetricGroup.gauge(
- MetricNames.TASK_SLOTS_AVAILABLE, () -> (long) getNumberFreeSlots());
- slotManagerMetricGroup.gauge(
- MetricNames.TASK_SLOTS_TOTAL, () -> (long) getNumberRegisteredSlots());
+ // Because taskManagerTracker is not thread-safe, metrics must be updated periodically on
+ // the main thread to prevent concurrent modification issues.
+ metricsUpdateFuture =
+ scheduledExecutor.scheduleAtFixedRate(
+ this::updateMetrics,
+ 0L,
+ METRICS_UPDATE_INTERVAL.toMillis(),
+ TimeUnit.MILLISECONDS);
+
+ slotManagerMetricGroup.gauge(MetricNames.TASK_SLOTS_AVAILABLE, () -> lastNumberFreeSlots);
+ slotManagerMetricGroup.gauge(MetricNames.TASK_SLOTS_TOTAL, () -> lastNumberRegisteredSlots);
+ }
+
+ private void updateMetrics() {
+ Objects.requireNonNull(mainThreadExecutor)
+ .execute(
+ () -> {
+ lastNumberFreeSlots = getNumberFreeSlots();
+ lastNumberRegisteredSlots = getNumberRegisteredSlots();
+ });
}
/** Suspends the component. This clears the internal state of the slot manager. */
@@ -250,6 +277,12 @@ public void suspend() {
clusterReconciliationCheck = null;
}
+ // stop the metrics updates
+ if (metricsUpdateFuture != null) {
+ metricsUpdateFuture.cancel(false);
+ metricsUpdateFuture = null;
+ }
+
slotStatusSyncer.close();
taskManagerTracker.clear();
resourceTracker.clear();
diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java b/flink-runtime/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java
index 09b12007d3349..05b279b66c17d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java
@@ -180,6 +180,7 @@ public class SourceOperator extends AbstractStr
private final Map splitCurrentWatermarks = new HashMap<>();
private final Set currentlyPausedSplits = new HashSet<>();
+ private final Set currentlyIdleSplits = new HashSet<>();
private final Map splitMetricGroups = new HashMap<>();
@@ -371,6 +372,7 @@ protected InternalSourceSplitMetricGroup getOrCreateSplitMetricGroup(String spli
InternalSourceSplitMetricGroup splitMetricGroup =
InternalSourceSplitMetricGroup.wrap(
getMetricGroup(),
+ processingTimeService.getClock(),
splitId,
() ->
splitCurrentWatermarks.getOrDefault(
@@ -723,6 +725,15 @@ public void updateCurrentEffectiveWatermark(long watermark) {
@Override
public void updateCurrentSplitWatermark(String splitId, long watermark) {
splitCurrentWatermarks.put(splitId, watermark);
+ if (!currentlyIdleSplits.contains(splitId)) {
+ maybePauseSplit(splitId);
+ }
+ }
+
+ private void maybePauseSplit(String splitId) {
+ final long watermark =
+ splitCurrentWatermarks.getOrDefault(
+ splitId, Watermark.UNINITIALIZED.getTimestamp());
if (watermark > currentMaxDesiredWatermark && !currentlyPausedSplits.contains(splitId)) {
pauseOrResumeSplits(Collections.singletonList(splitId), Collections.emptyList());
currentlyPausedSplits.add(splitId);
@@ -731,10 +742,22 @@ public void updateCurrentSplitWatermark(String splitId, long watermark) {
@Override
public void updateCurrentSplitIdle(String splitId, boolean idle) {
+ final InternalSourceSplitMetricGroup splitMetricGroup =
+ this.getOrCreateSplitMetricGroup(splitId);
+ if (idle == currentlyIdleSplits.contains(splitId)) {
+ return;
+ }
if (idle) {
- this.getOrCreateSplitMetricGroup(splitId).markIdle();
+ LOG.info("[{}] Marking split idle", splitId);
+ currentlyIdleSplits.add(splitId);
+ splitMetricGroup.markIdle();
} else {
- this.getOrCreateSplitMetricGroup(splitId).markNotIdle();
+ LOG.info("[{}] Marking split not idle", splitId);
+ currentlyIdleSplits.remove(splitId);
+ splitMetricGroup.markNotIdle();
+ // Since we skipped alignment check
+ // for this split while it was idle:
+ maybePauseSplit(splitId);
}
}
@@ -743,6 +766,7 @@ public void splitFinished(String splitId) {
splitCurrentWatermarks.remove(splitId);
getOrCreateSplitMetricGroup(splitId).onSplitFinished();
this.splitMetricGroups.remove(splitId);
+ currentlyIdleSplits.remove(splitId);
}
/**
@@ -756,6 +780,9 @@ private void checkSplitWatermarkAlignment() {
Collection splitsToResume = new ArrayList<>();
splitCurrentWatermarks.forEach(
(splitId, splitWatermark) -> {
+ if (currentlyIdleSplits.contains(splitId)) {
+ return;
+ }
if (splitWatermark > currentMaxDesiredWatermark) {
splitsToPause.add(splitId);
} else if (currentlyPausedSplits.contains(splitId)) {
diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/partitioner/CustomPartitionerWrapper.java b/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/partitioner/CustomPartitionerWrapper.java
index e3c5cb442a2b1..49ec4c5b6ea47 100644
--- a/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/partitioner/CustomPartitionerWrapper.java
+++ b/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/partitioner/CustomPartitionerWrapper.java
@@ -44,6 +44,7 @@ public class CustomPartitionerWrapper extends StreamPartitioner {
public CustomPartitionerWrapper(Partitioner partitioner, KeySelector keySelector) {
this.partitioner = partitioner;
this.keySelector = keySelector;
+ disableUnalignedCheckpoints();
}
@Override
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheDeleteTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheDeleteTest.java
index c8230bd0015ff..10c1ae7551346 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheDeleteTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheDeleteTest.java
@@ -24,7 +24,6 @@
import org.apache.flink.util.OperatingSystem;
import org.apache.flink.util.concurrent.FutureUtils;
-import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
@@ -194,13 +193,11 @@ private void testDeleteTransientAlreadyDeleted(@Nullable final JobID jobId) thro
}
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testDeleteTransientLocalFailsNoJob() throws IOException, InterruptedException {
testDeleteTransientLocalFails(null);
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testDeleteTransientLocalFailsForJob() throws IOException, InterruptedException {
testDeleteTransientLocalFails(new JobID());
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheGetTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheGetTest.java
index dc176b04d8df6..c336a8264f835 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheGetTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheGetTest.java
@@ -25,7 +25,6 @@
import org.apache.flink.util.concurrent.FutureUtils;
import org.apache.commons.lang3.exception.ExceptionUtils;
-import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
@@ -170,19 +169,16 @@ private void testGetFailsDuringLookup(
}
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testGetFailsIncomingNoJob() throws IOException {
testGetFailsIncoming(null, TRANSIENT_BLOB);
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testGetFailsIncomingForJob() throws IOException {
testGetFailsIncoming(new JobID(), TRANSIENT_BLOB);
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testGetFailsIncomingForJobHa() throws IOException {
testGetFailsIncoming(new JobID(), PERMANENT_BLOB);
@@ -268,19 +264,16 @@ private void testGetFailsIncoming(@Nullable final JobID jobId, BlobKey.BlobType
}
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testGetTransientFailsStoreNoJob() throws IOException, InterruptedException {
testGetFailsStore(null, TRANSIENT_BLOB);
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testGetTransientFailsStoreForJob() throws IOException, InterruptedException {
testGetFailsStore(new JobID(), TRANSIENT_BLOB);
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testGetPermanentFailsStoreForJob() throws IOException, InterruptedException {
testGetFailsStore(new JobID(), PERMANENT_BLOB);
@@ -402,13 +395,11 @@ void testGetFailsHaStoreForJobHa() throws IOException {
}
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testGetTransientRemoteDeleteFailsNoJob() throws IOException {
testGetTransientRemoteDeleteFails(null);
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testGetTransientRemoteDeleteFailsForJob() throws IOException {
testGetTransientRemoteDeleteFails(new JobID());
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCachePutTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCachePutTest.java
index 1c81a4e22d7f8..ce76e50901b32 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCachePutTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCachePutTest.java
@@ -28,7 +28,6 @@
import org.apache.flink.util.concurrent.FutureUtils;
import org.apache.commons.io.FileUtils;
-import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
@@ -521,19 +520,16 @@ private void testPutChunkedStreamTransientSuccessfulGet(
// --------------------------------------------------------------------------------------------
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testPutBufferFailsNoJob() throws IOException {
testPutBufferFails(null, TRANSIENT_BLOB);
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testPutBufferFailsForJob() throws IOException {
testPutBufferFails(new JobID(), TRANSIENT_BLOB);
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testPutBufferFailsForJobHa() throws IOException {
testPutBufferFails(new JobID(), PERMANENT_BLOB);
@@ -574,19 +570,16 @@ private void testPutBufferFails(@Nullable final JobID jobId, BlobKey.BlobType bl
}
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testPutBufferFailsIncomingNoJob() throws IOException {
testPutBufferFailsIncoming(null, TRANSIENT_BLOB);
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testPutBufferFailsIncomingForJob() throws IOException {
testPutBufferFailsIncoming(new JobID(), TRANSIENT_BLOB);
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testPutBufferFailsIncomingForJobHa() throws IOException {
testPutBufferFailsIncoming(new JobID(), PERMANENT_BLOB);
@@ -638,19 +631,16 @@ private void testPutBufferFailsIncoming(@Nullable final JobID jobId, BlobKey.Blo
}
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testPutBufferFailsStoreNoJob() throws IOException {
testPutBufferFailsStore(null, TRANSIENT_BLOB);
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testPutBufferFailsStoreForJob() throws IOException {
testPutBufferFailsStore(new JobID(), TRANSIENT_BLOB);
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testPutBufferFailsStoreForJobHa() throws IOException {
testPutBufferFailsStore(new JobID(), PERMANENT_BLOB);
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java
index 83b9f5fa49cb4..5d604183fa7fb 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java
@@ -24,7 +24,6 @@
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.concurrent.FutureUtils;
-import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
@@ -189,19 +188,16 @@ private void testDeleteBlobAlreadyDeleted(
}
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testDeleteTransientFailsNoJob() throws IOException {
testDeleteBlobFails(null, TRANSIENT_BLOB);
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testDeleteTransientFailsForJob() throws IOException {
testDeleteBlobFails(new JobID(), TRANSIENT_BLOB);
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testDeletePermanentFailsForJob() throws IOException {
testDeleteBlobFails(new JobID(), PERMANENT_BLOB);
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerGetTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerGetTest.java
index 7efae9ccfe018..8f50909bc0055 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerGetTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerGetTest.java
@@ -30,7 +30,6 @@
import org.apache.flink.util.concurrent.FutureUtils;
import org.apache.commons.io.FileUtils;
-import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
@@ -146,7 +145,6 @@ private void testGetFailsDuringLookup(
* Retrieves a BLOB from the HA store to a {@link BlobServer} which cannot create incoming
* files. File transfers should fail.
*/
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testGetFailsIncomingForJobHa() throws IOException {
assumeThat(OperatingSystem.isWindows()).as("setWritable doesn't work on Windows").isFalse();
@@ -219,7 +217,6 @@ void testGetFailsIncomingForJobHa() throws IOException {
* Retrieves a BLOB from the HA store to a {@link BlobServer} which cannot create the final
* storage file. File transfers should fail.
*/
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testGetFailsStoreForJobHa() throws IOException {
assumeThat(OperatingSystem.isWindows()).as("setWritable doesn't work on Windows").isFalse();
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerPutTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerPutTest.java
index 4ba0fb1de536e..1f1f26cedc0cb 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerPutTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerPutTest.java
@@ -29,7 +29,6 @@
import org.apache.flink.util.concurrent.FutureUtils;
import org.apache.commons.io.FileUtils;
-import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
@@ -403,19 +402,16 @@ private void testPutChunkedStreamSuccessfulGet(
// --------------------------------------------------------------------------------------------
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testPutBufferFailsNoJob() throws IOException {
testPutBufferFails(null, TRANSIENT_BLOB);
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testPutBufferFailsForJob() throws IOException {
testPutBufferFails(new JobID(), TRANSIENT_BLOB);
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testPutBufferFailsForJobHa() throws IOException {
testPutBufferFails(new JobID(), PERMANENT_BLOB);
@@ -453,19 +449,16 @@ private void testPutBufferFails(@Nullable final JobID jobId, BlobKey.BlobType bl
}
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testPutBufferFailsIncomingNoJob() throws IOException {
testPutBufferFailsIncoming(null, TRANSIENT_BLOB);
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testPutBufferFailsIncomingForJob() throws IOException {
testPutBufferFailsIncoming(new JobID(), TRANSIENT_BLOB);
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testPutBufferFailsIncomingForJobHa() throws IOException {
testPutBufferFailsIncoming(new JobID(), PERMANENT_BLOB);
@@ -514,19 +507,16 @@ private void testPutBufferFailsIncoming(@Nullable final JobID jobId, BlobKey.Blo
}
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testPutBufferFailsStoreNoJob() throws IOException {
testPutBufferFailsStore(null, TRANSIENT_BLOB);
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testPutBufferFailsStoreForJob() throws IOException {
testPutBufferFailsStore(new JobID(), TRANSIENT_BLOB);
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testPutBufferFailsStoreForJobHa() throws IOException {
testPutBufferFailsStore(new JobID(), PERMANENT_BLOB);
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobUtilsNonWritableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobUtilsNonWritableTest.java
index ae673c2b20290..5101001216548 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobUtilsNonWritableTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobUtilsNonWritableTest.java
@@ -24,7 +24,6 @@
import org.apache.flink.util.OperatingSystem;
import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
@@ -37,7 +36,6 @@
import static org.assertj.core.api.Assumptions.assumeThat;
/** Tests for {@link BlobUtils} working on non-writable directories. */
-@Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
class BlobUtilsNonWritableTest {
private static final String CANNOT_CREATE_THIS = "cannot-create-this";
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java
index 9fe89f97cf5de..f269b77543b5e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java
@@ -27,7 +27,6 @@
import org.apache.flink.runtime.blob.PermanentBlobKey;
import org.apache.flink.runtime.blob.PermanentBlobService;
import org.apache.flink.runtime.blob.VoidBlobStore;
-import org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser;
import org.apache.flink.util.FlinkUserCodeClassLoaders;
import org.apache.flink.util.OperatingSystem;
import org.apache.flink.util.TestLogger;
@@ -35,7 +34,6 @@
import org.junit.Rule;
import org.junit.Test;
-import org.junit.experimental.categories.Category;
import org.junit.rules.TemporaryFolder;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
@@ -325,7 +323,6 @@ public void testLibraryCacheManagerCleanup() throws Exception {
}
@Test
- @Category(FailsInGHAContainerWithRootUser.class)
public void testRegisterAndDownload() throws IOException {
assumeTrue(!OperatingSystem.isWindows()); // setWritable doesn't work on Windows.
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/PreferredAllocationRequestSlotMatchingStrategyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/PreferredAllocationRequestSlotMatchingStrategyTest.java
index cbb8d2cda4a10..3a047d8b45bbf 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/PreferredAllocationRequestSlotMatchingStrategyTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/PreferredAllocationRequestSlotMatchingStrategyTest.java
@@ -221,7 +221,7 @@ private static TestingPhysicalSlot createSlotAndGrainProfile(TaskManagerLocation
return createSlot(finedGrainProfile, new AllocationID(), tmLocation);
}
- private static TestingPhysicalSlot createSlot(
+ static TestingPhysicalSlot createSlot(
ResourceProfile profile, AllocationID allocationId, TaskManagerLocation tmLocation) {
return TestingPhysicalSlot.builder()
.withAllocationID(allocationId)
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/ResourceRequestPreMappingsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/ResourceRequestPreMappingsTest.java
new file mode 100644
index 0000000000000..c2bf3fd691b5f
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/ResourceRequestPreMappingsTest.java
@@ -0,0 +1,381 @@
+/*
+ * 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.runtime.jobmaster.slotpool;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.scheduler.adaptive.allocator.TestingSlot;
+import org.apache.flink.runtime.scheduler.loading.DefaultLoadingWeight;
+import org.apache.flink.util.Preconditions;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Test for {@link ResourceRequestPreMappings}. */
+class ResourceRequestPreMappingsTest {
+
+ private static final ResourceProfile smallFineGrainedResourceProfile =
+ ResourceProfile.newBuilder().setManagedMemoryMB(10).build();
+
+ private static final ResourceProfile bigGrainedResourceProfile =
+ ResourceProfile.newBuilder().setManagedMemoryMB(20).build();
+
+ @Test
+ void testIncludeInvalidProfileOfRequestOrResource() {
+ // For invalid resource.
+ ResourceProfile[] profiles =
+ new ResourceProfile[] {ResourceProfile.UNKNOWN, ResourceProfile.ZERO};
+ for (ResourceProfile profile : profiles) {
+ assertThatThrownBy(
+ () ->
+ ResourceRequestPreMappings.createFrom(
+ Collections.emptyList(), newTestingSlots(profile)))
+ .isInstanceOf(IllegalStateException.class);
+ }
+
+ // For invalid request.
+ profiles = new ResourceProfile[] {ResourceProfile.ANY, ResourceProfile.ZERO};
+ for (ResourceProfile profile : profiles) {
+ assertThatThrownBy(
+ () ->
+ ResourceRequestPreMappings.createFrom(
+ newPendingRequests(profile), Collections.emptyList()))
+ .isInstanceOf(IllegalStateException.class);
+ }
+ }
+
+ @Test
+ void testBuildWhenUnavailableTotalResourcesOrEmptyRequestsResources() {
+ // Testing for unavailable total resource
+ ResourceRequestPreMappings preMappings =
+ ResourceRequestPreMappings.createFrom(
+ newPendingRequests(ResourceProfile.UNKNOWN), Collections.emptyList());
+ assertThat(preMappings.isMatchingFulfilled()).isFalse();
+ assertThat(preMappings.getBaseRequiredResourcePreMappings()).isEmpty();
+
+ // Testing for empty slots or requests
+ preMappings =
+ ResourceRequestPreMappings.createFrom(
+ Collections.emptyList(), Collections.emptyList());
+ assertNotMatchable(preMappings);
+ }
+
+ @Test
+ void testBuildWhenMissingResourceToMatchFineGrainedRequest() {
+
+ // Testing for missing available fine-grained resources when only fine-grained request
+ ResourceRequestPreMappings preMappings =
+ ResourceRequestPreMappings.createFrom(
+ newPendingRequests(
+ smallFineGrainedResourceProfile,
+ smallFineGrainedResourceProfile,
+ bigGrainedResourceProfile),
+ newTestingSlots(
+ smallFineGrainedResourceProfile,
+ smallFineGrainedResourceProfile,
+ smallFineGrainedResourceProfile));
+ assertNotMatchable(preMappings);
+
+ // Testing for missing available fine-grained resources when fine-grained and unknown
+ // requests.
+ preMappings =
+ ResourceRequestPreMappings.createFrom(
+ newPendingRequests(
+ ResourceProfile.UNKNOWN,
+ smallFineGrainedResourceProfile,
+ bigGrainedResourceProfile),
+ newTestingSlots(
+ smallFineGrainedResourceProfile,
+ smallFineGrainedResourceProfile,
+ smallFineGrainedResourceProfile));
+ assertNotMatchable(preMappings);
+ }
+
+ @Test
+ void testBuildSuccessfullyThatFinedGrainedMatchedExactly() {
+ ResourceRequestPreMappings preMappings =
+ ResourceRequestPreMappings.createFrom(
+ newPendingRequests(
+ smallFineGrainedResourceProfile,
+ smallFineGrainedResourceProfile,
+ bigGrainedResourceProfile),
+ newTestingSlots(
+ bigGrainedResourceProfile,
+ smallFineGrainedResourceProfile,
+ smallFineGrainedResourceProfile,
+ smallFineGrainedResourceProfile));
+ assertThat(preMappings.isMatchingFulfilled()).isTrue();
+ assertThat(preMappings.getBaseRequiredResourcePreMappings())
+ .hasSize(2)
+ .contains(
+ new AbstractMap.SimpleEntry<>(
+ smallFineGrainedResourceProfile,
+ new HashMap<>() {
+ {
+ put(smallFineGrainedResourceProfile, 2);
+ }
+ }),
+ new AbstractMap.SimpleEntry<>(
+ bigGrainedResourceProfile,
+ new HashMap<>() {
+ {
+ put(bigGrainedResourceProfile, 1);
+ }
+ }));
+ assertThat(preMappings.getRemainingFlexibleResources())
+ .contains(new AbstractMap.SimpleEntry<>(smallFineGrainedResourceProfile, 1));
+ }
+
+ @Test
+ void testBuildSuccessfullyThatFinedGrainedToMatchedUnknownRequests() {
+
+ // Testing for available all resources and no UNKNOWN required resource.
+ ResourceRequestPreMappings preMappings =
+ ResourceRequestPreMappings.createFrom(
+ newPendingRequests(
+ ResourceProfile.UNKNOWN,
+ ResourceProfile.UNKNOWN,
+ smallFineGrainedResourceProfile,
+ bigGrainedResourceProfile),
+ newTestingSlots(
+ bigGrainedResourceProfile,
+ bigGrainedResourceProfile,
+ bigGrainedResourceProfile,
+ ResourceProfile.ANY,
+ smallFineGrainedResourceProfile,
+ smallFineGrainedResourceProfile));
+ assertThat(preMappings.isMatchingFulfilled()).isTrue();
+ assertThat(preMappings.getBaseRequiredResourcePreMappings())
+ .hasSize(3)
+ .contains(
+ new AbstractMap.SimpleEntry<>(
+ smallFineGrainedResourceProfile,
+ new HashMap<>() {
+ {
+ put(smallFineGrainedResourceProfile, 1);
+ }
+ }),
+ new AbstractMap.SimpleEntry<>(
+ bigGrainedResourceProfile,
+ new HashMap<>() {
+ {
+ put(bigGrainedResourceProfile, 1);
+ }
+ }));
+ Map unknownBaseMapping =
+ preMappings.getBaseRequiredResourcePreMappings().get(ResourceProfile.UNKNOWN);
+ assertThat(unknownBaseMapping.values().stream().reduce(0, Integer::sum)).isEqualTo(2);
+ assertThat(
+ preMappings.getRemainingFlexibleResources().values().stream()
+ .reduce(0, Integer::sum))
+ .isEqualTo(2);
+ }
+
+ @Test
+ void testBuildSuccessfullyThatAnyToMatchedUnknownAndFineGrainedRequests() {
+
+ // Testing for available all resources and no UNKNOWN required resource.
+ ResourceRequestPreMappings preMappings =
+ ResourceRequestPreMappings.createFrom(
+ newPendingRequests(
+ ResourceProfile.UNKNOWN,
+ ResourceProfile.UNKNOWN,
+ smallFineGrainedResourceProfile,
+ smallFineGrainedResourceProfile,
+ bigGrainedResourceProfile,
+ bigGrainedResourceProfile),
+ newTestingSlots(
+ bigGrainedResourceProfile,
+ smallFineGrainedResourceProfile,
+ ResourceProfile.ANY,
+ ResourceProfile.ANY,
+ ResourceProfile.ANY,
+ ResourceProfile.ANY));
+ assertThat(preMappings.isMatchingFulfilled()).isTrue();
+ assertThat(preMappings.getBaseRequiredResourcePreMappings())
+ .hasSize(3)
+ .contains(
+ new AbstractMap.SimpleEntry<>(
+ smallFineGrainedResourceProfile,
+ new HashMap<>() {
+ {
+ put(smallFineGrainedResourceProfile, 1);
+ put(ResourceProfile.ANY, 1);
+ }
+ }),
+ new AbstractMap.SimpleEntry<>(
+ bigGrainedResourceProfile,
+ new HashMap<>() {
+ {
+ put(bigGrainedResourceProfile, 1);
+ put(ResourceProfile.ANY, 1);
+ }
+ }),
+ new AbstractMap.SimpleEntry<>(
+ ResourceProfile.UNKNOWN,
+ new HashMap<>() {
+ {
+ put(ResourceProfile.ANY, 2);
+ }
+ }));
+ assertThat(
+ preMappings.getRemainingFlexibleResources().values().stream()
+ .reduce(0, Integer::sum))
+ .isZero();
+ }
+
+ @Test
+ void testHasAvailableProfile() {
+ ResourceRequestPreMappings mappings =
+ ResourceRequestPreMappings.createFrom(
+ newPendingRequests(ResourceProfile.UNKNOWN, ResourceProfile.UNKNOWN),
+ newTestingSlots(
+ smallFineGrainedResourceProfile,
+ smallFineGrainedResourceProfile,
+ smallFineGrainedResourceProfile));
+
+ // Testing available resource in flexible resources
+ assertThat(
+ mappings.hasAvailableProfile(
+ smallFineGrainedResourceProfile, smallFineGrainedResourceProfile))
+ .isTrue();
+ assertThat(
+ mappings.hasAvailableProfile(
+ smallFineGrainedResourceProfile, bigGrainedResourceProfile))
+ .isFalse();
+
+ // Testing available resource in base mapping resources
+ assertThat(
+ mappings.hasAvailableProfile(
+ ResourceProfile.UNKNOWN, smallFineGrainedResourceProfile))
+ .isTrue();
+ assertThat(mappings.hasAvailableProfile(ResourceProfile.UNKNOWN, bigGrainedResourceProfile))
+ .isFalse();
+ }
+
+ @Test
+ void testDecrease() {
+ // Testing decrease resource in base mapping
+ ResourceRequestPreMappings mappings =
+ ResourceRequestPreMappings.createFrom(
+ newPendingRequests(ResourceProfile.UNKNOWN, ResourceProfile.UNKNOWN),
+ newTestingSlots(
+ smallFineGrainedResourceProfile,
+ smallFineGrainedResourceProfile,
+ smallFineGrainedResourceProfile));
+
+ // Testing decrease resource in base mapping resources successfully
+ mappings.decrease(ResourceProfile.UNKNOWN, smallFineGrainedResourceProfile);
+ assertThat(
+ mappings.getAvailableResourceCntOfBasePreMappings(
+ ResourceProfile.UNKNOWN, smallFineGrainedResourceProfile))
+ .isOne();
+ // Testing decrease resource in base mapping resources failed
+ assertThatThrownBy(
+ () ->
+ mappings.decrease(
+ smallFineGrainedResourceProfile,
+ smallFineGrainedResourceProfile))
+ .isInstanceOf(IllegalStateException.class);
+
+ // Testing decrease resource in flexible resources
+ ResourceRequestPreMappings mappings2 =
+ ResourceRequestPreMappings.createFrom(
+ true,
+ new HashMap<>() {
+ {
+ put(
+ ResourceProfile.UNKNOWN,
+ new HashMap<>() {
+ {
+ put(smallFineGrainedResourceProfile, 2);
+ }
+ });
+ }
+ },
+ new HashMap<>() {
+ {
+ put(smallFineGrainedResourceProfile, 1);
+ put(bigGrainedResourceProfile, 2);
+ }
+ });
+ // Testing decrease resource in flexible resources successfully
+ mappings2.decrease(ResourceProfile.UNKNOWN, bigGrainedResourceProfile);
+ assertThat(
+ mappings2.getAvailableResourceCntOfRemainingFlexibleMapping(
+ bigGrainedResourceProfile))
+ .isOne();
+ assertThat(
+ mappings2.getAvailableResourceCntOfBasePreMappings(
+ ResourceProfile.UNKNOWN, smallFineGrainedResourceProfile))
+ .isOne();
+ assertThat(
+ mappings2.getAvailableResourceCntOfRemainingFlexibleMapping(
+ smallFineGrainedResourceProfile))
+ .isEqualTo(2);
+
+ // Testing decrease resource in flexible resources failed
+ mappings2.decrease(ResourceProfile.UNKNOWN, smallFineGrainedResourceProfile);
+ assertThatThrownBy(
+ () ->
+ mappings2.decrease(
+ ResourceProfile.UNKNOWN, smallFineGrainedResourceProfile))
+ .isInstanceOf(IllegalStateException.class);
+ }
+
+ private List newPendingRequests(ResourceProfile... requiredProfiles) {
+ ArrayList pendingRequests = new ArrayList<>();
+ if (requiredProfiles == null || requiredProfiles.length == 0) {
+ return pendingRequests;
+ }
+ for (ResourceProfile requiredProfile : requiredProfiles) {
+ pendingRequests.add(
+ PendingRequest.createNormalRequest(
+ new SlotRequestId(),
+ Preconditions.checkNotNull(requiredProfile),
+ DefaultLoadingWeight.EMPTY,
+ Collections.emptyList()));
+ }
+ return pendingRequests;
+ }
+
+ private List newTestingSlots(ResourceProfile... slotProfiles) {
+ ArrayList slots = new ArrayList<>();
+ if (slotProfiles == null || slotProfiles.length == 0) {
+ return slots;
+ }
+ for (ResourceProfile slotProfile : slotProfiles) {
+ slots.add(new TestingSlot(Preconditions.checkNotNull(slotProfile)));
+ }
+ return slots;
+ }
+
+ private void assertNotMatchable(ResourceRequestPreMappings preMappings) {
+ assertThat(preMappings.isMatchingFulfilled()).isFalse();
+ assertThat(preMappings.getBaseRequiredResourcePreMappings()).isEmpty();
+ }
+}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/TasksBalancedRequestSlotMatchingStrategyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/TasksBalancedRequestSlotMatchingStrategyTest.java
new file mode 100644
index 0000000000000..706c3c8fefe42
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/TasksBalancedRequestSlotMatchingStrategyTest.java
@@ -0,0 +1,114 @@
+/*
+ * 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.runtime.jobmaster.slotpool;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.scheduler.TestingPhysicalSlot;
+import org.apache.flink.runtime.scheduler.loading.DefaultLoadingWeight;
+import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+
+import static org.apache.flink.runtime.jobmaster.slotpool.PreferredAllocationRequestSlotMatchingStrategyTest.createSlot;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Testing for {@link TasksBalancedRequestSlotMatchingStrategy}. */
+class TasksBalancedRequestSlotMatchingStrategyTest {
+
+ private static final ResourceProfile smallFineGrainedProfile =
+ ResourceProfile.newBuilder().setCpuCores(1d).build();
+ private static final ResourceProfile bigFineGrainedProfile =
+ ResourceProfile.newBuilder().setCpuCores(2d).build();
+
+ private static final TaskManagerLocation tmLocation1 = new LocalTaskManagerLocation();
+ private static final TaskManagerLocation tmLocation2 = new LocalTaskManagerLocation();
+
+ @Test
+ void testMatchRequestsAndSlotsRiskOfFineGrainedResourcesMatchedToUnknownProfile() {
+ // The case is aiming to check when the numbers of requests and resources are equals but
+ // having the risk of matching resources that would be matched with fine-grained request
+ // with ResourceProfile>UNKNOWN.
+ final Collection pendingRequests =
+ Arrays.asList(
+ createRequest(ResourceProfile.UNKNOWN, 100),
+ createRequest(bigFineGrainedProfile, 1));
+ List slots =
+ Arrays.asList(
+ createSlot(bigFineGrainedProfile, new AllocationID(), tmLocation1),
+ createSlot(smallFineGrainedProfile, new AllocationID(), tmLocation2));
+ final Collection requestSlotMatches =
+ TasksBalancedRequestSlotMatchingStrategy.INSTANCE.matchRequestsAndSlots(
+ slots,
+ pendingRequests,
+ new HashMap<>() {
+ {
+ put(tmLocation1.getResourceID(), DefaultLoadingWeight.EMPTY);
+ put(tmLocation2.getResourceID(), new DefaultLoadingWeight(9));
+ }
+ });
+ assertThat(requestSlotMatches).hasSize(2);
+ }
+
+ @Test
+ void testMatchRequestsAndSlotsMissingFineGrainedResources() {
+
+ PendingRequest requestWithBigProfile = createRequest(bigFineGrainedProfile, 6);
+ PendingRequest requestWithUnknownProfile = createRequest(ResourceProfile.UNKNOWN, 6);
+ PendingRequest requestWithSmallProfile = createRequest(smallFineGrainedProfile, 6);
+
+ final Collection pendingRequests =
+ Arrays.asList(
+ requestWithSmallProfile, requestWithUnknownProfile, requestWithBigProfile);
+ List slots =
+ Arrays.asList(
+ createSlot(
+ bigFineGrainedProfile,
+ new AllocationID(),
+ new LocalTaskManagerLocation()),
+ createSlot(
+ bigFineGrainedProfile,
+ new AllocationID(),
+ new LocalTaskManagerLocation()),
+ createSlot(
+ bigFineGrainedProfile,
+ new AllocationID(),
+ new LocalTaskManagerLocation()));
+ final Collection requestSlotMatches =
+ TasksBalancedRequestSlotMatchingStrategy.INSTANCE.matchRequestsAndSlots(
+ slots, pendingRequests, new HashMap<>());
+ assertThat(requestSlotMatches).isEmpty();
+ }
+
+ private static PendingRequest createRequest(ResourceProfile requestProfile, float loading) {
+ return PendingRequest.createNormalRequest(
+ new SlotRequestId(),
+ requestProfile,
+ new DefaultLoadingWeight(loading),
+ Collections.emptyList());
+ }
+}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/net/SSLUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/net/SSLUtilsTest.java
index 183665fcb6a3c..a449c1c616e85 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/net/SSLUtilsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/net/SSLUtilsTest.java
@@ -518,11 +518,6 @@ public static String getRestCertificateFingerprint(
private static void addSslProviderConfig(Configuration config, String sslProvider) {
if (sslProvider.equalsIgnoreCase("OPENSSL")) {
OpenSsl.ensureAvailability();
-
- // Flink's default algorithm set is not available for openSSL - choose a different one:
- config.set(
- SecurityOptions.SSL_ALGORITHMS,
- "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384");
}
config.set(SecurityOptions.SSL_PROVIDER, sslProvider);
}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerTest.java
index a19f42177b14d..5f374f2f9d162 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerTest.java
@@ -20,12 +20,14 @@
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.metrics.Gauge;
import org.apache.flink.runtime.clusterframework.types.AllocationID;
import org.apache.flink.runtime.clusterframework.types.ResourceID;
import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
import org.apache.flink.runtime.clusterframework.types.SlotID;
import org.apache.flink.runtime.instance.InstanceID;
import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
import org.apache.flink.runtime.metrics.MetricRegistry;
import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
import org.apache.flink.runtime.metrics.util.TestingMetricRegistry;
@@ -38,6 +40,7 @@
import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGateway;
import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor;
import org.apache.flink.util.function.ThrowingConsumer;
import org.junit.jupiter.api.Test;
@@ -50,6 +53,7 @@
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Consumer;
import static org.apache.flink.core.testutils.FlinkAssertions.assertThatFuture;
@@ -1058,4 +1062,100 @@ void testClearResourceRequirementsWithPendingTaskManager() throws Exception {
}
};
}
+
+ @Test
+ void testMetricsUpdate() throws Exception {
+ final AtomicReference> slotsAvailableGauge = new AtomicReference<>();
+ final AtomicReference> slotsTotalGauge = new AtomicReference<>();
+
+ final MetricRegistry metricRegistry =
+ TestingMetricRegistry.builder()
+ .setRegisterConsumer(
+ (metric, name, group) -> {
+ if (name.equals(MetricNames.TASK_SLOTS_AVAILABLE)) {
+ slotsAvailableGauge.set((Gauge) metric);
+ } else if (name.equals(MetricNames.TASK_SLOTS_TOTAL)) {
+ slotsTotalGauge.set((Gauge) metric);
+ }
+ })
+ .build();
+
+ final Context context = new Context();
+ context.setSlotManagerMetricGroup(
+ SlotManagerMetricGroup.create(metricRegistry, "localhost"));
+ final ManuallyTriggeredScheduledExecutor scheduledExecutor =
+ new ManuallyTriggeredScheduledExecutor();
+ context.setScheduledExecutor(scheduledExecutor);
+ final TaskExecutorConnection executorConnection1 = createTaskExecutorConnection();
+ final TaskExecutorConnection executorConnection2 = createTaskExecutorConnection();
+
+ context.runTest(
+ () -> {
+ assertThat(slotsAvailableGauge.get().getValue()).isEqualTo(0);
+ assertThat(slotsTotalGauge.get().getValue()).isEqualTo(0);
+
+ final CompletableFuture
+ registerTaskManagerFuture1 = new CompletableFuture<>();
+ context.runInMainThreadAndWait(
+ () ->
+ registerTaskManagerFuture1.complete(
+ context.getSlotManager()
+ .registerTaskManager(
+ executorConnection1,
+ new SlotReport(),
+ DEFAULT_TOTAL_RESOURCE_PROFILE,
+ DEFAULT_SLOT_RESOURCE_PROFILE)));
+ assertThat(assertFutureCompleteAndReturn(registerTaskManagerFuture1))
+ .isEqualTo(SlotManager.RegistrationResult.SUCCESS);
+
+ final CompletableFuture
+ registerTaskManagerFuture2 = new CompletableFuture<>();
+ context.runInMainThreadAndWait(
+ () ->
+ registerTaskManagerFuture2.complete(
+ context.getSlotManager()
+ .registerTaskManager(
+ executorConnection2,
+ new SlotReport(
+ createAllocatedSlotStatus(
+ new JobID(),
+ new AllocationID(),
+ DEFAULT_SLOT_RESOURCE_PROFILE)),
+ DEFAULT_TOTAL_RESOURCE_PROFILE,
+ DEFAULT_SLOT_RESOURCE_PROFILE)));
+ assertThat(assertFutureCompleteAndReturn(registerTaskManagerFuture2))
+ .isEqualTo(SlotManager.RegistrationResult.SUCCESS);
+
+ // triggers the metric update task on the main thread and waits for the main
+ // thread to process queued up callbacks
+ scheduledExecutor.triggerPeriodicScheduledTasks();
+ context.runInMainThreadAndWait(() -> {});
+
+ assertThat(slotsTotalGauge.get().getValue())
+ .isEqualTo(2 * DEFAULT_NUM_SLOTS_PER_WORKER);
+ assertThat(slotsAvailableGauge.get().getValue())
+ .isEqualTo(2 * DEFAULT_NUM_SLOTS_PER_WORKER - 1);
+
+ final CompletableFuture unRegisterTaskManagerFuture =
+ new CompletableFuture<>();
+ context.runInMainThreadAndWait(
+ () ->
+ unRegisterTaskManagerFuture.complete(
+ context.getSlotManager()
+ .unregisterTaskManager(
+ executorConnection2.getInstanceID(),
+ TEST_EXCEPTION)));
+ assertThat(assertFutureCompleteAndReturn(unRegisterTaskManagerFuture)).isTrue();
+
+ // triggers the metric update task on the main thread and waits for the main
+ // thread to process queued up callbacks
+ scheduledExecutor.triggerPeriodicScheduledTasks();
+ context.runInMainThreadAndWait(() -> {});
+
+ assertThat(slotsTotalGauge.get().getValue())
+ .isEqualTo(DEFAULT_NUM_SLOTS_PER_WORKER);
+ assertThat(slotsAvailableGauge.get().getValue())
+ .isEqualTo(DEFAULT_NUM_SLOTS_PER_WORKER);
+ });
+ }
}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerTestBase.java
index 977b230863821..0edd01247f766 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerTestBase.java
@@ -152,7 +152,7 @@ protected class Context {
private SlotManagerMetricGroup slotManagerMetricGroup =
UnregisteredMetricGroups.createUnregisteredSlotManagerMetricGroup();
private BlockedTaskManagerChecker blockedTaskManagerChecker = resourceID -> false;
- private final ScheduledExecutor scheduledExecutor =
+ private ScheduledExecutor scheduledExecutor =
new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor());
private final Executor mainThreadExecutor = EXECUTOR_RESOURCE.getExecutor();
private FineGrainedSlotManager slotManager;
@@ -193,6 +193,10 @@ public void setBlockedTaskManagerChecker(
this.blockedTaskManagerChecker = blockedTaskManagerChecker;
}
+ public void setScheduledExecutor(ScheduledExecutor scheduledExecutor) {
+ this.scheduledExecutor = scheduledExecutor;
+ }
+
void runInMainThread(Runnable runnable) {
mainThreadExecutor.execute(runnable);
}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointTest.java
index 9289f3db0e391..05d86686a87f7 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointTest.java
@@ -18,7 +18,6 @@
package org.apache.flink.runtime.rest;
-import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
import org.slf4j.helpers.NOPLogger;
@@ -73,7 +72,6 @@ void testCreateUploadDir(@TempDir File file) throws Exception {
assertThat(Files.exists(testUploadDir)).isTrue();
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testCreateUploadDirFails(@TempDir File file) throws Exception {
assertThat(file.setWritable(false));
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStateOutputStreamTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStateOutputStreamTest.java
index 20c776b61ed44..6f3a30f9c3c95 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStateOutputStreamTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStateOutputStreamTest.java
@@ -33,7 +33,6 @@
import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
import org.apache.flink.testutils.junit.utils.TempDirUtils;
-import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.TestTemplate;
import org.junit.jupiter.api.extension.ExtendWith;
import org.junit.jupiter.api.io.TempDir;
@@ -376,7 +375,6 @@ void testMixedBelowAndAboveThreshold() throws Exception {
* This test checks that the stream does not check and clean the parent directory when
* encountering a write error.
*/
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@TestTemplate
void testStreamDoesNotTryToCleanUpParentOnError() throws Exception {
final File directory = TempDirUtils.newFolder(tempDir);
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunnerStartupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunnerStartupTest.java
index 61f344d9282c3..126767ded612d 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunnerStartupTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunnerStartupTest.java
@@ -50,7 +50,6 @@
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Assumptions;
import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.RegisterExtension;
import org.junit.jupiter.api.io.TempDir;
@@ -114,7 +113,6 @@ void tearDownTest() throws Exception {
* Tests that the TaskManagerRunner startup fails synchronously when the I/O directories are not
* writable.
*/
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testIODirectoryNotWritable() throws Exception {
File nonWritable = TempDirUtils.newFolder(tempFolder);
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/SerializedThrowableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/SerializedThrowableTest.java
index 2f81a91fd2076..64a9340937eed 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/SerializedThrowableTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/SerializedThrowableTest.java
@@ -19,12 +19,22 @@
package org.apache.flink.runtime.util;
import org.apache.flink.core.testutils.CommonTestUtils;
+import org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException;
import org.apache.flink.testutils.ClassLoaderUtils;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.InstantiationUtil;
import org.apache.flink.util.SerializedThrowable;
import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatThrownBy;
@@ -179,4 +189,62 @@ void testCopySuppressed() {
.isInstanceOf(SerializedThrowable.class)
.hasMessage("java.lang.Exception: suppressed");
}
+
+ @Test
+ void testCyclicSuppressedThrowableSerialized() {
+ SerializedThrowable serializedThrowable = new SerializedThrowable(mockThrowable());
+ assertThat(serializedThrowable).isNotNull();
+ }
+
+ @Test
+ @Timeout(value = 5, unit = TimeUnit.SECONDS)
+ void testCyclicSuppressedThrowableConcurrentSerialized() throws InterruptedException {
+ Throwable throwable = mockThrowable();
+ int threadNum = 16;
+ CountDownLatch countDownLatch = new CountDownLatch(threadNum);
+ List threads = new ArrayList<>();
+ for (int i = 0; i < threadNum; i++) {
+ String threadName = "thread-" + i;
+ Thread t = createThread(countDownLatch, throwable, threadName);
+ t.start();
+ countDownLatch.countDown();
+ threads.add(t);
+ }
+ for (Thread thread : threads) {
+ thread.join();
+ }
+ }
+
+ private static Thread createThread(
+ CountDownLatch countDownLatch, Throwable throwable, String threadName) {
+ Thread t =
+ new Thread(
+ () -> {
+ try {
+ countDownLatch.await();
+ SerializedThrowable serializedThrowable =
+ new SerializedThrowable(throwable);
+ assertThat(serializedThrowable).isNotNull();
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ });
+ t.setName(threadName);
+ return t;
+ }
+
+ private static Throwable mockThrowable() {
+ SocketAddress remoteAddr = new InetSocketAddress(80);
+ RemoteTransportException remoteTransportException =
+ new RemoteTransportException(
+ "Connection unexpectedly closed by remote task manager '"
+ + remoteAddr
+ + "'. "
+ + "This might indicate that the remote task manager was lost.",
+ remoteAddr,
+ new IOException("connection reset by peer."));
+ RuntimeException runtimeException = new RuntimeException(remoteTransportException);
+ remoteTransportException.addSuppressed(runtimeException);
+ return remoteTransportException;
+ }
}
diff --git a/flink-runtime/src/test/java/org/apache/flink/streaming/api/operators/SourceOperatorSplitWatermarkAlignmentTest.java b/flink-runtime/src/test/java/org/apache/flink/streaming/api/operators/SourceOperatorSplitWatermarkAlignmentTest.java
index 35621fbf5bcc8..f184c3fac14fb 100644
--- a/flink-runtime/src/test/java/org/apache/flink/streaming/api/operators/SourceOperatorSplitWatermarkAlignmentTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/streaming/api/operators/SourceOperatorSplitWatermarkAlignmentTest.java
@@ -72,6 +72,8 @@ Licensed to the Apache Software Foundation (ASF) under one
import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.createExecutionAttemptId;
import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
/** Unit test for split alignment in {@link SourceOperator}. */
class SourceOperatorSplitWatermarkAlignmentTest {
@@ -482,6 +484,69 @@ void testStateReportingForSingleSplitWatermarkAlignmentAndIdleness() throws Exce
assertThat(operator.getSplitMetricGroup(split0.splitId()).isActive()).isTrue();
}
+ @Test
+ void testAlignmentCheckIsDeferredForIdleSplits() throws Exception {
+ final long idleTimeout = 100;
+ final MockSourceReader sourceReader =
+ new MockSourceReader(WaitingForSplits.DO_NOT_WAIT_FOR_SPLITS, true, true);
+ final TestProcessingTimeService processingTimeService = new TestProcessingTimeService();
+ // Split states math assumes non-negative time
+ processingTimeService.setCurrentTime(0);
+ final SourceOperator operator =
+ createAndOpenSourceOperatorWithIdleness(
+ sourceReader, processingTimeService, idleTimeout);
+
+ final MockSourceSplit split0 = new MockSourceSplit(0, 0, 10);
+ final int allowedWatermark4 = 4;
+ final int allowedWatermark7 = 7;
+ split0.addRecord(5);
+ split0.addRecord(6);
+ split0.addRecord(7);
+ split0.addRecord(8);
+ operator.handleOperatorEvent(
+ new AddSplitEvent<>(Arrays.asList(split0), new MockSourceSplitSerializer()));
+ final CollectingDataOutput actualOutput = new CollectingDataOutput<>();
+
+ // Emit enough record to fill the sampler buffer
+ operator.emitNext(actualOutput);
+ operator.emitNext(actualOutput);
+ operator.emitNext(actualOutput);
+
+ // Transition the split to idle state:
+ for (int i = 0; i < 10; i++) {
+ processingTimeService.advance(idleTimeout);
+ }
+ assertThat(operator.getSplitMetricGroup(split0.splitId()).isIdle()).isTrue();
+
+ // Alignment check fires but doesn't pause the idle split
+ operator.handleOperatorEvent(new WatermarkAlignmentEvent(allowedWatermark4));
+ assertThat(operator.getSplitMetricGroup(split0.splitId()).isIdle()).isTrue();
+
+ // While the split is idle, we advance the allowed watermark to keep the source active
+ operator.handleOperatorEvent(new WatermarkAlignmentEvent(allowedWatermark7));
+ // The split is still idle:
+ assertThat(operator.getSplitMetricGroup(split0.splitId()).isIdle()).isTrue();
+
+ // updating timers values manually (in reality this is done by ViewUpdater)
+ operator.getSplitMetricGroup(split0.splitId()).updateTimers();
+ // Ensure the idle timer ticked, but not pause timer
+ assertNotEquals(
+ 0L, operator.getSplitMetricGroup(split0.splitId()).getAccumulatedIdleTime());
+ assertEquals(0L, operator.getSplitMetricGroup(split0.splitId()).getAccumulatedPausedTime());
+
+ // The split emits a record to break out of idleness
+ operator.emitNext(actualOutput);
+
+ // The split is marked not idle, then immediately paused by the deferred alignment check
+ assertThat(operator.getSplitMetricGroup(split0.splitId()).isPaused()).isTrue();
+
+ // Make pause timer tick
+ processingTimeService.advance(10);
+ operator.getSplitMetricGroup(split0.splitId()).updateTimers();
+ assertNotEquals(
+ 0L, operator.getSplitMetricGroup(split0.splitId()).getAccumulatedPausedTime());
+ }
+
private void assertOutput(
CollectingDataOutput actualOutput, List expectedOutput) {
assertThat(
diff --git a/flink-state-backends/flink-statebackend-forst/src/test/java/org/apache/flink/state/forst/ForStStateBackendConfigTest.java b/flink-state-backends/flink-statebackend-forst/src/test/java/org/apache/flink/state/forst/ForStStateBackendConfigTest.java
index 50f3ac0954bd0..624bbf0b577db 100644
--- a/flink-state-backends/flink-statebackend-forst/src/test/java/org/apache/flink/state/forst/ForStStateBackendConfigTest.java
+++ b/flink-state-backends/flink-statebackend-forst/src/test/java/org/apache/flink/state/forst/ForStStateBackendConfigTest.java
@@ -42,7 +42,6 @@
import org.apache.flink.runtime.state.filesystem.FsCheckpointStorageAccess;
import org.apache.flink.runtime.state.ttl.TtlTimeProvider;
import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo;
-import org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser;
import org.apache.flink.util.FileUtils;
import org.apache.commons.lang3.RandomUtils;
@@ -58,7 +57,6 @@
import org.junit.Assume;
import org.junit.Rule;
import org.junit.Test;
-import org.junit.experimental.categories.Category;
import org.junit.jupiter.api.Timeout;
import org.junit.rules.TemporaryFolder;
@@ -375,7 +373,6 @@ public void testUseTempDirectories() throws Exception {
// ------------------------------------------------------------------------
@Test
- @Category(FailsInGHAContainerWithRootUser.class)
public void testFailWhenNoLocalStorageDir() throws Exception {
final File targetDir = tempFolder.newFolder();
Assume.assumeTrue(
diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/state/rocksdb/RocksDBStateBackendConfigTest.java b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/state/rocksdb/RocksDBStateBackendConfigTest.java
index b1f78a0c0b85b..1a0a45f804e61 100644
--- a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/state/rocksdb/RocksDBStateBackendConfigTest.java
+++ b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/state/rocksdb/RocksDBStateBackendConfigTest.java
@@ -44,7 +44,6 @@
import org.apache.flink.runtime.state.heap.HeapPriorityQueueSetFactory;
import org.apache.flink.runtime.state.ttl.TtlTimeProvider;
import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo;
-import org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser;
import org.apache.flink.util.FileUtils;
import org.apache.flink.util.IOUtils;
@@ -53,7 +52,6 @@
import org.junit.Assume;
import org.junit.Rule;
import org.junit.Test;
-import org.junit.experimental.categories.Category;
import org.junit.jupiter.api.Timeout;
import org.junit.rules.TemporaryFolder;
import org.rocksdb.BlockBasedTableConfig;
@@ -474,7 +472,6 @@ public void testUseTempDirectories() throws Exception {
// ------------------------------------------------------------------------
@Test
- @Category(FailsInGHAContainerWithRootUser.class)
public void testFailWhenNoLocalStorageDir() throws Exception {
final File targetDir = tempFolder.newFolder();
Assume.assumeTrue(
diff --git a/flink-table/flink-sql-parser/src/main/codegen/templates/Parser.jj b/flink-table/flink-sql-parser/src/main/codegen/templates/Parser.jj
index 0de358fd15bce..f9e7cd74b2c45 100644
--- a/flink-table/flink-sql-parser/src/main/codegen/templates/Parser.jj
+++ b/flink-table/flink-sql-parser/src/main/codegen/templates/Parser.jj
@@ -2202,7 +2202,10 @@ SqlNode TableRef3(ExprContext exprContext, boolean lateral) :
[ tableRef = ExtendTable(tableRef) ]
tableRef = Over(tableRef)
[ tableRef = Snapshot(tableRef) ]
- [ tableRef = MatchRecognize(tableRef) ]
+ [
+ LOOKAHEAD(3)
+ tableRef = MatchRecognize(tableRef)
+ ]
)
|
LOOKAHEAD(2)
@@ -2210,7 +2213,10 @@ SqlNode TableRef3(ExprContext exprContext, boolean lateral) :
tableRef = ParenthesizedExpression(exprContext)
tableRef = Over(tableRef)
tableRef = addLateral(tableRef, lateral)
- [ tableRef = MatchRecognize(tableRef) ]
+ [
+ LOOKAHEAD(3)
+ tableRef = MatchRecognize(tableRef)
+ ]
|
LOOKAHEAD(2)
[ ] // "LATERAL" is implicit with "UNNEST", so ignore
@@ -3059,6 +3065,7 @@ void AddUnpivotValue(List list) :
SqlMatchRecognize MatchRecognize(SqlNode tableRef) :
{
final Span s, s0, s1, s2;
+ final SqlIdentifier aliasBeforeMatch;
final SqlNodeList measureList;
final SqlNodeList partitionList;
final SqlNodeList orderList;
@@ -3073,6 +3080,12 @@ SqlMatchRecognize MatchRecognize(SqlNode tableRef) :
final SqlLiteral isStrictEnds;
}
{
+ [
+ aliasBeforeMatch = SimpleIdentifier() {
+ tableRef = SqlStdOperatorTable.AS.createCall(
+ Span.of(tableRef).end(this), tableRef, aliasBeforeMatch);
+ }
+ ]
{ s = span(); checkNotJoin(tableRef); }
(
{ s2 = span(); }
@@ -7209,7 +7222,7 @@ SqlCall MatchRecognizeCallWithModifier() :
{
final Span s;
final SqlOperator runningOp;
- final SqlNode func;
+ final SqlNode e;
}
{
(
@@ -7218,8 +7231,8 @@ SqlCall MatchRecognizeCallWithModifier() :
{ runningOp = SqlStdOperatorTable.FINAL; }
)
{ s = span(); }
- func = NamedFunctionCall() {
- return runningOp.createCall(s.end(func), func);
+ e = Expression3(ExprContext.ACCEPT_NON_QUERY) {
+ return runningOp.createCall(s.end(e), e);
}
}
diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlRowTypeNameSpec.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlRowTypeNameSpec.java
index 9c6ac8868a38f..1ba03412c90ae 100644
--- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlRowTypeNameSpec.java
+++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlRowTypeNameSpec.java
@@ -121,7 +121,8 @@ public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
if (p.right.getNullable() != null && !p.right.getNullable()) {
writer.keyword("NOT NULL");
}
- if (comments.get(i) != null) {
+ // With bounds check - prevents IndexOutOfBoundsException
+ if (i < comments.size() && comments.get(i) != null) {
comments.get(i).unparse(writer, leftPrec, rightPrec);
}
i += 1;
diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/ExtendedSqlRowTypeNameSpecTest.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/ExtendedSqlRowTypeNameSpecTest.java
new file mode 100644
index 0000000000000..d67aca371a806
--- /dev/null
+++ b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/ExtendedSqlRowTypeNameSpecTest.java
@@ -0,0 +1,103 @@
+/*
+ * 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.sql.parser;
+
+import org.apache.flink.sql.parser.impl.FlinkSqlParserImpl;
+import org.apache.flink.sql.parser.type.ExtendedSqlRowTypeNameSpec;
+
+import org.apache.calcite.avatica.util.Casing;
+import org.apache.calcite.avatica.util.Quoting;
+import org.apache.calcite.sql.SqlBasicTypeNameSpec;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.dialect.CalciteSqlDialect;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.parser.SqlParserImplFactory;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.pretty.SqlPrettyWriter;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlConformanceEnum;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+import java.util.Map;
+
+/** Tests for {@link ExtendedSqlRowTypeNameSpec}. */
+class ExtendedSqlRowTypeNameSpecTest {
+ @Test
+ void testExtendedRowWithNoComments() {
+ final ExtendedSqlRowTypeNameSpec spec =
+ new ExtendedSqlRowTypeNameSpec(
+ SqlParserPos.ZERO,
+ List.of(
+ new SqlIdentifier("t1", SqlParserPos.ZERO),
+ new SqlIdentifier("t2", SqlParserPos.ZERO),
+ new SqlIdentifier("t3", SqlParserPos.ZERO)),
+ List.of(
+ new SqlDataTypeSpec(
+ new SqlBasicTypeNameSpec(
+ SqlTypeName.INTEGER, SqlParserPos.ZERO),
+ SqlParserPos.ZERO),
+ new SqlDataTypeSpec(
+ new SqlBasicTypeNameSpec(
+ SqlTypeName.DATE, SqlParserPos.ZERO),
+ SqlParserPos.ZERO),
+ new SqlDataTypeSpec(
+ new SqlBasicTypeNameSpec(
+ SqlTypeName.TIME, SqlParserPos.ZERO),
+ SqlParserPos.ZERO)),
+ List.of(),
+ false);
+ SqlWriter writer = getSqlWriter();
+ spec.unparse(writer, 0, 0);
+ }
+
+ private SqlWriter getSqlWriter() {
+ final Map options =
+ Map.ofEntries(
+ Map.entry("quoting", Quoting.BACK_TICK),
+ Map.entry("quotedCasing", Casing.UNCHANGED),
+ Map.entry("unquotedCasing", Casing.UNCHANGED),
+ Map.entry("caseSensitive", true),
+ Map.entry("enableTypeCoercion", false),
+ Map.entry("conformance", SqlConformanceEnum.DEFAULT),
+ Map.entry("operatorTable", SqlStdOperatorTable.instance()),
+ Map.entry("parserFactory", FlinkSqlParserImpl.FACTORY));
+ final SqlParser.Config parserConfig =
+ SqlParser.config()
+ .withQuoting((Quoting) options.get("quoting"))
+ .withUnquotedCasing((Casing) options.get("unquotedCasing"))
+ .withQuotedCasing((Casing) options.get("quotedCasing"))
+ .withConformance((SqlConformance) options.get("conformance"))
+ .withCaseSensitive((boolean) options.get("caseSensitive"))
+ .withParserFactory((SqlParserImplFactory) options.get("parserFactory"));
+
+ return new SqlPrettyWriter(
+ new CalciteSqlDialect(
+ SqlDialect.EMPTY_CONTEXT
+ .withQuotedCasing(parserConfig.unquotedCasing())
+ .withConformance(parserConfig.conformance())
+ .withUnquotedCasing(parserConfig.unquotedCasing())
+ .withIdentifierQuoteString(parserConfig.quoting().string)),
+ false);
+ }
+}
diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
index 42a86cee711a5..eb3fa67b997ac 100644
--- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
+++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
@@ -1087,8 +1087,8 @@ public OutType initCap() {
}
/**
- * Returns true, if a string matches the specified LIKE pattern with default escape character
- * '/'.
+ * Returns true, if a string matches the specified LIKE pattern. There is no default escape
+ * character.
*
* e.g. "Jo_n%" matches all strings that start with "Jo(arbitrary letter)n"
*/
diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/functions/SqlLikeUtils.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/functions/SqlLikeUtils.java
index 1cbb1b968842a..e13f87a549946 100644
--- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/functions/SqlLikeUtils.java
+++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/functions/SqlLikeUtils.java
@@ -18,6 +18,7 @@
package org.apache.flink.table.functions;
import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.ValidationException;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
@@ -95,8 +96,11 @@ public static String sqlToRegexLike(String sqlPattern, CharSequence escapeStr) {
throw invalidEscapeCharacter(escapeStr.toString());
}
escapeChar = escapeStr.charAt(0);
+ if (escapeChar == 0) {
+ throw invalidEscapeCharacter(escapeStr.toString());
+ }
} else {
- escapeChar = '\\';
+ escapeChar = 0;
}
return sqlToRegexLike(sqlPattern, escapeChar);
}
@@ -108,7 +112,7 @@ static String sqlToRegexLike(String sqlPattern, char escapeChar) {
final StringBuilder javaPattern = new StringBuilder(len + len);
for (i = 0; i < len; i++) {
char c = sqlPattern.charAt(i);
- if (c == escapeChar) {
+ if (c == escapeChar && escapeChar != 0) {
if (i == (sqlPattern.length() - 1)) {
throw invalidEscapeSequence(sqlPattern, i);
}
@@ -140,11 +144,11 @@ static String sqlToRegexLike(String sqlPattern, char escapeChar) {
}
public static RuntimeException invalidEscapeCharacter(String s) {
- return new RuntimeException("Invalid escape character '" + s + "'");
+ return new ValidationException("Invalid escape character '" + s + "'");
}
public static RuntimeException invalidEscapeSequence(String s, int i) {
- return new RuntimeException("Invalid escape sequence '" + s + "', " + i);
+ return new ValidationException("Invalid escape sequence '" + s + "', " + i);
}
private static void similarEscapeRuleChecking(String sqlPattern, char escapeChar) {
@@ -191,7 +195,7 @@ private static int sqlSimilarRewriteCharEnumeration(
char c = sqlPattern.charAt(i);
if (c == ']') {
return i - 1;
- } else if (c == escapeChar) {
+ } else if (c == escapeChar && escapeChar != 0) {
i++;
char nextChar = sqlPattern.charAt(i);
if (SQL_SIMILAR_SPECIALS.indexOf(nextChar) >= 0) {
@@ -240,6 +244,9 @@ public static String sqlToRegexSimilar(String sqlPattern, CharSequence escapeStr
throw invalidEscapeCharacter(escapeStr.toString());
}
escapeChar = escapeStr.charAt(0);
+ if (escapeChar == 0) {
+ throw invalidEscapeCharacter(escapeStr.toString());
+ }
} else {
escapeChar = 0;
}
@@ -255,7 +262,7 @@ public static String sqlToRegexSimilar(String sqlPattern, char escapeChar) {
final int len = sqlPattern.length();
for (int i = 0; i < len; i++) {
char c = sqlPattern.charAt(i);
- if (c == escapeChar) {
+ if (c == escapeChar && escapeChar != 0) {
if (i == (len - 1)) {
// It should never reach here after the escape rule
// checking.
diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/planner/loader/PlannerModule.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/planner/loader/PlannerModule.java
index 289a3f23fa60f..1a8375b5039ef 100644
--- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/planner/loader/PlannerModule.java
+++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/planner/loader/PlannerModule.java
@@ -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;
@@ -135,7 +136,7 @@ private PlannerModule() {
}
}
- public ClassLoader getSubmoduleClassLoader() {
+ public URLClassLoader getSubmoduleClassLoader() {
return this.submoduleClassLoader;
}
diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FileCatalogStoreTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FileCatalogStoreTest.java
index 2824605fd6222..011ce881bb84f 100644
--- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FileCatalogStoreTest.java
+++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FileCatalogStoreTest.java
@@ -23,7 +23,6 @@
import org.apache.flink.util.OperatingSystem;
import org.assertj.core.api.ThrowableAssert;
-import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
@@ -64,7 +63,6 @@ void testNotOpened() {
assertCatalogStoreNotOpened(() -> catalogStore.removeCatalog(DUMMY, true));
}
- @Tag("org.apache.flink.testutils.junit.FailsInGHAContainerWithRootUser")
@Test
void testCannotMakeStorePath() {
assumeThat(OperatingSystem.isWindows())
diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/NoCommonJoinKeyException.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/NoCommonJoinKeyException.java
new file mode 100644
index 0000000000000..21eb273b91171
--- /dev/null
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/NoCommonJoinKeyException.java
@@ -0,0 +1,40 @@
+/*
+ * 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.utils;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.util.FlinkRuntimeException;
+
+/** Thrown when a MultiJoin node has no common join key. */
+@Internal
+public class NoCommonJoinKeyException extends FlinkRuntimeException {
+ private static final long serialVersionUID = 1L;
+
+ public NoCommonJoinKeyException(String message) {
+ super(message);
+ }
+
+ public NoCommonJoinKeyException(String message, Throwable cause) {
+ super(message, cause);
+ }
+
+ public NoCommonJoinKeyException(Throwable cause) {
+ super(cause);
+ }
+}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
index 89317e352f6a4..db6b61421bb46 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
@@ -169,22 +169,35 @@
* Default implementation of {@link SqlValidator}, the class was copied over because of
* CALCITE-4554.
*
- *
Lines 202 ~ 205, Flink improves error message for functions without appropriate arguments in
+ *
Lines 219 ~ 222, Flink improves error message for functions without appropriate arguments in
* handleUnresolvedFunction.
*
- *
Lines 1270 ~ 1272, CALCITE-7217, should be removed after upgrading Calcite to 1.41.0.
+ *
Lines 1287 ~ 1289, CALCITE-7217, should be removed after upgrading Calcite to 1.41.0.
*
- *
Lines 2031 ~ 2045, Flink improves error message for functions without appropriate arguments in
+ *
Lines 2048 ~ 2062, Flink improves error message for functions without appropriate arguments in
* handleUnresolvedFunction at {@link SqlValidatorImpl#handleUnresolvedFunction}.
*
- *
Lines 2571 ~ 2588, CALCITE-7217, should be removed after upgrading Calcite to 1.41.0.
+ *
Lines 2475 ~ 2477, CALCITE-7471 should be removed after upgrading Calcite to 1.42.0.
*
- *
Line 2618 ~2631, set the correct scope for VECTOR_SEARCH.
+ *
Lines 2590 ~ 2609, CALCITE-7217, CALCITE-7312 should be removed after upgrading Calcite to
+ * 1.42.0.
*
- *
Lines 3920 ~ 3925, 6599 ~ 6606 Flink improves Optimize the retrieval of sub-operands in
+ *
Line 2640 ~2658, set the correct scope for VECTOR_SEARCH.
+ *
+ *
Lines 3937 ~ 3941, 6612 ~ 6618 Flink improves Optimize the retrieval of sub-operands in
* SqlCall when using NamedParameters at {@link SqlValidatorImpl#checkRollUp}.
*
- *
Lines 5340 ~ 5347, FLINK-24352 Add null check for temporal table check on SqlSnapshot.
+ *
Lines 5357 ~ 5363, FLINK-24352 Add null check for temporal table check on SqlSnapshot.
+ *
+ *
Lines 5782-5784, CALCITE-7466 should be removed after upgrading Calcite to 1.42.0.
+ *
+ *
Lines 5838-5840, CALCITE-7470 should be removed after upgrading Calcite to 1.42.0.
+ *
+ *
Lines 7267-7290, CALCITE-7486 should be removed after upgrading Calcite to 1.42.0.
+ *
+ *
Lines 7335-7352, CALCITE-7486 should be removed after upgrading Calcite to 1.42.0.
+ *
+ *
Lines 7397-7405, CALCITE-7486 should be removed after upgrading Calcite to 1.42.0.
*/
public class SqlValidatorImpl implements SqlValidatorWithHints {
// ~ Static fields/initializers ---------------------------------------------
@@ -2459,7 +2472,9 @@ private SqlNode registerFrom(
enclosingNode,
alias,
forceNullable);
- return node;
+ // ----- FLINK MODIFICATION BEGIN -----
+ return newNode;
+ // ----- FLINK MODIFICATION END -----
case PIVOT:
registerPivot(
@@ -5764,11 +5779,9 @@ private PairList validateMeasure(
setValidatedNodeType(measure, type);
fields.add(alias, type);
- sqlNodes.add(
- SqlStdOperatorTable.AS.createCall(
- SqlParserPos.ZERO,
- expand,
- new SqlIdentifier(alias, SqlParserPos.ZERO)));
+ // ----- FLINK MODIFICATION BEGIN -----
+ sqlNodes.add(expand);
+ // ----- FLINK MODIFICATION END -----
}
SqlNodeList list = new SqlNodeList(sqlNodes, measures.getParserPosition());
@@ -5822,11 +5835,9 @@ private void validateDefinitions(SqlMatchRecognize mr, MatchRecognizeScope scope
// Some extra work need required here.
// In PREV, NEXT, FINAL and LAST, only one pattern variable is allowed.
- sqlNodes.add(
- SqlStdOperatorTable.AS.createCall(
- SqlParserPos.ZERO,
- expand,
- new SqlIdentifier(alias, SqlParserPos.ZERO)));
+ // ----- FLINK MODIFICATION BEGIN -----
+ sqlNodes.add(expand);
+ // ----- FLINK MODIFICATION END -----
final RelDataType type = deriveType(scope, expand);
if (!SqlTypeUtil.inBooleanFamily(type)) {
@@ -7251,19 +7262,31 @@ private class PatternValidator extends SqlBasicVisitor<@Nullable Set> {
int firstLastCount;
int prevNextCount;
int aggregateCount;
+ // ----- FLINK MODIFICATION BEGIN -----
+ int index;
+ int argCount;
PatternValidator(boolean isMeasure) {
- this(isMeasure, 0, 0, 0);
+ this(isMeasure, 0, 0, 0, 0, 0);
}
PatternValidator(
- boolean isMeasure, int firstLastCount, int prevNextCount, int aggregateCount) {
+ boolean isMeasure,
+ int firstLastCount,
+ int prevNextCount,
+ int aggregateCount,
+ int index,
+ int argCount) {
this.isMeasure = isMeasure;
this.firstLastCount = firstLastCount;
this.prevNextCount = prevNextCount;
this.aggregateCount = aggregateCount;
+ this.index = index;
+ this.argCount = argCount;
}
+ // ----- FLINK MODIFICATION END -----
+
@Override
public Set visit(SqlCall call) {
boolean isSingle = false;
@@ -7309,7 +7332,9 @@ public Set visit(SqlCall call) {
call, Static.RESOURCE.patternRunningFunctionInDefine(call.toString()));
}
- for (SqlNode node : operands) {
+ // ----- FLINK MODIFICATION BEGIN -----
+ for (int i = 0; i < operands.size(); i++) {
+ SqlNode node = operands.get(i);
if (node != null) {
vars.addAll(
requireNonNull(
@@ -7318,10 +7343,13 @@ public Set visit(SqlCall call) {
isMeasure,
firstLastCount,
prevNextCount,
- aggregateCount)),
+ aggregateCount,
+ i,
+ operands.size())),
() -> "node.accept(PatternValidator) for node " + node));
}
}
+ // ----- FLINK MODIFICATION END -----
if (isSingle) {
switch (kind) {
@@ -7366,7 +7394,15 @@ public Set visit(SqlIdentifier identifier) {
@Override
public Set visit(SqlLiteral literal) {
- return ImmutableSet.of();
+ // ----- FLINK MODIFICATION BEGIN -----
+ if ((this.argCount == 1 || this.index < this.argCount - 1)
+ && (this.firstLastCount > 0 || this.prevNextCount > 0)
+ && !SqlUtil.isNull(literal)) {
+ return ImmutableSet.of(literal.toValue());
+ } else {
+ return ImmutableSet.of();
+ }
+ // ----- FLINK MODIFICATION END -----
}
@Override
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
index b226cc5701497..4f00abc858bb7 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -245,19 +245,21 @@
* FLINK modifications are at lines
*
*
- * Added in FLINK-29081, FLINK-28682, FLINK-33395: Lines 686 ~ 703
- * Added in Flink-24024: Lines 1453 ~ 1459
- * Added in Flink-24024: Lines 1473 ~ 1512
- * Added in Flink-37269: Lines 2250 ~ 2272
- * Added in FLINK-28682: Lines 2383 ~ 2400
- * Added in FLINK-28682: Lines 2437 ~ 2465
- * Added in FLINK-32474: Lines 2522 ~ 2524
- * Added in FLINK-32474: Lines 2528 ~ 2530
- * Added in FLINK-32474: Lines 2546 ~ 2548
- * Added in CALCITE-7217: Lines 2587 ~ 2595, should be dropped with upgrade to Calcite 1.41.0
- * Added in FLINK-32474: Lines 2970 ~ 2982
- * Added in FLINK-32474: Lines 3083 ~ 3117
- * Added in FLINK-34312: Lines 5947 ~ 5958
+ * Added in FLINK-29081, FLINK-28682, FLINK-33395: Lines 688 ~ 705
+ * Added in FLINK-24024: Lines 1455 ~ 1461
+ * Added in FLINK-24024: Lines 1475 ~ 1514
+ * Added in FLINK-37269: Lines 2252 ~ 2274
+ * Added in FLINK-28682: Lines 2385 ~ 2402
+ * Added in FLINK-28682: Lines 2439 ~ 2467
+ * Added in FLINK-32474: Lines 2524 ~ 2526
+ * Added in FLINK-32474: Lines 2530 ~ 2532
+ * Added in FLINK-32474: Lines 2548 ~ 2550
+ * Added in CALCITE-7217: Lines 2589 ~ 2597, should be dropped with upgrade to Calcite 1.41.0
+ * Added in FLINK-32474: Lines 2972 ~ 2984
+ * Added in FLINK-32474: Lines 3085 ~ 3119
+ * Added in FLINK-38720: Lines 4579 ~ 4585
+ * Added in FLINK-38720: Lines 4591 ~ 4607
+ * Added in FLINK-34312: Lines 5971 ~ 5982
*
*
* In official extension point (i.e. {@link #convertExtendedExpression(SqlNode, Blackboard)}):
@@ -4574,14 +4576,34 @@ private RexNode convertIdentifier(Blackboard bb, SqlIdentifier identifier) {
}
if (e0.left instanceof RexCorrelVariable) {
- assert e instanceof RexFieldAccess;
- final RexNode prev =
- bb.mapCorrelateToRex.put(((RexCorrelVariable) e0.left).id, (RexFieldAccess) e);
+ // ----- FLINK MODIFICATION BEGIN -----
+ // adjust the type to account for nulls introduced by FlinkRexBuilder#makeFieldAccess
+ final RexFieldAccess rfa = adjustRexFieldAccess(e);
+ final RexNode prev = bb.mapCorrelateToRex.put(((RexCorrelVariable) e0.left).id, rfa);
+ // ----- FLINK MODIFICATION END -----
assert prev == null;
}
return e;
}
+ // ----- FLINK MODIFICATION BEGIN -----
+ private RexFieldAccess adjustRexFieldAccess(RexNode rexNode) {
+ // Either RexFieldAccess or CAST of RexFieldAccess to nullable
+ assert rexNode instanceof RexFieldAccess
+ || rexNode instanceof RexCall
+ && rexNode.getKind() == SqlKind.CAST
+ && ((RexCall) rexNode).getOperands().size() == 1
+ && ((RexCall) rexNode).getOperands().get(0) instanceof RexFieldAccess;
+
+ if (rexNode instanceof RexFieldAccess) {
+ return (RexFieldAccess) rexNode;
+ } else {
+ return (RexFieldAccess) ((RexCall) rexNode).getOperands().get(0);
+ }
+ }
+
+ // ----- FLINK MODIFICATION END -----
+
/**
* Adjusts the type of a reference to an input field to account for nulls introduced by outer
* joins; and adjusts the offset to match the physical implementation.
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/adaptive/AdaptiveJoinOperatorGenerator.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/adaptive/AdaptiveJoinOperatorGenerator.java
index 78981a5e8ce98..b69220399d452 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/adaptive/AdaptiveJoinOperatorGenerator.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/adaptive/AdaptiveJoinOperatorGenerator.java
@@ -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;
@@ -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,
@@ -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;
@@ -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,
@@ -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;
- }
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.java
index 6403708c216c6..168d4df4fb003 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.java
@@ -65,6 +65,7 @@
import org.apache.calcite.rel.core.AggregateCall;
import org.apache.calcite.rel.core.SetOp;
import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.core.Window;
import org.apache.calcite.rel.logical.LogicalCalc;
import org.apache.calcite.rel.logical.LogicalTableModify;
import org.apache.calcite.rel.type.RelDataType;
@@ -78,6 +79,7 @@
import org.apache.calcite.rex.RexProgram;
import org.apache.calcite.rex.RexProgramBuilder;
import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.sql.SqlAggFunction;
import org.apache.calcite.sql.SqlOperator;
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
import org.apache.calcite.sql.type.SqlTypeName;
@@ -143,10 +145,11 @@ public RelNode visit(RelNode node) {
|| node instanceof FlinkLogicalDistribution
|| node instanceof FlinkLogicalWatermarkAssigner
|| node instanceof FlinkLogicalSort
- || node instanceof FlinkLogicalOverAggregate
|| node instanceof FlinkLogicalExpand
|| node instanceof FlinkLogicalScriptTransform) {
return visitSimpleRel(node);
+ } else if (node instanceof FlinkLogicalOverAggregate) {
+ return visitLogicalOverAggregate((FlinkLogicalOverAggregate) node);
} else if (node instanceof FlinkLogicalWindowAggregate) {
return visitWindowAggregate((FlinkLogicalWindowAggregate) node);
} else if (node instanceof FlinkLogicalWindowTableAggregate) {
@@ -236,6 +239,54 @@ private RelNode visitMatch(FlinkLogicalMatch match) {
newInterval);
}
+ private RelNode visitLogicalOverAggregate(FlinkLogicalOverAggregate logical) {
+ final RelNode newInput = logical.getInput().accept(this);
+ final List newDataTypeList =
+ new ArrayList<>(
+ newInput.getRowType().getFieldList().stream()
+ .map(RelDataTypeField::getType)
+ .collect(Collectors.toList()));
+
+ final List windowGroups = new ArrayList<>();
+ for (Window.Group group : logical.groups) {
+ final List winCalls = new ArrayList<>();
+ for (Window.RexWinAggCall call : group.aggCalls) {
+ RelDataType callType;
+ if (isTimeIndicatorType(call.getType())) {
+ callType =
+ timestamp(
+ call.getType().isNullable(),
+ isTimestampLtzType(call.getType()));
+ } else {
+ callType = call.getType();
+ }
+ winCalls.add(
+ new Window.RexWinAggCall(
+ (SqlAggFunction) call.op,
+ callType,
+ call.getOperands(),
+ call.ordinal,
+ call.distinct,
+ call.ignoreNulls));
+ newDataTypeList.add(callType);
+ }
+ windowGroups.add(
+ new Window.Group(
+ group.keys,
+ group.isRows,
+ group.lowerBound,
+ group.upperBound,
+ group.orderKeys,
+ winCalls));
+ }
+
+ final RelDataType newType =
+ logical.getCluster()
+ .getTypeFactory()
+ .createStructType(newDataTypeList, logical.getRowType().getFieldNames());
+ return logical.copy(logical.getTraitSet(), List.of(newInput), newType, windowGroups);
+ }
+
private RelNode visitCalc(FlinkLogicalCalc calc) {
// visit children and update inputs
RelNode newInput = calc.getInput().accept(this);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandChecker.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandChecker.java
index 467e2178cccc4..e20c2c0754835 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandChecker.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandChecker.java
@@ -19,10 +19,12 @@
package org.apache.flink.table.planner.functions.inference;
import org.apache.flink.annotation.Internal;
+import org.apache.flink.sql.parser.type.SqlRawTypeNameSpec;
import org.apache.flink.table.api.ValidationException;
import org.apache.flink.table.catalog.DataTypeFactory;
import org.apache.flink.table.functions.FunctionDefinition;
import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.plan.schema.RawRelDataType;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.inference.ArgumentCount;
import org.apache.flink.table.types.inference.CallContext;
@@ -32,16 +34,20 @@
import org.apache.flink.table.types.inference.TypeInference;
import org.apache.flink.table.types.inference.TypeInferenceUtil;
import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RawType;
import org.apache.flink.table.types.logical.utils.LogicalTypeChecks;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.rel.type.StructKind;
import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlDataTypeSpec;
import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
import org.apache.calcite.sql.SqlNode;
import org.apache.calcite.sql.SqlOperandCountRange;
import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlTypeNameSpec;
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
import org.apache.calcite.sql.parser.SqlParserPos;
import org.apache.calcite.sql.type.SqlOperandMetadata;
@@ -240,10 +246,28 @@ private void insertImplicitCasts(SqlCallBinding callBinding, List expe
/** Adopted from {@link org.apache.calcite.sql.validate.implicit.AbstractTypeCoercion}. */
private SqlNode castTo(SqlNode node, RelDataType type) {
- return SqlStdOperatorTable.CAST.createCall(
- SqlParserPos.ZERO,
- node,
- SqlTypeUtil.convertTypeToSpec(type).withNullable(type.isNullable()));
+ final SqlDataTypeSpec dataType;
+ if (type instanceof RawRelDataType) {
+ dataType = createRawDataTypeSpec((RawRelDataType) type);
+ } else {
+ dataType = SqlTypeUtil.convertTypeToSpec(type).withNullable(type.isNullable());
+ }
+
+ return SqlStdOperatorTable.CAST.createCall(SqlParserPos.ZERO, node, dataType);
+ }
+
+ private SqlDataTypeSpec createRawDataTypeSpec(RawRelDataType type) {
+ final RawType> rawType = type.getRawType();
+
+ SqlNode className =
+ SqlLiteral.createCharString(
+ rawType.getOriginatingClass().getName(), SqlParserPos.ZERO);
+ SqlNode serializer =
+ SqlLiteral.createCharString(rawType.getSerializerString(), SqlParserPos.ZERO);
+
+ SqlTypeNameSpec rawSpec = new SqlRawTypeNameSpec(className, serializer, SqlParserPos.ZERO);
+
+ return new SqlDataTypeSpec(rawSpec, null, type.isNullable(), SqlParserPos.ZERO);
}
/** Adopted from {@link org.apache.calcite.sql.validate.implicit.AbstractTypeCoercion}. */
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandInference.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandInference.java
index fba8d45034fec..51835f1004cb9 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandInference.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandInference.java
@@ -81,8 +81,10 @@ public void inferOperandTypes(
false)) {
inferOperandTypesOrError(unwrapTypeFactory(callBinding), callContext, operandTypes);
}
- } catch (ValidationException | CalciteContextException e) {
+ } catch (ValidationException e) {
// let operand checker fail
+ } catch (CalciteContextException e) {
+ throw e;
} catch (Throwable t) {
throw createUnexpectedException(callContext, t);
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlNodeConvertUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlNodeConvertUtils.java
index db9140a084a98..57ad2c281dac1 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlNodeConvertUtils.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlNodeConvertUtils.java
@@ -77,15 +77,17 @@ static CatalogView toCatalogView(
// This bug is fixed in CALCITE-3877 of Calcite 1.23.0.
String originalQuery = context.toQuotedSqlString(query);
SqlNode validateQuery = context.getSqlValidator().validate(query);
+ // FLINK-38950: SqlValidator.validate() mutates its input parameter. Always use the
+ // returned validateQuery instead of the mutated query for all subsequent operations.
// Check name is unique.
// Don't rely on the calcite because if the field names are duplicate, calcite will add
// index to identify the duplicate names.
SqlValidatorNamespace validatedNamespace =
context.getSqlValidator().getNamespace(validateQuery);
- validateDuplicatedColumnNames(query, viewFields, validatedNamespace);
+ validateDuplicatedColumnNames(validateQuery, viewFields, validatedNamespace);
- String expandedQuery = context.toQuotedSqlString(query);
+ String expandedQuery = context.toQuotedSqlString(validateQuery);
PlannerQueryOperation operation = toQueryOperation(validateQuery, context);
ResolvedSchema schema = operation.getResolvedSchema();
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecAdaptiveJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecAdaptiveJoin.java
index bf1122bde8696..a103a8e349bb9 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecAdaptiveJoin.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecAdaptiveJoin.java
@@ -19,6 +19,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.batch;
import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.configuration.CoreOptions;
import org.apache.flink.configuration.ReadableConfig;
import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
import org.apache.flink.table.api.TableException;
@@ -36,7 +37,7 @@
import org.apache.flink.table.planner.plan.utils.JoinUtil;
import org.apache.flink.table.planner.plan.utils.OperatorType;
import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
-import org.apache.flink.table.runtime.operators.join.adaptive.AdaptiveJoin;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
import org.apache.flink.table.runtime.operators.join.adaptive.AdaptiveJoinOperatorFactory;
import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
import org.apache.flink.table.types.logical.RowType;
@@ -45,6 +46,8 @@
import java.io.IOException;
import java.util.List;
+import static org.apache.flink.util.Preconditions.checkState;
+
/** {@link BatchExecNode} for adaptive join. */
public class BatchExecAdaptiveJoin extends ExecNodeBase
implements BatchExecNode, SingleTransformationTranslator {
@@ -87,6 +90,14 @@ public BatchExecAdaptiveJoin(
this.leftIsBuild = leftIsBuild;
this.tryDistinctBuildRow = tryDistinctBuildRow;
this.description = description;
+ 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.originalJoin = originalJoin;
}
@@ -113,11 +124,10 @@ protected Transformation translateToPlanInternal(
leftType,
rightType);
- AdaptiveJoinOperatorGenerator adaptiveJoin =
+ AdaptiveJoinOperatorGenerator adaptiveJoinGenerator =
new AdaptiveJoinOperatorGenerator(
joinSpec.getLeftKeys(),
joinSpec.getRightKeys(),
- joinSpec.getJoinType(),
joinSpec.getFilterNulls(),
leftType,
rightType,
@@ -127,16 +137,19 @@ protected Transformation translateToPlanInternal(
estimatedLeftRowCount,
estimatedRightRowCount,
tryDistinctBuildRow,
- managedMemory,
- leftIsBuild,
- originalJoin);
+ managedMemory);
return ExecNodeUtil.createTwoInputTransformation(
leftInputTransform,
rightInputTransform,
createTransformationName(config),
createTransformationDescription(config),
- getAdaptiveJoinOperatorFactory(adaptiveJoin),
+ getAdaptiveJoinOperatorFactory(
+ adaptiveJoinGenerator,
+ config.get(CoreOptions.CHECK_LEAKED_CLASSLOADER),
+ joinSpec.getJoinType(),
+ originalJoin,
+ leftIsBuild),
InternalTypeInfo.of(getOutputType()),
// Given that the probe side might be decided at runtime, we choose the larger
// parallelism here.
@@ -146,10 +159,20 @@ protected Transformation translateToPlanInternal(
}
private StreamOperatorFactory getAdaptiveJoinOperatorFactory(
- AdaptiveJoin adaptiveJoin) {
+ AdaptiveJoinOperatorGenerator adaptiveJoinGenerator,
+ boolean checkClassLoaderLeak,
+ FlinkJoinType joinType,
+ OperatorType originalJoin,
+ boolean leftIsBuild) {
try {
- byte[] adaptiveJoinSerialized = InstantiationUtil.serializeObject(adaptiveJoin);
- return new AdaptiveJoinOperatorFactory<>(adaptiveJoinSerialized);
+ byte[] adaptiveJoinGeneratorSerialized =
+ InstantiationUtil.serializeObject(adaptiveJoinGenerator);
+ return new AdaptiveJoinOperatorFactory<>(
+ adaptiveJoinGeneratorSerialized,
+ joinType,
+ originalJoin == OperatorType.SortMergeJoin,
+ leftIsBuild,
+ checkClassLoaderLeak);
} catch (IOException e) {
throw new TableException("The adaptive join operator failed to serialize.", e);
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecVectorSearchTableFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecVectorSearchTableFunction.java
index 51f32a685c50c..204da67ac2ed4 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecVectorSearchTableFunction.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecVectorSearchTableFunction.java
@@ -28,8 +28,6 @@
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecVectorSearchTableFunction;
import org.apache.flink.table.planner.plan.nodes.exec.spec.VectorSearchSpec;
import org.apache.flink.table.planner.plan.nodes.exec.spec.VectorSearchTableSourceSpec;
-import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecMLPredictTableFunction;
-import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecVectorSearchTableFunction;
import org.apache.flink.table.planner.plan.utils.FunctionCallUtil;
import org.apache.flink.table.types.logical.RowType;
@@ -50,7 +48,7 @@
"table.exec.async-vector-search.timeout",
"table.exec.async-vector-search.output-mode"
},
- producedTransformations = StreamExecMLPredictTableFunction.ML_PREDICT_TRANSFORMATION,
+ producedTransformations = CommonExecVectorSearchTableFunction.VECTOR_SEARCH_TRANSFORMATION,
minPlanVersion = FlinkVersion.v2_2,
minStateVersion = FlinkVersion.v2_2)
public class BatchExecVectorSearchTableFunction extends CommonExecVectorSearchTableFunction
@@ -66,9 +64,9 @@ public BatchExecVectorSearchTableFunction(
String description) {
this(
ExecNodeContext.newNodeId(),
- ExecNodeContext.newContext(StreamExecVectorSearchTableFunction.class),
+ ExecNodeContext.newContext(BatchExecVectorSearchTableFunction.class),
ExecNodeContext.newPersistedConfig(
- StreamExecVectorSearchTableFunction.class, tableConfig),
+ BatchExecVectorSearchTableFunction.class, tableConfig),
tableSourceSpec,
vectorSearchSpec,
asyncOptions,
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RexNodeJsonDeserializer.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RexNodeJsonDeserializer.java
index f06d2bfcfd431..f38548506238f 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RexNodeJsonDeserializer.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RexNodeJsonDeserializer.java
@@ -472,6 +472,7 @@ private static SqlOperator deserializeFunctionClass(
case SCALAR:
case ASYNC_SCALAR:
case TABLE:
+ case PROCESS_TABLE:
return BridgingSqlFunction.of(
serdeContext.getFlinkContext(),
serdeContext.getTypeFactory(),
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecVectorSearchTableFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecVectorSearchTableFunction.java
index 101a58dabdeca..849a6b68788e6 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecVectorSearchTableFunction.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecVectorSearchTableFunction.java
@@ -49,7 +49,7 @@
"table.exec.async-vector-search.timeout",
"table.exec.async-vector-search.output-mode"
},
- producedTransformations = StreamExecMLPredictTableFunction.ML_PREDICT_TRANSFORMATION,
+ producedTransformations = CommonExecVectorSearchTableFunction.VECTOR_SEARCH_TRANSFORMATION,
minPlanVersion = FlinkVersion.v2_2,
minStateVersion = FlinkVersion.v2_2)
public class StreamExecVectorSearchTableFunction extends CommonExecVectorSearchTableFunction
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/JoinToMultiJoinRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/JoinToMultiJoinRule.java
index b0e1c7de8e526..81f27d29788ab 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/JoinToMultiJoinRule.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/JoinToMultiJoinRule.java
@@ -18,15 +18,18 @@
package org.apache.flink.table.planner.plan.rules.logical;
-import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
import org.apache.flink.table.planner.hint.FlinkHints;
import org.apache.flink.table.planner.hint.StateTtlHint;
import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMultiJoin;
import org.apache.flink.table.planner.plan.utils.IntervalJoinUtil;
+import org.apache.flink.table.runtime.operators.join.stream.keyselector.AttributeBasedJoinKeyExtractor;
+import org.apache.flink.table.runtime.operators.join.stream.keyselector.JoinKeyExtractor;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.utils.NoCommonJoinKeyException;
import org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.calcite.plan.RelOptTable;
import org.apache.calcite.plan.RelOptUtil;
import org.apache.calcite.plan.RelRule;
import org.apache.calcite.plan.hep.HepRelVertex;
@@ -36,21 +39,15 @@
import org.apache.calcite.rel.core.Join;
import org.apache.calcite.rel.core.JoinInfo;
import org.apache.calcite.rel.core.JoinRelType;
-import org.apache.calcite.rel.core.TableFunctionScan;
-import org.apache.calcite.rel.core.TableScan;
-import org.apache.calcite.rel.core.Values;
import org.apache.calcite.rel.hint.RelHint;
import org.apache.calcite.rel.logical.LogicalJoin;
import org.apache.calcite.rel.logical.LogicalSnapshot;
-import org.apache.calcite.rel.metadata.RelColumnOrigin;
-import org.apache.calcite.rel.metadata.RelMetadataQuery;
import org.apache.calcite.rel.rules.CoreRules;
import org.apache.calcite.rel.rules.FilterMultiJoinMergeRule;
import org.apache.calcite.rel.rules.MultiJoin;
import org.apache.calcite.rel.rules.ProjectMultiJoinMergeRule;
import org.apache.calcite.rel.rules.TransformationRule;
import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.rex.RexCall;
import org.apache.calcite.rex.RexInputRef;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.rex.RexUtil;
@@ -65,14 +62,14 @@
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
-import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
-import java.util.Set;
import java.util.stream.Collectors;
+import java.util.stream.Stream;
import static org.apache.flink.table.planner.hint.StateTtlHint.STATE_TTL;
+import static org.apache.flink.table.planner.plan.utils.MultiJoinUtil.createJoinAttributeMap;
/**
* Flink Planner rule to flatten a tree of {@link Join}s into a single {@link MultiJoin} with N
@@ -442,134 +439,45 @@ private boolean canCombine(RelNode input, Join origJoin) {
/**
* Checks if original join and child multi-join have common join keys to decide if we can merge
- * them into a single MultiJoin with one more input.
+ * them into a single MultiJoin with one more input. The method uses {@link
+ * AttributeBasedJoinKeyExtractor} to try to create valid common join key extractors.
*
* @param origJoin original Join
* @param otherJoin child MultiJoin
* @return true if original Join and child multi-join have at least one common JoinKey
*/
private boolean haveCommonJoinKey(Join origJoin, MultiJoin otherJoin) {
- Set origJoinKeys = getJoinKeys(origJoin);
- Set otherJoinKeys = getJoinKeys(otherJoin);
-
- origJoinKeys.retainAll(otherJoinKeys);
-
- return !origJoinKeys.isEmpty();
- }
-
- /**
- * Returns a set of join keys as strings following this format [table_name.field_name].
- *
- * @param join Join or MultiJoin node
- * @return set of all the join keys (keys from join conditions)
- */
- public Set getJoinKeys(RelNode join) {
- Set joinKeys = new HashSet<>();
- List conditions = Collections.emptyList();
- List inputs = join.getInputs();
-
- if (join instanceof Join) {
- conditions = collectConjunctions(((Join) join).getCondition());
- } else if (join instanceof MultiJoin) {
- conditions =
- ((MultiJoin) join)
- .getOuterJoinConditions().stream()
- .flatMap(cond -> collectConjunctions(cond).stream())
- .collect(Collectors.toList());
+ final List combinedJoinInputs =
+ Stream.concat(otherJoin.getInputs().stream(), Stream.of(origJoin.getRight()))
+ .collect(Collectors.toUnmodifiableList());
+
+ final List combinedInputTypes =
+ combinedJoinInputs.stream()
+ .map(i -> FlinkTypeFactory.toLogicalRowType(i.getRowType()))
+ .collect(Collectors.toUnmodifiableList());
+
+ final List combinedJoinConditions =
+ Stream.concat(
+ otherJoin.getOuterJoinConditions().stream(),
+ List.of(origJoin.getCondition()).stream())
+ .collect(Collectors.toUnmodifiableList());
+
+ final Map>
+ joinAttributeMap =
+ createJoinAttributeMap(combinedJoinInputs, combinedJoinConditions);
+
+ boolean haveCommonJoinKey = false;
+ try {
+ // we probe to instantiate AttributeBasedJoinKeyExtractor's constructor to check whether
+ // it's possible to initialize common join key structures
+ final JoinKeyExtractor keyExtractor =
+ new AttributeBasedJoinKeyExtractor(joinAttributeMap, combinedInputTypes);
+ haveCommonJoinKey = keyExtractor.getCommonJoinKeyIndices(0).length > 0;
+ } catch (NoCommonJoinKeyException ignored) {
+ // failed to instantiate common join key structures => no common join key
}
- RelMetadataQuery mq = join.getCluster().getMetadataQuery();
-
- for (RexCall condition : conditions) {
- for (RexNode operand : condition.getOperands()) {
- if (operand instanceof RexInputRef) {
- addJoinKeysByOperand((RexInputRef) operand, inputs, mq, joinKeys);
- }
- }
- }
-
- return joinKeys;
- }
-
- /**
- * Retrieves conjunctions from joinCondition.
- *
- * @param joinCondition join condition
- * @return List of RexCalls representing conditions
- */
- private List collectConjunctions(RexNode joinCondition) {
- return RelOptUtil.conjunctions(joinCondition).stream()
- .map(rexNode -> (RexCall) rexNode)
- .collect(Collectors.toList());
- }
-
- /**
- * Appends join key's string representation to the set of join keys.
- *
- * @param ref input ref to the operand
- * @param inputs List of node's inputs
- * @param mq RelMetadataQuery needed to retrieve column origins
- * @param joinKeys Set of join keys to be added
- */
- private void addJoinKeysByOperand(
- RexInputRef ref, List inputs, RelMetadataQuery mq, Set joinKeys) {
- int inputRefIndex = ref.getIndex();
- Tuple2 targetInputAndIdx = getTargetInputAndIdx(inputRefIndex, inputs);
- RelNode targetInput = targetInputAndIdx.f0;
- int idxInTargetInput = targetInputAndIdx.f1;
-
- Set origins = mq.getColumnOrigins(targetInput, idxInTargetInput);
- if (origins != null) {
- for (RelColumnOrigin origin : origins) {
- RelOptTable originTable = origin.getOriginTable();
- List qualifiedName = originTable.getQualifiedName();
- String fieldName =
- originTable
- .getRowType()
- .getFieldList()
- .get(origin.getOriginColumnOrdinal())
- .getName();
- joinKeys.add(qualifiedName.get(qualifiedName.size() - 1) + "." + fieldName);
- }
- }
- }
-
- /**
- * Get real table that contains needed input ref (join key).
- *
- * @param inputRefIndex index of the required field
- * @param inputs inputs of the node
- * @return target input + idx of the required field as target input's
- */
- private Tuple2 getTargetInputAndIdx(int inputRefIndex, List inputs) {
- RelNode targetInput = null;
- int idxInTargetInput = 0;
- int inputFieldEnd = 0;
- for (RelNode input : inputs) {
- inputFieldEnd += input.getRowType().getFieldCount();
- if (inputRefIndex < inputFieldEnd) {
- targetInput = input;
- int targetInputStartIdx = inputFieldEnd - input.getRowType().getFieldCount();
- idxInTargetInput = inputRefIndex - targetInputStartIdx;
- break;
- }
- }
-
- targetInput =
- (targetInput instanceof HepRelVertex)
- ? ((HepRelVertex) targetInput).getCurrentRel()
- : targetInput;
-
- assert targetInput != null;
-
- if (targetInput instanceof TableScan
- || targetInput instanceof Values
- || targetInput instanceof TableFunctionScan
- || targetInput.getInputs().isEmpty()) {
- return new Tuple2<>(targetInput, idxInTargetInput);
- } else {
- return getTargetInputAndIdx(idxInTargetInput, targetInput.getInputs());
- }
+ return haveCommonJoinKey;
}
/**
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.java
index b9eb4f24f11c6..a0805e4b110df 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.java
@@ -33,17 +33,20 @@
import org.apache.calcite.plan.hep.HepRelVertex;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.core.Correlate;
+import org.apache.calcite.rel.core.JoinRelType;
import org.apache.calcite.rel.core.Uncollect;
import org.apache.calcite.rel.logical.LogicalCorrelate;
import org.apache.calcite.rel.logical.LogicalFilter;
import org.apache.calcite.rel.logical.LogicalProject;
import org.apache.calcite.rel.logical.LogicalTableFunctionScan;
import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexNode;
import org.immutables.value.Value;
import java.util.Collections;
import java.util.Map;
+import java.util.stream.Collectors;
import static org.apache.flink.table.types.logical.utils.LogicalTypeUtils.toRowType;
@@ -103,7 +106,10 @@ private RelNode convert(RelNode relNode, LogicalCorrelate correlate) {
relNode = convert(getRel(hepRelVertex), correlate);
}
if (relNode instanceof LogicalProject) {
- LogicalProject logicalProject = (LogicalProject) relNode;
+ final LogicalProject logicalProject =
+ correlate.getJoinType() == JoinRelType.LEFT
+ ? getLogicalProjectWithAdjustedNullability((LogicalProject) relNode)
+ : (LogicalProject) relNode;
return logicalProject.copy(
logicalProject.getTraitSet(),
ImmutableList.of(convert(getRel(logicalProject.getInput()), correlate)));
@@ -161,6 +167,35 @@ private RelNode getRel(RelNode rel) {
return rel;
}
+ /**
+ * If unnesting type is {@code NOT NULL} however at the same time {@code LEFT JOIN} makes it
+ * nullable, this method adjusts nullability by inserting extra {@code CAST}.
+ */
+ private LogicalProject getLogicalProjectWithAdjustedNullability(LogicalProject logicalProject) {
+ final RelOptCluster cluster = logicalProject.getCluster();
+ FlinkTypeFactory typeFactory = (FlinkTypeFactory) cluster.getTypeFactory();
+ RexBuilder rexBuilder = cluster.getRexBuilder();
+ final RelDataType rowType = logicalProject.getRowType();
+ return logicalProject.copy(
+ logicalProject.getTraitSet(),
+ logicalProject.getInput(),
+ logicalProject.getProjects().stream()
+ .map(
+ t -> {
+ if (t.getType().isNullable()) {
+ return t;
+ }
+ return rexBuilder.makeCast(
+ createNullableType(typeFactory, t.getType()), t);
+ })
+ .collect(Collectors.toList()),
+ rowType.isNullable() ? rowType : createNullableType(typeFactory, rowType));
+ }
+
+ private static RelDataType createNullableType(FlinkTypeFactory typeFactory, RelDataType type) {
+ return typeFactory.createTypeWithNullability(type, true);
+ }
+
/** Rule configuration. */
@Value.Immutable(singleton = false)
public interface LogicalUnnestRuleConfig extends RelRule.Config {
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalMultiJoinRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalMultiJoinRule.java
index 84b5de014b096..da3b99d669a73 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalMultiJoinRule.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalMultiJoinRule.java
@@ -32,18 +32,14 @@
import org.apache.calcite.plan.RelTraitSet;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.sql.SqlKind;
-import org.checkerframework.checker.nullness.qual.Nullable;
import java.util.ArrayList;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
+import static org.apache.flink.table.planner.plan.utils.MultiJoinUtil.createJoinAttributeMap;
+
/** Rule that converts {@link FlinkLogicalMultiJoin} to {@link StreamPhysicalMultiJoin}. */
public class StreamPhysicalMultiJoinRule extends ConverterRule {
public static final RelOptRule INSTANCE = new StreamPhysicalMultiJoinRule();
@@ -61,7 +57,7 @@ private StreamPhysicalMultiJoinRule() {
public RelNode convert(final RelNode rel) {
final FlinkLogicalMultiJoin multiJoin = (FlinkLogicalMultiJoin) rel;
final Map> joinAttributeMap =
- createJoinAttributeMap(multiJoin);
+ createJoinAttributeMap(multiJoin.getInputs(), multiJoin.getJoinConditions());
final List inputRowTypes =
multiJoin.getInputs().stream()
.map(i -> FlinkTypeFactory.toLogicalRowType(i.getRowType()))
@@ -117,120 +113,4 @@ private RelTraitSet createInputTraitSet(
return inputTraitSet;
}
-
- private Map> createJoinAttributeMap(
- final FlinkLogicalMultiJoin multiJoin) {
- final Map> joinAttributeMap = new HashMap<>();
- final List inputFieldCounts =
- multiJoin.getInputs().stream()
- .map(input -> input.getRowType().getFieldCount())
- .collect(Collectors.toList());
-
- final List inputOffsets = new ArrayList<>();
- int currentOffset = 0;
- for (final Integer count : inputFieldCounts) {
- inputOffsets.add(currentOffset);
- currentOffset += count;
- }
-
- final List extends RexNode> joinConditions = multiJoin.getJoinConditions();
- for (final RexNode condition : joinConditions) {
- extractEqualityConditions(condition, inputOffsets, inputFieldCounts, joinAttributeMap);
- }
- return joinAttributeMap;
- }
-
- private void extractEqualityConditions(
- final RexNode condition,
- final List inputOffsets,
- final List inputFieldCounts,
- final Map> joinAttributeMap) {
- if (!(condition instanceof RexCall)) {
- return;
- }
-
- final RexCall call = (RexCall) condition;
- final SqlKind kind = call.getOperator().getKind();
-
- if (kind != SqlKind.EQUALS) {
- for (final RexNode operand : call.getOperands()) {
- extractEqualityConditions(
- operand, inputOffsets, inputFieldCounts, joinAttributeMap);
- }
- return;
- }
-
- if (call.getOperands().size() != 2) {
- return;
- }
-
- final RexNode op1 = call.getOperands().get(0);
- final RexNode op2 = call.getOperands().get(1);
-
- if (!(op1 instanceof RexInputRef) || !(op2 instanceof RexInputRef)) {
- return;
- }
-
- final InputRef inputRef1 =
- findInputRef(((RexInputRef) op1).getIndex(), inputOffsets, inputFieldCounts);
- final InputRef inputRef2 =
- findInputRef(((RexInputRef) op2).getIndex(), inputOffsets, inputFieldCounts);
-
- if (inputRef1 == null || inputRef2 == null) {
- return;
- }
-
- final InputRef leftRef;
- final InputRef rightRef;
- if (inputRef1.inputIndex < inputRef2.inputIndex) {
- leftRef = inputRef1;
- rightRef = inputRef2;
- } else {
- leftRef = inputRef2;
- rightRef = inputRef1;
- }
-
- // Special case for input 0:
- // Since we are building attribute references that do left -> right index,
- // we need a special base case for input 0 which has no input to the left.
- // So we do {-1, -1} -> {0, attributeIndex}
- if (leftRef.inputIndex == 0) {
- final ConditionAttributeRef firstAttrRef =
- new ConditionAttributeRef(-1, -1, leftRef.inputIndex, leftRef.attributeIndex);
- joinAttributeMap
- .computeIfAbsent(leftRef.inputIndex, k -> new ArrayList<>())
- .add(firstAttrRef);
- }
-
- final ConditionAttributeRef attrRef =
- new ConditionAttributeRef(
- leftRef.inputIndex,
- leftRef.attributeIndex,
- rightRef.inputIndex,
- rightRef.attributeIndex);
- joinAttributeMap.computeIfAbsent(rightRef.inputIndex, k -> new ArrayList<>()).add(attrRef);
- }
-
- private @Nullable InputRef findInputRef(
- final int fieldIndex,
- final List inputOffsets,
- final List inputFieldCounts) {
- for (int i = 0; i < inputOffsets.size(); i++) {
- final int offset = inputOffsets.get(i);
- if (fieldIndex >= offset && fieldIndex < offset + inputFieldCounts.get(i)) {
- return new InputRef(i, fieldIndex - offset);
- }
- }
- return null;
- }
-
- private static final class InputRef {
- private final int inputIndex;
- private final int attributeIndex;
-
- private InputRef(final int inputIndex, final int attributeIndex) {
- this.inputIndex = inputIndex;
- this.attributeIndex = attributeIndex;
- }
- }
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/DeltaJoinUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/DeltaJoinUtil.java
index e96d5be5324fb..d433ba5ed9df0 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/DeltaJoinUtil.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/DeltaJoinUtil.java
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.plan.utils;
+import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.table.catalog.Index;
import org.apache.flink.table.catalog.ResolvedSchema;
@@ -335,12 +336,14 @@ private static boolean areJoinConditionsSupported(StreamPhysicalJoin join) {
return isFilterOnOneSetOfUpsertKeys(nonEquiCond.get(), upsertKeys);
}
- private static boolean isFilterOnOneSetOfUpsertKeys(
+ @VisibleForTesting
+ protected static boolean isFilterOnOneSetOfUpsertKeys(
RexNode filter, @Nullable Set upsertKeys) {
ImmutableBitSet fieldRefIndices =
ImmutableBitSet.of(
RexNodeExtractor.extractRefInputFields(Collections.singletonList(filter)));
- return upsertKeys.stream().anyMatch(uk -> uk.contains(fieldRefIndices));
+ return upsertKeys != null
+ && upsertKeys.stream().anyMatch(uk -> uk.contains(fieldRefIndices));
}
private static boolean areAllJoinTableScansSupported(StreamPhysicalJoin join) {
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java
index cd24cc9c82b55..2aaeb5e8ea087 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java
@@ -47,6 +47,7 @@
import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecTableSourceScan;
import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecUnion;
import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecValues;
+import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecVectorSearchTableFunction;
import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecWindowTableFunction;
import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecAsyncCalc;
import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecAsyncCorrelate;
@@ -201,6 +202,7 @@ private ExecNodeMetadataUtil() {
add(BatchExecMatch.class);
add(BatchExecOverAggregate.class);
add(BatchExecRank.class);
+ add(BatchExecVectorSearchTableFunction.class);
}
};
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/MultiJoinUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/MultiJoinUtil.java
new file mode 100644
index 0000000000000..59a69599381d9
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/MultiJoinUtil.java
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.plan.utils;
+
+import org.apache.flink.table.runtime.operators.join.stream.keyselector.AttributeBasedJoinKeyExtractor;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlKind;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class MultiJoinUtil {
+ public static Map>
+ createJoinAttributeMap(
+ List joinInputs, List extends RexNode> joinConditions) {
+ final Map>
+ joinAttributeMap = new HashMap<>();
+ final List inputFieldCounts =
+ joinInputs.stream()
+ .map(input -> input.getRowType().getFieldCount())
+ .collect(Collectors.toList());
+
+ final List inputOffsets = new ArrayList<>();
+ int currentOffset = 0;
+ for (final Integer count : inputFieldCounts) {
+ inputOffsets.add(currentOffset);
+ currentOffset += count;
+ }
+
+ for (final RexNode condition : joinConditions) {
+ extractEqualityConditions(condition, inputOffsets, inputFieldCounts, joinAttributeMap);
+ }
+ return joinAttributeMap;
+ }
+
+ private static void extractEqualityConditions(
+ final RexNode condition,
+ final List inputOffsets,
+ final List inputFieldCounts,
+ final Map>
+ joinAttributeMap) {
+ if (!(condition instanceof RexCall)) {
+ return;
+ }
+
+ final RexCall call = (RexCall) condition;
+ final SqlKind kind = call.getOperator().getKind();
+
+ if (kind != SqlKind.EQUALS) {
+ // Only conjunctions (AND) can contain equality conditions that are valid for multijoin.
+ // All other condition types are deferred to the postJoinFilter.
+ if (kind == SqlKind.AND) {
+ for (final RexNode operand : call.getOperands()) {
+ extractEqualityConditions(
+ operand, inputOffsets, inputFieldCounts, joinAttributeMap);
+ }
+ }
+ return;
+ }
+
+ if (call.getOperands().size() != 2) {
+ return;
+ }
+
+ final RexNode op1 = call.getOperands().get(0);
+ final RexNode op2 = call.getOperands().get(1);
+
+ if (!(op1 instanceof RexInputRef) || !(op2 instanceof RexInputRef)) {
+ return;
+ }
+
+ final InputRef inputRef1 =
+ findInputRef(((RexInputRef) op1).getIndex(), inputOffsets, inputFieldCounts);
+ final InputRef inputRef2 =
+ findInputRef(((RexInputRef) op2).getIndex(), inputOffsets, inputFieldCounts);
+
+ if (inputRef1 == null || inputRef2 == null) {
+ return;
+ }
+
+ final InputRef leftRef;
+ final InputRef rightRef;
+ if (inputRef1.inputIndex < inputRef2.inputIndex) {
+ leftRef = inputRef1;
+ rightRef = inputRef2;
+ } else {
+ leftRef = inputRef2;
+ rightRef = inputRef1;
+ }
+
+ // Special case for input 0:
+ // Since we are building attribute references that do left -> right index,
+ // we need a special base case for input 0 which has no input to the left.
+ // So we do {-1, -1} -> {0, attributeIndex}
+ if (leftRef.inputIndex == 0) {
+ final AttributeBasedJoinKeyExtractor.ConditionAttributeRef firstAttrRef =
+ new AttributeBasedJoinKeyExtractor.ConditionAttributeRef(
+ -1, -1, leftRef.inputIndex, leftRef.attributeIndex);
+ joinAttributeMap
+ .computeIfAbsent(leftRef.inputIndex, k -> new ArrayList<>())
+ .add(firstAttrRef);
+ }
+
+ final AttributeBasedJoinKeyExtractor.ConditionAttributeRef attrRef =
+ new AttributeBasedJoinKeyExtractor.ConditionAttributeRef(
+ leftRef.inputIndex,
+ leftRef.attributeIndex,
+ rightRef.inputIndex,
+ rightRef.attributeIndex);
+ joinAttributeMap.computeIfAbsent(rightRef.inputIndex, k -> new ArrayList<>()).add(attrRef);
+ }
+
+ private static @Nullable InputRef findInputRef(
+ final int fieldIndex,
+ final List inputOffsets,
+ final List inputFieldCounts) {
+ for (int i = 0; i < inputOffsets.size(); i++) {
+ final int offset = inputOffsets.get(i);
+ if (fieldIndex >= offset && fieldIndex < offset + inputFieldCounts.get(i)) {
+ return new InputRef(i, fieldIndex - offset);
+ }
+ }
+ return null;
+ }
+
+ private static final class InputRef {
+ private final int inputIndex;
+ private final int attributeIndex;
+
+ private InputRef(final int inputIndex, final int attributeIndex) {
+ this.inputIndex = inputIndex;
+ this.attributeIndex = attributeIndex;
+ }
+ }
+}
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala
index 537adb0746696..c4fac01871059 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala
@@ -301,7 +301,9 @@ object CodeGenUtils {
// ordered by type root definition
case CHAR | VARCHAR => s"$BINARY_STRING.EMPTY_UTF8"
case BOOLEAN => "false"
- case TINYINT | SMALLINT | INTEGER | DATE | TIME_WITHOUT_TIME_ZONE | INTERVAL_YEAR_MONTH => "-1"
+ case TINYINT => "((byte) -1)"
+ case SMALLINT => "((short) -1)"
+ case INTEGER | DATE | TIME_WITHOUT_TIME_ZONE | INTERVAL_YEAR_MONTH => "-1"
case BIGINT | INTERVAL_DAY_TIME => "-1L"
case FLOAT => "-1.0f"
case DOUBLE => "-1.0d"
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGeneratorContext.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGeneratorContext.scala
index 7cbdb50b134c2..3b034de76aac1 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGeneratorContext.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGeneratorContext.scala
@@ -32,7 +32,7 @@ import org.apache.flink.table.runtime.util.collections._
import org.apache.flink.table.types.DataType
import org.apache.flink.table.types.logical._
import org.apache.flink.table.types.logical.LogicalTypeRoot._
-import org.apache.flink.table.utils.DateTimeUtils
+import org.apache.flink.table.utils.{DateTimeUtils, EncodingUtils}
import org.apache.flink.util.InstantiationUtil
import java.time.ZoneId
@@ -645,9 +645,10 @@ class CodeGeneratorContext(
" This is a bug, please file an issue.")
})
+ val escapedQueryStartCurrentDatabase = EncodingUtils.escapeJava(queryStartCurrentDatabase);
reusableMemberStatements.add(s"""
|private static final $BINARY_STRING $fieldTerm =
- |$BINARY_STRING.fromString("$queryStartCurrentDatabase");
+ |$BINARY_STRING.fromString("$escapedQueryStartCurrentDatabase");
|""".stripMargin)
fieldTerm
@@ -982,22 +983,21 @@ class CodeGeneratorContext(
}
/**
- * Adds a reusable string constant to the member area of the generated class.
+ * Adds an already pre-escaped string constant to the reusable member area of the generated class.
*
- * The string must be already escaped with
- * [[org.apache.flink.table.utils.EncodingUtils.escapeJava()]].
+ * The string must be already escaped with [[EncodingUtils.escapeJava()]].
*/
- def addReusableEscapedStringConstant(value: String): String = {
- reusableStringConstants.get(value) match {
+ def addReusablePreEscapedStringConstant(alreadyEscapedValue: String): String = {
+ reusableStringConstants.get(alreadyEscapedValue) match {
case Some(field) => field
case None =>
val field = newName(this, "str")
val stmt =
s"""
- |private final $BINARY_STRING $field = $BINARY_STRING.fromString("$value");
+ |private final $BINARY_STRING $field = $BINARY_STRING.fromString("$alreadyEscapedValue");
""".stripMargin
reusableMemberStatements.add(stmt)
- reusableStringConstants(value) = field
+ reusableStringConstants(alreadyEscapedValue) = field
field
}
}
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala
index 9e3f9976f833c..6c6c1f5d4fe36 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala
@@ -306,10 +306,12 @@ object GenerateUtils {
// as they're not cheap to construct. For the other types, the return term is directly
// the literal value
case CHAR | VARCHAR =>
- val escapedValue =
- EncodingUtils.escapeJava(literalValue.asInstanceOf[BinaryStringData].toString)
- val field = ctx.addReusableEscapedStringConstant(escapedValue)
- generateNonNullLiteral(literalType, field, StringData.fromString(escapedValue))
+ val str = literalValue.asInstanceOf[BinaryStringData]
+ val field = ctx.addReusablePreEscapedStringConstant(EncodingUtils.escapeJava(str.toString))
+ // The original value should be passed as literalValue
+ // all required escaping should be done in corresponding code generation,
+ // so that the literalValue can be also used directly when needed
+ generateNonNullLiteral(literalType, field, str)
case BINARY | VARBINARY =>
val bytesVal = literalValue.asInstanceOf[Array[Byte]]
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/GeneratedExpression.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/GeneratedExpression.scala
index 108ea10d1ada6..325a7608c2706 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/GeneratedExpression.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/GeneratedExpression.scala
@@ -33,8 +33,12 @@ import org.apache.flink.table.types.logical.LogicalType
* @param resultType
* type of the resultTerm
* @param literalValue
- * None if the expression is not literal. Otherwise it represent the original object of the
- * literal.
+ * Contains the literal value (as internal data structure) for deep literal inspection if the
+ * originating expression was a literal. Literal inspection is useful for performance
+ * optimizations. For example, figuring out whether a time parsing function ever produces
+ * sub-second data by inspecting the "format" literal string. NOTE: The literal value is not
+ * intended to be used in generated code, use `resultTerm` for this purpose. The literal value is
+ * NOT escaped.
*/
case class GeneratedExpression(
resultTerm: String,
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/JsonGenerateUtils.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/JsonGenerateUtils.scala
index 823837d2f5ec1..f87e24188121c 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/JsonGenerateUtils.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/JsonGenerateUtils.scala
@@ -31,6 +31,7 @@ import org.apache.flink.table.runtime.typeutils.TypeCheckUtils.isCharacterString
import org.apache.flink.table.types.logical._
import org.apache.flink.table.types.logical.LogicalTypeRoot._
import org.apache.flink.table.types.logical.utils.LogicalTypeChecks
+import org.apache.flink.table.utils.EncodingUtils
import org.apache.calcite.rex.{RexCall, RexNode}
@@ -261,7 +262,7 @@ object JsonGenerateUtils {
/** Generates a method to convert rows into [[ObjectNode]]. */
private def generateRowConverter(ctx: CodeGeneratorContext, rowType: LogicalType): String = {
- val fieldNames = toScala(LogicalTypeChecks.getFieldNames(rowType))
+ val fieldNames = toScala(LogicalTypeChecks.getFieldNames(rowType)).map(EncodingUtils.escapeJava)
val fieldTypes = toScala(LogicalTypeChecks.getFieldTypes(rowType))
val populateObjectCode = fieldNames.zipWithIndex.map {
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/LikeCallGen.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/LikeCallGen.scala
index b3dc9a1911c28..f734389b62970 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/LikeCallGen.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/LikeCallGen.scala
@@ -23,6 +23,7 @@ import org.apache.flink.table.planner.codegen.CodeGenUtils.{className, newName,
import org.apache.flink.table.planner.codegen.GenerateUtils.generateCallIfArgsNotNull
import org.apache.flink.table.runtime.functions.SqlLikeChainChecker
import org.apache.flink.table.types.logical.{BooleanType, LogicalType}
+import org.apache.flink.table.utils.EncodingUtils
import java.util.regex.Pattern
@@ -57,44 +58,47 @@ class LikeCallGen extends CallGenerator {
!pattern.contains("_")
} else {
val escape = operands(2).literalValue.get.toString
- if ((escape.length == 2 && escape.charAt(0) != '\\') || escape.length > 2) {
- throw SqlLikeUtils.invalidEscapeCharacter(escape)
- }
- val escapeChar = escape.charAt(escape.length - 1)
- var matched = true
- var i = 0
- val newBuilder = new StringBuilder
- while (i < pattern.length && matched) {
- var c = pattern.charAt(i)
- if (c == '\\') {
- i += 1
- c = pattern.charAt(i)
+ if (escape.isEmpty) {
+ !pattern.contains("_")
+ } else {
+ if (escape.length > 1) {
+ throw SqlLikeUtils.invalidEscapeCharacter(escape)
}
- if (c == escapeChar) {
- if (i == (pattern.length - 1)) {
- throw SqlLikeUtils.invalidEscapeSequence(pattern, i)
- }
- val nextChar = pattern.charAt(i + 1)
- if (nextChar == '%') {
+ val escapeChar = escape.charAt(escape.length - 1)
+ if (escapeChar == 0) {
+ throw SqlLikeUtils.invalidEscapeCharacter(escape)
+ }
+ var matched = true
+ var i = 0
+ val newBuilder = new StringBuilder
+ while (i < pattern.length && matched) {
+ val c = pattern.charAt(i)
+ if (c == escapeChar) {
+ if (i == (pattern.length - 1)) {
+ throw SqlLikeUtils.invalidEscapeSequence(pattern, i)
+ }
+ val nextChar = pattern.charAt(i + 1)
+ if (nextChar == '%') {
+ matched = false
+ } else if ((nextChar == '_') || (nextChar == escapeChar)) {
+ newBuilder.append(nextChar)
+ i += 1
+ } else {
+ throw SqlLikeUtils.invalidEscapeSequence(pattern, i)
+ }
+ } else if (c == '_') {
matched = false
- } else if ((nextChar == '_') || (nextChar == escapeChar)) {
- newBuilder.append(nextChar)
- i += 1
} else {
- throw SqlLikeUtils.invalidEscapeSequence(pattern, i)
+ newBuilder.append(c)
}
- } else if (c == '_') {
- matched = false
- } else {
- newBuilder.append(c)
+ i += 1
}
- i += 1
- }
- if (matched) {
- newPattern = newBuilder.toString
+ if (matched) {
+ newPattern = newBuilder.toString
+ }
+ matched
}
- matched
}
if (allowQuick) {
@@ -102,23 +106,28 @@ class LikeCallGen extends CallGenerator {
val beginMatcher = BEGIN_PATTERN.matcher(newPattern)
val endMatcher = END_PATTERN.matcher(newPattern)
val middleMatcher = MIDDLE_PATTERN.matcher(newPattern)
+ val escapedNewPattern = EncodingUtils.escapeJava(newPattern)
if (noneMatcher.matches()) {
- val reusePattern = ctx.addReusableEscapedStringConstant(newPattern)
+ val reusePattern = ctx.addReusablePreEscapedStringConstant(escapedNewPattern)
s"${terms.head}.equals($reusePattern)"
} else if (beginMatcher.matches()) {
- val field = ctx.addReusableEscapedStringConstant(beginMatcher.group(1))
+ val escapedStartValue = EncodingUtils.escapeJava(beginMatcher.group(1))
+ val field = ctx.addReusablePreEscapedStringConstant(escapedStartValue)
s"${terms.head}.startsWith($field)"
} else if (endMatcher.matches()) {
- val field = ctx.addReusableEscapedStringConstant(endMatcher.group(1))
+ val escapedEndValue = EncodingUtils.escapeJava(endMatcher.group(1))
+ val field = ctx.addReusablePreEscapedStringConstant(escapedEndValue)
s"${terms.head}.endsWith($field)"
} else if (middleMatcher.matches()) {
- val field = ctx.addReusableEscapedStringConstant(middleMatcher.group(1))
+ val escapedMiddleValue = EncodingUtils.escapeJava(middleMatcher.group(1))
+ val field = ctx.addReusablePreEscapedStringConstant(escapedMiddleValue)
s"${terms.head}.contains($field)"
} else {
val field = className[SqlLikeChainChecker]
val checker = newName(ctx, "likeChainChecker")
- ctx.addReusableMember(s"$field $checker = new $field(${"\""}$newPattern${"\""});")
+ ctx.addReusableMember(
+ s"$field $checker = new $field(${"\""}$escapedNewPattern${"\""});")
s"$checker.check(${terms.head})"
}
} else {
@@ -129,15 +138,18 @@ class LikeCallGen extends CallGenerator {
val escape = if (operands.size == 2) {
"null"
} else {
+ val escapedEscapeLiteral =
+ EncodingUtils.escapeJava(operands(2).literalValue.get.toString)
s"""
- |"${operands(2).literalValue.get}"
+ |"$escapedEscapeLiteral"
""".stripMargin
}
+ val escapedPatternLiteral = EncodingUtils.escapeJava(pattern)
ctx.addReusableMember(
s"""
|$patternClass $patternName =
| $patternClass.compile(
- | $likeClass.sqlToRegexLike("${operands(1).literalValue.get}", $escape));
+ | $likeClass.sqlToRegexLike("$escapedPatternLiteral", $escape));
|""".stripMargin)
s"$patternName.matcher(${terms.head}.toString()).matches()"
}
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala
index 249e73fc4ca12..19812e318f3db 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala
@@ -19,7 +19,7 @@ package org.apache.flink.table.planner.codegen.calls
import org.apache.flink.table.api.ValidationException
import org.apache.flink.table.api.config.ExecutionConfigOptions
-import org.apache.flink.table.data.binary.BinaryArrayData
+import org.apache.flink.table.data.binary.{BinaryArrayData, BinaryStringData}
import org.apache.flink.table.data.util.MapDataUtil
import org.apache.flink.table.data.utils.CastExecutor
import org.apache.flink.table.data.writer.{BinaryArrayWriter, BinaryRowWriter}
@@ -41,6 +41,7 @@ import org.apache.flink.table.types.logical.utils.LogicalTypeChecks
import org.apache.flink.table.types.logical.utils.LogicalTypeChecks.{getFieldTypes, getPrecision, getScale}
import org.apache.flink.table.types.logical.utils.LogicalTypeMerging.findCommonType
import org.apache.flink.table.utils.DateTimeUtils.MILLIS_PER_DAY
+import org.apache.flink.table.utils.EncodingUtils
import org.apache.flink.types.ColumnList
import org.apache.flink.util.Preconditions.checkArgument
@@ -1699,6 +1700,7 @@ object ScalarOperatorGens {
}
try {
+ // No escaping here as it will be done in the primitiveLiteralForType according to the type of the literal value.
val result = castExecutor.cast(literalExpr.literalValue.get)
val resultTerm = newName(ctx, "stringToTime")
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalOverAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalOverAggregate.scala
index 899a171909484..61a35b2fb0044 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalOverAggregate.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalOverAggregate.scala
@@ -52,15 +52,22 @@ class FlinkLogicalOverAggregate(
with FlinkLogicalRel {
override def copy(traitSet: RelTraitSet, inputs: JList[RelNode]): RelNode = {
+ copy(traitSet, inputs, rowType, windowGroups)
+ }
+
+ def copy(
+ traitSet: RelTraitSet,
+ inputs: JList[RelNode],
+ rowType: RelDataType,
+ groups: JList[Window.Group]): RelNode = {
new FlinkLogicalOverAggregate(
cluster,
traitSet,
inputs.get(0),
windowConstants,
- getRowType,
- windowGroups)
+ rowType,
+ groups)
}
-
}
class FlinkLogicalOverAggregateConverter(config: Config) extends ConverterRule(config) {
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/RemoteCalcSplitRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/RemoteCalcSplitRule.scala
index ff60809cea680..26baa44f43d82 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/RemoteCalcSplitRule.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/RemoteCalcSplitRule.scala
@@ -434,6 +434,8 @@ class ScalarFunctionSplitter(
private var fieldsRexCall: Map[Int, Int] = Map[Int, Int]()
+ private val extractedRexNodeRefs: mutable.HashSet[RexNode] = mutable.HashSet[RexNode]()
+
override def visitCall(call: RexCall): RexNode = {
if (needConvert(call)) {
getExtractedRexNode(call)
@@ -454,7 +456,9 @@ class ScalarFunctionSplitter(
new RexInputRef(field.getIndex, field.getType)
case _ =>
val newFieldAccess =
- rexBuilder.makeFieldAccess(expr.accept(this), fieldAccess.getField.getIndex)
+ rexBuilder.makeFieldAccess(
+ convertInputRefToLocalRefIfNecessary(expr.accept(this)),
+ fieldAccess.getField.getIndex)
getExtractedRexNode(newFieldAccess)
}
} else {
@@ -468,9 +472,18 @@ class ScalarFunctionSplitter(
override def visitNode(rexNode: RexNode): RexNode = rexNode
+ private def convertInputRefToLocalRefIfNecessary(node: RexNode): RexNode = {
+ node match {
+ case inputRef: RexInputRef if extractedRexNodeRefs.contains(node) =>
+ new RexLocalRef(inputRef.getIndex, node.getType)
+ case _ => node
+ }
+ }
+
private def getExtractedRexNode(node: RexNode): RexNode = {
val newNode = new RexInputRef(extractedFunctionOffset + extractedRexNodes.length, node.getType)
extractedRexNodes.append(node)
+ extractedRexNodeRefs.add(newNode)
newNode
}
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalCorrelateRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalCorrelateRule.scala
index 9f714f2057c8c..471f40cce18e1 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalCorrelateRule.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalCorrelateRule.scala
@@ -64,7 +64,9 @@ class BatchPhysicalCorrelateRule(config: Config) extends ConverterRule(config) {
case calc: FlinkLogicalCalc =>
convertToCorrelate(
calc.getInput.asInstanceOf[RelSubset].getOriginal,
- Some(calc.getProgram.expandLocalRef(calc.getProgram.getCondition)))
+ if (calc.getProgram.getCondition == null) None
+ else Some(calc.getProgram.expandLocalRef(calc.getProgram.getCondition))
+ )
case scan: FlinkLogicalTableFunctionScan =>
new BatchPhysicalCorrelate(
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/adaptive/AdaptiveJoinOperatorGeneratorTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/adaptive/AdaptiveJoinOperatorGeneratorTest.java
index 3baa5030cbed1..d554465993b7c 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/adaptive/AdaptiveJoinOperatorGeneratorTest.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/adaptive/AdaptiveJoinOperatorGeneratorTest.java
@@ -28,7 +28,7 @@
import org.apache.flink.table.runtime.operators.join.HashJoinOperator;
import org.apache.flink.table.runtime.operators.join.Int2HashJoinOperatorTestBase;
import org.apache.flink.table.runtime.operators.join.SortMergeJoinOperator;
-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.runtime.util.UniformBinaryRowGenerator;
import org.apache.flink.table.types.logical.IntType;
import org.apache.flink.table.types.logical.RowType;
@@ -268,10 +268,15 @@ public Object newOperator(
boolean buildLeft,
boolean isBroadcast,
OperatorType operatorType) {
- AdaptiveJoin adaptiveJoin = genAdaptiveJoin(flinkJoinType, operatorType);
- adaptiveJoin.markAsBroadcastJoin(isBroadcast, buildLeft);
+ AdaptiveJoinGenerator adaptiveJoinGenerator = genAdaptiveJoinGenerator();
- return adaptiveJoin.genOperatorFactory(getClass().getClassLoader(), new Configuration());
+ return adaptiveJoinGenerator.genOperatorFactory(
+ getClass().getClassLoader(),
+ new Configuration(),
+ flinkJoinType,
+ operatorType == SortMergeJoin,
+ isBroadcast,
+ buildLeft);
}
public void assertOperatorType(Object operator, OperatorType expectedOperatorType) {
@@ -301,7 +306,7 @@ public void assertOperatorType(Object operator, OperatorType expectedOperatorTyp
}
}
- public AdaptiveJoin genAdaptiveJoin(FlinkJoinType flinkJoinType, OperatorType operatorType) {
+ public AdaptiveJoinGenerator genAdaptiveJoinGenerator() {
GeneratedJoinCondition condFuncCode =
new GeneratedJoinCondition(
Int2HashJoinOperatorTestBase.MyJoinCondition.class.getCanonicalName(),
@@ -316,7 +321,6 @@ public JoinCondition newInstance(ClassLoader classLoader) {
return new AdaptiveJoinOperatorGenerator(
new int[] {0},
new int[] {0},
- flinkJoinType,
new boolean[] {true},
RowType.of(new IntType(), new IntType()),
RowType.of(new IntType(), new IntType()),
@@ -326,8 +330,6 @@ public JoinCondition newInstance(ClassLoader classLoader) {
20,
10000,
false,
- TABLE_EXEC_RESOURCE_HASH_JOIN_MEMORY.defaultValue().getBytes(),
- true,
- operatorType);
+ TABLE_EXEC_RESOURCE_HASH_JOIN_MEMORY.defaultValue().getBytes());
}
}
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/calcite/FlinkSqlLikeUtilsTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/calcite/FlinkSqlLikeUtilsTest.java
index 86099925f8983..2270541974340 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/calcite/FlinkSqlLikeUtilsTest.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/calcite/FlinkSqlLikeUtilsTest.java
@@ -36,9 +36,14 @@ void testSqlLike() {
assertThat(SqlLikeUtils.like("abcd", "a.*d", "\\")).isEqualTo(false);
assertThat(SqlLikeUtils.like("abcde", "%c.e", "\\")).isEqualTo(false);
- // default escape character
+ // no default escape character - backslash is treated as a literal character
assertThat(SqlLikeUtils.like("a-c", "a\\_c")).isEqualTo(false);
- assertThat(SqlLikeUtils.like("a_c", "a\\_c")).isEqualTo(true);
+ assertThat(SqlLikeUtils.like("a_c", "a\\_c")).isEqualTo(false);
+ assertThat(SqlLikeUtils.like("a\\_c", "a\\_c")).isEqualTo(true);
+
+ // default escape also excludes \u0000
+ assertThat(SqlLikeUtils.like("_", "\u0000_", null)).isEqualTo(false);
+ assertThat(SqlLikeUtils.like("\u0000x", "\u0000_", null)).isEqualTo(true);
// -------------------------------- sqlToRegexLike ----------------------------------------
@@ -66,5 +71,8 @@ void testSqlLike() {
assertThat(SqlLikeUtils.similar("abc", "a.c", "\\")).isEqualTo(true);
assertThat(SqlLikeUtils.similar("a.c", "a.c", "\\")).isEqualTo(true);
assertThat(SqlLikeUtils.similar("abcd", "a.*d", "\\")).isEqualTo(true);
+ // default escape also excludes \u0000
+ assertThat(SqlLikeUtils.similar("_", "\u0000_", null)).isEqualTo(false);
+ assertThat(SqlLikeUtils.similar("\u0000x", "\u0000_", null)).isEqualTo(true);
}
}
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/calcite/SqlTypeUtilTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/calcite/SqlTypeUtilTest.java
new file mode 100644
index 0000000000000..58560bdee9a76
--- /dev/null
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/calcite/SqlTypeUtilTest.java
@@ -0,0 +1,66 @@
+/*
+ * 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.calcite;
+
+import org.apache.flink.table.planner.typeutils.LogicalRelDataTypeConverter;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.VarCharType;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.pretty.SqlPrettyWriter;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.junit.jupiter.api.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for {@link SqlTypeUtil}. */
+class SqlTypeUtilTest {
+ /**
+ * Test case for [FLINK-38913]
+ * ArrayIndexOutOfBoundsException when creating a table with computed rows including casts to
+ * null .
+ */
+ @Test
+ void testConvertRowTypeToSpecAndUnparse() {
+ FlinkTypeFactory typeFactory =
+ new FlinkTypeFactory(
+ Thread.currentThread().getContextClassLoader(), FlinkTypeSystem.INSTANCE);
+ RowType rowType =
+ RowType.of(
+ new LogicalType[] {new IntType(), new VarCharType(1)},
+ new String[] {"a", "b"});
+ RelDataType relDataType = LogicalRelDataTypeConverter.toRelDataType(rowType, typeFactory);
+ SqlDataTypeSpec typeSpec = SqlTypeUtil.convertTypeToSpec(relDataType);
+ SqlWriter writer =
+ new SqlPrettyWriter(
+ SqlPrettyWriter.config()
+ .withAlwaysUseParentheses(false)
+ .withSelectListItemsOnSeparateLines(false)
+ .withIndentation(0));
+ // unparse that will end up passing no comments through
+ typeSpec.unparse(writer, 0, 0);
+ String result = writer.toSqlString().getSql();
+ assertThat(result)
+ .hasToString("ROW(\"a\" INTEGER, \"b\" VARCHAR(1) CHARACTER SET \"UTF-16LE\")");
+ }
+}
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/JsonFunctionsITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/JsonFunctionsITCase.java
index e26401951c4de..02791725aaf86 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/JsonFunctionsITCase.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/JsonFunctionsITCase.java
@@ -53,6 +53,7 @@
import static org.apache.flink.table.api.DataTypes.BOOLEAN;
import static org.apache.flink.table.api.DataTypes.DECIMAL;
import static org.apache.flink.table.api.DataTypes.DOUBLE;
+import static org.apache.flink.table.api.DataTypes.FIELD;
import static org.apache.flink.table.api.DataTypes.INT;
import static org.apache.flink.table.api.DataTypes.MAP;
import static org.apache.flink.table.api.DataTypes.ROW;
@@ -690,6 +691,21 @@ private static List jsonStringSpec() {
jsonString($("f13")),
"JSON_STRING(f13)",
"{\"f0\":[{\"f0\":1,\"f1\":2}]}",
+ STRING().notNull()),
+ TestSetSpec.forFunction(BuiltInFunctionDefinitions.JSON_STRING)
+ .onFieldsWithData(Row.of("val1", "val2", "val3", "val4", "val5"))
+ .andDataTypes(
+ ROW(
+ FIELD("field\"quote", STRING()),
+ FIELD("field\\slash", STRING()),
+ FIELD("field\nline", STRING()),
+ FIELD("field\ttab", STRING()),
+ FIELD("field\rreturn", STRING()))
+ .notNull())
+ .testResult(
+ jsonString($("f0")),
+ "JSON_STRING(f0)",
+ "{\"field\\ttab\":\"val4\",\"field\\nline\":\"val3\",\"field\\rreturn\":\"val5\",\"field\\\"quote\":\"val1\",\"field\\\\slash\":\"val2\"}",
STRING().notNull()));
}
@@ -1062,6 +1078,21 @@ private static List jsonObjectSpec() {
+ "\"R\":{\"f0\":\"V\",\"f1\":null}"
+ "}",
STRING().notNull(),
+ STRING().notNull()),
+ TestSetSpec.forFunction(BuiltInFunctionDefinitions.JSON_OBJECT)
+ .onFieldsWithData(Row.of("val1", "val2", "val3", "val4", "val5"))
+ .andDataTypes(
+ ROW(
+ FIELD("field\"quote", STRING()),
+ FIELD("field\\slash", STRING()),
+ FIELD("field\nline", STRING()),
+ FIELD("field\ttab", STRING()),
+ FIELD("field\rreturn", STRING()))
+ .notNull())
+ .testResult(
+ jsonObject(JsonOnNull.NULL, "testRow", $("f0")),
+ "JSON_OBJECT(KEY 'testRow' VALUE f0 NULL ON NULL)",
+ "{\"testRow\":{\"field\\ttab\":\"val4\",\"field\\nline\":\"val3\",\"field\\rreturn\":\"val5\",\"field\\\"quote\":\"val1\",\"field\\\\slash\":\"val2\"}}",
STRING().notNull()));
}
@@ -1484,6 +1515,21 @@ private static List jsonArraySpec() {
+ "{\"age\":1,\"name\":\"V\",\"payload\":{\"M1\":\"V1\",\"M2\":\"V2\"}}"
+ "]",
STRING().notNull(),
+ STRING().notNull()),
+ TestSetSpec.forFunction(BuiltInFunctionDefinitions.JSON_ARRAY)
+ .onFieldsWithData(Row.of("val1", "val2", "val3", "val4", "val5"))
+ .andDataTypes(
+ ROW(
+ FIELD("field\"quote", STRING()),
+ FIELD("field\\slash", STRING()),
+ FIELD("field\nline", STRING()),
+ FIELD("field\ttab", STRING()),
+ FIELD("field\rreturn", STRING()))
+ .notNull())
+ .testResult(
+ jsonArray(JsonOnNull.NULL, $("f0")),
+ "JSON_ARRAY(f0 NULL ON NULL)",
+ "[{\"field\\ttab\":\"val4\",\"field\\nline\":\"val3\",\"field\\rreturn\":\"val5\",\"field\\\"quote\":\"val1\",\"field\\\\slash\":\"val2\"}]",
STRING().notNull()));
}
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/LikeFunctionITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/LikeFunctionITCase.java
new file mode 100644
index 0000000000000..17981f0fd8c88
--- /dev/null
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/LikeFunctionITCase.java
@@ -0,0 +1,213 @@
+/*
+ * 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.functions;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+
+import java.util.stream.Stream;
+
+/** Integration tests for {@code LIKE [ESCAPE ]} pattern matching operations. */
+class LikeFunctionITCase extends BuiltInFunctionTestBase {
+
+ @Override
+ Stream getTestSetSpecs() {
+ return Stream.of(withEscape(), withoutEscape()).flatMap(s -> s);
+ }
+
+ private Stream withoutEscape() {
+ return Stream.of(
+ TestSetSpec.forFunction(BuiltInFunctionDefinitions.LIKE)
+ .onFieldsWithData("test", "t\"est", "tes\"t", "t\"es\"t")
+ .andDataTypes(
+ DataTypes.STRING(),
+ DataTypes.STRING(),
+ DataTypes.STRING(),
+ DataTypes.STRING())
+
+ // Multiple % with quote in middle segment
+ .testSqlResult("f0 LIKE 'a%b\"c%d'", false, DataTypes.BOOLEAN())
+ .testSqlResult("f0 LIKE 't%es%t'", true, DataTypes.BOOLEAN())
+
+ // Quote in first segment
+ .testSqlResult("f0 LIKE 'a\"b%c%d'", false, DataTypes.BOOLEAN())
+ .testSqlResult("f1 LIKE 't\"e%s%t'", true, DataTypes.BOOLEAN())
+
+ // Quote in last segment
+ .testSqlResult("f0 LIKE 'a%b%c\"d'", false, DataTypes.BOOLEAN())
+ .testSqlResult("f2 LIKE 't%e%s\"t'", true, DataTypes.BOOLEAN())
+
+ // Multiple quotes
+ .testSqlResult("f0 LIKE 'a\"%b\"%c'", false, DataTypes.BOOLEAN())
+ .testSqlResult("f3 LIKE 't\"%s\"%t'", true, DataTypes.BOOLEAN())
+
+ // Pattern with underscore and quote
+ .testSqlResult("f0 LIKE 'te_t\"'", false, DataTypes.BOOLEAN())
+ .testSqlResult("f2 LIKE 'te_\"t'", true, DataTypes.BOOLEAN())
+
+ // Multiple underscores with quotes
+ .testSqlResult("f0 LIKE '_\"_test_\"_'", false, DataTypes.BOOLEAN())
+ .testSqlResult("f3 LIKE '_\"__\"_'", true, DataTypes.BOOLEAN()),
+ TestSetSpec.forFunction(BuiltInFunctionDefinitions.LIKE)
+ .onFieldsWithData("test", "abc%def", "test_123", "hello'world")
+ .andDataTypes(
+ DataTypes.STRING(),
+ DataTypes.STRING(),
+ DataTypes.STRING(),
+ DataTypes.STRING())
+
+ // Normal exact match - no special chars
+ .testSqlResult("f0 LIKE 'test'", true, DataTypes.BOOLEAN())
+
+ // Normal exact match - in case of empty strings
+ .testSqlResult("'' LIKE ''", true, DataTypes.BOOLEAN().notNull())
+ .testSqlResult("'' LIKE '%'", true, DataTypes.BOOLEAN().notNull())
+ .testSqlResult("f0 LIKE ''", false, DataTypes.BOOLEAN())
+ .testSqlResult("f0 LIKE '%%'", true, DataTypes.BOOLEAN())
+
+ // Starts with pattern
+ .testSqlResult("f0 LIKE 'te%'", true, DataTypes.BOOLEAN())
+
+ // Ends with pattern
+ .testSqlResult("f0 LIKE '%st'", true, DataTypes.BOOLEAN())
+
+ // Contains pattern
+ .testSqlResult("f0 LIKE '%es%'", true, DataTypes.BOOLEAN())
+
+ // Single quote in data (not pattern)
+ // SQL escapes single quote as ''
+ .testSqlResult("f3 LIKE '%''%'", true, DataTypes.BOOLEAN())
+
+ // Pattern with % in data matches literal
+ .testSqlResult("f1 LIKE 'abc%def'", true, DataTypes.BOOLEAN())
+
+ // Pattern doesn't match
+ .testSqlResult("f0 LIKE 'orange'", false, DataTypes.BOOLEAN()),
+ TestSetSpec.forFunction(BuiltInFunctionDefinitions.LIKE)
+ .onFieldsWithData("test")
+ .andDataTypes(DataTypes.STRING())
+
+ // With backslash and double quote in the middle
+ .testSqlResult("f0 LIKE 'test\\\"more'", false, DataTypes.BOOLEAN())
+
+ // With backslash at the end
+ .testSqlResult("f0 LIKE 'test\\\\'", false, DataTypes.BOOLEAN()),
+ TestSetSpec.forFunction(BuiltInFunctionDefinitions.LIKE)
+ .onFieldsWithData("test", "\"test", "te\"st", "test\"", "test\\")
+ .andDataTypes(
+ DataTypes.STRING(),
+ DataTypes.STRING(),
+ DataTypes.STRING(),
+ DataTypes.STRING(),
+ DataTypes.STRING())
+
+ // Quick path
+ .testSqlResult("f0 LIKE 'test\"quote'", false, DataTypes.BOOLEAN())
+ .testSqlResult("f2 LIKE 'te\"st'", true, DataTypes.BOOLEAN())
+ .testSqlResult("f0 LIKE '\"test'", false, DataTypes.BOOLEAN())
+ .testSqlResult("f1 LIKE '\"test'", true, DataTypes.BOOLEAN())
+ .testSqlResult("f0 LIKE 'test\"'", false, DataTypes.BOOLEAN())
+ .testSqlResult("f3 LIKE 'test\"'", true, DataTypes.BOOLEAN())
+ .testSqlResult("f0 LIKE 'start\"test%'", false, DataTypes.BOOLEAN())
+ .testSqlResult("f2 LIKE 'te\"s%'", true, DataTypes.BOOLEAN())
+ .testSqlResult("f0 LIKE '%test\"end'", false, DataTypes.BOOLEAN())
+ .testSqlResult("f2 LIKE '%te\"st'", true, DataTypes.BOOLEAN())
+ .testSqlResult("f0 LIKE '%mid\"dle%'", false, DataTypes.BOOLEAN())
+ .testSqlResult("f2 LIKE '%te\"st%'", true, DataTypes.BOOLEAN())
+
+ // Trailing backslash
+ .testSqlResult("f0 LIKE 'test\\'", false, DataTypes.BOOLEAN())
+ .testSqlResult("f4 LIKE 'test\\'", true, DataTypes.BOOLEAN()));
+ }
+
+ private Stream withEscape() {
+ return Stream.of(
+ TestSetSpec.forFunction(BuiltInFunctionDefinitions.LIKE)
+ .onFieldsWithData("test", "test%", "te_st", "te\"st", "test\\", "✅test✅")
+ .andDataTypes(
+ DataTypes.STRING(),
+ DataTypes.STRING(),
+ DataTypes.STRING(),
+ DataTypes.STRING(),
+ DataTypes.STRING(),
+ DataTypes.STRING())
+ // Empty strings in pattern or escape
+ .testSqlResult("f0 LIKE 'test\"end' ESCAPE ''", false, DataTypes.BOOLEAN())
+ .testSqlResult("f0 LIKE '' ESCAPE ''", false, DataTypes.BOOLEAN())
+ // Escaping with emoji
+ .testSqlResult("f0 LIKE 'test' ESCAPE '✅'", true, DataTypes.BOOLEAN())
+ .testSqlResult("f1 LIKE 'test✅%' ESCAPE '✅'", true, DataTypes.BOOLEAN())
+ .testSqlResult("f1 LIKE 'test!%' ESCAPE '!'", true, DataTypes.BOOLEAN())
+ .testSqlResult("f0 LIKE '✅test' ESCAPE '!'", false, DataTypes.BOOLEAN())
+ .testSqlResult("f0 LIKE '✅test' ESCAPE '\\'", false, DataTypes.BOOLEAN())
+ .testSqlResult("f5 LIKE '✅test✅' ESCAPE '\\'", true, DataTypes.BOOLEAN())
+ .testSqlResult("f5 LIKE '✅%✅' ESCAPE '\\'", true, DataTypes.BOOLEAN())
+ .testSqlResult("f5 LIKE '✅%' ESCAPE '\\'", true, DataTypes.BOOLEAN())
+ .testSqlResult("f5 LIKE '%st✅' ESCAPE '\\'", true, DataTypes.BOOLEAN())
+ // Mixed escaped symbols
+ .testSqlResult("f2 LIKE 'te_st' ESCAPE '!'", true, DataTypes.BOOLEAN())
+ .testSqlResult("f2 LIKE 'te__st' ESCAPE '_'", true, DataTypes.BOOLEAN())
+ .testSqlResult("f1 LIKE 'test_%' ESCAPE '_'", true, DataTypes.BOOLEAN())
+ .testSqlResult("f2 LIKE 'te%_st' ESCAPE '%'", true, DataTypes.BOOLEAN())
+ .testSqlResult("f1 LIKE 'test%%' ESCAPE '%'", true, DataTypes.BOOLEAN())
+ .testSqlValidationError(
+ "f2 LIKE 'te_st' ESCAPE '_'", "Invalid escape sequence 'te_st', 2")
+ .testSqlValidationError(
+ "f1 LIKE 'test_' ESCAPE '_'", "Invalid escape sequence 'test_', 4")
+ .testSqlValidationError(
+ "f2 LIKE 'te%st' ESCAPE '%'", "Invalid escape sequence 'te%st', 2")
+ .testSqlValidationError(
+ "f1 LIKE 'test%' ESCAPE '%'", "Invalid escape sequence 'test%', 4")
+ .testSqlValidationError(
+ "f0 LIKE 'test\\\"end' ESCAPE '\\'",
+ "Invalid escape sequence 'test\\\"end', 4")
+ .testSqlValidationError(
+ "f0 LIKE '%e_t%' ESCAPE 'ab'", "Invalid escape character 'ab'")
+ // Mixed
+ .testSqlResult("f0 LIKE 'test\"end' ESCAPE '!'", false, DataTypes.BOOLEAN())
+ .testSqlResult("f3 LIKE 'te\"st' ESCAPE '!'", true, DataTypes.BOOLEAN())
+ .testSqlResult("f4 LIKE 'test\\' ESCAPE '!'", true, DataTypes.BOOLEAN())
+ .testSqlResult(
+ "'a1bc' LIKE CAST('a%\"+1+\"b%c' AS STRING) ESCAPE '!'",
+ false, DataTypes.BOOLEAN().notNull())
+ .testSqlResult(
+ "'a1\"+1+\"bc' LIKE CAST('a%\"+1+\"b%c' AS STRING) ESCAPE '!'",
+ true, DataTypes.BOOLEAN().notNull())
+ // Unicode like sequence
+ .testSqlResult(
+ "f0 LIKE 'test" + "\\u" + "000Aend' ESCAPE '!'",
+ false,
+ DataTypes.BOOLEAN())
+ .testSqlResult(
+ "'test\\u000Aend' LIKE 'test" + "\\u" + "000Aend' ESCAPE '!'",
+ true,
+ DataTypes.BOOLEAN().notNull())
+ // Special characters
+ .testSqlResult(
+ "f0 LIKE '\btest\ne\\nd\f' ESCAPE '!'", false, DataTypes.BOOLEAN())
+ .testSqlResult(
+ "'\btest\ne\\nd\f' LIKE '\btest\ne\\nd\f' ESCAPE '!'",
+ true,
+ DataTypes.BOOLEAN().notNull())
+ // Invalid escape character
+ .testSqlValidationError(
+ "f0 LIKE 'test' ESCAPE '\u0000'",
+ "Invalid escape character '\u0000'"));
+ }
+}
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/common/JoinSemanticTestPrograms.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/common/JoinSemanticTestPrograms.java
new file mode 100644
index 0000000000000..b687f5d4a18b3
--- /dev/null
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/common/JoinSemanticTestPrograms.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.plan.nodes.exec.common;
+
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecJoin;
+import org.apache.flink.table.test.program.SinkTestStep;
+import org.apache.flink.table.test.program.SourceTestStep;
+import org.apache.flink.table.test.program.TableTestProgram;
+import org.apache.flink.types.Row;
+import org.apache.flink.types.RowKind;
+
+import java.util.stream.IntStream;
+
+/** {@link TableTestProgram} definitions for semantic testing {@link StreamExecJoin}. */
+public class JoinSemanticTestPrograms {
+ public static final TableTestProgram OUTER_JOIN_CHANGELOG_TEST =
+ TableTestProgram.of("join-duplicate-emission-bug", "bug with CTE and left join")
+ .setupTableSource(
+ SourceTestStep.newBuilder("upsert_table_with_duplicates")
+ .addSchema(
+ "`execution_plan_id` VARCHAR(2147483647) NOT NULL",
+ "`workflow_id` VARCHAR(2147483647) NOT NULL",
+ "`event_section_id` VARCHAR(2147483647) NOT NULL",
+ "CONSTRAINT `PRIMARY` PRIMARY KEY (`execution_plan_id`, `event_section_id`) NOT ENFORCED")
+ .addOption("changelog-mode", "I, UA,D")
+ .producedValues(
+ IntStream.range(0, 13)
+ .mapToObj(
+ i ->
+ Row.ofKind(
+ RowKind.UPDATE_AFTER,
+ "section_id_1",
+ "section_id_2",
+ "section_id_3"))
+ .toArray(Row[]::new))
+ .build())
+ .setupTableSink(
+ SinkTestStep.newBuilder("sink")
+ .addSchema("event_element_id STRING", "cnt BIGINT")
+ .testMaterializedData()
+ .consumedValues(Row.of("pk-1", 1), Row.of("pk-2", 1))
+ .build())
+ .runSql(
+ "INSERT INTO sink WITH\n"
+ + " section_detail as (\n"
+ + " SELECT s.event_section_id\n"
+ + " \n"
+ + " FROM upsert_table_with_duplicates s\n"
+ + " ),\n"
+ + "\n"
+ + " event_element as (\n"
+ + " SELECT\n"
+ + " ed.id as event_element_id\n"
+ + " FROM (\n"
+ + " SELECT\n"
+ + " 'pk-2' id,\n"
+ + " 'section_id_3' section_id\n"
+ + " UNION ALL\n"
+ + " SELECT\n"
+ + " 'pk-1' id,\n"
+ + " 'section_id_3' section_id\n"
+ + " ) ed \n"
+ + " LEFT JOIN\n"
+ + " section_detail as s\n"
+ + " ON s.event_section_id = ed.section_id\n"
+ + " )\n"
+ + "\n"
+ + "SELECT event_element_id, COUNT(*) cnt\n"
+ + "FROM event_element\n"
+ + "GROUP BY event_element_id")
+ .build();
+
+ public static final TableTestProgram ANTI_JOIN_ON_NESTED =
+ TableTestProgram.of("anti-join-on-nested", "anti join on nested fields")
+ .setupTableSource(
+ SourceTestStep.newBuilder("source_t1")
+ .addSchema("`ext` ROW<`nested` STRING NOT NULL>")
+ .producedValues(Row.of(Row.of("test_same")))
+ .build())
+ .setupTableSource(
+ SourceTestStep.newBuilder("source_t2")
+ .addSchema(
+ "`ext` ROW<`nested` ROW<`nested1` ROW<`nested2` STRING NOT NULL>>>")
+ .producedValues(
+ Row.of(Row.of(Row.of(Row.of("test_diff")))),
+ Row.of(Row.of(Row.of(Row.of("test_same")))))
+ .build())
+ .setupTableSink(
+ SinkTestStep.newBuilder("sink_t")
+ .addSchema("output STRING")
+ .consumedValues("+I[test_diff]")
+ .build())
+ .runSql(
+ "INSERT INTO sink_t SELECT t2.ext.nested.nested1.nested2 FROM source_t2 t2 WHERE"
+ + " NOT EXISTS (SELECT 1 FROM source_t1 t1 WHERE t1.ext.nested = t2.ext.nested.nested1.nested2)")
+ .build();
+}
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/common/JoinTestPrograms.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/common/JoinTestPrograms.java
index 6f41ef4ff1eaf..02d3e92b7110b 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/common/JoinTestPrograms.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/common/JoinTestPrograms.java
@@ -23,9 +23,6 @@
import org.apache.flink.table.test.program.SourceTestStep;
import org.apache.flink.table.test.program.TableTestProgram;
import org.apache.flink.types.Row;
-import org.apache.flink.types.RowKind;
-
-import java.util.stream.IntStream;
/** {@link TableTestProgram} definitions for testing {@link StreamExecJoin}. */
public class JoinTestPrograms {
@@ -45,63 +42,6 @@ public class JoinTestPrograms {
public static final TableTestProgram JOIN_WITH_STATE_TTL_HINT;
public static final TableTestProgram SEMI_ANTI_JOIN_WITH_LITERAL_AGG;
- public static final TableTestProgram OUTER_JOIN_CHANGELOG_TEST =
- TableTestProgram.of("join-duplicate-emission-bug", "bug with CTE and left join")
- .setupTableSource(
- SourceTestStep.newBuilder("upsert_table_with_duplicates")
- .addSchema(
- "`execution_plan_id` VARCHAR(2147483647) NOT NULL",
- "`workflow_id` VARCHAR(2147483647) NOT NULL",
- "`event_section_id` VARCHAR(2147483647) NOT NULL",
- "CONSTRAINT `PRIMARY` PRIMARY KEY (`execution_plan_id`, `event_section_id`) NOT ENFORCED")
- .addOption("changelog-mode", "I, UA,D")
- .producedValues(
- IntStream.range(0, 13)
- .mapToObj(
- i ->
- Row.ofKind(
- RowKind.UPDATE_AFTER,
- "section_id_1",
- "section_id_2",
- "section_id_3"))
- .toArray(Row[]::new))
- .build())
- .setupTableSink(
- SinkTestStep.newBuilder("sink")
- .addSchema("event_element_id STRING", "cnt BIGINT")
- .testMaterializedData()
- .consumedValues(Row.of("pk-1", 1), Row.of("pk-2", 1))
- .build())
- .runSql(
- "INSERT INTO sink WITH\n"
- + " section_detail as (\n"
- + " SELECT s.event_section_id\n"
- + " \n"
- + " FROM upsert_table_with_duplicates s\n"
- + " ),\n"
- + "\n"
- + " event_element as (\n"
- + " SELECT\n"
- + " ed.id as event_element_id\n"
- + " FROM (\n"
- + " SELECT\n"
- + " 'pk-2' id,\n"
- + " 'section_id_3' section_id\n"
- + " UNION ALL\n"
- + " SELECT\n"
- + " 'pk-1' id,\n"
- + " 'section_id_3' section_id\n"
- + " ) ed \n"
- + " LEFT JOIN\n"
- + " section_detail as s\n"
- + " ON s.event_section_id = ed.section_id\n"
- + " )\n"
- + "\n"
- + "SELECT event_element_id, COUNT(*) cnt\n"
- + "FROM event_element\n"
- + "GROUP BY event_element_id")
- .build();
-
static final SourceTestStep EMPLOYEE =
SourceTestStep.newBuilder("EMPLOYEE")
.addSchema("deptno int", "salary bigint", "name varchar")
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RexNodeJsonSerdeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RexNodeJsonSerdeTest.java
index 76b0066256b0d..ad8583a23a50c 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RexNodeJsonSerdeTest.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RexNodeJsonSerdeTest.java
@@ -36,6 +36,7 @@
import org.apache.flink.table.functions.FunctionDefinition;
import org.apache.flink.table.functions.FunctionIdentifier;
import org.apache.flink.table.functions.FunctionKind;
+import org.apache.flink.table.functions.ProcessTableFunction;
import org.apache.flink.table.functions.ScalarFunction;
import org.apache.flink.table.module.Module;
import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
@@ -113,6 +114,7 @@ public class RexNodeJsonSerdeTest {
RexNodeJsonSerdeTest.class.getClassLoader(), FlinkTypeSystem.INSTANCE);
private static final String FUNCTION_NAME = "MyFunc";
private static final String ASYNC_FUNCTION_NAME = "MyAsyncFunc";
+ private static final String PROCESS_TABLE_FUNCTION_NAME = "MyProcessTableFunc";
private static final FunctionIdentifier FUNCTION_SYS_ID = FunctionIdentifier.of(FUNCTION_NAME);
private static final FunctionIdentifier FUNCTION_CAT_ID =
FunctionIdentifier.of(
@@ -120,23 +122,35 @@ public class RexNodeJsonSerdeTest {
private static final FunctionIdentifier ASYNC_FUNCTION_CAT_ID =
FunctionIdentifier.of(
ObjectIdentifier.of(DEFAULT_CATALOG, DEFAULT_DATABASE, ASYNC_FUNCTION_NAME));
+ private static final FunctionIdentifier PROCESS_TABLE_FUNCTION_CAT_ID =
+ FunctionIdentifier.of(
+ ObjectIdentifier.of(
+ DEFAULT_CATALOG, DEFAULT_DATABASE, PROCESS_TABLE_FUNCTION_NAME));
private static final UnresolvedIdentifier UNRESOLVED_FUNCTION_CAT_ID =
UnresolvedIdentifier.of(FUNCTION_CAT_ID.toList());
private static final UnresolvedIdentifier UNRESOLVED_ASYNC_FUNCTION_CAT_ID =
UnresolvedIdentifier.of(ASYNC_FUNCTION_CAT_ID.toList());
+ private static final UnresolvedIdentifier UNRESOLVED_PROCESS_TABLE_FUNCTION_CAT_ID =
+ UnresolvedIdentifier.of(PROCESS_TABLE_FUNCTION_CAT_ID.toList());
private static final SerializableScalarFunction SER_UDF_IMPL = new SerializableScalarFunction();
private static final SerializableAsyncScalarFunction SER_ASYNC_UDF_IMPL =
new SerializableAsyncScalarFunction();
+ private static final SerializableProcessTableFunction SER_PTF_UDF_IMPL =
+ new SerializableProcessTableFunction();
private static final Class SER_UDF_CLASS =
SerializableScalarFunction.class;
private static final Class SER_ASYNC_UDF_CLASS =
SerializableAsyncScalarFunction.class;
+ private static final Class SER_PTF_UDF_CLASS =
+ SerializableProcessTableFunction.class;
private static final OtherSerializableScalarFunction SER_UDF_IMPL_OTHER =
new OtherSerializableScalarFunction();
private static final Class SER_UDF_CLASS_OTHER =
OtherSerializableScalarFunction.class;
private static final NonSerializableScalarFunction NON_SER_UDF_IMPL =
new NonSerializableScalarFunction(true);
+ private static final NonSerializableProcessTableFunction NON_SER_PTF_UDF_IMPL =
+ new NonSerializableProcessTableFunction(true);
private static final NonSerializableFunctionDefinition NON_SER_FUNCTION_DEF_IMPL =
new NonSerializableFunctionDefinition();
private static final ContextResolvedFunction PERMANENT_FUNCTION =
@@ -180,7 +194,13 @@ public void testInlineFunction() throws IOException {
serdeContext, ContextResolvedFunction.anonymous(SER_ASYNC_UDF_IMPL)),
RexNode.class);
- // Non-serializable function due to fields
+ // Serializable process table function
+ testJsonRoundTrip(
+ createFunctionCall(
+ serdeContext, ContextResolvedFunction.anonymous(SER_PTF_UDF_IMPL)),
+ RexNode.class);
+
+ // Non-serializable scalar function due to fields
assertThatThrownBy(
() ->
toJson(
@@ -193,6 +213,20 @@ public void testInlineFunction() throws IOException {
anyCauseMatches(
TableException.class,
"The function's implementation class must not be stateful"));
+
+ // Non-serializable process table function due to fields
+ assertThatThrownBy(
+ () ->
+ toJson(
+ serdeContext,
+ createFunctionCall(
+ serdeContext,
+ ContextResolvedFunction.anonymous(
+ NON_SER_PTF_UDF_IMPL))))
+ .satisfies(
+ anyCauseMatches(
+ TableException.class,
+ "The function's implementation class must not be stateful"));
}
@Test
@@ -863,6 +897,14 @@ private static SerdeContext serdeContextWithPermanentFunction(
UNRESOLVED_ASYNC_FUNCTION_CAT_ID,
FunctionDescriptor.forFunctionClass(SER_ASYNC_UDF_CLASS).build(),
false);
+ serdeContext
+ .getFlinkContext()
+ .getFunctionCatalog()
+ .registerCatalogFunction(
+ UNRESOLVED_PROCESS_TABLE_FUNCTION_CAT_ID,
+ FunctionDescriptor.forFunctionClass(SER_PTF_UDF_CLASS).build(),
+ false);
+
return serdeContext;
}
@@ -977,6 +1019,24 @@ public void eval(CompletableFuture res, Integer i) {
}
}
+ /** Serializable process table function. */
+ public static class SerializableProcessTableFunction extends ProcessTableFunction {
+
+ @SuppressWarnings("unused")
+ public void eval(Integer i) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public TypeInference getTypeInference(DataTypeFactory typeFactory) {
+ return TypeInference.newBuilder()
+ .typedArguments(DataTypes.INT())
+ .outputTypeStrategy(TypeStrategies.explicit(DataTypes.STRING()))
+ .disableSystemArguments(true)
+ .build();
+ }
+ }
+
/** Non-serializable function. */
public static class NonSerializableScalarFunction extends ScalarFunction {
@SuppressWarnings({"FieldCanBeLocal", "unused"})
@@ -992,6 +1052,30 @@ public String eval(Integer i) {
}
}
+ /** Non-serializable process table function. */
+ public static class NonSerializableProcessTableFunction extends ProcessTableFunction {
+ @SuppressWarnings({"FieldCanBeLocal", "unused"})
+ private final boolean flag;
+
+ public NonSerializableProcessTableFunction(boolean flag) {
+ this.flag = flag;
+ }
+
+ @SuppressWarnings("unused")
+ public void eval(Integer i) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public TypeInference getTypeInference(DataTypeFactory typeFactory) {
+ return TypeInference.newBuilder()
+ .typedArguments(DataTypes.INT())
+ .outputTypeStrategy(TypeStrategies.explicit(DataTypes.STRING()))
+ .disableSystemArguments(true)
+ .build();
+ }
+ }
+
/** Non-serializable function definition. */
public static class NonSerializableFunctionDefinition implements FunctionDefinition {
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinSemanticTests.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinSemanticTests.java
index b7970e3aa3fd5..2501a7e626140 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinSemanticTests.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinSemanticTests.java
@@ -18,7 +18,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
-import org.apache.flink.table.planner.plan.nodes.exec.common.JoinTestPrograms;
+import org.apache.flink.table.planner.plan.nodes.exec.common.JoinSemanticTestPrograms;
import org.apache.flink.table.planner.plan.nodes.exec.testutils.SemanticTestBase;
import org.apache.flink.table.test.program.TableTestProgram;
@@ -28,6 +28,8 @@
public class JoinSemanticTests extends SemanticTestBase {
@Override
public List programs() {
- return List.of(JoinTestPrograms.OUTER_JOIN_CHANGELOG_TEST);
+ return List.of(
+ JoinSemanticTestPrograms.OUTER_JOIN_CHANGELOG_TEST,
+ JoinSemanticTestPrograms.ANTI_JOIN_ON_NESTED);
}
}
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRuleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRuleTest.java
index 444c17fee7e49..53ba394f5f07d 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRuleTest.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRuleTest.java
@@ -62,7 +62,8 @@ public void setup() {
+ " a int,\n"
+ " b bigint,\n"
+ " c string,\n"
- + " d ARRAY\n"
+ + " d ARRAY,\n"
+ + " e ROW, g string>"
+ ") WITH (\n"
+ " 'connector' = 'test-simple-table-source'\n"
+ ") ;");
@@ -182,6 +183,12 @@ public void testFieldAccessAfter() {
util.verifyRelPlan(sqlQuery);
}
+ @Test
+ public void testCompositeFieldAsInput() {
+ String sqlQuery = "SELECT func1(e.f.h) from MyTable";
+ util.verifyRelPlan(sqlQuery);
+ }
+
@Test
public void testFieldOperand() {
String sqlQuery = "SELECT func1(func5(a).f0) from MyTable";
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRuleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRuleTest.java
index b98403a28466a..862aedb421b24 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRuleTest.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRuleTest.java
@@ -62,7 +62,8 @@ public void setup() {
+ " a int,\n"
+ " b bigint,\n"
+ " c string,\n"
- + " d ARRAY\n"
+ + " d ARRAY,\n"
+ + " e ROW, g string>\n"
+ ") WITH (\n"
+ " 'connector' = 'test-simple-table-source'\n"
+ ") ;");
@@ -110,6 +111,12 @@ public void testCorrelateWithCast() {
util.verifyRelPlan(sqlQuery);
}
+ @Test
+ public void testCorrelateWithCompositeFieldAsInput() {
+ String sqlQuery = "select * FROM MyTable, LATERAL TABLE(asyncTableFunc(e.f.h))";
+ util.verifyRelPlan(sqlQuery);
+ }
+
/** Test function. */
public static class AsyncFunc extends AsyncTableFunction {
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.java
index a832696fbda74..81cc1a10ef9e3 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.java
@@ -27,6 +27,7 @@
import org.apache.flink.table.planner.utils.TableTestBase;
import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import scala.Enumeration;
@@ -50,7 +51,7 @@ void setup() {
util.tableEnv()
.executeSql(
"CREATE TABLE Users ("
- + " user_id_0 STRING PRIMARY KEY NOT ENFORCED,"
+ + " user_id STRING PRIMARY KEY NOT ENFORCED,"
+ " name STRING,"
+ " cash INT"
+ ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
@@ -59,7 +60,7 @@ void setup() {
.executeSql(
"CREATE TABLE Orders ("
+ " order_id STRING PRIMARY KEY NOT ENFORCED,"
- + " user_id_1 STRING,"
+ + " user_id STRING,"
+ " product STRING"
+ ") WITH ('connector' = 'values', 'changelog-mode' = 'I,D')");
@@ -68,16 +69,26 @@ void setup() {
"CREATE TABLE Payments ("
+ " payment_id STRING PRIMARY KEY NOT ENFORCED,"
+ " price INT,"
- + " user_id_2 STRING"
+ + " user_id STRING"
+ ") WITH ('connector' = 'values', 'changelog-mode' = 'I')");
util.tableEnv()
.executeSql(
"CREATE TABLE Shipments ("
+ " location STRING,"
- + " user_id_3 STRING"
+ + " user_id STRING"
+ ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,UB,D')");
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE Detail ("
+ + " detail_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " description STRING,"
+ + " user_id STRING,"
+ + " data STRING,"
+ + " `timestamp` BIGINT"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I')");
+
// Tables for testing temporal join exclusion
util.tableEnv()
.executeSql(
@@ -101,8 +112,8 @@ void setup() {
"CREATE TABLE EventTable1 ("
+ " id STRING,"
+ " val INT,"
- + " rowtime TIMESTAMP(3),"
- + " WATERMARK FOR rowtime AS rowtime - INTERVAL '5' SECOND"
+ + " `$rowtime` TIMESTAMP(3),"
+ + " WATERMARK FOR `$rowtime` AS `$rowtime` - INTERVAL '5' SECOND"
+ ") WITH ('connector' = 'values', 'changelog-mode' = 'I')");
util.tableEnv()
@@ -110,15 +121,15 @@ void setup() {
"CREATE TABLE EventTable2 ("
+ " id STRING,"
+ " price DOUBLE,"
- + " rowtime TIMESTAMP(3),"
- + " WATERMARK FOR rowtime AS rowtime - INTERVAL '5' SECOND"
+ + " `$rowtime` TIMESTAMP(3),"
+ + " WATERMARK FOR `$rowtime` AS `$rowtime` - INTERVAL '5' SECOND"
+ ") WITH ('connector' = 'values', 'changelog-mode' = 'I')");
// Tables for testing time attribute materialization in multi-join
util.tableEnv()
.executeSql(
"CREATE TABLE UsersWithProctime ("
- + " user_id_0 STRING PRIMARY KEY NOT ENFORCED,"
+ + " user_id STRING PRIMARY KEY NOT ENFORCED,"
+ " name STRING,"
+ " proctime AS PROCTIME()"
+ ") WITH ('connector' = 'values', 'changelog-mode' = 'I')");
@@ -127,9 +138,9 @@ void setup() {
.executeSql(
"CREATE TABLE OrdersWithRowtime ("
+ " order_id STRING PRIMARY KEY NOT ENFORCED,"
- + " user_id_1 STRING,"
- + " rowtime TIMESTAMP(3),"
- + " WATERMARK FOR rowtime AS rowtime"
+ + " user_id STRING,"
+ + " `$rowtime` TIMESTAMP(3),"
+ + " WATERMARK FOR `$rowtime` AS `$rowtime`"
+ ") WITH ('connector' = 'values', 'changelog-mode' = 'I')");
// Tables for testing upsert key preservation
util.tableEnv()
@@ -170,64 +181,109 @@ void setup() {
@Test
void testThreeWayInnerJoinRelPlan() {
util.verifyRelPlan(
- "SELECT u.user_id_0, u.name, o.order_id, p.payment_id "
- + "FROM Users u "
- + "INNER JOIN Orders o ON u.user_id_0 = o.user_id_1 "
- + "INNER JOIN Payments p ON u.user_id_0 = p.user_id_2");
+ "\nSELECT\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " o.order_id,\n"
+ + " p.payment_id\n"
+ + "FROM Users u\n"
+ + "INNER JOIN Orders o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "INNER JOIN Payments p\n"
+ + " ON u.user_id = p.user_id");
}
@Test
- void testThreeWayInnerJoinNoCommonJoinKeyRelPlan() {
+ @Tag("no-common-join-key")
+ void testThreeWayInnerJoinRelPlanNoCommonJoinKey() {
util.verifyRelPlan(
- "SELECT u.user_id_0, u.name, o.order_id, p.payment_id "
- + "FROM Users u "
- + "INNER JOIN Orders o ON u.user_id_0 = o.user_id_1 "
- + "INNER JOIN Payments p ON u.cash = p.price");
+ "\nSELECT\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " o.order_id,\n"
+ + " p.payment_id\n"
+ + "FROM Users u\n"
+ + "INNER JOIN Orders o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "INNER JOIN Payments p\n"
+ + " ON u.cash = p.price");
}
@Test
void testThreeWayInnerJoinExecPlan() {
util.verifyExecPlan(
- "SELECT u.user_id_0, u.name, o.order_id, p.payment_id "
- + "FROM Users u "
- + "INNER JOIN Orders o ON u.user_id_0 = o.user_id_1 "
- + "INNER JOIN Payments p ON u.user_id_0 = p.user_id_2");
+ "\nSELECT\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " o.order_id,\n"
+ + " p.payment_id\n"
+ + "FROM Users u\n"
+ + "INNER JOIN Orders o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "INNER JOIN Payments p\n"
+ + " ON u.user_id = p.user_id");
}
@Test
void testThreeWayLeftOuterJoinRelPlan() {
util.verifyRelPlan(
- "SELECT u.user_id_0, u.name, o.order_id, p.payment_id "
- + "FROM Users u "
- + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 "
- + "LEFT JOIN Payments p ON u.user_id_0 = p.user_id_2");
+ "\nSELECT\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " o.order_id,\n"
+ + " p.payment_id\n"
+ + "FROM Users u\n"
+ + "LEFT JOIN Orders o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "LEFT JOIN Payments p\n"
+ + " ON u.user_id = p.user_id");
}
@Test
void testThreeWayInnerJoinWithTttlHints() {
util.verifyRelPlan(
- "SELECT /*+ STATE_TTL(u='1d', o='2d', p='1h') */u.user_id_0, u.name, o.order_id, p.payment_id "
- + "FROM Users u "
- + "INNER JOIN Orders o ON u.user_id_0 = o.user_id_1 "
- + "INNER JOIN Payments p ON u.user_id_0 = p.user_id_2");
+ "\nSELECT\n"
+ + " /*+ STATE_TTL(u='1d', o='2d', p='1h') */\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " o.order_id,\n"
+ + " p.payment_id\n"
+ + "FROM Users u\n"
+ + "INNER JOIN Orders o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "INNER JOIN Payments p\n"
+ + " ON u.user_id = p.user_id");
}
@Test
void testThreeWayInnerJoinWithSingleTttlHint() {
util.verifyRelPlan(
- "SELECT /*+ STaTE_tTL(o='2d') */u.user_id_0, u.name, o.order_id, p.payment_id "
- + "FROM Users u "
- + "INNER JOIN Orders o ON u.user_id_0 = o.user_id_1 "
- + "INNER JOIN Payments p ON u.user_id_0 = p.user_id_2");
+ "\nSELECT\n"
+ + " /*+ STaTE_tTL(o='2d') */\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " o.order_id,\n"
+ + " p.payment_id\n"
+ + "FROM Users u\n"
+ + "INNER JOIN Orders o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "INNER JOIN Payments p\n"
+ + " ON u.user_id = p.user_id");
}
@Test
void testThreeWayLeftOuterJoinExecPlan() {
util.verifyExecPlan(
- "SELECT u.user_id_0, u.name, o.order_id, p.payment_id "
- + "FROM Users u "
- + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 "
- + "LEFT JOIN Payments p ON u.user_id_0 = p.user_id_2");
+ "\nSELECT\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " o.order_id,\n"
+ + " p.payment_id\n"
+ + "FROM Users u\n"
+ + "LEFT JOIN Orders o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "LEFT JOIN Payments p\n"
+ + " ON u.user_id = p.user_id");
}
@Test
@@ -236,154 +292,241 @@ void testTwoWayJoinWithUnion() {
.executeSql(
"CREATE TABLE Orders2 ("
+ " order_id STRING PRIMARY KEY NOT ENFORCED,"
- + " user_id_1 STRING,"
+ + " user_id STRING,"
+ " product STRING"
+ ") WITH ('connector' = 'values', 'changelog-mode' = 'I,D')");
util.verifyRelPlan(
- "WITH OrdersUnion as ("
- + "SELECT * FROM Orders "
- + "UNION ALL "
- + "SELECT * FROM Orders2"
- + ") "
- + "SELECT * FROM OrdersUnion o "
- + "LEFT JOIN Users u "
- + "ON o.user_id_1 = u.user_id_0");
+ "\nWITH OrdersUnion as (\n"
+ + "SELECT * FROM Orders\n"
+ + "UNION ALL\n"
+ + "SELECT * FROM Orders2\n"
+ + ")\n"
+ + "SELECT * FROM OrdersUnion o\n"
+ + "LEFT JOIN Users u\n"
+ + " ON o.user_id = u.user_id");
}
@Test
void testTwoWayJoinWithRank() {
- util.getTableEnv()
- .getConfig()
- .set(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true);
-
- util.verifyRelPlan(
- "WITH JoinedEvents as ("
- + "SELECT e1.id as id, e1.val, e1.rowtime as `rowtime`, e2.price "
- + "FROM EventTable1 e1 "
- + "JOIN EventTable2 e2 ON e1.id = e2.id) "
- + "SELECT id, val, `rowtime` FROM ("
- + "SELECT *, "
- + "ROW_NUMBER() OVER (PARTITION BY id ORDER BY `rowtime` DESC) as ts "
- + "FROM JoinedEvents) "
+ util.verifyRelPlan(
+ "\nWITH JoinedEvents as (\n"
+ + "SELECT\n"
+ + " e1.id as id,\n"
+ + " e1.val,\n"
+ + " e1.`$rowtime` as `$rowtime`,\n"
+ + " e2.price\n"
+ + "FROM EventTable1 e1\n"
+ + "JOIN EventTable2 e2\n"
+ + " ON e1.id = e2.id)\n"
+ + "SELECT\n"
+ + " id,\n"
+ + " val,\n"
+ + " `$rowtime`\n"
+ + "FROM (\n"
+ + " SELECT\n"
+ + " *,\n"
+ + " ROW_NUMBER() OVER (PARTITION BY id ORDER BY `$rowtime` DESC) as ts\n"
+ + " FROM JoinedEvents)\n"
+ "WHERE ts = 1");
}
@Test
void testFourWayComplexJoinRelPlan() {
util.verifyRelPlan(
- "SELECT u.user_id_0, u.name, o.order_id, p.payment_id, s.location "
- + "FROM Users u "
- + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 "
- + "INNER JOIN Payments p ON u.user_id_0 = p.user_id_2 AND (u.cash >= p.price OR p.price < 0) "
- + "LEFT JOIN Shipments s ON p.user_id_2 = s.user_id_3");
+ "\nSELECT\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " o.order_id,\n"
+ + " p.payment_id,\n"
+ + " s.location\n"
+ + "FROM Users u\n"
+ + "LEFT JOIN Orders o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "INNER JOIN Payments p\n"
+ + " ON u.user_id = p.user_id\n"
+ + " AND (u.cash >= p.price OR p.price < 0)\n"
+ + "LEFT JOIN Shipments s\n"
+ + " ON p.user_id = s.user_id");
}
@Test
- void testThreeWayJoinNoJoinKeyExecPlan() {
+ @Tag("no-common-join-key")
+ void testThreeWayJoinExecPlanNoCommonJoinKey() {
util.verifyExecPlan(
- "SELECT u.user_id_0, u.name, o.order_id, p.payment_id "
- + "FROM Users u "
- + "LEFT JOIN Orders o ON TRUE "
- + "INNER JOIN Payments p ON TRUE ");
+ "\nSELECT\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " o.order_id,\n"
+ + " p.payment_id\n"
+ + "FROM Users u\n"
+ + "LEFT JOIN Orders o ON TRUE\n"
+ + "INNER JOIN Payments p ON TRUE");
}
@Test
- void testFourWayJoinNoCommonJoinKeyRelPlan() {
+ @Tag("no-common-join-key")
+ void testFourWayJoinRelPlanNoCommonJoinKey() {
util.verifyRelPlan(
- "SELECT u.user_id_0, u.name, o.order_id, p.payment_id, s.location "
- + "FROM Users u "
- + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 "
- + "INNER JOIN Payments p ON u.user_id_0 = p.user_id_2 "
- + "LEFT JOIN Shipments s ON p.payment_id = s.user_id_3");
+ "\nSELECT\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " o.order_id,\n"
+ + " p.payment_id,\n"
+ + " s.location\n"
+ + "FROM Users u\n"
+ + "LEFT JOIN Orders o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "LEFT JOIN LookupTable\n"
+ + " ON u.name = LookupTable.name\n"
+ + "LEFT JOIN Payments p\n"
+ + " ON u.user_id = p.user_id\n"
+ + "LEFT JOIN Shipments s\n"
+ + " ON o.user_id = s.user_id");
}
@Test
void testFourWayComplexJoinExecPlan() {
util.verifyExecPlan(
- "SELECT u.user_id_0, u.name, o.order_id, p.payment_id, s.location "
- + "FROM Users u "
- + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 "
- + "INNER JOIN Payments p ON u.user_id_0 = p.user_id_2 AND (u.cash >= p.price OR p.price < 0) "
- + "LEFT JOIN Shipments s ON p.user_id_2 = s.user_id_3");
+ "\nSELECT\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " o.order_id,\n"
+ + " p.payment_id,\n"
+ + " s.location\n"
+ + "FROM Users u\n"
+ + "LEFT JOIN Orders o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "INNER JOIN Payments p\n"
+ + " ON u.user_id = p.user_id\n"
+ + " AND (u.cash >= p.price OR p.price < 0)\n"
+ + "LEFT JOIN Shipments s\n"
+ + " ON p.user_id = s.user_id");
}
@Test
void testThreeWayInnerJoinExplain() {
util.verifyExplain(
- "SELECT u.user_id_0, u.name, o.order_id, p.payment_id "
- + "FROM Users u "
- + "INNER JOIN Orders o ON u.user_id_0 = o.user_id_1 "
- + "INNER JOIN Payments p ON u.user_id_0 = p.user_id_2");
+ "\nSELECT\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " o.order_id,\n"
+ + " p.payment_id\n"
+ + "FROM Users u\n"
+ + "INNER JOIN Orders o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "INNER JOIN Payments p\n"
+ + " ON u.user_id = p.user_id");
}
@Test
void testThreeWayLeftOuterJoinExplain() {
util.verifyExplain(
- "SELECT u.user_id_0, u.name, o.order_id, p.payment_id "
- + "FROM Users u "
- + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 "
- + "LEFT JOIN Payments p ON u.user_id_0 = p.user_id_2");
+ "\nSELECT\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " o.order_id,\n"
+ + " p.payment_id\n"
+ + "FROM Users u\n"
+ + "LEFT JOIN Orders o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "LEFT JOIN Payments p\n"
+ + " ON u.user_id = p.user_id");
}
@Test
void testFourWayComplexJoinExplain() {
util.verifyExplain(
- "SELECT u.user_id_0, u.name, o.order_id, p.payment_id, s.location "
- + "FROM Users u "
- + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 "
- + "INNER JOIN Payments p ON u.user_id_0 = p.user_id_2 AND (u.cash >= p.price OR p.price < 0) "
- + "LEFT JOIN Shipments s ON p.user_id_2 = s.user_id_3");
+ "\nSELECT\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " o.order_id,\n"
+ + " p.payment_id,\n"
+ + " s.location\n"
+ + "FROM Users u\n"
+ + "LEFT JOIN Orders o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "INNER JOIN Payments p\n"
+ + " ON u.user_id = p.user_id\n"
+ + " AND (u.cash >= p.price OR p.price < 0)\n"
+ + "LEFT JOIN Shipments s\n"
+ + " ON p.user_id = s.user_id");
}
@Test
void testTemporalJoinExcludedFromMultiJoin() {
// Temporal joins should remain as lookup joins, not be merged into MultiJoin
util.verifyRelPlan(
- "SELECT s.user_id, s.amount, l.name, l.age "
- + "FROM StreamTable s "
- + "JOIN LookupTable FOR SYSTEM_TIME AS OF s.proctime AS l "
- + "ON s.user_id = l.id");
+ "\nSELECT\n"
+ + " s.user_id,\n"
+ + " s.amount,\n"
+ + " l.name,\n"
+ + " l.age\n"
+ + "FROM StreamTable s\n"
+ + "JOIN LookupTable FOR SYSTEM_TIME AS OF s.proctime AS l\n"
+ + " ON s.user_id = l.id");
}
@Test
void testIntervalJoinExcludedFromMultiJoin() {
// Interval joins (event-time and processing-time) should remain as interval joins
util.verifyRelPlan(
- "SELECT e1.id, e1.val, e2.price "
- + "FROM EventTable1 e1 "
- + "JOIN EventTable2 e2 ON e1.id = e2.id "
- + "AND e1.rowtime BETWEEN e2.rowtime - INTERVAL '1' MINUTE "
- + "AND e2.rowtime + INTERVAL '1' MINUTE");
+ "\nSELECT\n"
+ + " e1.id,\n"
+ + " e1.val,\n"
+ + " e2.price\n"
+ + "FROM EventTable1 e1\n"
+ + "JOIN EventTable2 e2\n"
+ + " ON e1.id = e2.id\n"
+ + " AND e1.`$rowtime` BETWEEN e2.`$rowtime` - INTERVAL '1' MINUTE\n"
+ + " AND e2.`$rowtime` + INTERVAL '1' MINUTE");
}
@Test
void testThreeWayLeftOuterJoinWithWhereClauseRelPlan() {
util.verifyRelPlan(
- "SELECT u.user_id_0, u.name, o.order_id, p.payment_id "
- + "FROM Users u "
- + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 "
- + "LEFT JOIN Payments p ON u.user_id_0 = p.user_id_2 "
+ "\nSELECT\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " o.order_id,\n"
+ + " p.payment_id\n"
+ + "FROM Users u\n"
+ + "LEFT JOIN Orders o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "LEFT JOIN Payments p\n"
+ + " ON u.user_id = p.user_id\n"
+ "WHERE u.name = 'Gus' AND p.price > 10");
}
@Test
void testThreeWayLeftOuterJoinWithWhereClauseExecPlan() {
util.verifyExecPlan(
- "SELECT u.user_id_0, u.name, o.order_id, p.payment_id "
- + "FROM Users u "
- + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 "
- + "LEFT JOIN Payments p ON u.user_id_0 = p.user_id_2 "
+ "\nSELECT\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " o.order_id,\n"
+ + " p.payment_id\n"
+ + "FROM Users u\n"
+ + "LEFT JOIN Orders o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "LEFT JOIN Payments p\n"
+ + " ON u.user_id = p.user_id\n"
+ "WHERE u.name = 'Gus' AND p.price > 10");
}
@Test
void testThreeWayLeftOuterJoinWithWhereClauseExplain() {
util.verifyExplain(
- "SELECT u.user_id_0, u.name, o.order_id, p.payment_id "
- + "FROM Users u "
- + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 "
- + "LEFT JOIN Payments p ON u.user_id_0 = p.user_id_2 "
+ "\nSELECT\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " o.order_id,\n"
+ + " p.payment_id\n"
+ + "FROM Users u\n"
+ + "LEFT JOIN Orders o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "LEFT JOIN Payments p\n"
+ + " ON u.user_id = p.user_id\n"
+ "WHERE u.name = 'Gus' AND p.price > 10");
}
@@ -391,53 +534,84 @@ void testThreeWayLeftOuterJoinWithWhereClauseExplain() {
void testRegularJoinsAreMergedApartFromTemporalJoin() {
// Regular joins should still be eligible for MultiJoin but not mixed with temporal joins
util.verifyRelPlan(
- "SELECT u.user_id_0, u.name, o.order_id, temporal.age "
- + "FROM Users u "
- + "INNER JOIN Orders o ON u.user_id_0 = o.user_id_1 "
- + "INNER JOIN ("
- + " SELECT s.user_id, l.age "
- + " FROM StreamTable s "
- + " JOIN LookupTable FOR SYSTEM_TIME AS OF s.proctime AS l "
- + " ON s.user_id = l.id"
- + ") temporal ON u.user_id_0 = temporal.user_id");
+ "\nSELECT\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " o.order_id,\n"
+ + " temporal.age "
+ + "FROM Users u\n"
+ + "INNER JOIN Orders o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "INNER JOIN (\n"
+ + " SELECT s.user_id, l.age\n"
+ + " FROM StreamTable s\n"
+ + " JOIN LookupTable FOR SYSTEM_TIME AS OF s.proctime AS l\n"
+ + " ON s.user_id = l.id\n"
+ + ") temporal ON u.user_id = temporal.user_id");
}
@Test
void testFourWayJoinTransitiveCommonJoinKeyRelPlan() {
util.verifyRelPlan(
- "SELECT u.user_id_0, u.name, o.order_id, p.payment_id, s.location "
- + "FROM Users u "
- + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 "
- + "LEFT JOIN Payments p ON o.user_id_1 = p.user_id_2 "
- + "LEFT JOIN Shipments s ON p.user_id_2 = s.user_id_3");
+ "\nSELECT\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " o.order_id,\n"
+ + " p.payment_id,\n"
+ + " s.location\n"
+ + "FROM Users u\n"
+ + "LEFT JOIN Orders o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "LEFT JOIN Payments p\n"
+ + " ON o.user_id = p.user_id\n"
+ + "LEFT JOIN Shipments s\n"
+ + " ON p.user_id = s.user_id");
}
/* Update this to supported with FLINK-37973 https://issues.apache.org/jira/browse/FLINK-37973 */
@Test
void testRightJoinNotSupported() {
util.verifyRelPlan(
- "SELECT u.user_id_0, u.name, o.order_id, p.payment_id "
- + "FROM Users u "
- + "RIGHT JOIN Orders o ON u.user_id_0 = o.user_id_1 "
- + "RIGHT JOIN Payments p ON o.user_id_1 = p.user_id_2");
+ "\nSELECT\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " o.order_id,\n"
+ + " p.payment_id\n"
+ + "FROM Users u\n"
+ + "RIGHT JOIN Orders o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "RIGHT JOIN Payments p\n"
+ + " ON o.user_id = p.user_id");
}
@Test
void testFullOuterNotSupported() {
util.verifyRelPlan(
- "SELECT u.user_id_0, u.name, o.order_id, p.payment_id "
- + "FROM Users u "
- + "FULL OUTER JOIN Orders o ON u.user_id_0 = o.user_id_1 "
- + "FULL OUTER JOIN Payments p ON o.user_id_1 = p.user_id_2");
+ "\nSELECT\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " o.order_id,\n"
+ + " p.payment_id\n"
+ + "FROM Users u\n"
+ + "FULL OUTER JOIN Orders o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "FULL OUTER JOIN Payments p\n"
+ + " ON o.user_id = p.user_id");
}
@Test
void testThreeWayJoinWithTimeAttributesMaterialization() {
util.verifyRelPlan(
- "SELECT u.name, u.proctime, o.rowtime, p.price "
- + "FROM UsersWithProctime u "
- + "JOIN OrdersWithRowtime o ON u.user_id_0 = o.user_id_1 "
- + "JOIN Payments p ON u.user_id_0 = p.user_id_2");
+ "\nSELECT\n"
+ + " u.name,\n"
+ + " u.proctime,\n"
+ + " o.`$rowtime`,\n"
+ + " p.price\n"
+ + "FROM UsersWithProctime u\n"
+ + "JOIN OrdersWithRowtime o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "JOIN Payments p\n"
+ + " ON u.user_id = p.user_id");
}
@Test
@@ -456,15 +630,15 @@ void testPreservesUpsertKeyTwoWayLeftJoinOrders() {
+ ")");
util.verifyRelPlanInsert(
- "INSERT INTO sink_two_way "
- + "SELECT"
- + " o.user_id,"
- + " o.order_id,"
- + " o.product,"
- + " u.region_id "
- + "FROM OrdersPK o "
- + "LEFT JOIN UsersPK u"
- + " ON u.user_id = o.user_id");
+ "\nINSERT INTO sink_two_way\n"
+ + "SELECT\n"
+ + " o.user_id,\n"
+ + " o.order_id,\n"
+ + " o.product,\n"
+ + " u.region_id\n"
+ + "FROM OrdersPK o\n"
+ + "LEFT JOIN UsersPK u\n"
+ + " ON u.user_id = o.user_id");
}
@Test
@@ -483,15 +657,15 @@ void testPreservesUpsertKeyTwoWayInnerJoinOrders() {
+ ")");
util.verifyRelPlanInsert(
- "INSERT INTO sink_two_way "
- + "SELECT"
- + " o.user_id,"
- + " o.order_id,"
- + " o.product,"
- + " u.region_id "
- + "FROM UsersPK u "
- + "INNER JOIN OrdersPK o "
- + " ON u.user_id = o.user_id");
+ "\nINSERT INTO sink_two_way\n"
+ + "SELECT\n"
+ + " o.user_id,\n"
+ + " o.order_id,\n"
+ + " o.product,\n"
+ + " u.region_id\n"
+ + "FROM UsersPK u\n"
+ + "INNER JOIN OrdersPK o\n"
+ + " ON u.user_id = o.user_id");
}
@Test
@@ -519,15 +693,15 @@ void testPreservesUpsertKeyTwoWayInnerJoinOrdersDoesNot() {
+ ")");
util.verifyRelPlanInsert(
- "INSERT INTO sink_two_way "
- + "SELECT"
- + " o.user_id,"
- + " o.order_id,"
- + " o.product,"
- + " u.region_id "
- + "FROM UsersPK u "
- + "INNER JOIN OrdersSimplePK o "
- + " ON u.user_id = o.user_id");
+ "\nINSERT INTO sink_two_way\n"
+ + "SELECT\n"
+ + " o.user_id,\n"
+ + " o.order_id,\n"
+ + " o.product,\n"
+ + " u.region_id\n"
+ + "FROM UsersPK u\n"
+ + "INNER JOIN OrdersSimplePK o\n"
+ + " ON u.user_id = o.user_id");
}
@Test
@@ -548,19 +722,19 @@ void testPreservesUpsertKeyThreeWayJoin() {
+ ")");
util.verifyRelPlanInsert(
- "INSERT INTO sink_three_way "
- + "SELECT"
- + " o.user_id,"
- + " o.order_id,"
- + " p.user_id,"
- + " p.payment_id,"
- + " u.user_id,"
- + " u.description "
- + "FROM UsersPK u "
- + "JOIN OrdersPK o"
- + " ON o.user_id = u.user_id "
- + "JOIN PaymentsPK p"
- + " ON o.user_id = p.user_id");
+ "\nINSERT INTO sink_three_way\n"
+ + "SELECT\n"
+ + " o.user_id,\n"
+ + " o.order_id,\n"
+ + " p.user_id,\n"
+ + " p.payment_id,\n"
+ + " u.user_id,\n"
+ + " u.description\n"
+ + "FROM UsersPK u\n"
+ + "JOIN OrdersPK o\n"
+ + " ON o.user_id = u.user_id\n"
+ + "JOIN PaymentsPK p\n"
+ + " ON o.user_id = p.user_id");
}
@Test
@@ -568,36 +742,39 @@ void testPreservesUpsertKeyFourWayComplex() {
util.tableEnv()
.executeSql(
"CREATE TABLE sink_four_way ("
- + " user_id_0 STRING NOT NULL,"
+ + " user_id STRING NOT NULL,"
+ " order_id STRING NOT NULL,"
- + " user_id_1 STRING NOT NULL,"
+ + " user_id1 STRING NOT NULL,"
+ " payment_id STRING NOT NULL,"
- + " user_id_2 STRING NOT NULL,"
+ + " user_id2 STRING NOT NULL,"
+ " name STRING,"
+ " location STRING,"
- + " CONSTRAINT `PRIMARY` PRIMARY KEY (`user_id_0`, `order_id`, `user_id_1`, `payment_id`, `user_id_2`) NOT ENFORCED"
+ + " CONSTRAINT `PRIMARY` PRIMARY KEY (`user_id`, `order_id`, `user_id1`, `payment_id`, `user_id2`) NOT ENFORCED"
+ ") WITH ("
+ " 'connector' = 'values',"
+ " 'sink-insert-only' = 'false'"
+ ")");
util.verifyRelPlanInsert(
- "INSERT INTO sink_four_way "
- + "SELECT"
- + " u.user_id,"
- + " o.order_id,"
- + " o.user_id,"
- + " p.payment_id,"
- + " p.user_id,"
- + " u.name,"
- + " a.location "
- + "FROM UsersPK u "
- + "JOIN OrdersPK o"
- + " ON u.user_id = o.user_id AND o.product IS NOT NULL "
- + "JOIN PaymentsPK p"
- + " ON u.user_id = p.user_id AND p.price >= 0 "
- + "JOIN AddressPK a"
- + " ON u.user_id = a.user_id AND a.location IS NOT NULL");
+ "\nINSERT INTO sink_four_way\n"
+ + "SELECT\n"
+ + " u.user_id,\n"
+ + " o.order_id,\n"
+ + " o.user_id,\n"
+ + " p.payment_id,\n"
+ + " p.user_id,\n"
+ + " u.name,\n"
+ + " a.location\n"
+ + "FROM UsersPK u\n"
+ + "JOIN OrdersPK o\n"
+ + " ON u.user_id = o.user_id\n"
+ + " AND o.product IS NOT NULL\n"
+ + "JOIN PaymentsPK p\n"
+ + " ON u.user_id = p.user_id\n"
+ + " AND p.price >= 0\n"
+ + "JOIN AddressPK a\n"
+ + " ON u.user_id = a.user_id\n"
+ + " AND a.location IS NOT NULL");
}
@Test
@@ -684,4 +861,768 @@ void testMultiSinkOnMultiJoinedView() {
false,
false);
}
+
+ /*
+ * Calcite adds a LogicalProject to compute expressions such as UPPER and FLOOR
+ * on the necessary fields. As a result, the planner cannot fuse all joins into
+ * a single MultiJoin node initially.
+ */
+ @Test
+ @Tag("multijoin-chain-expected")
+ void testFourWayJoinWithFunctionInConditionMultiJoinChainExpected() {
+ util.verifyRelPlan(
+ "\nSELECT\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " o.order_id,\n"
+ + " p.payment_id,\n"
+ + " s.location\n"
+ + "FROM Users u\n"
+ + "LEFT JOIN Orders o\n"
+ + " ON o.user_id = u.user_id\n"
+ + "LEFT JOIN Payments p\n"
+ + " ON u.user_id = p.user_id\n"
+ + " AND UPPER(u.name) = UPPER(p.payment_id)\n"
+ + " AND (FLOOR(u.cash) >= FLOOR(p.price) OR p.price < 0)\n"
+ + "LEFT JOIN Shipments s\n"
+ + " ON p.payment_id = s.location");
+ }
+
+ /*
+ * We expect the join inputs to **not** merge into a single MultiJoin node in this case,
+ * because `documents.common_id` is different from `other_documents.common_id`.
+ */
+ @Test
+ @Tag("no-common-join-key")
+ void testComplexCommonJoinKeyMissingProjectionNoCommonJoinKey() {
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE Assignments ("
+ + " assignment_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " user_id STRING,"
+ + " detail_id STRING,"
+ + " common_id STRING"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE Documents ("
+ + " detail_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " creator_nm STRING,"
+ + " common_id STRING"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.verifyRelPlan(
+ "\nSELECT *\n"
+ + "FROM Assignments assignments\n"
+ + "LEFT JOIN Documents AS documents\n"
+ + " ON assignments.detail_id = documents.detail_id\n"
+ + " AND assignments.common_id = documents.common_id\n"
+ + "LEFT JOIN Documents AS other_documents\n"
+ + " ON assignments.user_id = other_documents.common_id");
+ }
+
+ @Test
+ void testComplexCommonJoinKey() {
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE Assignments ("
+ + " assignment_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " user_id STRING,"
+ + " detail_id STRING,"
+ + " common_id STRING"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE Customers ("
+ + " user_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " name STRING,"
+ + " depart_num STRING,"
+ + " common_id STRING"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE Documents ("
+ + " detail_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " creator_nm STRING,"
+ + " common_id STRING"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE PhaseDetails ("
+ + " phase_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " common_id STRING"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE Organizations ("
+ + " org_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " org_name STRING,"
+ + " common_id STRING"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.verifyExecPlan(
+ "\nSELECT *\n"
+ + "FROM Assignments assignments\n"
+ + "LEFT JOIN Customers AS customer\n"
+ + " ON assignments.user_id = customer.user_id\n"
+ + " AND assignments.common_id = customer.common_id\n"
+ + "LEFT JOIN Documents AS documents\n"
+ + " ON assignments.detail_id = documents.detail_id\n"
+ + " AND assignments.common_id = documents.common_id\n"
+ + "LEFT JOIN PhaseDetails AS phase_details\n"
+ + " ON documents.common_id = phase_details.common_id\n"
+ + "LEFT JOIN Organizations AS organizations\n"
+ + " ON customer.depart_num = organizations.org_id\n"
+ + " AND customer.common_id = organizations.common_id\n"
+ + "LEFT JOIN Customers AS creators\n"
+ + " ON documents.creator_nm = creators.depart_num\n"
+ + " AND documents.common_id = creators.common_id");
+ }
+
+ @Test
+ @Tag("no-common-join-key")
+ void testComplexConditionalLogicWithMultiJoinNoCommonJoinKey() {
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE ProductCategories ("
+ + " category_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " category_name STRING,"
+ + " is_premium BOOLEAN,"
+ + " discount_rate DOUBLE"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE ProductReviews ("
+ + " review_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " product_id STRING,"
+ + " rating INT,"
+ + " is_verified BOOLEAN"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.verifyRelPlan(
+ "\nSELECT\n"
+ + " u.user_id,\n"
+ + " o.order_id,\n"
+ + " p.payment_id,\n"
+ + " pc.category_name,\n"
+ + " CASE\n"
+ + " WHEN pc.is_premium = true AND p.price > 1000 THEN 'High-Value Premium'\n"
+ + " WHEN pc.is_premium = true THEN 'Premium'\n"
+ + " WHEN p.price > 500 THEN 'Standard High-Value'\n"
+ + " ELSE 'Standard'\n"
+ + " END AS product_tier,\n"
+ + " CASE\n"
+ + " WHEN pr.rating >= 4 AND pr.is_verified = true THEN 'Highly Recommended'\n"
+ + " WHEN pr.rating >= 3 THEN 'Recommended'\n"
+ + " WHEN pr.rating >= 2 THEN 'Average'\n"
+ + " ELSE 'Not Recommended'\n"
+ + " END AS recommendation_status,\n"
+ + " CASE\n"
+ + " WHEN pc.discount_rate > 0.2 THEN p.price * (1 - pc.discount_rate)\n"
+ + " ELSE p.price\n"
+ + " END AS final_price\n"
+ + "FROM Users u\n"
+ + "LEFT JOIN Orders o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "LEFT JOIN Payments p\n"
+ + " ON u.user_id = p.user_id\n"
+ + "LEFT JOIN ProductCategories pc\n"
+ + " ON o.product = pc.category_id\n"
+ + "LEFT JOIN ProductReviews pr\n"
+ + " ON o.product = pr.product_id");
+ }
+
+ @Test
+ @Tag("no-common-join-key")
+ void testComplexCTEWithMultiJoinNoCommonJoinKey() {
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE OrderStatus ("
+ + " status_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " status_name STRING,"
+ + " is_final BOOLEAN"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE PaymentMethods ("
+ + " method_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " method_name STRING,"
+ + " processing_fee DOUBLE"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.verifyRelPlan(
+ "\nWITH user_orders AS (\n"
+ + " SELECT u.user_id, u.name, o.order_id, o.product, p.payment_id, p.price\n"
+ + " FROM Users u\n"
+ + " LEFT JOIN Orders o ON\n"
+ + " u.user_id = o.user_id\n"
+ + " LEFT JOIN Payments p\n"
+ + " ON u.user_id = p.user_id\n"
+ + "),\n"
+ + "order_details AS (\n"
+ + " SELECT uo.*, os.status_name, os.is_final, pm.method_name, pm.processing_fee\n"
+ + " FROM user_orders uo\n"
+ + " LEFT JOIN OrderStatus os\n"
+ + " ON uo.order_id = os.status_id\n"
+ + " LEFT JOIN PaymentMethods pm\n"
+ + " ON uo.payment_id = pm.method_id\n"
+ + "),\n"
+ + "final_summary AS (\n"
+ + " SELECT\n"
+ + " user_id,\n"
+ + " name,\n"
+ + " COUNT(order_id) as total_orders,\n"
+ + " SUM(price) as total_spent,\n"
+ + " AVG(price) as avg_order_value,\n"
+ + " COUNT(CASE WHEN is_final = true THEN 1 END) as completed_orders\n"
+ + " FROM order_details\n"
+ + " GROUP BY user_id, name\n"
+ + ")\n"
+ + "SELECT * FROM final_summary");
+ }
+
+ @Test
+ @Tag("no-common-join-key")
+ void testAggregationAndGroupingWithMultiJoinNoCommonJoinKey() {
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE OrderItems ("
+ + " item_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " order_id STRING,"
+ + " product_name STRING,"
+ + " quantity INT,"
+ + " unit_price DOUBLE"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE ProductCategories ("
+ + " category_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " category_name STRING,"
+ + " parent_category STRING"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.verifyRelPlan(
+ "\nSELECT\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " pc.category_name,\n"
+ + " COUNT(DISTINCT o.order_id) as order_count,\n"
+ + " SUM(oi.quantity) as total_items,\n"
+ + " SUM(oi.quantity * oi.unit_price) as total_value,\n"
+ + " AVG(oi.unit_price) as avg_item_price,\n"
+ + " MAX(p.price) as max_payment,\n"
+ + " MIN(p.price) as min_payment,\n"
+ + " COUNT(CASE WHEN oi.quantity > 5 THEN 1 END) as bulk_orders\n"
+ + "FROM Users u\n"
+ + "LEFT JOIN Orders o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "LEFT JOIN OrderItems oi\n"
+ + " ON o.order_id = oi.order_id\n"
+ + "LEFT JOIN ProductCategories pc\n"
+ + " ON oi.product_name = pc.category_id\n"
+ + "LEFT JOIN Payments p\n"
+ + " ON u.user_id = p.user_id\n"
+ + "GROUP BY u.user_id, u.name, pc.category_name\n"
+ + "HAVING COUNT(DISTINCT o.order_id) > 0");
+ }
+
+ @Test
+ @Tag("no-common-join-key")
+ void testFunctionAndExpressionWithMultiJoinNoCommonJoinKey() {
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE ProductDetails ("
+ + " product_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " product_name STRING,"
+ + " description STRING,"
+ + " created_date BIGINT,"
+ + " tags STRING"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE UserPreferences ("
+ + " user_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " preferred_category STRING,"
+ + " notification_level STRING"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.verifyRelPlan(
+ "\nSELECT\n"
+ + " u.user_id,\n"
+ + " UPPER(u.name) as user_name_upper,\n"
+ + " LOWER(o.product) as product_lower,\n"
+ + " CONCAT(u.name, ' - ', o.product) as user_product,\n"
+ + " SUBSTRING(pd.description, 1, 50) as description_preview,\n"
+ + " CHAR_LENGTH(pd.description) as description_length,\n"
+ + " FLOOR(p.price / 100.0) * 100 as price_rounded,\n"
+ + " CASE\n"
+ + " WHEN p.price > 1000 THEN 'High'\n"
+ + " WHEN p.price > 500 THEN 'Medium'\n"
+ + " ELSE 'Low'\n"
+ + " END as price_tier,\n"
+ + " REGEXP_REPLACE(pd.tags, ',', ' | ') as formatted_tags,\n"
+ + " TO_TIMESTAMP_LTZ(pd.created_date, 3) as product_created,\n"
+ + " COALESCE(up.preferred_category, 'None') as user_preference,\n"
+ + " CASE\n"
+ + " WHEN up.notification_level = 'HIGH' THEN 'Frequent Updates'\n"
+ + " WHEN up.notification_level = 'MEDIUM' THEN 'Daily Updates'\n"
+ + " ELSE 'Weekly Updates'\n"
+ + " END as notification_frequency\n"
+ + "FROM Users u\n"
+ + "LEFT JOIN Orders o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "LEFT JOIN Payments p\n"
+ + " ON u.user_id = p.user_id\n"
+ + "LEFT JOIN ProductDetails pd\n"
+ + " ON o.product = pd.product_id\n"
+ + "LEFT JOIN UserPreferences up\n"
+ + " ON u.user_id = up.user_id");
+ }
+
+ /*
+ * Calcite automatically generates LogicalProject nodes for nested field access.
+ * As a result, each join input in this test is wrapped in a projection, which prevents
+ * the planner from fusing all joins into a single MultiJoin node initially.
+ * Therefore, in this test, each Join is still converted to a MultiJoin individually.
+ */
+ @Test
+ @Tag("multijoin-chain-expected")
+ void testJoinConditionHasNestedFieldsMultiJoinChainExpected() {
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE Developers ("
+ + " developer_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " person ROW>,"
+ + " experience_years INT"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE SupportTickets ("
+ + " ticket_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " reporter ROW>,"
+ + " issue STRING"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE Feedback ("
+ + " feedback_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " author ROW>,"
+ + " message STRING"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE Subscriptions ("
+ + " sub_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " subscriber ROW>,"
+ + " active BOOLEAN"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.verifyRelPlan(
+ "\nSELECT\n"
+ + " d.developer_id,\n"
+ + " d.person.info.name AS developer_name,\n"
+ + " s.ticket_id,\n"
+ + " s.reporter.info.priority AS ticket_priority,\n"
+ + " f.feedback_id,\n"
+ + " f.author.info.rating AS feedback_rating,\n"
+ + " sub.sub_id,\n"
+ + " sub.subscriber.info.plan AS subscription_plan\n"
+ + "FROM Developers AS d\n"
+ + "LEFT JOIN SupportTickets AS s\n"
+ + " ON d.person.info.id = s.reporter.info.id\n"
+ + "LEFT JOIN Feedback AS f\n"
+ + " ON d.person.info.id = f.author.info.id\n"
+ + "LEFT JOIN Subscriptions AS sub\n"
+ + " ON d.person.info.id = sub.subscriber.info.id");
+ }
+
+ @Test
+ @Tag("multijoin-chain-expected")
+ void testComplexNestedCTEWithAggregationAndFunctionsMultiJoinChainExpected() {
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE OrderMetrics ("
+ + " metric_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " order_id STRING,"
+ + " metric_type STRING,"
+ + " metric_value DOUBLE"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.verifyRelPlan(
+ "\nWITH base_orders AS (\n"
+ + " SELECT u.user_id, u.name, o.order_id, p.payment_id, p.price\n"
+ + " FROM Users u\n"
+ + " INNER JOIN Orders o\n"
+ + " ON u.user_id = o.user_id\n"
+ + " INNER JOIN Payments p\n"
+ + " ON u.user_id = p.user_id\n"
+ + "),\n"
+ + "enriched_orders AS (\n"
+ + " SELECT\n"
+ + " bo.*,\n"
+ + " om.metric_type,\n"
+ + " om.metric_value,\n"
+ + " CASE\n"
+ + " WHEN bo.price > 1000 THEN 'Premium'\n"
+ + " WHEN bo.price > 500 THEN 'Standard'\n"
+ + " ELSE 'Basic'\n"
+ + " END as order_tier\n"
+ + " FROM base_orders bo\n"
+ + " LEFT JOIN OrderMetrics om\n"
+ + " ON bo.order_id = om.order_id\n"
+ + "),\n"
+ + "aggregated_metrics AS (\n"
+ + " SELECT\n"
+ + " user_id,\n"
+ + " name,\n"
+ + " COUNT(DISTINCT order_id) as total_orders,\n"
+ + " SUM(price) as total_spent,\n"
+ + " AVG(price) as avg_order_value,\n"
+ + " MAX(metric_value) as max_metric,\n"
+ + " MIN(metric_value) as min_metric,\n"
+ + " COUNT(CASE WHEN order_tier = 'Premium' THEN 1 END) as premium_orders\n"
+ + " FROM enriched_orders\n"
+ + " GROUP BY user_id, name\n"
+ + ")\n"
+ + "SELECT\n"
+ + " user_id,\n"
+ + " UPPER(name) as user_name,\n"
+ + " total_orders,\n"
+ + " ROUND(total_spent, 2) as total_spent_rounded,\n"
+ + " ROUND(avg_order_value, 2) as avg_order_value_rounded,\n"
+ + " CONCAT('User: ', name, ' has ', CAST(total_orders AS STRING), ' orders') as summary,\n"
+ + " CASE\n"
+ + " WHEN total_orders > 10 THEN 'Frequent Customer'\n"
+ + " WHEN total_orders > 5 THEN 'Regular Customer'\n"
+ + " ELSE 'Occasional Customer'\n"
+ + " END as customer_type\n"
+ + "FROM aggregated_metrics\n"
+ + "WHERE total_spent > 0");
+ }
+
+ @Test
+ void testJoinOfProjections() {
+ util.verifyRelPlan(
+ "\nSELECT u.user_id, o.order_id, o.product, p.price, s.location\n"
+ + "FROM (SELECT user_id, name, cash FROM Users WHERE cash > 100) AS u\n"
+ + "JOIN (SELECT user_id, order_id, product FROM Orders WHERE product IS NOT NULL) AS o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "LEFT JOIN (SELECT user_id, price FROM Payments WHERE price > 50) AS p\n"
+ + " ON u.user_id = p.user_id\n"
+ + "LEFT JOIN (SELECT user_id, location FROM Shipments WHERE location IS NOT NULL) AS s\n"
+ + " ON u.user_id = s.user_id");
+ }
+
+ @Test
+ @Tag("multijoin-chain-expected")
+ void testJoinWithNestedSubqueryMultiJoinChainExpected() {
+ util.verifyRelPlan(
+ "\nSELECT *\n"
+ + "FROM Users u\n"
+ + "JOIN (\n"
+ + " SELECT o.user_id, o.order_id, p.payment_id, p.price\n"
+ + " FROM Orders o\n"
+ + " JOIN (\n"
+ + " SELECT payment_id, user_id, price\n"
+ + " FROM Payments\n"
+ + " WHERE price > 100\n"
+ + " ) AS p\n"
+ + " ON o.user_id = p.user_id\n"
+ + ") AS op\n"
+ + "ON u.user_id = op.user_id");
+ }
+
+ @Test
+ void testCTEWithMultiJoinV2() {
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE Departments ("
+ + " dept_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " dept_name STRING,"
+ + " budget DOUBLE"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE Projects ("
+ + " project_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " project_name STRING,"
+ + " dept_id STRING,"
+ + " status STRING"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.verifyRelPlan(
+ "\nWITH high_budget_depts AS (\n"
+ + " SELECT dept_id, dept_name, budget\n"
+ + " FROM Departments\n"
+ + " WHERE budget > 600000\n"
+ + "),\n"
+ + "active_projects AS (\n"
+ + " SELECT project_id, project_name, dept_id\n"
+ + " FROM Projects\n"
+ + " WHERE status = 'ACTIVE'\n"
+ + ")\n"
+ + "SELECT\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " o.order_id,\n"
+ + " hbd.dept_name,\n"
+ + " ap.project_name,\n"
+ + " hbd.budget\n"
+ + "FROM Users u\n"
+ + "LEFT JOIN Orders o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "LEFT JOIN high_budget_depts hbd\n"
+ + " ON o.user_id = hbd.dept_id\n"
+ + "LEFT JOIN active_projects ap\n"
+ + " ON hbd.dept_id = ap.dept_id");
+ }
+
+ @Test
+ void testWithOrInJoinCondition() {
+ util.verifyRelPlan(
+ "\nSELECT\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " o.order_id,\n"
+ + " p.payment_id,\n"
+ + " s.location\n"
+ + "FROM Users u\n"
+ + "LEFT JOIN Orders o\n"
+ + " ON o.user_id = u.user_id\n"
+ + "LEFT JOIN Payments p\n"
+ + " ON u.user_id = p.user_id OR u.name = p.payment_id\n"
+ + "LEFT JOIN Shipments s\n"
+ + " ON p.user_id = s.user_id");
+ }
+
+ @Test
+ @Tag("multijoin-chain-expected")
+ void testWithCastCommonJoinKeyToIntegerMultiJoinChainExpected() {
+ util.verifyRelPlan(
+ "\nSELECT\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " o.order_id,\n"
+ + " p.payment_id,\n"
+ + " s.location\n"
+ + "FROM Users u\n"
+ + "LEFT JOIN Orders o\n"
+ + " ON o.user_id = u.user_id\n"
+ + "LEFT JOIN Payments p\n"
+ + " ON CAST(u.user_id as INTEGER) = CAST(p.user_id as INTEGER)\n"
+ + "LEFT JOIN Shipments s\n"
+ + " ON u.user_id = s.user_id");
+ }
+
+ @Test
+ void testWithCastCommonJoinKeyToVarchar() {
+ util.verifyRelPlan(
+ "\nSELECT\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " o.order_id,\n"
+ + " p.payment_id,\n"
+ + " s.location\n"
+ + "FROM Users u\n"
+ + "LEFT JOIN Orders o\n"
+ + " ON o.user_id = u.user_id\n"
+ + "LEFT JOIN Payments p\n"
+ + " ON CAST(u.user_id as VARCHAR) = CAST(p.user_id as VARCHAR)\n"
+ + "LEFT JOIN Shipments s\n"
+ + " ON u.user_id = s.user_id");
+ }
+
+ @Test
+ void testAggregationAndGroupingWithMultiJoinV2() {
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE Categories ("
+ + " category_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " category_name STRING,"
+ + " parent_category STRING,"
+ + " user_id STRING"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE Sales ("
+ + " sale_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " user_id STRING,"
+ + " product_id STRING,"
+ + " amount DOUBLE,"
+ + " sale_date DATE"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.verifyRelPlan(
+ "\nSELECT\n"
+ + " c.category_name,\n"
+ + " COUNT(DISTINCT u.user_id) AS unique_users,\n"
+ + " COUNT(s.sale_id) AS total_sales,\n"
+ + " SUM(s.amount) AS total_revenue,\n"
+ + " AVG(s.amount) AS avg_sale_amount,\n"
+ + " MAX(s.amount) AS max_sale_amount\n"
+ + "FROM Users u\n"
+ + "LEFT JOIN Orders o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "LEFT JOIN Categories c\n"
+ + " ON u.user_id = c.user_id AND o.product = c.category_id\n"
+ + "LEFT JOIN Sales s\n"
+ + " ON u.user_id = s.user_id\n"
+ + "GROUP BY c.category_name\n"
+ + "HAVING COUNT(s.sale_id) > 0");
+ }
+
+ @Test
+ void testSameTableMultipleAliases() {
+ util.verifyRelPlan(
+ "\nSELECT *\n"
+ + "FROM Users u\n"
+ + "LEFT JOIN Users u1\n"
+ + " ON u.user_id = u1.user_id\n"
+ + "LEFT JOIN Users u2\n"
+ + " ON u1.user_id = u2.user_id\n"
+ + "LEFT JOIN Users u3\n"
+ + " ON u2.user_id = u3.user_id");
+ }
+
+ @Test
+ @Tag("multijoin-chain-expected")
+ void testWithExpressionInJoinConditionMultiJoinChainExpected() {
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE Products ("
+ + " product_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " price DOUBLE,"
+ + " discount DOUBLE"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE Sales ("
+ + " sale_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " product_key DOUBLE,"
+ + " quantity INT"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE Promotions ("
+ + " promo_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " product_key DOUBLE,"
+ + " promo_text STRING"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.verifyRelPlan(
+ "\nSELECT\n"
+ + " p.product_id,\n"
+ + " (p.price - p.discount) AS net_price,\n"
+ + " s.quantity,\n"
+ + " pr.promo_text\n"
+ + "FROM Products AS p\n"
+ + "LEFT JOIN Sales AS s\n"
+ + " ON (p.price - p.discount) = s.product_key\n"
+ + "LEFT JOIN Promotions AS pr\n"
+ + " ON (p.price - p.discount) = pr.product_key\n"
+ + "WHERE (p.price - p.discount) > 100");
+ }
+
+ @Test
+ @Tag("no-common-join-key")
+ void testFunctionAndExpressionWithMultiJoinNoCommonJoinKeyV2() {
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE ProductDetails ("
+ + " product_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " product_name STRING,"
+ + " price DOUBLE,"
+ + " weight DOUBLE,"
+ + " created_date DATE"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE Reviews ("
+ + " review_id STRING PRIMARY KEY NOT ENFORCED,"
+ + " product_id STRING,"
+ + " rating INT,"
+ + " review_text STRING,"
+ + " review_date DATE"
+ + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')");
+
+ util.verifyRelPlan(
+ "\nSELECT\n"
+ + " u.user_id,\n"
+ + " u.name,\n"
+ + " o.order_id,\n"
+ + " pd.product_name,\n"
+ + " pd.price,\n"
+ + " ROUND(pd.price * 1.1, 2) AS price_with_tax,\n"
+ + " CONCAT('Product: ', pd.product_name) AS product_description,\n"
+ + " CHAR_LENGTH(r.review_text) AS review_length,\n"
+ + " UPPER(SUBSTRING(r.review_text, 1, 10)) AS review_preview,\n"
+ + " CASE\n"
+ + " WHEN r.rating >= 4 THEN 'High Rating'\n"
+ + " WHEN r.rating >= 3 THEN 'Medium Rating'\n"
+ + " ELSE 'Low Rating'\n"
+ + " END AS rating_category,\n"
+ + " TIMESTAMPDIFF(DAY, pd.created_date, CURRENT_DATE) AS days_since_created\n"
+ + "FROM Users u\n"
+ + "LEFT JOIN Orders o\n"
+ + " ON u.user_id = o.user_id\n"
+ + "LEFT JOIN ProductDetails pd\n"
+ + " ON o.product = pd.product_id\n"
+ + "LEFT JOIN Reviews r\n"
+ + " ON pd.product_id = r.product_id");
+ }
+
+ @Test
+ void testCrossJoinUnnestWithMultiJoinInsert() {
+ util.tableEnv()
+ .executeSql(
+ "CREATE TABLE UnnestSink ("
+ + " detail_id STRING,"
+ + " element_data STRING,"
+ + " data_value_id INT,"
+ + " user_id STRING,"
+ + " order_id STRING"
+ + ") WITH ('connector' = 'values', 'sink-insert-only' = 'false')");
+
+ util.verifyRelPlanInsert(
+ "\nINSERT INTO UnnestSink\n"
+ + "(\n"
+ + " detail_id,\n"
+ + " element_data,\n"
+ + " data_value_id,\n"
+ + " user_id,\n"
+ + " order_id\n"
+ + ")\n"
+ + "SELECT\n"
+ + " d.detail_id,\n"
+ + " TRIM(REGEXP_REPLACE(edata, '[\\[\\]\\\"]', '')) AS element_data,\n"
+ + " ARRAY_POSITION(split(REGEXP_REPLACE(d.data, '^\\[\"|\"\\]$', '') , '\", \"'), edata) as data_value_id,\n"
+ + " d.user_id,\n"
+ + " o.order_id\n"
+ + "FROM Detail d\n"
+ + "INNER JOIN Orders o\n"
+ + " ON o.user_id = d.user_id\n"
+ + "INNER JOIN Payments p\n"
+ + " ON p.user_id = d.user_id\n"
+ + "LEFT JOIN Shipments s\n"
+ + " ON s.user_id = d.user_id\n"
+ + "CROSS JOIN UNNEST(split(REGEXP_REPLACE(d.data, '^\\[\"|\"\\]$', '') , '\", \"')) AS T(edata)\n"
+ + "WHERE NOT (s.location IS NOT NULL)");
+ }
}
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/DeltaJoinUtilTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/DeltaJoinUtilTest.java
new file mode 100644
index 0000000000000..9a5d8fde20227
--- /dev/null
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/DeltaJoinUtilTest.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.plan.utils;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.calcite.FlinkTypeSystem;
+import org.apache.flink.table.types.utils.TypeConversions;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.planner.plan.utils.DeltaJoinUtil.isFilterOnOneSetOfUpsertKeys;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for {@link DeltaJoinUtil}. */
+class DeltaJoinUtilTest {
+
+ @Test
+ void testIsFilterOnOneSetOfUpsertKeys() {
+ FlinkTypeFactory typeFactory =
+ new FlinkTypeFactory(
+ Thread.currentThread().getContextClassLoader(), FlinkTypeSystem.INSTANCE);
+ // input schema:
+ // a string,
+ // b bigint,
+ // c bigint
+ List allFieldTypes =
+ Stream.of(DataTypes.VARCHAR(100), DataTypes.BIGINT(), DataTypes.BIGINT())
+ .map(TypeConversions::fromDataToLogicalType)
+ .map(typeFactory::createFieldTypeFromLogicalType)
+ .collect(Collectors.toList());
+
+ RexBuilder rexBuilder = new RexBuilder(typeFactory);
+
+ // a = 'jim'
+ RexNode filter =
+ rexBuilder.makeCall(
+ SqlStdOperatorTable.EQUALS,
+ rexBuilder.makeInputRef(allFieldTypes.get(0), 0),
+ rexBuilder.makeLiteral("jim", allFieldTypes.get(0)));
+
+ assertThat(isFilterOnOneSetOfUpsertKeys(filter, Set.of(ImmutableBitSet.of(0)))).isTrue();
+ assertThat(isFilterOnOneSetOfUpsertKeys(filter, Set.of(ImmutableBitSet.of(2)))).isFalse();
+ assertThat(isFilterOnOneSetOfUpsertKeys(filter, Set.of(ImmutableBitSet.of(0, 1)))).isTrue();
+ assertThat(isFilterOnOneSetOfUpsertKeys(filter, Set.of(ImmutableBitSet.of(1, 2))))
+ .isFalse();
+ assertThat(
+ isFilterOnOneSetOfUpsertKeys(
+ filter, Set.of(ImmutableBitSet.of(1), ImmutableBitSet.of(2))))
+ .isFalse();
+ assertThat(
+ isFilterOnOneSetOfUpsertKeys(
+ filter, Set.of(ImmutableBitSet.of(1), ImmutableBitSet.of(0))))
+ .isTrue();
+ assertThat(isFilterOnOneSetOfUpsertKeys(filter, null)).isFalse();
+ }
+}
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/FunctionITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/FunctionITCase.java
index e07ec1372dac8..a85cf50c50ce0 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/FunctionITCase.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/FunctionITCase.java
@@ -18,7 +18,9 @@
package org.apache.flink.table.planner.runtime.batch.sql;
+import org.apache.flink.table.annotation.DataTypeHint;
import org.apache.flink.table.api.Table;
+import org.apache.flink.table.functions.ScalarFunction;
import org.apache.flink.table.planner.factories.utils.TestCollectionTableFactory;
import org.apache.flink.table.planner.runtime.utils.BatchTestBase;
import org.apache.flink.types.Row;
@@ -99,6 +101,49 @@ void testUserDefinedTemporarySystemFunctionByUsingJar() throws Exception {
testUserDefinedFunctionByUsingJar(functionDDL, dropFunctionDDL);
}
+ @Test
+ void testOrderByScopeRawTypeCast() throws Exception {
+ final List sourceData = List.of(Row.of(1), Row.of(2), Row.of(3), Row.of(4), Row.of(5));
+ TestCollectionTableFactory.reset();
+ TestCollectionTableFactory.initData(sourceData);
+
+ tEnv().executeSql("CREATE TABLE Source(i INT) WITH ('connector' = 'COLLECTION')");
+ tEnv().executeSql("CREATE TABLE Sink(i INT) WITH ('connector' = 'COLLECTION')");
+
+ tEnv().createTemporarySystemFunction("CustomIntUdf", new CustomIntUdf());
+
+ tEnv().executeSql(
+ "INSERT INTO Sink"
+ + " SELECT i FROM Source"
+ + " ORDER BY CustomIntUdf(NULL)")
+ .await();
+
+ assertThat(TestCollectionTableFactory.getResult()).hasSize(5);
+ }
+
+ @Test
+ void testHavingScopeRawTypeCast() throws Exception {
+ final List sourceData = List.of(Row.of(1), Row.of(2), Row.of(3), Row.of(4), Row.of(5));
+ TestCollectionTableFactory.reset();
+ TestCollectionTableFactory.initData(sourceData);
+
+ tEnv().executeSql("CREATE TABLE Source(i INT) WITH ('connector' = 'COLLECTION')");
+ tEnv().executeSql("CREATE TABLE Sink(i INT) WITH ('connector' = 'COLLECTION')");
+
+ tEnv().createTemporarySystemFunction("CustomIntUdf", new CustomIntUdf());
+
+ tEnv().executeSql(
+ "INSERT INTO Sink"
+ + " SELECT SUM(i) AS s FROM Source"
+ + " HAVING CustomIntUdf(NULL) = 0")
+ .await();
+
+ assertThat(TestCollectionTableFactory.getResult())
+ .singleElement()
+ .asString()
+ .contains("15");
+ }
+
private void testUserDefinedFunctionByUsingJar(String createFunctionDDL, String dropFunctionDDL)
throws Exception {
List sourceData =
@@ -123,7 +168,7 @@ private void testUserDefinedFunctionByUsingJar(String createFunctionDDL, String
Table t2 = tEnv().sqlQuery(query);
t2.executeInsert("t2").await();
- List result = TestCollectionTableFactory.RESULT();
+ List result = TestCollectionTableFactory.getResult();
List expected =
Arrays.asList(
Row.of(1, "jark"),
@@ -139,4 +184,21 @@ private void testUserDefinedFunctionByUsingJar(String createFunctionDDL, String
// delete the function
tEnv().executeSql(dropFunctionDDL);
}
+
+ // ----- Test types / UDF -----
+
+ @DataTypeHint(value = "RAW", bridgedTo = CustomInt.class)
+ public static class CustomInt {
+ public Integer value;
+
+ public CustomInt(Integer v) {
+ this.value = v;
+ }
+ }
+
+ public static class CustomIntUdf extends ScalarFunction {
+ public Integer eval(CustomInt v) {
+ return 0;
+ }
+ }
}
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/MatchRecognizeITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/MatchRecognizeITCase.java
index 5aae60f527904..db4d7a3705e6d 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/MatchRecognizeITCase.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/MatchRecognizeITCase.java
@@ -41,8 +41,6 @@
import java.time.Instant;
import java.time.LocalDateTime;
import java.time.ZoneOffset;
-import java.util.Comparator;
-import java.util.List;
import static org.apache.flink.api.common.typeinfo.Types.DOUBLE;
import static org.apache.flink.api.common.typeinfo.Types.INSTANT;
@@ -90,24 +88,23 @@ void testSimplePatternInProcTime() {
.column("name", DataTypes.STRING())
.columnByExpression("proctime", "PROCTIME()")
.build()));
- TableResult tableResult =
- tEnv.executeSql(
- "SELECT T.aid, T.bid, T.cid\n"
- + "FROM MyTable\n"
- + "MATCH_RECOGNIZE (\n"
- + " ORDER BY proctime\n"
- + " MEASURES\n"
- + " `A\"`.id AS aid,\n"
- + " \u006C.id AS bid,\n"
- + " C.id AS cid\n"
- + " PATTERN (`A\"` \u006C C)\n"
- + " DEFINE\n"
- + " `A\"` AS name = 'a',\n"
- + " \u006C AS name = 'b',\n"
- + " C AS name = 'c'\n"
- + ") AS T");
- assertThat(CollectionUtil.iteratorToList(tableResult.collect()))
- .containsExactly(Row.of(6, 7, 8));
+ final String sql =
+ "SELECT T.aid, T.bid, T.cid\n"
+ + "FROM MyTable\n"
+ + "MATCH_RECOGNIZE (\n"
+ + " ORDER BY proctime\n"
+ + " MEASURES\n"
+ + " `A\"`.id AS aid,\n"
+ + " \u006C.id AS bid,\n"
+ + " C.id AS cid\n"
+ + " PATTERN (`A\"` \u006C C)\n"
+ + " DEFINE\n"
+ + " `A\"` AS name = 'a',\n"
+ + " \u006C AS name = 'b',\n"
+ + " C AS name = 'c'\n"
+ + ") AS T";
+
+ assertTableResult(sql, Row.of(6, 7, 8));
}
@Test
@@ -137,24 +134,23 @@ void testSimplePatternInEventTime() {
.column("name", DataTypes.STRING())
.column("ts", DataTypes.TIMESTAMP_LTZ(3))
.build()));
- TableResult tableResult =
- tEnv.executeSql(
- "SELECT T.aid, T.bid, T.cid\n"
- + "FROM MyTable\n"
- + "MATCH_RECOGNIZE (\n"
- + " ORDER BY ts\n"
- + " MEASURES\n"
- + " `A\"`.id AS aid,\n"
- + " \u006C.id AS bid,\n"
- + " C.id AS cid\n"
- + " PATTERN (`A\"` \u006C C)\n"
- + " DEFINE\n"
- + " `A\"` AS name = 'a',\n"
- + " \u006C AS name = 'b',\n"
- + " C AS name = 'c'\n"
- + ") AS T");
- assertThat(CollectionUtil.iteratorToList(tableResult.collect()))
- .containsExactly(Row.of(6, 7, 8));
+ final String sql =
+ "SELECT T.aid, T.bid, T.cid\n"
+ + "FROM MyTable\n"
+ + "MATCH_RECOGNIZE (\n"
+ + " ORDER BY ts\n"
+ + " MEASURES\n"
+ + " `A\"`.id AS aid,\n"
+ + " \u006C.id AS bid,\n"
+ + " C.id AS cid\n"
+ + " PATTERN (`A\"` \u006C C)\n"
+ + " DEFINE\n"
+ + " `A\"` AS name = 'a',\n"
+ + " \u006C AS name = 'b',\n"
+ + " C AS name = 'c'\n"
+ + ") AS T";
+
+ assertTableResult(sql, Row.of(6, 7, 8));
}
@Test
@@ -186,24 +182,23 @@ void testTimeConstraint() {
.column("name", DataTypes.STRING())
.column("ts", DataTypes.TIMESTAMP(3))
.build()));
- TableResult tableResult =
- tEnv.executeSql(
- "SELECT T.aid, T.bid, T.cid\n"
- + "FROM MyTable\n"
- + "MATCH_RECOGNIZE (\n"
- + " ORDER BY ts\n"
- + " MEASURES\n"
- + " A.id AS aid,\n"
- + " B.id AS bid,\n"
- + " C.id AS cid\n"
- + " PATTERN (A B C) WITHIN INTERVAL '1' MINUTE\n"
- + " DEFINE\n"
- + " A AS name = 'a',\n"
- + " B AS name = 'b',\n"
- + " C AS name = 'c'\n"
- + ") AS T");
- assertThat(CollectionUtil.iteratorToList(tableResult.collect()))
- .containsExactly(Row.of(2, 3, 4));
+
+ final String sql =
+ "SELECT T.aid, T.bid, T.cid\n"
+ + "FROM MyTable\n"
+ + "MATCH_RECOGNIZE (\n"
+ + " ORDER BY ts\n"
+ + " MEASURES\n"
+ + " A.id AS aid,\n"
+ + " B.id AS bid,\n"
+ + " C.id AS cid\n"
+ + " PATTERN (A B C) WITHIN INTERVAL '1' MINUTE\n"
+ + " DEFINE\n"
+ + " A AS name = 'a',\n"
+ + " B AS name = 'b',\n"
+ + " C AS name = 'c'\n"
+ + ") AS T";
+ assertTableResult(sql, Row.of(2, 3, 4));
}
@Test
@@ -235,25 +230,25 @@ void testSimplePatternWithNulls() {
.column("nullField", DataTypes.STRING())
.column("ts", DataTypes.TIMESTAMP_LTZ(3))
.build()));
- TableResult tableResult =
- tEnv.executeSql(
- "SELECT T.aid, T.bNull, T.cid, T.aNull\n"
- + "FROM MyTable\n"
- + "MATCH_RECOGNIZE (\n"
- + " ORDER BY ts\n"
- + " MEASURES\n"
- + " A.id AS aid,\n"
- + " A.nullField AS aNull,\n"
- + " LAST(B.nullField) AS bNull,\n"
- + " C.id AS cid\n"
- + " PATTERN (A B C)\n"
- + " DEFINE\n"
- + " A AS name = 'a' AND nullField IS NULL,\n"
- + " B AS name = 'b' AND LAST(A.nullField) IS NULL,\n"
- + " C AS name = 'c'\n"
- + ") AS T");
- assertThat(CollectionUtil.iteratorToList(tableResult.collect()))
- .containsExactly(Row.of(1, null, 3, null), Row.of(6, null, 8, null));
+
+ final String sql =
+ "SELECT T.aid, T.bNull, T.cid, T.aNull\n"
+ + "FROM MyTable\n"
+ + "MATCH_RECOGNIZE (\n"
+ + " ORDER BY ts\n"
+ + " MEASURES\n"
+ + " A.id AS aid,\n"
+ + " A.nullField AS aNull,\n"
+ + " LAST(B.nullField) AS bNull,\n"
+ + " C.id AS cid\n"
+ + " PATTERN (A B C)\n"
+ + " DEFINE\n"
+ + " A AS name = 'a' AND nullField IS NULL,\n"
+ + " B AS name = 'b' AND LAST(A.nullField) IS NULL,\n"
+ + " C AS name = 'c'\n"
+ + ") AS T";
+
+ assertTableResult(sql, Row.of(1, null, 3, null), Row.of(6, null, 8, null));
}
@Test
@@ -288,31 +283,29 @@ void testCodeSplitsAreProperlyGenerated() {
.column("key2", DataTypes.STRING())
.column("ts", DataTypes.TIMESTAMP_LTZ(3))
.build()));
- TableResult tableResult =
- tEnv.executeSql(
- "SELECT *\n"
- + "FROM MyTable\n"
- + "MATCH_RECOGNIZE (\n"
- + " PARTITION BY key1, key2\n"
- + " ORDER BY ts\n"
- + " MEASURES\n"
- + " A.id AS aid,\n"
- + " A.key1 AS akey1,\n"
- + " LAST(B.id) AS bid,\n"
- + " C.id AS cid,\n"
- + " C.key2 AS ckey2\n"
- + " PATTERN (A B C)\n"
- + " DEFINE\n"
- + " A AS name = 'a' AND key1 LIKE '%key%' AND id > 0,\n"
- + " B AS name = 'b' AND LAST(A.name, 2) IS NULL,\n"
- + " C AS name = 'c' AND LAST(A.name) = 'a'\n"
- + ") AS T");
- List actual = CollectionUtil.iteratorToList(tableResult.collect());
- actual.sort(Comparator.comparing(o -> String.valueOf(o.getField(0))));
- assertThat(actual)
- .containsExactly(
- Row.of("key1", "second_key3", 1, "key1", 2, 3, "second_key3"),
- Row.of("key2", "second_key4", 6, "key2", 7, 8, "second_key4"));
+ final String sql =
+ "SELECT *\n"
+ + "FROM MyTable\n"
+ + "MATCH_RECOGNIZE (\n"
+ + " PARTITION BY key1, key2\n"
+ + " ORDER BY ts\n"
+ + " MEASURES\n"
+ + " A.id AS aid,\n"
+ + " A.key1 AS akey1,\n"
+ + " LAST(B.id) AS bid,\n"
+ + " C.id AS cid,\n"
+ + " C.key2 AS ckey2\n"
+ + " PATTERN (A B C)\n"
+ + " DEFINE\n"
+ + " A AS name = 'a' AND key1 LIKE '%key%' AND id > 0,\n"
+ + " B AS name = 'b' AND LAST(A.name, 2) IS NULL,\n"
+ + " C AS name = 'c' AND LAST(A.name) = 'a'\n"
+ + ") AS T";
+
+ assertTableResult(
+ sql,
+ Row.of("key1", "second_key3", 1, "key1", 2, 3, "second_key3"),
+ Row.of("key2", "second_key4", 6, "key2", 7, 8, "second_key4"));
}
@Test
@@ -413,31 +406,30 @@ void testMatchRecognizeAppliedToWindowedGrouping() {
.column("tax", DataTypes.INT())
.build()));
- TableResult tableResult =
- tEnv.executeSql(
- "SELECT *\n"
- + "FROM (\n"
- + " SELECT\n"
- + " symbol,\n"
- + " SUM(price) as price,\n"
- + " TUMBLE_ROWTIME(ts, interval '3' second) as rowTime,\n"
- + " TUMBLE_START(ts, interval '3' second) as startTime\n"
- + " FROM Ticker\n"
- + " GROUP BY symbol, TUMBLE(ts, interval '3' second)\n"
- + ")\n"
- + "MATCH_RECOGNIZE (\n"
- + " PARTITION BY symbol\n"
- + " ORDER BY rowTime\n"
- + " MEASURES\n"
- + " B.price as dPrice,\n"
- + " B.startTime as dTime\n"
- + " ONE ROW PER MATCH\n"
- + " PATTERN (A B)\n"
- + " DEFINE\n"
- + " B AS B.price < A.price\n"
- + ")");
- assertThat(CollectionUtil.iteratorToList(tableResult.collect()))
- .containsExactly(Row.of("ACME", 2, now.plusSeconds(3)));
+ final String sql =
+ "SELECT *\n"
+ + "FROM (\n"
+ + " SELECT\n"
+ + " symbol,\n"
+ + " SUM(price) as price,\n"
+ + " TUMBLE_ROWTIME(ts, interval '3' second) as rowTime,\n"
+ + " TUMBLE_START(ts, interval '3' second) as startTime\n"
+ + " FROM Ticker\n"
+ + " GROUP BY symbol, TUMBLE(ts, interval '3' second)\n"
+ + ")\n"
+ + "MATCH_RECOGNIZE (\n"
+ + " PARTITION BY symbol\n"
+ + " ORDER BY rowTime\n"
+ + " MEASURES\n"
+ + " B.price as dPrice,\n"
+ + " B.startTime as dTime\n"
+ + " ONE ROW PER MATCH\n"
+ + " PATTERN (A B)\n"
+ + " DEFINE\n"
+ + " B AS B.price < A.price\n"
+ + ")";
+
+ assertTableResult(sql, Row.of("ACME", 2, now.plusSeconds(3)));
}
@Test
@@ -467,39 +459,39 @@ void testWindowedGroupingAppliedToMatchRecognize() {
.column("tax", DataTypes.INT())
.build()));
- TableResult tableResult =
- tEnv.executeSql(
- "SELECT\n"
- + " symbol,\n"
- + " SUM(price) as price,\n"
- + " TUMBLE_ROWTIME(matchRowtime, interval '3' second) as rowTime,\n"
- + " TUMBLE_START(matchRowtime, interval '3' second) as startTime\n"
- + "FROM Ticker\n"
- + "MATCH_RECOGNIZE (\n"
- + " PARTITION BY symbol\n"
- + " ORDER BY ts\n"
- + " MEASURES\n"
- + " A.price as price,\n"
- + " A.tax as tax,\n"
- + " MATCH_ROWTIME() as matchRowtime\n"
- + " ONE ROW PER MATCH\n"
- + " PATTERN (A)\n"
- + " DEFINE\n"
- + " A AS A.price > 0\n"
- + ") AS T\n"
- + "GROUP BY symbol, TUMBLE(matchRowtime, interval '3' second)");
- assertThat(CollectionUtil.iteratorToList(tableResult.collect()))
- .containsExactly(
- Row.of(
- "ACME",
- 3,
- LocalDateTime.parse("1970-01-01T00:00:02.999"),
- LocalDateTime.parse("1970-01-01T00:00")),
- Row.of(
- "ACME",
- 2,
- LocalDateTime.parse("1970-01-01T00:00:05.999"),
- LocalDateTime.parse("1970-01-01T00:00:03")));
+ final String sql =
+ "SELECT\n"
+ + " symbol,\n"
+ + " SUM(price) as price,\n"
+ + " TUMBLE_ROWTIME(matchRowtime, interval '3' second) as rowTime,\n"
+ + " TUMBLE_START(matchRowtime, interval '3' second) as startTime\n"
+ + "FROM Ticker\n"
+ + "MATCH_RECOGNIZE (\n"
+ + " PARTITION BY symbol\n"
+ + " ORDER BY ts\n"
+ + " MEASURES\n"
+ + " A.price as price,\n"
+ + " A.tax as tax,\n"
+ + " MATCH_ROWTIME() as matchRowtime\n"
+ + " ONE ROW PER MATCH\n"
+ + " PATTERN (A)\n"
+ + " DEFINE\n"
+ + " A AS A.price > 0\n"
+ + ") AS T\n"
+ + "GROUP BY symbol, TUMBLE(matchRowtime, interval '3' second)";
+
+ assertTableResult(
+ sql,
+ Row.of(
+ "ACME",
+ 3,
+ LocalDateTime.parse("1970-01-01T00:00:02.999"),
+ LocalDateTime.parse("1970-01-01T00:00")),
+ Row.of(
+ "ACME",
+ 2,
+ LocalDateTime.parse("1970-01-01T00:00:05.999"),
+ LocalDateTime.parse("1970-01-01T00:00:03")));
}
@Test
@@ -530,27 +522,27 @@ void testLogicalOffsets() {
.column("tax", DataTypes.INT())
.columnByExpression("ts", "TO_TIMESTAMP_LTZ(tstamp, 3)")
.build()));
- TableResult tableResult =
- tEnv.executeSql(
- "SELECT *\n"
- + "FROM Ticker\n"
- + "MATCH_RECOGNIZE (\n"
- + " ORDER BY ts\n"
- + " MEASURES\n"
- + " FIRST(DOWN.tstamp) AS start_tstamp,\n"
- + " LAST(DOWN.tstamp) AS bottom_tstamp,\n"
- + " UP.tstamp AS end_tstamp,\n"
- + " FIRST(DOWN.price + DOWN.tax + 1) AS bottom_total,\n"
- + " UP.price + UP.tax AS end_total\n"
- + " ONE ROW PER MATCH\n"
- + " AFTER MATCH SKIP PAST LAST ROW\n"
- + " PATTERN (DOWN{2,} UP)\n"
- + " DEFINE\n"
- + " DOWN AS price < LAST(DOWN.price, 1) OR LAST(DOWN.price, 1) IS NULL,\n"
- + " UP AS price < FIRST(DOWN.price)\n"
- + ") AS T");
- assertThat(CollectionUtil.iteratorToList(tableResult.collect()))
- .containsExactly(Row.of(6L, 7L, 8L, 33, 33));
+
+ final String sql =
+ "SELECT *\n"
+ + "FROM Ticker\n"
+ + "MATCH_RECOGNIZE (\n"
+ + " ORDER BY ts\n"
+ + " MEASURES\n"
+ + " FIRST(DOWN.tstamp) AS start_tstamp,\n"
+ + " LAST(DOWN.tstamp) AS bottom_tstamp,\n"
+ + " UP.tstamp AS end_tstamp,\n"
+ + " FIRST(DOWN.price + DOWN.tax + 1) AS bottom_total,\n"
+ + " UP.price + UP.tax AS end_total\n"
+ + " ONE ROW PER MATCH\n"
+ + " AFTER MATCH SKIP PAST LAST ROW\n"
+ + " PATTERN (DOWN{2,} UP)\n"
+ + " DEFINE\n"
+ + " DOWN AS price < LAST(DOWN.price, 1) OR LAST(DOWN.price, 1) IS NULL,\n"
+ + " UP AS price < FIRST(DOWN.price)\n"
+ + ") AS T";
+
+ assertTableResult(sql, Row.of(6L, 7L, 8L, 33, 33));
}
@Test
@@ -579,25 +571,24 @@ void testPartitionByWithParallelSource() {
.column("tax", DataTypes.INT())
.build()));
- TableResult tableResult =
- tEnv.executeSql(
- "SELECT *\n"
- + "FROM Ticker\n"
- + "MATCH_RECOGNIZE (\n"
- + " PARTITION BY symbol\n"
- + " ORDER BY ts\n"
- + " MEASURES\n"
- + " DOWN.tax AS bottom_tax,\n"
- + " UP.tax AS end_tax\n"
- + " ONE ROW PER MATCH\n"
- + " AFTER MATCH SKIP PAST LAST ROW\n"
- + " PATTERN (DOWN UP)\n"
- + " DEFINE\n"
- + " DOWN AS DOWN.price = 13,\n"
- + " UP AS UP.price = 20\n"
- + ") AS T");
- assertThat(CollectionUtil.iteratorToList(tableResult.collect()))
- .containsExactly(Row.of("ACME", 3, 4));
+ final String sql =
+ "SELECT *\n"
+ + "FROM Ticker\n"
+ + "MATCH_RECOGNIZE (\n"
+ + " PARTITION BY symbol\n"
+ + " ORDER BY ts\n"
+ + " MEASURES\n"
+ + " DOWN.tax AS bottom_tax,\n"
+ + " UP.tax AS end_tax\n"
+ + " ONE ROW PER MATCH\n"
+ + " AFTER MATCH SKIP PAST LAST ROW\n"
+ + " PATTERN (DOWN UP)\n"
+ + " DEFINE\n"
+ + " DOWN AS DOWN.price = 13,\n"
+ + " UP AS UP.price = 20\n"
+ + ") AS T";
+
+ assertTableResult(sql, Row.of("ACME", 3, 4));
}
@Test
@@ -628,38 +619,37 @@ void testLogicalOffsetsWithStarVariable() {
.column("price", DataTypes.INT())
.columnByExpression("ts", "TO_TIMESTAMP_LTZ(tstamp, 3)")
.build()));
- TableResult tableResult =
- tEnv.executeSql(
- "SELECT *\n"
- + "FROM Ticker\n"
- + "MATCH_RECOGNIZE (\n"
- + " ORDER BY ts\n"
- + " MEASURES\n"
- + " FIRST(id, 0) as id0,\n"
- + " FIRST(id, 1) as id1,\n"
- + " FIRST(id, 2) as id2,\n"
- + " FIRST(id, 3) as id3,\n"
- + " FIRST(id, 4) as id4,\n"
- + " FIRST(id, 5) as id5,\n"
- + " FIRST(id, 6) as id6,\n"
- + " FIRST(id, 7) as id7,\n"
- + " LAST(id, 0) as id8,\n"
- + " LAST(id, 1) as id9,\n"
- + " LAST(id, 2) as id10,\n"
- + " LAST(id, 3) as id11,\n"
- + " LAST(id, 4) as id12,\n"
- + " LAST(id, 5) as id13,\n"
- + " LAST(id, 6) as id14,\n"
- + " LAST(id, 7) as id15\n"
- + " ONE ROW PER MATCH\n"
- + " AFTER MATCH SKIP PAST LAST ROW\n"
- + " PATTERN (`DOWN\"`{2,} UP)\n"
- + " DEFINE\n"
- + " `DOWN\"` AS price < LAST(price, 1) OR LAST(price, 1) IS NULL,\n"
- + " UP AS price = FIRST(price) AND price > FIRST(price, 3) AND price = LAST(price, 7)\n"
- + ") AS T");
- assertThat(CollectionUtil.iteratorToList(tableResult.collect()))
- .containsExactly(Row.of(1, 2, 3, 4, 5, 6, 7, 8, 8, 7, 6, 5, 4, 3, 2, 1));
+ final String sql =
+ "SELECT *\n"
+ + "FROM Ticker\n"
+ + "MATCH_RECOGNIZE (\n"
+ + " ORDER BY ts\n"
+ + " MEASURES\n"
+ + " FIRST(id, 0) as id0,\n"
+ + " FIRST(id, 1) as id1,\n"
+ + " FIRST(id, 2) as id2,\n"
+ + " FIRST(id, 3) as id3,\n"
+ + " FIRST(id, 4) as id4,\n"
+ + " FIRST(id, 5) as id5,\n"
+ + " FIRST(id, 6) as id6,\n"
+ + " FIRST(id, 7) as id7,\n"
+ + " LAST(id, 0) as id8,\n"
+ + " LAST(id, 1) as id9,\n"
+ + " LAST(id, 2) as id10,\n"
+ + " LAST(id, 3) as id11,\n"
+ + " LAST(id, 4) as id12,\n"
+ + " LAST(id, 5) as id13,\n"
+ + " LAST(id, 6) as id14,\n"
+ + " LAST(id, 7) as id15\n"
+ + " ONE ROW PER MATCH\n"
+ + " AFTER MATCH SKIP PAST LAST ROW\n"
+ + " PATTERN (`DOWN\"`{2,} UP)\n"
+ + " DEFINE\n"
+ + " `DOWN\"` AS price < LAST(price, 1) OR LAST(price, 1) IS NULL,\n"
+ + " UP AS price = FIRST(price) AND price > FIRST(price, 3) AND price = LAST(price, 7)\n"
+ + ") AS T";
+
+ assertTableResult(sql, Row.of(1, 2, 3, 4, 5, 6, 7, 8, 8, 7, 6, 5, 4, 3, 2, 1));
}
@Test
@@ -686,25 +676,25 @@ void testLogicalOffsetOutsideOfRangeInMeasures() {
.column("tax", DataTypes.INT())
.columnByExpression("ts", "TO_TIMESTAMP_LTZ(tstamp, 3)")
.build()));
- TableResult tableResult =
- tEnv.executeSql(
- "SELECT *\n"
- + "FROM Ticker\n"
- + "MATCH_RECOGNIZE (\n"
- + " ORDER BY ts\n"
- + " MEASURES\n"
- + " FIRST(DOWN.price) as first,\n"
- + " LAST(DOWN.price) as last,\n"
- + " FIRST(DOWN.price, 5) as nullPrice\n"
- + " ONE ROW PER MATCH\n"
- + " AFTER MATCH SKIP PAST LAST ROW\n"
- + " PATTERN (DOWN{2,} UP)\n"
- + " DEFINE\n"
- + " DOWN AS price < LAST(DOWN.price, 1) OR LAST(DOWN.price, 1) IS NULL,\n"
- + " UP AS price > LAST(DOWN.price)\n"
- + ") AS T");
- assertThat(CollectionUtil.iteratorToList(tableResult.collect()))
- .containsExactly(Row.of(19, 13, null));
+
+ final String sql =
+ "SELECT *\n"
+ + "FROM Ticker\n"
+ + "MATCH_RECOGNIZE (\n"
+ + " ORDER BY ts\n"
+ + " MEASURES\n"
+ + " FIRST(DOWN.price) as first,\n"
+ + " LAST(DOWN.price) as last,\n"
+ + " FIRST(DOWN.price, 5) as nullPrice\n"
+ + " ONE ROW PER MATCH\n"
+ + " AFTER MATCH SKIP PAST LAST ROW\n"
+ + " PATTERN (DOWN{2,} UP)\n"
+ + " DEFINE\n"
+ + " DOWN AS price < LAST(DOWN.price, 1) OR LAST(DOWN.price, 1) IS NULL,\n"
+ + " UP AS price > LAST(DOWN.price)\n"
+ + ") AS T";
+
+ assertTableResult(sql, Row.of(19, 13, null));
}
/**
@@ -754,35 +744,36 @@ void testAggregates() {
.column("ts", DataTypes.TIMESTAMP_LTZ(3))
.build()));
tEnv.createTemporarySystemFunction("weightedAvg", new WeightedAvg());
- TableResult tableResult =
- tEnv.executeSql(
- "SELECT *\n"
- + "FROM MyTable\n"
- + "MATCH_RECOGNIZE (\n"
- + " ORDER BY ts\n"
- + " MEASURES\n"
- + " FIRST(id) as startId,\n"
- + " SUM(A.price) AS sumA,\n"
- + " COUNT(D.price) AS countD,\n"
- + " SUM(D.price) as sumD,\n"
- + " weightedAvg(price, weight) as wAvg,\n"
- + " AVG(B.price) AS avgB,\n"
- + " SUM(B.price * B.rate) as sumExprB,\n"
- + " LAST(id) as endId\n"
- + " AFTER MATCH SKIP PAST LAST ROW\n"
- + " PATTERN (A+ B+ C D? E)\n"
- + " DEFINE\n"
- + " A AS SUM(A.price) < 6,\n"
- + " B AS SUM(B.price * B.rate) < SUM(A.price) AND\n"
- + " SUM(B.price * B.rate) > 0.2 AND\n"
- + " SUM(B.price) >= 1 AND\n"
- + " AVG(B.price) >= 1 AND\n"
- + " weightedAvg(price, weight) > 1\n"
- + ") AS T");
- assertThat(CollectionUtil.iteratorToList(tableResult.collect()))
- .containsExactly(
- Row.of(1, 5, 0L, null, 2L, 3, 3.4D, 8),
- Row.of(9, 4, 0L, null, 3L, 4, 3.2D, 12));
+
+ final String sql =
+ "SELECT *\n"
+ + "FROM MyTable\n"
+ + "MATCH_RECOGNIZE (\n"
+ + " ORDER BY ts\n"
+ + " MEASURES\n"
+ + " FIRST(id) as startId,\n"
+ + " SUM(A.price) AS sumA,\n"
+ + " COUNT(D.price) AS countD,\n"
+ + " SUM(D.price) as sumD,\n"
+ + " weightedAvg(price, weight) as wAvg,\n"
+ + " AVG(B.price) AS avgB,\n"
+ + " SUM(B.price * B.rate) as sumExprB,\n"
+ + " LAST(id) as endId\n"
+ + " AFTER MATCH SKIP PAST LAST ROW\n"
+ + " PATTERN (A+ B+ C D? E)\n"
+ + " DEFINE\n"
+ + " A AS SUM(A.price) < 6,\n"
+ + " B AS SUM(B.price * B.rate) < SUM(A.price) AND\n"
+ + " SUM(B.price * B.rate) > 0.2 AND\n"
+ + " SUM(B.price) >= 1 AND\n"
+ + " AVG(B.price) >= 1 AND\n"
+ + " weightedAvg(price, weight) > 1\n"
+ + ") AS T";
+
+ assertTableResult(
+ sql,
+ Row.of(1, 5, 0L, null, 2L, 3, 3.4D, 8),
+ Row.of(9, 4, 0L, null, 3L, 4, 3.2D, 12));
}
@Test
@@ -816,27 +807,27 @@ void testAggregatesWithNullInputs() {
.column("ts", DataTypes.TIMESTAMP_LTZ(3))
.build()));
tEnv.createTemporarySystemFunction("weightedAvg", new WeightedAvg());
- TableResult tableResult =
- tEnv.executeSql(
- "SELECT *\n"
- + "FROM MyTable\n"
- + "MATCH_RECOGNIZE (\n"
- + " ORDER BY ts\n"
- + " MEASURES\n"
- + " SUM(A.price) as sumA,\n"
- + " COUNT(A.id) as countAId,\n"
- + " COUNT(A.price) as countAPrice,\n"
- + " COUNT(*) as countAll,\n"
- + " COUNT(price) as countAllPrice,\n"
- + " LAST(id) as endId\n"
- + " AFTER MATCH SKIP PAST LAST ROW\n"
- + " PATTERN (A+ C)\n"
- + " DEFINE\n"
- + " A AS SUM(A.price) < 30,\n"
- + " C AS C.name = 'c'\n"
- + ") AS T");
- assertThat(CollectionUtil.iteratorToList(tableResult.collect()))
- .containsExactly(Row.of(29, 7L, 5L, 8L, 6L, 8));
+
+ final String sql =
+ "SELECT *\n"
+ + "FROM MyTable\n"
+ + "MATCH_RECOGNIZE (\n"
+ + " ORDER BY ts\n"
+ + " MEASURES\n"
+ + " SUM(A.price) as sumA,\n"
+ + " COUNT(A.id) as countAId,\n"
+ + " COUNT(A.price) as countAPrice,\n"
+ + " COUNT(*) as countAll,\n"
+ + " COUNT(price) as countAllPrice,\n"
+ + " LAST(id) as endId\n"
+ + " AFTER MATCH SKIP PAST LAST ROW\n"
+ + " PATTERN (A+ C)\n"
+ + " DEFINE\n"
+ + " A AS SUM(A.price) < 30,\n"
+ + " C AS C.name = 'c'\n"
+ + ") AS T";
+
+ assertTableResult(sql, Row.of(29, 7L, 5L, 8L, 6L, 8));
}
@Test
@@ -851,21 +842,21 @@ void testAccessingCurrentTime() {
.column("name", DataTypes.STRING())
.columnByExpression("proctime", "PROCTIME()")
.build()));
- TableResult tableResult =
- tEnv.executeSql(
- "SELECT T.aid\n"
- + "FROM MyTable\n"
- + "MATCH_RECOGNIZE (\n"
- + " ORDER BY proctime\n"
- + " MEASURES\n"
- + " A.id AS aid,\n"
- + " A.proctime AS aProctime,\n"
- + " LAST(A.proctime + INTERVAL '1' second) as calculatedField\n"
- + " PATTERN (A)\n"
- + " DEFINE\n"
- + " A AS proctime >= (CURRENT_TIMESTAMP - INTERVAL '1' day)\n"
- + ") AS T");
- assertThat(CollectionUtil.iteratorToList(tableResult.collect())).containsExactly(Row.of(1));
+
+ final String sql =
+ "SELECT T.aid\n"
+ + "FROM MyTable\n"
+ + "MATCH_RECOGNIZE (\n"
+ + " ORDER BY proctime\n"
+ + " MEASURES\n"
+ + " A.id AS aid,\n"
+ + " A.proctime AS aProctime,\n"
+ + " LAST(A.proctime + INTERVAL '1' second) as calculatedField\n"
+ + " PATTERN (A)\n"
+ + " DEFINE\n"
+ + " A AS proctime >= (CURRENT_TIMESTAMP - INTERVAL '1' day)\n"
+ + ") AS T";
+ assertTableResult(sql, Row.of(1));
}
@Test
@@ -906,29 +897,28 @@ void testUserDefinedFunctions() {
jobParameters.setString("prefix", prefix);
jobParameters.setString("start", Integer.toString(startFrom));
env.getConfig().setGlobalJobParameters(jobParameters);
- TableResult tableResult =
- tEnv.executeSql(
- String.format(
- "SELECT *\n"
- + "FROM MyTable\n"
- + "MATCH_RECOGNIZE (\n"
- + " ORDER BY ts\n"
- + " MEASURES\n"
- + " FIRST(id) as firstId,\n"
- + " prefix(A.name) as prefixedNameA,\n"
- + " countFrom(A.price) as countFromA,\n"
- + " LAST(id) as lastId\n"
- + " AFTER MATCH SKIP PAST LAST ROW\n"
- + " PATTERN (A+ C)\n"
- + " DEFINE\n"
- + " A AS prefix(A.name) = '%s:a' AND countFrom(A.price) <= %d\n"
- + ") AS T",
- prefix, 4 + 4));
- assertThat(CollectionUtil.iteratorToList(tableResult.collect()))
- .containsExactly(Row.of(1, "PREF:a", 8, 5), Row.of(7, "PREF:a", 6, 9));
+ String sql =
+ String.format(
+ "SELECT *\n"
+ + "FROM MyTable\n"
+ + "MATCH_RECOGNIZE (\n"
+ + " ORDER BY ts\n"
+ + " MEASURES\n"
+ + " FIRST(id) as firstId,\n"
+ + " prefix(A.name) as prefixedNameA,\n"
+ + " countFrom(A.price) as countFromA,\n"
+ + " LAST(id) as lastId\n"
+ + " AFTER MATCH SKIP PAST LAST ROW\n"
+ + " PATTERN (A+ C)\n"
+ + " DEFINE\n"
+ + " A AS prefix(A.name) = '%s:a' AND countFrom(A.price) <= %d\n"
+ + ") AS T",
+ prefix, 4 + 4);
+
+ assertTableResult(sql, Row.of(1, "PREF:a", 8, 5), Row.of(7, "PREF:a", 6, 9));
}
- /** Test prefixing function.. */
+ /** Test prefixing function... */
public static class PrefixingScalarFunc extends ScalarFunction {
private String prefix = "ERROR_VALUE";
@@ -981,4 +971,17 @@ public void accumulate(CountAcc countAcc, Integer value) {
countAcc.count += value;
}
}
+
+ private void assertTableResult(String sql, Row... expected) {
+ TableResult tableResult = tEnv.executeSql(sql);
+ assertThat(CollectionUtil.iteratorToList(tableResult.collect())).containsExactly(expected);
+
+ // Also check that same query is able to compile and return same result if it is used in
+ // view
+ // test cases for FLINK-39293
+ tEnv.executeSql("CREATE VIEW test_view AS \n" + sql);
+ TableResult tableResultWithView = tEnv.executeSql("SELECT * FROM test_view");
+ assertThat(CollectionUtil.iteratorToList(tableResultWithView.collect()))
+ .containsExactly(expected);
+ }
}
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml
index 2b8cfaaffc10c..c0c753fbe87bc 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml
@@ -128,6 +128,144 @@ Calc(select=[a, f0 AS s])
+- Sort(orderBy=[a ASC])
+- Calc(select=[a, b], where=[(a < 5)])
+- BoundedStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c])
+]]>
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRuleTest.xml
index fa4f17124edbc..726b02847c8a5 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRuleTest.xml
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRuleTest.xml
@@ -34,7 +34,25 @@ AsyncCalc(select=[a, func3($f1) AS EXPR$1])
+- GroupAggregate(groupBy=[a], select=[a, COUNT(*) AS $f1])
+- Exchange(distribution=[hash[a]])
+- Calc(select=[a])
- +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+ +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e])
+]]>
+
+
+
+
+
+
+
+
+
+
+
@@ -52,7 +70,7 @@ LogicalProject(EXPR$0=[func5($0).f0])
@@ -71,7 +89,7 @@ LogicalProject(EXPR$0=[func1(func5($0).f0)])
AsyncCalc(select=[func1(f0) AS EXPR$0])
+- Calc(select=[f0.f0 AS f0])
+- AsyncCalc(select=[func5(a) AS f0])
- +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+ +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e])
]]>
@@ -88,7 +106,7 @@ LogicalProject(a=[$0], EXPR$1=[func1($0)])
@@ -107,7 +125,7 @@ LogicalProject(EXPR$0=[func1($0)], EXPR$1=[func2($0)], EXPR$2=[func1($0)], EXPR$
Calc(select=[f0 AS EXPR$0, f00 AS EXPR$1, f0 AS EXPR$2, f00 AS EXPR$3])
+- AsyncCalc(select=[f0, func2(a) AS f00])
+- AsyncCalc(select=[a, func1(a) AS f0])
- +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+ +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e])
]]>
@@ -118,9 +136,9 @@ Calc(select=[f0 AS EXPR$0, f00 AS EXPR$1, f0 AS EXPR$2, f00 AS EXPR$3])
(func6($0, $4), 10))], joinType=[inner])
++- LogicalJoin(condition=[AND(=($0, $5), >(func6($0, $5), 10))], joinType=[inner])
:- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
]]>
@@ -159,7 +177,7 @@ Calc(select=[a], where=[>(f0, 10)])
+- Join(joinType=[InnerJoin], where=[=(a, a2)], select=[a, a2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])
:- Exchange(distribution=[hash[a]])
: +- Calc(select=[a])
- : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+ : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e])
+- Exchange(distribution=[hash[a2]])
+- Calc(select=[a2])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a2, b2, c2, d2])
@@ -174,7 +192,7 @@ Calc(select=[a], where=[>(f0, 10)])
@@ -186,7 +204,7 @@ Calc(select=[a])
:- Exchange(distribution=[hash[a]])
: +- Calc(select=[a], where=[REGEXP(f0, 'val (2|3)')])
: +- AsyncCalc(select=[a, func2(a) AS f0])
- : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+ : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e])
+- Exchange(distribution=[hash[a2]])
+- Calc(select=[a2], where=[REGEXP(f0, 'val (2|3)')])
+- AsyncCalc(select=[a2, func2(a2) AS f0])
@@ -201,8 +219,8 @@ Calc(select=[a])
(func6($0, $4), 10)])
- +- LogicalJoin(condition=[=($0, $4)], joinType=[inner])
++- LogicalFilter(condition=[>(func6($0, $5), 10)])
+ +- LogicalJoin(condition=[=($0, $5)], joinType=[inner])
:- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
]]>
@@ -214,7 +232,7 @@ Calc(select=[a], where=[>(f0, 10)])
+- Join(joinType=[InnerJoin], where=[=(a, a2)], select=[a, a2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])
:- Exchange(distribution=[hash[a]])
: +- Calc(select=[a])
- : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+ : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e])
+- Exchange(distribution=[hash[a2]])
+- Calc(select=[a2])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a2, b2, c2, d2])
@@ -228,7 +246,7 @@ Calc(select=[a], where=[>(f0, 10)])
@@ -239,7 +257,7 @@ AsyncCalc(select=[func1(a) AS EXPR$0])
+- Join(joinType=[InnerJoin], where=[=(a, a2)], select=[a, a2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])
:- Exchange(distribution=[hash[a]])
: +- Calc(select=[a])
- : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+ : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e])
+- Exchange(distribution=[hash[a2]])
+- Calc(select=[a2])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a2, b2, c2, d2])
@@ -270,8 +288,8 @@ AsyncCalc(select=[func1(1) AS EXPR$0])
(func6($0, $4), 10))])
- +- LogicalJoin(condition=[=($0, $4)], joinType=[left])
++- LogicalFilter(condition=[AND(=($0, $5), >(func6($0, $5), 10))])
+ +- LogicalJoin(condition=[=($0, $5)], joinType=[left])
:- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
]]>
@@ -283,7 +301,7 @@ Calc(select=[a], where=[>(f0, 10)])
+- Join(joinType=[InnerJoin], where=[=(a, a2)], select=[a, a2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])
:- Exchange(distribution=[hash[a]])
: +- Calc(select=[a])
- : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+ : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e])
+- Exchange(distribution=[hash[a2]])
+- Calc(select=[a2])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a2, b2, c2, d2])
@@ -296,10 +314,10 @@ Calc(select=[a], where=[>(f0, 10)])
@@ -337,7 +355,7 @@ Join(joinType=[LeftOuterJoin], where=[=(a, a2)], select=[a, a2], leftInputSpec=[
:- Exchange(distribution=[hash[a]])
: +- Calc(select=[a], where=[REGEXP(f0, 'string (2|3)')])
: +- AsyncCalc(select=[a, func2(a) AS f0])
-: +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+: +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e])
+- Exchange(distribution=[hash[a2]])
+- Calc(select=[a2])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a2, b2, c2, d2])
@@ -351,8 +369,8 @@ Join(joinType=[LeftOuterJoin], where=[=(a, a2)], select=[a, a2], leftInputSpec=[
(func6($0, $4), 10)])
- +- LogicalJoin(condition=[=($0, $4)], joinType=[left])
++- LogicalFilter(condition=[>(func6($0, $5), 10)])
+ +- LogicalJoin(condition=[=($0, $5)], joinType=[left])
:- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
]]>
@@ -364,7 +382,7 @@ Calc(select=[a], where=[>(f0, 10)])
+- Join(joinType=[LeftOuterJoin], where=[=(a, a2)], select=[a, a2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])
:- Exchange(distribution=[hash[a]])
: +- Calc(select=[a])
- : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+ : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e])
+- Exchange(distribution=[hash[a2]])
+- Calc(select=[a2])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a2, b2, c2, d2])
@@ -378,7 +396,7 @@ Calc(select=[a], where=[>(f0, 10)])
(func1($4), 10))], joinType=[left])
++- LogicalJoin(condition=[AND(=($0, $5), >(func1($5), 10))], joinType=[left])
:- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
]]>
@@ -389,7 +407,7 @@ Calc(select=[a])
+- Join(joinType=[LeftOuterJoin], where=[=(a, a2)], select=[a, a2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])
:- Exchange(distribution=[hash[a]])
: +- Calc(select=[a])
- : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+ : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e])
+- Exchange(distribution=[hash[a2]])
+- Calc(select=[a2], where=[>(f0, 10)])
+- AsyncCalc(select=[a2, func1(a2) AS f0])
@@ -404,8 +422,8 @@ Calc(select=[a])
(func1($4), 10)])
- +- LogicalJoin(condition=[=($0, $4)], joinType=[left])
++- LogicalFilter(condition=[>(func1($5), 10)])
+ +- LogicalJoin(condition=[=($0, $5)], joinType=[left])
:- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
]]>
@@ -417,7 +435,7 @@ Calc(select=[a], where=[>(f0, 10)])
+- Join(joinType=[LeftOuterJoin], where=[=(a, a2)], select=[a, a2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])
:- Exchange(distribution=[hash[a]])
: +- Calc(select=[a])
- : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+ : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e])
+- Exchange(distribution=[hash[a2]])
+- Calc(select=[a2])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a2, b2, c2, d2])
@@ -438,7 +456,7 @@ LogicalProject(EXPR$0=[_UTF-16LE'foo'], EXPR$1=[func1($0)])
@@ -457,7 +475,7 @@ LogicalProject(EXPR$0=[func1(func1(func1($0)))])
AsyncCalc(select=[func1(f0) AS EXPR$0])
+- AsyncCalc(select=[func1(f0) AS f0])
+- AsyncCalc(select=[func1(a) AS f0])
- +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+ +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e])
]]>
@@ -493,7 +511,7 @@ LogicalProject(EXPR$0=[CONCAT(func2($0), _UTF-16LE'foo')])
@@ -513,7 +531,7 @@ LogicalProject(blah=[$0])
@@ -524,8 +542,8 @@ Calc(select=[f0 AS blah], where=[REGEXP(f0, 'string (2|3)')])
(func6($0, $4), 10))])
- +- LogicalJoin(condition=[=($0, $4)], joinType=[right])
++- LogicalFilter(condition=[AND(=($0, $5), >(func6($0, $5), 10))])
+ +- LogicalJoin(condition=[=($0, $5)], joinType=[right])
:- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
]]>
@@ -537,7 +555,7 @@ Calc(select=[a], where=[>(f0, 10)])
+- Join(joinType=[InnerJoin], where=[=(a, a2)], select=[a, a2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])
:- Exchange(distribution=[hash[a]])
: +- Calc(select=[a])
- : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+ : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e])
+- Exchange(distribution=[hash[a2]])
+- Calc(select=[a2])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a2, b2, c2, d2])
@@ -550,10 +568,10 @@ Calc(select=[a], where=[>(f0, 10)])
(func6($0, $4), 10)])
- +- LogicalJoin(condition=[=($0, $4)], joinType=[right])
++- LogicalFilter(condition=[>(func6($0, $5), 10)])
+ +- LogicalJoin(condition=[=($0, $5)], joinType=[right])
:- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
]]>
@@ -592,7 +610,7 @@ Calc(select=[a], where=[>(f0, 10)])
+- Join(joinType=[RightOuterJoin], where=[=(a, a2)], select=[a, a2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])
:- Exchange(distribution=[hash[a]])
: +- Calc(select=[a])
- : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+ : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e])
+- Exchange(distribution=[hash[a2]])
+- Calc(select=[a2])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a2, b2, c2, d2])
@@ -606,7 +624,7 @@ Calc(select=[a], where=[>(f0, 10)])
(func1($0), 10))], joinType=[right])
++- LogicalJoin(condition=[AND(=($0, $5), >(func1($0), 10))], joinType=[right])
:- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
]]>
@@ -618,7 +636,7 @@ Calc(select=[a])
:- Exchange(distribution=[hash[a]])
: +- Calc(select=[a], where=[>(f0, 10)])
: +- AsyncCalc(select=[a, func1(a) AS f0])
- : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+ : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e])
+- Exchange(distribution=[hash[a2]])
+- Calc(select=[a2])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a2, b2, c2, d2])
@@ -633,7 +651,7 @@ Calc(select=[a])
(func1($0), 10)])
- +- LogicalJoin(condition=[=($0, $4)], joinType=[right])
+ +- LogicalJoin(condition=[=($0, $5)], joinType=[right])
:- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
]]>
@@ -645,7 +663,7 @@ Calc(select=[a], where=[>(f0, 10)])
+- Join(joinType=[RightOuterJoin], where=[=(a, a2)], select=[a, a2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])
:- Exchange(distribution=[hash[a]])
: +- Calc(select=[a])
- : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+ : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e])
+- Exchange(distribution=[hash[a2]])
+- Calc(select=[a2])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a2, b2, c2, d2])
@@ -665,7 +683,7 @@ LogicalProject(EXPR$0=[func4($3)])
@@ -682,7 +700,7 @@ LogicalProject(EXPR$0=[func1($0)])
@@ -703,7 +721,7 @@ AsyncCalc(select=[f0 AS EXPR$0, func1(f1) AS EXPR$1, f2 AS EXPR$2])
+- AsyncCalc(select=[f2, f1, func1(f0) AS f0])
+- Calc(select=[f0, f0 AS f1, f0 AS f2])
+- AsyncCalc(select=[func1(a) AS f0])
- +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+ +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e])
]]>
@@ -721,7 +739,7 @@ LogicalProject(EXPR$0=[func1($0)], EXPR$1=[func1($0)])
@@ -740,7 +758,7 @@ LogicalProject(a=[$0])
@@ -759,7 +777,7 @@ LogicalProject(EXPR$0=[func2($0)])
@@ -779,7 +797,7 @@ LogicalProject(blah=[$0])
@@ -798,7 +816,7 @@ LogicalProject(a=[$0])
=(f0, 12)])
+- AsyncCalc(select=[a, func1(a) AS f0])
- +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+ +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e])
]]>
@@ -822,7 +840,7 @@ AsyncCalc(select=[func1(a) AS EXPR$0])
+- Join(joinType=[LeftAntiJoin], where=[OR(IS NULL(a), IS NULL(a2), =(a, a2))], select=[a], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])
:- Exchange(distribution=[single])
: +- Calc(select=[a])
- : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+ : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e])
+- Exchange(distribution=[single])
+- Calc(select=[a2])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a2, b2, c2, d2])
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRuleTest.xml
index dcbe743d2c2c7..2cdbfbaf162f8 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRuleTest.xml
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRuleTest.xml
@@ -22,7 +22,7 @@ limitations under the License.
@@ -43,7 +43,7 @@ Calc(select=[a, b, c, d, EXPR$0])
@@ -64,7 +64,7 @@ Calc(select=[a, b, c, d, EXPR$0])
@@ -86,7 +86,7 @@ Calc(select=[a, b, c, d, EXPR$0])
+
+
+
+
+
+
+
+
+
+
+
@@ -107,7 +128,7 @@ Calc(select=[a, b, c, d, EXPR$0])
@@ -128,7 +149,7 @@ Calc(select=[a, b, c, d, EXPR$0])
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml
index 6c744e43ca0ca..930a6d94bf5ad 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml
@@ -133,6 +133,156 @@ LogicalProject(a=[$0], s=[$2])
: +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+- LogicalProject(s=[$0])
+- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS$1($cor0.set)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) f0)])
+]]>
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml
index aae5d7ca42901..c9cfa9694a791 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml
@@ -16,13 +16,648 @@ See the License for the specific language governing permissions and
limitations under the License.
-->
+
+
+ 5 THEN 1 END) as bulk_orders
+FROM Users u
+LEFT JOIN Orders o
+ ON u.user_id = o.user_id
+LEFT JOIN OrderItems oi
+ ON o.order_id = oi.order_id
+LEFT JOIN ProductCategories pc
+ ON oi.product_name = pc.category_id
+LEFT JOIN Payments p
+ ON u.user_id = p.user_id
+GROUP BY u.user_id, u.name, pc.category_name
+HAVING COUNT(DISTINCT o.order_id) > 0]]>
+
+
+ ($3, 0)])
++- LogicalAggregate(group=[{0, 1, 2}], order_count=[COUNT(DISTINCT $3)], total_items=[SUM($4)], total_value=[SUM($5)], avg_item_price=[AVG($6)], max_payment=[MAX($7)], min_payment=[MIN($7)], bulk_orders=[COUNT($8)])
+ +- LogicalProject(user_id=[$0], name=[$1], category_name=[$12], order_id=[$3], quantity=[$9], $f5=[*($9, $10)], unit_price=[$10], price=[$15], $f8=[CASE(>($9, 5), 1, null:INTEGER)])
+ +- LogicalJoin(condition=[=($0, $16)], joinType=[left])
+ :- LogicalJoin(condition=[=($8, $11)], joinType=[left])
+ : :- LogicalJoin(condition=[=($3, $7)], joinType=[left])
+ : : :- LogicalJoin(condition=[=($0, $4)], joinType=[left])
+ : : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]])
+ : : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]])
+ : : +- LogicalTableScan(table=[[default_catalog, default_database, OrderItems]])
+ : +- LogicalTableScan(table=[[default_catalog, default_database, ProductCategories]])
+ +- LogicalTableScan(table=[[default_catalog, default_database, Payments]])
+]]>
+
+
+ (order_count, 0)])
++- GroupAggregate(groupBy=[user_id, name, category_name], select=[user_id, name, category_name, COUNT_RETRACT(DISTINCT order_id) AS order_count, SUM_RETRACT(quantity) AS total_items, SUM_RETRACT($f5) AS total_value, AVG_RETRACT(unit_price) AS avg_item_price, MAX_RETRACT(price) AS max_payment, MIN_RETRACT(price) AS min_payment, COUNT_RETRACT($f8) AS bulk_orders])
+ +- Exchange(distribution=[hash[user_id, name, category_name]])
+ +- Calc(select=[user_id, name, category_name, order_id, quantity, *(quantity, unit_price) AS $f5, unit_price, price, CASE(>(quantity, 5), 1, null:INTEGER) AS $f8])
+ +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, noUniqueKey], joinConditions=[=(user_id, user_id1)], select=[user_id,name,order_id,quantity,unit_price,category_name,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, INTEGER quantity, DOUBLE unit_price, VARCHAR(2147483647) category_name, INTEGER price, VARCHAR(2147483647) user_id1)])
+ :- Exchange(distribution=[hash[user_id]])
+ : +- Calc(select=[user_id, name, order_id, quantity, unit_price, category_name])
+ : +- MultiJoin(commonJoinKey=[product_name], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, (category_id)], joinConditions=[=(product_name, category_id)], select=[user_id,name,order_id,product_name,quantity,unit_price,category_id,category_name], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) product_name, INTEGER quantity, DOUBLE unit_price, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name)])
+ : :- Exchange(distribution=[hash[product_name]])
+ : : +- Calc(select=[user_id, name, order_id, product_name, quantity, unit_price])
+ : : +- MultiJoin(commonJoinKey=[order_id], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, noUniqueKey], joinConditions=[=(order_id, order_id0)], select=[user_id,name,order_id,order_id0,product_name,quantity,unit_price], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) order_id0, VARCHAR(2147483647) product_name, INTEGER quantity, DOUBLE unit_price)])
+ : : :- Exchange(distribution=[hash[order_id]])
+ : : : +- Calc(select=[user_id, name, order_id])
+ : : : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[(user_id), (order_id)], joinConditions=[=(user_id, user_id0)], select=[user_id,name,order_id,user_id0], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0)])
+ : : : :- Exchange(distribution=[hash[user_id]])
+ : : : : +- ChangelogNormalize(key=[user_id])
+ : : : : +- Exchange(distribution=[hash[user_id]])
+ : : : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name])
+ : : : +- Exchange(distribution=[hash[user_id]])
+ : : : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id])
+ : : +- Exchange(distribution=[hash[order_id]])
+ : : +- Calc(select=[order_id, product_name, quantity, unit_price])
+ : : +- ChangelogNormalize(key=[item_id])
+ : : +- Exchange(distribution=[hash[item_id]])
+ : : +- TableSourceScan(table=[[default_catalog, default_database, OrderItems]], fields=[item_id, order_id, product_name, quantity, unit_price])
+ : +- Exchange(distribution=[hash[category_id]])
+ : +- ChangelogNormalize(key=[category_id])
+ : +- Exchange(distribution=[hash[category_id]])
+ : +- TableSourceScan(table=[[default_catalog, default_database, ProductCategories, project=[category_id, category_name], metadata=[]]], fields=[category_id, category_name])
+ +- Exchange(distribution=[hash[user_id]])
+ +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[price, user_id], metadata=[]]], fields=[price, user_id])
+]]>
+
+
+
+
+ 0]]>
+
+
+ ($2, 0)])
++- LogicalAggregate(group=[{0}], unique_users=[COUNT(DISTINCT $1)], total_sales=[COUNT($2)], total_revenue=[SUM($3)], avg_sale_amount=[AVG($3)], max_sale_amount=[MAX($3)])
+ +- LogicalProject(category_name=[$7], user_id=[$0], sale_id=[$10], amount=[$13])
+ +- LogicalJoin(condition=[=($0, $11)], joinType=[left])
+ :- LogicalJoin(condition=[AND(=($0, $9), =($5, $6))], joinType=[left])
+ : :- LogicalJoin(condition=[=($0, $4)], joinType=[left])
+ : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]])
+ : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]])
+ : +- LogicalTableScan(table=[[default_catalog, default_database, Categories]])
+ +- LogicalTableScan(table=[[default_catalog, default_database, Sales]])
+]]>
+
+
+ (total_sales, 0)])
++- GroupAggregate(groupBy=[category_name], select=[category_name, COUNT_RETRACT(DISTINCT user_id) AS unique_users, COUNT_RETRACT(sale_id) AS total_sales, SUM_RETRACT(amount) AS total_revenue, AVG_RETRACT(amount) AS avg_sale_amount, MAX_RETRACT(amount) AS max_sale_amount])
+ +- Exchange(distribution=[hash[category_name]])
+ +- Calc(select=[category_name, user_id, sale_id, amount])
+ +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, LEFT, LEFT], inputUniqueKeys=[(user_id), noUniqueKey, (category_id), (sale_id)], joinConditions=[=(user_id, user_id0), AND(=(user_id, user_id1), =(product, category_id)), =(user_id, user_id2)], select=[user_id,user_id0,product,category_id,category_name,user_id1,sale_id,user_id2,amount], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) sale_id, VARCHAR(2147483647) user_id2, DOUBLE amount)])
+ :- Exchange(distribution=[hash[user_id]])
+ : +- ChangelogNormalize(key=[user_id])
+ : +- Exchange(distribution=[hash[user_id]])
+ : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id], metadata=[]]], fields=[user_id])
+ :- Exchange(distribution=[hash[user_id]])
+ : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[user_id, product], metadata=[]]], fields=[user_id, product])
+ :- Exchange(distribution=[hash[user_id]])
+ : +- ChangelogNormalize(key=[category_id])
+ : +- Exchange(distribution=[hash[category_id]])
+ : +- TableSourceScan(table=[[default_catalog, default_database, Categories, project=[category_id, category_name, user_id], metadata=[]]], fields=[category_id, category_name, user_id])
+ +- Exchange(distribution=[hash[user_id]])
+ +- ChangelogNormalize(key=[sale_id])
+ +- Exchange(distribution=[hash[sale_id]])
+ +- TableSourceScan(table=[[default_catalog, default_database, Sales, project=[sale_id, user_id, amount], metadata=[]]], fields=[sale_id, user_id, amount])
+]]>
+
+
+
+
+ 600000
+),
+active_projects AS (
+ SELECT project_id, project_name, dept_id
+ FROM Projects
+ WHERE status = 'ACTIVE'
+)
+SELECT
+ u.user_id,
+ u.name,
+ o.order_id,
+ hbd.dept_name,
+ ap.project_name,
+ hbd.budget
+FROM Users u
+LEFT JOIN Orders o
+ ON u.user_id = o.user_id
+LEFT JOIN high_budget_depts hbd
+ ON o.user_id = hbd.dept_id
+LEFT JOIN active_projects ap
+ ON hbd.dept_id = ap.dept_id]]>
+
+
+ ($2, 600000)])
+ : +- LogicalTableScan(table=[[default_catalog, default_database, Departments]])
+ +- LogicalProject(project_id=[$0], project_name=[$1], dept_id=[$2])
+ +- LogicalFilter(condition=[=($3, _UTF-16LE'ACTIVE')])
+ +- LogicalTableScan(table=[[default_catalog, default_database, Projects]])
+]]>
+
+
+ (budget, 600000)])
+ : +- Exchange(distribution=[hash[dept_id]])
+ : +- TableSourceScan(table=[[default_catalog, default_database, Departments, filter=[]]], fields=[dept_id, dept_name, budget])
+ +- Exchange(distribution=[hash[dept_id]])
+ +- Calc(select=[project_name, dept_id])
+ +- ChangelogNormalize(key=[project_id], condition=[=(status, 'ACTIVE')])
+ +- Exchange(distribution=[hash[project_id]])
+ +- TableSourceScan(table=[[default_catalog, default_database, Projects, filter=[]]], fields=[project_id, project_name, dept_id, status])
+]]>
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+ 1000 THEN 'High-Value Premium'
+ WHEN pc.is_premium = true THEN 'Premium'
+ WHEN p.price > 500 THEN 'Standard High-Value'
+ ELSE 'Standard'
+ END AS product_tier,
+ CASE
+ WHEN pr.rating >= 4 AND pr.is_verified = true THEN 'Highly Recommended'
+ WHEN pr.rating >= 3 THEN 'Recommended'
+ WHEN pr.rating >= 2 THEN 'Average'
+ ELSE 'Not Recommended'
+ END AS recommendation_status,
+ CASE
+ WHEN pc.discount_rate > 0.2 THEN p.price * (1 - pc.discount_rate)
+ ELSE p.price
+ END AS final_price
+FROM Users u
+LEFT JOIN Orders o
+ ON u.user_id = o.user_id
+LEFT JOIN Payments p
+ ON u.user_id = p.user_id
+LEFT JOIN ProductCategories pc
+ ON o.product = pc.category_id
+LEFT JOIN ProductReviews pr
+ ON o.product = pr.product_id]]>
+
+
+ ($7, 1000)), _UTF-16LE'High-Value Premium':VARCHAR(19) CHARACTER SET "UTF-16LE", $11, _UTF-16LE'Premium':VARCHAR(19) CHARACTER SET "UTF-16LE", >($7, 500), _UTF-16LE'Standard High-Value':VARCHAR(19) CHARACTER SET "UTF-16LE", _UTF-16LE'Standard':VARCHAR(19) CHARACTER SET "UTF-16LE")], recommendation_status=[CASE(AND(>=($15, 4), $16), _UTF-16LE'Highly Recommended':VARCHAR(18) CHARACTER SET "UTF-16LE", >=($15, 3), _UTF-16LE'Recommended':VARCHAR(18) CHARACTER SET "UTF-16LE", >=($15, 2), _UTF-16LE'Average':VARCHAR(18) CHARACTER SET "UTF-16LE", _UTF-16LE'Not Recommended':VARCHAR(18) CHARACTER SET "UTF-16LE")], final_price=[CASE(>($12, 0.2:DECIMAL(2, 1)), *($7, -(1, $12)), CAST($7):DOUBLE)])
++- LogicalJoin(condition=[=($5, $14)], joinType=[left])
+ :- LogicalJoin(condition=[=($5, $9)], joinType=[left])
+ : :- LogicalJoin(condition=[=($0, $8)], joinType=[left])
+ : : :- LogicalJoin(condition=[=($0, $4)], joinType=[left])
+ : : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]])
+ : : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]])
+ : : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]])
+ : +- LogicalTableScan(table=[[default_catalog, default_database, ProductCategories]])
+ +- LogicalTableScan(table=[[default_catalog, default_database, ProductReviews]])
+]]>
+
+
+ (price, 1000)), 'High-Value Premium', is_premium, 'Premium', >(price, 500), 'Standard High-Value', 'Standard') AS product_tier, CASE(AND(>=(rating, 4), is_verified), 'Highly Recommended', >=(rating, 3), 'Recommended', >=(rating, 2), 'Average', 'Not Recommended') AS recommendation_status, CASE(>(discount_rate, 0.2), *(price, -(1, discount_rate)), CAST(price AS DOUBLE)) AS final_price])
++- MultiJoin(commonJoinKey=[product], joinTypes=[LEFT, LEFT], inputUniqueKeys=[noUniqueKey, (category_id), noUniqueKey], joinConditions=[=(product, category_id), =(product, product_id)], select=[user_id,order_id,product,payment_id,price,category_id,category_name,is_premium,discount_rate,product_id,rating,is_verified], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, BOOLEAN is_premium, DOUBLE discount_rate, VARCHAR(2147483647) product_id, INTEGER rating, BOOLEAN is_verified)])
+ :- Exchange(distribution=[hash[product]])
+ : +- Calc(select=[user_id, order_id, product, payment_id, price])
+ : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)])
+ : :- Exchange(distribution=[hash[user_id]])
+ : : +- ChangelogNormalize(key=[user_id])
+ : : +- Exchange(distribution=[hash[user_id]])
+ : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id], metadata=[]]], fields=[user_id])
+ : :- Exchange(distribution=[hash[user_id]])
+ : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product])
+ : +- Exchange(distribution=[hash[user_id]])
+ : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id])
+ :- Exchange(distribution=[hash[category_id]])
+ : +- ChangelogNormalize(key=[category_id])
+ : +- Exchange(distribution=[hash[category_id]])
+ : +- TableSourceScan(table=[[default_catalog, default_database, ProductCategories]], fields=[category_id, category_name, is_premium, discount_rate])
+ +- Exchange(distribution=[hash[product_id]])
+ +- Calc(select=[product_id, rating, is_verified])
+ +- ChangelogNormalize(key=[review_id])
+ +- Exchange(distribution=[hash[review_id]])
+ +- TableSourceScan(table=[[default_catalog, default_database, ProductReviews]], fields=[review_id, product_id, rating, is_verified])
+]]>
+
+
+
+
+ 1000 THEN 'Premium'
+ WHEN bo.price > 500 THEN 'Standard'
+ ELSE 'Basic'
+ END as order_tier
+ FROM base_orders bo
+ LEFT JOIN OrderMetrics om
+ ON bo.order_id = om.order_id
+),
+aggregated_metrics AS (
+ SELECT
+ user_id,
+ name,
+ COUNT(DISTINCT order_id) as total_orders,
+ SUM(price) as total_spent,
+ AVG(price) as avg_order_value,
+ MAX(metric_value) as max_metric,
+ MIN(metric_value) as min_metric,
+ COUNT(CASE WHEN order_tier = 'Premium' THEN 1 END) as premium_orders
+ FROM enriched_orders
+ GROUP BY user_id, name
+)
+SELECT
+ user_id,
+ UPPER(name) as user_name,
+ total_orders,
+ ROUND(total_spent, 2) as total_spent_rounded,
+ ROUND(avg_order_value, 2) as avg_order_value_rounded,
+ CONCAT('User: ', name, ' has ', CAST(total_orders AS STRING), ' orders') as summary,
+ CASE
+ WHEN total_orders > 10 THEN 'Frequent Customer'
+ WHEN total_orders > 5 THEN 'Regular Customer'
+ ELSE 'Occasional Customer'
+ END as customer_type
+FROM aggregated_metrics
+WHERE total_spent > 0]]>
+
+
+ ($2, 10), _UTF-16LE'Frequent Customer':VARCHAR(19) CHARACTER SET "UTF-16LE", >($2, 5), _UTF-16LE'Regular Customer':VARCHAR(19) CHARACTER SET "UTF-16LE", _UTF-16LE'Occasional Customer':VARCHAR(19) CHARACTER SET "UTF-16LE")])
++- LogicalFilter(condition=[>($3, 0)])
+ +- LogicalAggregate(group=[{0, 1}], total_orders=[COUNT(DISTINCT $2)], total_spent=[SUM($3)], avg_order_value=[AVG($3)], max_metric=[MAX($4)], min_metric=[MIN($4)], premium_orders=[COUNT($5)])
+ +- LogicalProject(user_id=[$0], name=[$1], order_id=[$2], price=[$4], metric_value=[$6], $f5=[CASE(=($7, _UTF-16LE'Premium'), 1, null:INTEGER)])
+ +- LogicalProject(user_id=[$0], name=[$1], order_id=[$2], payment_id=[$3], price=[$4], metric_type=[$7], metric_value=[$8], order_tier=[CASE(>($4, 1000), _UTF-16LE'Premium':VARCHAR(8) CHARACTER SET "UTF-16LE", >($4, 500), _UTF-16LE'Standard':VARCHAR(8) CHARACTER SET "UTF-16LE", _UTF-16LE'Basic':VARCHAR(8) CHARACTER SET "UTF-16LE")])
+ +- LogicalJoin(condition=[=($2, $6)], joinType=[left])
+ :- LogicalProject(user_id=[$0], name=[$1], order_id=[$3], payment_id=[$6], price=[$7])
+ : +- LogicalJoin(condition=[=($0, $8)], joinType=[inner])
+ : :- LogicalJoin(condition=[=($0, $4)], joinType=[inner])
+ : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]])
+ : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]])
+ : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]])
+ +- LogicalTableScan(table=[[default_catalog, default_database, OrderMetrics]])
+]]>
+
+
+ (total_orders, 10), 'Frequent Customer', >(total_orders, 5), 'Regular Customer', 'Occasional Customer') AS customer_type], where=[>(total_spent, 0)])
++- GroupAggregate(groupBy=[user_id, name], select=[user_id, name, COUNT_RETRACT(DISTINCT order_id) AS total_orders, SUM_RETRACT(price) AS total_spent, AVG_RETRACT(price) AS avg_order_value])
+ +- Exchange(distribution=[hash[user_id, name]])
+ +- MultiJoin(commonJoinKey=[order_id], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, noUniqueKey], joinConditions=[=(order_id, order_id0)], select=[user_id,name,order_id,price,order_id0,metric_value], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, INTEGER price, VARCHAR(2147483647) order_id0, DOUBLE metric_value)])
+ :- Exchange(distribution=[hash[order_id]])
+ : +- Calc(select=[user_id, name, order_id, price])
+ : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[INNER, INNER], inputUniqueKeys=[(user_id), (order_id), noUniqueKey], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,name,order_id,user_id0,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, INTEGER price, VARCHAR(2147483647) user_id1)])
+ : :- Exchange(distribution=[hash[user_id]])
+ : : +- ChangelogNormalize(key=[user_id])
+ : : +- Exchange(distribution=[hash[user_id]])
+ : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name])
+ : :- Exchange(distribution=[hash[user_id]])
+ : : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id])
+ : +- Exchange(distribution=[hash[user_id]])
+ : +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[price, user_id], metadata=[]]], fields=[price, user_id])
+ +- Exchange(distribution=[hash[order_id]])
+ +- Calc(select=[order_id, metric_value])
+ +- ChangelogNormalize(key=[metric_id])
+ +- Exchange(distribution=[hash[metric_id]])
+ +- TableSourceScan(table=[[default_catalog, default_database, OrderMetrics, project=[order_id, metric_value, metric_id], metadata=[]]], fields=[order_id, metric_value, metric_id])
+]]>
+
+
+
+
+
+
+
+
+
+
+
+
+
- = p.price OR p.price < 0) LEFT JOIN Shipments s ON p.user_id_2 = s.user_id_3]]>
+ = p.price OR p.price < 0)
+LEFT JOIN Shipments s
+ ON p.user_id = s.user_id]]>
=($2, $7), <($7, 0)))], joinType=[inner])
: :- LogicalJoin(condition=[=($0, $4)], joinType=[left])
@@ -34,25 +669,25 @@ LogicalProject(user_id_0=[$0], name=[$1], order_id=[$3], payment_id=[$6], locati
= price) OR (price < 0))), (user_id_2 = user_id_3)], select=[user_id_0,name,cash,order_id,user_id_1,payment_id,price,user_id_2,location,user_id_3], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id_2, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id_3)])
- :- Exchange(distribution=[hash[user_id_0]])
- : +- ChangelogNormalize(key=[user_id_0])
- : +- Exchange(distribution=[hash[user_id_0]])
- : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id_0, name, cash])
- :- Exchange(distribution=[hash[user_id_1]])
- : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id_1], metadata=[]]], fields=[order_id, user_id_1])
- :- Exchange(distribution=[hash[user_id_2]])
- : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id_2])
- +- Exchange(distribution=[hash[user_id_3]])
- +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id_3])
+Calc(select=[user_id, name, order_id, payment_id, location])
++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id), noUniqueKey], joinConditions=[(user_id = user_id0), ((user_id = user_id1) AND ((cash >= price) OR (price < 0))), (user_id1 = user_id2)], select=[user_id,name,cash,order_id,user_id0,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)])
+ :- Exchange(distribution=[hash[user_id]])
+ : +- ChangelogNormalize(key=[user_id])
+ : +- Exchange(distribution=[hash[user_id]])
+ : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash])
+ :- Exchange(distribution=[hash[user_id]])
+ : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id])
+ :- Exchange(distribution=[hash[user_id]])
+ : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id])
+ +- Exchange(distribution=[hash[user_id]])
+ +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id])
]]>
=($2, $7), <($7, 0)))], joinType=[inner])
: :- LogicalJoin(condition=[=($0, $4)], joinType=[left])
@@ -62,42 +697,56 @@ LogicalProject(user_id_0=[$0], name=[$1], order_id=[$3], payment_id=[$6], locati
+- LogicalTableScan(table=[[default_catalog, default_database, Shipments]])
== Optimized Physical Plan ==
-Calc(select=[user_id_0, name, order_id, payment_id, location])
-+- MultiJoin(commonJoinKey=[user_id_0], joinTypes=[LEFT, INNER, LEFT], inputUniqueKeys=[(user_id_0), (order_id), (payment_id), noUniqueKey], joinConditions=[=(user_id_0, user_id_1), AND(=(user_id_0, user_id_2), OR(>=(cash, price), <(price, 0))), =(user_id_2, user_id_3)], select=[user_id_0,name,cash,order_id,user_id_1,payment_id,price,user_id_2,location,user_id_3], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id_2, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id_3)])
- :- Exchange(distribution=[hash[user_id_0]])
- : +- ChangelogNormalize(key=[user_id_0])
- : +- Exchange(distribution=[hash[user_id_0]])
- : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id_0, name, cash])
- :- Exchange(distribution=[hash[user_id_1]])
- : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id_1], metadata=[]]], fields=[order_id, user_id_1])
- :- Exchange(distribution=[hash[user_id_2]])
- : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id_2])
- +- Exchange(distribution=[hash[user_id_3]])
- +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id_3])
+Calc(select=[user_id, name, order_id, payment_id, location])
++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id), noUniqueKey], joinConditions=[=(user_id, user_id0), AND(=(user_id, user_id1), OR(>=(cash, price), <(price, 0))), =(user_id1, user_id2)], select=[user_id,name,cash,order_id,user_id0,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)])
+ :- Exchange(distribution=[hash[user_id]])
+ : +- ChangelogNormalize(key=[user_id])
+ : +- Exchange(distribution=[hash[user_id]])
+ : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash])
+ :- Exchange(distribution=[hash[user_id]])
+ : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id])
+ :- Exchange(distribution=[hash[user_id]])
+ : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id])
+ +- Exchange(distribution=[hash[user_id]])
+ +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id])
== Optimized Execution Plan ==
-Calc(select=[user_id_0, name, order_id, payment_id, location])
-+- MultiJoin(commonJoinKey=[user_id_0], joinTypes=[LEFT, INNER, LEFT], inputUniqueKeys=[(user_id_0), (order_id), (payment_id), noUniqueKey], joinConditions=[(user_id_0 = user_id_1), ((user_id_0 = user_id_2) AND ((cash >= price) OR (price < 0))), (user_id_2 = user_id_3)], select=[user_id_0,name,cash,order_id,user_id_1,payment_id,price,user_id_2,location,user_id_3], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id_2, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id_3)])
- :- Exchange(distribution=[hash[user_id_0]])
- : +- ChangelogNormalize(key=[user_id_0])
- : +- Exchange(distribution=[hash[user_id_0]])
- : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id_0, name, cash])
- :- Exchange(distribution=[hash[user_id_1]])
- : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id_1], metadata=[]]], fields=[order_id, user_id_1])
- :- Exchange(distribution=[hash[user_id_2]])
- : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id_2])
- +- Exchange(distribution=[hash[user_id_3]])
- +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id_3])
+Calc(select=[user_id, name, order_id, payment_id, location])
++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id), noUniqueKey], joinConditions=[(user_id = user_id0), ((user_id = user_id1) AND ((cash >= price) OR (price < 0))), (user_id1 = user_id2)], select=[user_id,name,cash,order_id,user_id0,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)])
+ :- Exchange(distribution=[hash[user_id]])
+ : +- ChangelogNormalize(key=[user_id])
+ : +- Exchange(distribution=[hash[user_id]])
+ : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash])
+ :- Exchange(distribution=[hash[user_id]])
+ : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id])
+ :- Exchange(distribution=[hash[user_id]])
+ : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id])
+ +- Exchange(distribution=[hash[user_id]])
+ +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id])
]]>
- = p.price OR p.price < 0) LEFT JOIN Shipments s ON p.user_id_2 = s.user_id_3]]>
+ = p.price OR p.price < 0)
+LEFT JOIN Shipments s
+ ON p.user_id = s.user_id]]>
=($2, $7), <($7, 0)))], joinType=[inner])
: :- LogicalJoin(condition=[=($0, $4)], joinType=[left])
@@ -109,64 +758,101 @@ LogicalProject(user_id_0=[$0], name=[$1], order_id=[$3], payment_id=[$6], locati
=(cash, price), <(price, 0))), =(user_id_2, user_id_3)], select=[user_id_0,name,cash,order_id,user_id_1,payment_id,price,user_id_2,location,user_id_3], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id_2, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id_3)])
- :- Exchange(distribution=[hash[user_id_0]])
- : +- ChangelogNormalize(key=[user_id_0])
- : +- Exchange(distribution=[hash[user_id_0]])
- : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id_0, name, cash])
- :- Exchange(distribution=[hash[user_id_1]])
- : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id_1], metadata=[]]], fields=[order_id, user_id_1])
- :- Exchange(distribution=[hash[user_id_2]])
- : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id_2])
- +- Exchange(distribution=[hash[user_id_3]])
- +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id_3])
+Calc(select=[user_id, name, order_id, payment_id, location])
++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id), noUniqueKey], joinConditions=[=(user_id, user_id0), AND(=(user_id, user_id1), OR(>=(cash, price), <(price, 0))), =(user_id1, user_id2)], select=[user_id,name,cash,order_id,user_id0,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)])
+ :- Exchange(distribution=[hash[user_id]])
+ : +- ChangelogNormalize(key=[user_id])
+ : +- Exchange(distribution=[hash[user_id]])
+ : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash])
+ :- Exchange(distribution=[hash[user_id]])
+ : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id])
+ :- Exchange(distribution=[hash[user_id]])
+ : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id])
+ +- Exchange(distribution=[hash[user_id]])
+ +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id])
]]>
-
+
-
+
-
+
+
+
+
+
+ = FLOOR(p.price) OR p.price < 0)
+LEFT JOIN Shipments s
+ ON p.payment_id = s.location]]>
+
+
+ =(FLOOR($2), FLOOR($8)), <($8, 0)))], joinType=[left])
+ : :- LogicalProject(user_id=[$0], name=[$1], cash=[$2], order_id=[$3], user_id0=[$4], product=[$5], $f6=[UPPER($1)])
+ : : +- LogicalJoin(condition=[=($4, $0)], joinType=[left])
+ : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]])
+ : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]])
+ : +- LogicalProject(payment_id=[$0], price=[$1], user_id=[$2], $f3=[UPPER($0)])
+ : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]])
+ +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]])
+]]>
+
+
+ =(FLOOR(cash), FLOOR(price)), <(price, 0)))], select=[user_id,name,cash,order_id,$f6,payment_id,price,user_id1,$f3], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) $f6, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) $f3)])
+: :- Exchange(distribution=[hash[user_id, $f6]])
+: : +- Calc(select=[user_id, name, cash, order_id, UPPER(name) AS $f6])
+: : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[(user_id), (order_id)], joinConditions=[=(user_id0, user_id)], select=[user_id,name,cash,order_id,user_id0], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0)])
+: : :- Exchange(distribution=[hash[user_id]])
+: : : +- ChangelogNormalize(key=[user_id])
+: : : +- Exchange(distribution=[hash[user_id]])
+: : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash])
+: : +- Exchange(distribution=[hash[user_id]])
+: : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id])
+: +- Exchange(distribution=[hash[user_id, $f3]])
+: +- Calc(select=[payment_id, price, user_id, UPPER(payment_id) AS $f3])
+: +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id])
++- Exchange(distribution=[hash[location]])
+ +- TableSourceScan(table=[[default_catalog, default_database, Shipments, project=[location], metadata=[]]], fields=[location])
]]>
-
+
+
+
+
+
+ 1000 THEN 'High'
+ WHEN p.price > 500 THEN 'Medium'
+ ELSE 'Low'
+ END as price_tier,
+ REGEXP_REPLACE(pd.tags, ',', ' | ') as formatted_tags,
+ TO_TIMESTAMP_LTZ(pd.created_date, 3) as product_created,
+ COALESCE(up.preferred_category, 'None') as user_preference,
+ CASE
+ WHEN up.notification_level = 'HIGH' THEN 'Frequent Updates'
+ WHEN up.notification_level = 'MEDIUM' THEN 'Daily Updates'
+ ELSE 'Weekly Updates'
+ END as notification_frequency
+FROM Users u
+LEFT JOIN Orders o
+ ON u.user_id = o.user_id
+LEFT JOIN Payments p
+ ON u.user_id = p.user_id
+LEFT JOIN ProductDetails pd
+ ON o.product = pd.product_id
+LEFT JOIN UserPreferences up
+ ON u.user_id = up.user_id]]>
+
+
+ ($7, 1000), _UTF-16LE'High':VARCHAR(6) CHARACTER SET "UTF-16LE", >($7, 500), _UTF-16LE'Medium':VARCHAR(6) CHARACTER SET "UTF-16LE", _UTF-16LE'Low':VARCHAR(6) CHARACTER SET "UTF-16LE")], formatted_tags=[REGEXP_REPLACE($13, _UTF-16LE',', _UTF-16LE' | ')], product_created=[TO_TIMESTAMP_LTZ($12, 3)], user_preference=[COALESCE($15, _UTF-16LE'None')], notification_frequency=[CASE(=($16, _UTF-16LE'HIGH'), _UTF-16LE'Frequent Updates':VARCHAR(16) CHARACTER SET "UTF-16LE", =($16, _UTF-16LE'MEDIUM'), _UTF-16LE'Daily Updates':VARCHAR(16) CHARACTER SET "UTF-16LE", _UTF-16LE'Weekly Updates':VARCHAR(16) CHARACTER SET "UTF-16LE")])
++- LogicalJoin(condition=[=($0, $14)], joinType=[left])
+ :- LogicalJoin(condition=[=($5, $9)], joinType=[left])
+ : :- LogicalJoin(condition=[=($0, $8)], joinType=[left])
+ : : :- LogicalJoin(condition=[=($0, $4)], joinType=[left])
+ : : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]])
+ : : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]])
+ : : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]])
+ : +- LogicalTableScan(table=[[default_catalog, default_database, ProductDetails]])
+ +- LogicalTableScan(table=[[default_catalog, default_database, UserPreferences]])
+]]>
+
+
+ (price, 1000), 'High', >(price, 500), 'Medium', 'Low') AS price_tier, REGEXP_REPLACE(tags, ',', ' | ') AS formatted_tags, TO_TIMESTAMP_LTZ(created_date, 3) AS product_created, COALESCE(preferred_category, 'None') AS user_preference, CASE(=(notification_level, 'HIGH'), 'Frequent Updates', =(notification_level, 'MEDIUM'), 'Daily Updates', 'Weekly Updates') AS notification_frequency])
++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, (user_id)], joinConditions=[=(user_id, user_id2)], select=[user_id,name,product,price,description,created_date,tags,user_id2,preferred_category,notification_level], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) product, INTEGER price, VARCHAR(2147483647) description, BIGINT created_date, VARCHAR(2147483647) tags, VARCHAR(2147483647) user_id2, VARCHAR(2147483647) preferred_category, VARCHAR(2147483647) notification_level)])
+ :- Exchange(distribution=[hash[user_id]])
+ : +- Calc(select=[user_id, name, product, price, description, created_date, tags])
+ : +- MultiJoin(commonJoinKey=[product], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, (product_id)], joinConditions=[=(product, product_id)], select=[user_id,name,product,price,product_id,description,created_date,tags], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) product, INTEGER price, VARCHAR(2147483647) product_id, VARCHAR(2147483647) description, BIGINT created_date, VARCHAR(2147483647) tags)])
+ : :- Exchange(distribution=[hash[product]])
+ : : +- Calc(select=[user_id, name, product, price])
+ : : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, LEFT], inputUniqueKeys=[(user_id), noUniqueKey, noUniqueKey], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,name,user_id0,product,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, INTEGER price, VARCHAR(2147483647) user_id1)])
+ : : :- Exchange(distribution=[hash[user_id]])
+ : : : +- ChangelogNormalize(key=[user_id])
+ : : : +- Exchange(distribution=[hash[user_id]])
+ : : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name])
+ : : :- Exchange(distribution=[hash[user_id]])
+ : : : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[user_id, product], metadata=[]]], fields=[user_id, product])
+ : : +- Exchange(distribution=[hash[user_id]])
+ : : +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[price, user_id], metadata=[]]], fields=[price, user_id])
+ : +- Exchange(distribution=[hash[product_id]])
+ : +- ChangelogNormalize(key=[product_id])
+ : +- Exchange(distribution=[hash[product_id]])
+ : +- TableSourceScan(table=[[default_catalog, default_database, ProductDetails, project=[product_id, description, created_date, tags], metadata=[]]], fields=[product_id, description, created_date, tags])
+ +- Exchange(distribution=[hash[user_id]])
+ +- ChangelogNormalize(key=[user_id])
+ +- Exchange(distribution=[hash[user_id]])
+ +- TableSourceScan(table=[[default_catalog, default_database, UserPreferences]], fields=[user_id, preferred_category, notification_level])
+]]>
+
+
+
+
+ = 4 THEN 'High Rating'
+ WHEN r.rating >= 3 THEN 'Medium Rating'
+ ELSE 'Low Rating'
+ END AS rating_category,
+ TIMESTAMPDIFF(DAY, pd.created_date, CURRENT_DATE) AS days_since_created
+FROM Users u
+LEFT JOIN Orders o
+ ON u.user_id = o.user_id
+LEFT JOIN ProductDetails pd
+ ON o.product = pd.product_id
+LEFT JOIN Reviews r
+ ON pd.product_id = r.product_id]]>
+
+
+ =($13, 4), _UTF-16LE'High Rating':VARCHAR(13) CHARACTER SET "UTF-16LE", >=($13, 3), _UTF-16LE'Medium Rating':VARCHAR(13) CHARACTER SET "UTF-16LE", _UTF-16LE'Low Rating':VARCHAR(13) CHARACTER SET "UTF-16LE")], days_since_created=[CAST(/INT(Reinterpret(-(CURRENT_DATE, $10)), 86400000)):INTEGER])
++- LogicalJoin(condition=[=($6, $12)], joinType=[left])
+ :- LogicalJoin(condition=[=($5, $6)], joinType=[left])
+ : :- LogicalJoin(condition=[=($0, $4)], joinType=[left])
+ : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]])
+ : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]])
+ : +- LogicalTableScan(table=[[default_catalog, default_database, ProductDetails]])
+ +- LogicalTableScan(table=[[default_catalog, default_database, Reviews]])
+]]>
+
+
+ =(rating, 4), 'High Rating', >=(rating, 3), 'Medium Rating', 'Low Rating') AS rating_category, CAST(/INT(Reinterpret(-(CURRENT_DATE(), created_date)), 86400000) AS INTEGER) AS days_since_created])
++- MultiJoin(commonJoinKey=[product], joinTypes=[LEFT, LEFT], inputUniqueKeys=[noUniqueKey, (product_id), noUniqueKey], joinConditions=[=(product, product_id), =(product_id, product_id0)], select=[user_id,name,order_id,product,product_id,product_name,price,created_date,product_id0,rating,review_text], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) product, VARCHAR(2147483647) product_id, VARCHAR(2147483647) product_name, DOUBLE price, DATE created_date, VARCHAR(2147483647) product_id0, INTEGER rating, VARCHAR(2147483647) review_text)])
+ :- Exchange(distribution=[hash[product]])
+ : +- Calc(select=[user_id, name, order_id, product])
+ : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[(user_id), (order_id)], joinConditions=[=(user_id, user_id0)], select=[user_id,name,order_id,user_id0,product], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product)])
+ : :- Exchange(distribution=[hash[user_id]])
+ : : +- ChangelogNormalize(key=[user_id])
+ : : +- Exchange(distribution=[hash[user_id]])
+ : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name])
+ : +- Exchange(distribution=[hash[user_id]])
+ : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product])
+ :- Exchange(distribution=[hash[product_id]])
+ : +- ChangelogNormalize(key=[product_id])
+ : +- Exchange(distribution=[hash[product_id]])
+ : +- TableSourceScan(table=[[default_catalog, default_database, ProductDetails, project=[product_id, product_name, price, created_date], metadata=[]]], fields=[product_id, product_name, price, created_date])
+ +- Exchange(distribution=[hash[product_id]])
+ +- Calc(select=[product_id, rating, review_text])
+ +- ChangelogNormalize(key=[review_id])
+ +- Exchange(distribution=[hash[review_id]])
+ +- TableSourceScan(table=[[default_catalog, default_database, Reviews, project=[product_id, rating, review_text, review_id], metadata=[]]], fields=[product_id, rating, review_text, review_id])
]]>
-
+
=($2, -($5, 60000:INTERVAL MINUTE)), <=($2, +($5, 60000:INTERVAL MINUTE)))], joinType=[inner])
- :- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($2, 5000:INTERVAL SECOND)])
+ :- LogicalWatermarkAssigner(rowtime=[$rowtime], watermark=[-($2, 5000:INTERVAL SECOND)])
: +- LogicalTableScan(table=[[default_catalog, default_database, EventTable1]])
- +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($2, 5000:INTERVAL SECOND)])
+ +- LogicalWatermarkAssigner(rowtime=[$rowtime], watermark=[-($2, 5000:INTERVAL SECOND)])
+- LogicalTableScan(table=[[default_catalog, default_database, EventTable2]])
]]>
=(rowtime, -(rowtime0, 60000:INTERVAL MINUTE)), <=(rowtime, +(rowtime0, 60000:INTERVAL MINUTE)))], select=[id, val, rowtime, id0, price, rowtime0])
++- IntervalJoin(joinType=[InnerJoin], windowBounds=[isRowTime=true, leftLowerBound=-60000, leftUpperBound=60000, leftTimeIndex=2, rightTimeIndex=2], where=[AND(=(id, id0), >=($rowtime, -($rowtime0, 60000:INTERVAL MINUTE)), <=($rowtime, +($rowtime0, 60000:INTERVAL MINUTE)))], select=[id, val, $rowtime, id0, price, $rowtime0])
:- Exchange(distribution=[hash[id]])
- : +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 5000:INTERVAL SECOND)])
- : +- TableSourceScan(table=[[default_catalog, default_database, EventTable1]], fields=[id, val, rowtime])
+ : +- WatermarkAssigner(rowtime=[$rowtime], watermark=[-($rowtime, 5000:INTERVAL SECOND)])
+ : +- TableSourceScan(table=[[default_catalog, default_database, EventTable1]], fields=[id, val, $rowtime])
+- Exchange(distribution=[hash[id]])
- +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 5000:INTERVAL SECOND)])
- +- TableSourceScan(table=[[default_catalog, default_database, EventTable2]], fields=[id, price, rowtime])
+ +- WatermarkAssigner(rowtime=[$rowtime], watermark=[-($rowtime, 5000:INTERVAL SECOND)])
+ +- TableSourceScan(table=[[default_catalog, default_database, EventTable2]], fields=[id, price, $rowtime])
+]]>
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+ 100) AS u
+JOIN (SELECT user_id, order_id, product FROM Orders WHERE product IS NOT NULL) AS o
+ ON u.user_id = o.user_id
+LEFT JOIN (SELECT user_id, price FROM Payments WHERE price > 50) AS p
+ ON u.user_id = p.user_id
+LEFT JOIN (SELECT user_id, location FROM Shipments WHERE location IS NOT NULL) AS s
+ ON u.user_id = s.user_id]]>
+
+
+ ($2, 100)])
+ : : : +- LogicalTableScan(table=[[default_catalog, default_database, Users]])
+ : : +- LogicalProject(user_id=[$1], order_id=[$0], product=[$2])
+ : : +- LogicalFilter(condition=[IS NOT NULL($2)])
+ : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]])
+ : +- LogicalProject(user_id=[$2], price=[$1])
+ : +- LogicalFilter(condition=[>($1, 50)])
+ : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]])
+ +- LogicalProject(user_id=[$1], location=[$0])
+ +- LogicalFilter(condition=[IS NOT NULL($0)])
+ +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]])
+]]>
+
+
+ (cash, 100)])
+ : +- Exchange(distribution=[hash[user_id]])
+ : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[], project=[user_id, cash], metadata=[]]], fields=[user_id, cash])
+ :- Exchange(distribution=[hash[user_id]])
+ : +- Calc(select=[user_id, order_id, product], where=[IS NOT NULL(product)])
+ : +- TableSourceScan(table=[[default_catalog, default_database, Orders, filter=[]]], fields=[order_id, user_id, product])
+ :- Exchange(distribution=[hash[user_id]])
+ : +- Calc(select=[user_id, price], where=[>(price, 50)])
+ : +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[], project=[price, user_id], metadata=[]]], fields=[price, user_id])
+ +- Exchange(distribution=[hash[user_id]])
+ +- Calc(select=[user_id, location], where=[IS NOT NULL(location)])
+ +- TableSourceScan(table=[[default_catalog, default_database, Shipments, filter=[]]], fields=[location, user_id])
+]]>
+
+
+
+
+ 100
+ ) AS p
+ ON o.user_id = p.user_id
+) AS op
+ON u.user_id = op.user_id]]>
+
+
+ ($1, 100)])
+ +- LogicalTableScan(table=[[default_catalog, default_database, Payments]])
+]]>
+
+
+ (price, 100)])
+ +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id])
]]>
@@ -288,7 +1364,26 @@ Calc(select=[a, DATE_FORMAT(CURRENT_TIMESTAMP(), _UTF-16LE'yyMMdd') AS day], cha
- = 0 JOIN AddressPK a ON u.user_id = a.user_id AND a.location IS NOT NULL]]>
+ = 0
+JOIN AddressPK a
+ ON u.user_id = a.user_id
+ AND a.location IS NOT NULL]]>
-
+
-
+
-
+
-
+