dailai commented on code in PR #7003:
URL: https://github.com/apache/seatunnel/pull/7003#discussion_r1643574218


##########
seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceStreamFetcher.java:
##########
@@ -349,4 +287,99 @@ private void configureFilter() {
         this.maxSplitHighWatermarkMap = tableIdBinlogPositionMap;
         this.pureBinlogPhaseTables.clear();
     }
+
+    class SchemaChangeStreamSplitter {
+        private List<SourceRecords> blockSet;
+        private List<SourceRecord> currentBlock;
+        private SourceRecord previousRecord;
+
+        void reset() {

Review Comment:
   Here use the method of reset is a bit odd, because every time is newed a new 
SchemaChangeStreamSplitter instance of a batch of data processing.
   
   A choice between two suggestions:
   
   Suggested 1: Keep reset method, don't need new  a SchemaChangeStreamSplitter 
instance every time, sharing the same class instance.
   
   Suggested 2: Change the reset method to a constructor.
   
   



##########
seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceStreamFetcher.java:
##########
@@ -349,4 +287,99 @@ private void configureFilter() {
         this.maxSplitHighWatermarkMap = tableIdBinlogPositionMap;
         this.pureBinlogPhaseTables.clear();
     }
+
+    class SchemaChangeStreamSplitter {
+        private List<SourceRecords> blockSet;
+        private List<SourceRecord> currentBlock;
+        private SourceRecord previousRecord;
+
+        void reset() {
+            blockSet = new ArrayList<>();
+            currentBlock = new ArrayList<>();
+            previousRecord = null;
+        }
+
+        public Iterator<SourceRecords> split(List<DataChangeEvent> 
batchEvents) {
+            reset();
+
+            for (int i = 0; i < batchEvents.size(); i++) {
+                DataChangeEvent event = batchEvents.get(i);
+                SourceRecord currentRecord = event.getRecord();
+                if (!shouldEmit(currentRecord)) {
+                    continue;
+                }
+
+                if (SourceRecordUtils.isSchemaChangeEvent(currentRecord)) {
+                    if (!schemaChangeResolver.support(currentRecord)) {
+                        continue;
+                    }
+
+                    if (previousRecord == null) {
+                        // add schema-change-before to first
+                        currentBlock.add(
+                                
WatermarkEvent.createSchemaChangeBeforeWatermark(currentRecord));
+                        flipBlock();
+
+                        currentBlock.add(currentRecord);
+                    } else if 
(SourceRecordUtils.isSchemaChangeEvent(previousRecord)) {
+                        currentBlock.add(currentRecord);
+                    } else {
+                        currentBlock.add(
+                                
WatermarkEvent.createSchemaChangeBeforeWatermark(currentRecord));
+                        flipBlock();
+
+                        currentBlock.add(currentRecord);
+                    }
+                } else if (SourceRecordUtils.isDataChangeRecord(currentRecord)
+                        || SourceRecordUtils.isHeartbeatRecord(currentRecord)) 
{
+                    if (previousRecord == null
+                            || 
SourceRecordUtils.isDataChangeRecord(previousRecord)
+                            || 
SourceRecordUtils.isHeartbeatRecord(previousRecord)) {
+                        currentBlock.add(currentRecord);
+                    } else {
+                        endBlock(previousRecord);
+                        flipBlock();
+
+                        currentBlock.add(currentRecord);
+                    }
+                }
+
+                previousRecord = currentRecord;
+                if (i == batchEvents.size() - 1) {
+                    endBlock(currentRecord);
+                    flipBlock();
+                }
+            }
+
+            endLastBlock(previousRecord);
+
+            if (blockSet.size() > 1) {
+                log.debug(
+                        "Split events stream into {} batches and mark schema 
change checkpoint",
+                        blockSet.size());
+            }
+
+            return blockSet.iterator();
+        }
+
+        void flipBlock() {
+            if (currentBlock != null && !currentBlock.isEmpty()) {
+                blockSet.add(new SourceRecords(currentBlock));
+            }
+            currentBlock = new ArrayList<>();

Review Comment:
   Here, a new ArrayList is created each time whether currentBlock is empty or 
not, so if currentBlock is empty, don't you need to create a new one.



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