sandynz commented on code in PR #23168:
URL: https://github.com/apache/shardingsphere/pull/23168#discussion_r1059850530
##########
kernel/data-pipeline/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/core/ack/CDCAckHolder.java:
##########
@@ -15,34 +15,48 @@
* limitations under the License.
*/
-package org.apache.shardingsphere.data.pipeline.cdc.holder;
+package org.apache.shardingsphere.data.pipeline.cdc.core.ack;
-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;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
/**
* CDC ack holder.
*/
-@NoArgsConstructor(access = AccessLevel.PRIVATE)
public final class CDCAckHolder {
+ private static final int TIMEOUT_ACK_MILLIS = 60 * 1000 * 5;
+
private static final CDCAckHolder INSTANCE = new CDCAckHolder();
- private final Map<String, Map<CDCImporter, Record>> ackIdImporterMap = new
ConcurrentHashMap<>();
+ private final Map<String, Map<CDCImporter, CDCAckPosition>>
ackIdImporterMap = new ConcurrentHashMap<>();
+
+ private final ScheduledThreadPoolExecutor scheduleExecutor = new
ScheduledThreadPoolExecutor(1);
+
+ private CDCAckHolder() {
+ scheduleExecutor.scheduleWithFixedDelay(this::cleanUpTimeoutAckId,
60L, 60L, TimeUnit.SECONDS);
+ }
+
+ private void cleanUpTimeoutAckId() {
+ if (ackIdImporterMap.isEmpty()) {
+ return;
+ }
+ long now = System.currentTimeMillis();
+ ackIdImporterMap.entrySet().removeIf(entry ->
entry.getValue().values().stream().anyMatch(each -> now -
each.getCreateTimeMills() >= TIMEOUT_ACK_MILLIS));
+ }
Review Comment:
Could we do cleanup by connection or job lifecycle, but not hard-coded
timeout
##########
kernel/data-pipeline/cdc/core/src/test/java/org/apache/shardingsphere/data/pipeline/cdc/core/ack/CDCAckHolderTest.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.core.ack;
+
+import
org.apache.shardingsphere.data.pipeline.api.ingest.position.FinishedPosition;
+import
org.apache.shardingsphere.data.pipeline.api.ingest.record.FinishedRecord;
+import org.apache.shardingsphere.data.pipeline.cdc.core.importer.CDCImporter;
+import org.apache.shardingsphere.infra.util.reflection.ReflectionUtil;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
+public final class CDCAckHolderTest {
+
+ @Test
+ public void assertBindAckIdWithPositionAndAck() {
+ CDCAckHolder cdcAckHolder = CDCAckHolder.getInstance();
+ final Map<CDCImporter, CDCAckPosition> importerDataRecordMap = new
HashMap<>();
+ CDCImporter cdcImporter = mock(CDCImporter.class);
+ importerDataRecordMap.put(cdcImporter, new CDCAckPosition(new
FinishedRecord(new FinishedPosition()), 0));
+ Optional<Map<String, Map<CDCImporter, CDCAckPosition>>>
ackIdImporterMap = ReflectionUtil.getFieldValue(cdcAckHolder,
"ackIdImporterMap");
Review Comment:
It's better to add `actual` for `ackIdImporterMap`
##########
kernel/data-pipeline/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/core/importer/connector/CDCImporterConnector.java:
##########
@@ -181,38 +208,62 @@ private final class CDCIncrementalImporterTask implements
Runnable {
@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()) {
+ final Map<CDCImporter, CDCAckPosition> cdcAckPositionMap = new
HashMap<>();
Review Comment:
`final` is not necessary
##########
kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/util/RecordUtil.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.core.util;
+
+import
org.apache.shardingsphere.data.pipeline.api.ingest.position.PlaceholderPosition;
+import org.apache.shardingsphere.data.pipeline.api.ingest.record.Record;
+
+import java.util.List;
+
+/**
+ * Record util.
+ */
+public final class RecordUtil {
Review Comment:
There's already `RecordUtil`, is it possible to merge them into one?
##########
kernel/data-pipeline/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/core/ack/CDCAckPosition.java:
##########
@@ -0,0 +1,47 @@
+/*
+ * 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.core.ack;
+
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.data.pipeline.api.ingest.record.Record;
+
+/**
+ * CDC ack position.
+ */
+@Getter
+public final class CDCAckPosition {
+
+ @Setter
+ private Record lastRecord;
+
+ @Setter
+ private int dataRecordCount;
+
+ private final long createTimeMills;
+
+ public CDCAckPosition(final Record lastRecord, final int dataRecordCount) {
+ this(lastRecord, dataRecordCount, System.currentTimeMillis());
+ }
+
+ public CDCAckPosition(final Record lastRecord, final int dataRecordCount,
final long createTimeMills) {
+ this.lastRecord = lastRecord;
+ this.dataRecordCount = dataRecordCount;
+ this.createTimeMills = createTimeMills;
+ }
Review Comment:
Could it be simplified as AllArgsConstructor annotation
##########
kernel/data-pipeline/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/core/importer/connector/CDCImporterConnector.java:
##########
@@ -181,38 +208,62 @@ private final class CDCIncrementalImporterTask implements
Runnable {
@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()) {
+ final Map<CDCImporter, CDCAckPosition> cdcAckPositionMap = new
HashMap<>();
+ List<DataRecord> dataRecords = new LinkedList<>();
+ for (int i = 0; i < batchSize; i++) {
+ DataRecord minimumDataRecord =
findMinimumDataRecordAndSavePosition(cdcAckPositionMap);
+ if (null == minimumDataRecord) {
break;
}
- DataRecord minDataRecord = Collections.min(filterRecord,
dataRecordComparator);
- dataRecords.add(minDataRecord);
-
pipelineChannelPositions.put(recordChannelMap.get(minDataRecord),
minDataRecord);
- index++;
+ dataRecords.add(minimumDataRecord);
}
if (dataRecords.isEmpty()) {
ThreadUtil.sleep(200, TimeUnit.MILLISECONDS);
} else {
- writeImmediately(dataRecords, pipelineChannelPositions);
+ writeImmediately(dataRecords, cdcAckPositionMap);
+ }
+ }
+ }
+
+ private DataRecord findMinimumDataRecordAndSavePosition(final
Map<CDCImporter, CDCAckPosition> cdcAckPositionMap) {
Review Comment:
It needs unit test, if it's not convenient to do unit test, could we extract
it as public class
--
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]