You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/01/04 15:10:52 UTC

[GitHub] [flink] slinkydeveloper opened a new pull request #18264: [FLINK-25518][table-planner] Harden JSON utilities for serde of the persisted plan

slinkydeveloper opened a new pull request #18264:
URL: https://github.com/apache/flink/pull/18264


   ## What is the purpose of the change
   
   This PR contains several fixes to harden our json utilities and ser/de for the persisted plan
   
   ## Brief change log
   
   * Fix `JsonSerdeUtil` to create just one instance of `ObjectMapper`, as its allocation tends to be heavy and error prone. This also allows us to test in the unit tests the same `ObjectMapper` that is going to be used in the real environment (eventually testing conflicts of any kind)
   * Removed `FlinkDeserializationContext` as `SerdeContext` is now passed through the Jackson built-in `DeserializationContext`. Now `SerdeContext` is also passed to the serialization context.
   * Removed all usages of `ObjectMapper` within the `Deserializer`s, replacing them with proper usage of `DeserializationContext#readValue`.
   
   ## Verifying this change
   
   This change is a trivial rework without additional test coverage.
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): no
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: no
     - The serializers: no
     - The runtime per-record code paths (performance sensitive): no
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: no
     - The S3 file system connector: no
   
   ## Documentation
   
     - Does this pull request introduce a new feature? no
     - If yes, how is the feature documented? not applicable


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #18264: [FLINK-25518][table-planner] Harden JSON utilities for serde of the persisted plan

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18264:
URL: https://github.com/apache/flink/pull/18264#issuecomment-1004894702


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28940",
       "triggerID" : "eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8a3bf88ea132dcebec6c4d8026dfaaa593ccb3cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28968",
       "triggerID" : "8a3bf88ea132dcebec6c4d8026dfaaa593ccb3cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac9d8edbd2ed771be272b4ded41f0e8f20fb635f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28986",
       "triggerID" : "ac9d8edbd2ed771be272b4ded41f0e8f20fb635f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5f24958e8fb012149cf0b381569d3c08af023e",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29032",
       "triggerID" : "dd5f24958e8fb012149cf0b381569d3c08af023e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * dd5f24958e8fb012149cf0b381569d3c08af023e Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29032) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #18264: [FLINK-25518][table-planner] Harden JSON utilities for serde of the persisted plan

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18264:
URL: https://github.com/apache/flink/pull/18264#issuecomment-1004894702


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28940",
       "triggerID" : "eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8a3bf88ea132dcebec6c4d8026dfaaa593ccb3cb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28968",
       "triggerID" : "8a3bf88ea132dcebec6c4d8026dfaaa593ccb3cb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8a3bf88ea132dcebec6c4d8026dfaaa593ccb3cb Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28968) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #18264: [FLINK-25518][table-planner] Harden JSON utilities for serde of the persisted plan

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18264:
URL: https://github.com/apache/flink/pull/18264#issuecomment-1004894702


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28940",
       "triggerID" : "eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8a3bf88ea132dcebec6c4d8026dfaaa593ccb3cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28968",
       "triggerID" : "8a3bf88ea132dcebec6c4d8026dfaaa593ccb3cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac9d8edbd2ed771be272b4ded41f0e8f20fb635f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28986",
       "triggerID" : "ac9d8edbd2ed771be272b4ded41f0e8f20fb635f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ac9d8edbd2ed771be272b4ded41f0e8f20fb635f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28986) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #18264: [FLINK-25518][table-planner] Harden JSON utilities for serde of the persisted plan

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18264:
URL: https://github.com/apache/flink/pull/18264#issuecomment-1004894702


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28940",
       "triggerID" : "eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8a3bf88ea132dcebec6c4d8026dfaaa593ccb3cb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28968",
       "triggerID" : "8a3bf88ea132dcebec6c4d8026dfaaa593ccb3cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac9d8edbd2ed771be272b4ded41f0e8f20fb635f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ac9d8edbd2ed771be272b4ded41f0e8f20fb635f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8a3bf88ea132dcebec6c4d8026dfaaa593ccb3cb Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28968) 
   * ac9d8edbd2ed771be272b4ded41f0e8f20fb635f UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #18264: [FLINK-25518][table-planner] Harden JSON utilities for serde of the persisted plan

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #18264:
URL: https://github.com/apache/flink/pull/18264#issuecomment-1004896631


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358 (Tue Jan 04 15:15:50 UTC 2022)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #18264: [FLINK-25518][table-planner] Harden JSON utilities for serde of the persisted plan

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18264:
URL: https://github.com/apache/flink/pull/18264#issuecomment-1004894702


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28940",
       "triggerID" : "eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8a3bf88ea132dcebec6c4d8026dfaaa593ccb3cb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28968",
       "triggerID" : "8a3bf88ea132dcebec6c4d8026dfaaa593ccb3cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac9d8edbd2ed771be272b4ded41f0e8f20fb635f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28986",
       "triggerID" : "ac9d8edbd2ed771be272b4ded41f0e8f20fb635f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8a3bf88ea132dcebec6c4d8026dfaaa593ccb3cb Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28968) 
   * ac9d8edbd2ed771be272b4ded41f0e8f20fb635f Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28986) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #18264: [FLINK-25518][table-planner] Harden JSON utilities for serde of the persisted plan

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18264:
URL: https://github.com/apache/flink/pull/18264#issuecomment-1004894702


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28940",
       "triggerID" : "eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8a3bf88ea132dcebec6c4d8026dfaaa593ccb3cb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8a3bf88ea132dcebec6c4d8026dfaaa593ccb3cb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28940) 
   * 8a3bf88ea132dcebec6c4d8026dfaaa593ccb3cb UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] twalthr commented on a change in pull request #18264: [FLINK-25518][table-planner] Harden JSON utilities for serde of the persisted plan

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #18264:
URL: https://github.com/apache/flink/pull/18264#discussion_r779434345



