This is an automated email from the ASF dual-hosted git repository. chesnay pushed a commit to branch release-1.18 in repository https://gitbox.apache.org/repos/asf/flink.git
The following commit(s) were added to refs/heads/release-1.18 by this push: new 39ed3cf279d [FLINK-34496] Break circular dependency in static initialization 39ed3cf279d is described below commit 39ed3cf279d61e4472e1c30a17927992236df467 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/nodes/exec/serde/JsonSerdeUtil.java | 15 --------------- .../planner/plan/utils/ExecNodeMetadataUtil.java | 19 +++++++++++++++++-- .../planner/plan/utils/ExecNodeMetadataUtilTest.java | 3 +-- 3 files changed, 18 insertions(+), 19 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/JsonSerdeUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/JsonSerdeUtil.java index dffd134e97a..2d34c710a76 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/JsonSerdeUtil.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/JsonSerdeUtil.java @@ -42,7 +42,6 @@ import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.jackson.JacksonMapperFactory; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.ObjectCodec; @@ -68,26 +67,12 @@ import org.apache.calcite.rex.RexNode; import org.apache.calcite.rex.RexWindowBound; import java.io.IOException; -import java.lang.annotation.Annotation; -import java.lang.reflect.Constructor; import java.util.Optional; /** A utility class that provide abilities for JSON serialization and deserialization. */ @Internal public class JsonSerdeUtil { - /** Return true if the given class's constructors have @JsonCreator annotation, else false. */ - public static boolean hasJsonCreatorAnnotation(Class<?> clazz) { - for (Constructor<?> constructor : clazz.getDeclaredConstructors()) { - for (Annotation annotation : constructor.getAnnotations()) { - if (annotation instanceof JsonCreator) { - return true; - } - } - } - return false; - } - /** * Object mapper shared instance to serialize and deserialize the plan. Note that creating and * copying of object mappers is expensive and should be avoided. 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 5c467f29583..84ce3fbf2a9 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.StreamExecCalc; import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecChangelogNormalize; import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCorrelate; @@ -78,8 +77,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; @@ -230,7 +233,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 " @@ -364,4 +367,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 9301f79e725..bc43ee09925 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 @@ public 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); }