Hisoka-X commented on code in PR #9628:
URL: https://github.com/apache/seatunnel/pull/9628#discussion_r2239979852
##########
seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlConnectionUtils.java:
##########
@@ -185,4 +192,87 @@ private static Map<String, String> querySystemVariables(
return variables;
}
+
+ public static BinlogOffset findBinlogOffsetBytimestamp(
+ JdbcConnection jdbc, BinaryLogClient client, long timestamp) {
+ final String showBinaryLogStmt = "SHOW BINARY LOGS";
+ List<String> binlogFiles = new ArrayList<>();
+ JdbcConnection.ResultSetConsumer rsc =
+ rs -> {
+ while (rs.next()) {
+ String fileName = rs.getString(1);
+ long fileSize = rs.getLong(2);
+ if (fileSize > 0) {
+ binlogFiles.add(fileName);
+ }
+ }
+ };
+ try {
+ jdbc.query(showBinaryLogStmt, rsc);
+ if (binlogFiles.isEmpty()) {
+ return BinlogOffset.INITIAL_OFFSET;
+ }
+ String binlogName = searchBinlogName(client, timestamp,
binlogFiles);
+ return new BinlogOffset(binlogName, 0);
Review Comment:
So, our logic cannot precisely specify the binlog offset, and can only
locate the file closest to the specified timestamp? This might result in data
before the timestamp being read, or data after the timestamp being discarded.
Is my understanding correct?
##########
seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlConnectionUtils.java:
##########
@@ -185,4 +192,87 @@ private static Map<String, String> querySystemVariables(
return variables;
}
+
+ public static BinlogOffset findBinlogOffsetBytimestamp(
+ JdbcConnection jdbc, BinaryLogClient client, long timestamp) {
+ final String showBinaryLogStmt = "SHOW BINARY LOGS";
+ List<String> binlogFiles = new ArrayList<>();
+ JdbcConnection.ResultSetConsumer rsc =
+ rs -> {
+ while (rs.next()) {
+ String fileName = rs.getString(1);
+ long fileSize = rs.getLong(2);
+ if (fileSize > 0) {
+ binlogFiles.add(fileName);
+ }
+ }
+ };
+ try {
+ jdbc.query(showBinaryLogStmt, rsc);
+ if (binlogFiles.isEmpty()) {
+ return BinlogOffset.INITIAL_OFFSET;
+ }
+ String binlogName = searchBinlogName(client, timestamp,
binlogFiles);
+ return new BinlogOffset(binlogName, 0);
+ } catch (Exception e) {
+ throw new SeaTunnelException(e);
+ }
+ }
+
+ 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);
+ }
+
+ public static long getBinlogTimestamp(BinaryLogClient client, String
binlogFile)
+ throws IOException, InterruptedException {
+
+ ArrayBlockingQueue<Long> binlogTimestamps = new
ArrayBlockingQueue<>(1);
Review Comment:
Why use `ArrayBlockingQueue` to store the only one value? It's weird.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]