liuxiao2shf commented on code in PR #3619:
URL: https://github.com/apache/flink-cdc/pull/3619#discussion_r1870538039


##########
flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/MySqlEventDeserializer.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.cdc.connectors.base.source;
+
+import org.apache.flink.cdc.common.annotation.Internal;
+import org.apache.flink.cdc.common.data.binary.BinaryStringData;
+import org.apache.flink.cdc.common.event.SchemaChangeEvent;
+import org.apache.flink.cdc.common.event.TableId;
+import 
org.apache.flink.cdc.connectors.base.source.parser.CustomMySqlAntlrDdlParser;
+import org.apache.flink.cdc.debezium.event.DebeziumEventDeserializationSchema;
+import org.apache.flink.cdc.debezium.table.DebeziumChangelogMode;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.debezium.data.Envelope;
+import io.debezium.relational.Tables;
+import io.debezium.relational.history.HistoryRecord;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.data.Struct;
+import org.apache.kafka.connect.source.SourceRecord;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import static 
org.apache.flink.cdc.connectors.base.utils.SourceRecordUtils.getHistoryRecord;
+
+/** Event deserializer for {@link MySqlDataSource}. */
+@Internal
+public class MySqlEventDeserializer extends DebeziumEventDeserializationSchema 
{
+
+    private static final long serialVersionUID = 1L;
+
+    public static final String SCHEMA_CHANGE_EVENT_KEY_NAME =
+            "io.debezium.connector.mysql.SchemaChangeKey";
+
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+    private final boolean includeSchemaChanges;
+
+    private transient Tables tables;
+    private transient CustomMySqlAntlrDdlParser customParser;
+
+    public MySqlEventDeserializer(
+            DebeziumChangelogMode changelogMode, boolean includeSchemaChanges) 
{
+        super(new MySqlSchemaDataTypeInference(), changelogMode);
+        this.includeSchemaChanges = includeSchemaChanges;
+    }
+
+    @Override
+    protected List<SchemaChangeEvent> 
deserializeSchemaChangeRecord(SourceRecord record) {
+        if (includeSchemaChanges) {
+            if (customParser == null) {
+                customParser = new CustomMySqlAntlrDdlParser();
+                tables = new Tables();
+            }
+
+            try {
+                HistoryRecord historyRecord = getHistoryRecord(record);
+
+                String databaseName =
+                        
historyRecord.document().getString(HistoryRecord.Fields.DATABASE_NAME);
+                String ddl =
+                        
historyRecord.document().getString(HistoryRecord.Fields.DDL_STATEMENTS);
+                customParser.setCurrentDatabase(databaseName);
+                customParser.parse(ddl, tables);
+                return customParser.getAndClearParsedEvents();
+            } catch (IOException e) {
+                throw new IllegalStateException("Failed to parse the schema 
change : " + record, e);
+            }
+        }
+        return Collections.emptyList();
+    }
+
+    @Override
+    protected boolean isDataChangeRecord(SourceRecord record) {
+        Schema valueSchema = record.valueSchema();
+        Struct value = (Struct) record.value();
+        return value != null
+                && valueSchema != null
+                && valueSchema.field(Envelope.FieldName.OPERATION) != null
+                && value.getString(Envelope.FieldName.OPERATION) != null;
+    }
+
+    @Override
+    protected boolean isSchemaChangeRecord(SourceRecord record) {
+        Schema keySchema = record.keySchema();
+        return keySchema != null && 
SCHEMA_CHANGE_EVENT_KEY_NAME.equalsIgnoreCase(keySchema.name());
+    }
+
+    @Override
+    protected TableId getTableId(SourceRecord record) {
+        String[] parts = record.topic().split("\\.");
+        return TableId.tableId(parts[1], parts[2]);
+    }
+
+    @Override
+    protected Map<String, String> getMetadata(SourceRecord record) {
+        return Collections.emptyMap();
+    }
+
+    @Override
+    protected Object convertToString(Object dbzObj, Schema schema) {
+        // the Geometry datatype in MySQL will be converted to
+        // a String with Json format
+        // if (Point.LOGICAL_NAME.equals(schema.name())
+        //        || Geometry.LOGICAL_NAME.equals(schema.name())) {
+        //    try {
+        //        Struct geometryStruct = (Struct) dbzObj;
+        //        byte[] wkb = geometryStruct.getBytes("wkb");
+        //        String geoJson = 
OGCGeometry.fromBinary(ByteBuffer.wrap(wkb)).asGeoJson();
+        //        JsonNode originGeoNode = OBJECT_MAPPER.readTree(geoJson);
+        //        Optional<Integer> srid = 
Optional.ofNullable(geometryStruct.getInt32("srid"));
+        //        Map<String, Object> geometryInfo = new HashMap<>();
+        //        String geometryType = originGeoNode.get("type").asText();
+        //        geometryInfo.put("type", geometryType);
+        //        if (geometryType.equals("GeometryCollection")) {
+        //            geometryInfo.put("geometries", 
originGeoNode.get("geometries"));
+        //        } else {
+        //            geometryInfo.put("coordinates", 
originGeoNode.get("coordinates"));
+        //        }
+        //        geometryInfo.put("srid", srid.orElse(0));
+        //        return BinaryStringData.fromString(
+        //                
OBJECT_MAPPER.writer().writeValueAsString(geometryInfo));
+        //    } catch (Exception e) {
+        //        throw new IllegalArgumentException(
+        //                String.format("Failed to convert %s to geometry 
JSON.", dbzObj), e);
+        //    }
+        // } else {
+        //    return BinaryStringData.fromString(dbzObj.toString());
+        // }

