This is an automated email from the ASF dual-hosted git repository.

snuyanzin pushed a commit to branch release-2.2
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/release-2.2 by this push:
     new ce68d4a9622 [FLINK-39547][table] Fix table-planner class loading order
ce68d4a9622 is described below

commit ce68d4a96221c518b8a3e8e3ad8c6a6856655033
Author: Piotr Przybylski <[email protected]>
AuthorDate: Wed Apr 29 09:27:18 2026 +0200

    [FLINK-39547][table] Fix table-planner class loading order
---
 .../flink-end-to-end-tests-table-api/pom.xml       |  7 ++++++
 .../test/join/JoinWithCustomTypeExampleTest.java   | 29 ++++++++++++++++++++++
 .../join/adaptive/AdaptiveJoinOperatorFactory.java |  3 ++-
 3 files changed, 38 insertions(+), 1 deletion(-)

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 2e9afd9f7de..9bc236942cf 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.
                        <scope>provided</scope>
                </dependency>
 
+               <!-- Incompatible Scala library in user code shouldn't break 
table-planner -->
+               <dependency>
+                       <groupId>org.scala-lang</groupId>
+                       <artifactId>scala-library</artifactId>
+                       <version>2.13.18</version>
+               </dependency>
+
                <!-- Flink Table API -->
                <dependency>
                        <groupId>org.apache.flink</groupId>
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 00000000000..0e51c0dbc5f
--- /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-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/adaptive/AdaptiveJoinOperatorFactory.java
 
b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/adaptive/AdaptiveJoinOperatorFactory.java
index 7246f3718c0..890060082ae 100644
--- 
a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/adaptive/AdaptiveJoinOperatorFactory.java
+++ 
b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/adaptive/AdaptiveJoinOperatorFactory.java
@@ -102,9 +102,10 @@ public class AdaptiveJoinOperatorFactory<OUT> extends 
AbstractStreamOperatorFact
         ClassLoader classLoader =
                 plannerModule == null
                         ? userClassLoader
-                        : FlinkUserCodeClassLoaders.parentFirst(
+                        : FlinkUserCodeClassLoaders.childFirst(
                                 
plannerModule.getSubmoduleClassLoader().getURLs(),
                                 userClassLoader,
+                                new String[0],
                                 NOOP_EXCEPTION_HANDLER,
                                 checkClassLoaderLeak);
 

Reply via email to