##########
File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RelDataTypeJsonDeserializer.java
##########
@@ -176,14 +173,32 @@ private RelDataType deserialize(JsonNode jsonNode, FlinkDeserializationContext c
                                 : null;
                 final RelDataType type;
                 if (sqlTypeName == SqlTypeName.ARRAY) {
-                    RelDataType elementType = deserialize(objectNode.get(FIELD_NAME_ELEMENT), ctx);
+                    RelDataType elementType =
+                            deserialize(

Review comment:
       This makes the code quite verbose. Now `.traverse(jsonParser.getCodec())` is everywhere in the code. Isn't it possible to restore the original code at this location?

##########
File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/JsonSerdeUtil.java
##########
@@ -42,22 +60,83 @@ public static boolean hasJsonCreatorAnnotation(Class<?> clazz) {
         return false;
     }
 
-    /** Create an {@link ObjectMapper} which DeserializationContext wraps a {@link SerdeContext}. */
-    public static ObjectMapper createObjectMapper(SerdeContext serdeCtx) {
-        FlinkDeserializationContext ctx =
-                new FlinkDeserializationContext(
-                        new DefaultDeserializationContext.Impl(BeanDeserializerFactory.instance),
-                        serdeCtx);
-        ObjectMapper mapper =
-                new ObjectMapper(
-                        null, // JsonFactory
-                        null, // DefaultSerializerProvider
-                        ctx);
-        mapper.setTypeFactory(
-                mapper.getTypeFactory().withClassLoader(JsonSerdeUtil.class.getClassLoader()));
-        mapper.configure(MapperFeature.USE_GETTERS_AS_SETTERS, false);
-        ctx.setObjectMapper(mapper);
-        return mapper;
+    // Object mapper shared instance to serialize and deserialize the plan
+    private static final ObjectMapper OBJECT_MAPPER_INSTANCE;
+
+    static {
+        OBJECT_MAPPER_INSTANCE = new ObjectMapper();
+
+        OBJECT_MAPPER_INSTANCE.setTypeFactory(
+                // Make sure to register the classloader of the planner
+                OBJECT_MAPPER_INSTANCE
+                        .getTypeFactory()
+                        .withClassLoader(JsonSerdeUtil.class.getClassLoader()));
+        OBJECT_MAPPER_INSTANCE.configure(MapperFeature.USE_GETTERS_AS_SETTERS, false);
+        OBJECT_MAPPER_INSTANCE.registerModule(createFlinkTableJacksonModule());
+    }
+
+    /** Get the {@link ObjectMapper} instance. */
+    public static ObjectMapper getObjectMapper() {
+        return OBJECT_MAPPER_INSTANCE;
+    }
+
+    public static ObjectReader createObjectReader(SerdeContext serdeContext) {
+        return OBJECT_MAPPER_INSTANCE
+                .reader()
+                .withAttribute(SerdeContext.SERDE_CONTEXT_KEY, serdeContext);
+    }
+
+    public static ObjectWriter createObjectWriter(SerdeContext serdeContext) {
+        return OBJECT_MAPPER_INSTANCE
+                .writer()
+                .withAttribute(SerdeContext.SERDE_CONTEXT_KEY, serdeContext);
+    }
+
+    private static Module createFlinkTableJacksonModule() {
+        final SimpleModule module = new SimpleModule("Flink table types");

Review comment:
       nit: `types` is used at too many locations, call it `Flink table module`

##########
File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LogicalWindowJsonDeserializer.java
##########
@@ -68,24 +66,25 @@ public LogicalWindowJsonDeserializer() {
     @Override
     public LogicalWindow deserialize(
             JsonParser jsonParser, DeserializationContext deserializationContext)
-            throws IOException, JsonProcessingException {
-        FlinkDeserializationContext flinkDeserializationContext =
-                (FlinkDeserializationContext) deserializationContext;
-        ObjectMapper mapper = flinkDeserializationContext.getObjectMapper();
+            throws IOException {
         JsonNode jsonNode = jsonParser.readValueAsTree();
         String kind = jsonNode.get(FIELD_NAME_KIND).asText().toUpperCase();
         WindowReference alias =
-                mapper.readValue(jsonNode.get(FIELD_NAME_ALIAS).toString(), WindowReference.class);
+                jsonParser

Review comment:
       What is the difference between this and
   ```
   deserializationContext.readValue(
                                       jsonNode.get(FIELD_NAME_SIZE).traverse(jsonParser.getCodec()),
                                       Duration.class);
   ```
   Why is this not consistent?

##########
File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RelDataTypeJsonDeserializer.java
##########
@@ -107,17 +103,18 @@ private RelDataType deserialize(JsonNode jsonNode, FlinkDeserializationContext c
             } else if (objectNode.has(FIELD_NAME_STRUCTURED_TYPE)) {
                 JsonNode structuredTypeNode = objectNode.get(FIELD_NAME_STRUCTURED_TYPE);
                 LogicalType structuredType =
-                        ctx.getObjectMapper()
-                                .readValue(structuredTypeNode.toPrettyString(), LogicalType.class);
+                        ctx.readValue(
+                                structuredTypeNode.traverse(jsonParser.getCodec()),
+                                LogicalType.class);
                 checkArgument(structuredType instanceof StructuredType);
-                return ctx.getSerdeContext()
-                        .getTypeFactory()
-                        .createFieldTypeFromLogicalType(structuredType);
+                return serdeContext.getTypeFactory().createFieldTypeFromLogicalType(structuredType);
             } else if (objectNode.has(FIELD_NAME_STRUCT_KIND)) {
                 ArrayNode arrayNode = (ArrayNode) objectNode.get(FIELD_NAME_FIELDS);
                 RelDataTypeFactory.Builder builder = typeFactory.builder();
                 for (JsonNode node : arrayNode) {
-                    builder.add(node.get(FIELD_NAME_FILED_NAME).asText(), deserialize(node, ctx));
+                    builder.add(
+                            node.get(FIELD_NAME_FILED_NAME).asText(),
+                            deserialize(node.traverse(jsonParser.getCodec()), ctx));

Review comment:
       let's reintroduce a method to avoid calling 
   
   ```
   deserialize(node.traverse(jsonParser.getCodec()), ctx)
   ```
   repeatedly. in general it would be good if every deserializer has a public `deserialize(JsonNode, SerdeContext)` method that can be called by other deserializers directly without having to go through Jackson one more time.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #18264: [FLINK-25518][table-planner] Harden JSON utilities for serde of the persisted plan

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18264:
URL: https://github.com/apache/flink/pull/18264#issuecomment-1004894702


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28940",
       "triggerID" : "eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8a3bf88ea132dcebec6c4d8026dfaaa593ccb3cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28968",
       "triggerID" : "8a3bf88ea132dcebec6c4d8026dfaaa593ccb3cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac9d8edbd2ed771be272b4ded41f0e8f20fb635f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28986",
       "triggerID" : "ac9d8edbd2ed771be272b4ded41f0e8f20fb635f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5f24958e8fb012149cf0b381569d3c08af023e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29032",
       "triggerID" : "dd5f24958e8fb012149cf0b381569d3c08af023e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ac9d8edbd2ed771be272b4ded41f0e8f20fb635f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28986) 
   * dd5f24958e8fb012149cf0b381569d3c08af023e Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29032) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #18264: [FLINK-25518][table-planner] Harden JSON utilities for serde of the persisted plan

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18264:
URL: https://github.com/apache/flink/pull/18264#issuecomment-1004894702


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28940",
       "triggerID" : "eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28940) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] slinkydeveloper commented on a change in pull request #18264: [FLINK-25518][table-planner] Harden JSON utilities for serde of the persisted plan

Posted by GitBox <gi...@apache.org>.
slinkydeveloper commented on a change in pull request #18264:
URL: https://github.com/apache/flink/pull/18264#discussion_r779457785



##########
File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LogicalWindowJsonDeserializer.java
##########
@@ -68,24 +66,25 @@ public LogicalWindowJsonDeserializer() {
     @Override
     public LogicalWindow deserialize(
             JsonParser jsonParser, DeserializationContext deserializationContext)
-            throws IOException, JsonProcessingException {
-        FlinkDeserializationContext flinkDeserializationContext =
-                (FlinkDeserializationContext) deserializationContext;
-        ObjectMapper mapper = flinkDeserializationContext.getObjectMapper();
+            throws IOException {
         JsonNode jsonNode = jsonParser.readValueAsTree();
         String kind = jsonNode.get(FIELD_NAME_KIND).asText().toUpperCase();
         WindowReference alias =
-                mapper.readValue(jsonNode.get(FIELD_NAME_ALIAS).toString(), WindowReference.class);
+                jsonParser

Review comment:
       Good question. For some reason, only in this specific case, I'm forced to use the codec directly. I feel like this has something to do with the fact that the class `WindowReference` is in scala. I also havent't spent much time investigating, as my understanding is that we don't need to serialize this class anymore once we disable `GroupWindow`s.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #18264: [FLINK-25518][table-planner] Harden JSON utilities for serde of the persisted plan

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18264:
URL: https://github.com/apache/flink/pull/18264#issuecomment-1004894702


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28940",
       "triggerID" : "eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8a3bf88ea132dcebec6c4d8026dfaaa593ccb3cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28968",
       "triggerID" : "8a3bf88ea132dcebec6c4d8026dfaaa593ccb3cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac9d8edbd2ed771be272b4ded41f0e8f20fb635f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28986",
       "triggerID" : "ac9d8edbd2ed771be272b4ded41f0e8f20fb635f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dd5f24958e8fb012149cf0b381569d3c08af023e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "dd5f24958e8fb012149cf0b381569d3c08af023e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ac9d8edbd2ed771be272b4ded41f0e8f20fb635f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28986) 
   * dd5f24958e8fb012149cf0b381569d3c08af023e UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] twalthr closed pull request #18264: [FLINK-25518][table-planner] Harden JSON utilities for serde of the persisted plan

Posted by GitBox <gi...@apache.org>.
twalthr closed pull request #18264:
URL: https://github.com/apache/flink/pull/18264


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #18264: [FLINK-25518][table-planner] Harden JSON utilities for serde of the persisted plan

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18264:
URL: https://github.com/apache/flink/pull/18264#issuecomment-1004894702


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28940",
       "triggerID" : "eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8a3bf88ea132dcebec6c4d8026dfaaa593ccb3cb",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28968",
       "triggerID" : "8a3bf88ea132dcebec6c4d8026dfaaa593ccb3cb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28940) 
   * 8a3bf88ea132dcebec6c4d8026dfaaa593ccb3cb Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28968) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #18264: [FLINK-25518][table-planner] Harden JSON utilities for serde of the persisted plan

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18264:
URL: https://github.com/apache/flink/pull/18264#issuecomment-1004894702


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28940",
       "triggerID" : "eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=28940) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #18264: [FLINK-25518][table-planner] Harden JSON utilities for serde of the persisted plan

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #18264:
URL: https://github.com/apache/flink/pull/18264#issuecomment-1004894702


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * eb5e1b5f6e345c1cf1ca64a429b1426ef71cb358 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org