Review Comment:
   Processed



##########
flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/assigner/SnapshotSplitAssigner.java:
##########
@@ -397,6 +491,27 @@ && allSnapshotSplitsFinished()) {
             }
             LOG.info("Snapshot split assigner is turn into finished status.");
         }
+
+        if (splitFinishedCheckpointIds != null && 
!splitFinishedCheckpointIds.isEmpty()) {
+            Iterator<Map.Entry<String, Long>> iterator =
+                    splitFinishedCheckpointIds.entrySet().iterator();
+            while (iterator.hasNext()) {
+                Map.Entry<String, Long> splitFinishedCheckpointId = 
iterator.next();
+                String splitId = splitFinishedCheckpointId.getKey();
+                Long splitCheckpointId = splitFinishedCheckpointId.getValue();
+                if (splitCheckpointId != UNDEFINED_CHECKPOINT_ID
+                        && checkpointId >= splitCheckpointId) {
+                    // record table-level splits metrics
+                    TableId tableId = SnapshotSplit.parseTableId(splitId);

Review Comment:
   Processed



##########
flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/meta/split/SnapshotSplit.java:
##########
@@ -95,6 +95,10 @@ public TableId getTableId() {
         return tableId;
     }
 
+    public static TableId parseTableId(String splitId) {
+        return TableId.parse(splitId.substring(0, splitId.lastIndexOf(":")));
+    }

Review Comment:
   Processed



##########
flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/assigner/SnapshotSplitAssigner.java:
##########
@@ -335,6 +402,12 @@ public List<FinishedSnapshotSplitInfo> 
getFinishedSplitInfos() {
     @Override
     public void onFinishedSplits(Map<String, Offset> splitFinishedOffsets) {
         this.splitFinishedOffsets.putAll(splitFinishedOffsets);
+        for (String splitId : splitFinishedOffsets.keySet()) {
+            splitFinishedCheckpointIds.put(splitId, UNDEFINED_CHECKPOINT_ID);
+        }
+        LOG.info(
+                "onFinishedSplits, splitFinishedCheckpointIds size: {}",
+                splitFinishedCheckpointIds == null ? 0 : 
splitFinishedCheckpointIds.size());

Review Comment:
   Processed



##########
flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/assigner/SnapshotSplitAssigner.java:
##########
@@ -269,6 +279,52 @@ private void captureNewlyAddedTables() {
         }
     }
 
+    /** This should be invoked after this class's open method. */
+    public void initEnumeratorMetrics(SourceEnumeratorMetrics 
enumeratorMetrics) {
+        this.enumeratorMetrics = enumeratorMetrics;
+
+        this.enumeratorMetrics.enterSnapshotPhase();
+        this.enumeratorMetrics.registerMetrics(
+                alreadyProcessedTables::size, assignedSplits::size, 
remainingSplits::size);
+        this.enumeratorMetrics.addNewTables(computeTablesPendingSnapshot());
+        for (SchemalessSnapshotSplit snapshotSplit : remainingSplits) {
+            this.enumeratorMetrics
+                    .getTableMetrics(snapshotSplit.getTableId())
+                    .addNewSplit(snapshotSplit.splitId());
+        }
+        for (SchemalessSnapshotSplit snapshotSplit : assignedSplits.values()) {
+            this.enumeratorMetrics
+                    .getTableMetrics(snapshotSplit.getTableId())
+                    .addProcessedSplit(snapshotSplit.splitId());
+        }
+        for (String splitId : splitFinishedOffsets.keySet()) {
+            String[] splits = splitId.split(":");

Review Comment:
   Processed



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

Reply via email to