godfreyhe commented on a change in pull request #18274:
URL: https://github.com/apache/flink/pull/18274#discussion_r781013103



##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DataTypeJsonDeserializer.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.planner.plan.nodes.exec.serde;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.types.CollectionDataType;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.FieldsDataType;
+import org.apache.flink.table.types.KeyValueDataType;
+import org.apache.flink.table.types.extraction.ExtractionUtils;
+import org.apache.flink.table.types.logical.DistinctType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeChecks;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser;
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext;
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.StdDeserializer;
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static 
org.apache.flink.table.planner.plan.nodes.exec.serde.DataTypeJsonSerializer.FIELD_NAME_CONVERSION_CLASS;
+import static 
org.apache.flink.table.planner.plan.nodes.exec.serde.DataTypeJsonSerializer.FIELD_NAME_ELEMENT_CLASS;
+import static 
org.apache.flink.table.planner.plan.nodes.exec.serde.DataTypeJsonSerializer.FIELD_NAME_FIELDS;
+import static 
org.apache.flink.table.planner.plan.nodes.exec.serde.DataTypeJsonSerializer.FIELD_NAME_FIELD_NAME;
+import static 
org.apache.flink.table.planner.plan.nodes.exec.serde.DataTypeJsonSerializer.FIELD_NAME_KEY_CLASS;
+import static 
org.apache.flink.table.planner.plan.nodes.exec.serde.DataTypeJsonSerializer.FIELD_NAME_TYPE;
+import static 
org.apache.flink.table.planner.plan.nodes.exec.serde.DataTypeJsonSerializer.FIELD_NAME_VALUE_CLASS;
+
+/**
+ * JSON deserializer for {@link DataType}.
+ *
+ * @see DataTypeJsonSerializer for the reverse operation
+ */
+@Internal
+public class DataTypeJsonDeserializer extends StdDeserializer<DataType> {
+
+    public DataTypeJsonDeserializer() {
+        super(DataType.class);
+    }
+
+    @Override
+    public DataType deserialize(JsonParser jsonParser, DeserializationContext 
ctx)
+            throws IOException {
+        final JsonNode dataTypeNode = jsonParser.readValueAsTree();
+        final SerdeContext serdeContext = SerdeContext.get(ctx);
+        return deserialize(dataTypeNode, serdeContext);
+    }
+
+    public static DataType deserialize(JsonNode dataTypeNode, SerdeContext 
serdeContext) {
+        if (dataTypeNode.isTextual()) {
+            return deserializeWithInternalClass(dataTypeNode, serdeContext);
+        } else {
+            return deserializeWithExternalClass(dataTypeNode, serdeContext);
+        }
+    }
+
+    private static DataType deserializeWithInternalClass(
+            JsonNode logicalTypeNode, SerdeContext serdeContext) {
+        final LogicalType logicalType =
+                LogicalTypeJsonDeserializer.deserialize(logicalTypeNode, 
serdeContext);
+        return DataTypes.of(logicalType).toInternal();
+    }
+
+    private static DataType deserializeWithExternalClass(
+            JsonNode dataTypeNode, SerdeContext serdeContext) {
+        final LogicalType logicalType =
+                LogicalTypeJsonDeserializer.deserialize(
+                        dataTypeNode.get(FIELD_NAME_TYPE), serdeContext);
+        return deserializeClass(logicalType, dataTypeNode, serdeContext);
+    }
+
+    private static DataType deserializeClass(
+            LogicalType logicalType, @Nullable JsonNode classNode, 
SerdeContext serdeContext) {

Review comment:
       nit: rename `classNode` to `parentNode` ? 

##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DataTypeJsonSerializer.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.planner.plan.nodes.exec.serde;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.DataTypes.Field;
+import org.apache.flink.table.types.CollectionDataType;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.KeyValueDataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.utils.DataTypeUtils;
+
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator;
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.SerializerProvider;
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ser.std.StdSerializer;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * JSON serializer for {@link LogicalType}.

Review comment:
       `JSON serializer for {@link DataType}.`




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to