|
|
|
@ -41,6 +41,7 @@ import java.util.Map;
|
|
|
|
|
import java.util.concurrent.ExecutorService;
|
|
|
|
|
import java.util.concurrent.Executors;
|
|
|
|
|
import java.util.concurrent.ThreadFactory;
|
|
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
|
|
|
|
|
|
|
|
|
import static com.ververica.cdc.connectors.base.utils.SourceRecordUtils.formatMessageTimestamp;
|
|
|
|
@ -64,7 +65,7 @@ public class JdbcSourceScanFetcher implements Fetcher<SourceRecords, SourceSplit
|
|
|
|
|
public AtomicBoolean reachEnd;
|
|
|
|
|
|
|
|
|
|
private final JdbcSourceFetchTaskContext taskContext;
|
|
|
|
|
private final ExecutorService executor;
|
|
|
|
|
private final ExecutorService executorService;
|
|
|
|
|
private volatile ChangeEventQueue<DataChangeEvent> queue;
|
|
|
|
|
private volatile Throwable readException;
|
|
|
|
|
|
|
|
|
@ -73,13 +74,15 @@ public class JdbcSourceScanFetcher implements Fetcher<SourceRecords, SourceSplit
|
|
|
|
|
private SnapshotSplit currentSnapshotSplit;
|
|
|
|
|
private SchemaNameAdjuster nameAdjuster;
|
|
|
|
|
|
|
|
|
|
private static final long READER_CLOSE_TIMEOUT_SECONDS = 30L;
|
|
|
|
|
|
|
|
|
|
public JdbcSourceScanFetcher(JdbcSourceFetchTaskContext taskContext, int subtaskId) {
|
|
|
|
|
this.taskContext = taskContext;
|
|
|
|
|
ThreadFactory threadFactory =
|
|
|
|
|
new ThreadFactoryBuilder()
|
|
|
|
|
.setNameFormat("debezium-snapshot-reader-" + subtaskId)
|
|
|
|
|
.build();
|
|
|
|
|
this.executor = Executors.newSingleThreadExecutor(threadFactory);
|
|
|
|
|
this.executorService = Executors.newSingleThreadExecutor(threadFactory);
|
|
|
|
|
this.hasNextElement = new AtomicBoolean(false);
|
|
|
|
|
this.reachEnd = new AtomicBoolean(false);
|
|
|
|
|
}
|
|
|
|
@ -93,7 +96,7 @@ public class JdbcSourceScanFetcher implements Fetcher<SourceRecords, SourceSplit
|
|
|
|
|
this.queue = taskContext.getQueue();
|
|
|
|
|
this.hasNextElement.set(true);
|
|
|
|
|
this.reachEnd.set(false);
|
|
|
|
|
executor.submit(
|
|
|
|
|
executorService.submit(
|
|
|
|
|
() -> {
|
|
|
|
|
try {
|
|
|
|
|
snapshotSplitReadTask.execute(taskContext);
|
|
|
|
@ -190,7 +193,21 @@ public class JdbcSourceScanFetcher implements Fetcher<SourceRecords, SourceSplit
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Override
|
|
|
|
|
public void close() {}
|
|
|
|
|
public void close() {
|
|
|
|
|
try {
|
|
|
|
|
if (executorService != null) {
|
|
|
|
|
executorService.shutdown();
|
|
|
|
|
if (executorService.awaitTermination(
|
|
|
|
|
READER_CLOSE_TIMEOUT_SECONDS, TimeUnit.SECONDS)) {
|
|
|
|
|
LOG.warn(
|
|
|
|
|
"Failed to close the scan fetcher in {} seconds.",
|
|
|
|
|
READER_CLOSE_TIMEOUT_SECONDS);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
} catch (Exception e) {
|
|
|
|
|
LOG.error("Close scan fetcher error", e);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private void assertLowWatermark(SourceRecord lowWatermark) {
|
|
|
|
|
checkState(
|
|
|
|
|