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

chesnay pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
     new 2d78c102112 [FLINK-34496] Break circular dependency in static 
initialization
2d78c102112 is described below

commit 2d78c10211272a264712e86192c4dfc59c6a5521
Author: Chesnay Schepler <ches...@apache.org>
AuthorDate: Fri Feb 23 10:26:45 2024 +0100

    [FLINK-34496] Break circular dependency in static initialization
---
 .../planner/plan/utils/ExecNodeMetadataUtil.java      | 19 +++++++++++++++++--
 .../planner/plan/utils/ExecNodeMetadataUtilTest.java  |  3 +--
 2 files changed, 18 insertions(+), 4 deletions(-)

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 efded08a82e..a7e1d2840ad 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
@@ -28,7 +28,6 @@ import org.apache.flink.table.api.config.TableConfigOptions;
 import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
 import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
 import org.apache.flink.table.planner.plan.nodes.exec.MultipleExecNodeMetadata;
-import org.apache.flink.table.planner.plan.nodes.exec.serde.JsonSerdeUtil;
 import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecAsyncCalc;
 import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc;
 import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecChangelogNormalize;
@@ -79,8 +78,12 @@ import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowJoi
 import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowRank;
 import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowTableFunction;
 
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+
 import javax.annotation.Nullable;
 
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Constructor;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Comparator;
@@ -232,7 +235,7 @@ public final class ExecNodeMetadataUtil {
     }
 
     private static void addToLookupMap(Class<? extends ExecNode<?>> 
execNodeClass) {
-        if (!JsonSerdeUtil.hasJsonCreatorAnnotation(execNodeClass)) {
+        if (!hasJsonCreatorAnnotation(execNodeClass)) {
             throw new IllegalStateException(
                     String.format(
                             "ExecNode: %s does not implement @JsonCreator 
annotation on "
@@ -366,4 +369,16 @@ public final class ExecNodeMetadataUtil {
             return Objects.hash(name, version);
         }
     }
+
+    /** Return true if the given class's constructors have @JsonCreator 
annotation, else false. */
+    static boolean hasJsonCreatorAnnotation(Class<?> clazz) {
+        for (Constructor<?> constructor : clazz.getDeclaredConstructors()) {
+            for (Annotation annotation : constructor.getAnnotations()) {
+                if (annotation instanceof JsonCreator) {
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
 }
diff --git 
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtilTest.java
 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtilTest.java
index 323a037f310..3e2a8c3275b 100644
--- 
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtilTest.java
+++ 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtilTest.java
@@ -32,7 +32,6 @@ import 
org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
 import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.MultipleExecNodeMetadata;
-import org.apache.flink.table.planner.plan.nodes.exec.serde.JsonSerdeUtil;
 import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecNode;
 import org.apache.flink.table.types.logical.LogicalType;
 
@@ -239,7 +238,7 @@ class ExecNodeMetadataUtilTest {
         List<Class<? extends ExecNode<?>>> 
classesWithJsonCreatorInUnsupportedList =
                 new ArrayList<>();
         for (Class<? extends ExecNode<?>> clazz : subClasses) {
-            boolean hasJsonCreator = 
JsonSerdeUtil.hasJsonCreatorAnnotation(clazz);
+            boolean hasJsonCreator = 
ExecNodeMetadataUtil.hasJsonCreatorAnnotation(clazz);
             if (hasJsonCreator && 
UNSUPPORTED_JSON_SERDE_CLASSES.contains(clazz)) {
                 classesWithJsonCreatorInUnsupportedList.add(clazz);
             }

Reply via email to