[cdc-base] Release the scan fetcher thread resource after reading finished (#1619)

pull/1156/head
Hang Ruan 2 years ago committed by GitHub
parent 806a8505d3
commit f44d2092c7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23

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

@ -47,6 +47,7 @@ import java.util.Set;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
import static com.ververica.cdc.connectors.base.utils.SourceRecordUtils.getTableId; import static com.ververica.cdc.connectors.base.utils.SourceRecordUtils.getTableId;
import static com.ververica.cdc.connectors.base.utils.SourceRecordUtils.isDataChangeRecord; import static com.ververica.cdc.connectors.base.utils.SourceRecordUtils.isDataChangeRecord;
@ -57,7 +58,7 @@ public class JdbcSourceStreamFetcher implements Fetcher<SourceRecords, SourceSpl
private static final Logger LOG = LoggerFactory.getLogger(JdbcSourceStreamFetcher.class); private static final Logger LOG = LoggerFactory.getLogger(JdbcSourceStreamFetcher.class);
private final JdbcSourceFetchTaskContext taskContext; private final JdbcSourceFetchTaskContext taskContext;
private final ExecutorService executor; private final ExecutorService executorService;
private final Set<TableId> pureBinlogPhaseTables; private final Set<TableId> pureBinlogPhaseTables;
private volatile ChangeEventQueue<DataChangeEvent> queue; private volatile ChangeEventQueue<DataChangeEvent> queue;
@ -70,11 +71,13 @@ public class JdbcSourceStreamFetcher implements Fetcher<SourceRecords, SourceSpl
private Map<TableId, Offset> maxSplitHighWatermarkMap; private Map<TableId, Offset> maxSplitHighWatermarkMap;
private Tables.TableFilter capturedTableFilter; private Tables.TableFilter capturedTableFilter;
private static final long READER_CLOSE_TIMEOUT_SECONDS = 30L;
public JdbcSourceStreamFetcher(JdbcSourceFetchTaskContext taskContext, int subTaskId) { public JdbcSourceStreamFetcher(JdbcSourceFetchTaskContext taskContext, int subTaskId) {
this.taskContext = taskContext; this.taskContext = taskContext;
ThreadFactory threadFactory = ThreadFactory threadFactory =
new ThreadFactoryBuilder().setNameFormat("debezium-reader-" + subTaskId).build(); new ThreadFactoryBuilder().setNameFormat("debezium-reader-" + subTaskId).build();
this.executor = Executors.newSingleThreadExecutor(threadFactory); this.executorService = Executors.newSingleThreadExecutor(threadFactory);
this.pureBinlogPhaseTables = new HashSet<>(); this.pureBinlogPhaseTables = new HashSet<>();
} }
@ -85,7 +88,7 @@ public class JdbcSourceStreamFetcher implements Fetcher<SourceRecords, SourceSpl
configureFilter(); configureFilter();
taskContext.configure(currentStreamSplit); taskContext.configure(currentStreamSplit);
this.queue = taskContext.getQueue(); this.queue = taskContext.getQueue();
executor.submit( executorService.submit(
() -> { () -> {
try { try {
streamFetchTask.execute(taskContext); streamFetchTask.execute(taskContext);
@ -134,7 +137,21 @@ public class JdbcSourceStreamFetcher implements Fetcher<SourceRecords, SourceSpl
} }
@Override @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 stream fetcher in {} seconds.",
READER_CLOSE_TIMEOUT_SECONDS);
}
}
} catch (Exception e) {
LOG.error("Close stream fetcher error", e);
}
}
/** /**
* Returns the record should emit or not. * Returns the record should emit or not.

Loading…
Cancel
Save