Skip to content

Commit

Permalink
[FLINK-34496] Break circular dependency in static initialization
Browse files Browse the repository at this point in the history
  • Loading branch information
zentol committed Feb 23, 2024
1 parent f21ee01 commit dd77ee5
Show file tree
Hide file tree
Showing 3 changed files with 18 additions and 19 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,6 @@
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;
Expand All @@ -69,26 +68,12 @@
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.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,6 @@
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;
Expand Down Expand Up @@ -79,8 +78,12 @@
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;
Expand Down Expand Up @@ -232,7 +235,7 @@ public static <T extends ExecNode<?>> List<ExecNodeMetadata> extractMetadataFrom
}

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 "
Expand Down Expand Up @@ -366,4 +369,16 @@ public int hashCode() {
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;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,6 @@
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;

Expand Down Expand Up @@ -239,7 +238,7 @@ void testStreamExecNodeJsonSerdeCoverage() {
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);
}
Expand Down

0 comments on commit dd77ee5

Please sign in to comment.