leonardBang commented on a change in pull request #18738: URL: https://github.com/apache/flink/pull/18738#discussion_r806712589
########## File path: flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/ogg/OggJsonDeserializationSchema.java ########## @@ -0,0 +1,276 @@ +/* + * 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.ogg; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.common.TimestampFormat; +import org.apache.flink.formats.json.JsonRowDataDeserializationSchema; +import org.apache.flink.formats.json.ogg.OggJsonDecodingFormat.ReadableMetadata; +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.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.DataTypeUtils; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.Collector; + +import java.io.IOException; +import java.io.Serializable; +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; + +import static java.lang.String.format; + +/** + * Deserialization schema from Ogg JSON to Flink Table/SQL internal data structure {@link RowData}. + * The deserialization schema knows Ogg's schema definition and can extract the database data and + * convert into {@link RowData} with {@link RowKind}. + * + * <p>Deserializes a <code>byte[]</code> message as a JSON object and reads the specified fields. + * + * <p>Failures during deserialization are forwarded as wrapped IOExceptions. + * + * @see <a href="https://www.oracle.com/cn/middleware/technologies/goldengate/overview.html">Ogg</a> + */ +@Internal +public final class OggJsonDeserializationSchema implements DeserializationSchema<RowData> { + private static final long serialVersionUID = 1L; + + private static final String OP_CREATE = "I"; // insert + private static final String OP_UPDATE = "U"; // update + private static final String OP_DELETE = "D"; // delete + private static final String OP_TRUNCATE = "T"; // truncate + + private static final String REPLICA_IDENTITY_EXCEPTION = + "The \"before\" field of %s message is null, " + + "if you are using Ogg Postgres Connector, " + + "please check the Postgres table has been set REPLICA IDENTITY to FULL level."; + + /** The deserializer to deserialize Ogg JSON data. */ + private final JsonRowDataDeserializationSchema jsonDeserializer; + + /** Flag that indicates that an additional projection is required for metadata. */ + private final boolean hasMetadata; + + /** Metadata to be extracted for every record. */ + private final MetadataConverter[] metadataConverters; + + /** {@link TypeInformation} of the produced {@link RowData} (physical + metadata). */ + private final TypeInformation<RowData> producedTypeInfo; + + /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ + private final boolean ignoreParseErrors; + + public OggJsonDeserializationSchema( + DataType physicalDataType, + List<ReadableMetadata> requestedMetadata, + TypeInformation<RowData> producedTypeInfo, + boolean ignoreParseErrors, + TimestampFormat timestampFormat) { + final RowType jsonRowType = createJsonRowType(physicalDataType, requestedMetadata); + this.jsonDeserializer = + new JsonRowDataDeserializationSchema( + jsonRowType, + // the result type is never used, so it's fine to pass in the produced type + // info + producedTypeInfo, + false, // ignoreParseErrors already contains the functionality of + // failOnMissingField + ignoreParseErrors, + timestampFormat); + this.hasMetadata = requestedMetadata.size() > 0; + this.metadataConverters = createMetadataConverters(jsonRowType, requestedMetadata); + this.producedTypeInfo = producedTypeInfo; + this.ignoreParseErrors = ignoreParseErrors; + } + + private static RowType createJsonRowType( + DataType physicalDataType, List<ReadableMetadata> readableMetadata) { + DataType root = + DataTypes.ROW( + DataTypes.FIELD("before", physicalDataType), + DataTypes.FIELD("after", physicalDataType), + DataTypes.FIELD("op_type", DataTypes.STRING())); + // append fields that are required for reading metadata in the root + final List<DataTypes.Field> rootMetadataFields = + readableMetadata.stream() + .map(m -> m.requiredJsonField) + .distinct() + .collect(Collectors.toList()); + return (RowType) DataTypeUtils.appendRowFields(root, rootMetadataFields).getLogicalType(); + } + + private static MetadataConverter[] createMetadataConverters( + RowType jsonRowType, List<ReadableMetadata> requestedMetadata) { + return requestedMetadata.stream() + .map(m -> convertInRoot(jsonRowType, m)) + .toArray(MetadataConverter[]::new); + } + + private static MetadataConverter convertInRoot(RowType jsonRowType, ReadableMetadata metadata) { + final int pos = findFieldPos(metadata, jsonRowType); + return new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(GenericRowData root, int unused) { + return metadata.converter.convert(root, pos); + } + }; + } + + private static int findFieldPos(ReadableMetadata metadata, RowType jsonRowType) { + return jsonRowType.getFieldNames().indexOf(metadata.requiredJsonField.getName()); + } + + @Override + public RowData deserialize(byte[] message) { + throw new RuntimeException( + "Please invoke DeserializationSchema#deserialize(byte[], Collector<RowData>) instead."); + } + + @Override + public void deserialize(byte[] message, Collector<RowData> out) throws IOException { + if (message == null || message.length == 0) { + // skip tombstone messages + return; + } + try { + GenericRowData row = (GenericRowData) jsonDeserializer.deserialize(message); + + GenericRowData before = (GenericRowData) row.getField(0); + GenericRowData after = (GenericRowData) row.getField(1); + String op = row.getField(2).toString(); + if (OP_CREATE.equals(op)) { + after.setRowKind(RowKind.INSERT); + emitRow(row, after, out); + } else if (OP_UPDATE.equals(op)) { + if (before == null) { + throw new IllegalStateException( + String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); + } + before.setRowKind(RowKind.UPDATE_BEFORE); + after.setRowKind(RowKind.UPDATE_AFTER); + emitRow(row, before, out); + emitRow(row, after, out); + } else if (OP_DELETE.equals(op)) { + if (before == null) { + throw new IllegalStateException( + String.format(REPLICA_IDENTITY_EXCEPTION, "DELETE")); + } + before.setRowKind(RowKind.DELETE); + emitRow(row, before, out); + } else if (OP_TRUNCATE.equals(op)) { + // flink can't interpret such type of record, skip the record Review comment: print a WARN LOG ? -- 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]
