DBZ-140 Improved locking logic to support RDS

Improved the MySQL connector's logic to better handle Amazon RDS that does not allow giving user `SUPER` privileges. As before, the connector starts a transaction and attempts to get a global read lock via `FLUSH TABLES WITH READ LOCK` to prevent writes to the database so that the binlog position can be accurately read _and_ the table schemas can be read without interference from other clients. Once that is done, the connector releases the global read lock and continues in the same transaction to read all table rows. This means that our snapshot is consistent, but we maintain the global read lock for a very short period of time.

Amazon's RDS and Aurora are hosted MySQL instances that do not allow users to have the `SUPER` privilege, which means the user cannot get a global read lock. In this case, the connector detects this error, continues to read the database and table names (without any lock), and _then_ uses `FLUSH TABLES <tableName> WITH READ LOCK` on each table that satisfies the filters to prevent changes from other clients. The connector then reads the table schemas, reads _all_ table rows, commits the transaction, and _finally_ releases the table locks.

Therefore, there are two very different behaviors/requirements when the user can't obtain a global read lock because of lack of privilege, like on RDS:

# The RDS user that the connector makes use of must also have the `LOCK TABLES` privilege; without it the connector will fail during the snapshot.
# The connector must hold the table read locks _until it has completed reading all of the tables_, since release the table locks using `UNLOCK TABLES` would prematurely commit our transaction and prevent us from getting a consistent snapshot. From the [MySQL documentation](https://dev.mysql.com/doc/refman/5.7/en/flush.html):
> `UNLOCK TABLES` implicitly commits any active transaction only if any tables currently have been locked with `LOCK TABLES`. The commit does not occur for `UNLOCK TABLES` following `FLUSH TABLES WITH READ LOCK` because the latter statement does not acquire table locks.
This commit is contained in:
Randall Hauch 2017-02-06 13:56:55 -06:00
parent 3ff9ca8344
commit bb0800ca3a
2 changed files with 177 additions and 90 deletions

View File

@ -11,6 +11,7 @@
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.kafka.connect.errors.ConnectException;
@ -146,6 +147,33 @@ public String knownGtidSet() {
return result != null ? result : "";
}
/**
* Determine if the current user has the named privilege. Note that if the user has the "ALL" privilege this method
* returns {@code true}.
*
* @param grantName the name of the MySQL privilege; may not be null
* @return {@code true} if the user has the named privilege, or {@code false} otherwise
*/
public boolean userHasPrivileges(String grantName) {
AtomicBoolean result = new AtomicBoolean(false);
try {
jdbc.query("SHOW GRANTS FOR CURRENT_USER", rs -> {
if (rs.next()) {
String grants = rs.getString(1);
logger.debug(grants);
if (grants == null) return;
grants = grants.toUpperCase();
if (grants.contains("ALL") || grants.contains(grantName.toUpperCase())) {
result.set(true);
}
}
});
} catch (SQLException e) {
throw new ConnectException("Unexpected error while connecting to MySQL and looking at privileges for current user: ", e);
}
return result.get();
}
protected String connectionString() {
return jdbc.connectionString(MYSQL_CONNECTION_URL);
}

View File

