Details
-
Bug
-
Status: Resolved
-
Blocker
-
Resolution: Fixed
-
cdc-3.1.1
-
flink-cdc-3.1.x
Description
When a task is started by multiple mysql connector timestamp start mode at the same time, when searching for binlog timestamp, there will be task blocking problem, which may cause source to be unable to obtain data all the time.
1、 I have four tables(products,orders,orders_copy,shipments,)to capture in a task . For these four tables, I made a lot of binlogs,such as 10 million。
2、I try start it with timestamp mode and the products table could not get any records .
3、I try start it with timestamp mode ,but the orders_copy table could not get any records
3、I debug code and find some problems
# Class: org.apache.flink.cdc.connectors.mysql.debezium.DebeziumUtils private static String searchBinlogName( BinaryLogClient client, long targetMs, List<String> binlogFiles) throws IOException, InterruptedException { int startIdx = 0; int endIdx = binlogFiles.size() - 1; while (startIdx <= endIdx) { int mid = startIdx + (endIdx - startIdx) / 2; long midTs = getBinlogTimestamp(client, binlogFiles.get(mid)); if (midTs < targetMs) { startIdx = mid + 1; } else if (targetMs < midTs) { endIdx = mid - 1; } else { return binlogFiles.get(mid); } } return endIdx < 0 ? binlogFiles.get(0) : binlogFiles.get(endIdx); } private static long getBinlogTimestamp(BinaryLogClient client, String binlogFile) throws IOException, InterruptedException { ArrayBlockingQueue<Long> binlogTimestamps = new ArrayBlockingQueue<>(1); BinaryLogClient.EventListener eventListener = event -> { EventData data = event.getData(); if (data instanceof RotateEventData) { // We skip RotateEventData because it does not contain the timestamp we are // interested in. return; } EventHeaderV4 header = event.getHeader(); long timestamp = header.getTimestamp(); if (timestamp > 0) { binlogTimestamps.offer(timestamp); try { client.disconnect(); } catch (IOException e) { throw new RuntimeException(e); } } }; try { client.registerEventListener(eventListener); client.setBinlogFilename(binlogFile); client.setBinlogPosition(0); LOG.info("begin parse binlog: {}", binlogFile); client.connect(); } finally { client.unregisterEventListener(eventListener); } return binlogTimestamps.take(); }
5、 the funciton binlogTimestamps.take() is blocking until the queue has records.
6、the binlogTimestamps queue is always blocking and cannot get any data.
Attachments
Attachments
Issue Links
- links to