wuchong commented on a change in pull request #13333: URL: https://github.com/apache/flink/pull/13333#discussion_r484857382
########## File path: flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java ########## @@ -0,0 +1,112 @@ +/* + * 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.debezium; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.formats.json.JsonRowDataSerializationSchema; +import org.apache.flink.formats.json.TimestampFormat; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; + +import java.util.Objects; + +import static java.lang.String.format; +import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; + +/** + * Serialization schema from Flink Table/SQL internal data structure {@link RowData} to Debezium JSON. + * + * @see <a href="https://debezium.io/">Debezium</a> + */ +public class DebeziumJsonSerializationSchema implements SerializationSchema<RowData> { + private static final long serialVersionUID = 1L; + + private static final StringData OP_INSERT = StringData.fromString("INSERT"); // insert + private static final StringData OP_DELETE = StringData.fromString("DELETE"); // delete + + /** The serializer to serialize Debezium JSON data. **/ + private final JsonRowDataSerializationSchema jsonSerializer; + + private transient GenericRowData genericRowData; + + public DebeziumJsonSerializationSchema(RowType rowType, TimestampFormat timestampFormat) { + jsonSerializer = new JsonRowDataSerializationSchema( + createJsonRowType(fromLogicalToDataType(rowType)), + timestampFormat); + } + + @Override + public void open(InitializationContext context) { + genericRowData = new GenericRowData(2); + } + + @Override + public byte[] serialize(RowData rowData) { + try { + genericRowData.setField(0, rowData); + genericRowData.setField(1, rowKind2String(rowData.getRowKind())); + return jsonSerializer.serialize(genericRowData); + } catch (Throwable t) { + throw new RuntimeException(format("Could not serialize row '%s'.", rowData), t); + } + } + + private StringData rowKind2String(RowKind rowKind) { + switch (rowKind) { + case INSERT: + case UPDATE_AFTER: + return OP_INSERT; + case UPDATE_BEFORE: + case DELETE: + return OP_DELETE; + default: + throw new UnsupportedOperationException(format("Unsupported operation '%s' for row kind.", rowKind)); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DebeziumJsonSerializationSchema that = (DebeziumJsonSerializationSchema) o; + return Objects.equals(jsonSerializer, that.jsonSerializer); + } + + @Override + public int hashCode() { + return Objects.hash(jsonSerializer); + } + + private static RowType createJsonRowType(DataType databaseSchema) { + // Debezium JSON contains some other information, e.g. "source", "ts_ms" + // but we don't need them. + return (RowType) DataTypes.ROW( + DataTypes.FIELD("after", databaseSchema), Review comment: This is wrong. Debezium serialize insert data in "after", but delete data in "before". ########## File path: flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java ########## @@ -0,0 +1,112 @@ +/* + * 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.debezium; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.formats.json.JsonRowDataSerializationSchema; +import org.apache.flink.formats.json.TimestampFormat; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; + +import java.util.Objects; + +import static java.lang.String.format; +import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; + +/** + * Serialization schema from Flink Table/SQL internal data structure {@link RowData} to Debezium JSON. + * + * @see <a href="https://debezium.io/">Debezium</a> + */ +public class DebeziumJsonSerializationSchema implements SerializationSchema<RowData> { + private static final long serialVersionUID = 1L; + + private static final StringData OP_INSERT = StringData.fromString("INSERT"); // insert + private static final StringData OP_DELETE = StringData.fromString("DELETE"); // delete + + /** The serializer to serialize Debezium JSON data. **/ + private final JsonRowDataSerializationSchema jsonSerializer; + + private transient GenericRowData genericRowData; + + public DebeziumJsonSerializationSchema(RowType rowType, TimestampFormat timestampFormat) { + jsonSerializer = new JsonRowDataSerializationSchema( + createJsonRowType(fromLogicalToDataType(rowType)), + timestampFormat); + } + + @Override + public void open(InitializationContext context) { + genericRowData = new GenericRowData(2); + } + + @Override + public byte[] serialize(RowData rowData) { + try { + genericRowData.setField(0, rowData); Review comment: This is wrong. Debezium serialize insert data in "after", but delete data in "before". ########## File path: flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonFormatFactory.java ########## @@ -104,7 +105,31 @@ public ChangelogMode getChangelogMode() { public EncodingFormat<SerializationSchema<RowData>> createEncodingFormat( DynamicTableFactory.Context context, ReadableConfig formatOptions) { - throw new UnsupportedOperationException("Debezium format doesn't support as a sink format yet."); + FactoryUtil.validateFactoryOptions(this, formatOptions); + TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions); + + return new EncodingFormat<SerializationSchema<RowData>>() { + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.UPDATE_BEFORE) + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + } + + @Override + public SerializationSchema<RowData> createRuntimeEncoder( + DynamicTableSink.Context context, DataType consumedDataType) { Review comment: Indent. ########## File path: flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/debezium/DebeziumJsonSerializationSchema.java ########## @@ -0,0 +1,112 @@ +/* + * 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.debezium; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.formats.json.JsonRowDataSerializationSchema; +import org.apache.flink.formats.json.TimestampFormat; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; + +import java.util.Objects; + +import static java.lang.String.format; +import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; + +/** + * Serialization schema from Flink Table/SQL internal data structure {@link RowData} to Debezium JSON. + * + * @see <a href="https://debezium.io/">Debezium</a> + */ +public class DebeziumJsonSerializationSchema implements SerializationSchema<RowData> { + private static final long serialVersionUID = 1L; + + private static final StringData OP_INSERT = StringData.fromString("INSERT"); // insert + private static final StringData OP_DELETE = StringData.fromString("DELETE"); // delete Review comment: The insert and delete operation string in debezium should be "c" and "d". ---------------------------------------------------------------- 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: [email protected]
