You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by GitBox <gi...@apache.org> on 2022/12/29 13:02:56 UTC

[GitHub] [shardingsphere] sandynz commented on a diff in pull request #23168: Add CDC server push data record implementation and refactor pipeline importer

sandynz commented on code in PR #23168:
URL: https://github.com/apache/shardingsphere/pull/23168#discussion_r1058910337


##########
kernel/data-pipeline/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/core/importer/CDCImporter.java:
##########
@@ -19,42 +19,101 @@
 
 import lombok.AccessLevel;
 import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.data.pipeline.api.config.ImporterConfiguration;
 import org.apache.shardingsphere.data.pipeline.api.executor.AbstractLifecycleExecutor;
 import org.apache.shardingsphere.data.pipeline.api.importer.Importer;
 import org.apache.shardingsphere.data.pipeline.api.ingest.channel.PipelineChannel;
+import org.apache.shardingsphere.data.pipeline.api.ingest.record.FinishedRecord;
+import org.apache.shardingsphere.data.pipeline.api.ingest.record.PlaceholderRecord;
+import org.apache.shardingsphere.data.pipeline.api.ingest.record.Record;
+import org.apache.shardingsphere.data.pipeline.api.job.JobOperationType;
 import org.apache.shardingsphere.data.pipeline.api.job.progress.listener.PipelineJobProgressListener;
+import org.apache.shardingsphere.data.pipeline.api.job.progress.listener.PipelineJobProgressUpdatedParameter;
+import org.apache.shardingsphere.data.pipeline.cdc.core.importer.connector.CDCImporterConnector;
+import org.apache.shardingsphere.data.pipeline.spi.importer.ImporterType;
 import org.apache.shardingsphere.data.pipeline.spi.importer.connector.ImporterConnector;
 import org.apache.shardingsphere.data.pipeline.spi.ratelimit.JobRateLimitAlgorithm;
 
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
 /**
  * CDC importer.
  */
