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



##########
File path: 
flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/utils/LogicalTypeDataTypeConverter.java
##########
@@ -201,9 +200,29 @@ public DataType visit(RowType rowType) {
 
         @Override
         public DataType visit(DistinctType distinctType) {
-            return new FieldsDataType(
-                    distinctType,
-                    
Collections.singletonList(distinctType.getSourceType().accept(this)));
+            final DataType sourceDataType = 
distinctType.getSourceType().accept(this);
+            if (sourceDataType instanceof AtomicDataType) {
+                return new AtomicDataType(distinctType, 
sourceDataType.getConversionClass());
+            } else if (sourceDataType instanceof CollectionDataType) {
+                final CollectionDataType collectionDataType = 
(CollectionDataType) sourceDataType;
+                return new CollectionDataType(
+                        distinctType,
+                        collectionDataType.getConversionClass(),
+                        collectionDataType.getElementDataType());
+            } else if (sourceDataType instanceof KeyValueDataType) {
+                final KeyValueDataType keyValueDataType = (KeyValueDataType) 
sourceDataType;
+                return new KeyValueDataType(
+                        distinctType,
+                        keyValueDataType.getConversionClass(),
+                        keyValueDataType.getKeyDataType(),
+                        keyValueDataType.getValueDataType());
+            } else if (sourceDataType instanceof FieldsDataType) {
+                return new FieldsDataType(
+                        distinctType,
+                        sourceDataType.getConversionClass(),
+                        sourceDataType.getChildren());
+            }
+            throw new IllegalStateException("Unexpected data type instance.");

Review comment:
       Ok, that's a detail I didn't knew about our type system, makes sense

##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/FlinkSerializationProvider.java
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.shaded.jackson2.com.fasterxml.jackson.databind.SerializationConfig;
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.SerializerProvider;
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ser.DefaultSerializerProvider;
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ser.SerializerFactory;
+
+/** {@link SerializerProvider} that offers a Flink-specific {@link 
SerdeContext}. */
+class FlinkSerializationProvider extends DefaultSerializerProvider {

Review comment:
       You don't need this one anymore

##########
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) {
+        if (classNode == null) {
+            return DataTypes.of(logicalType).toInternal();
+        }
+
+        final DataType dataType;
+        switch (logicalType.getTypeRoot()) {
+            case ARRAY:
+            case MULTISET:
+                final DataType elementDataType =
+                        deserializeClass(
+                                logicalType.getChildren().get(0),
+                                classNode.get(FIELD_NAME_ELEMENT_CLASS),
+                                serdeContext);
+                dataType = new CollectionDataType(logicalType, 
elementDataType);
+                break;
+
+            case MAP:
+                final MapType mapType = (MapType) logicalType;
+                final DataType keyDataType =
+                        deserializeClass(
+                                mapType.getKeyType(),
+                                classNode.get(FIELD_NAME_KEY_CLASS),
+                                serdeContext);
+                final DataType valueDataType =
+                        deserializeClass(
+                                mapType.getValueType(),
+                                classNode.get(FIELD_NAME_VALUE_CLASS),
+                                serdeContext);
+                dataType = new KeyValueDataType(mapType, keyDataType, 
valueDataType);
+                break;
+
+            case ROW:
+            case STRUCTURED_TYPE:
+                final List<String> fieldNames = 
LogicalTypeChecks.getFieldNames(logicalType);
+                final List<LogicalType> fieldTypes = 
LogicalTypeChecks.getFieldTypes(logicalType);
+
+                final ArrayNode fieldNodes = (ArrayNode) 
classNode.get(FIELD_NAME_FIELDS);
+                final Map<String, JsonNode> fieldNodesByName = new HashMap<>();
+                if (fieldNodes != null) {
+                    fieldNodes.forEach(
+                            fieldNode ->
+                                    fieldNodesByName.put(
+                                            
fieldNode.get(FIELD_NAME_FIELD_NAME).asText(),
+                                            fieldNode));
+                }
+
+                final List<DataType> fieldDataTypes =
+                        IntStream.range(0, fieldNames.size())
+                                .mapToObj(
+                                        i -> {
+                                            final String fieldName = 
fieldNames.get(i);
+                                            final LogicalType fieldType = 
fieldTypes.get(i);
+                                            return deserializeClass(
+                                                    fieldType,
+                                                    
fieldNodesByName.get(fieldName),
+                                                    serdeContext);
+                                        })
+                                .collect(Collectors.toList());
+
+                dataType = new FieldsDataType(logicalType, fieldDataTypes);
+                break;
+
+            case DISTINCT_TYPE:
+                final DistinctType distinctType = (DistinctType) logicalType;
+                dataType = deserializeClass(distinctType.getSourceType(), 
classNode, serdeContext);
+                break;
+
+            default:
+                dataType = DataTypes.of(logicalType).toInternal();
+        }
+
+        final Class<?> conversionClass =
+                loadClass(
+                        classNode.get(FIELD_NAME_CONVERSION_CLASS).asText(),
+                        serdeContext,
+                        String.format("conversion class of data type '%s'", 
dataType));
+        return dataType.bridgedTo(conversionClass);
+    }
+
+    private static Class<?> loadClass(
+            String className, SerdeContext serdeContext, String explanation) {
+        try {
+            return ExtractionUtils.classForName(className, true, 
serdeContext.getClassLoader());
+        } catch (ClassNotFoundException e) {
+            throw new TableException(
+                    String.format("Could not load class '%s' for %s.", 
className, explanation));

Review comment:
       Can you propagate `e` in the exception cause?

##########
File path: 
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LogicalTypeJsonSerdeTest.java
##########
@@ -249,28 +331,44 @@ public void testLogicalTypeSerde() throws IOException {
                                         ObjectIdentifier.of("cat", "db", 
"distinctType"),
                                         new VarCharType(false, 5))
                                 .build(),
+                        // custom RawType
+                        new RawType<>(Integer.class, IntSerializer.INSTANCE),

Review comment:
       Can you add a test case here for a "real" raw type? For example take 
`RAW(LocalDateTime.class, LocalDateTimeSerializer.INSTANCE)` (I use it in 
`CastRulesTest`)

##########
File path: 
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LogicalTypeJsonSerdeTest.java
##########
@@ -60,68 +58,142 @@
 import org.apache.flink.table.types.logical.TimestampKind;
 import org.apache.flink.table.types.logical.TimestampType;
 import org.apache.flink.table.types.logical.TinyIntType;
-import org.apache.flink.table.types.logical.TypeInformationRawType;
 import org.apache.flink.table.types.logical.VarBinaryType;
 import org.apache.flink.table.types.logical.VarCharType;
 import org.apache.flink.table.types.logical.YearMonthIntervalType;
 import org.apache.flink.table.types.logical.ZonedTimestampType;
-import org.apache.flink.table.types.utils.DataTypeUtils;
+import org.apache.flink.table.types.utils.DataTypeFactoryMock;
+import org.apache.flink.table.utils.CatalogManagerMocks;
+import org.apache.flink.types.Row;
 
-import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator;
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException;
 import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectReader;
 import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectWriter;
 
 import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
+import org.junit.runners.Parameterized.Parameters;
 
 import java.io.IOException;
-import java.io.StringWriter;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
+import java.util.Optional;
 
-import static 
org.apache.flink.table.types.utils.LogicalTypeDataTypeConverter.toDataType;
-import static 
org.apache.flink.table.types.utils.LogicalTypeDataTypeConverter.toLogicalType;
-import static org.junit.Assert.assertEquals;
+import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches;
+import static 
org.apache.flink.table.api.config.TableConfigOptions.CatalogPlanCompilation.ALL;
+import static 
org.apache.flink.table.api.config.TableConfigOptions.CatalogPlanCompilation.IDENTIFIER;
+import static 
org.apache.flink.table.planner.plan.nodes.exec.serde.DataTypeJsonSerdeTest.configuredSerdeContext;
+import static 
org.apache.flink.table.utils.CatalogManagerMocks.preparedCatalogManager;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
 /** Tests for {@link LogicalType} serialization and deserialization. */
-@RunWith(Parameterized.class)
-public class LogicalTypeSerdeTest {
+public class LogicalTypeJsonSerdeTest {
 
-    @Parameterized.Parameter public LogicalType logicalType;
+    @ParameterizedTest
+    @MethodSource("testLogicalTypeSerde")
+    public void testLogicalTypeSerde(LogicalType logicalType) throws 
IOException {
+        final SerdeContext serdeContext = configuredSerdeContext();
+
+        final String json = toJson(serdeContext, logicalType);
+        final LogicalType actual = toLogicalType(serdeContext, json);
+
+        assertThat(actual).isEqualTo(logicalType);
+    }
 
     @Test
-    public void testLogicalTypeSerde() throws IOException {
-        SerdeContext serdeCtx =
-                new SerdeContext(
-                        new FlinkContextImpl(
-                                false,
-                                TableConfig.getDefault(),
-                                new ModuleManager(),
-                                null,
-                                null,
-                                null),
-                        Thread.currentThread().getContextClassLoader(),
-                        FlinkTypeFactory.INSTANCE(),
-                        FlinkSqlOperatorTable.instance());
-        ObjectReader objectReader = JsonSerdeUtil.createObjectReader(serdeCtx);
-        ObjectWriter objectWriter = JsonSerdeUtil.createObjectWriter(serdeCtx);
-
-        StringWriter writer = new StringWriter(100);
-        try (JsonGenerator gen = 
objectWriter.getFactory().createGenerator(writer)) {
-            gen.writeObject(logicalType);
-        }
-        String json = writer.toString();
-        LogicalType actual = objectReader.readValue(json, LogicalType.class);
-        assertEquals(logicalType, actual);
-        assertEquals(logicalType.asSummaryString(), actual.asSummaryString());
+    public void testIdentifierSerde() {
+        final DataTypeFactoryMock dataTypeFactoryMock = new 
DataTypeFactoryMock();
+        final TableConfig tableConfig = TableConfig.getDefault();
+        final Configuration config = tableConfig.getConfiguration();
+        final CatalogManager catalogManager =
+                
preparedCatalogManager().dataTypeFactory(dataTypeFactoryMock).build();
+        final SerdeContext serdeContext = 
configuredSerdeContext(catalogManager, tableConfig);
+
+        // minimal plan content
+        config.set(TableConfigOptions.PLAN_COMPILE_CATALOG_OBJECTS, 
IDENTIFIER);
+        final String minimalJson = toJson(serdeContext, STRUCTURED_TYPE);
+        
assertThat(minimalJson).isEqualTo("\"`default_catalog`.`default_database`.`MyType`\"");
+
+        // catalog lookup with miss
+        config.set(
+                TableConfigOptions.PLAN_RESTORE_CATALOG_OBJECTS,
+                TableConfigOptions.CatalogPlanRestore.IDENTIFIER);
+        dataTypeFactoryMock.logicalType = Optional.empty();
+        assertThatThrownBy(() -> toLogicalType(serdeContext, minimalJson))
+                .satisfies(anyCauseMatches(ValidationException.class, "No type 
found."));
+
+        // catalog lookup
+        config.set(
+                TableConfigOptions.PLAN_RESTORE_CATALOG_OBJECTS,
+                TableConfigOptions.CatalogPlanRestore.IDENTIFIER);
+        dataTypeFactoryMock.logicalType = Optional.of(STRUCTURED_TYPE);
+        assertThat(toLogicalType(serdeContext, 
minimalJson)).isEqualTo(STRUCTURED_TYPE);
+
+        // maximum plan content
+        config.set(TableConfigOptions.PLAN_COMPILE_CATALOG_OBJECTS, ALL);
+        final String maximumJson = toJson(serdeContext, STRUCTURED_TYPE);
+        assertThat(maximumJson)
+                .isEqualTo(
+                        "{\"type\":\"STRUCTURED_TYPE\","
+                                + "\"objectIdentifier\":"
+                                + "{\"catalogName\":\"default_catalog\","
+                                + "\"databaseName\":\"default_database\","
+                                + "\"tableName\":\"MyType\"},"
+                                + "\"description\":\"My original type.\","
+                                + "\"attributes\":[]}");

Review comment:
       Let's avoid these tests, but instead execute equalities on `ObjectNode`




-- 
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