@ -20,6 +20,7 @@
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.kafka.connect.errors.ConnectException;
import org.apache.kafka.connect.source.SourceRecord;
import io.debezium.connector.mysql.RecordMakers.RecordsForTable;
@ -139,6 +140,7 @@ protected void execute() {
logServerInformation(mysql);
boolean isLocked = false;
boolean isTxnStarted = false;
boolean tableLocks = false;
try {
metrics.startSnapshot();
@ -160,13 +162,13 @@ protected void execute() {
mysql.setAutoCommit(false);
sql.set("SET TRANSACTION ISOLATION LEVEL REPEATABLE READ");
mysql.execute(sql.get());
metrics.globalLockAcquired();
// Generate the DDL statements that set the charset-related system variables ...
Map<String, String> systemVariables = context.readMySqlCharsetSystemVariables(sql);
String setSystemVariablesStatement = context.setStatementFor(systemVariables);
AtomicBoolean interrupted = new AtomicBoolean(false);
long lockAcquired = 0L;
int step = 1;
try {
// ------
@ -180,59 +182,40 @@ protected void execute() {
mysql.execute(sql.get());
isTxnStarted = true;
// ------
// STEP 2
// ------
// ------------------------------------
// LOCK TABLES and READ BINLOG POSITION
// ------------------------------------
// Obtain read lock on all tables. This statement closes all open tables and locks all tables
// for all databases with a global read lock, and it prevents ALL updates while we have this lock.
// It also ensures that everything we do while we have this lock will be consistent.
if (!isRunning()) return;
lockAcquired = clock.currentTimeInMillis();
logger.info("Step 2: flush and obtain global read lock (preventing writes to database)");
sql.set("FLUSH TABLES WITH READ LOCK");
mysql.execute(sql.get());
isLocked = true;
try {
logger.info("Step 2: flush and obtain global read lock to prevent writes to database");
sql.set("FLUSH TABLES WITH READ LOCK");
mysql.execute(sql.get());
lockAcquired = clock.currentTimeInMillis();
metrics.globalLockAcquired();
isLocked = true;
} catch (SQLException e) {
logger.info("Step 2: unable to flush and acquire global read lock, will use table read locks after reading table names");
// Continue anyway, since RDS (among others) don't allow setting a global lock
assert !isLocked;
}
// ------
// STEP 3
// ------
// Obtain the binlog position and update the SourceInfo in the context. This means that all source records
// generated
// as part of the snapshot will contain the binlog position of the snapshot.
if (!isRunning()) return;
logger.info("Step 3: read binlog position of MySQL master");
String showMasterStmt = "SHOW MASTER STATUS";
sql.set(showMasterStmt);
mysql.query(sql.get(), rs -> {
if (rs.next()) {
String binlogFilename = rs.getString(1);
long binlogPosition = rs.getLong(2);
source.setBinlogStartPoint(binlogFilename, binlogPosition);
if (rs.getMetaData().getColumnCount() > 4) {
// This column exists only in MySQL 5.6.5 or later ...
String gtidSet = rs.getString(5);// GTID set, may be null, blank, or contain a GTID set
source.setCompletedGtidSet(gtidSet);
logger.info("\t using binlog '{}' at position '{}' and gtid '{}'", binlogFilename, binlogPosition,
gtidSet);
} else {
logger.info("\t using binlog '{}' at position '{}'", binlogFilename, binlogPosition);
}
source.startSnapshot();
} else {
throw new IllegalStateException("Cannot read the binlog filename and position via '" + showMasterStmt
+ "'. Make sure your server is correctly configured");
}
});
step = 3;
if (isLocked) {
// Obtain the binlog position and update the SourceInfo in the context. This means that all source records
// generated as part of the snapshot will contain the binlog position of the snapshot.
readBinlogPosition(step++, source, mysql, sql);
}
// From this point forward, all source records produced by this connector will have an offset that includes a
// "snapshot" field (with value of "true").
// ------
// STEP 4
// ------
// -------------------
// READ DATABASE NAMES
// -------------------
// Get the list of databases ...
if (!isRunning()) return;
logger.info("Step 4: read list of available databases");
logger.info("Step {}: read list of available databases", step++);
final List<String> databaseNames = new ArrayList<>();
sql.set("SHOW DATABASES");
mysql.query(sql.get(), rs -> {
@ -242,14 +225,14 @@ protected void execute() {
});
logger.info("\t list of available databases is: {}", databaseNames);
// ------
// STEP 5
// ------
// ----------------
// READ TABLE NAMES
// ----------------
// Get the list of table IDs for each database. We can't use a prepared statement with MySQL, so we have to
// build the SQL statement each time. Although in other cases this might lead to SQL injection, in our case
// we are reading the database names from the database and not taking them from the user ...
if (!isRunning()) return;
logger.info("Step 5: read list of available tables in each database");
logger.info("Step {}: read list of available tables in each database", step++);
List<TableId> tableIds = new ArrayList<>();
final Map<String, List<TableId>> tableIdsByDbName = new HashMap<>();
final Set<String> readableDatabaseNames = new HashSet<>();
@ -278,12 +261,42 @@ protected void execute() {
}
logger.info("\t snapshot continuing with databases: {}", readableDatabaseNames);
if (!isLocked) {
// ------------------------------------
// LOCK TABLES and READ BINLOG POSITION
// ------------------------------------
// We were not able to acquire the global read lock, so instead we have to obtain a read lock on each table.
// This requires different privileges than normal, and also means we can't unlock the tables without
// implicitly committing our transaction ...
if (!context.userHasPrivileges("LOCK TABLES")) {
// We don't have the right privileges
throw new ConnectException("User does not have the 'LOCK TABLES' privilege required to obtain a "
+ "consistent snapshot by preventing concurrent writes to tables.");
}
// We have the required privileges, so try to lock all of the tables we're interested in ...
logger.info("Step {}: flush and obtain read lock for {} tables (preventing writes)", step++, tableIds.size());
for (TableId tableId : tableIds) {
sql.set("FLUSH TABLES " + quote(tableId) + " WITH READ LOCK");
mysql.execute(sql.get());
}
lockAcquired = clock.currentTimeInMillis();
metrics.globalLockAcquired();
isLocked = true;
tableLocks = true;
// Our tables are locked, so read the binlog position ...
readBinlogPosition(step++, source, mysql, sql);
}
// From this point forward, all source records produced by this connector will have an offset that includes a
// "snapshot" field (with value of "true").
// ------
// STEP 6
// ------
// Transform the current schema so that it reflects the *current* state of the MySQL server's contents.
// First, get the DROP TABLE and CREATE TABLE statement (with keys and constraint definitions) for our tables ...
logger.info("Step 6: generating DROP and CREATE statements to reflect current database schemas:");
logger.info("Step {}: generating DROP and CREATE statements to reflect current database schemas:", step++);
schema.applyDdl(source, null, setSystemVariablesStatement, this::enqueueSchemaChanges);
// Add DROP TABLE statements for all tables that we knew about AND those tables found in the databases ...
@ -327,17 +340,28 @@ protected void execute() {
// STEP 7
// ------
if (minimalBlocking && isLocked) {
// We are doing minimal blocking, then we should release the read lock now. All subsequent SELECT
// should still use the MVCC snapshot obtained when we started our transaction (since we started it
// "...with consistent snapshot"). So, since we're only doing very simple SELECT without WHERE predicates,
// we can release the lock now ...
logger.info("Step 7: releasing global read lock to enable MySQL writes");
sql.set("UNLOCK TABLES");
mysql.execute(sql.get());
isLocked = false;
long lockReleased = clock.currentTimeInMillis();
metrics.globalLockReleased();
logger.info("Step 7: blocked writes to MySQL for a total of {}", Strings.duration(lockReleased - lockAcquired));
if (tableLocks) {
// We could not acquire a global read lock and instead had to obtain individual table-level read locks
// using 'FLUSH TABLE <tableName> WITH READ LOCK'. However, if we were to do this, the 'UNLOCK TABLES'
// would implicitly commit our active transaction, and this would break our consistent snapshot logic.
// Therefore, we cannot unlock the tables here!
// https://dev.mysql.com/doc/refman/5.7/en/flush.html
logger.info("Step {}: tables were locked explicitly, but to get a consistent snapshot we cannot "
+ "release the locks until we've read all tables.", step++);
} else {
// We are doing minimal blocking via a global read lock, so we should release the global read lock now.
// All subsequent SELECT should still use the MVCC snapshot obtained when we started our transaction
// (since we started it "...with consistent snapshot"). So, since we're only doing very simple SELECT
// without WHERE predicates, we can release the lock now ...
logger.info("Step {}: releasing global read lock to enable MySQL writes", step);
sql.set("UNLOCK TABLES");
mysql.execute(sql.get());
isLocked = false;
long lockReleased = clock.currentTimeInMillis();
metrics.globalLockReleased();
logger.info("Step {}: blocked writes to MySQL for a total of {}", step++,
Strings.duration(lockReleased - lockAcquired));
}
}
// ------
@ -350,7 +374,7 @@ protected void execute() {
BufferedBlockingConsumer<SourceRecord> bufferedRecordQueue = BufferedBlockingConsumer.bufferLast(super::enqueueRecord);
// Dump all of the tables and generate source records ...
logger.info("Step 8: scanning contents of {} tables", tableIds.size());
logger.info("Step {}: scanning contents of {} tables while still in transaction", step, tableIds.size());
metrics.setTableCount(tableIds.size());
long startScan = clock.currentTimeInMillis();
@ -395,9 +419,10 @@ protected void execute() {
// Scan the rows in the table ...
long start = clock.currentTimeInMillis();
logger.info("Step 8: - scanning table '{}' ({} of {} tables)", tableId, ++counter, tableIds.size());
logger.info("Step {}: - scanning table '{}' ({} of {} tables)", step, tableId, ++counter, tableIds.size());
sql.set("SELECT * FROM " + quote(tableId));
try {
int stepNum = step;
mysql.query(sql.get(), statementFactory, rs -> {
long rowNum = 0;
try {
@ -418,21 +443,21 @@ protected void execute() {
}
if (rowNum % 10_000 == 0) {
long stop = clock.currentTimeInMillis();
logger.info("Step 8: - {} of {} rows scanned from table '{}' after {}",
rowNum, rowCountStr, tableId, Strings.duration(stop - start));
logger.info("Step {}: - {} of {} rows scanned from table '{}' after {}",
stepNum, rowNum, rowCountStr, tableId, Strings.duration(stop - start));
}
}
totalRowCount.addAndGet(rowNum);
if (isRunning()) {
long stop = clock.currentTimeInMillis();
logger.info("Step 8: - Completed scanning a total of {} rows from table '{}' after {}",
rowNum, tableId, Strings.duration(stop - start));
logger.info("Step {}: - Completed scanning a total of {} rows from table '{}' after {}",
stepNum, rowNum, tableId, Strings.duration(stop - start));
}
} catch (InterruptedException e) {
Thread.interrupted();
// We were not able to finish all rows in all tables ...
logger.info("Step 8: Stopping the snapshot due to thread interruption");
logger.info("Step {}: Stopping the snapshot due to thread interruption", stepNum);
interrupted.set(true);
}
});
@ -454,41 +479,27 @@ protected void execute() {
long stop = clock.currentTimeInMillis();
try {
bufferedRecordQueue.flush(this::replaceOffset);
logger.info("Step 8: scanned {} rows in {} tables in {}",
totalRowCount, tableIds.size(), Strings.duration(stop - startScan));
logger.info("Step {}: scanned {} rows in {} tables in {}",
step, totalRowCount, tableIds.size(), Strings.duration(stop - startScan));
} catch (InterruptedException e) {
Thread.interrupted();
// We were not able to finish all rows in all tables ...
logger.info("Step 8: aborting the snapshot after {} rows in {} of {} tables {}",
totalRowCount, completedCounter, tableIds.size(), Strings.duration(stop - startScan));
logger.info("Step {}: aborting the snapshot after {} rows in {} of {} tables {}",
step, totalRowCount, completedCounter, tableIds.size(), Strings.duration(stop - startScan));
interrupted.set(true);
}
} else {
// source.markLastSnapshot(); Think we will not be needing this here it is used to mark last row entry?
logger.info("Step 8: encountered only schema based snapshot, skipping data snapshot");
logger.info("Step {}: encountered only schema based snapshot, skipping data snapshot", step);
}
step++;
} finally {
// No matter what, we always want to do these steps if necessary ...
// ------
// STEP 9
// ------
// Release the read lock if we have not yet done so. Locks are not released when committing/rolling back ...
int step = 9;
if (isLocked) {
logger.info("Step {}: releasing global read lock to enable MySQL writes", step++);
sql.set("UNLOCK TABLES");
mysql.execute(sql.get());
isLocked = false;
long lockReleased = clock.currentTimeInMillis();
metrics.globalLockReleased();
logger.info("Writes to MySQL prevented for a total of {}", Strings.duration(lockReleased - lockAcquired));
}
// -------
// STEP 10
// -------
// Either commit or roll back the transaction ...
// Either commit or roll back the transaction, BEFORE releasing the locks ...
if (isTxnStarted) {
if (interrupted.get() || !isRunning()) {
// We were interrupted or were stopped while reading the tables,
@ -504,7 +515,29 @@ protected void execute() {
sql.set("COMMIT");
mysql.execute(sql.get());
metrics.completeSnapshot();
} else {}
}
// -------
// STEP 10
// -------
// Release the read lock(s) if we have not yet done so. Locks are not released when committing/rolling back ...
if (isLocked) {
if (tableLocks) {
logger.info("Step {}: releasing table read locks to enable MySQL writes", step++);
} else {
logger.info("Step {}: releasing global read lock to enable MySQL writes", step++);
}
sql.set("UNLOCK TABLES");
mysql.execute(sql.get());
isLocked = false;
long lockReleased = clock.currentTimeInMillis();
metrics.globalLockReleased();
if (tableLocks) {
logger.info("Writes to MySQL prevented for a total of {}", Strings.duration(lockReleased - lockAcquired));
} else {
logger.info("Writes to MySQL tables prevented for a total of {}", Strings.duration(lockReleased - lockAcquired));
}
}
}
if (!isRunning()) {
@ -536,6 +569,32 @@ protected void execute() {
}
}
protected void readBinlogPosition(int step, SourceInfo source, JdbcConnection mysql, AtomicReference<String> sql) throws SQLException {
logger.info("Step {}: read binlog position of MySQL master", step);
String showMasterStmt = "SHOW MASTER STATUS";
sql.set(showMasterStmt);
mysql.query(sql.get(), rs -> {
if (rs.next()) {
String binlogFilename = rs.getString(1);
long binlogPosition = rs.getLong(2);
source.setBinlogStartPoint(binlogFilename, binlogPosition);
if (rs.getMetaData().getColumnCount() > 4) {
// This column exists only in MySQL 5.6.5 or later ...
String gtidSet = rs.getString(5);// GTID set, may be null, blank, or contain a GTID set
source.setCompletedGtidSet(gtidSet);
logger.info("\t using binlog '{}' at position '{}' and gtid '{}'", binlogFilename, binlogPosition,
gtidSet);
} else {
logger.info("\t using binlog '{}' at position '{}'", binlogFilename, binlogPosition);
}
source.startSnapshot();
} else {
throw new IllegalStateException("Cannot read the binlog filename and position via '" + showMasterStmt
+ "'. Make sure your server is correctly configured");
}
});
}
protected String quote(String dbOrTableName) {
return "`" + dbOrTableName + "`";
}