From 6bfc454ee6d045f4fc0697234b62203eeb073a80 Mon Sep 17 00:00:00 2001 From: wudi Date: Fri, 23 Jan 2026 17:01:04 +0800 Subject: [PATCH 01/11] =?UTF-8?q?1.=20change=20produce=20-consumer=20to=20?= =?UTF-8?q?poll=20split=20record=EF=BC=8Cpoll=20until=20headerbeat?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../streaming/StreamingMultiTblTask.java | 2 +- .../doris/cdcclient/common/Constants.java | 2 +- .../service/PipelineCoordinator.java | 363 ++++++++++++------ .../reader/JdbcIncrementalSourceReader.java | 187 +++------ .../cdcclient/source/reader/SourceReader.java | 12 +- .../source/reader/SplitReadResult.java | 4 - .../reader/mysql/MySqlSourceReader.java | 95 ++--- .../reader/postgres/PostgresSourceReader.java | 5 +- 8 files changed, 361 insertions(+), 309 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java index 76dc77dfab7836..1f955f0a2c3d3d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java @@ -306,7 +306,7 @@ public String getTimeoutReason() { log.warn("Failed to get task timeout reason, response: {}", response); } } catch (ExecutionException | InterruptedException ex) { - log.error("Send get task fail reason request failed: ", ex); + log.error("Send get fail reason request failed: ", ex); } return ""; } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Constants.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Constants.java index 93b141ae600c6a..b4c2284833ceb1 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Constants.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Constants.java @@ -22,7 +22,7 @@ public class Constants { public static final long POLL_SPLIT_RECORDS_TIMEOUTS = 15000L; // Debezium default properties - public static final long DEBEZIUM_HEARTBEAT_INTERVAL_MS = 10000L; + public static final long DEBEZIUM_HEARTBEAT_INTERVAL_MS = 3000L; public static final String DEBEZIUM_MAX_QUEUE_SIZE = "162580"; public static final String DEBEZIUM_MAX_BATCH_SIZE = "40960"; public static final String DEBEZIUM_POLL_INTERVAL_MS = "50"; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java index 3245e00d4ccb74..ecf66089da721e 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java @@ -17,6 +17,7 @@ package org.apache.doris.cdcclient.service; +import org.apache.doris.cdcclient.common.Constants; import org.apache.doris.cdcclient.common.Env; import org.apache.doris.cdcclient.model.response.RecordWithMeta; import org.apache.doris.cdcclient.sink.DorisBatchStreamLoad; @@ -29,6 +30,7 @@ import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.source.SourceRecord; @@ -42,6 +44,8 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import static org.apache.flink.cdc.connectors.base.utils.SourceRecordUtils.SCHEMA_HEARTBEAT_EVENT_KEY_NAME; + import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import io.debezium.data.Envelope; @@ -83,67 +87,101 @@ public PipelineCoordinator() { public RecordWithMeta fetchRecords(FetchRecordRequest fetchRecordRequest) throws Exception { SourceReader sourceReader = Env.getCurrentEnv().getReader(fetchRecordRequest); - SplitReadResult readResult = sourceReader.readSplitRecords(fetchRecordRequest); + SplitReadResult readResult = sourceReader.prepareAndSubmitSplit(fetchRecordRequest); return buildRecordResponse(sourceReader, fetchRecordRequest, readResult); } - /** build RecordWithMeta */ + /** + * Build RecordWithMeta response + * + *

This method polls records until: 1. Data is received AND heartbeat is received (normal + * case) 2. Timeout is reached (with heartbeat wait protection) + */ private RecordWithMeta buildRecordResponse( SourceReader sourceReader, FetchRecordRequest fetchRecord, SplitReadResult readResult) throws Exception { RecordWithMeta recordResponse = new RecordWithMeta(); - SourceSplit split = readResult.getSplit(); - int count = 0; try { - // Serialize records and add them to the response (collect from iterator) - Iterator iterator = readResult.getRecordIterator(); - while (iterator != null && iterator.hasNext()) { - SourceRecord element = iterator.next(); - List serializedRecords = - sourceReader.deserialize(fetchRecord.getConfig(), element); - if (!CollectionUtils.isEmpty(serializedRecords)) { - recordResponse.getRecords().addAll(serializedRecords); - count += serializedRecords.size(); - if (sourceReader.isBinlogSplit(split)) { - // put offset for event - Map lastMeta = - sourceReader.extractBinlogStateOffset(readResult.getSplitState()); - lastMeta.put(SPLIT_ID, BinlogSplit.BINLOG_SPLIT_ID); - recordResponse.setMeta(lastMeta); + long startTime = System.currentTimeMillis(); + boolean shouldStop = false; + boolean hasReceivedData = false; + int heartbeatCount = 0; + int recordCount = 0; + + while (!shouldStop) { + long elapsedTime = System.currentTimeMillis() - startTime; + boolean timeoutReached = elapsedTime > Constants.POLL_SPLIT_RECORDS_TIMEOUTS; + + // Timeout protection: force stop if waiting for heartbeat exceeds threshold + // After reaching timeout, wait at most DEBEZIUM_HEARTBEAT_INTERVAL_MS * 3 for + // heartbeat + if (timeoutReached + && elapsedTime + > Constants.POLL_SPLIT_RECORDS_TIMEOUTS + + Constants.DEBEZIUM_HEARTBEAT_INTERVAL_MS * 3) { + LOG.warn( + "Heartbeat wait timeout after {} ms since timeout reached, force stopping. " + + "Total elapsed: {} ms", + elapsedTime - Constants.POLL_SPLIT_RECORDS_TIMEOUTS, + elapsedTime); + break; + } + + Iterator recordIterator = + sourceReader.pollRecords(readResult.getSplitState()); + + if (!recordIterator.hasNext()) { + // If we have data and reached timeout, continue waiting for heartbeat + if (hasReceivedData && timeoutReached) { + Thread.sleep(100); + continue; } + // Otherwise, just wait for more data + Thread.sleep(100); + continue; } - } - } finally { - // The LSN in the commit is the current offset, which is the offset from the last - // successful write. - // Therefore, even if a subsequent write fails, it will not affect the commit. - sourceReader.commitSourceOffset(fetchRecord.getJobId(), readResult.getSplit()); - // This must be called after commitSourceOffset; otherwise, - // PG's confirmed lsn will not proceed. - sourceReader.finishSplitRecords(); - } + while (recordIterator.hasNext()) { + SourceRecord element = recordIterator.next(); - if (readResult.getSplitState() != null) { - // Set meta information for hw - if (sourceReader.isSnapshotSplit(split)) { - Map offsetRes = - sourceReader.extractSnapshotStateOffset(readResult.getSplitState()); - offsetRes.put(SPLIT_ID, split.splitId()); - recordResponse.setMeta(offsetRes); - } + // Check if this is a heartbeat message + if (isHeartbeatEvent(element)) { + heartbeatCount++; + if (hasReceivedData || timeoutReached) { + LOG.info( + "Heartbeat received after {} data records, stopping", + recordResponse.getRecords().size()); + shouldStop = true; + break; + } + // Skip heartbeat messages if we haven't received data yet + continue; + } - // set meta for binlog event - if (sourceReader.isBinlogSplit(split)) { - Map offsetRes = - sourceReader.extractBinlogStateOffset(readResult.getSplitState()); - offsetRes.put(SPLIT_ID, BinlogSplit.BINLOG_SPLIT_ID); - recordResponse.setMeta(offsetRes); + // Process data messages + List serializedRecords = + sourceReader.deserialize(fetchRecord.getConfig(), element); + if (!CollectionUtils.isEmpty(serializedRecords)) { + recordCount++; + recordResponse.getRecords().addAll(serializedRecords); + hasReceivedData = true; + } + } } - } else { - throw new RuntimeException("split state is null"); + LOG.info( + "Fetched {} records and {} heartbeats in {} ms for jobId={}", + recordCount, + heartbeatCount, + System.currentTimeMillis() - startTime, + fetchRecord.getJobId()); + } finally { + cleanupReaderResources(sourceReader, fetchRecord.getJobId(), readResult); } + // Extract and set offset metadata + Map offsetMeta = extractOffsetMeta(sourceReader, readResult); + recordResponse.setMeta(offsetMeta); + return recordResponse; } @@ -177,107 +215,134 @@ public CompletableFuture writeRecordsAsync(WriteRecordRequest writeRecordR executor); } - /** Read data from SourceReader and write it to Doris, while returning meta information. */ + /** + * Read data from SourceReader and write it to Doris, while returning meta information. 1. poll + * interval record. 2. Try to retrieve the heartbeat message, as it returns the latest offset, + * preventing the next task from having to skip a large number of records. + */ public void writeRecords(WriteRecordRequest writeRecordRequest) throws Exception { SourceReader sourceReader = Env.getCurrentEnv().getReader(writeRecordRequest); DorisBatchStreamLoad batchStreamLoad = null; Map metaResponse = new HashMap<>(); long scannedRows = 0L; long scannedBytes = 0L; + int heartbeatCount = 0; SplitReadResult readResult = null; try { - readResult = sourceReader.readSplitRecords(writeRecordRequest); - batchStreamLoad = - getOrCreateBatchStreamLoad( - writeRecordRequest.getJobId(), writeRecordRequest.getTargetDb()); - batchStreamLoad.setCurrentTaskId(writeRecordRequest.getTaskId()); - batchStreamLoad.setFrontendAddress(writeRecordRequest.getFrontendAddress()); - batchStreamLoad.setToken(writeRecordRequest.getToken()); - - // Record start time for maxInterval check + // 1. submit split async + readResult = sourceReader.prepareAndSubmitSplit(writeRecordRequest); + batchStreamLoad = getOrCreateBatchStreamLoad(writeRecordRequest); long startTime = System.currentTimeMillis(); long maxIntervalMillis = writeRecordRequest.getMaxInterval() * 1000; + boolean shouldStop = false; - // Use iterators to read and write. - Iterator iterator = readResult.getRecordIterator(); - while (iterator != null && iterator.hasNext()) { - SourceRecord element = iterator.next(); - List serializedRecords = - sourceReader.deserialize(writeRecordRequest.getConfig(), element); - - if (!CollectionUtils.isEmpty(serializedRecords)) { - String database = writeRecordRequest.getTargetDb(); - String table = extractTable(element); - for (String record : serializedRecords) { - scannedRows++; - byte[] dataBytes = record.getBytes(); - scannedBytes += dataBytes.length; - batchStreamLoad.writeRecord(database, table, dataBytes); - } - } - // Check if maxInterval has been exceeded + // 2. poll record + while (!shouldStop) { long elapsedTime = System.currentTimeMillis() - startTime; - if (maxIntervalMillis > 0 && elapsedTime >= maxIntervalMillis) { - LOG.info( - "Max interval {} seconds reached, stopping data reading", - writeRecordRequest.getMaxInterval()); + boolean timeoutReached = maxIntervalMillis > 0 && elapsedTime >= maxIntervalMillis; + + // Timeout protection: force stop if waiting for heartbeat exceeds threshold + // After reaching maxInterval, wait at most DEBEZIUM_HEARTBEAT_INTERVAL_MS * 3 for + // heartbeat + if (timeoutReached + && elapsedTime + > maxIntervalMillis + + Constants.DEBEZIUM_HEARTBEAT_INTERVAL_MS * 3) { + LOG.warn( + "Heartbeat wait timeout after {} ms since timeout reached, force stopping. " + + "Total elapsed: {} ms", + elapsedTime - maxIntervalMillis, + elapsedTime); break; } - } - } finally { - if (readResult != null) { - // The LSN in the commit is the current offset, which is the offset from the last - // successful write. - // Therefore, even if a subsequent write fails, it will not affect the commit. - sourceReader.commitSourceOffset( - writeRecordRequest.getJobId(), readResult.getSplit()); - } - // This must be called after commitSourceOffset; otherwise, - // PG's confirmed lsn will not proceed. - // This operation must be performed before batchStreamLoad.commitOffset; - // otherwise, fe might issue the next task for this job. - sourceReader.finishSplitRecords(); - } - // get offset from split state - try { - if (readResult.getSplitState() != null) { - // Set meta information for hw - if (sourceReader.isSnapshotSplit(readResult.getSplit())) { - Map offsetRes = - sourceReader.extractSnapshotStateOffset(readResult.getSplitState()); - offsetRes.put(SPLIT_ID, readResult.getSplit().splitId()); - metaResponse = offsetRes; + Iterator recordIterator = + sourceReader.pollRecords(readResult.getSplitState()); + + if (!recordIterator.hasNext()) { + Thread.sleep(100); + continue; } - // set meta for binlog event - if (sourceReader.isBinlogSplit(readResult.getSplit())) { - Map offsetRes = - sourceReader.extractBinlogStateOffset(readResult.getSplitState()); - offsetRes.put(SPLIT_ID, BinlogSplit.BINLOG_SPLIT_ID); - metaResponse = offsetRes; + while (recordIterator.hasNext()) { + SourceRecord element = recordIterator.next(); + // Check if this is a heartbeat message + if (isHeartbeatEvent(element)) { + heartbeatCount++; + if (timeoutReached) { + LOG.info( + "Max interval reached and heartbeat received, stopping data reading"); + shouldStop = true; + break; + } + // Skip heartbeat messages during normal processing + continue; + } + + // Process data messages + List serializedRecords = + sourceReader.deserialize(writeRecordRequest.getConfig(), element); + + if (!CollectionUtils.isEmpty(serializedRecords)) { + String database = writeRecordRequest.getTargetDb(); + String table = extractTable(element); + for (String record : serializedRecords) { + scannedRows++; + byte[] dataBytes = record.getBytes(); + scannedBytes += dataBytes.length; + batchStreamLoad.writeRecord(database, table, dataBytes); + } + } } - } else { - throw new RuntimeException("split state is null"); } + LOG.info( + "Fetched {} records and {} heartbeats in {} ms for jobId={} taskId={}", + scannedRows, + heartbeatCount, + System.currentTimeMillis() - startTime, + writeRecordRequest.getJobId(), + writeRecordRequest.getTaskId()); + + } finally { + cleanupReaderResources(sourceReader, writeRecordRequest.getJobId(), readResult); + } - // wait all stream load finish + // 3. Extract offset from split state + metaResponse = extractOffsetMeta(sourceReader, readResult); + try { + // 4. wait all stream load finish batchStreamLoad.forceFlush(); - // request fe api + // 5. request fe api update offset batchStreamLoad.commitOffset(metaResponse, scannedRows, scannedBytes); } finally { batchStreamLoad.resetTaskId(); } } - private DorisBatchStreamLoad getOrCreateBatchStreamLoad(Long jobId, String targetDb) { - return batchStreamLoadMap.computeIfAbsent( - jobId, - k -> { - LOG.info("Create DorisBatchStreamLoad for jobId={}", jobId); - return new DorisBatchStreamLoad(jobId, targetDb); - }); + public static boolean isHeartbeatEvent(SourceRecord record) { + Schema valueSchema = record.valueSchema(); + return valueSchema != null + && SCHEMA_HEARTBEAT_EVENT_KEY_NAME.equalsIgnoreCase(valueSchema.name()); + } + + private synchronized DorisBatchStreamLoad getOrCreateBatchStreamLoad( + WriteRecordRequest writeRecordRequest) { + DorisBatchStreamLoad batchStreamLoad = + batchStreamLoadMap.computeIfAbsent( + writeRecordRequest.getJobId(), + k -> { + LOG.info( + "Create DorisBatchStreamLoad for jobId={}", + writeRecordRequest.getJobId()); + return new DorisBatchStreamLoad( + writeRecordRequest.getJobId(), + writeRecordRequest.getTargetDb()); + }); + batchStreamLoad.setCurrentTaskId(writeRecordRequest.getTaskId()); + batchStreamLoad.setFrontendAddress(writeRecordRequest.getFrontendAddress()); + batchStreamLoad.setToken(writeRecordRequest.getToken()); + return batchStreamLoad; } public void closeJobStreamLoad(Long jobId) { @@ -298,4 +363,66 @@ public String getTaskFailReason(String taskId) { String taskReason = taskErrorMaps.remove(taskId); return taskReason == null ? "" : taskReason; } + + /** + * Clean up reader resources: commit source offset and finish split records. + * + * @param sourceReader the source reader + * @param jobId the job id + * @param readResult the read result containing split information + */ + private void cleanupReaderResources( + SourceReader sourceReader, Long jobId, SplitReadResult readResult) { + try { + // The LSN in the commit is the current offset, which is the offset from the last + // successful write. + // Therefore, even if a subsequent write fails, it will not affect the commit. + if (readResult != null && readResult.getSplit() != null) { + sourceReader.commitSourceOffset(jobId, readResult.getSplit()); + } + } finally { + // This must be called after `commitSourceOffset`; otherwise, + // PG's confirmed lsn will not proceed. + // This operation must be performed before `batchStreamLoad.commitOffset`; + // otherwise, fe might create the next task for this job. + sourceReader.finishSplitRecords(); + } + } + + /** + * Extract offset metadata from split state. + * + *

This method handles both snapshot splits and binlog splits, extracting the appropriate + * offset information and adding the split ID. + * + * @param sourceReader the source reader + * @param readResult the read result containing split and split state + * @return offset metadata map + * @throws RuntimeException if split state is null or split type is unknown + */ + private Map extractOffsetMeta( + SourceReader sourceReader, SplitReadResult readResult) { + Preconditions.checkNotNull(readResult, "readResult must not be null"); + + if (readResult.getSplitState() == null) { + throw new RuntimeException("split state is null"); + } + + SourceSplit split = readResult.getSplit(); + Map offsetRes; + + // Set meta information for hw (high watermark) + if (sourceReader.isSnapshotSplit(split)) { + offsetRes = sourceReader.extractSnapshotStateOffset(readResult.getSplitState()); + offsetRes.put(SPLIT_ID, split.splitId()); + } else if (sourceReader.isBinlogSplit(split)) { + // Set meta for binlog event + offsetRes = sourceReader.extractBinlogStateOffset(readResult.getSplitState()); + offsetRes.put(SPLIT_ID, BinlogSplit.BINLOG_SPLIT_ID); + } else { + throw new RuntimeException("Unknown split type: " + split.getClass().getName()); + } + + return offsetRes; + } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/JdbcIncrementalSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/JdbcIncrementalSourceReader.java index 5b404932c3f2d7..888c6d90df5727 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/JdbcIncrementalSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/JdbcIncrementalSourceReader.java @@ -17,7 +17,6 @@ package org.apache.doris.cdcclient.source.reader; -import org.apache.doris.cdcclient.common.Constants; import org.apache.doris.cdcclient.source.deserialize.DebeziumJsonDeserializer; import org.apache.doris.cdcclient.source.deserialize.SourceRecordDeserializer; import org.apache.doris.cdcclient.source.factory.DataSource; @@ -51,8 +50,6 @@ import org.apache.flink.cdc.connectors.base.source.meta.wartermark.WatermarkEvent; import org.apache.flink.cdc.connectors.base.source.reader.external.FetchTask; import org.apache.flink.cdc.connectors.base.source.reader.external.Fetcher; -import org.apache.flink.cdc.connectors.base.source.reader.external.IncrementalSourceScanFetcher; -import org.apache.flink.cdc.connectors.base.source.reader.external.IncrementalSourceStreamFetcher; import org.apache.flink.cdc.connectors.base.utils.SourceRecordUtils; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.types.DataType; @@ -61,6 +58,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; @@ -149,64 +147,70 @@ public List getSourceSplits(FetchTableSplitsRequest ftsReq) } @Override - public SplitReadResult readSplitRecords(JobBaseRecordRequest baseReq) throws Exception { + public SplitReadResult prepareAndSubmitSplit(JobBaseRecordRequest baseReq) throws Exception { Map offsetMeta = baseReq.getMeta(); if (offsetMeta == null || offsetMeta.isEmpty()) { throw new RuntimeException("miss meta offset"); } - LOG.info("Job {} read split records with offset: {}", baseReq.getJobId(), offsetMeta); - - // If there is an active split being consumed, reuse it directly; - // Otherwise, create a new snapshot/stream split based on offset and start the reader. - SourceSplitBase split = null; - SplitRecords currentSplitRecords = this.getCurrentSplitRecords(); - if (currentSplitRecords == null) { - Fetcher currentReader = this.getCurrentReader(); - if (baseReq.isReload() || currentReader == null) { - LOG.info( - "No current reader or reload {}, create new split reader for job {}", - baseReq.isReload(), - baseReq.getJobId()); - // build split - Tuple2 splitFlag = createSourceSplit(offsetMeta, baseReq); - split = splitFlag.f0; - // closeBinlogReader(); - currentSplitRecords = pollSplitRecordsWithSplit(split, baseReq); - this.setCurrentSplitRecords(currentSplitRecords); - this.setCurrentSplit(split); - } else if (currentReader instanceof IncrementalSourceStreamFetcher) { - LOG.info("Continue poll records with current binlog reader"); - // only for binlog reader - currentSplitRecords = pollSplitRecordsWithCurrentReader(currentReader); - split = this.getCurrentSplit(); - } else { - throw new RuntimeException("Should not happen"); - } - } else { - LOG.info( - "Continue read records with current split records, splitId: {}", - currentSplitRecords.getSplitId()); + LOG.info("Job {} prepare and submit split with offset: {}", baseReq.getJobId(), offsetMeta); + // Build split + Tuple2 splitFlag = createSourceSplit(offsetMeta, baseReq); + this.currentSplit = splitFlag.f0; + LOG.info("Get a split: {}", this.currentSplit.toString()); + + // Create reader based on split type + if (this.currentSplit.isSnapshotSplit()) { + this.currentReader = getSnapshotSplitReader(baseReq); + } else if (this.currentSplit.isStreamSplit()) { + this.currentReader = getBinlogSplitReader(baseReq); } - // build response with iterator - SplitReadResult result = new SplitReadResult(); - SourceSplitState currentSplitState = null; - SourceSplitBase currentSplit = this.getCurrentSplit(); - if (currentSplit.isSnapshotSplit()) { - currentSplitState = new SnapshotSplitState(currentSplit.asSnapshotSplit()); + // Submit split + FetchTask splitFetchTask = + createFetchTaskFromSplit(baseReq, this.currentSplit); + this.currentReader.submitTask(splitFetchTask); + this.setCurrentFetchTask(splitFetchTask); + + // Create split state + SourceSplitState currentSplitState; + if (this.currentSplit.isSnapshotSplit()) { + currentSplitState = new SnapshotSplitState(this.currentSplit.asSnapshotSplit()); } else { - currentSplitState = new StreamSplitState(currentSplit.asStreamSplit()); + currentSplitState = new StreamSplitState(this.currentSplit.asStreamSplit()); } - Iterator filteredIterator = - new FilteredRecordIterator(currentSplitRecords, currentSplitState); - - result.setRecordIterator(filteredIterator); + // Return result without iterator + SplitReadResult result = new SplitReadResult(); + result.setSplit(this.currentSplit); result.setSplitState(currentSplitState); - result.setSplit(split); return result; } + @Override + public Iterator pollRecords(Object splitState) throws InterruptedException { + Preconditions.checkState(this.currentReader != null, "currentReader is null"); + Preconditions.checkNotNull(splitState, "splitState is null"); + Preconditions.checkState( + splitState instanceof SourceSplitState, + "splitState type is invalid " + splitState.getClass()); + + // Poll data from Debezium queue + Iterator dataIt = currentReader.pollSplitRecords(); + if (dataIt == null || !dataIt.hasNext()) { + return Collections.emptyIterator(); // No data available + } + + SourceRecords sourceRecords = dataIt.next(); + SplitRecords splitRecords = + new SplitRecords(this.currentSplit.splitId(), sourceRecords.iterator()); + if (!sourceRecords.getSourceRecordList().isEmpty()) { + LOG.info("{} Records received.", sourceRecords.getSourceRecordList().size()); + } + + // Return filtered iterator + return new FilteredRecordIterator(splitRecords, (SourceSplitState) splitState); + } + protected abstract DataType fromDbzColumn(Column splitColumn); protected abstract Fetcher getSnapshotSplitReader( @@ -453,91 +457,6 @@ private static void closeChunkSplitterOnly(HybridSplitAssigner splitAssigner) } } - private SplitRecords pollSplitRecordsWithSplit(SourceSplitBase split, JobBaseConfig jobConfig) - throws Exception { - Preconditions.checkState(split != null, "split is null"); - SourceRecords sourceRecords = null; - String currentSplitId = null; - Fetcher currentReader = null; - LOG.info("Get a split: {}", split.splitId()); - if (split.isSnapshotSplit()) { - currentReader = getSnapshotSplitReader(jobConfig); - } else if (split.isStreamSplit()) { - currentReader = getBinlogSplitReader(jobConfig); - } - this.setCurrentReader(currentReader); - FetchTask splitFetchTask = createFetchTaskFromSplit(jobConfig, split); - currentReader.submitTask(splitFetchTask); - currentSplitId = split.splitId(); - this.setCurrentFetchTask(splitFetchTask); - // make split record available - sourceRecords = - pollUntilDataAvailable(currentReader, Constants.POLL_SPLIT_RECORDS_TIMEOUTS, 500); - if (currentReader instanceof IncrementalSourceScanFetcher) { - closeCurrentReader(); - } - return new SplitRecords(currentSplitId, sourceRecords.iterator()); - } - - private SplitRecords pollSplitRecordsWithCurrentReader( - Fetcher currentReader) throws Exception { - Iterator dataIt = null; - if (currentReader instanceof IncrementalSourceStreamFetcher) { - dataIt = currentReader.pollSplitRecords(); - return dataIt == null - ? null - : new SplitRecords(STREAM_SPLIT_ID, dataIt.next().iterator()); - } else { - throw new IllegalStateException("Unsupported reader type."); - } - } - - /** - * Split tasks are submitted asynchronously, and data is sent to the Debezium queue. Therefore, - * there will be a time interval between retrieving data; it's necessary to fetch data until the - * queue has data. - */ - private SourceRecords pollUntilDataAvailable( - Fetcher reader, long maxWaitTimeMs, long pollIntervalMs) - throws InterruptedException { - long startTime = System.currentTimeMillis(); - long elapsedTime = 0; - int attemptCount = 0; - LOG.info("Polling until data available"); - Iterator lastDataIt = null; - while (elapsedTime < maxWaitTimeMs) { - attemptCount++; - lastDataIt = reader.pollSplitRecords(); - if (lastDataIt != null && lastDataIt.hasNext()) { - SourceRecords sourceRecords = lastDataIt.next(); - if (sourceRecords != null && !sourceRecords.getSourceRecordList().isEmpty()) { - LOG.info( - "Data available after {} ms ({} attempts). {} Records received.", - elapsedTime, - attemptCount, - sourceRecords.getSourceRecordList().size()); - // todo: poll until heartbeat ? - return sourceRecords; - } - } - - // No records yet, continue polling - if (elapsedTime + pollIntervalMs < maxWaitTimeMs) { - Thread.sleep(pollIntervalMs); - elapsedTime = System.currentTimeMillis() - startTime; - } else { - // Last attempt before timeout - break; - } - } - - LOG.warn( - "Timeout: No data (heartbeat or data change) received after {} ms ({} attempts).", - elapsedTime, - attemptCount); - return new SourceRecords(new ArrayList<>()); - } - private void closeCurrentReader() { Fetcher currentReader = this.getCurrentReader(); if (currentReader != null) { @@ -672,6 +591,8 @@ public boolean hasNext() { Offset position = createOffset(element.sourceOffset()); splitState.asStreamSplitState().setStartingOffset(position); } + nextRecord = element; + return true; } else if (SourceRecordUtils.isDataChangeRecord(element)) { if (splitState.isStreamSplitState()) { Offset position = createOffset(element.sourceOffset()); diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java index d5feeef45a32c8..1ac5f0287222f3 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java @@ -28,6 +28,7 @@ import org.apache.kafka.connect.source.SourceRecord; import java.io.IOException; +import java.util.Iterator; import java.util.List; import java.util.Map; @@ -41,12 +42,11 @@ public interface SourceReader { /** Divide the data to be read. For example: split mysql to chunks */ List getSourceSplits(FetchTableSplitsRequest config); - /** - * 1. If the SplitRecords iterator has it, read the iterator directly. 2. If there is a stream - * reader, poll it. 3. If there is none, resubmit split. 4. If reload is true, need to reset - * streamSplitReader and submit split. - */ - SplitReadResult readSplitRecords(JobBaseRecordRequest baseReq) throws Exception; + /** Construct a split and submit a split reading task. */ + SplitReadResult prepareAndSubmitSplit(JobBaseRecordRequest baseReq) throws Exception; + + /** Retrieve data from the current split. */ + Iterator pollRecords(Object splitState) throws InterruptedException; /** Extract offset information from snapshot split state. */ Map extractSnapshotStateOffset(Object splitState); diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java index 39f386f89886c1..df591468d49a9c 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java @@ -18,16 +18,12 @@ package org.apache.doris.cdcclient.source.reader; import org.apache.flink.api.connector.source.SourceSplit; -import org.apache.kafka.connect.source.SourceRecord; - -import java.util.Iterator; import lombok.Data; /** The result of reading a split with iterator. */ @Data public class SplitReadResult { - private Iterator recordIterator; // MySqlSplitState, SourceSplitState private Object splitState; // MySqlSplit SourceSplitBase diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java index d20225c0974cbb..70e71e69c021ce 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java @@ -69,8 +69,10 @@ import java.io.IOException; import java.sql.SQLException; +import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; @@ -81,6 +83,7 @@ import java.util.Properties; import java.util.concurrent.ConcurrentHashMap; +import static org.apache.doris.cdcclient.common.Constants.DEBEZIUM_HEARTBEAT_INTERVAL_MS; import static org.apache.doris.cdcclient.utils.ConfigUtil.is13Timestamp; import static org.apache.doris.cdcclient.utils.ConfigUtil.isJson; import static org.apache.doris.cdcclient.utils.ConfigUtil.toStringMap; @@ -91,6 +94,7 @@ import com.github.shyiko.mysql.binlog.BinaryLogClient; import com.mysql.cj.conf.ConnectionUrl; import io.debezium.connector.mysql.MySqlConnection; +import io.debezium.connector.mysql.MySqlConnectorConfig; import io.debezium.connector.mysql.MySqlPartition; import io.debezium.document.Array; import io.debezium.relational.Column; @@ -165,62 +169,56 @@ public List getSourceSplits(FetchTableSplitsRequest ftsReq) } @Override - public SplitReadResult readSplitRecords(JobBaseRecordRequest baseReq) throws Exception { + public SplitReadResult prepareAndSubmitSplit(JobBaseRecordRequest baseReq) throws Exception { Map offsetMeta = baseReq.getMeta(); if (offsetMeta == null || offsetMeta.isEmpty()) { throw new RuntimeException("miss meta offset"); } LOG.info("Job {} read split records with offset: {}", baseReq.getJobId(), offsetMeta); - - // If there is an active split being consumed, reuse it directly; - // Otherwise, create a new snapshot/binlog split based on offset and start the reader. - MySqlSplit split = null; - SplitRecords currentSplitRecords = this.getCurrentSplitRecords(); - if (currentSplitRecords == null) { - DebeziumReader currentReader = this.getCurrentReader(); - if (baseReq.isReload() || currentReader == null) { - LOG.info( - "No current reader or reload {}, create new split reader", - baseReq.isReload()); - // build split - Tuple2 splitFlag = createMySqlSplit(offsetMeta, baseReq); - split = splitFlag.f0; - // reset binlog reader - // closeBinlogReader(); - currentSplitRecords = pollSplitRecordsWithSplit(split, baseReq); - this.setCurrentSplitRecords(currentSplitRecords); - this.setCurrentSplit(split); - } else if (currentReader instanceof BinlogSplitReader) { - LOG.info("Continue poll records with current binlog reader"); - // only for binlog reader - currentSplitRecords = pollSplitRecordsWithCurrentReader(currentReader); - split = this.getCurrentSplit(); - } else { - throw new RuntimeException("Should not happen"); - } + // build split + Tuple2 splitFlag = createMySqlSplit(offsetMeta, baseReq); + this.currentSplit = splitFlag.f0; + LOG.info("Get a split: {}", this.currentSplit.toString()); + if (this.currentSplit instanceof MySqlSnapshotSplit) { + this.currentReader = getSnapshotSplitReader(baseReq); + } else if (this.currentSplit instanceof MySqlBinlogSplit) { + this.currentReader = getBinlogSplitReader(baseReq); + } + this.currentReader.submitSplit(this.currentSplit); + MySqlSplitState currentSplitState = null; + if (this.currentSplit.isSnapshotSplit()) { + currentSplitState = new MySqlSnapshotSplitState(this.currentSplit.asSnapshotSplit()); } else { - LOG.info( - "Continue read records with current split records, splitId: {}", - currentSplitRecords.getSplitId()); + currentSplitState = new MySqlBinlogSplitState(this.currentSplit.asBinlogSplit()); } - - // build response with iterator SplitReadResult result = new SplitReadResult(); - MySqlSplitState currentSplitState = null; - MySqlSplit currentSplit = this.getCurrentSplit(); - if (currentSplit.isSnapshotSplit()) { - currentSplitState = new MySqlSnapshotSplitState(currentSplit.asSnapshotSplit()); - } else { - currentSplitState = new MySqlBinlogSplitState(currentSplit.asBinlogSplit()); + result.setSplit(this.currentSplit); + result.setSplitState(currentSplitState); + return result; + } + + @Override + public Iterator pollRecords(Object splitState) throws InterruptedException { + Preconditions.checkState(this.currentReader != null, "currentReader is null"); + Preconditions.checkNotNull(splitState, "splitState is null"); + Preconditions.checkState( + splitState instanceof MySqlSplitState, + "splitState type is invalid " + splitState.getClass()); + + // Poll data from Debezium queue + Iterator dataIt = currentReader.pollSplitRecords(); + if (dataIt == null || !dataIt.hasNext()) { + return Collections.emptyIterator(); // No data available } - Iterator filteredIterator = - new FilteredRecordIterator(currentSplitRecords, currentSplitState); + SourceRecords sourceRecords = dataIt.next(); + SplitRecords splitRecords = + new SplitRecords(this.currentSplit.splitId(), sourceRecords.iterator()); + if (!sourceRecords.getSourceRecordList().isEmpty()) { + LOG.info("{} Records received", sourceRecords.getSourceRecordList().size()); + } - result.setRecordIterator(filteredIterator); - result.setSplitState(currentSplitState); - result.setSplit(split); - return result; + return new FilteredRecordIterator(splitRecords, (MySqlSplitState) splitState); } /** @@ -620,7 +618,12 @@ private MySqlSourceConfig generateMySqlConfig(Map cdcConfig, Str configFactory.jdbcProperties(jdbcProperteis); Properties dbzProps = ConfigUtil.getDefaultDebeziumProps(); + dbzProps.setProperty( + MySqlConnectorConfig.KEEP_ALIVE_INTERVAL_MS.name(), + DEBEZIUM_HEARTBEAT_INTERVAL_MS + ""); configFactory.debeziumProperties(dbzProps); + configFactory.heartbeatInterval(Duration.ofMillis(DEBEZIUM_HEARTBEAT_INTERVAL_MS)); + if (cdcConfig.containsKey(DataSourceConfigKeys.SPLIT_SIZE)) { configFactory.splitSize( Integer.parseInt(cdcConfig.get(DataSourceConfigKeys.SPLIT_SIZE))); @@ -796,6 +799,8 @@ public boolean hasNext() { BinlogOffset position = RecordUtils.getBinlogPosition(element); splitState.asBinlogSplitState().setStartingOffset(position); } + nextRecord = element; + return true; } else if (RecordUtils.isDataChangeRecord(element)) { if (splitState.isBinlogSplitState()) { BinlogOffset position = RecordUtils.getBinlogPosition(element); diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java index c1e062df5a6b34..7c1894de21cd1d 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java @@ -17,6 +17,7 @@ package org.apache.doris.cdcclient.source.reader.postgres; +import org.apache.doris.cdcclient.common.Constants; import org.apache.doris.cdcclient.exception.CdcClientException; import org.apache.doris.cdcclient.source.factory.DataSource; import org.apache.doris.cdcclient.source.reader.JdbcIncrementalSourceReader; @@ -50,6 +51,7 @@ import org.apache.flink.cdc.connectors.postgres.source.utils.TableDiscoveryUtils; import org.apache.flink.table.types.DataType; +import java.time.Duration; import java.time.Instant; import java.util.Arrays; import java.util.HashMap; @@ -208,7 +210,8 @@ private PostgresSourceConfig generatePostgresConfig(Map cdcConfi ConfigUtil.getPostgresServerTimeZoneFromProps(props).toString()); configFactory.slotName(getSlotName(jobId)); configFactory.decodingPluginName("pgoutput"); - // configFactory.heartbeatInterval(Duration.ofMillis(Constants.POLL_SPLIT_RECORDS_TIMEOUTS)); + configFactory.heartbeatInterval( + Duration.ofMillis(Constants.DEBEZIUM_HEARTBEAT_INTERVAL_MS)); return configFactory.create(0); } From 56dabdea3282b772d63225c7ffc6092bb2e29b2f Mon Sep 17 00:00:00 2001 From: wudi Date: Fri, 23 Jan 2026 20:27:32 +0800 Subject: [PATCH 02/11] fix --- .../service/PipelineCoordinator.java | 213 +++++++++++++----- .../cdcclient/sink/DorisBatchStreamLoad.java | 13 +- .../reader/JdbcIncrementalSourceReader.java | 3 + .../reader/mysql/MySqlSourceReader.java | 90 +------- 4 files changed, 171 insertions(+), 148 deletions(-) diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java index ecf66089da721e..d20e61e9e6bdd8 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java @@ -102,42 +102,38 @@ private RecordWithMeta buildRecordResponse( throws Exception { RecordWithMeta recordResponse = new RecordWithMeta(); try { + boolean isSnapshotSplit = sourceReader.isSnapshotSplit(readResult.getSplit()); long startTime = System.currentTimeMillis(); boolean shouldStop = false; boolean hasReceivedData = false; + boolean lastMessageIsHeartbeat = false; int heartbeatCount = 0; int recordCount = 0; + LOG.info( + "Start fetching records for jobId={}, isSnapshotSplit={}", + fetchRecord.getJobId(), + isSnapshotSplit); while (!shouldStop) { - long elapsedTime = System.currentTimeMillis() - startTime; - boolean timeoutReached = elapsedTime > Constants.POLL_SPLIT_RECORDS_TIMEOUTS; - - // Timeout protection: force stop if waiting for heartbeat exceeds threshold - // After reaching timeout, wait at most DEBEZIUM_HEARTBEAT_INTERVAL_MS * 3 for - // heartbeat - if (timeoutReached - && elapsedTime - > Constants.POLL_SPLIT_RECORDS_TIMEOUTS - + Constants.DEBEZIUM_HEARTBEAT_INTERVAL_MS * 3) { - LOG.warn( - "Heartbeat wait timeout after {} ms since timeout reached, force stopping. " - + "Total elapsed: {} ms", - elapsedTime - Constants.POLL_SPLIT_RECORDS_TIMEOUTS, - elapsedTime); - break; - } - Iterator recordIterator = sourceReader.pollRecords(readResult.getSplitState()); if (!recordIterator.hasNext()) { - // If we have data and reached timeout, continue waiting for heartbeat - if (hasReceivedData && timeoutReached) { - Thread.sleep(100); - continue; - } - // Otherwise, just wait for more data Thread.sleep(100); + + // Check if should stop + long elapsedTime = System.currentTimeMillis() - startTime; + boolean timeoutReached = elapsedTime > Constants.POLL_SPLIT_RECORDS_TIMEOUTS; + + if (shouldStop( + isSnapshotSplit, + hasReceivedData, + lastMessageIsHeartbeat, + elapsedTime, + Constants.POLL_SPLIT_RECORDS_TIMEOUTS, + timeoutReached)) { + break; + } continue; } @@ -147,6 +143,17 @@ private RecordWithMeta buildRecordResponse( // Check if this is a heartbeat message if (isHeartbeatEvent(element)) { heartbeatCount++; + + // Mark last message as heartbeat + if (!isSnapshotSplit) { + lastMessageIsHeartbeat = true; + } + + // If already have data or timeout, stop when heartbeat received + long elapsedTime = System.currentTimeMillis() - startTime; + boolean timeoutReached = + elapsedTime > Constants.POLL_SPLIT_RECORDS_TIMEOUTS; + if (hasReceivedData || timeoutReached) { LOG.info( "Heartbeat received after {} data records, stopping", @@ -165,6 +172,7 @@ private RecordWithMeta buildRecordResponse( recordCount++; recordResponse.getRecords().addAll(serializedRecords); hasReceivedData = true; + lastMessageIsHeartbeat = false; } } } @@ -216,9 +224,16 @@ public CompletableFuture writeRecordsAsync(WriteRecordRequest writeRecordR } /** - * Read data from SourceReader and write it to Doris, while returning meta information. 1. poll - * interval record. 2. Try to retrieve the heartbeat message, as it returns the latest offset, - * preventing the next task from having to skip a large number of records. + * Read data from SourceReader and write it to Doris, while returning meta information. + * + *

Snapshot split: Returns immediately after reading; otherwise, returns after the + * maxInterval. + * + *

Binlog split: Fetches data at the maxInterval. Returns immediately if no data is found; if + * found, checks if the last record is a heartbeat record. If it is, returns immediately; + * otherwise, fetches again until the heartbeat deadline. + * + *

Heartbeat events will carry the latest offset. */ public void writeRecords(WriteRecordRequest writeRecordRequest) throws Exception { SourceReader sourceReader = Env.getCurrentEnv().getReader(writeRecordRequest); @@ -232,46 +247,63 @@ public void writeRecords(WriteRecordRequest writeRecordRequest) throws Exception // 1. submit split async readResult = sourceReader.prepareAndSubmitSplit(writeRecordRequest); batchStreamLoad = getOrCreateBatchStreamLoad(writeRecordRequest); + + boolean isSnapshotSplit = sourceReader.isSnapshotSplit(readResult.getSplit()); long startTime = System.currentTimeMillis(); long maxIntervalMillis = writeRecordRequest.getMaxInterval() * 1000; boolean shouldStop = false; + boolean lastMessageIsHeartbeat = false; + LOG.info( + "Start polling records for jobId={} taskId={}, isSnapshotSplit={}", + writeRecordRequest.getJobId(), + writeRecordRequest.getTaskId(), + isSnapshotSplit); // 2. poll record while (!shouldStop) { - long elapsedTime = System.currentTimeMillis() - startTime; - boolean timeoutReached = maxIntervalMillis > 0 && elapsedTime >= maxIntervalMillis; - - // Timeout protection: force stop if waiting for heartbeat exceeds threshold - // After reaching maxInterval, wait at most DEBEZIUM_HEARTBEAT_INTERVAL_MS * 3 for - // heartbeat - if (timeoutReached - && elapsedTime - > maxIntervalMillis - + Constants.DEBEZIUM_HEARTBEAT_INTERVAL_MS * 3) { - LOG.warn( - "Heartbeat wait timeout after {} ms since timeout reached, force stopping. " - + "Total elapsed: {} ms", - elapsedTime - maxIntervalMillis, - elapsedTime); - break; - } - Iterator recordIterator = sourceReader.pollRecords(readResult.getSplitState()); if (!recordIterator.hasNext()) { Thread.sleep(100); + + // Check if should stop + long elapsedTime = System.currentTimeMillis() - startTime; + boolean timeoutReached = + maxIntervalMillis > 0 && elapsedTime >= maxIntervalMillis; + + if (shouldStop( + isSnapshotSplit, + scannedRows > 0, + lastMessageIsHeartbeat, + elapsedTime, + maxIntervalMillis, + timeoutReached)) { + break; + } continue; } while (recordIterator.hasNext()) { SourceRecord element = recordIterator.next(); + // Check if this is a heartbeat message if (isHeartbeatEvent(element)) { heartbeatCount++; - if (timeoutReached) { + + // Mark last message as heartbeat (only for binlog split) + if (!isSnapshotSplit) { + lastMessageIsHeartbeat = true; + } + + // If already timeout, stop immediately when heartbeat received + long elapsedTime = System.currentTimeMillis() - startTime; + boolean timeoutReached = + maxIntervalMillis > 0 && elapsedTime >= maxIntervalMillis; + + if (!isSnapshotSplit && timeoutReached) { LOG.info( - "Max interval reached and heartbeat received, stopping data reading"); + "Binlog split max interval reached and heartbeat received, stopping data reading"); shouldStop = true; break; } @@ -292,6 +324,8 @@ public void writeRecords(WriteRecordRequest writeRecordRequest) throws Exception scannedBytes += dataBytes.length; batchStreamLoad.writeRecord(database, table, dataBytes); } + // Mark last message as data (not heartbeat) + lastMessageIsHeartbeat = false; } } } @@ -309,15 +343,15 @@ public void writeRecords(WriteRecordRequest writeRecordRequest) throws Exception // 3. Extract offset from split state metaResponse = extractOffsetMeta(sourceReader, readResult); - try { - // 4. wait all stream load finish - batchStreamLoad.forceFlush(); - - // 5. request fe api update offset - batchStreamLoad.commitOffset(metaResponse, scannedRows, scannedBytes); - } finally { - batchStreamLoad.resetTaskId(); - } + // 4. wait all stream load finish + batchStreamLoad.forceFlush(); + + // 5. request fe api update offset + String currentTaskId = batchStreamLoad.getCurrentTaskId(); + // The offset must be reset before commitOffset to prevent the next taskId from being create + // by the fe. + batchStreamLoad.resetTaskId(); + batchStreamLoad.commitOffset(currentTaskId, metaResponse, scannedRows, scannedBytes); } public static boolean isHeartbeatEvent(SourceRecord record) { @@ -326,6 +360,73 @@ public static boolean isHeartbeatEvent(SourceRecord record) { && SCHEMA_HEARTBEAT_EVENT_KEY_NAME.equalsIgnoreCase(valueSchema.name()); } + /** + * Determine if we should stop polling. + * + * @param isSnapshotSplit whether this is a snapshot split + * @param hasData whether we have received any data + * @param lastMessageIsHeartbeat whether the last message is a heartbeat + * @param elapsedTime total elapsed time in milliseconds + * @param maxIntervalMillis max interval in milliseconds + * @param timeoutReached whether timeout is reached + * @return true if should stop, false if should continue + */ + private boolean shouldStop( + boolean isSnapshotSplit, + boolean hasData, + boolean lastMessageIsHeartbeat, + long elapsedTime, + long maxIntervalMillis, + boolean timeoutReached) { + + // 1. Snapshot split with data: if no more data in queue, stop immediately (no need to wait + // for timeout) + // snapshot split will be written to the debezium queue all at once. + if (isSnapshotSplit && hasData) { + LOG.info( + "Snapshot split finished, no more data available. Total elapsed: {} ms", + elapsedTime); + return true; + } + + // 2. Not timeout yet: continue waiting + if (!timeoutReached) { + return false; + } + + // === Below are checks after timeout is reached === + + // 3. No data received after timeout: stop + if (!hasData) { + LOG.info("No data received after timeout, stopping. Elapsed: {} ms", elapsedTime); + return true; + } + + // 4. Snapshot split after timeout (should not reach here, but keep as safety check) + if (isSnapshotSplit) { + LOG.info("Snapshot split timeout reached, stopping. Elapsed: {} ms", elapsedTime); + return true; + } + + // 5. Binlog split + last message is heartbeat: stop immediately + if (lastMessageIsHeartbeat) { + LOG.info("Binlog split timeout and last message is heartbeat, stopping"); + return true; + } + + // 6. Binlog split + no heartbeat yet: wait for heartbeat with timeout protection + if (elapsedTime > maxIntervalMillis + Constants.DEBEZIUM_HEARTBEAT_INTERVAL_MS * 3) { + LOG.warn( + "Binlog split heartbeat wait timeout after {} ms, force stopping. Total elapsed: {} ms", + elapsedTime - maxIntervalMillis, + elapsedTime); + return true; + } + + // Continue waiting for heartbeat + return false; + } + private synchronized DorisBatchStreamLoad getOrCreateBatchStreamLoad( WriteRecordRequest writeRecordRequest) { DorisBatchStreamLoad batchStreamLoad = diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java index 1604b1c030539e..9686364c71fbca 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java @@ -56,6 +56,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import com.fasterxml.jackson.databind.ObjectMapper; +import lombok.Getter; import lombok.Setter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -88,7 +89,7 @@ public class DorisBatchStreamLoad implements Serializable { private final Lock lock = new ReentrantLock(); private final Condition block = lock.newCondition(); private final Map bufferMapLock = new ConcurrentHashMap<>(); - @Setter private String currentTaskId; + @Setter @Getter private String currentTaskId; private String targetDb; private long jobId; @Setter private String token; @@ -481,13 +482,14 @@ public void resetTaskId() { } /** commit offfset to frontends. */ - public void commitOffset(Map meta, long scannedRows, long scannedBytes) { + public void commitOffset( + String taskId, Map meta, long scannedRows, long scannedBytes) { try { String url = String.format(COMMIT_URL_PATTERN, frontendAddress, targetDb); Map commitParams = new HashMap<>(); commitParams.put("offset", OBJECT_MAPPER.writeValueAsString(meta)); commitParams.put("jobId", jobId); - commitParams.put("taskId", currentTaskId); + commitParams.put("taskId", taskId); commitParams.put("scannedRows", scannedRows); commitParams.put("scannedBytes", scannedBytes); String param = OBJECT_MAPPER.writeValueAsString(commitParams); @@ -501,8 +503,7 @@ public void commitOffset(Map meta, long scannedRows, long scanne .commit() .setEntity(new StringEntity(param)); - LOG.info( - "commit offset for jobId {} taskId {}, params {}", jobId, currentTaskId, param); + LOG.info("commit offset for jobId {} taskId {}, params {}", jobId, taskId, param); Throwable resEx = null; int retry = 0; while (retry <= RETRY) { @@ -516,7 +517,7 @@ public void commitOffset(Map meta, long scannedRows, long scanne : ""; LOG.info("commit result {}", responseBody); if (statusCode == 200) { - LOG.info("commit offset for jobId {} taskId {}", jobId, currentTaskId); + LOG.info("commit offset for jobId {} taskId {}", jobId, taskId); // A 200 response indicates that the request was successful, and // information such as offset and statistics may have already been // updated. Retrying may result in repeated updates. diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/JdbcIncrementalSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/JdbcIncrementalSourceReader.java index 888c6d90df5727..27db10bccc6101 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/JdbcIncrementalSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/JdbcIncrementalSourceReader.java @@ -163,6 +163,9 @@ public SplitReadResult prepareAndSubmitSplit(JobBaseRecordRequest baseReq) throw this.currentReader = getSnapshotSplitReader(baseReq); } else if (this.currentSplit.isStreamSplit()) { this.currentReader = getBinlogSplitReader(baseReq); + } else { + throw new IllegalStateException( + "Unknown split type: " + this.currentSplit.getClass().getName()); } // Submit split diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java index 70e71e69c021ce..2a66fe871d1155 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java @@ -17,7 +17,6 @@ package org.apache.doris.cdcclient.source.reader.mysql; -import org.apache.doris.cdcclient.common.Constants; import org.apache.doris.cdcclient.source.deserialize.DebeziumJsonDeserializer; import org.apache.doris.cdcclient.source.deserialize.SourceRecordDeserializer; import org.apache.doris.cdcclient.source.factory.DataSource; @@ -183,7 +182,11 @@ public SplitReadResult prepareAndSubmitSplit(JobBaseRecordRequest baseReq) throw this.currentReader = getSnapshotSplitReader(baseReq); } else if (this.currentSplit instanceof MySqlBinlogSplit) { this.currentReader = getBinlogSplitReader(baseReq); + } else { + throw new IllegalStateException( + "Unsupported MySqlSplit type: " + this.currentSplit.getClass().getName()); } + this.currentReader.submitSplit(this.currentSplit); MySqlSplitState currentSplitState = null; if (this.currentSplit.isSnapshotSplit()) { @@ -423,91 +426,6 @@ private void closeChunkSplitterOnly(MySqlSnapshotSplitAssigner splitAssigner) { } } - private SplitRecords pollSplitRecordsWithSplit(MySqlSplit split, JobBaseConfig jobConfig) - throws Exception { - Preconditions.checkState(split != null, "split is null"); - SourceRecords sourceRecords = null; - String currentSplitId = null; - DebeziumReader currentReader = null; - LOG.info("Get a split: {}", split.toString()); - if (split instanceof MySqlSnapshotSplit) { - currentReader = getSnapshotSplitReader(jobConfig); - } else if (split instanceof MySqlBinlogSplit) { - currentReader = getBinlogSplitReader(jobConfig); - } - this.setCurrentReader(currentReader); - currentReader.submitSplit(split); - currentSplitId = split.splitId(); - // make split record available - sourceRecords = - pollUntilDataAvailable(currentReader, Constants.POLL_SPLIT_RECORDS_TIMEOUTS, 500); - if (currentReader instanceof SnapshotSplitReader) { - closeCurrentReader(); - } - return new SplitRecords(currentSplitId, sourceRecords.iterator()); - } - - private SplitRecords pollSplitRecordsWithCurrentReader( - DebeziumReader currentReader) throws Exception { - Iterator dataIt = null; - if (currentReader instanceof BinlogSplitReader) { - dataIt = currentReader.pollSplitRecords(); - return dataIt == null - ? null - : new SplitRecords(BINLOG_SPLIT_ID, dataIt.next().iterator()); - } else { - throw new IllegalStateException("Unsupported reader type."); - } - } - - /** - * Split tasks are submitted asynchronously, and data is sent to the Debezium queue. Therefore, - * there will be a time interval between retrieving data; it's necessary to fetch data until the - * queue has data. - */ - private SourceRecords pollUntilDataAvailable( - DebeziumReader reader, - long maxWaitTimeMs, - long pollIntervalMs) - throws InterruptedException { - long startTime = System.currentTimeMillis(); - long elapsedTime = 0; - int attemptCount = 0; - LOG.info("Polling until data available"); - Iterator lastDataIt = null; - while (elapsedTime < maxWaitTimeMs) { - attemptCount++; - lastDataIt = reader.pollSplitRecords(); - if (lastDataIt != null && lastDataIt.hasNext()) { - SourceRecords sourceRecords = lastDataIt.next(); - if (sourceRecords != null && !sourceRecords.getSourceRecordList().isEmpty()) { - LOG.info( - "Data available after {} ms ({} attempts). {} Records received.", - elapsedTime, - attemptCount, - sourceRecords.getSourceRecordList().size()); - // todo: Until debezium_heartbeat is consumed - return sourceRecords; - } - } - - // No records yet, continue polling - if (elapsedTime + pollIntervalMs < maxWaitTimeMs) { - Thread.sleep(pollIntervalMs); - elapsedTime = System.currentTimeMillis() - startTime; - } else { - // Last attempt before timeout - break; - } - } - - LOG.warn( - "Timeout: No data (heartbeat or data change) received after {} ms ({} attempts).", - elapsedTime, - attemptCount); - return new SourceRecords(new ArrayList<>()); - } - private SnapshotSplitReader getSnapshotSplitReader(JobBaseConfig config) { MySqlSourceConfig sourceConfig = getSourceConfig(config); final MySqlConnection jdbcConnection = DebeziumUtils.createMySqlConnection(sourceConfig); From 56f63e3f7127808d2252eda9379133f98d253aa6 Mon Sep 17 00:00:00 2001 From: wudi Date: Mon, 26 Jan 2026 11:25:40 +0800 Subject: [PATCH 03/11] add log --- .../insert/streaming/StreamingInsertJob.java | 2 + fs_brokers/cdc_client/pom.xml | 3 + .../MySqlStreamingChangeEventSource.java | 1571 +++++++++++++++++ .../service/PipelineCoordinator.java | 1 + .../cdcclient/sink/DorisBatchStreamLoad.java | 14 +- 5 files changed, 1589 insertions(+), 2 deletions(-) create mode 100644 fs_brokers/cdc_client/src/main/java/io/debezium/connector/mysql/MySqlStreamingChangeEventSource.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 4dade2b4ec8b11..cef86e56d630ac 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -1116,6 +1116,8 @@ public void commitOffset(CommitOffsetRequest offsetRequest) throws JobException } persistOffsetProviderIfNeed(); + log.info("Streaming multi table job {} task {} commit offset successfully, offset: {}", + getJobId(), offsetRequest.getTaskId(), offsetRequest.getOffset()); ((StreamingMultiTblTask) this.runningStreamTask).successCallback(offsetRequest); } diff --git a/fs_brokers/cdc_client/pom.xml b/fs_brokers/cdc_client/pom.xml index c2e3580be76013..96c27aba3e526f 100644 --- a/fs_brokers/cdc_client/pom.xml +++ b/fs_brokers/cdc_client/pom.xml @@ -209,6 +209,9 @@ under the License. ${spotless.version} + + src/main/java/org/apache/doris/*.java + 1.17.0 diff --git a/fs_brokers/cdc_client/src/main/java/io/debezium/connector/mysql/MySqlStreamingChangeEventSource.java b/fs_brokers/cdc_client/src/main/java/io/debezium/connector/mysql/MySqlStreamingChangeEventSource.java new file mode 100644 index 00000000000000..bbf973641e4cd0 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/io/debezium/connector/mysql/MySqlStreamingChangeEventSource.java @@ -0,0 +1,1571 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ + +package io.debezium.connector.mysql; + +import com.github.shyiko.mysql.binlog.BinaryLogClient; +import com.github.shyiko.mysql.binlog.BinaryLogClient.LifecycleListener; +import com.github.shyiko.mysql.binlog.event.DeleteRowsEventData; +import com.github.shyiko.mysql.binlog.event.Event; +import com.github.shyiko.mysql.binlog.event.EventData; +import com.github.shyiko.mysql.binlog.event.EventHeader; +import com.github.shyiko.mysql.binlog.event.EventHeaderV4; +import com.github.shyiko.mysql.binlog.event.EventType; +import com.github.shyiko.mysql.binlog.event.GtidEventData; +import com.github.shyiko.mysql.binlog.event.QueryEventData; +import com.github.shyiko.mysql.binlog.event.RotateEventData; +import com.github.shyiko.mysql.binlog.event.RowsQueryEventData; +import com.github.shyiko.mysql.binlog.event.TableMapEventData; +import com.github.shyiko.mysql.binlog.event.UpdateRowsEventData; +import com.github.shyiko.mysql.binlog.event.WriteRowsEventData; +import com.github.shyiko.mysql.binlog.event.deserialization.EventDataDeserializationException; +import com.github.shyiko.mysql.binlog.event.deserialization.EventDeserializer; +import com.github.shyiko.mysql.binlog.event.deserialization.GtidEventDataDeserializer; +import com.github.shyiko.mysql.binlog.io.ByteArrayInputStream; +import com.github.shyiko.mysql.binlog.network.AuthenticationException; +import com.github.shyiko.mysql.binlog.network.DefaultSSLSocketFactory; +import com.github.shyiko.mysql.binlog.network.SSLMode; +import com.github.shyiko.mysql.binlog.network.SSLSocketFactory; +import com.github.shyiko.mysql.binlog.network.ServerException; +import io.debezium.DebeziumException; +import io.debezium.annotation.SingleThreadAccess; +import io.debezium.config.CommonConnectorConfig.EventProcessingFailureHandlingMode; +import io.debezium.config.Configuration; +import io.debezium.connector.mysql.MySqlConnectorConfig.GtidNewChannelPosition; +import io.debezium.connector.mysql.MySqlConnectorConfig.SecureConnectionMode; +import io.debezium.connector.mysql.util.ErrorMessageUtils; +import io.debezium.data.Envelope.Operation; +import io.debezium.function.BlockingConsumer; +import io.debezium.pipeline.ErrorHandler; +import io.debezium.pipeline.EventDispatcher; +import io.debezium.pipeline.source.spi.StreamingChangeEventSource; +import io.debezium.relational.TableId; +import io.debezium.schema.SchemaChangeEvent; +import io.debezium.util.Clock; +import io.debezium.util.Metronome; +import io.debezium.util.Strings; +import io.debezium.util.Threads; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.slf4j.event.Level; + +import javax.net.ssl.KeyManager; +import javax.net.ssl.KeyManagerFactory; +import javax.net.ssl.SSLContext; +import javax.net.ssl.TrustManager; +import javax.net.ssl.TrustManagerFactory; +import javax.net.ssl.X509TrustManager; + +import java.io.IOException; +import java.security.GeneralSecurityException; +import java.security.KeyStore; +import java.security.KeyStoreException; +import java.security.NoSuchAlgorithmException; +import java.security.UnrecoverableKeyException; +import java.security.cert.CertificateException; +import java.security.cert.X509Certificate; +import java.sql.SQLException; +import java.time.Duration; +import java.time.Instant; +import java.util.EnumMap; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Predicate; + +import static io.debezium.util.Strings.isNullOrEmpty; + +/** + * Copied from FlinkCDC project(3.5.0). + * + *

Line 924 : change Log Level info to debug. + */ +public class MySqlStreamingChangeEventSource + implements StreamingChangeEventSource { + + private static final Logger LOGGER = + LoggerFactory.getLogger(MySqlStreamingChangeEventSource.class); + + private static final String KEEPALIVE_THREAD_NAME = "blc-keepalive"; + + private final EnumMap> eventHandlers = + new EnumMap<>(EventType.class); + private final BinaryLogClient client; + private final MySqlStreamingChangeEventSourceMetrics metrics; + private final Clock clock; + private final EventProcessingFailureHandlingMode eventDeserializationFailureHandlingMode; + private final EventProcessingFailureHandlingMode inconsistentSchemaHandlingMode; + + private int startingRowNumber = 0; + private long initialEventsToSkip = 0L; + private boolean skipEvent = false; + private boolean ignoreDmlEventByGtidSource = false; + private final Predicate gtidDmlSourceFilter; + private final AtomicLong totalRecordCounter = new AtomicLong(); + private volatile Map lastOffset = null; + private com.github.shyiko.mysql.binlog.GtidSet gtidSet; + private final float heartbeatIntervalFactor = 0.8f; + private final Map binaryLogClientThreads = new ConcurrentHashMap<>(4); + private final MySqlTaskContext taskContext; + private final MySqlConnectorConfig connectorConfig; + private final MySqlConnection connection; + private final EventDispatcher eventDispatcher; + private final ErrorHandler errorHandler; + + @SingleThreadAccess("binlog client thread") + private Instant eventTimestamp; + + /** Describe binlog position. */ + public static class BinlogPosition { + final String filename; + final long position; + + public BinlogPosition(String filename, long position) { + assert filename != null; + + this.filename = filename; + this.position = position; + } + + public String getFilename() { + return filename; + } + + public long getPosition() { + return position; + } + + @Override + public String toString() { + return filename + "/" + position; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + filename.hashCode(); + result = prime * result + (int) (position ^ (position >>> 32)); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + BinlogPosition other = (BinlogPosition) obj; + if (!filename.equals(other.filename)) { + return false; + } + if (position != other.position) { + return false; + } + return true; + } + } + + @FunctionalInterface + private interface BinlogChangeEmitter { + void emit(TableId tableId, T data) throws InterruptedException; + } + + public MySqlStreamingChangeEventSource( + MySqlConnectorConfig connectorConfig, + MySqlConnection connection, + EventDispatcher dispatcher, + ErrorHandler errorHandler, + Clock clock, + MySqlTaskContext taskContext, + MySqlStreamingChangeEventSourceMetrics metrics) { + + this.taskContext = taskContext; + this.connectorConfig = connectorConfig; + this.connection = connection; + this.clock = clock; + this.eventDispatcher = dispatcher; + this.errorHandler = errorHandler; + this.metrics = metrics; + + eventDeserializationFailureHandlingMode = + connectorConfig.getEventProcessingFailureHandlingMode(); + inconsistentSchemaHandlingMode = connectorConfig.inconsistentSchemaFailureHandlingMode(); + + // Set up the log reader ... + client = taskContext.getBinaryLogClient(); + // BinaryLogClient will overwrite thread names later + client.setThreadFactory( + Threads.threadFactory( + MySqlConnector.class, + connectorConfig.getLogicalName(), + "binlog-client", + false, + false, + x -> binaryLogClientThreads.put(x.getName(), x))); + client.setServerId(connectorConfig.serverId()); + client.setSSLMode(sslModeFor(connectorConfig.sslMode())); + if (connectorConfig.sslModeEnabled()) { + SSLSocketFactory sslSocketFactory = + getBinlogSslSocketFactory(connectorConfig, connection); + if (sslSocketFactory != null) { + client.setSslSocketFactory(sslSocketFactory); + } + } + Configuration configuration = connectorConfig.getConfig(); + client.setKeepAlive(configuration.getBoolean(MySqlConnectorConfig.KEEP_ALIVE)); + final long keepAliveInterval = + configuration.getLong(MySqlConnectorConfig.KEEP_ALIVE_INTERVAL_MS); + client.setKeepAliveInterval(keepAliveInterval); + // Considering heartbeatInterval should be less than keepAliveInterval, we use the + // heartbeatIntervalFactor + // multiply by keepAliveInterval and set the result value to heartbeatInterval.The default + // value of heartbeatIntervalFactor + // is 0.8, and we believe the left time (0.2 * keepAliveInterval) is enough to process the + // packet received from the MySQL server. + client.setHeartbeatInterval((long) (keepAliveInterval * heartbeatIntervalFactor)); + + boolean filterDmlEventsByGtidSource = + configuration.getBoolean(MySqlConnectorConfig.GTID_SOURCE_FILTER_DML_EVENTS); + gtidDmlSourceFilter = + filterDmlEventsByGtidSource ? connectorConfig.gtidSourceFilter() : null; + + // Set up the event deserializer with additional type(s) ... + final Map tableMapEventByTableId = + new HashMap(); + EventDeserializer eventDeserializer = + new EventDeserializer() { + @Override + public Event nextEvent(ByteArrayInputStream inputStream) throws IOException { + try { + // Delegate to the superclass ... + Event event = super.nextEvent(inputStream); + + // We have to record the most recent TableMapEventData for each table + // number for our custom deserializers ... + if (event.getHeader().getEventType() == EventType.TABLE_MAP) { + TableMapEventData tableMapEvent = event.getData(); + tableMapEventByTableId.put( + tableMapEvent.getTableId(), tableMapEvent); + } + + // DBZ-5126 Clean cache on rotate event to prevent it from growing + // indefinitely. + if (event.getHeader().getEventType() == EventType.ROTATE + && event.getHeader().getTimestamp() != 0) { + tableMapEventByTableId.clear(); + } + return event; + } + // DBZ-217 In case an event couldn't be read we create a pseudo-event for + // the sake of logging + catch (EventDataDeserializationException edde) { + // DBZ-3095 As of Java 15, when reaching EOF in the binlog stream, the + // polling loop in + // BinaryLogClient#listenForEventPackets() keeps returning values != -1 + // from peek(); + // this causes the loop to never finish + // Propagating the exception (either EOF or socket closed) causes the + // loop to be aborted + // in this case + if (edde.getCause() instanceof IOException) { + throw edde; + } + + EventHeaderV4 header = new EventHeaderV4(); + header.setEventType(EventType.INCIDENT); + header.setTimestamp(edde.getEventHeader().getTimestamp()); + header.setServerId(edde.getEventHeader().getServerId()); + + if (edde.getEventHeader() instanceof EventHeaderV4) { + header.setEventLength( + ((EventHeaderV4) edde.getEventHeader()).getEventLength()); + header.setNextPosition( + ((EventHeaderV4) edde.getEventHeader()).getNextPosition()); + header.setFlags(((EventHeaderV4) edde.getEventHeader()).getFlags()); + } + + EventData data = new EventDataDeserializationExceptionData(edde); + return new Event(header, data); + } + } + }; + + // Add our custom deserializers ... + eventDeserializer.setEventDataDeserializer(EventType.STOP, new StopEventDataDeserializer()); + eventDeserializer.setEventDataDeserializer(EventType.GTID, new GtidEventDataDeserializer()); + eventDeserializer.setEventDataDeserializer( + EventType.WRITE_ROWS, + new RowDeserializers.WriteRowsDeserializer(tableMapEventByTableId)); + eventDeserializer.setEventDataDeserializer( + EventType.UPDATE_ROWS, + new RowDeserializers.UpdateRowsDeserializer(tableMapEventByTableId)); + eventDeserializer.setEventDataDeserializer( + EventType.DELETE_ROWS, + new RowDeserializers.DeleteRowsDeserializer(tableMapEventByTableId)); + eventDeserializer.setEventDataDeserializer( + EventType.EXT_WRITE_ROWS, + new RowDeserializers.WriteRowsDeserializer(tableMapEventByTableId) + .setMayContainExtraInformation(true)); + eventDeserializer.setEventDataDeserializer( + EventType.EXT_UPDATE_ROWS, + new RowDeserializers.UpdateRowsDeserializer(tableMapEventByTableId) + .setMayContainExtraInformation(true)); + eventDeserializer.setEventDataDeserializer( + EventType.EXT_DELETE_ROWS, + new RowDeserializers.DeleteRowsDeserializer(tableMapEventByTableId) + .setMayContainExtraInformation(true)); + client.setEventDeserializer(eventDeserializer); + } + + protected void onEvent(MySqlOffsetContext offsetContext, Event event) { + long ts = 0; + + if (event.getHeader().getEventType() == EventType.HEARTBEAT) { + // HEARTBEAT events have no timestamp but are fired only when + // there is no traffic on the connection which means we are caught-up + // https://dev.mysql.com/doc/internals/en/heartbeat-event.html + metrics.setMilliSecondsBehindSource(ts); + return; + } + + // MySQL has seconds resolution but mysql-binlog-connector-java returns + // a value in milliseconds + long eventTs = event.getHeader().getTimestamp(); + + if (eventTs == 0) { + LOGGER.trace("Received unexpected event with 0 timestamp: {}", event); + return; + } + + ts = clock.currentTimeInMillis() - eventTs; + LOGGER.trace("Current milliseconds behind source: {} ms", ts); + metrics.setMilliSecondsBehindSource(ts); + } + + protected void ignoreEvent(MySqlOffsetContext offsetContext, Event event) { + LOGGER.trace("Ignoring event due to missing handler: {}", event); + } + + protected void handleEvent( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) { + if (event == null) { + return; + } + + final EventHeader eventHeader = event.getHeader(); + // Update the source offset info. Note that the client returns the value in *milliseconds*, + // even though the binlog + // contains only *seconds* precision ... + // HEARTBEAT events have no timestamp; only set the timestamp if the event is not a + // HEARTBEAT + eventTimestamp = + !eventHeader.getEventType().equals(EventType.HEARTBEAT) + ? Instant.ofEpochMilli(eventHeader.getTimestamp()) + : null; + offsetContext.setBinlogServerId(eventHeader.getServerId()); + + final EventType eventType = eventHeader.getEventType(); + if (eventType == EventType.ROTATE) { + EventData eventData = event.getData(); + RotateEventData rotateEventData; + if (eventData instanceof EventDeserializer.EventDataWrapper) { + rotateEventData = + (RotateEventData) + ((EventDeserializer.EventDataWrapper) eventData).getInternal(); + } else { + rotateEventData = (RotateEventData) eventData; + } + offsetContext.setBinlogStartPoint( + rotateEventData.getBinlogFilename(), rotateEventData.getBinlogPosition()); + } else if (eventHeader instanceof EventHeaderV4) { + EventHeaderV4 trackableEventHeader = (EventHeaderV4) eventHeader; + offsetContext.setEventPosition( + trackableEventHeader.getPosition(), trackableEventHeader.getEventLength()); + } + + // If there is a handler for this event, forward the event to it ... + try { + // Forward the event to the handler ... + eventHandlers + .getOrDefault(eventType, (e) -> ignoreEvent(offsetContext, e)) + .accept(event); + + // Generate heartbeat message if the time is right + eventDispatcher.dispatchHeartbeatEvent(partition, offsetContext); + + // Capture that we've completed another event ... + offsetContext.completeEvent(); + + // update last offset used for logging + lastOffset = offsetContext.getOffset(); + + if (skipEvent) { + // We're in the mode of skipping events and we just skipped this one, so decrement + // our skip count ... + --initialEventsToSkip; + skipEvent = initialEventsToSkip > 0; + } + } catch (RuntimeException e) { + // There was an error in the event handler, so propagate the failure to Kafka Connect + // ... + logStreamingSourceState(); + errorHandler.setProducerThrowable( + new DebeziumException("Error processing binlog event", e)); + // Do not stop the client, since Kafka Connect should stop the connector on it's own + // (and doing it here may cause problems the second time it is stopped). + // We can clear the listeners though so that we ignore all future events ... + eventHandlers.clear(); + LOGGER.info( + "Error processing binlog event, and propagating to Kafka Connect so it stops this connector. Future binlog events read before connector is shutdown will be ignored."); + } catch (InterruptedException e) { + // Most likely because this reader was stopped and our thread was interrupted ... + Thread.currentThread().interrupt(); + eventHandlers.clear(); + LOGGER.info("Stopped processing binlog events due to thread interruption"); + } + } + + @SuppressWarnings("unchecked") + protected T unwrapData(Event event) { + EventData eventData = event.getData(); + if (eventData instanceof EventDeserializer.EventDataWrapper) { + eventData = ((EventDeserializer.EventDataWrapper) eventData).getInternal(); + } + return (T) eventData; + } + + /** + * Handle the supplied event that signals that mysqld has stopped. + * + * @param event the server stopped event to be processed; may not be null + */ + protected void handleServerStop(MySqlOffsetContext offsetContext, Event event) { + LOGGER.debug("Server stopped: {}", event); + } + + /** + * Handle the supplied event that is sent by a primary to a replica to let the replica know that + * the primary is still alive. Not written to a binary log. + * + * @param event the server stopped event to be processed; may not be null + */ + protected void handleServerHeartbeat( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) + throws InterruptedException { + LOGGER.trace("Server heartbeat: {}", event); + eventDispatcher.dispatchServerHeartbeatEvent(partition, offsetContext); + } + + /** + * Handle the supplied event that signals that an out of the ordinary event that occurred on the + * master. It notifies the replica that something happened on the primary that might cause data + * to be in an inconsistent state. + * + * @param event the server stopped event to be processed; may not be null + */ + protected void handleServerIncident( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) { + if (event.getData() instanceof EventDataDeserializationExceptionData) { + metrics.onErroneousEvent(partition, "source = " + event); + EventDataDeserializationExceptionData data = event.getData(); + + EventHeaderV4 eventHeader = + (EventHeaderV4) + data.getCause() + .getEventHeader(); // safe cast, instantiated that ourselves + + // logging some additional context but not the exception itself, this will happen in + // handleEvent() + if (eventDeserializationFailureHandlingMode + == EventProcessingFailureHandlingMode.FAIL) { + LOGGER.error( + "Error while deserializing binlog event at offset {}.{}" + + "Use the mysqlbinlog tool to view the problematic event: mysqlbinlog --start-position={} --stop-position={} --verbose {}", + offsetContext.getOffset(), + System.lineSeparator(), + eventHeader.getPosition(), + eventHeader.getNextPosition(), + offsetContext.getSource().binlogFilename()); + + throw new RuntimeException(data.getCause()); + } else if (eventDeserializationFailureHandlingMode + == EventProcessingFailureHandlingMode.WARN) { + LOGGER.warn( + "Error while deserializing binlog event at offset {}.{}" + + "This exception will be ignored and the event be skipped.{}" + + "Use the mysqlbinlog tool to view the problematic event: mysqlbinlog --start-position={} --stop-position={} --verbose {}", + offsetContext.getOffset(), + System.lineSeparator(), + System.lineSeparator(), + eventHeader.getPosition(), + eventHeader.getNextPosition(), + offsetContext.getSource().binlogFilename(), + data.getCause()); + } + } else { + LOGGER.error("Server incident: {}", event); + } + } + + /** + * Handle the supplied event with a {@link RotateEventData} that signals the logs are being + * rotated. This means that either the server was restarted, or the binlog has transitioned to a + * new file. In either case, subsequent table numbers will be different than those seen to this + * point. + * + * @param event the database change data event to be processed; may not be null + */ + protected void handleRotateLogsEvent(MySqlOffsetContext offsetContext, Event event) { + LOGGER.debug("Rotating logs: {}", event); + RotateEventData command = unwrapData(event); + assert command != null; + taskContext.getSchema().clearTableMappings(); + } + + /** + * Handle the supplied event with a {@link GtidEventData} that signals the beginning of a GTID + * transaction. We don't yet know whether this transaction contains any events we're interested + * in, but we have to record it so that we know the position of this event and know we've + * processed the binlog to this point. + * + *

Note that this captures the current GTID and complete GTID set, regardless of whether the + * connector is {@link MySqlTaskContext#gtidSourceFilter() filtering} the GTID set upon + * connection. We do this because we actually want to capture all GTID set values found in the + * binlog, whether or not we process them. However, only when we connect do we actually want to + * pass to MySQL only those GTID ranges that are applicable per the configuration. + * + * @param event the GTID event to be processed; may not be null + */ + protected void handleGtidEvent(MySqlOffsetContext offsetContext, Event event) { + LOGGER.debug("GTID transaction: {}", event); + GtidEventData gtidEvent = unwrapData(event); + String gtid = gtidEvent.getGtid(); + gtidSet.add(gtid); + offsetContext.startGtid(gtid, gtidSet.toString()); // rather than use the client's GTID set + ignoreDmlEventByGtidSource = false; + if (gtidDmlSourceFilter != null && gtid != null) { + String uuid = gtid.trim().substring(0, gtid.indexOf(":")); + if (!gtidDmlSourceFilter.test(uuid)) { + ignoreDmlEventByGtidSource = true; + } + } + metrics.onGtidChange(gtid); + } + + /** + * Handle the supplied event with an {@link RowsQueryEventData} by recording the original SQL + * query that generated the event. + * + * @param event the database change data event to be processed; may not be null + */ + protected void handleRowsQuery(MySqlOffsetContext offsetContext, Event event) { + // Unwrap the RowsQueryEvent + final RowsQueryEventData lastRowsQueryEventData = unwrapData(event); + + // Set the query on the source + offsetContext.setQuery(lastRowsQueryEventData.getQuery()); + } + + /** + * Handle the supplied event with an {@link QueryEventData} by possibly recording the DDL + * statements as changes in the MySQL schemas. + * + * @param partition the partition in which the even occurred + * @param event the database change data event to be processed; may not be null + * @throws InterruptedException if this thread is interrupted while recording the DDL statements + */ + protected void handleQueryEvent( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) + throws InterruptedException { + QueryEventData command = unwrapData(event); + LOGGER.debug("Received query command: {}", event); + String sql = command.getSql().trim(); + if (sql.equalsIgnoreCase("BEGIN")) { + // We are starting a new transaction ... + offsetContext.startNextTransaction(); + eventDispatcher.dispatchTransactionStartedEvent( + partition, offsetContext.getTransactionId(), offsetContext); + offsetContext.setBinlogThread(command.getThreadId()); + if (initialEventsToSkip != 0) { + LOGGER.debug( + "Restarting partially-processed transaction; change events will not be created for the first {} events plus {} more rows in the next event", + initialEventsToSkip, + startingRowNumber); + // We are restarting, so we need to skip the events in this transaction that we + // processed previously... + skipEvent = true; + } + return; + } + if (sql.equalsIgnoreCase("COMMIT")) { + handleTransactionCompletion(partition, offsetContext, event); + return; + } + + String upperCasedStatementBegin = Strings.getBegin(sql, 7).toUpperCase(); + + if (upperCasedStatementBegin.startsWith("XA ")) { + // This is an XA transaction, and we currently ignore these and do nothing ... + return; + } + if (connectorConfig.getDdlFilter().test(sql)) { + LOGGER.debug("DDL '{}' was filtered out of processing", sql); + return; + } + if (upperCasedStatementBegin.equals("INSERT ") + || upperCasedStatementBegin.equals("UPDATE ") + || upperCasedStatementBegin.equals("DELETE ")) { + LOGGER.warn( + "Received DML '" + + sql + + "' for processing, binlog probably contains events generated with statement or mixed based replication format"); + return; + } + if (sql.equalsIgnoreCase("ROLLBACK")) { + // We have hit a ROLLBACK which is not supported + LOGGER.warn( + "Rollback statements cannot be handled without binlog buffering, the connector will fail. Please check '{}' to see how to enable buffering", + MySqlConnectorConfig.BUFFER_SIZE_FOR_BINLOG_READER.name()); + } + + final List schemaChangeEvents = + taskContext + .getSchema() + .parseStreamingDdl( + partition, + sql, + command.getDatabase(), + offsetContext, + clock.currentTimeAsInstant()); + try { + for (SchemaChangeEvent schemaChangeEvent : schemaChangeEvents) { + if (taskContext.getSchema().skipSchemaChangeEvent(schemaChangeEvent)) { + continue; + } + + final TableId tableId = + schemaChangeEvent.getTables().isEmpty() + ? null + : schemaChangeEvent.getTables().iterator().next().id(); + eventDispatcher.dispatchSchemaChangeEvent( + partition, + tableId, + (receiver) -> { + try { + receiver.schemaChangeEvent(schemaChangeEvent); + } catch (Exception e) { + throw new DebeziumException(e); + } + }); + } + } catch (InterruptedException e) { + LOGGER.info("Processing interrupted"); + } + } + + private void handleTransactionCompletion( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) + throws InterruptedException { + // We are completing the transaction ... + eventDispatcher.dispatchTransactionCommittedEvent(partition, offsetContext); + offsetContext.commitTransaction(); + offsetContext.setBinlogThread(-1L); + skipEvent = false; + ignoreDmlEventByGtidSource = false; + } + + /** + * Handle a change in the table metadata. + * + *

This method should be called whenever we consume a TABLE_MAP event, and every transaction + * in the log should include one of these for each table affected by the transaction. Each table + * map event includes a monotonically-increasing numeric identifier, and this identifier is used + * within subsequent events within the same transaction. This table identifier can change when: + * + *

    + *
  1. the table structure is modified (e.g., via an {@code ALTER TABLE ...} command); or + *
  2. MySQL rotates to a new binary log file, even if the table structure does not change. + *
+ * + * @param event the update event; never null + */ + protected void handleUpdateTableMetadata( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) + throws InterruptedException { + TableMapEventData metadata = unwrapData(event); + long tableNumber = metadata.getTableId(); + String databaseName = metadata.getDatabase(); + String tableName = metadata.getTable(); + TableId tableId = new TableId(databaseName, null, tableName); + if (taskContext.getSchema().assignTableNumber(tableNumber, tableId)) { + LOGGER.debug("Received update table metadata event: {}", event); + } else { + informAboutUnknownTableIfRequired(partition, offsetContext, event, tableId); + } + } + + /** + * If we receive an event for a table that is monitored but whose metadata we don't know, either + * ignore that event or raise a warning or error as per the {@link + * MySqlConnectorConfig#INCONSISTENT_SCHEMA_HANDLING_MODE} configuration. + */ + private void informAboutUnknownTableIfRequired( + MySqlPartition partition, + MySqlOffsetContext offsetContext, + Event event, + TableId tableId, + Operation operation) + throws InterruptedException { + if (tableId != null + && connectorConfig.getTableFilters().dataCollectionFilter().isIncluded(tableId)) { + metrics.onErroneousEvent( + partition, "source = " + tableId + ", event " + event, operation); + EventHeaderV4 eventHeader = event.getHeader(); + + if (inconsistentSchemaHandlingMode == EventProcessingFailureHandlingMode.FAIL) { + LOGGER.error( + "Encountered change event '{}' at offset {} for table {} whose schema isn't known to this connector. One possible cause is an incomplete database history topic. Take a new snapshot in this case.{}" + + "Use the mysqlbinlog tool to view the problematic event: mysqlbinlog --start-position={} --stop-position={} --verbose {}", + event, + offsetContext.getOffset(), + tableId, + System.lineSeparator(), + eventHeader.getPosition(), + eventHeader.getNextPosition(), + offsetContext.getSource().binlogFilename()); + throw new DebeziumException( + "Encountered change event for table " + + tableId + + " whose schema isn't known to this connector"); + } else if (inconsistentSchemaHandlingMode == EventProcessingFailureHandlingMode.WARN) { + LOGGER.warn( + "Encountered change event '{}' at offset {} for table {} whose schema isn't known to this connector. One possible cause is an incomplete database history topic. Take a new snapshot in this case.{}" + + "The event will be ignored.{}" + + "Use the mysqlbinlog tool to view the problematic event: mysqlbinlog --start-position={} --stop-position={} --verbose {}", + event, + offsetContext.getOffset(), + tableId, + System.lineSeparator(), + System.lineSeparator(), + eventHeader.getPosition(), + eventHeader.getNextPosition(), + offsetContext.getSource().binlogFilename()); + } else { + LOGGER.debug( + "Encountered change event '{}' at offset {} for table {} whose schema isn't known to this connector. One possible cause is an incomplete database history topic. Take a new snapshot in this case.{}" + + "The event will be ignored.{}" + + "Use the mysqlbinlog tool to view the problematic event: mysqlbinlog --start-position={} --stop-position={} --verbose {}", + event, + offsetContext.getOffset(), + tableId, + System.lineSeparator(), + System.lineSeparator(), + eventHeader.getPosition(), + eventHeader.getNextPosition(), + offsetContext.getSource().binlogFilename()); + } + } else { + if (tableId == null) { + EventData eventData = unwrapData(event); + if (eventData instanceof WriteRowsEventData) { + tableId = + taskContext + .getSchema() + .getExcludeTableId( + ((WriteRowsEventData) eventData).getTableId()); + } else if (eventData instanceof UpdateRowsEventData) { + tableId = + taskContext + .getSchema() + .getExcludeTableId( + ((UpdateRowsEventData) eventData).getTableId()); + } else if (eventData instanceof DeleteRowsEventData) { + tableId = + taskContext + .getSchema() + .getExcludeTableId( + ((DeleteRowsEventData) eventData).getTableId()); + } + } + LOGGER.trace("Filtered {} event for {}", event.getHeader().getEventType(), tableId); + metrics.onFilteredEvent(partition, "source = " + tableId, operation); + eventDispatcher.dispatchFilteredEvent(partition, offsetContext); + } + } + + private void informAboutUnknownTableIfRequired( + MySqlPartition partition, + MySqlOffsetContext offsetContext, + Event event, + TableId tableId) + throws InterruptedException { + informAboutUnknownTableIfRequired(partition, offsetContext, event, tableId, null); + } + + /** + * Generate source records for the supplied event with an {@link WriteRowsEventData}. + * + * @param partition the partition in which the even occurred + * @param event the database change data event to be processed; may not be null + * @throws InterruptedException if this thread is interrupted while blocking + */ + protected void handleInsert( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) + throws InterruptedException { + handleChange( + partition, + offsetContext, + event, + Operation.CREATE, + WriteRowsEventData.class, + x -> taskContext.getSchema().getTableId(x.getTableId()), + WriteRowsEventData::getRows, + (tableId, row) -> + eventDispatcher.dispatchDataChangeEvent( + partition, + tableId, + new MySqlChangeRecordEmitter( + partition, + offsetContext, + clock, + Operation.CREATE, + null, + row))); + } + + /** + * Generate source records for the supplied event with an {@link UpdateRowsEventData}. + * + * @param partition the partition in which the even occurred + * @param event the database change data event to be processed; may not be null + * @throws InterruptedException if this thread is interrupted while blocking + */ + protected void handleUpdate( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) + throws InterruptedException { + handleChange( + partition, + offsetContext, + event, + Operation.UPDATE, + UpdateRowsEventData.class, + x -> taskContext.getSchema().getTableId(x.getTableId()), + UpdateRowsEventData::getRows, + (tableId, row) -> + eventDispatcher.dispatchDataChangeEvent( + partition, + tableId, + new MySqlChangeRecordEmitter( + partition, + offsetContext, + clock, + Operation.UPDATE, + row.getKey(), + row.getValue()))); + } + + /** + * Generate source records for the supplied event with an {@link DeleteRowsEventData}. + * + * @param partition the partition in which the even occurred + * @param event the database change data event to be processed; may not be null + * @throws InterruptedException if this thread is interrupted while blocking + */ + protected void handleDelete( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) + throws InterruptedException { + handleChange( + partition, + offsetContext, + event, + Operation.DELETE, + DeleteRowsEventData.class, + x -> taskContext.getSchema().getTableId(x.getTableId()), + DeleteRowsEventData::getRows, + (tableId, row) -> + eventDispatcher.dispatchDataChangeEvent( + partition, + tableId, + new MySqlChangeRecordEmitter( + partition, + offsetContext, + clock, + Operation.DELETE, + row, + null))); + } + + private void handleChange( + MySqlPartition partition, + MySqlOffsetContext offsetContext, + Event event, + Operation operation, + Class eventDataClass, + TableIdProvider tableIdProvider, + RowsProvider rowsProvider, + BinlogChangeEmitter changeEmitter) + throws InterruptedException { + if (skipEvent) { + // We can skip this because we should already be at least this far ... + LOGGER.debug("Skipping previously processed row event: {}", event); + return; + } + if (ignoreDmlEventByGtidSource) { + LOGGER.debug("Skipping DML event because this GTID source is filtered: {}", event); + return; + } + final T data = unwrapData(event); + final TableId tableId = tableIdProvider.getTableId(data); + final List rows = rowsProvider.getRows(data); + String changeType = operation.name(); + + if (tableId != null && taskContext.getSchema().schemaFor(tableId) != null) { + int count = 0; + int numRows = rows.size(); + if (startingRowNumber < numRows) { + for (int row = startingRowNumber; row != numRows; ++row) { + offsetContext.setRowNumber(row, numRows); + offsetContext.event(tableId, eventTimestamp); + changeEmitter.emit(tableId, rows.get(row)); + count++; + } + if (LOGGER.isDebugEnabled()) { + if (startingRowNumber != 0) { + LOGGER.debug( + "Emitted {} {} record(s) for last {} row(s) in event: {}", + count, + changeType, + numRows - startingRowNumber, + event); + } else { + LOGGER.debug( + "Emitted {} {} record(s) for event: {}", count, changeType, event); + } + } + offsetContext.changeEventCompleted(); + } else { + // All rows were previously processed ... + LOGGER.debug("Skipping previously processed {} event: {}", changeType, event); + } + } else { + informAboutUnknownTableIfRequired(partition, offsetContext, event, tableId, operation); + } + startingRowNumber = 0; + } + + /** + * Handle a {@link EventType#VIEW_CHANGE} event. + * + * @param event the database change data event to be processed; may not be null + * @throws InterruptedException if this thread is interrupted while blocking + */ + protected void viewChange(MySqlOffsetContext offsetContext, Event event) + throws InterruptedException { + LOGGER.debug("View Change event: {}", event); + // do nothing + } + + /** + * Handle a {@link EventType#XA_PREPARE} event. + * + * @param event the database change data event to be processed; may not be null + * @throws InterruptedException if this thread is interrupted while blocking + */ + protected void prepareTransaction(MySqlOffsetContext offsetContext, Event event) + throws InterruptedException { + LOGGER.debug("XA Prepare event: {}", event); + // do nothing + } + + private SSLMode sslModeFor(SecureConnectionMode mode) { + switch (mode) { + case DISABLED: + return SSLMode.DISABLED; + case PREFERRED: + return SSLMode.PREFERRED; + case REQUIRED: + return SSLMode.REQUIRED; + case VERIFY_CA: + return SSLMode.VERIFY_CA; + case VERIFY_IDENTITY: + return SSLMode.VERIFY_IDENTITY; + } + return null; + } + + @Override + public void execute( + ChangeEventSourceContext context, + MySqlPartition partition, + MySqlOffsetContext offsetContext) + throws InterruptedException { + if (!connectorConfig.getSnapshotMode().shouldStream()) { + LOGGER.info( + "Streaming is disabled for snapshot mode {}", + connectorConfig.getSnapshotMode()); + return; + } + if (connectorConfig.getSnapshotMode() != MySqlConnectorConfig.SnapshotMode.NEVER) { + taskContext.getSchema().assureNonEmptySchema(); + } + final Set skippedOperations = connectorConfig.getSkippedOperations(); + + final MySqlOffsetContext effectiveOffsetContext = + offsetContext != null ? offsetContext : MySqlOffsetContext.initial(connectorConfig); + + // Register our event handlers ... + eventHandlers.put( + EventType.STOP, (event) -> handleServerStop(effectiveOffsetContext, event)); + eventHandlers.put( + EventType.HEARTBEAT, + (event) -> handleServerHeartbeat(partition, effectiveOffsetContext, event)); + eventHandlers.put( + EventType.INCIDENT, + (event) -> handleServerIncident(partition, effectiveOffsetContext, event)); + eventHandlers.put( + EventType.ROTATE, (event) -> handleRotateLogsEvent(effectiveOffsetContext, event)); + eventHandlers.put( + EventType.TABLE_MAP, + (event) -> handleUpdateTableMetadata(partition, effectiveOffsetContext, event)); + eventHandlers.put( + EventType.QUERY, + (event) -> handleQueryEvent(partition, effectiveOffsetContext, event)); + + if (!skippedOperations.contains(Operation.CREATE)) { + eventHandlers.put( + EventType.WRITE_ROWS, + (event) -> handleInsert(partition, effectiveOffsetContext, event)); + eventHandlers.put( + EventType.EXT_WRITE_ROWS, + (event) -> handleInsert(partition, effectiveOffsetContext, event)); + } + + if (!skippedOperations.contains(Operation.UPDATE)) { + eventHandlers.put( + EventType.UPDATE_ROWS, + (event) -> handleUpdate(partition, effectiveOffsetContext, event)); + eventHandlers.put( + EventType.EXT_UPDATE_ROWS, + (event) -> handleUpdate(partition, effectiveOffsetContext, event)); + } + + if (!skippedOperations.contains(Operation.DELETE)) { + eventHandlers.put( + EventType.DELETE_ROWS, + (event) -> handleDelete(partition, effectiveOffsetContext, event)); + eventHandlers.put( + EventType.EXT_DELETE_ROWS, + (event) -> handleDelete(partition, effectiveOffsetContext, event)); + } + + eventHandlers.put( + EventType.VIEW_CHANGE, (event) -> viewChange(effectiveOffsetContext, event)); + eventHandlers.put( + EventType.XA_PREPARE, (event) -> prepareTransaction(effectiveOffsetContext, event)); + eventHandlers.put( + EventType.XID, + (event) -> handleTransactionCompletion(partition, effectiveOffsetContext, event)); + + // Conditionally register ROWS_QUERY handler to parse SQL statements. + if (connectorConfig.includeSqlQuery()) { + eventHandlers.put( + EventType.ROWS_QUERY, + (event) -> handleRowsQuery(effectiveOffsetContext, event)); + } + + BinaryLogClient.EventListener listener; + if (connectorConfig.bufferSizeForStreamingChangeEventSource() == 0) { + listener = (event) -> handleEvent(partition, effectiveOffsetContext, event); + } else { + EventBuffer buffer = + new EventBuffer( + connectorConfig.bufferSizeForStreamingChangeEventSource(), + this, + context); + listener = (event) -> buffer.add(partition, effectiveOffsetContext, event); + } + client.registerEventListener(listener); + + client.registerLifecycleListener(new ReaderThreadLifecycleListener(effectiveOffsetContext)); + client.registerEventListener((event) -> onEvent(effectiveOffsetContext, event)); + if (LOGGER.isDebugEnabled()) { + client.registerEventListener((event) -> logEvent(effectiveOffsetContext, event)); + } + + final boolean isGtidModeEnabled = connection.isGtidModeEnabled(); + metrics.setIsGtidModeEnabled(isGtidModeEnabled); + + // Get the current GtidSet from MySQL so we can get a filtered/merged GtidSet based off of + // the last Debezium checkpoint. + String availableServerGtidStr = connection.knownGtidSet(); + if (isGtidModeEnabled) { + // The server is using GTIDs, so enable the handler ... + eventHandlers.put( + EventType.GTID, (event) -> handleGtidEvent(effectiveOffsetContext, event)); + + // Now look at the GTID set from the server and what we've previously seen ... + GtidSet availableServerGtidSet = new GtidSet(availableServerGtidStr); + + // also take into account purged GTID logs + GtidSet purgedServerGtidSet = connection.purgedGtidSet(); + LOGGER.info("GTID set purged on server: {}", purgedServerGtidSet); + + GtidSet filteredGtidSet = + filterGtidSet( + effectiveOffsetContext, availableServerGtidSet, purgedServerGtidSet); + if (filteredGtidSet != null) { + // We've seen at least some GTIDs, so start reading from the filtered GTID set ... + LOGGER.info("Registering binlog reader with GTID set: {}", filteredGtidSet); + String filteredGtidSetStr = filteredGtidSet.toString(); + client.setGtidSet(filteredGtidSetStr); + effectiveOffsetContext.setCompletedGtidSet(filteredGtidSetStr); + gtidSet = new com.github.shyiko.mysql.binlog.GtidSet(filteredGtidSetStr); + } else { + // We've not yet seen any GTIDs, so that means we have to start reading the binlog + // from the beginning ... + client.setBinlogFilename(effectiveOffsetContext.getSource().binlogFilename()); + client.setBinlogPosition(effectiveOffsetContext.getSource().binlogPosition()); + gtidSet = new com.github.shyiko.mysql.binlog.GtidSet(""); + } + } else { + // The server is not using GTIDs, so start reading the binlog based upon where we last + // left off ... + client.setBinlogFilename(effectiveOffsetContext.getSource().binlogFilename()); + client.setBinlogPosition(effectiveOffsetContext.getSource().binlogPosition()); + } + + // We may be restarting in the middle of a transaction, so see how far into the transaction + // we have already processed... + initialEventsToSkip = effectiveOffsetContext.eventsToSkipUponRestart(); + LOGGER.info("Skip {} events on streaming start", initialEventsToSkip); + + // Set the starting row number, which is the next row number to be read ... + startingRowNumber = effectiveOffsetContext.rowsToSkipUponRestart(); + LOGGER.info("Skip {} rows on streaming start", startingRowNumber); + + // Only when we reach the first BEGIN event will we start to skip events ... + skipEvent = false; + + try { + // Start the log reader, which starts background threads ... + if (context.isRunning()) { + long timeout = connectorConfig.getConnectionTimeout().toMillis(); + long started = clock.currentTimeInMillis(); + try { + LOGGER.debug( + "Attempting to establish binlog reader connection with timeout of {} ms", + timeout); + client.connect(timeout); + // Need to wait for keepalive thread to be running, otherwise it can be left + // orphaned + // The problem is with timing. When the close is called too early after connect + // then + // the keepalive thread is not terminated + if (client.isKeepAlive()) { + LOGGER.info("Waiting for keepalive thread to start"); + final Metronome metronome = Metronome.parker(Duration.ofMillis(100), clock); + int waitAttempts = 50; + boolean keepAliveThreadRunning = false; + while (!keepAliveThreadRunning && waitAttempts-- > 0) { + for (Thread t : binaryLogClientThreads.values()) { + if (t.getName().startsWith(KEEPALIVE_THREAD_NAME) && t.isAlive()) { + LOGGER.info("Keepalive thread is running"); + keepAliveThreadRunning = true; + } + } + metronome.pause(); + } + } + } catch (TimeoutException e) { + // If the client thread is interrupted *before* the client could connect, the + // client throws a timeout exception + // The only way we can distinguish this is if we get the timeout exception + // before the specified timeout has + // elapsed, so we simply check this (within 10%) ... + long duration = clock.currentTimeInMillis() - started; + if (duration > (0.9 * timeout)) { + double actualSeconds = TimeUnit.MILLISECONDS.toSeconds(duration); + throw new DebeziumException( + "Timed out after " + + actualSeconds + + " seconds while waiting to connect to MySQL at " + + connectorConfig.hostname() + + ":" + + connectorConfig.port() + + " with user '" + + connectorConfig.username() + + "'", + e); + } + // Otherwise, we were told to shutdown, so we don't care about the timeout + // exception + } catch (AuthenticationException e) { + throw new DebeziumException( + "Failed to authenticate to the MySQL database at " + + connectorConfig.hostname() + + ":" + + connectorConfig.port() + + " with user '" + + connectorConfig.username() + + "'", + e); + } catch (Throwable e) { + throw new DebeziumException( + "Unable to connect to the MySQL database at " + + connectorConfig.hostname() + + ":" + + connectorConfig.port() + + " with user '" + + connectorConfig.username() + + "': " + + e.getMessage(), + e); + } + } + while (context.isRunning()) { + Thread.sleep(100); + } + } finally { + try { + client.disconnect(); + } catch (Exception e) { + LOGGER.info("Exception while stopping binary log client", e); + } + } + } + + private SSLSocketFactory getBinlogSslSocketFactory( + MySqlConnectorConfig connectorConfig, MySqlConnection connection) { + String acceptedTlsVersion = connection.getSessionVariableForSslVersion(); + if (!isNullOrEmpty(acceptedTlsVersion)) { + SSLMode sslMode = sslModeFor(connectorConfig.sslMode()); + LOGGER.info( + "Enable ssl " + + sslMode + + " mode for connector " + + connectorConfig.getLogicalName()); + + final char[] keyPasswordArray = connection.connectionConfig().sslKeyStorePassword(); + final String keyFilename = connection.connectionConfig().sslKeyStore(); + final char[] trustPasswordArray = connection.connectionConfig().sslTrustStorePassword(); + final String trustFilename = connection.connectionConfig().sslTrustStore(); + KeyManager[] keyManagers = null; + if (keyFilename != null) { + try { + KeyStore ks = connection.loadKeyStore(keyFilename, keyPasswordArray); + + KeyManagerFactory kmf = KeyManagerFactory.getInstance("NewSunX509"); + kmf.init(ks, keyPasswordArray); + + keyManagers = kmf.getKeyManagers(); + } catch (KeyStoreException + | NoSuchAlgorithmException + | UnrecoverableKeyException e) { + throw new DebeziumException("Could not load keystore", e); + } + } + TrustManager[] trustManagers; + try { + KeyStore ks = null; + if (trustFilename != null) { + ks = connection.loadKeyStore(trustFilename, trustPasswordArray); + } + + if (ks == null && (sslMode == SSLMode.PREFERRED || sslMode == SSLMode.REQUIRED)) { + trustManagers = + new TrustManager[] { + new X509TrustManager() { + + @Override + public void checkClientTrusted( + X509Certificate[] x509Certificates, String s) + throws CertificateException {} + + @Override + public void checkServerTrusted( + X509Certificate[] x509Certificates, String s) + throws CertificateException {} + + @Override + public X509Certificate[] getAcceptedIssuers() { + return new X509Certificate[0]; + } + } + }; + } else { + TrustManagerFactory tmf = + TrustManagerFactory.getInstance( + TrustManagerFactory.getDefaultAlgorithm()); + tmf.init(ks); + trustManagers = tmf.getTrustManagers(); + } + } catch (KeyStoreException | NoSuchAlgorithmException e) { + throw new DebeziumException("Could not load truststore", e); + } + // DBZ-1208 Resembles the logic from the upstream BinaryLogClient, only that + // the accepted TLS version is passed to the constructed factory + final KeyManager[] finalKMS = keyManagers; + return new DefaultSSLSocketFactory(acceptedTlsVersion) { + + @Override + protected void initSSLContext(SSLContext sc) throws GeneralSecurityException { + sc.init(finalKMS, trustManagers, null); + } + }; + } + + return null; + } + + private void logStreamingSourceState() { + logStreamingSourceState(Level.ERROR); + } + + protected void logEvent(MySqlOffsetContext offsetContext, Event event) { + LOGGER.trace("Received event: {}", event); + } + + private void logStreamingSourceState(Level severity) { + final Object position = + client == null + ? "N/A" + : client.getBinlogFilename() + "/" + client.getBinlogPosition(); + final String message = + "Error during binlog processing. Last offset stored = {}, binlog reader near position = {}"; + switch (severity) { + case WARN: + LOGGER.warn(message, lastOffset, position); + break; + case DEBUG: + LOGGER.debug(message, lastOffset, position); + break; + default: + LOGGER.error(message, lastOffset, position); + } + } + + /** + * Apply the include/exclude GTID source filters to the current {@link #source() GTID set} and + * merge them onto the currently available GTID set from a MySQL server. + * + *

The merging behavior of this method might seem a bit strange at first. It's required in + * order for Debezium to consume a MySQL binlog that has multi-source replication enabled, if a + * failover has to occur. In such a case, the server that Debezium is failed over to might have + * a different set of sources, but still include the sources required for Debezium to continue + * to function. MySQL does not allow downstream replicas to connect if the GTID set does not + * contain GTIDs for all channels that the server is replicating from, even if the server does + * have the data needed by the client. To get around this, we can have Debezium merge its GTID + * set with whatever is on the server, so that MySQL will allow it to connect. See DBZ-143 for details. + * + *

This method does not mutate any state in the context. + * + * @param availableServerGtidSet the GTID set currently available in the MySQL server + * @param purgedServerGtid the GTID set already purged by the MySQL server + * @return A GTID set meant for consuming from a MySQL binlog; may return null if the SourceInfo + * has no GTIDs and therefore none were filtered + */ + public GtidSet filterGtidSet( + MySqlOffsetContext offsetContext, + GtidSet availableServerGtidSet, + GtidSet purgedServerGtid) { + String gtidStr = offsetContext.gtidSet(); + if (gtidStr == null) { + return null; + } + LOGGER.info("Attempting to generate a filtered GTID set"); + LOGGER.info("GTID set from previous recorded offset: {}", gtidStr); + GtidSet filteredGtidSet = new GtidSet(gtidStr); + Predicate gtidSourceFilter = connectorConfig.gtidSourceFilter(); + if (gtidSourceFilter != null) { + filteredGtidSet = filteredGtidSet.retainAll(gtidSourceFilter); + LOGGER.info( + "GTID set after applying GTID source includes/excludes to previous recorded offset: {}", + filteredGtidSet); + } + LOGGER.info("GTID set available on server: {}", availableServerGtidSet); + + GtidSet mergedGtidSet; + + if (connectorConfig.gtidNewChannelPosition() == GtidNewChannelPosition.EARLIEST) { + final GtidSet knownGtidSet = filteredGtidSet; + LOGGER.info("Using first available positions for new GTID channels"); + final GtidSet relevantAvailableServerGtidSet = + (gtidSourceFilter != null) + ? availableServerGtidSet.retainAll(gtidSourceFilter) + : availableServerGtidSet; + LOGGER.info( + "Relevant GTID set available on server: {}", relevantAvailableServerGtidSet); + + // Since the GTID recorded in the checkpoint represents the CDC-executed records, in + // certain scenarios + // (such as when the startup mode is earliest/timestamp/binlogfile), the recorded GTID + // may not start from + // the beginning. For example, A:300-500. However, during job recovery, we usually only + // need to focus on + // the last consumed point instead of consuming A:1-299. Therefore, some adjustments + // need to be made to the + // recorded offset in the checkpoint, and the available GTID for other MySQL instances + // should be completed. + mergedGtidSet = + GtidUtils.fixRestoredGtidSet( + GtidUtils.mergeGtidSetInto( + relevantAvailableServerGtidSet.retainAll( + uuid -> knownGtidSet.forServerWithId(uuid) != null), + purgedServerGtid), + filteredGtidSet); + } else { + mergedGtidSet = availableServerGtidSet.with(filteredGtidSet); + } + + LOGGER.info("Final merged GTID set to use when connecting to MySQL: {}", mergedGtidSet); + return mergedGtidSet; + } + + MySqlStreamingChangeEventSourceMetrics getMetrics() { + return metrics; + } + + void rewindBinaryLogClient(ChangeEventSourceContext context, BinlogPosition position) { + try { + if (context.isRunning()) { + LOGGER.debug("Rewinding binlog to position {}", position); + client.disconnect(); + client.setBinlogFilename(position.getFilename()); + client.setBinlogPosition(position.getPosition()); + client.connect(); + } + } catch (IOException e) { + LOGGER.error("Unexpected error when re-connecting to the MySQL binary log reader", e); + } + } + + BinlogPosition getCurrentBinlogPosition() { + return new BinlogPosition(client.getBinlogFilename(), client.getBinlogPosition()); + } + + /** + * Wraps the specified exception in a {@link DebeziumException}, ensuring that all useful state + * is captured inside the new exception's message. + * + * @param error the exception; may not be null + * @return the wrapped Kafka Connect exception + */ + protected DebeziumException wrap(Throwable error) { + assert error != null; + String msg = error.getMessage(); + if (error instanceof ServerException) { + ServerException e = (ServerException) error; + msg = msg + " Error code: " + e.getErrorCode() + "; SQLSTATE: " + e.getSqlState() + "."; + } else if (error instanceof SQLException) { + SQLException e = (SQLException) error; + msg = + e.getMessage() + + " Error code: " + + e.getErrorCode() + + "; SQLSTATE: " + + e.getSQLState() + + "."; + } + msg = ErrorMessageUtils.optimizeErrorMessage(msg); + return new DebeziumException(msg, error); + } + + /** LifecycleListener for Reader Thread. */ + protected final class ReaderThreadLifecycleListener implements LifecycleListener { + private final MySqlOffsetContext offsetContext; + + ReaderThreadLifecycleListener(MySqlOffsetContext offsetContext) { + this.offsetContext = offsetContext; + } + + @Override + public void onDisconnect(BinaryLogClient client) { + if (LOGGER.isInfoEnabled()) { + taskContext.temporaryLoggingContext( + connectorConfig, + "binlog", + () -> { + Map offset = lastOffset; + if (offset != null) { + LOGGER.info( + "Stopped reading binlog after {} events, last recorded offset: {}", + totalRecordCounter, + offset); + } else { + LOGGER.info( + "Stopped reading binlog after {} events, no new offset was recorded", + totalRecordCounter); + } + }); + } + } + + @Override + public void onConnect(BinaryLogClient client) { + // Set up the MDC logging context for this thread ... + taskContext.configureLoggingContext("binlog"); + + // The event row number will be used when processing the first event ... + LOGGER.info( + "Connected to MySQL binlog at {}:{}, starting at {}", + connectorConfig.hostname(), + connectorConfig.port(), + offsetContext); + } + + @Override + public void onCommunicationFailure(BinaryLogClient client, Exception ex) { + LOGGER.debug("A communication failure event arrived", ex); + logStreamingSourceState(); + try { + // Stop BinaryLogClient background threads + client.disconnect(); + } catch (final Exception e) { + LOGGER.debug("Exception while closing client", e); + } + errorHandler.setProducerThrowable(wrap(ex)); + } + + @Override + public void onEventDeserializationFailure(BinaryLogClient client, Exception ex) { + if (eventDeserializationFailureHandlingMode + == EventProcessingFailureHandlingMode.FAIL) { + LOGGER.debug("A deserialization failure event arrived", ex); + logStreamingSourceState(); + errorHandler.setProducerThrowable(wrap(ex)); + } else if (eventDeserializationFailureHandlingMode + == EventProcessingFailureHandlingMode.WARN) { + LOGGER.warn("A deserialization failure event arrived", ex); + logStreamingSourceState(Level.WARN); + } else { + LOGGER.debug("A deserialization failure event arrived", ex); + logStreamingSourceState(Level.DEBUG); + } + } + } + + @FunctionalInterface + private interface TableIdProvider { + TableId getTableId(E data); + } + + @FunctionalInterface + private interface RowsProvider { + List getRows(E data); + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java index d20e61e9e6bdd8..5785a022f17415 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java @@ -266,6 +266,7 @@ public void writeRecords(WriteRecordRequest writeRecordRequest) throws Exception if (!recordIterator.hasNext()) { Thread.sleep(100); + LOG.info("No records polled, continuing..."); // Check if should stop long elapsedTime = System.currentTimeMillis() - startTime; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java index 9686364c71fbca..f54e7d5f39d11e 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java @@ -163,19 +163,29 @@ public void writeRecord(String database, String table, byte[] record) { lock.unlock(); } } + + // Single table flush according to the STREAM_LOAD_MAX_BYTES + if (buffer.getBufferSizeBytes() >= STREAM_LOAD_MAX_BYTES) { + boolean flush = bufferFullFlush(); + LOG.info("trigger flush by buffer full, flush: {}", flush); + } } public boolean cacheFullFlush() { return doFlush(true, true); } + public boolean bufferFullFlush() { + return doFlush(false, true); + } + public boolean forceFlush() { return doFlush(true, false); } - private synchronized boolean doFlush(boolean waitUtilDone, boolean cacheFull) { + private synchronized boolean doFlush(boolean waitUtilDone, boolean bufferFull) { checkFlushException(); - if (waitUtilDone || cacheFull) { + if (waitUtilDone || bufferFull) { return flush(waitUtilDone); } return false; From 1e727935827a2fefde5f0b9512e5b71603c80744 Mon Sep 17 00:00:00 2001 From: wudi Date: Mon, 26 Jan 2026 11:43:35 +0800 Subject: [PATCH 04/11] fix --- .licenserc.yaml | 1 + .../doris/cdcclient/source/reader/SplitReadResult.java | 5 ++++- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/.licenserc.yaml b/.licenserc.yaml index 8bdc7d8113bc4c..ef44aa8e5818ad 100644 --- a/.licenserc.yaml +++ b/.licenserc.yaml @@ -102,4 +102,5 @@ header: - "pytest/deploy/*.conf" - "tools/jeprof" - "tools/FlameGraph/*" + - "fs_brokers/cdc_client/src/main/java/io/debezium/**" comment: on-failure diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java index df591468d49a9c..fc90c2d953aa90 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java @@ -21,7 +21,10 @@ import lombok.Data; -/** The result of reading a split with iterator. */ +/** + * Container for a source split and its associated state. + * Iteration over records for this split is handled separately (for example via pollRecords). + */ @Data public class SplitReadResult { // MySqlSplitState, SourceSplitState From dbd5a0fd0b61c744f383ac4b8ef6fb1bf1456b41 Mon Sep 17 00:00:00 2001 From: wudi Date: Mon, 26 Jan 2026 16:54:39 +0800 Subject: [PATCH 05/11] fix --- .../doris/cdcclient/common/Constants.java | 4 +- .../service/PipelineCoordinator.java | 2 +- .../cdcclient/sink/DorisBatchStreamLoad.java | 44 ++++++++++--------- 3 files changed, 26 insertions(+), 24 deletions(-) diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Constants.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Constants.java index b4c2284833ceb1..c17c6414c0012b 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Constants.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Constants.java @@ -23,7 +23,7 @@ public class Constants { // Debezium default properties public static final long DEBEZIUM_HEARTBEAT_INTERVAL_MS = 3000L; - public static final String DEBEZIUM_MAX_QUEUE_SIZE = "162580"; - public static final String DEBEZIUM_MAX_BATCH_SIZE = "40960"; + public static final String DEBEZIUM_MAX_QUEUE_SIZE = "327680"; + public static final String DEBEZIUM_MAX_BATCH_SIZE = "81920"; public static final String DEBEZIUM_POLL_INTERVAL_MS = "50"; } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java index 5785a022f17415..9340e8afe3b469 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java @@ -265,7 +265,7 @@ public void writeRecords(WriteRecordRequest writeRecordRequest) throws Exception sourceReader.pollRecords(readResult.getSplitState()); if (!recordIterator.hasNext()) { - Thread.sleep(100); + Thread.sleep(50); LOG.info("No records polled, continuing..."); // Check if should stop diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java index f54e7d5f39d11e..0b505658ae99a0 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java @@ -166,49 +166,51 @@ public void writeRecord(String database, String table, byte[] record) { // Single table flush according to the STREAM_LOAD_MAX_BYTES if (buffer.getBufferSizeBytes() >= STREAM_LOAD_MAX_BYTES) { - boolean flush = bufferFullFlush(); - LOG.info("trigger flush by buffer full, flush: {}", flush); + bufferFullFlush(bufferKey); } } - public boolean cacheFullFlush() { - return doFlush(true, true); + public void bufferFullFlush(String bufferKey) { + doFlush(bufferKey, false, true); } - public boolean bufferFullFlush() { - return doFlush(false, true); + public void forceFlush() { + doFlush(null, true, false); } - public boolean forceFlush() { - return doFlush(true, false); + public void cacheFullFlush() { + doFlush(null, true, true); } - private synchronized boolean doFlush(boolean waitUtilDone, boolean bufferFull) { + + private synchronized void doFlush(String bufferKey, boolean waitUtilDone, boolean bufferFull) { checkFlushException(); if (waitUtilDone || bufferFull) { - return flush(waitUtilDone); + flush(bufferKey, waitUtilDone); } - return false; } - private synchronized boolean flush(boolean waitUtilDone) { + private synchronized void flush(String bufferKey, boolean waitUtilDone) { if (!waitUtilDone && bufferMap.isEmpty()) { // bufferMap may have been flushed by other threads LOG.info("bufferMap is empty, no need to flush"); - return false; + return; } - for (String key : bufferMap.keySet()) { - if (waitUtilDone) { - // Ensure that the interval satisfies intervalMS - flushBuffer(key); + + if (null == bufferKey) { + for (String key : bufferMap.keySet()) { + if (waitUtilDone) { + flushBuffer(key); + } } - } - if (!waitUtilDone) { - return false; + } else if (bufferMap.containsKey(bufferKey)) { + flushBuffer(bufferKey); } else { + LOG.warn("buffer not found for key: {}, may be already flushed.", bufferKey); + } + if (waitUtilDone) { waitAsyncLoadFinish(); } - return true; } private synchronized void flushBuffer(String bufferKey) { From d34f43685f19697d135468bd334b01659932312c Mon Sep 17 00:00:00 2001 From: wudi Date: Mon, 26 Jan 2026 21:32:45 +0800 Subject: [PATCH 06/11] fix --- .../java/org/apache/doris/cdcclient/common/Constants.java | 4 ++-- .../apache/doris/cdcclient/service/PipelineCoordinator.java | 3 +-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Constants.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Constants.java index c17c6414c0012b..c9d006a90aea18 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Constants.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Constants.java @@ -23,7 +23,7 @@ public class Constants { // Debezium default properties public static final long DEBEZIUM_HEARTBEAT_INTERVAL_MS = 3000L; - public static final String DEBEZIUM_MAX_QUEUE_SIZE = "327680"; - public static final String DEBEZIUM_MAX_BATCH_SIZE = "81920"; + public static final String DEBEZIUM_MAX_QUEUE_SIZE = "809200"; + public static final String DEBEZIUM_MAX_BATCH_SIZE = "204800"; public static final String DEBEZIUM_POLL_INTERVAL_MS = "50"; } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java index 9340e8afe3b469..d20e61e9e6bdd8 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java @@ -265,8 +265,7 @@ public void writeRecords(WriteRecordRequest writeRecordRequest) throws Exception sourceReader.pollRecords(readResult.getSplitState()); if (!recordIterator.hasNext()) { - Thread.sleep(50); - LOG.info("No records polled, continuing..."); + Thread.sleep(100); // Check if should stop long elapsedTime = System.currentTimeMillis() - startTime; From 2ff3249be20deff878498c279b28779087ce69c9 Mon Sep 17 00:00:00 2001 From: wudi Date: Tue, 27 Jan 2026 10:52:31 +0800 Subject: [PATCH 07/11] fix --- .../java/org/apache/doris/cdcclient/common/Constants.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Constants.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Constants.java index c9d006a90aea18..b4c2284833ceb1 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Constants.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Constants.java @@ -23,7 +23,7 @@ public class Constants { // Debezium default properties public static final long DEBEZIUM_HEARTBEAT_INTERVAL_MS = 3000L; - public static final String DEBEZIUM_MAX_QUEUE_SIZE = "809200"; - public static final String DEBEZIUM_MAX_BATCH_SIZE = "204800"; + public static final String DEBEZIUM_MAX_QUEUE_SIZE = "162580"; + public static final String DEBEZIUM_MAX_BATCH_SIZE = "40960"; public static final String DEBEZIUM_POLL_INTERVAL_MS = "50"; } From 3d0a563d13e7422f5e103d8a26d457567224ab92 Mon Sep 17 00:00:00 2001 From: wudi Date: Tue, 27 Jan 2026 17:07:14 +0800 Subject: [PATCH 08/11] test default prams --- .../java/org/apache/doris/cdcclient/utils/ConfigUtil.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java index 016c9ddf312e0d..037cc2180504e9 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java @@ -99,9 +99,9 @@ public static ZoneId getPostgresServerTimeZoneFromProps(java.util.Properties pro /** Optimized debezium parameters */ public static Properties getDefaultDebeziumProps() { Properties properties = new Properties(); - properties.setProperty("max.queue.size", Constants.DEBEZIUM_MAX_QUEUE_SIZE); - properties.setProperty("max.batch.size", Constants.DEBEZIUM_MAX_BATCH_SIZE); - properties.setProperty("poll.interval.ms", Constants.DEBEZIUM_POLL_INTERVAL_MS); + // properties.setProperty("max.queue.size", Constants.DEBEZIUM_MAX_QUEUE_SIZE); + // properties.setProperty("max.batch.size", Constants.DEBEZIUM_MAX_BATCH_SIZE); + // properties.setProperty("poll.interval.ms", Constants.DEBEZIUM_POLL_INTERVAL_MS); return properties; } From 65fa89d60a21e1e69fbe5c4ff817df6057a845f5 Mon Sep 17 00:00:00 2001 From: wudi Date: Tue, 27 Jan 2026 19:58:59 +0800 Subject: [PATCH 09/11] final fix --- .../job/extensions/insert/streaming/StreamingInsertJob.java | 3 +++ .../main/java/org/apache/doris/cdcclient/common/Constants.java | 3 --- .../cdcclient/source/reader/JdbcIncrementalSourceReader.java | 3 --- .../doris/cdcclient/source/reader/mysql/MySqlSourceReader.java | 3 --- .../main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java | 3 --- 5 files changed, 3 insertions(+), 12 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index cef86e56d630ac..07ede21a5b8155 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -785,6 +785,9 @@ private String getShowSQL() { sb.append("FROM ").append(dataSourceType.name()); sb.append("("); for (Map.Entry entry : sourceProperties.entrySet()) { + if (entry.getKey().equalsIgnoreCase("password")) { + continue; + } sb.append("'").append(entry.getKey()) .append("'='").append(entry.getValue()).append("',"); } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Constants.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Constants.java index b4c2284833ceb1..04ec118a6c2d4e 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Constants.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Constants.java @@ -23,7 +23,4 @@ public class Constants { // Debezium default properties public static final long DEBEZIUM_HEARTBEAT_INTERVAL_MS = 3000L; - public static final String DEBEZIUM_MAX_QUEUE_SIZE = "162580"; - public static final String DEBEZIUM_MAX_BATCH_SIZE = "40960"; - public static final String DEBEZIUM_POLL_INTERVAL_MS = "50"; } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/JdbcIncrementalSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/JdbcIncrementalSourceReader.java index 27db10bccc6101..4be0c20dfadd03 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/JdbcIncrementalSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/JdbcIncrementalSourceReader.java @@ -206,9 +206,6 @@ public Iterator pollRecords(Object splitState) throws InterruptedE SourceRecords sourceRecords = dataIt.next(); SplitRecords splitRecords = new SplitRecords(this.currentSplit.splitId(), sourceRecords.iterator()); - if (!sourceRecords.getSourceRecordList().isEmpty()) { - LOG.info("{} Records received.", sourceRecords.getSourceRecordList().size()); - } // Return filtered iterator return new FilteredRecordIterator(splitRecords, (SourceSplitState) splitState); diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java index 2a66fe871d1155..1bf7b02ffe21a1 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java @@ -217,9 +217,6 @@ public Iterator pollRecords(Object splitState) throws InterruptedE SourceRecords sourceRecords = dataIt.next(); SplitRecords splitRecords = new SplitRecords(this.currentSplit.splitId(), sourceRecords.iterator()); - if (!sourceRecords.getSourceRecordList().isEmpty()) { - LOG.info("{} Records received", sourceRecords.getSourceRecordList().size()); - } return new FilteredRecordIterator(splitRecords, (MySqlSplitState) splitState); } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java index 037cc2180504e9..7fb3f92edd9723 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java @@ -99,9 +99,6 @@ public static ZoneId getPostgresServerTimeZoneFromProps(java.util.Properties pro /** Optimized debezium parameters */ public static Properties getDefaultDebeziumProps() { Properties properties = new Properties(); - // properties.setProperty("max.queue.size", Constants.DEBEZIUM_MAX_QUEUE_SIZE); - // properties.setProperty("max.batch.size", Constants.DEBEZIUM_MAX_BATCH_SIZE); - // properties.setProperty("poll.interval.ms", Constants.DEBEZIUM_POLL_INTERVAL_MS); return properties; } From 8ac6729d3338b685ee4def39f011fb95d63c857c Mon Sep 17 00:00:00 2001 From: wudi Date: Thu, 29 Jan 2026 17:39:24 +0800 Subject: [PATCH 10/11] add snapshot split size and parallelism --- .../doris/job/cdc/DataSourceConfigKeys.java | 3 +- .../streaming/DataSourceConfigValidator.java | 3 +- .../streaming/StreamingMultiTblTask.java | 49 ++- .../doris/job/offset/jdbc/JdbcOffset.java | 76 +++- .../offset/jdbc/JdbcSourceOffsetProvider.java | 132 ++++--- .../controller/ClientController.java | 4 +- .../model/response/RecordWithMeta.java | 4 +- .../service/PipelineCoordinator.java | 58 +-- .../cdcclient/sink/DorisBatchStreamLoad.java | 2 +- .../doris/cdcclient/sink/HttpPutBuilder.java | 9 + .../reader/JdbcIncrementalSourceReader.java | 353 ++++++++++++++---- .../source/reader/SnapshotReaderContext.java | 37 ++ .../cdcclient/source/reader/SourceReader.java | 4 +- .../source/reader/SplitReadResult.java | 35 +- .../reader/mysql/MySqlSourceReader.java | 325 ++++++++++++---- .../reader/postgres/PostgresSourceReader.java | 26 +- .../src/main/resources/log4j.properties | 2 +- regression-test/conf/regression-conf.groovy | 13 +- .../cdc/test_streaming_postgres_job_split.out | 8 + .../cdc/test_streaming_mysql_job.groovy | 3 +- .../cdc/test_streaming_postgres_job.groovy | 5 +- .../test_streaming_postgres_job_priv.groovy | 2 +- .../test_streaming_postgres_job_split.groovy | 115 ++++++ 23 files changed, 969 insertions(+), 299 deletions(-) create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SnapshotReaderContext.java create mode 100644 regression-test/data/job_p0/streaming_job/cdc/test_streaming_postgres_job_split.out create mode 100644 regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_split.groovy diff --git a/fe/fe-common/src/main/java/org/apache/doris/job/cdc/DataSourceConfigKeys.java b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/DataSourceConfigKeys.java index 9e1918e561431b..42c7fabee60033 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/job/cdc/DataSourceConfigKeys.java +++ b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/DataSourceConfigKeys.java @@ -32,5 +32,6 @@ public class DataSourceConfigKeys { public static final String OFFSET_INITIAL = "initial"; public static final String OFFSET_EARLIEST = "earliest"; public static final String OFFSET_LATEST = "latest"; - public static final String SPLIT_SIZE = "split_size"; + public static final String SNAPSHOT_SPLIT_SIZE = "snapshot_split_size"; + public static final String SNAPSHOT_PARALLELISM = "snapshot_parallelism"; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidator.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidator.java index fcabcb82898d75..ddaf8456271499 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidator.java @@ -37,7 +37,8 @@ public class DataSourceConfigValidator { DataSourceConfigKeys.SCHEMA, DataSourceConfigKeys.INCLUDE_TABLES, DataSourceConfigKeys.EXCLUDE_TABLES, - DataSourceConfigKeys.SPLIT_SIZE + DataSourceConfigKeys.SNAPSHOT_SPLIT_SIZE, + DataSourceConfigKeys.SNAPSHOT_PARALLELISM ); public static void validateSource(Map input) throws IllegalArgumentException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java index 1f955f0a2c3d3d..10364fe465f8b4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java @@ -47,11 +47,13 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; import com.google.gson.Gson; import lombok.Getter; import lombok.extern.log4j.Log4j2; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; @@ -177,9 +179,9 @@ private WriteRecordRequest buildRequestParams() throws JobException { request.setTaskId(getTaskId() + ""); request.setToken(getToken()); request.setTargetDb(targetDb); - Map splitMeta = objectMapper.convertValue(offset.getSplit(), - new TypeReference>() { - }); + + Map splitMeta = offset.generateMeta(); + Preconditions.checkArgument(!splitMeta.isEmpty(), "split meta is empty"); request.setMeta(splitMeta); String feAddr = Env.getCurrentEnv().getMasterHost() + ":" + Env.getCurrentEnv().getMasterHttpPort(); request.setFrontendAddress(feAddr); @@ -210,28 +212,39 @@ public void successCallback(CommitOffsetRequest offsetRequest) { return; } // set end offset to running offset - Map offsetMeta; + // binlogSplit : [{"splitId":"binlog-split"}] only 1 element + // snapshotSplit:[{"splitId":"table-0"},...],...}] + List> offsetMeta; try { - offsetMeta = objectMapper.readValue(offsetRequest.getOffset(), new TypeReference>() { - }); + offsetMeta = objectMapper.readValue(offsetRequest.getOffset(), + new TypeReference>>() {}); } catch (JsonProcessingException e) { log.warn("Failed to parse offset meta from request: {}", offsetRequest.getOffset(), e); throw new RuntimeException(e); } - String splitId = offsetMeta.remove(JdbcSourceOffsetProvider.SPLIT_ID); - if (runOffset.getSplit().snapshotSplit() - && !BinlogSplit.BINLOG_SPLIT_ID.equals(splitId)) { - SnapshotSplit split = (SnapshotSplit) runOffset.getSplit(); - split.setHighWatermark(offsetMeta); - } else if (!runOffset.getSplit().snapshotSplit() - && BinlogSplit.BINLOG_SPLIT_ID.equals(splitId)) { - BinlogSplit split = (BinlogSplit) runOffset.getSplit(); - split.setEndingOffset(offsetMeta); + + Preconditions.checkState(offsetMeta.size() == runOffset.getSplits().size(), "offset meta size " + + offsetMeta.size() + " is not equal to running offset splits size " + + runOffset.getSplits().size()); + + if (runOffset.snapshotSplit()) { + for (int i = 0; i < runOffset.getSplits().size(); i++) { + SnapshotSplit split = (SnapshotSplit) runOffset.getSplits().get(i); + Map splitOffsetMeta = offsetMeta.get(i); + String splitId = splitOffsetMeta.remove(JdbcSourceOffsetProvider.SPLIT_ID); + Preconditions.checkState(split.getSplitId().equals(splitId), + "split id " + split.getSplitId() + " is not equal to offset meta split id " + splitId); + split.setHighWatermark(splitOffsetMeta); + } } else { - log.warn("Split id is not consistent, task running split id {}," - + " offset commit request split id {}", runOffset.getSplit().getSplitId(), splitId); - throw new RuntimeException("Split id is not consistent"); + Map offsetMap = offsetMeta.get(0); + String splitId = offsetMap.remove(JdbcSourceOffsetProvider.SPLIT_ID); + Preconditions.checkState(BinlogSplit.BINLOG_SPLIT_ID.equals(splitId), + "split id is not equal to binlog split id"); + BinlogSplit split = (BinlogSplit) runOffset.getSplits().get(0); + split.setEndingOffset(offsetMap); } + this.scannedRows = offsetRequest.getScannedRows(); this.scannedBytes = offsetRequest.getScannedBytes(); Job job = Env.getCurrentEnv().getJobManager().getJob(getJobId()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java index ba83dd1c8cd1f5..dea2c244d6d78c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java @@ -21,6 +21,9 @@ import org.apache.doris.job.cdc.split.BinlogSplit; import org.apache.doris.job.offset.Offset; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; import com.google.gson.Gson; import lombok.AllArgsConstructor; import lombok.Getter; @@ -30,6 +33,8 @@ import org.apache.commons.collections4.MapUtils; import java.util.HashMap; +import java.util.List; +import java.util.Map; @Getter @Setter @@ -37,7 +42,9 @@ @NoArgsConstructor public class JdbcOffset implements Offset { - private AbstractSourceSplit split; + // There may be multiple snapshot splits, + // but only one binlog split, with the ID fixed as "binlog-split". + private List splits; @Override public String toSerializedJson() { @@ -56,30 +63,63 @@ public boolean isValidOffset() { @Override public String showRange() { - if (split.snapshotSplit()) { - // need to show hw - return new Gson().toJson(split); - } else { - BinlogSplit binlogSplit = (BinlogSplit) split; - HashMap showMap = new HashMap<>(); - showMap.put(JdbcSourceOffsetProvider.SPLIT_ID, BinlogSplit.BINLOG_SPLIT_ID); - if (binlogSplit.getStartingOffset() != null) { - showMap.put("startOffset", binlogSplit.getStartingOffset()); - } else if (CollectionUtils.isNotEmpty(binlogSplit.getFinishedSplits())) { - showMap.put("finishedSplitSize", binlogSplit.getFinishedSplits().size()); + if (!splits.isEmpty()) { + AbstractSourceSplit split = splits.get(0); + if (split.snapshotSplit()) { + // need to show hw + return new Gson().toJson(splits); + } else { + BinlogSplit binlogSplit = (BinlogSplit) split; + HashMap showMap = new HashMap<>(); + showMap.put(JdbcSourceOffsetProvider.SPLIT_ID, BinlogSplit.BINLOG_SPLIT_ID); + if (binlogSplit.getStartingOffset() != null) { + showMap.put("startOffset", binlogSplit.getStartingOffset()); + } else if (CollectionUtils.isNotEmpty(binlogSplit.getFinishedSplits())) { + showMap.put("finishedSplitSize", binlogSplit.getFinishedSplits().size()); + } + if (MapUtils.isNotEmpty(binlogSplit.getEndingOffset())) { + showMap.put("endOffset", binlogSplit.getEndingOffset()); + } + return new Gson().toJson(showMap); } - if (MapUtils.isNotEmpty(binlogSplit.getEndingOffset())) { - showMap.put("endOffset", binlogSplit.getEndingOffset()); - } - return new Gson().toJson(showMap); } + return "split is null"; } @Override public String toString() { return "JdbcOffset{" - + "split=" - + split + + "splits=" + + splits + '}'; } + + public boolean snapshotSplit() { + Preconditions.checkState(splits != null && !splits.isEmpty(), "splits is null or empty"); + AbstractSourceSplit split = splits.get(0); + return split.snapshotSplit(); + } + + /** + * Generate meta info for offset + * Snapshot: {splits:[{splitId:"tb:0"},...]} + * Binlog: {"splitId:":"binlog-split",...} + * @return + */ + public Map generateMeta() { + if (!splits.isEmpty()) { + AbstractSourceSplit split = splits.get(0); + if (split.snapshotSplit()) { + // need to show hw + return new ObjectMapper().convertValue(this, + new TypeReference>() { + }); + } else { + return new ObjectMapper().convertValue(split, + new TypeReference>() { + }); + } + } + return new HashMap<>(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java index 51f3c07aa41bf0..66ec5e58d5dc0b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java @@ -54,6 +54,7 @@ import org.apache.commons.collections4.MapUtils; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; @@ -68,6 +69,8 @@ public class JdbcSourceOffsetProvider implements SourceOffsetProvider { public static final String SPLIT_ID = "splitId"; private static final ObjectMapper objectMapper = new ObjectMapper(); + private static final int SNAPSHOT_PARALLELISM_DEFAULT = 4; + private final int snapshotParallelism; private Long jobId; private DataSourceType sourceType; private Map sourceProperties = new HashMap<>(); @@ -91,6 +94,9 @@ public JdbcSourceOffsetProvider(Long jobId, DataSourceType sourceType, Map(); + this.snapshotParallelism = Integer.parseInt( + sourceProperties.getOrDefault(DataSourceConfigKeys.SNAPSHOT_PARALLELISM, + SNAPSHOT_PARALLELISM_DEFAULT + "")); } @Override @@ -102,30 +108,30 @@ public String getSourceType() { public Offset getNextOffset(StreamingJobProperties jobProps, Map properties) { JdbcOffset nextOffset = new JdbcOffset(); if (!remainingSplits.isEmpty()) { - // snapshot read - SnapshotSplit snapshotSplit = remainingSplits.get(0); - nextOffset.setSplit(snapshotSplit); + int splitsNum = Math.min(remainingSplits.size(), snapshotParallelism); + List snapshotSplits = new ArrayList<>(remainingSplits.subList(0, splitsNum)); + nextOffset.setSplits(snapshotSplits); return nextOffset; - } else if (currentOffset != null && currentOffset.getSplit().snapshotSplit()) { + } else if (currentOffset != null && currentOffset.snapshotSplit()) { // snapshot to binlog BinlogSplit binlogSplit = new BinlogSplit(); binlogSplit.setFinishedSplits(finishedSplits); - nextOffset.setSplit(binlogSplit); + nextOffset.setSplits(Collections.singletonList(binlogSplit)); return nextOffset; } else { // only binlog - return currentOffset == null ? new JdbcOffset(new BinlogSplit()) : currentOffset; + return currentOffset == null ? new JdbcOffset(Collections.singletonList(new BinlogSplit())) : currentOffset; } } @Override public String getShowCurrentOffset() { if (this.currentOffset != null) { - AbstractSourceSplit split = this.currentOffset.getSplit(); - if (split.snapshotSplit()) { - return new Gson().toJson(split); + if (currentOffset.snapshotSplit()) { + List splits = currentOffset.getSplits(); + return new Gson().toJson(splits); } else { - BinlogSplit binlogSplit = (BinlogSplit) split; + BinlogSplit binlogSplit = (BinlogSplit) currentOffset.getSplits().get(0); HashMap showMap = new HashMap<>(); showMap.put(SPLIT_ID, BinlogSplit.BINLOG_SPLIT_ID); if (binlogSplit.getStartingOffset() != null) { @@ -154,29 +160,31 @@ public InsertIntoTableCommand rewriteTvfParams(InsertIntoTableCommand originComm @Override public void updateOffset(Offset offset) { this.currentOffset = (JdbcOffset) offset; - AbstractSourceSplit split = currentOffset.getSplit(); - if (split.snapshotSplit()) { - SnapshotSplit snapshotSplit = (SnapshotSplit) split; - String splitId = split.getSplitId(); - boolean remove = remainingSplits.removeIf(v -> { - if (v.getSplitId().equals(splitId)) { - snapshotSplit.setTableId(v.getTableId()); - snapshotSplit.setSplitKey(v.getSplitKey()); - snapshotSplit.setSplitStart(v.getSplitStart()); - snapshotSplit.setSplitEnd(v.getSplitEnd()); - return true; + if (currentOffset.snapshotSplit()) { + List splits = currentOffset.getSplits(); + for (AbstractSourceSplit split : splits) { + SnapshotSplit snapshotSplit = (SnapshotSplit) split; + String splitId = split.getSplitId(); + boolean remove = remainingSplits.removeIf(v -> { + if (v.getSplitId().equals(splitId)) { + snapshotSplit.setTableId(v.getTableId()); + snapshotSplit.setSplitKey(v.getSplitKey()); + snapshotSplit.setSplitStart(v.getSplitStart()); + snapshotSplit.setSplitEnd(v.getSplitEnd()); + return true; + } + return false; + }); + if (remove) { + finishedSplits.add(snapshotSplit); + chunkHighWatermarkMap.computeIfAbsent(snapshotSplit.getTableId(), k -> new HashMap<>()) + .put(snapshotSplit.getSplitId(), snapshotSplit.getHighWatermark()); + } else { + log.warn("Cannot find snapshot split {} in remainingSplits for job {}", splitId, getJobId()); } - return false; - }); - if (remove) { - finishedSplits.add(snapshotSplit); - chunkHighWatermarkMap.computeIfAbsent(snapshotSplit.getTableId(), k -> new HashMap<>()) - .put(snapshotSplit.getSplitId(), snapshotSplit.getHighWatermark()); - } else { - log.warn("Cannot find snapshot split {} in remainingSplits for job {}", splitId, getJobId()); } } else { - BinlogSplit binlogSplit = (BinlogSplit) split; + BinlogSplit binlogSplit = (BinlogSplit) currentOffset.getSplits().get(0); binlogOffsetPersist = new HashMap<>(binlogSplit.getStartingOffset()); binlogOffsetPersist.put(SPLIT_ID, BinlogSplit.BINLOG_SPLIT_ID); } @@ -241,20 +249,21 @@ public boolean hasMoreDataToConsume() { return false; } try { - if (!currentOffset.getSplit().snapshotSplit()) { - BinlogSplit binlogSplit = (BinlogSplit) currentOffset.getSplit(); + if (!currentOffset.snapshotSplit()) { + BinlogSplit binlogSplit = (BinlogSplit) currentOffset.getSplits().get(0); + if (MapUtils.isEmpty(binlogSplit.getStartingOffset())) { + // snapshot to binlog phase + return true; + } return compareOffset(endBinlogOffset, new HashMap<>(binlogSplit.getStartingOffset())); } else { - SnapshotSplit snapshotSplit = (SnapshotSplit) currentOffset.getSplit(); - if (MapUtils.isNotEmpty(snapshotSplit.getHighWatermark())) { - return compareOffset(endBinlogOffset, new HashMap<>(snapshotSplit.getHighWatermark())); - } + // snapshot means has data to consume + return true; } } catch (Exception ex) { log.info("Compare offset error: ", ex); return false; } - return false; } private boolean compareOffset(Map offsetFirst, Map offsetSecond) @@ -300,20 +309,23 @@ private boolean compareOffset(Map offsetFirst, Map offsetMeta = objectMapper.readValue(offset, new TypeReference>() { - }); - String splitId = offsetMeta.remove(SPLIT_ID); - if (BinlogSplit.BINLOG_SPLIT_ID.equals(splitId)) { - BinlogSplit binlogSplit = new BinlogSplit(); - binlogSplit.setSplitId(splitId); - binlogSplit.setStartingOffset(offsetMeta); - return new JdbcOffset(binlogSplit); - } else { + List> offsetMeta = + objectMapper.readValue(offset, new TypeReference>>() {}); + List snapshotSplits = new ArrayList<>(offsetMeta.size()); + for (Map ot : offsetMeta) { + String splitId = ot.remove(SPLIT_ID); + if (BinlogSplit.BINLOG_SPLIT_ID.equals(splitId)) { + BinlogSplit binlogSplit = new BinlogSplit(); + binlogSplit.setSplitId(splitId); + binlogSplit.setStartingOffset(ot); + return new JdbcOffset(Collections.singletonList(binlogSplit)); + } SnapshotSplit snapshotSplit = new SnapshotSplit(); snapshotSplit.setSplitId(splitId); - snapshotSplit.setHighWatermark(offsetMeta); - return new JdbcOffset(snapshotSplit); + snapshotSplit.setHighWatermark(ot); + snapshotSplits.add(snapshotSplit); } + return new JdbcOffset(snapshotSplits); } catch (JsonProcessingException e) { log.warn("Failed to deserialize offset: {}", offset, e); throw new RuntimeException(e); @@ -343,14 +355,22 @@ public void replayIfNeed(StreamingInsertJob job) throws JobException { chunkHighWatermarkMap == null ? 0 : chunkHighWatermarkMap.size()); if (MapUtils.isNotEmpty(binlogOffsetPersist)) { currentOffset = new JdbcOffset(); - currentOffset.setSplit(new BinlogSplit(binlogOffsetPersist)); + currentOffset.setSplits(Collections.singletonList(new BinlogSplit(binlogOffsetPersist))); } else { Map> snapshotSplits = StreamingJobUtils.restoreSplitsToJob(job.getJobId()); if (MapUtils.isNotEmpty(chunkHighWatermarkMap) && MapUtils.isNotEmpty(snapshotSplits)) { - SnapshotSplit lastSnapshotSplit = recalculateRemainingSplits(chunkHighWatermarkMap, snapshotSplits); + List lastSnapshotSplits = + recalculateRemainingSplits(chunkHighWatermarkMap, snapshotSplits); if (this.remainingSplits.isEmpty()) { currentOffset = new JdbcOffset(); - currentOffset.setSplit(lastSnapshotSplit); + if (!lastSnapshotSplits.isEmpty()) { + currentOffset.setSplits(lastSnapshotSplits); + } else { + // when snapshot to binlog phase fe restarts + BinlogSplit binlogSplit = new BinlogSplit(); + binlogSplit.setFinishedSplits(finishedSplits); + currentOffset.setSplits(Collections.singletonList(binlogSplit)); + } } } } @@ -374,13 +394,12 @@ public void replayIfNeed(StreamingInsertJob job) throws JobException { * Assign the HW value to the synchronized Split, * and remove the Split from remainSplit and place it in finishedSplit. */ - private SnapshotSplit recalculateRemainingSplits( + private List recalculateRemainingSplits( Map>> chunkHighWatermarkMap, Map> snapshotSplits) { if (this.finishedSplits == null) { this.finishedSplits = new ArrayList<>(); } - SnapshotSplit lastSnapshotSplit = null; for (Map.Entry>> entry : chunkHighWatermarkMap.entrySet()) { String tableId = entry.getKey(); Map> splitIdToHighWatermark = entry.getValue(); @@ -396,7 +415,6 @@ private SnapshotSplit recalculateRemainingSplits( if (CollectionUtils.isEmpty(tableSplits)) { continue; } - lastSnapshotSplit = tableSplits.get(tableSplits.size() - 1); tableSplits.removeIf(split -> { String splitId = split.getSplitId(); Map highWatermark = splitIdToHighWatermark.get(splitId); @@ -412,7 +430,11 @@ private SnapshotSplit recalculateRemainingSplits( this.remainingSplits = snapshotSplits.values().stream() .flatMap(List::stream) .collect(Collectors.toList()); - return lastSnapshotSplit; + + // The splits that were last syncing before the restart + int splitsNum = Math.min(remainingSplits.size(), snapshotParallelism); + List lastSnapshotSplits = new ArrayList<>(remainingSplits.subList(0, splitsNum)); + return lastSnapshotSplits; } private String getTableName(String tableId) { diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java index b3302e2c78519c..d0c2c17e457629 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java @@ -65,9 +65,9 @@ public Object fetchSplits(@RequestBody FetchTableSplitsRequest ftsReq) { SourceReader reader = Env.getCurrentEnv().getReader(ftsReq); List splits = reader.getSourceSplits(ftsReq); return RestResponse.success(splits); - } catch (IllegalArgumentException ex) { + } catch (Exception ex) { LOG.error("Failed to fetch splits, jobId={}", ftsReq.getJobId(), ex); - return RestResponse.internalError(ex.getMessage()); + return RestResponse.internalError(ExceptionUtils.getRootCauseMessage(ex)); } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/response/RecordWithMeta.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/response/RecordWithMeta.java index 4c6d0e4ee7fe00..b3c50486700215 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/response/RecordWithMeta.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/response/RecordWithMeta.java @@ -26,11 +26,11 @@ @Data public class RecordWithMeta { - private Map meta; + private List> meta; private List records; public RecordWithMeta() { - this.meta = new HashMap<>(); + this.meta = new ArrayList<>(); this.records = new ArrayList<>(); } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java index d20e61e9e6bdd8..76dd0c34e323ab 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java @@ -27,6 +27,7 @@ import org.apache.doris.job.cdc.request.WriteRecordRequest; import org.apache.doris.job.cdc.split.BinlogSplit; +import com.fasterxml.jackson.core.JsonProcessingException; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.flink.api.connector.source.SourceSplit; @@ -34,6 +35,7 @@ import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.source.SourceRecord; +import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -113,10 +115,9 @@ private RecordWithMeta buildRecordResponse( "Start fetching records for jobId={}, isSnapshotSplit={}", fetchRecord.getJobId(), isSnapshotSplit); - while (!shouldStop) { Iterator recordIterator = - sourceReader.pollRecords(readResult.getSplitState()); + sourceReader.pollRecords(); if (!recordIterator.hasNext()) { Thread.sleep(100); @@ -187,7 +188,7 @@ private RecordWithMeta buildRecordResponse( } // Extract and set offset metadata - Map offsetMeta = extractOffsetMeta(sourceReader, readResult); + List> offsetMeta = extractOffsetMeta(sourceReader, readResult); recordResponse.setMeta(offsetMeta); return recordResponse; @@ -238,7 +239,6 @@ public CompletableFuture writeRecordsAsync(WriteRecordRequest writeRecordR public void writeRecords(WriteRecordRequest writeRecordRequest) throws Exception { SourceReader sourceReader = Env.getCurrentEnv().getReader(writeRecordRequest); DorisBatchStreamLoad batchStreamLoad = null; - Map metaResponse = new HashMap<>(); long scannedRows = 0L; long scannedBytes = 0L; int heartbeatCount = 0; @@ -261,8 +261,7 @@ public void writeRecords(WriteRecordRequest writeRecordRequest) throws Exception // 2. poll record while (!shouldStop) { - Iterator recordIterator = - sourceReader.pollRecords(readResult.getSplitState()); + Iterator recordIterator = sourceReader.pollRecords(); if (!recordIterator.hasNext()) { Thread.sleep(100); @@ -342,7 +341,7 @@ public void writeRecords(WriteRecordRequest writeRecordRequest) throws Exception } // 3. Extract offset from split state - metaResponse = extractOffsetMeta(sourceReader, readResult); + List> metaResponse = extractOffsetMeta(sourceReader, readResult); // 4. wait all stream load finish batchStreamLoad.forceFlush(); @@ -382,6 +381,7 @@ private boolean shouldStop( // 1. Snapshot split with data: if no more data in queue, stop immediately (no need to wait // for timeout) // snapshot split will be written to the debezium queue all at once. + // multiple snapshot splits are handled in the source reader. if (isSnapshotSplit && hasData) { LOG.info( "Snapshot split finished, no more data available. Total elapsed: {} ms", @@ -493,16 +493,21 @@ private void cleanupReaderResources( /** * Extract offset metadata from split state. * - *

This method handles both snapshot splits and binlog splits, extracting the appropriate - * offset information and adding the split ID. + *

This method handles both snapshot splits and binlog splits, + * extracting the appropriate offset information through the SourceReader interface. + * For snapshot splits: + * {"highWatermarks":[{"splitId":"tbl:1",...},...]} + * + * For Binlog Split: + * {"splitId":"binlog_split","fileName":"mysql-bin.000001","pos":"12345",...} * * @param sourceReader the source reader - * @param readResult the read result containing split and split state + * @param readResult the read result containing splits and split states * @return offset metadata map * @throws RuntimeException if split state is null or split type is unknown */ - private Map extractOffsetMeta( - SourceReader sourceReader, SplitReadResult readResult) { + private List> extractOffsetMeta( + SourceReader sourceReader, SplitReadResult readResult) throws JsonProcessingException { Preconditions.checkNotNull(readResult, "readResult must not be null"); if (readResult.getSplitState() == null) { @@ -510,20 +515,31 @@ private Map extractOffsetMeta( } SourceSplit split = readResult.getSplit(); - Map offsetRes; - - // Set meta information for hw (high watermark) + List> commitOffsets = new ArrayList<>(); if (sourceReader.isSnapshotSplit(split)) { - offsetRes = sourceReader.extractSnapshotStateOffset(readResult.getSplitState()); - offsetRes.put(SPLIT_ID, split.splitId()); + // Unified format for both single and multiple splits + List allSplits = readResult.getSplits(); + Map allStates = readResult.getSplitStates(); + + for (SourceSplit currentSplit : allSplits) { + String splitId = currentSplit.splitId(); + Object currentState = allStates.get(splitId); + Preconditions.checkNotNull(currentState, "Split state not found for splitId: " + splitId); + + Map highWatermark = + sourceReader.extractSnapshotStateOffset(currentState); + Map splitInfo = new HashMap<>(); + splitInfo.put(SourceReader.SPLIT_ID, splitId); + splitInfo.putAll(highWatermark); + commitOffsets.add(splitInfo); + } } else if (sourceReader.isBinlogSplit(split)) { - // Set meta for binlog event - offsetRes = sourceReader.extractBinlogStateOffset(readResult.getSplitState()); + Map offsetRes = sourceReader.extractBinlogStateOffset(readResult.getSplitState()); offsetRes.put(SPLIT_ID, BinlogSplit.BINLOG_SPLIT_ID); + commitOffsets.add(offsetRes); } else { throw new RuntimeException("Unknown split type: " + split.getClass().getName()); } - - return offsetRes; + return commitOffsets; } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java index 0b505658ae99a0..8d850e63b76b4e 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java @@ -495,7 +495,7 @@ public void resetTaskId() { /** commit offfset to frontends. */ public void commitOffset( - String taskId, Map meta, long scannedRows, long scannedBytes) { + String taskId, List> meta, long scannedRows, long scannedBytes) { try { String url = String.format(COMMIT_URL_PATTERN, frontendAddress, targetDb); Map commitParams = new HashMap<>(); diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/HttpPutBuilder.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/HttpPutBuilder.java index c1594e04ffc30e..b5174d8279870e 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/HttpPutBuilder.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/HttpPutBuilder.java @@ -19,12 +19,14 @@ import org.apache.doris.cdcclient.common.Constants; +import org.apache.commons.codec.binary.Base64; import org.apache.flink.util.Preconditions; import org.apache.http.HttpEntity; import org.apache.http.HttpHeaders; import org.apache.http.client.methods.HttpPut; import org.apache.http.entity.StringEntity; +import java.nio.charset.StandardCharsets; import java.util.HashMap; import java.util.Map; import java.util.Properties; @@ -72,6 +74,13 @@ public HttpPutBuilder addTokenAuth(String token) { return this; } + public HttpPutBuilder baseAuth(String user, String password) { + final String authInfo = user + ":" + password; + byte[] encoded = Base64.encodeBase64(authInfo.getBytes(StandardCharsets.UTF_8)); + header.put(HttpHeaders.AUTHORIZATION, "Basic " + new String(encoded)); + return this; + } + public HttpPutBuilder addTxnId(long txnID) { header.put("txn_id", String.valueOf(txnID)); return this; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/JdbcIncrementalSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/JdbcIncrementalSourceReader.java index 4be0c20dfadd03..de553889370163 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/JdbcIncrementalSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/JdbcIncrementalSourceReader.java @@ -82,14 +82,22 @@ public abstract class JdbcIncrementalSourceReader implements SourceReader { private static final Logger LOG = LoggerFactory.getLogger(JdbcIncrementalSourceReader.class); private static ObjectMapper objectMapper = new ObjectMapper(); private SourceRecordDeserializer> serializer; - private Fetcher currentReader; private Map tableSchemas; - private SplitRecords currentSplitRecords; - private SourceSplitBase currentSplit; + + // Support for multiple snapshot splits + private List, SnapshotSplitState>> snapshotReaderContexts; + private int currentReaderIndex = 0; + + // Stream/binlog reader (single reader for stream split) + private Fetcher streamReader; + private StreamSplit streamSplit; + private StreamSplitState streamSplitState; protected FetchTask currentFetchTask; public JdbcIncrementalSourceReader() { this.serializer = new DebeziumJsonDeserializer(); + this.snapshotReaderContexts = new ArrayList<>(); } @Override @@ -152,69 +160,246 @@ public SplitReadResult prepareAndSubmitSplit(JobBaseRecordRequest baseReq) throw if (offsetMeta == null || offsetMeta.isEmpty()) { throw new RuntimeException("miss meta offset"); } - LOG.info("Job {} prepare and submit split with offset: {}", baseReq.getJobId(), offsetMeta); - // Build split - Tuple2 splitFlag = createSourceSplit(offsetMeta, baseReq); - this.currentSplit = splitFlag.f0; - LOG.info("Get a split: {}", this.currentSplit.toString()); - - // Create reader based on split type - if (this.currentSplit.isSnapshotSplit()) { - this.currentReader = getSnapshotSplitReader(baseReq); - } else if (this.currentSplit.isStreamSplit()) { - this.currentReader = getBinlogSplitReader(baseReq); + + LOG.info("Job {} read split records with offset: {}", baseReq.getJobId(), offsetMeta); + + String splitId = String.valueOf(offsetMeta.get(SPLIT_ID)); + if (BinlogSplit.BINLOG_SPLIT_ID.equals(splitId)){ + // Stream split mode + return prepareStreamSplit(offsetMeta, baseReq); } else { - throw new IllegalStateException( - "Unknown split type: " + this.currentSplit.getClass().getName()); + // Extract snapshot split list + List snapshotSplits = + extractSnapshotSplits(offsetMeta, baseReq); + return prepareSnapshotSplits(snapshotSplits, baseReq); } - + } + + /** + * Extract snapshot splits from meta. + * Only supports format: {"splits": [{"splitId": "xxx", ...}, ...]} + * + * @return List of snapshot splits + */ + private List extractSnapshotSplits( + Map offsetMeta, + JobBaseRecordRequest baseReq) { + + // Check if it contains "splits" array + Object splitsObj = offsetMeta.get("splits"); + if (splitsObj == null) { + throw new RuntimeException("Invalid meta format: missing 'splits' array"); + } + + if (!(splitsObj instanceof List)) { + throw new RuntimeException("Invalid meta format: 'splits' must be an array"); + } + + // Parse splits array + List> splitMetaList = (List>) splitsObj; + if (splitMetaList.isEmpty()) { + throw new RuntimeException("Invalid meta format: 'splits' array is empty"); + } + + List snapshotSplits = + new ArrayList<>(); + for (Map splitMeta : splitMetaList) { + org.apache.flink.cdc.connectors.base.source.meta.split.SnapshotSplit split = + createSnapshotSplit(splitMeta, baseReq); + snapshotSplits.add(split); + } + + LOG.info("Extracted {} snapshot split(s) from meta", snapshotSplits.size()); + return snapshotSplits; + } + + /** + * Prepare snapshot splits (unified handling for single or multiple splits) + */ + private SplitReadResult prepareSnapshotSplits( + List splits, + JobBaseRecordRequest baseReq) throws Exception { + + LOG.info("Preparing {} snapshot split(s) for reading", splits.size()); + + // Clear previous contexts + this.snapshotReaderContexts.clear(); + this.currentReaderIndex = 0; + + // Create reader for each split and submit + for (int i = 0; i < splits.size(); i++) { + org.apache.flink.cdc.connectors.base.source.meta.split.SnapshotSplit split = splits.get(i); + + // Create independent reader (each has its own Debezium queue) + Fetcher reader = getSnapshotSplitReader(baseReq, i); + + // Submit split (triggers async reading, data goes into reader's Debezium queue) + FetchTask splitFetchTask = createFetchTaskFromSplit(baseReq, split); + reader.submitTask(splitFetchTask); + + // Create split state + SnapshotSplitState splitState = new SnapshotSplitState(split); + + // Save context using generic SnapshotReaderContext + SnapshotReaderContext, SnapshotSplitState> context = + new SnapshotReaderContext<>(split, reader, splitState); + snapshotReaderContexts.add(context); + + LOG.info("Created reader {}/{} and submitted split: {} (table: {})", + i + 1, splits.size(), split.splitId(), split.getTableId().identifier()); + } + + // Construct return result with all splits and states + SplitReadResult result = new SplitReadResult(); + + List allSplits = new ArrayList<>(); + Map allStates = new HashMap<>(); + + for (SnapshotReaderContext, SnapshotSplitState> context + : snapshotReaderContexts) { + org.apache.flink.cdc.connectors.base.source.meta.split.SnapshotSplit split = context.getSplit(); + allSplits.add(split); + allStates.put(split.splitId(), context.getSplitState()); + } + + result.setSplits(allSplits); + result.setSplitStates(allStates); + + return result; + } + + /** + * Prepare stream split + */ + private SplitReadResult prepareStreamSplit( + Map offsetMeta, + JobBaseRecordRequest baseReq) throws Exception { + Tuple2 splitFlag = createStreamSplit(offsetMeta, baseReq); + this.streamSplit = splitFlag.f0.asStreamSplit(); + this.streamReader = getBinlogSplitReader(baseReq); + + LOG.info("Prepare stream split: {}", this.streamSplit.toString()); + // Submit split - FetchTask splitFetchTask = - createFetchTaskFromSplit(baseReq, this.currentSplit); - this.currentReader.submitTask(splitFetchTask); + FetchTask splitFetchTask = createFetchTaskFromSplit(baseReq, this.streamSplit); + this.streamReader.submitTask(splitFetchTask); this.setCurrentFetchTask(splitFetchTask); - - // Create split state - SourceSplitState currentSplitState; - if (this.currentSplit.isSnapshotSplit()) { - currentSplitState = new SnapshotSplitState(this.currentSplit.asSnapshotSplit()); - } else { - currentSplitState = new StreamSplitState(this.currentSplit.asStreamSplit()); - } - - // Return result without iterator + + this.streamSplitState = new StreamSplitState(this.streamSplit); + SplitReadResult result = new SplitReadResult(); - result.setSplit(this.currentSplit); - result.setSplitState(currentSplitState); + result.setSplits(Collections.singletonList(this.streamSplit)); + + Map statesMap = new HashMap<>(); + statesMap.put(this.streamSplit.splitId(), this.streamSplitState); + result.setSplitStates(statesMap); + return result; } @Override - public Iterator pollRecords(Object splitState) throws InterruptedException { - Preconditions.checkState(this.currentReader != null, "currentReader is null"); - Preconditions.checkNotNull(splitState, "splitState is null"); - Preconditions.checkState( - splitState instanceof SourceSplitState, - "splitState type is invalid " + splitState.getClass()); + public Iterator pollRecords() throws InterruptedException { + if (!snapshotReaderContexts.isEmpty()) { + // Snapshot split mode + return pollRecordsFromSnapshotReaders(); + } else if (streamReader != null) { + // Stream split mode + return pollRecordsFromStreamReader(); + } else { + throw new RuntimeException("No active snapshot or stream reader available"); + } + } + + /** + * Poll records from multiple snapshot readers sequentially. + * Directly pulls data from each reader's Debezium queue without extra queue. + * + *

This implementation loops through all readers in a single call. + * If a reader has no data, it automatically tries the next one. + * Each snapshot split is read only once. + */ + private Iterator pollRecordsFromSnapshotReaders() + throws InterruptedException { + if (snapshotReaderContexts.isEmpty()) { + return Collections.emptyIterator(); + } + + // Try all remaining readers (starting from currentReaderIndex) + int totalReaders = snapshotReaderContexts.size(); + int attempts = totalReaders - currentReaderIndex; + + for (int i = 0; i < attempts; i++) { + // Check if we've processed all splits + if (currentReaderIndex >= totalReaders) { + LOG.info("All {} snapshot splits have been processed", totalReaders); + return Collections.emptyIterator(); + } + + // Get current reader context + SnapshotReaderContext, SnapshotSplitState> context = + snapshotReaderContexts.get(currentReaderIndex); + + // Poll data directly from this reader's Debezium queue + Iterator dataIt = context.getReader().pollSplitRecords(); + + if (dataIt == null || !dataIt.hasNext()) { + // This reader has no data currently, move to next reader and continue + LOG.debug("Split {} has no data, trying next split", context.getSplit().splitId()); + currentReaderIndex++; + continue; + } + + // Has data, process and return + SourceRecords sourceRecords = dataIt.next(); + + // Move to next reader for next call + currentReaderIndex++; + + // Wrap as SplitRecords + SplitRecords splitRecords = + new SplitRecords(context.getSplit().splitId(), sourceRecords.iterator()); + + // Use FilteredRecordIterator to filter and update state + return new FilteredRecordIterator(splitRecords, context.getSplitState(), this); + } - // Poll data from Debezium queue - Iterator dataIt = currentReader.pollSplitRecords(); + LOG.info("All {} snapshot splits have no data currently", totalReaders); + return Collections.emptyIterator(); + } + + /** + * Poll records from stream reader + */ + private Iterator pollRecordsFromStreamReader() + throws InterruptedException { + + Preconditions.checkState(streamReader != null, "streamReader is null"); + Preconditions.checkNotNull(streamSplitState, "streamSplitState is null"); + + Iterator dataIt = streamReader.pollSplitRecords(); if (dataIt == null || !dataIt.hasNext()) { - return Collections.emptyIterator(); // No data available + return Collections.emptyIterator(); } - + SourceRecords sourceRecords = dataIt.next(); - SplitRecords splitRecords = - new SplitRecords(this.currentSplit.splitId(), sourceRecords.iterator()); - - // Return filtered iterator - return new FilteredRecordIterator(splitRecords, (SourceSplitState) splitState); + SplitRecords splitRecords = + new SplitRecords(streamSplit.splitId(), sourceRecords.iterator()); + + if (!sourceRecords.getSourceRecordList().isEmpty()) { + LOG.info("{} Records received from stream", + sourceRecords.getSourceRecordList().size()); + } + + return new FilteredRecordIterator(splitRecords, streamSplitState, this); } protected abstract DataType fromDbzColumn(Column splitColumn); protected abstract Fetcher getSnapshotSplitReader( - JobBaseConfig jobConfig); + JobBaseConfig jobConfig, int subtaskId); protected abstract Fetcher getBinlogSplitReader( JobBaseConfig jobConfig); @@ -229,20 +414,6 @@ protected abstract Fetcher getBinlogSplitReader( protected abstract JdbcDataSourceDialect getDialect(JdbcSourceConfig sourceConfig); - protected Tuple2 createSourceSplit( - Map offsetMeta, JobBaseConfig jobConfig) { - Tuple2 splitRes = null; - String splitId = String.valueOf(offsetMeta.get(SPLIT_ID)); - if (!BinlogSplit.BINLOG_SPLIT_ID.equals(splitId)) { - org.apache.flink.cdc.connectors.base.source.meta.split.SnapshotSplit split = - createSnapshotSplit(offsetMeta, jobConfig); - splitRes = Tuple2.of(split, false); - } else { - splitRes = createStreamSplit(offsetMeta, jobConfig); - } - return splitRes; - } - private org.apache.flink.cdc.connectors.base.source.meta.split.SnapshotSplit createSnapshotSplit(Map offset, JobBaseConfig jobConfig) { SnapshotSplit snapshotSplit = objectMapper.convertValue(offset, SnapshotSplit.class); @@ -457,14 +628,7 @@ private static void closeChunkSplitterOnly(HybridSplitAssigner splitAssigner) } } - private void closeCurrentReader() { - Fetcher currentReader = this.getCurrentReader(); - if (currentReader != null) { - LOG.info("Close current reader {}", currentReader.getClass().getCanonicalName()); - currentReader.close(); - this.setCurrentReader(null); - } - } + // Method removed - reader cleanup is now handled in finishSplitRecords() protected abstract FetchTask createFetchTaskFromSplit( JobBaseConfig jobConfig, SourceSplitBase split); @@ -472,6 +636,9 @@ protected abstract FetchTask createFetchTaskFromSplit( /** Get source config - to be implemented by subclasses */ protected abstract JdbcSourceConfig getSourceConfig(JobBaseConfig config); + /** Get source config - to be implemented by subclasses */ + protected abstract JdbcSourceConfig getSourceConfig(JobBaseConfig config, int subtaskId); + @Override public Map extractSnapshotStateOffset(Object splitState) { Preconditions.checkNotNull(splitState, "splitState is null"); @@ -514,9 +681,35 @@ public boolean isSnapshotSplit(SourceSplit split) { @Override public void finishSplitRecords() { - this.setCurrentSplitRecords(null); - // Close after each read, the binlog client will occupy the connection. - closeCurrentReader(); + // Clean up snapshot readers + if (!snapshotReaderContexts.isEmpty()) { + LOG.info("Closing {} snapshot readers", snapshotReaderContexts.size()); + for (SnapshotReaderContext, SnapshotSplitState> context + : snapshotReaderContexts) { + if (context.getReader() != null) { + closeReaderInternal(context.getReader()); + } + } + snapshotReaderContexts.clear(); + currentReaderIndex = 0; + } + + // Clean up stream reader + if (streamReader != null) { + LOG.info("Closing stream reader"); + closeReaderInternal(streamReader); + streamReader = null; + streamSplit = null; + streamSplitState = null; + } + } + + private void closeReaderInternal(Fetcher reader) { + if (reader != null) { + LOG.info("Close reader {}", reader.getClass().getCanonicalName()); + reader.close(); + } } private Map getTableSchemas(JobBaseConfig config) { @@ -534,10 +727,9 @@ protected abstract Map discoverTableSchemas( @Override public void close(JobBaseConfig jobConfig) { LOG.info("Close source reader for job {}", jobConfig.getJobId()); - closeCurrentReader(); - currentReader = null; - currentSplitRecords = null; - currentSplit = null; + // Clean up all readers + finishSplitRecords(); + if (tableSchemas != null) { tableSchemas.clear(); tableSchemas = null; @@ -552,15 +744,16 @@ public List deserialize(Map config, SourceRecord element /** * Filtered record iterator that only returns data change records, filtering out watermark, - * heartbeat and other events. This is a private inner class that encapsulates record filtering - * logic, making the main method cleaner. + * heartbeat and other events. This is a private static inner class that encapsulates record + * filtering logic, making the main method cleaner. */ private class FilteredRecordIterator implements Iterator { private final Iterator sourceIterator; private final SourceSplitState splitState; private SourceRecord nextRecord; - FilteredRecordIterator(SplitRecords currentSplitRecords, SourceSplitState splitState) { + FilteredRecordIterator(SplitRecords currentSplitRecords, SourceSplitState splitState, + JdbcIncrementalSourceReader reader) { this.sourceIterator = currentSplitRecords != null && !currentSplitRecords.isEmpty() ? currentSplitRecords.getIterator() diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SnapshotReaderContext.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SnapshotReaderContext.java new file mode 100644 index 00000000000000..6bd04d17743dbc --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SnapshotReaderContext.java @@ -0,0 +1,37 @@ +// 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.doris.cdcclient.source.reader; + +import lombok.Data; +import org.apache.flink.api.connector.source.SourceSplit; + +/** + * Generic context for managing a single snapshot split reader. + * + *

This class encapsulates the split, reader, and state for a single snapshot split + * + * @param Split type (extends MySqlSnapshotSplit) + * @param Reader type (e.g., SnapshotSplitReader) + * @param Split state type like MySqlSnapshotSplitState + */ +@Data +public class SnapshotReaderContext { + private final S split; + private final R reader; + private final ST splitState; +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java index 1ac5f0287222f3..d11e9c2db21d88 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java @@ -45,8 +45,8 @@ public interface SourceReader { /** Construct a split and submit a split reading task. */ SplitReadResult prepareAndSubmitSplit(JobBaseRecordRequest baseReq) throws Exception; - /** Retrieve data from the current split. */ - Iterator pollRecords(Object splitState) throws InterruptedException; + /** Retrieve data from the current split(s). */ + Iterator pollRecords() throws InterruptedException; /** Extract offset information from snapshot split state. */ Map extractSnapshotStateOffset(Object splitState); diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java index fc90c2d953aa90..0b8257b7902263 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java @@ -19,16 +19,39 @@ import org.apache.flink.api.connector.source.SourceSplit; +import java.util.List; +import java.util.Map; + import lombok.Data; /** - * Container for a source split and its associated state. - * Iteration over records for this split is handled separately (for example via pollRecords). + * Container for source splits and their associated states. + * Supports both single split (binlog) and multiple splits (snapshot). + * Iteration over records for these splits is handled separately (for example via pollRecords). */ @Data public class SplitReadResult { - // MySqlSplitState, SourceSplitState - private Object splitState; - // MySqlSplit SourceSplitBase - private SourceSplit split; + // List of splits (size=1 for binlog, size>=1 for snapshot) + private List splits; + + // Map of split states (key: splitId, value: state) + private Map splitStates; + + /** + * Get the first split ( The types in `splits` are the same.) + */ + public SourceSplit getSplit() { + return splits != null && !splits.isEmpty() ? splits.get(0) : null; + } + + /** + * Get the state of the first split + */ + public Object getSplitState() { + if (splits == null || splits.isEmpty() || splitStates == null) { + return null; + } + String firstSplitId = splits.get(0).splitId(); + return splitStates.get(firstSplitId); + } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java index 1bf7b02ffe21a1..0125e77960676d 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java @@ -21,6 +21,7 @@ import org.apache.doris.cdcclient.source.deserialize.SourceRecordDeserializer; import org.apache.doris.cdcclient.source.factory.DataSource; import org.apache.doris.cdcclient.source.reader.SourceReader; +import org.apache.doris.cdcclient.source.reader.SnapshotReaderContext; import org.apache.doris.cdcclient.source.reader.SplitReadResult; import org.apache.doris.cdcclient.source.reader.SplitRecords; import org.apache.doris.cdcclient.utils.ConfigUtil; @@ -81,6 +82,7 @@ import java.util.Optional; import java.util.Properties; import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; import static org.apache.doris.cdcclient.common.Constants.DEBEZIUM_HEARTBEAT_INTERVAL_MS; import static org.apache.doris.cdcclient.utils.ConfigUtil.is13Timestamp; @@ -111,13 +113,21 @@ public class MySqlSourceReader implements SourceReader { private static final FlinkJsonTableChangeSerializer TABLE_CHANGE_SERIALIZER = new FlinkJsonTableChangeSerializer(); private SourceRecordDeserializer> serializer; - private DebeziumReader currentReader; private Map tableSchemas; - private SplitRecords currentSplitRecords; - private MySqlSplit currentSplit; + + // Support for multiple snapshot splits with Round-Robin polling + private List> + snapshotReaderContexts; + private int currentReaderIndex = 0; + + // Binlog reader (single reader for binlog split) + private BinlogSplitReader binlogReader; + private MySqlBinlogSplit binlogSplit; + private MySqlBinlogSplitState binlogSplitState; public MySqlSourceReader() { this.serializer = new DebeziumJsonDeserializer(); + this.snapshotReaderContexts = new ArrayList<>(); } @Override @@ -173,52 +183,229 @@ public SplitReadResult prepareAndSubmitSplit(JobBaseRecordRequest baseReq) throw if (offsetMeta == null || offsetMeta.isEmpty()) { throw new RuntimeException("miss meta offset"); } + LOG.info("Job {} read split records with offset: {}", baseReq.getJobId(), offsetMeta); - // build split - Tuple2 splitFlag = createMySqlSplit(offsetMeta, baseReq); - this.currentSplit = splitFlag.f0; - LOG.info("Get a split: {}", this.currentSplit.toString()); - if (this.currentSplit instanceof MySqlSnapshotSplit) { - this.currentReader = getSnapshotSplitReader(baseReq); - } else if (this.currentSplit instanceof MySqlBinlogSplit) { - this.currentReader = getBinlogSplitReader(baseReq); - } else { - throw new IllegalStateException( - "Unsupported MySqlSplit type: " + this.currentSplit.getClass().getName()); - } - this.currentReader.submitSplit(this.currentSplit); - MySqlSplitState currentSplitState = null; - if (this.currentSplit.isSnapshotSplit()) { - currentSplitState = new MySqlSnapshotSplitState(this.currentSplit.asSnapshotSplit()); + String splitId = String.valueOf(offsetMeta.get(SPLIT_ID)); + if (BINLOG_SPLIT_ID.equals(splitId)){ + // Binlog split mode + return prepareBinlogSplit(offsetMeta, baseReq); } else { - currentSplitState = new MySqlBinlogSplitState(this.currentSplit.asBinlogSplit()); + // Extract snapshot split list + List snapshotSplits = extractSnapshotSplits(offsetMeta, baseReq); + return prepareSnapshotSplits(snapshotSplits, baseReq); } + } + + /** + * Extract snapshot splits from meta. + * Only supports format: {"splits": [{"splitId": "xxx", ...}, ...]} + * + * @return List of snapshot splits, or null if it's a binlog split + */ + private List extractSnapshotSplits( + Map offsetMeta, + JobBaseRecordRequest baseReq) throws JsonProcessingException { + + // Check if it contains "splits" array + Object splitsObj = offsetMeta.get("splits"); + if (splitsObj == null) { + throw new RuntimeException("Invalid meta format: missing 'splits' array"); + } + + if (!(splitsObj instanceof List)) { + throw new RuntimeException("Invalid meta format: 'splits' must be an array"); + } + + // Parse splits array + List> splitMetaList = (List>) splitsObj; + if (splitMetaList.isEmpty()) { + throw new RuntimeException("Invalid meta format: 'splits' array is empty"); + } + + List snapshotSplits = new ArrayList<>(); + for (Map splitMeta : splitMetaList) { + MySqlSnapshotSplit split = createSnapshotSplit(splitMeta, baseReq); + snapshotSplits.add(split); + } + + LOG.info("Extracted {} snapshot split(s) from meta", snapshotSplits.size()); + return snapshotSplits; + } + + /** + * Prepare snapshot splits (unified handling for single or multiple splits) + */ + private SplitReadResult prepareSnapshotSplits( + List splits, + JobBaseRecordRequest baseReq) throws Exception { + + LOG.info("Preparing {} snapshot split(s) for reading", splits.size()); + + // Clear previous contexts + this.snapshotReaderContexts.clear(); + this.currentReaderIndex = 0; + + // Create reader for each split and submit + for (int i = 0; i < splits.size(); i++) { + MySqlSnapshotSplit split = splits.get(i); + + // Create independent reader (each has its own Debezium queue) + SnapshotSplitReader reader = getSnapshotSplitReader(baseReq, i); + + // Submit split (triggers async reading, data goes into reader's Debezium queue) + reader.submitSplit(split); + + // Create split state + MySqlSnapshotSplitState splitState = new MySqlSnapshotSplitState(split); + + // Save context using generic SnapshotReaderContext + SnapshotReaderContext context = + new SnapshotReaderContext<>(split, reader, splitState); + snapshotReaderContexts.add(context); + + LOG.info("Created reader {}/{} and submitted split: {} (table: {})", + i + 1, splits.size(), split.splitId(), split.getTableId().identifier()); + } + + // Construct return result with all splits and states + SplitReadResult result = new SplitReadResult(); + + List allSplits = new ArrayList<>(); + Map allStates = new HashMap<>(); + + for (SnapshotReaderContext context + : snapshotReaderContexts) { + MySqlSnapshotSplit split = context.getSplit(); + allSplits.add(split); + allStates.put(split.splitId(), context.getSplitState()); + } + + result.setSplits(allSplits); + result.setSplitStates(allStates); + + return result; + } + + /** + * Prepare binlog split + */ + private SplitReadResult prepareBinlogSplit( + Map offsetMeta, + JobBaseRecordRequest baseReq) throws Exception { + Tuple2 splitFlag = createBinlogSplit(offsetMeta, baseReq); + this.binlogSplit = (MySqlBinlogSplit) splitFlag.f0; + this.binlogReader = getBinlogSplitReader(baseReq); + + LOG.info("Prepare binlog split: {}", this.binlogSplit.toString()); + + this.binlogReader.submitSplit(this.binlogSplit); + this.binlogSplitState = new MySqlBinlogSplitState(this.binlogSplit); + SplitReadResult result = new SplitReadResult(); - result.setSplit(this.currentSplit); - result.setSplitState(currentSplitState); + result.setSplits(Collections.singletonList(this.binlogSplit)); + + Map statesMap = new HashMap<>(); + statesMap.put(this.binlogSplit.splitId(), this.binlogSplitState); + result.setSplitStates(statesMap); + return result; } @Override - public Iterator pollRecords(Object splitState) throws InterruptedException { - Preconditions.checkState(this.currentReader != null, "currentReader is null"); - Preconditions.checkNotNull(splitState, "splitState is null"); - Preconditions.checkState( - splitState instanceof MySqlSplitState, - "splitState type is invalid " + splitState.getClass()); + public Iterator pollRecords() throws InterruptedException { + if (!snapshotReaderContexts.isEmpty()) { + // Snapshot split mode + return pollRecordsFromSnapshotReaders(); + } else if (binlogReader != null) { + // Binlog split mode + return pollRecordsFromBinlogReader(); + } else { + throw new RuntimeException("No active snapshot or binlog reader available"); + } + } + + /** + * Poll records from multiple snapshot readers sequentially. + * Directly pulls data from each reader's Debezium queue without extra queue. + * + *

This implementation loops through all readers in a single call. + * If a reader has no data, it automatically tries the next one. + * Each snapshot split is read only once. + */ + private Iterator pollRecordsFromSnapshotReaders() + throws InterruptedException { + if (snapshotReaderContexts.isEmpty()) { + return Collections.emptyIterator(); + } + + // Try all remaining readers (starting from currentReaderIndex) + int totalReaders = snapshotReaderContexts.size(); + int attempts = totalReaders - currentReaderIndex; + + for (int i = 0; i < attempts; i++) { + // Check if we've processed all splits + if (currentReaderIndex >= totalReaders) { + LOG.info("All {} snapshot splits have been processed", totalReaders); + return Collections.emptyIterator(); + } + + // Get current reader context + SnapshotReaderContext context = + snapshotReaderContexts.get(currentReaderIndex); + + // Poll data directly from this reader's Debezium queue + Iterator dataIt = context.getReader().pollSplitRecords(); + + if (dataIt == null || !dataIt.hasNext()) { + // This reader has no data currently, move to next reader and continue + LOG.debug("Split {} has no data, trying next split", context.getSplit().splitId()); + currentReaderIndex++; + continue; + } + + // Has data, process and return + SourceRecords sourceRecords = dataIt.next(); + + // Move to next reader for next call + currentReaderIndex++; + + // Wrap as SplitRecords + SplitRecords splitRecords = + new SplitRecords(context.getSplit().splitId(), sourceRecords.iterator()); + + // Use FilteredRecordIterator to filter and update state + return new FilteredRecordIterator(splitRecords, context.getSplitState()); + } - // Poll data from Debezium queue - Iterator dataIt = currentReader.pollSplitRecords(); + LOG.info("All {} snapshot splits have no data currently", totalReaders); + return Collections.emptyIterator(); + } + + /** + * Poll records from binlog reader + */ + private Iterator pollRecordsFromBinlogReader() + throws InterruptedException { + + Preconditions.checkState(binlogReader != null, "binlogReader is null"); + Preconditions.checkNotNull(binlogSplitState, "binlogSplitState is null"); + + Iterator dataIt = binlogReader.pollSplitRecords(); if (dataIt == null || !dataIt.hasNext()) { - return Collections.emptyIterator(); // No data available + return Collections.emptyIterator(); } - + SourceRecords sourceRecords = dataIt.next(); - SplitRecords splitRecords = - new SplitRecords(this.currentSplit.splitId(), sourceRecords.iterator()); - - return new FilteredRecordIterator(splitRecords, (MySqlSplitState) splitState); + SplitRecords splitRecords = + new SplitRecords(binlogSplit.splitId(), sourceRecords.iterator()); + + if (!sourceRecords.getSourceRecordList().isEmpty()) { + LOG.info("{} Records received from binlog", + sourceRecords.getSourceRecordList().size()); + } + + return new FilteredRecordIterator(splitRecords, binlogSplitState); } /** @@ -242,19 +429,6 @@ private void refreshTableChanges(SourceRecord element, Long jobId) throws IOExce } } - private Tuple2 createMySqlSplit( - Map offsetMeta, JobBaseConfig jobConfig) - throws JsonProcessingException { - Tuple2 splitRes = null; - String splitId = String.valueOf(offsetMeta.get(SPLIT_ID)); - if (!BINLOG_SPLIT_ID.equals(splitId)) { - MySqlSnapshotSplit split = createSnapshotSplit(offsetMeta, jobConfig); - splitRes = Tuple2.of(split, false); - } else { - splitRes = createBinlogSplit(offsetMeta, jobConfig); - } - return splitRes; - } private MySqlSnapshotSplit createSnapshotSplit( Map offset, JobBaseConfig jobConfig) throws JsonProcessingException { @@ -423,14 +597,14 @@ private void closeChunkSplitterOnly(MySqlSnapshotSplitAssigner splitAssigner) { } } - private SnapshotSplitReader getSnapshotSplitReader(JobBaseConfig config) { + private SnapshotSplitReader getSnapshotSplitReader(JobBaseConfig config, int subtaskId) { MySqlSourceConfig sourceConfig = getSourceConfig(config); final MySqlConnection jdbcConnection = DebeziumUtils.createMySqlConnection(sourceConfig); final BinaryLogClient binaryLogClient = DebeziumUtils.createBinaryClient(sourceConfig.getDbzConfiguration()); final StatefulTaskContext statefulTaskContext = new StatefulTaskContext(sourceConfig, binaryLogClient, jdbcConnection); - SnapshotSplitReader snapshotReader = new SnapshotSplitReader(statefulTaskContext, 0); + SnapshotSplitReader snapshotReader = new SnapshotSplitReader(statefulTaskContext, subtaskId); return snapshotReader; } @@ -445,15 +619,6 @@ private BinlogSplitReader getBinlogSplitReader(JobBaseConfig config) { return binlogReader; } - private void closeCurrentReader() { - DebeziumReader currentReader = this.getCurrentReader(); - if (currentReader != null) { - LOG.info("Close current reader {}", currentReader.getClass().getCanonicalName()); - currentReader.close(); - } - this.setCurrentReader(null); - } - private MySqlSourceConfig getSourceConfig(JobBaseConfig config) { return generateMySqlConfig(config); } @@ -539,9 +704,9 @@ private MySqlSourceConfig generateMySqlConfig(Map cdcConfig, Str configFactory.debeziumProperties(dbzProps); configFactory.heartbeatInterval(Duration.ofMillis(DEBEZIUM_HEARTBEAT_INTERVAL_MS)); - if (cdcConfig.containsKey(DataSourceConfigKeys.SPLIT_SIZE)) { + if (cdcConfig.containsKey(DataSourceConfigKeys.SNAPSHOT_SPLIT_SIZE)) { configFactory.splitSize( - Integer.parseInt(cdcConfig.get(DataSourceConfigKeys.SPLIT_SIZE))); + Integer.parseInt(cdcConfig.get(DataSourceConfigKeys.SNAPSHOT_SPLIT_SIZE))); } return configFactory.createConfig(0); @@ -560,10 +725,9 @@ private BinlogOffset initializeEffectiveOffset( @Override public Map extractSnapshotStateOffset(Object splitState) { Preconditions.checkNotNull(splitState, "splitState is null"); - MySqlSplitState mysqlSplitState = (MySqlSplitState) splitState; - BinlogOffset highWatermark = mysqlSplitState.asSnapshotSplitState().getHighWatermark(); - Map offsetRes = new HashMap<>(highWatermark.getOffset()); - return offsetRes; + MySqlSnapshotSplitState snapshotState = (MySqlSnapshotSplitState) splitState; + BinlogOffset highWatermark = snapshotState.getHighWatermark(); + return new HashMap<>(highWatermark.getOffset()); } @Override @@ -599,9 +763,27 @@ public boolean isSnapshotSplit(SourceSplit split) { @Override public void finishSplitRecords() { - this.setCurrentSplitRecords(null); - // Close after each read, the binlog client will occupy the connection. - closeCurrentReader(); + // Clean up snapshot readers + if (!snapshotReaderContexts.isEmpty()) { + LOG.info("Closing {} snapshot readers", snapshotReaderContexts.size()); + for (SnapshotReaderContext context + : snapshotReaderContexts) { + if (context.getReader() != null) { + context.getReader().close(); + } + } + snapshotReaderContexts.clear(); + currentReaderIndex = 0; + } + + // Clean up binlog reader + if (binlogReader != null) { + LOG.info("Closing binlog reader"); + binlogReader.close(); + binlogReader = null; + binlogSplit = null; + binlogSplitState = null; + } } @Override @@ -658,9 +840,8 @@ private Map discoverTableSchemas(JobBaseConfi @Override public void close(JobBaseConfig jobConfig) { LOG.info("Close source reader for job {}", jobConfig.getJobId()); - closeCurrentReader(); - currentReader = null; - currentSplitRecords = null; + + finishSplitRecords(); if (tableSchemas != null) { tableSchemas.clear(); tableSchemas = null; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java index 7c1894de21cd1d..5d89aca97707cb 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java @@ -86,7 +86,7 @@ public PostgresSourceReader() { @Override public void initialize(long jobId, DataSource dataSource, Map config) { - PostgresSourceConfig sourceConfig = generatePostgresConfig(config, jobId); + PostgresSourceConfig sourceConfig = generatePostgresConfig(config, jobId, 0); PostgresDialect dialect = new PostgresDialect(sourceConfig); LOG.info("Creating slot for job {}, user {}", jobId, sourceConfig.getUsername()); createSlotForGlobalStreamSplit(dialect); @@ -133,13 +133,18 @@ protected PostgresSourceConfig getSourceConfig(JobBaseConfig config) { return generatePostgresConfig(config); } + @Override + protected PostgresSourceConfig getSourceConfig(JobBaseConfig config, int subtaskId) { + return generatePostgresConfig(config.getConfig(), config.getJobId(), subtaskId); + } + /** Generate PostgreSQL source config from JobBaseConfig */ private PostgresSourceConfig generatePostgresConfig(JobBaseConfig config) { - return generatePostgresConfig(config.getConfig(), config.getJobId()); + return generatePostgresConfig(config.getConfig(), config.getJobId(), 0); } /** Generate PostgreSQL source config from Map config */ - private PostgresSourceConfig generatePostgresConfig(Map cdcConfig, Long jobId) { + private PostgresSourceConfig generatePostgresConfig(Map cdcConfig, Long jobId, int subtaskId) { PostgresSourceConfigFactory configFactory = new PostgresSourceConfigFactory(); // Parse JDBC URL to extract connection info @@ -197,9 +202,9 @@ private PostgresSourceConfig generatePostgresConfig(Map cdcConfi } // Set split size if provided - if (cdcConfig.containsKey(DataSourceConfigKeys.SPLIT_SIZE)) { + if (cdcConfig.containsKey(DataSourceConfigKeys.SNAPSHOT_SPLIT_SIZE)) { configFactory.splitSize( - Integer.parseInt(cdcConfig.get(DataSourceConfigKeys.SPLIT_SIZE))); + Integer.parseInt(cdcConfig.get(DataSourceConfigKeys.SNAPSHOT_SPLIT_SIZE))); } Properties dbzProps = ConfigUtil.getDefaultDebeziumProps(); @@ -212,7 +217,9 @@ private PostgresSourceConfig generatePostgresConfig(Map cdcConfi configFactory.decodingPluginName("pgoutput"); configFactory.heartbeatInterval( Duration.ofMillis(Constants.DEBEZIUM_HEARTBEAT_INTERVAL_MS)); - return configFactory.create(0); + + // subtaskId use pg create slot in snapshot phase, slotname is slot_name_subtaskId + return configFactory.create(subtaskId); } private String getSlotName(Long jobId) { @@ -220,13 +227,13 @@ private String getSlotName(Long jobId) { } @Override - protected IncrementalSourceScanFetcher getSnapshotSplitReader(JobBaseConfig config) { - PostgresSourceConfig sourceConfig = getSourceConfig(config); + protected IncrementalSourceScanFetcher getSnapshotSplitReader(JobBaseConfig config, int subtaskId) { + PostgresSourceConfig sourceConfig = getSourceConfig(config, subtaskId); PostgresDialect dialect = new PostgresDialect(sourceConfig); PostgresSourceFetchTaskContext taskContext = new PostgresSourceFetchTaskContext(sourceConfig, dialect); IncrementalSourceScanFetcher snapshotReader = - new IncrementalSourceScanFetcher(taskContext, 0); + new IncrementalSourceScanFetcher(taskContext, subtaskId); return snapshotReader; } @@ -236,6 +243,7 @@ protected IncrementalSourceStreamFetcher getBinlogSplitReader(JobBaseConfig conf PostgresDialect dialect = new PostgresDialect(sourceConfig); PostgresSourceFetchTaskContext taskContext = new PostgresSourceFetchTaskContext(sourceConfig, dialect); + // subTaskId maybe add jobId? IncrementalSourceStreamFetcher binlogReader = new IncrementalSourceStreamFetcher(taskContext, 0); return binlogReader; diff --git a/fs_brokers/cdc_client/src/main/resources/log4j.properties b/fs_brokers/cdc_client/src/main/resources/log4j.properties index 1ac680d4ddc5a2..f4c8faf2ff6130 100644 --- a/fs_brokers/cdc_client/src/main/resources/log4j.properties +++ b/fs_brokers/cdc_client/src/main/resources/log4j.properties @@ -16,7 +16,7 @@ # limitations under the License. ################################################################################ -log4j.rootLogger=INFO, FILE +log4j.rootLogger=INFO, STDOUT, FILE log.path=./logs diff --git a/regression-test/conf/regression-conf.groovy b/regression-test/conf/regression-conf.groovy index 79b4e5c20bc93d..99bf8199ce08f9 100644 --- a/regression-test/conf/regression-conf.groovy +++ b/regression-test/conf/regression-conf.groovy @@ -24,7 +24,7 @@ defaultDb = "regression_test" // init cmd like: select @@session.tx_read_only // at each time we connect. // add allowLoadLocalInfile so that the jdbc can execute mysql load data from client. -jdbcUrl = "jdbc:mysql://127.0.0.1:9030/?useLocalSessionState=true&allowLoadLocalInfile=true&zeroDateTimeBehavior=round" +jdbcUrl = "jdbc:mysql://10.16.10.6:29939/?useLocalSessionState=true&allowLoadLocalInfile=true&zeroDateTimeBehavior=round" targetJdbcUrl = "jdbc:mysql://127.0.0.1:9030/?useLocalSessionState=true&allowLoadLocalInfile=true&zeroDateTimeBehavior=round" jdbcUser = "root" jdbcPassword = "" @@ -35,7 +35,7 @@ syncerAddress = "127.0.0.1:9190" feSyncerUser = "root" feSyncerPassword = "" -feHttpAddress = "127.0.0.1:8030" +feHttpAddress = "10.16.10.6:28939" feHttpUser = "root" feHttpPassword = "" @@ -124,9 +124,9 @@ sk="***********" // jdbc connector test config // To enable jdbc test, you need first start mysql/pg container. // See `docker/thirdparties/run-thirdparties-docker.sh` -enableJdbcTest=false -mysql_57_port=3316 -pg_14_port=5442 +enableJdbcTest=true +mysql_57_port=3308 +pg_14_port=5438 oracle_11_port=1521 sqlserver_2022_port=1433 clickhouse_22_port=8123 @@ -258,7 +258,8 @@ max_failure_num=0 // used for exporting test s3ExportBucketName = "" -externalEnvIp="127.0.0.1" +externalEnvIp="10.16.10.6" + // trino-connector catalog test config enableTrinoConnectorTest = false diff --git a/regression-test/data/job_p0/streaming_job/cdc/test_streaming_postgres_job_split.out b/regression-test/data/job_p0/streaming_job/cdc/test_streaming_postgres_job_split.out new file mode 100644 index 00000000000000..6c5c602edc4fd8 --- /dev/null +++ b/regression-test/data/job_p0/streaming_job/cdc/test_streaming_postgres_job_split.out @@ -0,0 +1,8 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select_snapshot_table1 -- +1 A1 1 +2 B1 2 +3 C1 3 +4 D1 4 +5 E1 5 + diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job.groovy index 2febce4cb36909..bbd1037afe9421 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job.groovy @@ -91,7 +91,8 @@ suite("test_streaming_mysql_job", "p0,external,mysql,external_docker,external_do "password" = "123456", "database" = "${mysqlDb}", "include_tables" = "${table3},${table1},${table2}", - "offset" = "initial" + "offset" = "initial", + "snapshot_parallelism" = "1" ) TO DATABASE ${currentDb} ( "table.create.properties.replication_num" = "1" diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job.groovy index 7fe8cb73daaec2..9554ce4e973c14 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job.groovy @@ -28,7 +28,7 @@ suite("test_streaming_postgres_job", "p0,external,pg,external_docker,external_do def pgDB = "postgres" def pgSchema = "cdc_test" def pgUser = "postgres" - def pgPassword = "123456" + def pgPassword = "postgres" sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" sql """drop table if exists ${currentDb}.${table1} force""" @@ -87,7 +87,8 @@ suite("test_streaming_postgres_job", "p0,external,pg,external_docker,external_do "database" = "${pgDB}", "schema" = "${pgSchema}", "include_tables" = "${table1},${table2}", - "offset" = "initial" + "offset" = "initial", + "snapshot_parallelism" = "1" ) TO DATABASE ${currentDb} ( "table.create.properties.replication_num" = "1" diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_priv.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_priv.groovy index 9c0cd6a464c8ca..58d82f98430f27 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_priv.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_priv.groovy @@ -26,7 +26,7 @@ suite("test_streaming_postgres_job_priv", "p0,external,pg,external_docker,extern def pgDB = "postgres" def pgSchema = "cdc_test" def pgUser = "postgres" - def pgPassword = "123456" + def pgPassword = "postgres" sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" sql """drop table if exists ${currentDb}.${tableName} force""" diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_split.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_split.groovy new file mode 100644 index 00000000000000..dba580d5c79db2 --- /dev/null +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_split.groovy @@ -0,0 +1,115 @@ +// 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. + + +import org.awaitility.Awaitility + +import static java.util.concurrent.TimeUnit.SECONDS + +suite("test_streaming_postgres_job_split", "p0,external,pg,external_docker,external_docker_pg,nondatalake") { + def jobName = "test_streaming_postgres_job_split_name" + def currentDb = (sql "select database()")[0][0] + def table1 = "user_info_pg_split" + def pgDB = "postgres" + def pgSchema = "cdc_test" + def pgUser = "postgres" + def pgPassword = "postgres" + + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + String enabled = context.config.otherConfigs.get("enableJdbcTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String pg_port = context.config.otherConfigs.get("pg_14_port"); + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String s3_endpoint = getS3Endpoint() + String bucket = getS3BucketName() + String driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/postgresql-42.5.0.jar" + + // create test + connect("${pgUser}", "${pgPassword}", "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}") { + // sql """CREATE SCHEMA IF NOT EXISTS ${pgSchema}""" + sql """DROP TABLE IF EXISTS ${pgDB}.${pgSchema}.${table1}""" + sql """CREATE TABLE ${pgDB}.${pgSchema}.${table1} ( + "id" int2 PRIMARY KEY, + "name" varchar(200), + "age" int2 + )""" + // will split to 4 splits + sql """INSERT INTO ${pgDB}.${pgSchema}.${table1} (id, name, age) VALUES (1, 'A1', 1);""" + sql """INSERT INTO ${pgDB}.${pgSchema}.${table1} (id, name, age) VALUES (2, 'B1', 2);""" + sql """INSERT INTO ${pgDB}.${pgSchema}.${table1} (id, name, age) VALUES (3, 'C1', 3);""" + sql """INSERT INTO ${pgDB}.${pgSchema}.${table1} (id, name, age) VALUES (4, 'D1', 4);""" + sql """INSERT INTO ${pgDB}.${pgSchema}.${table1} (id, name, age) VALUES (5, 'E1', 5);""" + } + + sql """CREATE JOB ${jobName} + ON STREAMING + FROM POSTGRES ( + "jdbc_url" = "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}", + "driver_url" = "${driver_url}", + "driver_class" = "org.postgresql.Driver", + "user" = "${pgUser}", + "password" = "${pgPassword}", + "database" = "${pgDB}", + "schema" = "${pgSchema}", + "include_tables" = "${table1}", + "offset" = "initial", + "snapshot_split_size" = "1", + "snapshot_parallelism" = "2" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + + // check job running + try { + Awaitility.await().atMost(300, SECONDS) + .pollInterval(1, SECONDS).until( + { + def jobSuccendCount = sql """ select SucceedTaskCount from jobs("type"="insert") where Name = '${jobName}' and ExecuteType='STREAMING' """ + log.info("jobSuccendCount: " + jobSuccendCount) + // check job status and succeed task count larger than 2 + jobSuccendCount.size() == 1 && '2' <= jobSuccendCount.get(0).get(0) + } + ) + } catch (Exception ex){ + def showjob = sql """select * from jobs("type"="insert") where Name='${jobName}'""" + def showtask = sql """select * from tasks("type"="insert") where JobName='${jobName}'""" + log.info("show job: " + showjob) + log.info("show task: " + showtask) + throw ex; + } + + // check snapshot data + qt_select_snapshot_table1 """ SELECT * FROM ${table1} order by name asc """ + + def jobInfo = sql """ + select loadStatistic, status from jobs("type"="insert") where Name='${jobName}' + """ + log.info("jobInfo: " + jobInfo) + assert jobInfo.get(0).get(0) == "{\"scannedRows\":5,\"loadBytes\":270,\"fileNumber\":0,\"fileSize\":0}" + + sql """ + DROP JOB IF EXISTS where jobname = '${jobName}' + """ + + def jobCountRsp = sql """select count(1) from jobs("type"="insert") where Name ='${jobName}'""" + assert jobCountRsp.get(0).get(0) == 0 + } +} From a90a88c73eb945d1727bdf7d75f4e8e86f7f1095 Mon Sep 17 00:00:00 2001 From: wudi Date: Thu, 29 Jan 2026 17:47:38 +0800 Subject: [PATCH 11/11] fix --- .../apache/doris/cdcclient/service/PipelineCoordinator.java | 4 ++-- .../streaming_job/cdc/test_streaming_postgres_job.groovy | 2 +- .../streaming_job/cdc/test_streaming_postgres_job_priv.groovy | 2 +- .../cdc/test_streaming_postgres_job_split.groovy | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java index 76dd0c34e323ab..66825ed27d4084 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java @@ -496,10 +496,10 @@ private void cleanupReaderResources( *

This method handles both snapshot splits and binlog splits, * extracting the appropriate offset information through the SourceReader interface. * For snapshot splits: - * {"highWatermarks":[{"splitId":"tbl:1",...},...]} + * [{"splitId":"tbl:1",...},...] * * For Binlog Split: - * {"splitId":"binlog_split","fileName":"mysql-bin.000001","pos":"12345",...} + * [{"splitId":"binlog_split","fileName":"mysql-bin.000001","pos":"12345",...}] * * @param sourceReader the source reader * @param readResult the read result containing splits and split states diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job.groovy index 9554ce4e973c14..fdd414a6e2869a 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job.groovy @@ -28,7 +28,7 @@ suite("test_streaming_postgres_job", "p0,external,pg,external_docker,external_do def pgDB = "postgres" def pgSchema = "cdc_test" def pgUser = "postgres" - def pgPassword = "postgres" + def pgPassword = "123456" sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" sql """drop table if exists ${currentDb}.${table1} force""" diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_priv.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_priv.groovy index 58d82f98430f27..9c0cd6a464c8ca 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_priv.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_priv.groovy @@ -26,7 +26,7 @@ suite("test_streaming_postgres_job_priv", "p0,external,pg,external_docker,extern def pgDB = "postgres" def pgSchema = "cdc_test" def pgUser = "postgres" - def pgPassword = "postgres" + def pgPassword = "123456" sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" sql """drop table if exists ${currentDb}.${tableName} force""" diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_split.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_split.groovy index dba580d5c79db2..30efe01313caa8 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_split.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_split.groovy @@ -27,7 +27,7 @@ suite("test_streaming_postgres_job_split", "p0,external,pg,external_docker,exter def pgDB = "postgres" def pgSchema = "cdc_test" def pgUser = "postgres" - def pgPassword = "postgres" + def pgPassword = "123456" sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" sql """drop table if exists ${currentDb}.${table1} force"""