+@Slf4j
 public final class CDCImporter extends AbstractLifecycleExecutor implements Importer {
     
     @Getter(AccessLevel.PROTECTED)
     private final ImporterConfiguration importerConfig;
     
     private final PipelineChannel channel;
     
+    private final CDCImporterConnector importerConnector;
+    
     private final PipelineJobProgressListener jobProgressListener;
     
+    @Getter
+    private final ImporterType importerType;
+    
     private final JobRateLimitAlgorithm rateLimitAlgorithm;
     
-    public CDCImporter(final ImporterConfiguration importerConfig, final ImporterConnector importerConnector, final PipelineChannel channel, final PipelineJobProgressListener jobProgressListener) {
+    public CDCImporter(final ImporterConfiguration importerConfig, final ImporterConnector importerConnector, final PipelineChannel channel, final PipelineJobProgressListener jobProgressListener,
+                       final ImporterType importerType) {
         this.importerConfig = importerConfig;
         rateLimitAlgorithm = importerConfig.getRateLimitAlgorithm();
         this.channel = channel;
+        this.importerConnector = (CDCImporterConnector) importerConnector;
         this.jobProgressListener = jobProgressListener;
+        this.importerType = importerType;
     }
     
     @Override
     protected void runBlocking() {
-        // TODO to be implemented
+        int batchSize = importerConfig.getBatchSize();
+        if (ImporterType.INCREMENTAL == importerType) {
+            importerConnector.sendIncrementalStartEvent(batchSize);
+        }
+        while (isRunning()) {
+            List<Record> records = channel.fetchRecords(batchSize, 3);
+            if (null != records && !records.isEmpty()) {
+                List<Record> recordList = records.stream().filter(each -> !(each instanceof PlaceholderRecord)).collect(Collectors.toList());
+                try {
+                    processDataRecords(recordList);
+                } catch (final SQLException ex) {
+                    log.error("process data records failed", ex);
+                }
+                if (FinishedRecord.class.equals(records.get(records.size() - 1).getClass())) {
+                    break;
+                }
+            }
+        }
+    }
+    
+    private void processDataRecords(final List<Record> recordList) throws SQLException {
+        if (null == recordList || recordList.isEmpty()) {
+            return;
+        }
+        if (null != rateLimitAlgorithm) {
+            rateLimitAlgorithm.intercept(JobOperationType.INSERT, 1);
+        }
+        importerConnector.write(recordList, this, importerType);
+    }
+    
+    /**
+     * Ack with last data record.
+     *
+     * @param lastDataRecord last data record
+     */
+    public void ackWithLastDataRecord(final Record lastDataRecord) {
+        channel.ack(Collections.singletonList(lastDataRecord));
+        jobProgressListener.onProgressUpdated(new PipelineJobProgressUpdatedParameter(0));

Review Comment:
   Why `processedRecordsCount` is `0`



##########
kernel/data-pipeline/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/holder/CDCAckHolder.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.shardingsphere.data.pipeline.cdc.holder;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.data.pipeline.api.ingest.record.Record;
+import org.apache.shardingsphere.data.pipeline.cdc.core.importer.CDCImporter;
+
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * CDC ack holder.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class CDCAckHolder {
+    
+    private static final CDCAckHolder INSTANCE = new CDCAckHolder();
+    
+    private final Map<String, Map<CDCImporter, Record>> ackIdImporterMap = new ConcurrentHashMap<>();

Review Comment:
   It needs some cleanup mechanism, else it might cause memory leak



##########
kernel/data-pipeline/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/core/importer/CDCImporter.java:
##########
@@ -19,42 +19,101 @@
 
 import lombok.AccessLevel;
 import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.data.pipeline.api.config.ImporterConfiguration;
 import org.apache.shardingsphere.data.pipeline.api.executor.AbstractLifecycleExecutor;
 import org.apache.shardingsphere.data.pipeline.api.importer.Importer;
 import org.apache.shardingsphere.data.pipeline.api.ingest.channel.PipelineChannel;
+import org.apache.shardingsphere.data.pipeline.api.ingest.record.FinishedRecord;
+import org.apache.shardingsphere.data.pipeline.api.ingest.record.PlaceholderRecord;
+import org.apache.shardingsphere.data.pipeline.api.ingest.record.Record;
+import org.apache.shardingsphere.data.pipeline.api.job.JobOperationType;
 import org.apache.shardingsphere.data.pipeline.api.job.progress.listener.PipelineJobProgressListener;
+import org.apache.shardingsphere.data.pipeline.api.job.progress.listener.PipelineJobProgressUpdatedParameter;
+import org.apache.shardingsphere.data.pipeline.cdc.core.importer.connector.CDCImporterConnector;
+import org.apache.shardingsphere.data.pipeline.spi.importer.ImporterType;
 import org.apache.shardingsphere.data.pipeline.spi.importer.connector.ImporterConnector;
 import org.apache.shardingsphere.data.pipeline.spi.ratelimit.JobRateLimitAlgorithm;
 
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
 /**
  * CDC importer.
  */
+@Slf4j
 public final class CDCImporter extends AbstractLifecycleExecutor implements Importer {
     
     @Getter(AccessLevel.PROTECTED)
     private final ImporterConfiguration importerConfig;
     
     private final PipelineChannel channel;
     
+    private final CDCImporterConnector importerConnector;
+    
     private final PipelineJobProgressListener jobProgressListener;
     
+    @Getter
+    private final ImporterType importerType;
+    
     private final JobRateLimitAlgorithm rateLimitAlgorithm;
     
-    public CDCImporter(final ImporterConfiguration importerConfig, final ImporterConnector importerConnector, final PipelineChannel channel, final PipelineJobProgressListener jobProgressListener) {
+    public CDCImporter(final ImporterConfiguration importerConfig, final ImporterConnector importerConnector, final PipelineChannel channel, final PipelineJobProgressListener jobProgressListener,
+                       final ImporterType importerType) {
         this.importerConfig = importerConfig;
         rateLimitAlgorithm = importerConfig.getRateLimitAlgorithm();
         this.channel = channel;
+        this.importerConnector = (CDCImporterConnector) importerConnector;
         this.jobProgressListener = jobProgressListener;
+        this.importerType = importerType;
     }
     
     @Override
     protected void runBlocking() {
-        // TODO to be implemented
+        int batchSize = importerConfig.getBatchSize();
+        if (ImporterType.INCREMENTAL == importerType) {
+            importerConnector.sendIncrementalStartEvent(batchSize);
+        }
+        while (isRunning()) {
+            List<Record> records = channel.fetchRecords(batchSize, 3);
+            if (null != records && !records.isEmpty()) {
+                List<Record> recordList = records.stream().filter(each -> !(each instanceof PlaceholderRecord)).collect(Collectors.toList());
+                try {
+                    processDataRecords(recordList);
+                } catch (final SQLException ex) {
+                    log.error("process data records failed", ex);
+                }

Review Comment:
   SQLException is ignored, it might cause issue



##########
kernel/data-pipeline/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/holder/CDCAckHolder.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.shardingsphere.data.pipeline.cdc.holder;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.data.pipeline.api.ingest.record.Record;
+import org.apache.shardingsphere.data.pipeline.cdc.core.importer.CDCImporter;
+
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * CDC ack holder.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class CDCAckHolder {
+    
+    private static final CDCAckHolder INSTANCE = new CDCAckHolder();
+    
+    private final Map<String, Map<CDCImporter, Record>> ackIdImporterMap = new ConcurrentHashMap<>();
+    
+    /**
+     * the ack of CDC.
+     *
+     * @param ackId ack id
+     */
+    public void ack(final String ackId) {
+        Map<CDCImporter, Record> importerDataRecordMap = ackIdImporterMap.get(ackId);
+        if (null != importerDataRecordMap) {
+            importerDataRecordMap.forEach(CDCImporter::ackWithLastDataRecord);
+        }
+    }
+    
+    /**
+     * Bind ack id.
+     *
+     * @param importerDataRecordMap import data record map
+     * @return ack id
+     */
+    public String bindAckId(final Map<CDCImporter, Record> importerDataRecordMap) {
+        String ackId = generateAckId();
+        ackIdImporterMap.put(ackId, importerDataRecordMap);
+        return ackId;
+    }

Review Comment:
   1, `ackId` should be `result`
   
   2, Seems it might need to persist to registry center, in case of current proxy instance crash. We could add TODO for now if it's necessary
   



##########
kernel/data-pipeline/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/core/importer/connector/CDCImporterConnector.java:
##########
@@ -18,24 +18,201 @@
 package org.apache.shardingsphere.data.pipeline.cdc.core.importer.connector;
 
 import io.netty.channel.Channel;
+import lombok.Getter;
 import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.shardingsphere.data.pipeline.api.ingest.record.DataRecord;
+import org.apache.shardingsphere.data.pipeline.api.ingest.record.FinishedRecord;
+import org.apache.shardingsphere.data.pipeline.api.ingest.record.Record;
+import org.apache.shardingsphere.data.pipeline.cdc.core.importer.CDCImporter;
+import org.apache.shardingsphere.data.pipeline.cdc.generator.CDCResponseGenerator;
+import org.apache.shardingsphere.data.pipeline.cdc.holder.CDCAckHolder;
+import org.apache.shardingsphere.data.pipeline.cdc.protocol.response.DataRecordResult;
+import org.apache.shardingsphere.data.pipeline.cdc.util.DataRecordResultConvertUtil;
+import org.apache.shardingsphere.data.pipeline.core.util.ThreadUtil;
+import org.apache.shardingsphere.data.pipeline.spi.importer.ImporterType;
 import org.apache.shardingsphere.data.pipeline.spi.importer.connector.ImporterConnector;
 
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
 /**
  * CDC importer connector.
  */
-@RequiredArgsConstructor
+@Slf4j
 public final class CDCImporterConnector implements ImporterConnector {
     
+    private static final long DEFAULT_TIMEOUT_MILLISECONDS = 200L;
+    
+    private final Lock lock = new ReentrantLock();
+    
+    private final Condition condition = lock.newCondition();
+    
+    @Setter
+    private volatile boolean running = true;
+    
+    @Getter
+    private final String database;
+    
     private final Channel channel;
     
+    private final int jobShardingCount;
+    
+    private final Comparator<DataRecord> dataRecordComparator;
+    
+    private final Map<String, String> tableNameSchemaMap = new HashMap<>();
+    
+    private final Map<CDCImporter, BlockingQueue<Record>> incrementalRecordMap = new ConcurrentHashMap<>();
+    
+    private final AtomicInteger runningIncrementalTaskCount = new AtomicInteger(0);
+    
+    private Thread incrementalImporterTask;
+    
+    public CDCImporterConnector(final Channel channel, final String database, final int jobShardingCount, final List<String> tableNames, final Comparator<DataRecord> dataRecordComparator) {
+        this.channel = channel;
+        this.database = database;
+        this.jobShardingCount = jobShardingCount;
+        tableNames.stream().filter(each -> each.contains(".")).forEach(each -> {
+            String[] split = each.split("\\.");
+            tableNameSchemaMap.put(split[0], split[1]);
+        });
+        this.dataRecordComparator = dataRecordComparator;
+    }
+    
     @Override
     public Object getConnector() {
         return channel;
     }
     
+    /**
+     * Write data record into channel.
+     *
+     * @param recordList data records
+     * @param cdcImporter cdc importer
+     * @param importerType importer type
+     */
+    public void write(final List<Record> recordList, final CDCImporter cdcImporter, final ImporterType importerType) {
+        if (ImporterType.INVENTORY == importerType || null == dataRecordComparator) {
+            Map<CDCImporter, Record> importerDataRecordMap = new HashMap<>();
+            importerDataRecordMap.put(cdcImporter, recordList.get(recordList.size() - 1));
+            writeImmediately(recordList, importerDataRecordMap);
+        } else if (ImporterType.INCREMENTAL == importerType) {
+            writeIntoQueue(recordList, cdcImporter);
+        }
+    }
+    
+    private void writeImmediately(final List<Record> recordList, final Map<CDCImporter, Record> importerDataRecordMap) {
+        while (!channel.isWritable() && channel.isActive()) {
+            doAwait();
+        }
+        List<DataRecordResult.Record> records = new LinkedList<>();
+        for (Record each : recordList) {
+            if (each instanceof DataRecord) {
+                DataRecord dataRecord = (DataRecord) each;
+                records.add(DataRecordResultConvertUtil.convertDataRecordToRecord(database, tableNameSchemaMap.get(dataRecord.getTableName()), dataRecord));
+            }
+        }
+        String ackId = CDCAckHolder.getInstance().bindAckId(importerDataRecordMap);
+        DataRecordResult dataRecordResult = DataRecordResult.newBuilder().addAllRecords(records).setAckId(ackId).build();
+        channel.writeAndFlush(CDCResponseGenerator.succeedBuilder("").setDataRecordResult(dataRecordResult).build());
+    }
+    
+    private void doAwait() {
+        lock.lock();
+        try {
+            condition.await(DEFAULT_TIMEOUT_MILLISECONDS, TimeUnit.MILLISECONDS);
+        } catch (final InterruptedException ignored) {
+        } finally {
+            lock.unlock();
+        }
+    }
+    
+    @SneakyThrows(InterruptedException.class)
+    private void writeIntoQueue(final List<Record> dataRecords, final CDCImporter cdcImporter) {
+        BlockingQueue<Record> blockingQueue = incrementalRecordMap.computeIfAbsent(cdcImporter, ignored -> new ArrayBlockingQueue<>(500));
+        for (Record each : dataRecords) {
+            blockingQueue.put(each);
+        }
+    }
+    
+    /**
+     * Send finished record event.
+     *
+     * @param batchSize batch size
+     */
+    public void sendIncrementalStartEvent(final int batchSize) {
+        int count = runningIncrementalTaskCount.incrementAndGet();
+        if (count < jobShardingCount || null == dataRecordComparator) {
+            return;
+        }
+        log.debug("start CDC incremental importer");
+        if (null == incrementalImporterTask) {
+            incrementalImporterTask = new Thread(new CDCIncrementalImporterTask(batchSize));
+            incrementalImporterTask.start();
+        }
+    }
+    
     @Override
     public String getType() {
         return "CDC";
     }
+    
+    @RequiredArgsConstructor
+    private final class CDCIncrementalImporterTask implements Runnable {
+        
+        private final int batchSize;
+        
+        @Override
+        public void run() {
+            while (running && null != dataRecordComparator) {
+                int index = 0;
+                List<Record> dataRecords = new LinkedList<>();
+                Map<CDCImporter, Record> pipelineChannelPositions = new HashMap<>(incrementalRecordMap.size(), 1);
+                while (index < batchSize) {

Review Comment:
   Could we just use for loop to replace while? Then `index` changes will be in the same line



##########
kernel/data-pipeline/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/util/ColumnValueConvertUtil.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.shardingsphere.data.pipeline.cdc.util;
+
+import com.google.protobuf.BoolValue;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.BytesValue;
+import com.google.protobuf.DoubleValue;
+import com.google.protobuf.FloatValue;
+import com.google.protobuf.Int32Value;
+import com.google.protobuf.Int64Value;
+import com.google.protobuf.Message;
+import com.google.protobuf.StringValue;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.shardingsphere.data.pipeline.cdc.protocol.response.BigDecimalValue;
+import org.apache.shardingsphere.data.pipeline.cdc.protocol.response.BigIntegerValue;
+import org.apache.shardingsphere.data.pipeline.cdc.protocol.response.ClobValue;
+import org.apache.shardingsphere.data.pipeline.cdc.protocol.response.NullValue;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.sql.Clob;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.ZonedDateTime;
+import java.util.Date;
+
+/**
+ * Column value convert util.
+ */
+@Slf4j
+public final class ColumnValueConvertUtil {
+    
+    /**
+     * Convert java object to protobuf message.
+     *
+     * @param object object
+     * @return protobuf message
+     */
+    public static Message convertToProtobufMessage(final Object object) {
+        if (null == object) {
+            return NullValue.newBuilder().build();
+        }
+        if (object instanceof Integer) {
+            return Int32Value.newBuilder().setValue((int) object).build();
+        }
+        if (object instanceof Short) {
+            return Int32Value.newBuilder().setValue(((Short) object).intValue()).build();
+        }
+        if (object instanceof Byte) {
+            return Int32Value.newBuilder().setValue(((Byte) object).intValue()).build();
+        }
+        if (object instanceof Long) {
+            return Int64Value.newBuilder().setValue((long) object).build();
+        }
+        if (object instanceof BigInteger) {
+            return BigIntegerValue.newBuilder().setValue(ByteString.copyFrom(((BigInteger) object).toByteArray())).build();
+        }
+        if (object instanceof Float) {
+            return FloatValue.newBuilder().setValue((float) object).build();
+        }
+        if (object instanceof Double) {
+            return DoubleValue.newBuilder().setValue((double) object).build();
+        }
+        if (object instanceof BigDecimal) {
+            return BigDecimalValue.newBuilder().setValue(object.toString()).build();
+        }
+        if (object instanceof String) {
+            return StringValue.newBuilder().setValue(object.toString()).build();
+        }
+        if (object instanceof Boolean) {
+            return BoolValue.newBuilder().setValue((boolean) object).build();
+        }
+        if (object instanceof byte[]) {
+            return BytesValue.newBuilder().setValue(ByteString.copyFrom((byte[]) object)).build();
+        }
+        if (object instanceof Date) {
+            return converToProtobufTimestamp((Date) object);
+        }
+        if (object instanceof LocalDateTime) {
+            return converToProtobufTimestamp(Timestamp.valueOf((LocalDateTime) object));
+        }
+        if (object instanceof LocalDate) {
+            return converToProtobufTimestamp(Timestamp.valueOf(((LocalDate) object).atStartOfDay()));
+        }
+        if (object instanceof ZonedDateTime) {
+            return converToProtobufTimestamp(Timestamp.valueOf(((ZonedDateTime) object).toLocalDateTime()));
+        }
+        if (object instanceof Instant) {
+            Instant instant = (Instant) object;
+            return com.google.protobuf.Timestamp.newBuilder().setSeconds(instant.getEpochSecond()).setNanos(instant.getNano()).build();
+        }
+        if (object instanceof Clob) {
+            Clob clob = (Clob) object;
+            try {
+                return ClobValue.newBuilder().setValue(clob.getSubString(1, (int) clob.length())).build();
+            } catch (final SQLException ex) {
+                log.error("get clob length failed", ex);
+                throw new RuntimeException(ex);
+            }
+        }
+        log.warn(" {} can't convert to protobuf message, value {}", object.getClass().getName(), object);
+        throw new UnsupportedOperationException(String.format("Not support convert %s to protobuf message now", object.getClass()));

Review Comment:
   1, Are all date and time types included? e.g. LocalTime etc
   
   2, Is there any other types? e.g. Blob
   
   3, Is there common Message for other types, similar as Object or Any
   



##########
kernel/data-pipeline/cdc/core/src/test/java/org/apache/shardingsphere/data/pipeline/cdc/util/ColumnValueConvertUtilTest.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.shardingsphere.data.pipeline.cdc.util;
+
+import com.google.protobuf.BoolValue;
+import com.google.protobuf.BytesValue;
+import com.google.protobuf.DoubleValue;
+import com.google.protobuf.FloatValue;
+import com.google.protobuf.Int32Value;
+import com.google.protobuf.Int64Value;
+import com.google.protobuf.Message;
+import com.google.protobuf.StringValue;
+import org.apache.shardingsphere.data.pipeline.cdc.protocol.response.BigDecimalValue;
+import org.apache.shardingsphere.data.pipeline.cdc.protocol.response.BigIntegerValue;
+import org.apache.shardingsphere.data.pipeline.cdc.protocol.response.NullValue;
+import org.junit.Test;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.util.Date;
+
+import static org.junit.Assert.assertTrue;
+
+public final class ColumnValueConvertUtilTest {
+    
+    @Test
+    public void assertConvertToProtobufMessage() {

Review Comment:
   It needs to assert actual value, `instanceof` is not enough



##########
kernel/data-pipeline/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/util/DataRecordResultConvertUtil.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.shardingsphere.data.pipeline.cdc.util;
+
+import com.google.common.base.Strings;
+import com.google.protobuf.Any;
+import org.apache.shardingsphere.data.pipeline.api.ingest.record.Column;
+import org.apache.shardingsphere.data.pipeline.api.ingest.record.DataRecord;
+import org.apache.shardingsphere.data.pipeline.cdc.protocol.response.DataRecordResult;
+import org.apache.shardingsphere.data.pipeline.cdc.protocol.response.DataRecordResult.Record;
+import org.apache.shardingsphere.data.pipeline.cdc.protocol.response.DataRecordResult.Record.DataChangeType;
+import org.apache.shardingsphere.data.pipeline.cdc.protocol.response.DataRecordResult.Record.TableMetaData;
+import org.apache.shardingsphere.data.pipeline.core.ingest.IngestDataChangeType;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Data record result convert util.
+ */
+public final class DataRecordResultConvertUtil {
+    
+    /**
+     * Convert data record to record.
+     *
+     * @param database database
+     * @param schema schema
+     * @param dataRecord data record
+     * @return record
+     */
+    public static Record convertDataRecordToRecord(final String database, final String schema, final DataRecord dataRecord) {
+        Map<String, Any> beforeMap = new HashMap<>();
+        Map<String, Any> afterMap = new HashMap<>();
+        for (Column column : dataRecord.getColumns()) {
+            beforeMap.put(column.getName(), Any.pack(ColumnValueConvertUtil.convertToProtobufMessage(column.getOldValue())));
+            afterMap.put(column.getName(), Any.pack(ColumnValueConvertUtil.convertToProtobufMessage(column.getValue())));
+        }
+        TableMetaData metaData = TableMetaData.newBuilder().setDatabase(database).setSchema(Strings.nullToEmpty(schema)).setTableName(dataRecord.getTableName()).build();
+        DataChangeType dataChangeType = DataChangeType.UNKNOWN;
+        if (IngestDataChangeType.INSERT.equals(dataRecord.getType())) {
+            dataChangeType = DataChangeType.INSERT;
+        } else if (IngestDataChangeType.UPDATE.equals(dataRecord.getType())) {
+            dataChangeType = DataChangeType.INSERT;
+        } else if (IngestDataChangeType.DELETE.equals(dataRecord.getType())) {
+            dataChangeType = DataChangeType.INSERT;

Review Comment:
   `dataChangeType` should be `UPDATE` or `DELETE`



##########
kernel/data-pipeline/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/core/importer/connector/CDCImporterConnector.java:
##########
@@ -18,24 +18,201 @@
 package org.apache.shardingsphere.data.pipeline.cdc.core.importer.connector;
 
 import io.netty.channel.Channel;
+import lombok.Getter;
 import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.shardingsphere.data.pipeline.api.ingest.record.DataRecord;
+import org.apache.shardingsphere.data.pipeline.api.ingest.record.FinishedRecord;
+import org.apache.shardingsphere.data.pipeline.api.ingest.record.Record;
+import org.apache.shardingsphere.data.pipeline.cdc.core.importer.CDCImporter;
+import org.apache.shardingsphere.data.pipeline.cdc.generator.CDCResponseGenerator;
+import org.apache.shardingsphere.data.pipeline.cdc.holder.CDCAckHolder;
+import org.apache.shardingsphere.data.pipeline.cdc.protocol.response.DataRecordResult;
+import org.apache.shardingsphere.data.pipeline.cdc.util.DataRecordResultConvertUtil;
+import org.apache.shardingsphere.data.pipeline.core.util.ThreadUtil;
+import org.apache.shardingsphere.data.pipeline.spi.importer.ImporterType;
 import org.apache.shardingsphere.data.pipeline.spi.importer.connector.ImporterConnector;
 
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
 /**
  * CDC importer connector.
  */
-@RequiredArgsConstructor
+@Slf4j
 public final class CDCImporterConnector implements ImporterConnector {
     
+    private static final long DEFAULT_TIMEOUT_MILLISECONDS = 200L;
+    
+    private final Lock lock = new ReentrantLock();
+    
+    private final Condition condition = lock.newCondition();
+    
+    @Setter
+    private volatile boolean running = true;
+    
+    @Getter
+    private final String database;
+    
     private final Channel channel;
     
+    private final int jobShardingCount;
+    
+    private final Comparator<DataRecord> dataRecordComparator;
+    
+    private final Map<String, String> tableNameSchemaMap = new HashMap<>();
+    
+    private final Map<CDCImporter, BlockingQueue<Record>> incrementalRecordMap = new ConcurrentHashMap<>();
+    
+    private final AtomicInteger runningIncrementalTaskCount = new AtomicInteger(0);
+    
+    private Thread incrementalImporterTask;
+    
+    public CDCImporterConnector(final Channel channel, final String database, final int jobShardingCount, final List<String> tableNames, final Comparator<DataRecord> dataRecordComparator) {
+        this.channel = channel;
+        this.database = database;
+        this.jobShardingCount = jobShardingCount;
+        tableNames.stream().filter(each -> each.contains(".")).forEach(each -> {
+            String[] split = each.split("\\.");
+            tableNameSchemaMap.put(split[0], split[1]);
+        });
+        this.dataRecordComparator = dataRecordComparator;
+    }
+    
     @Override
     public Object getConnector() {
         return channel;
     }
     
+    /**
+     * Write data record into channel.
+     *
+     * @param recordList data records
+     * @param cdcImporter cdc importer
+     * @param importerType importer type
+     */
+    public void write(final List<Record> recordList, final CDCImporter cdcImporter, final ImporterType importerType) {
+        if (ImporterType.INVENTORY == importerType || null == dataRecordComparator) {
+            Map<CDCImporter, Record> importerDataRecordMap = new HashMap<>();
+            importerDataRecordMap.put(cdcImporter, recordList.get(recordList.size() - 1));
+            writeImmediately(recordList, importerDataRecordMap);
+        } else if (ImporterType.INCREMENTAL == importerType) {
+            writeIntoQueue(recordList, cdcImporter);
+        }
+    }
+    
+    private void writeImmediately(final List<Record> recordList, final Map<CDCImporter, Record> importerDataRecordMap) {
+        while (!channel.isWritable() && channel.isActive()) {
+            doAwait();
+        }
+        List<DataRecordResult.Record> records = new LinkedList<>();
+        for (Record each : recordList) {
+            if (each instanceof DataRecord) {
+                DataRecord dataRecord = (DataRecord) each;
+                records.add(DataRecordResultConvertUtil.convertDataRecordToRecord(database, tableNameSchemaMap.get(dataRecord.getTableName()), dataRecord));
+            }
+        }
+        String ackId = CDCAckHolder.getInstance().bindAckId(importerDataRecordMap);
+        DataRecordResult dataRecordResult = DataRecordResult.newBuilder().addAllRecords(records).setAckId(ackId).build();
+        channel.writeAndFlush(CDCResponseGenerator.succeedBuilder("").setDataRecordResult(dataRecordResult).build());
+    }
+    
+    private void doAwait() {
+        lock.lock();
+        try {
+            condition.await(DEFAULT_TIMEOUT_MILLISECONDS, TimeUnit.MILLISECONDS);
+        } catch (final InterruptedException ignored) {
+        } finally {
+            lock.unlock();
+        }
+    }
+    
+    @SneakyThrows(InterruptedException.class)
+    private void writeIntoQueue(final List<Record> dataRecords, final CDCImporter cdcImporter) {
+        BlockingQueue<Record> blockingQueue = incrementalRecordMap.computeIfAbsent(cdcImporter, ignored -> new ArrayBlockingQueue<>(500));
+        for (Record each : dataRecords) {
+            blockingQueue.put(each);
+        }
+    }
+    
+    /**
+     * Send finished record event.
+     *
+     * @param batchSize batch size
+     */
+    public void sendIncrementalStartEvent(final int batchSize) {
+        int count = runningIncrementalTaskCount.incrementAndGet();
+        if (count < jobShardingCount || null == dataRecordComparator) {
+            return;
+        }
+        log.debug("start CDC incremental importer");
+        if (null == incrementalImporterTask) {
+            incrementalImporterTask = new Thread(new CDCIncrementalImporterTask(batchSize));
+            incrementalImporterTask.start();
+        }
+    }
+    
     @Override
     public String getType() {
         return "CDC";
     }
+    
+    @RequiredArgsConstructor
+    private final class CDCIncrementalImporterTask implements Runnable {
+        
+        private final int batchSize;
+        
+        @Override
+        public void run() {
+            while (running && null != dataRecordComparator) {
+                int index = 0;
+                List<Record> dataRecords = new LinkedList<>();
+                Map<CDCImporter, Record> pipelineChannelPositions = new HashMap<>(incrementalRecordMap.size(), 1);
+                while (index < batchSize) {
+                    Map<Record, CDCImporter> recordChannelMap = new HashMap<>(incrementalRecordMap.size(), 1);
+                    for (Entry<CDCImporter, BlockingQueue<Record>> entry : incrementalRecordMap.entrySet()) {
+                        BlockingQueue<Record> blockingQueue = entry.getValue();
+                        if (null == blockingQueue.peek()) {
+                            continue;
+                        }
+                        Record record = blockingQueue.poll();
+                        if (record instanceof FinishedRecord) {
+                            continue;
+                        }
+                        recordChannelMap.put(record, entry.getKey());
+                        pipelineChannelPositions.computeIfAbsent(entry.getKey(), key -> record);
+                    }
+                    List<DataRecord> filterRecord = recordChannelMap.keySet().stream().filter(each -> each instanceof DataRecord).map(each -> (DataRecord) each).collect(Collectors.toList());
+                    if (filterRecord.isEmpty()) {
+                        break;
+                    }
+                    DataRecord minDataRecord = Collections.min(filterRecord, dataRecordComparator);
+                    dataRecords.add(minDataRecord);

Review Comment:
   1, There's `blockingQueue.poll()` from all CDCImporter and put into `recordChannelMap`, but it just add `minDataRecord` into `dataRecords`, seems it missed some records
   
   2, Could we use streaming poll and sort, and remove `Collections.min`?
   
   3, Could we extract these codes into dividual method and add related unit test? It's critical logic codes
   



##########
kernel/data-pipeline/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/core/importer/connector/CDCImporterConnector.java:
##########
@@ -18,24 +18,201 @@
 package org.apache.shardingsphere.data.pipeline.cdc.core.importer.connector;
 
 import io.netty.channel.Channel;
+import lombok.Getter;
 import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.shardingsphere.data.pipeline.api.ingest.record.DataRecord;
+import org.apache.shardingsphere.data.pipeline.api.ingest.record.FinishedRecord;
+import org.apache.shardingsphere.data.pipeline.api.ingest.record.Record;
+import org.apache.shardingsphere.data.pipeline.cdc.core.importer.CDCImporter;
+import org.apache.shardingsphere.data.pipeline.cdc.generator.CDCResponseGenerator;
+import org.apache.shardingsphere.data.pipeline.cdc.holder.CDCAckHolder;
+import org.apache.shardingsphere.data.pipeline.cdc.protocol.response.DataRecordResult;
+import org.apache.shardingsphere.data.pipeline.cdc.util.DataRecordResultConvertUtil;
+import org.apache.shardingsphere.data.pipeline.core.util.ThreadUtil;
+import org.apache.shardingsphere.data.pipeline.spi.importer.ImporterType;
 import org.apache.shardingsphere.data.pipeline.spi.importer.connector.ImporterConnector;
 
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
 /**
  * CDC importer connector.
  */
-@RequiredArgsConstructor
+@Slf4j
 public final class CDCImporterConnector implements ImporterConnector {
     
+    private static final long DEFAULT_TIMEOUT_MILLISECONDS = 200L;
+    
+    private final Lock lock = new ReentrantLock();
+    
+    private final Condition condition = lock.newCondition();
+    
+    @Setter
+    private volatile boolean running = true;
+    
+    @Getter
+    private final String database;
+    
     private final Channel channel;
     
+    private final int jobShardingCount;
+    
+    private final Comparator<DataRecord> dataRecordComparator;
+    
+    private final Map<String, String> tableNameSchemaMap = new HashMap<>();
+    
+    private final Map<CDCImporter, BlockingQueue<Record>> incrementalRecordMap = new ConcurrentHashMap<>();
+    
+    private final AtomicInteger runningIncrementalTaskCount = new AtomicInteger(0);
+    
+    private Thread incrementalImporterTask;
+    
+    public CDCImporterConnector(final Channel channel, final String database, final int jobShardingCount, final List<String> tableNames, final Comparator<DataRecord> dataRecordComparator) {
+        this.channel = channel;
+        this.database = database;
+        this.jobShardingCount = jobShardingCount;
+        tableNames.stream().filter(each -> each.contains(".")).forEach(each -> {
+            String[] split = each.split("\\.");
+            tableNameSchemaMap.put(split[0], split[1]);
+        });
+        this.dataRecordComparator = dataRecordComparator;
+    }
+    
     @Override
     public Object getConnector() {
         return channel;
     }
     
+    /**
+     * Write data record into channel.
+     *
+     * @param recordList data records
+     * @param cdcImporter cdc importer
+     * @param importerType importer type
+     */
+    public void write(final List<Record> recordList, final CDCImporter cdcImporter, final ImporterType importerType) {
+        if (ImporterType.INVENTORY == importerType || null == dataRecordComparator) {
+            Map<CDCImporter, Record> importerDataRecordMap = new HashMap<>();
+            importerDataRecordMap.put(cdcImporter, recordList.get(recordList.size() - 1));
+            writeImmediately(recordList, importerDataRecordMap);
+        } else if (ImporterType.INCREMENTAL == importerType) {
+            writeIntoQueue(recordList, cdcImporter);
+        }
+    }
+    
+    private void writeImmediately(final List<Record> recordList, final Map<CDCImporter, Record> importerDataRecordMap) {
+        while (!channel.isWritable() && channel.isActive()) {
+            doAwait();
+        }

Review Comment:
   How to break when running is false



##########
kernel/data-pipeline/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/holder/CDCAckHolder.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.shardingsphere.data.pipeline.cdc.holder;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.data.pipeline.api.ingest.record.Record;
+import org.apache.shardingsphere.data.pipeline.cdc.core.importer.CDCImporter;
+
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * CDC ack holder.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class CDCAckHolder {

Review Comment:
   It needs some unit tests



##########
kernel/data-pipeline/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/core/importer/CDCImporter.java:
##########
@@ -19,42 +19,101 @@
 
 import lombok.AccessLevel;
 import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.data.pipeline.api.config.ImporterConfiguration;
 import org.apache.shardingsphere.data.pipeline.api.executor.AbstractLifecycleExecutor;
 import org.apache.shardingsphere.data.pipeline.api.importer.Importer;
 import org.apache.shardingsphere.data.pipeline.api.ingest.channel.PipelineChannel;
+import org.apache.shardingsphere.data.pipeline.api.ingest.record.FinishedRecord;
+import org.apache.shardingsphere.data.pipeline.api.ingest.record.PlaceholderRecord;
+import org.apache.shardingsphere.data.pipeline.api.ingest.record.Record;
+import org.apache.shardingsphere.data.pipeline.api.job.JobOperationType;
 import org.apache.shardingsphere.data.pipeline.api.job.progress.listener.PipelineJobProgressListener;
+import org.apache.shardingsphere.data.pipeline.api.job.progress.listener.PipelineJobProgressUpdatedParameter;
+import org.apache.shardingsphere.data.pipeline.cdc.core.importer.connector.CDCImporterConnector;
+import org.apache.shardingsphere.data.pipeline.spi.importer.ImporterType;
 import org.apache.shardingsphere.data.pipeline.spi.importer.connector.ImporterConnector;
 import org.apache.shardingsphere.data.pipeline.spi.ratelimit.JobRateLimitAlgorithm;
 
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
 /**
  * CDC importer.
  */
+@Slf4j
 public final class CDCImporter extends AbstractLifecycleExecutor implements Importer {
     
     @Getter(AccessLevel.PROTECTED)
     private final ImporterConfiguration importerConfig;
     
     private final PipelineChannel channel;
     
+    private final CDCImporterConnector importerConnector;
+    
     private final PipelineJobProgressListener jobProgressListener;
     
+    @Getter
+    private final ImporterType importerType;
+    
     private final JobRateLimitAlgorithm rateLimitAlgorithm;
     
-    public CDCImporter(final ImporterConfiguration importerConfig, final ImporterConnector importerConnector, final PipelineChannel channel, final PipelineJobProgressListener jobProgressListener) {
+    public CDCImporter(final ImporterConfiguration importerConfig, final ImporterConnector importerConnector, final PipelineChannel channel, final PipelineJobProgressListener jobProgressListener,
+                       final ImporterType importerType) {
         this.importerConfig = importerConfig;
         rateLimitAlgorithm = importerConfig.getRateLimitAlgorithm();
         this.channel = channel;
+        this.importerConnector = (CDCImporterConnector) importerConnector;
         this.jobProgressListener = jobProgressListener;
+        this.importerType = importerType;
     }
     
     @Override
     protected void runBlocking() {
-        // TODO to be implemented
+        int batchSize = importerConfig.getBatchSize();
+        if (ImporterType.INCREMENTAL == importerType) {
+            importerConnector.sendIncrementalStartEvent(batchSize);
+        }
+        while (isRunning()) {
+            List<Record> records = channel.fetchRecords(batchSize, 3);
+            if (null != records && !records.isEmpty()) {
+                List<Record> recordList = records.stream().filter(each -> !(each instanceof PlaceholderRecord)).collect(Collectors.toList());
+                try {
+                    processDataRecords(recordList);
+                } catch (final SQLException ex) {
+                    log.error("process data records failed", ex);
+                }
+                if (FinishedRecord.class.equals(records.get(records.size() - 1).getClass())) {
+                    break;
+                }
+            }
+        }
+    }
+    
+    private void processDataRecords(final List<Record> recordList) throws SQLException {
+        if (null == recordList || recordList.isEmpty()) {
+            return;
+        }
+        if (null != rateLimitAlgorithm) {
+            rateLimitAlgorithm.intercept(JobOperationType.INSERT, 1);
+        }
+        importerConnector.write(recordList, this, importerType);
+    }
+    
+    /**
+     * Ack with last data record.
+     *
+     * @param lastDataRecord last data record
+     */
+    public void ackWithLastDataRecord(final Record lastDataRecord) {
+        channel.ack(Collections.singletonList(lastDataRecord));
+        jobProgressListener.onProgressUpdated(new PipelineJobProgressUpdatedParameter(0));
     }
     
     @Override
     protected void doStop() {
-        // TODO to be implemented
+        

Review Comment:
   Empty line could be removed



-- 
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: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org