twalthr commented on a change in pull request #18427: URL: https://github.com/apache/flink/pull/18427#discussion_r793562935
########## File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSourceSpec.java ########## @@ -110,39 +106,73 @@ private DynamicTableSource getTableSource(FlinkContext flinkContext) { return tableSource; } - @JsonIgnore public ScanTableSource getScanTableSource(FlinkContext flinkContext) { DynamicTableSource tableSource = getTableSource(flinkContext); if (tableSource instanceof ScanTableSource) { return (ScanTableSource) tableSource; } else { throw new TableException( String.format( - "%s is not a ScanTableSource.\nplease check it.", + "%s is not a ScanTableSource.\n" + "please check it.", Review comment: if we change this line, then let's remove the new line at all? ########## File path: flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DynamicTableSinkSpecSerdeTest.java ########## @@ -139,39 +139,166 @@ public void testDynamicTableSinkSpecSerde() throws IOException { put("p", "A"); } }))); - spec2.setReadableConfig(new Configuration()); - Map<String, String> properties3 = new HashMap<>(); - properties3.put("connector", "values"); - properties3.put("schema.0.name", "a"); - properties3.put("schema.0.data-type", "BIGINT"); - properties3.put("schema.1.name", "b"); - properties3.put("schema.1.data-type", "INT"); - properties3.put("schema.2.name", "m"); - properties3.put("schema.2.data-type", "STRING"); - properties3.put("writable-metadata", "m:STRING"); - - final CatalogTable catalogTable3 = CatalogTable.fromProperties(properties3); + Map<String, String> options3 = new HashMap<>(); + options3.put("connector", TestValuesTableFactory.IDENTIFIER); + options3.put("writable-metadata", "m:STRING"); final ResolvedSchema resolvedSchema3 = new ResolvedSchema( Arrays.asList( Column.physical("a", DataTypes.BIGINT()), Column.physical("b", DataTypes.INT()), - Column.physical("m", DataTypes.STRING())), + Column.metadata("m", DataTypes.STRING(), null, false)), Collections.emptyList(), null); + final CatalogTable catalogTable3 = + CatalogTable.of( + Schema.newBuilder().fromResolvedSchema(resolvedSchema3).build(), + null, + Collections.emptyList(), + options3); DynamicTableSinkSpec spec3 = new DynamicTableSinkSpec( - ObjectIdentifier.of("default_catalog", "default_db", "MyTable"), - new ResolvedCatalogTable(catalogTable3, resolvedSchema3), + ContextResolvedTable.temporary( + ObjectIdentifier.of( + DEFAULT_BUILTIN_CATALOG, + DEFAULT_BUILTIN_DATABASE, + "MyTable"), + new ResolvedCatalogTable(catalogTable3, resolvedSchema3)), Collections.singletonList( new WritingMetadataSpec( Collections.singletonList("m"), RowType.of(new BigIntType(), new IntType())))); - spec3.setReadableConfig(new Configuration()); - return Arrays.asList(spec1, spec2, spec3); + return Stream.of(spec1, spec2, spec3); + } + + @ParameterizedTest + @MethodSource("testDynamicTableSinkSpecSerde") + void testDynamicTableSinkSpecSerde(DynamicTableSinkSpec spec) throws IOException { + TableEnvironmentImpl tableEnv = Review comment: use `PlannerMocks` instead? It is not perfect yet, but you can expose the `Configuration` (not `TableConfig`) and should be able to access everything you need. ########## File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ContextResolvedTableJsonDeserializer.java ########## @@ -0,0 +1,238 @@ +/* + * 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.table.api.ValidationException; +import org.apache.flink.table.api.config.TableConfigOptions.CatalogPlanCompilation; +import org.apache.flink.table.api.config.TableConfigOptions.CatalogPlanRestore; +import org.apache.flink.table.catalog.CatalogManager; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ContextResolvedTable; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.ResolvedSchema; + +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.deser.std.StdDeserializer; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; + +import java.io.IOException; +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +import static org.apache.flink.table.api.config.TableConfigOptions.CatalogPlanRestore.IDENTIFIER; +import static org.apache.flink.table.api.config.TableConfigOptions.PLAN_COMPILE_CATALOG_OBJECTS; +import static org.apache.flink.table.api.config.TableConfigOptions.PLAN_RESTORE_CATALOG_OBJECTS; +import static org.apache.flink.table.planner.plan.nodes.exec.serde.ContextResolvedTableJsonSerializer.FIELD_NAME_CATALOG_TABLE; +import static org.apache.flink.table.planner.plan.nodes.exec.serde.ContextResolvedTableJsonSerializer.FIELD_NAME_IDENTIFIER; +import static org.apache.flink.table.planner.plan.nodes.exec.serde.ResolvedCatalogTableJsonSerializer.OPTIONS; + +class ContextResolvedTableJsonDeserializer extends StdDeserializer<ContextResolvedTable> { + private static final long serialVersionUID = 1L; + + public ContextResolvedTableJsonDeserializer() { + super(ContextResolvedTable.class); + } + + @Override + public ContextResolvedTable deserialize(JsonParser jsonParser, DeserializationContext ctx) + throws IOException { + final CatalogPlanRestore planRestoreOption = + SerdeContext.get(ctx).getConfiguration().get(PLAN_RESTORE_CATALOG_OBJECTS); + final CatalogManager catalogManager = + SerdeContext.get(ctx).getFlinkContext().getCatalogManager(); + final ObjectNode objectNode = jsonParser.readValueAsTree(); + + // Deserialize the two fields, if available + final ObjectIdentifier identifier = + JsonSerdeUtil.deserializeOptionalField( + objectNode, + FIELD_NAME_IDENTIFIER, + ObjectIdentifier.class, + jsonParser.getCodec(), + ctx) + .orElse(null); + ResolvedCatalogTable resolvedCatalogTable = + JsonSerdeUtil.deserializeOptionalField( + objectNode, + FIELD_NAME_CATALOG_TABLE, + ResolvedCatalogTable.class, + jsonParser.getCodec(), + ctx) + .orElse(null); + + if (identifier == null && resolvedCatalogTable == null) { + throw new ValidationException( + String.format( + "The input json is invalid because it doesn't contain '%s', nor the '%s'.", Review comment: nit: to be consistent we should always right `JSON` instead of `json`. esp in exceptions. ########## File path: flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DynamicTableSourceSpecSerdeTest.java ########## @@ -238,6 +231,135 @@ public void testDynamicTableSourceSpecSerde() throws IOException { put("p", "B"); } })))); - return Arrays.asList(spec1, spec2); + return Stream.of(spec1, spec2); + } + + @ParameterizedTest + @MethodSource("testDynamicTableSinkSpecSerde") + public void testDynamicTableSourceSpecSerde(DynamicTableSourceSpec spec) throws IOException { + TableEnvironmentImpl tableEnv = + (TableEnvironmentImpl) TableEnvironment.create(inStreamingMode()); + + CatalogManager catalogManager = tableEnv.getCatalogManager(); Review comment: is it possible to unify with this sink test? I see a lot of duplicate code here. ########## File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ResolvedCatalogTableJsonSerializer.java ########## @@ -0,0 +1,91 @@ +/* + * 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.table.api.ValidationException; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.ExternalCatalogTable; +import org.apache.flink.table.catalog.ResolvedCatalogTable; + +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; + +class ResolvedCatalogTableJsonSerializer extends StdSerializer<ResolvedCatalogTable> { + private static final long serialVersionUID = 1L; + + static final String SERIALIZE_OPTIONS = "serialize_options"; + + public static final String RESOLVED_SCHEMA = "schema"; + public static final String PARTITION_KEYS = "partitionKeys"; + public static final String OPTIONS = "options"; + public static final String COMMENT = "comment"; + + public ResolvedCatalogTableJsonSerializer() { + super(ResolvedCatalogTable.class); + } + + @Override + public void serialize( + ResolvedCatalogTable resolvedCatalogTable, + JsonGenerator jsonGenerator, + SerializerProvider serializerProvider) + throws IOException { + boolean serializeOptions = + serializerProvider.getAttribute(SERIALIZE_OPTIONS) == null + || (boolean) serializerProvider.getAttribute(SERIALIZE_OPTIONS); + + serialize(resolvedCatalogTable, serializeOptions, jsonGenerator, serializerProvider); + } + + static void serialize( + ResolvedCatalogTable resolvedCatalogTable, + boolean serializeOptions, + JsonGenerator jsonGenerator, + SerializerProvider serializerProvider) + throws IOException { + // Thia should never happen anyway, but we keep this assertion for sanity check Review comment: typo ########## File path: flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testWritingMetadata.out ########## @@ -22,20 +37,35 @@ }, { "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink", "dynamicTableSink" : { - "identifier" : "`default_catalog`.`default_database`.`MySink`", - "catalogTable" : { - "schema.2.data-type" : "VARCHAR(2147483647)", - "schema.2.virtual" : "false", - "connector" : "values", - "schema.0.data-type" : "BIGINT", - "writable-metadata" : "m:STRING", - "schema.2.metadata" : "m", - "schema.2.name" : "m", - "schema.1.name" : "b", - "schema.0.name" : "a", - "schema.1.data-type" : "INT" + "table" : { + "identifier" : "`default_catalog`.`default_database`.`MySink`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "BIGINT" + }, { + "name" : "b", + "dataType" : "INT" + }, { + "name" : "m", + "kind" : "metadata", Review comment: put constants in upper case ########## File path: flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DynamicTableSinkSpecSerdeTest.java ########## @@ -139,39 +139,166 @@ public void testDynamicTableSinkSpecSerde() throws IOException { put("p", "A"); } }))); - spec2.setReadableConfig(new Configuration()); - Map<String, String> properties3 = new HashMap<>(); - properties3.put("connector", "values"); - properties3.put("schema.0.name", "a"); - properties3.put("schema.0.data-type", "BIGINT"); - properties3.put("schema.1.name", "b"); - properties3.put("schema.1.data-type", "INT"); - properties3.put("schema.2.name", "m"); - properties3.put("schema.2.data-type", "STRING"); - properties3.put("writable-metadata", "m:STRING"); - - final CatalogTable catalogTable3 = CatalogTable.fromProperties(properties3); + Map<String, String> options3 = new HashMap<>(); + options3.put("connector", TestValuesTableFactory.IDENTIFIER); + options3.put("writable-metadata", "m:STRING"); final ResolvedSchema resolvedSchema3 = new ResolvedSchema( Arrays.asList( Column.physical("a", DataTypes.BIGINT()), Column.physical("b", DataTypes.INT()), - Column.physical("m", DataTypes.STRING())), + Column.metadata("m", DataTypes.STRING(), null, false)), Collections.emptyList(), null); + final CatalogTable catalogTable3 = + CatalogTable.of( + Schema.newBuilder().fromResolvedSchema(resolvedSchema3).build(), + null, + Collections.emptyList(), + options3); DynamicTableSinkSpec spec3 = new DynamicTableSinkSpec( - ObjectIdentifier.of("default_catalog", "default_db", "MyTable"), - new ResolvedCatalogTable(catalogTable3, resolvedSchema3), + ContextResolvedTable.temporary( + ObjectIdentifier.of( + DEFAULT_BUILTIN_CATALOG, + DEFAULT_BUILTIN_DATABASE, + "MyTable"), + new ResolvedCatalogTable(catalogTable3, resolvedSchema3)), Collections.singletonList( new WritingMetadataSpec( Collections.singletonList("m"), RowType.of(new BigIntType(), new IntType())))); - spec3.setReadableConfig(new Configuration()); - return Arrays.asList(spec1, spec2, spec3); + return Stream.of(spec1, spec2, spec3); + } + + @ParameterizedTest + @MethodSource("testDynamicTableSinkSpecSerde") + void testDynamicTableSinkSpecSerde(DynamicTableSinkSpec spec) throws IOException { + TableEnvironmentImpl tableEnv = + (TableEnvironmentImpl) TableEnvironment.create(inStreamingMode()); + + CatalogManager catalogManager = tableEnv.getCatalogManager(); + catalogManager.initSchemaResolver(true, ExpressionResolverMocks.dummyResolver()); + catalogManager.createTable( + spec.getContextResolvedTable().getResolvedTable(), + spec.getContextResolvedTable().getIdentifier(), + false); + + SerdeContext serdeCtx = configuredSerdeContext(catalogManager, tableEnv.getConfig()); + + // Re-init the spec to be permanent with correct catalog + spec = + new DynamicTableSinkSpec( + ContextResolvedTable.permanent( + spec.getContextResolvedTable().getIdentifier(), + catalogManager.getCatalog(catalogManager.getCurrentCatalog()).get(), + spec.getContextResolvedTable().getResolvedTable()), + spec.getSinkAbilities()); + + String actualJson = toJson(serdeCtx, spec); + DynamicTableSinkSpec actual = toObject(serdeCtx, actualJson, DynamicTableSinkSpec.class); + + assertThat(actual.getContextResolvedTable()).isEqualTo(spec.getContextResolvedTable()); + assertThat(actual.getSinkAbilities()).isEqualTo(spec.getSinkAbilities()); + + assertThat(actual.getTableSink(((PlannerBase) tableEnv.getPlanner()).getFlinkContext())) + .isNotNull(); + } + + @Test + void testDynamicTableSinkSpecSerdeWithEnrichmentOptions() throws Exception { + // Test model + ObjectIdentifier identifier = + ObjectIdentifier.of( + CatalogManagerMocks.DEFAULT_CATALOG, + CatalogManagerMocks.DEFAULT_DATABASE, + "my_table"); + ResolvedSchema resolvedSchema = + new ResolvedSchema( + Arrays.asList( + Column.physical("a", DataTypes.STRING()), + Column.physical("b", DataTypes.INT()), + Column.physical("c", DataTypes.BOOLEAN())), + Collections.emptyList(), + null); + Schema schema = Schema.newBuilder().fromResolvedSchema(resolvedSchema).build(); + + String formatPrefix = FactoryUtil.getFormatPrefix(FORMAT, TestFormatFactory.IDENTIFIER); + + Map<String, String> planOptions = new HashMap<>(); + planOptions.put(CONNECTOR.key(), TestDynamicTableFactory.IDENTIFIER); + planOptions.put(TARGET.key(), "abc"); + planOptions.put(BUFFER_SIZE.key(), "1000"); + planOptions.put(FORMAT.key(), TestFormatFactory.IDENTIFIER); + planOptions.put(formatPrefix + DELIMITER.key(), "|"); + + Map<String, String> catalogOptions = new HashMap<>(); + catalogOptions.put(CONNECTOR.key(), TestDynamicTableFactory.IDENTIFIER); + catalogOptions.put(TARGET.key(), "xyz"); + catalogOptions.put(BUFFER_SIZE.key(), "2000"); + catalogOptions.put(FORMAT.key(), TestFormatFactory.IDENTIFIER); + catalogOptions.put(formatPrefix + DELIMITER.key(), ","); + + ResolvedCatalogTable planResolvedCatalogTable = + new ResolvedCatalogTable( + CatalogTable.of(schema, null, Collections.emptyList(), planOptions), + resolvedSchema); + ResolvedCatalogTable catalogResolvedCatalogTable = + new ResolvedCatalogTable( + CatalogTable.of(schema, null, Collections.emptyList(), catalogOptions), + resolvedSchema); + + // Create table env + TableEnvironmentImpl tableEnv = + (TableEnvironmentImpl) TableEnvironment.create(inStreamingMode()); + + // Create mock catalog + CatalogManager catalogManager = tableEnv.getCatalogManager(); + catalogManager.initSchemaResolver(true, ExpressionResolverMocks.dummyResolver()); Review comment: this should not be necessary? ########## File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSourceSpec.java ########## @@ -110,39 +106,73 @@ private DynamicTableSource getTableSource(FlinkContext flinkContext) { return tableSource; } - @JsonIgnore public ScanTableSource getScanTableSource(FlinkContext flinkContext) { DynamicTableSource tableSource = getTableSource(flinkContext); if (tableSource instanceof ScanTableSource) { return (ScanTableSource) tableSource; } else { throw new TableException( String.format( - "%s is not a ScanTableSource.\nplease check it.", + "%s is not a ScanTableSource.\n" + "please check it.", tableSource.getClass().getName())); } } - @JsonIgnore public LookupTableSource getLookupTableSource(FlinkContext flinkContext) { DynamicTableSource tableSource = getTableSource(flinkContext); if (tableSource instanceof LookupTableSource) { return (LookupTableSource) tableSource; } else { throw new TableException( String.format( - "%s is not a LookupTableSource.\nplease check it.", + "%s is not a LookupTableSource.\n" + "please check it.", Review comment: same here -- 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-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org