wuchong commented on a change in pull request #14464: URL: https://github.com/apache/flink/pull/14464#discussion_r548441095
########## File path: flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java ########## @@ -0,0 +1,226 @@ +/* + * 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.formats.json.canal; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.json.TimestampFormat; +import org.apache.flink.formats.json.canal.CanalJsonDeserializationSchema.MetadataConverter; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.utils.DataTypeUtils; +import org.apache.flink.types.RowKind; + +import javax.annotation.Nullable; + +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * {@link DecodingFormat} for Canal using JSON encoding. + */ +public class CanalJsonDecodingFormat implements DecodingFormat<DeserializationSchema<RowData>> { + + // -------------------------------------------------------------------------------------------- + // Mutable attributes + // -------------------------------------------------------------------------------------------- + + private List<String> metadataKeys; + + // -------------------------------------------------------------------------------------------- + // Canal-specific attributes + // -------------------------------------------------------------------------------------------- + + private final @Nullable String database; + + private final @Nullable String table; + + private final boolean ignoreParseErrors; + + private final TimestampFormat timestampFormat; + + public CanalJsonDecodingFormat( + String database, + String table, + boolean ignoreParseErrors, + TimestampFormat timestampFormat) { + this.database = database; + this.table = table; + this.ignoreParseErrors = ignoreParseErrors; + this.timestampFormat = timestampFormat; + this.metadataKeys = Collections.emptyList(); + } + + @Override + public DeserializationSchema<RowData> createRuntimeDecoder( + DynamicTableSource.Context context, + DataType physicalDataType) { Review comment: Add indent. ########## File path: flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java ########## @@ -0,0 +1,226 @@ +/* + * 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.formats.json.canal; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.json.TimestampFormat; +import org.apache.flink.formats.json.canal.CanalJsonDeserializationSchema.MetadataConverter; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.utils.DataTypeUtils; +import org.apache.flink.types.RowKind; + +import javax.annotation.Nullable; + +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * {@link DecodingFormat} for Canal using JSON encoding. + */ +public class CanalJsonDecodingFormat implements DecodingFormat<DeserializationSchema<RowData>> { + + // -------------------------------------------------------------------------------------------- + // Mutable attributes + // -------------------------------------------------------------------------------------------- + + private List<String> metadataKeys; + + // -------------------------------------------------------------------------------------------- + // Canal-specific attributes + // -------------------------------------------------------------------------------------------- + + private final @Nullable String database; + + private final @Nullable String table; + + private final boolean ignoreParseErrors; + + private final TimestampFormat timestampFormat; + + public CanalJsonDecodingFormat( + String database, + String table, + boolean ignoreParseErrors, + TimestampFormat timestampFormat) { + this.database = database; + this.table = table; + this.ignoreParseErrors = ignoreParseErrors; + this.timestampFormat = timestampFormat; + this.metadataKeys = Collections.emptyList(); + } + + @Override + public DeserializationSchema<RowData> createRuntimeDecoder( + DynamicTableSource.Context context, + DataType physicalDataType) { + final List<ReadableMetadata> readableMetadata = metadataKeys.stream() + .map(k -> + Stream.of(ReadableMetadata.values()) + .filter(rm -> rm.key.equals(k)) + .findFirst() + .orElseThrow(IllegalStateException::new)) + .collect(Collectors.toList()); + final List<DataTypes.Field> metadataFields = readableMetadata.stream() + .map(m -> DataTypes.FIELD(m.key, m.dataType)) + .collect(Collectors.toList()); + final DataType producedDataType = DataTypeUtils.appendRowFields(physicalDataType, metadataFields); + final TypeInformation<RowData> producedTypeInfo = context.createTypeInformation(producedDataType); + return CanalJsonDeserializationSchema.builder(physicalDataType, readableMetadata, producedTypeInfo) + .setDatabase(database) + .setTable(table) + .setIgnoreParseErrors(ignoreParseErrors) + .setTimestampFormat(timestampFormat) + .build(); + } + + @Override + public Map<String, DataType> listReadableMetadata() { + final Map<String, DataType> metadataMap = new LinkedHashMap<>(); + Stream.of(ReadableMetadata.values()).forEachOrdered(m -> metadataMap.put(m.key, m.dataType)); + return metadataMap; + } + + @Override + public void applyReadableMetadata(List<String> metadataKeys) { + this.metadataKeys = metadataKeys; + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.UPDATE_BEFORE) + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + } + + // -------------------------------------------------------------------------------------------- + // Metadata handling + // -------------------------------------------------------------------------------------------- + + /** + * List of metadata that can be read with this format. + */ + enum ReadableMetadata { + DATABASE( + "database", + DataTypes.STRING().nullable(), + DataTypes.FIELD("database", DataTypes.STRING()), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData row, int pos) { + return row.getString(pos); + } + } + ), + + TABLE( + "table", + DataTypes.STRING().nullable(), + DataTypes.FIELD("table", DataTypes.STRING()), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData row, int pos) { + return row.getString(pos); + } + } + ), + + SQL_TYPE( + "sql-type", + DataTypes.STRING().nullable(), + DataTypes.FIELD("sqlType", DataTypes.STRING()), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData row, int pos) { + return row.getString(pos); + } + } + ), + + PK_NAMES( + "pk-names", + DataTypes.STRING().nullable(), + DataTypes.FIELD("pkNames", DataTypes.ARRAY(DataTypes.STRING())), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData row, int pos) { + return row.getArray(pos); + } + } + ), + + INGESTION_TIMESTAMP( + "ingestion-timestamp", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(), + DataTypes.FIELD("ts", DataTypes.BIGINT()), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData row, int pos) { + if (row.isNullAt(pos)) { + return null; + } + return TimestampData.fromEpochMillis(row.getLong(pos)); + } + } + ); + + final String key; + + final DataType dataType; + + final DataTypes.Field requiredJsonField; + + final MetadataConverter converter; + + ReadableMetadata( + String key, + DataType dataType, + DataTypes.Field requiredJsonField, + MetadataConverter converter) { Review comment: Add indent. ########## File path: flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java ########## @@ -262,24 +296,71 @@ public boolean equals(Object o) { return false; } CanalJsonDeserializationSchema that = (CanalJsonDeserializationSchema) o; - return ignoreParseErrors == that.ignoreParseErrors && - fieldCount == that.fieldCount && - Objects.equals(jsonDeserializer, that.jsonDeserializer) && - Objects.equals(resultTypeInfo, that.resultTypeInfo); + return Objects.equals(jsonDeserializer, that.jsonDeserializer) + && hasMetadata == that.hasMetadata + && Objects.equals(producedTypeInfo, that.producedTypeInfo) + && Objects.equals(database, that.database) + && Objects.equals(table, that.table) + && ignoreParseErrors == that.ignoreParseErrors + && fieldCount == that.fieldCount; } @Override public int hashCode() { - return Objects.hash(jsonDeserializer, resultTypeInfo, ignoreParseErrors, fieldCount); + return Objects.hash(jsonDeserializer, hasMetadata, producedTypeInfo, database, table, ignoreParseErrors, fieldCount); } - private static RowType createJsonRowType(DataType databaseSchema) { + // -------------------------------------------------------------------------------------------- + + private static RowType createJsonRowType(DataType physicalDataType, List<ReadableMetadata> readableMetadata) { // Canal JSON contains other information, e.g. "ts", "sql", but we don't need them - return (RowType) DataTypes.ROW( - DataTypes.FIELD("data", DataTypes.ARRAY(databaseSchema)), - DataTypes.FIELD("old", DataTypes.ARRAY(databaseSchema)), + DataType root = DataTypes.ROW( + DataTypes.FIELD("data", DataTypes.ARRAY(physicalDataType)), + DataTypes.FIELD("old", DataTypes.ARRAY(physicalDataType)), DataTypes.FIELD("type", DataTypes.STRING()), - DataTypes.FIELD("database", DataTypes.STRING()), - DataTypes.FIELD("table", DataTypes.STRING())).getLogicalType(); + ReadableMetadata.DATABASE.requiredJsonField, + ReadableMetadata.TABLE.requiredJsonField); + // append fields that are required for reading metadata in the root + final List<DataTypes.Field> rootMetadataFields = readableMetadata.stream() + .filter(m -> m != ReadableMetadata.DATABASE && m != ReadableMetadata.TABLE) + .map(m -> m.requiredJsonField) + .distinct() + .collect(Collectors.toList()); + return (RowType) DataTypeUtils.appendRowFields(root, rootMetadataFields).getLogicalType(); + } + + private static MetadataConverter[] createMetadataConverters( + RowType jsonRowType, + List<ReadableMetadata> requestedMetadata) { Review comment: Add indent. ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org