|
|
|
@ -19,6 +19,9 @@ package com.ververica.cdc.connectors.mysql.debezium;
|
|
|
|
|
import org.apache.flink.util.FlinkRuntimeException;
|
|
|
|
|
|
|
|
|
|
import com.github.shyiko.mysql.binlog.BinaryLogClient;
|
|
|
|
|
import com.github.shyiko.mysql.binlog.event.EventData;
|
|
|
|
|
import com.github.shyiko.mysql.binlog.event.EventHeaderV4;
|
|
|
|
|
import com.github.shyiko.mysql.binlog.event.RotateEventData;
|
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.config.MySqlSourceConfig;
|
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.connection.JdbcConnectionFactory;
|
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.offset.BinlogOffset;
|
|
|
|
@ -41,11 +44,14 @@ import io.debezium.util.SchemaNameAdjuster;
|
|
|
|
|
import org.slf4j.Logger;
|
|
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
|
|
|
|
import java.io.IOException;
|
|
|
|
|
import java.sql.SQLException;
|
|
|
|
|
import java.util.ArrayList;
|
|
|
|
|
import java.util.HashMap;
|
|
|
|
|
import java.util.List;
|
|
|
|
|
import java.util.Map;
|
|
|
|
|
import java.util.Properties;
|
|
|
|
|
import java.util.concurrent.ArrayBlockingQueue;
|
|
|
|
|
import java.util.function.Predicate;
|
|
|
|
|
|
|
|
|
|
import static com.ververica.cdc.connectors.mysql.source.utils.TableDiscoveryUtils.listTables;
|
|
|
|
@ -233,4 +239,96 @@ public class DebeziumUtils {
|
|
|
|
|
|
|
|
|
|
return variables;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
public static BinlogOffset findBinlogOffset(long targetMs, MySqlConnection connection) {
|
|
|
|
|
MySqlConnection.MySqlConnectionConfiguration config = connection.connectionConfig();
|
|
|
|
|
BinaryLogClient client =
|
|
|
|
|
new BinaryLogClient(
|
|
|
|
|
config.hostname(), config.port(), config.username(), config.password());
|
|
|
|
|
|
|
|
|
|
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 {
|
|
|
|
|
connection.query("SHOW BINARY LOGS", rsc);
|
|
|
|
|
LOG.info("Total search binlog: {}", binlogFiles);
|
|
|
|
|
|
|
|
|
|
if (binlogFiles.isEmpty()) {
|
|
|
|
|
return BinlogOffset.ofBinlogFilePosition("", 0);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
String binlogName = searchBinlogName(client, targetMs, binlogFiles);
|
|
|
|
|
return BinlogOffset.ofBinlogFilePosition(binlogName, 0);
|
|
|
|
|
} catch (Exception e) {
|
|
|
|
|
throw new FlinkRuntimeException(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);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
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();
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|