MonsterChenzhuo commented on code in PR #4923:
URL: https://github.com/apache/seatunnel/pull/4923#discussion_r1257811161


##########
seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/sender/MongoDBConnectorDeserializationSchema.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * 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.seatunnel.connectors.seatunnel.cdc.mongodb.sender;
+
+import org.apache.seatunnel.api.source.Collector;
+import org.apache.seatunnel.api.table.type.ArrayType;
+import org.apache.seatunnel.api.table.type.DecimalType;
+import org.apache.seatunnel.api.table.type.MapType;
+import org.apache.seatunnel.api.table.type.RowKind;
+import org.apache.seatunnel.api.table.type.SeaTunnelDataType;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+import org.apache.seatunnel.api.table.type.SqlType;
+import 
org.apache.seatunnel.connectors.cdc.debezium.DebeziumDeserializationSchema;
+import 
org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.exception.MongodbConnectorException;
+
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.data.Struct;
+import org.apache.kafka.connect.source.SourceRecord;
+
+import org.bson.BsonDocument;
+import org.bson.BsonType;
+import org.bson.BsonValue;
+import org.bson.json.JsonMode;
+import org.bson.json.JsonWriterSettings;
+import org.bson.types.Decimal128;
+import org.jetbrains.annotations.Contract;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import com.mongodb.client.model.changestream.OperationType;
+
+import java.io.Serializable;
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.math.RoundingMode;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import static 
org.apache.seatunnel.common.exception.CommonErrorCode.ILLEGAL_ARGUMENT;
+import static 
org.apache.seatunnel.common.exception.CommonErrorCode.UNSUPPORTED_DATA_TYPE;
+import static 
org.apache.seatunnel.common.exception.CommonErrorCode.UNSUPPORTED_OPERATION;
+import static 
org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.DEFAULT_JSON_WRITER_SETTINGS;
+import static 
org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.DOCUMENT_KEY;
+import static 
org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ENCODE_VALUE_FIELD;
+import static 
org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.FULL_DOCUMENT;
+import static 
org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ID_FIELD;
+import static 
org.apache.seatunnel.shade.com.google.common.base.Preconditions.checkNotNull;
+
+public class MongoDBConnectorDeserializationSchema
+        implements DebeziumDeserializationSchema<SeaTunnelRow> {
+
+    private final SeaTunnelDataType<SeaTunnelRow> resultTypeInfo;
+
+    private final DeserializationRuntimeConverter physicalConverter;
+
+    public MongoDBConnectorDeserializationSchema(
+            SeaTunnelDataType<SeaTunnelRow> physicalDataType,
+            SeaTunnelDataType<SeaTunnelRow> resultTypeInfo) {
+        this.physicalConverter = createConverter(physicalDataType);
+        this.resultTypeInfo = resultTypeInfo;
+    }
+
+    @Override
+    public void deserialize(@NotNull SourceRecord record, 
Collector<SeaTunnelRow> out) {
+        Struct value = (Struct) record.value();
+        Schema valueSchema = record.valueSchema();
+
+        OperationType op = operationTypeFor(record);
+        BsonDocument documentKey =
+                checkNotNull(
+                        Objects.requireNonNull(
+                                extractBsonDocument(value, valueSchema, 
DOCUMENT_KEY)));
+        BsonDocument fullDocument = extractBsonDocument(value, valueSchema, 
FULL_DOCUMENT);
+
+        switch (op) {
+            case INSERT:
+                SeaTunnelRow insert = extractRowData(fullDocument);
+                insert.setRowKind(RowKind.INSERT);
+                emit(record, insert, out);
+                break;
+            case DELETE:
+                SeaTunnelRow delete =
+                        new SeaTunnelRow(
+                                new Object[] {
+                                    
documentKey.get(ID_FIELD).asObjectId().getValue().toString()
+                                });
+                delete.setRowKind(RowKind.DELETE);
+                emit(record, delete, out);
+                break;
+            case UPDATE:
+                if (fullDocument == null) {
+                    break;
+                }
+                SeaTunnelRow updateAfter = extractRowData(fullDocument);
+                updateAfter.setRowKind(RowKind.UPDATE_AFTER);
+                emit(record, updateAfter, out);
+                break;
+            case REPLACE:
+                SeaTunnelRow replaceAfter = extractRowData(fullDocument);
+                replaceAfter.setRowKind(RowKind.UPDATE_AFTER);
+                emit(record, replaceAfter, out);
+                break;
+            case INVALIDATE:
+            case DROP:
+            case DROP_DATABASE:
+            case RENAME:
+            case OTHER:
+            default:
+                break;
+        }
+    }
+
+    @Override
+    public SeaTunnelDataType<SeaTunnelRow> getProducedType() {
+        return resultTypeInfo;
+    }
+
+    private @NotNull OperationType operationTypeFor(@NotNull SourceRecord 
record) {
+        Struct value = (Struct) record.value();
+        return OperationType.fromString(value.getString("operationType"));
+    }
+
+    // TODO:The dynamic schema will be completed based on this method later.
+    private void emit(
+            SourceRecord inRecord,
+            SeaTunnelRow physicalRow,
+            @NotNull Collector<SeaTunnelRow> collector) {
+        collector.collect(physicalRow);
+    }
+
+    private SeaTunnelRow extractRowData(BsonDocument document) {
+        checkNotNull(document);
+        return (SeaTunnelRow) physicalConverter.convert(document);
+    }
+
+    private BsonDocument extractBsonDocument(
+            Struct value, @NotNull Schema valueSchema, String fieldName) {
+        if (valueSchema.field(fieldName) != null) {
+            String docString = value.getString(fieldName);
+            if (docString != null) {
+                return BsonDocument.parse(docString);
+            }
+        }
+        return null;
+    }
+
+    // 
-------------------------------------------------------------------------------------
+    // Runtime Converters
+    // 
-------------------------------------------------------------------------------------
+
+    @FunctionalInterface
+    public interface DeserializationRuntimeConverter extends Serializable {
+        Object convert(BsonValue bsonValue);
+    }
+
+    public DeserializationRuntimeConverter 
createConverter(SeaTunnelDataType<?> type) {
+        SerializableFunction<BsonValue, Object> internalRowConverter =
+                createNullSafeInternalConverter(type);
+        return new DeserializationRuntimeConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                return internalRowConverter.apply(bsonValue);
+            }
+        };
+    }
+
+    @Contract("_ -> new")

Review Comment:
   Oh, okay. the original intention of writing is that I think this kind of 
annotation will provide better code quality.



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