Commit 5bbda971 authored by Libing Song's avatar Libing Song Committed by Marko Mäkelä

MDEV-33853 Async rollback prepared transactions during binlog

           crash recovery

Summary
=======
When doing server recovery, the active transactions will be rolled
back by InnoDB background rollback thread automatically. The
prepared transactions will be committed or rolled back accordingly
by binlog recovery. Binlog recovery is done in main thread before
the server can provide service to users. If there is a big
transaction to rollback, the server will not available for a long
time.

This patch provides a way to rollback the prepared transactions
asynchronously. Thus the rollback will not block server startup.

Design
======
- Handler::recover_rollback_by_xid()
  This patch provides a new handler interface to rollback transactions
  in recover phase. InnoDB just set the transaction's state to active.
  Then the transaction will be rolled back by the background rollback
  thread.

- Handler::signal_tc_log_recover_done()
  This function is called after tc log is opened(typically binlog opened)
  has done. When this function is called, all transactions will be rolled
  back have been reverted to ACTIVE state. Thus it starts rollback thread
  to rollback the transactions.

- Background rollback thread
  With this patch, background rollback thread is defered to run until binlog
  recovery is finished. It is started by innobase_tc_log_recovery_done().
parent db5d1cde
CREATE TABLE t1 (id int primary key, data int) ENGINE = InnoDB;
INSERT INTO t1 VALUES (0, 1);
#
# 1. Check DML in prepared state can rollback correctly.
#
connect con1, localhost, root,,;
SET debug_sync = "ha_commit_trans_after_prepare SIGNAL prepared1 WAIT_FOR continue";
INSERT INTO t1 VALUES(1, 1);;
connect con2, localhost, root,,;
SET debug_sync = "now WAIT_FOR prepared1";
SET debug_sync = "ha_commit_trans_after_prepare SIGNAL prepared2 WAIT_FOR continue";
UPDATE t1 SET data = data + 1 WHERE id = 0;
connection default;
SET debug_sync = "now WAIT_FOR prepared2";
# Kill the server
disconnect con1;
disconnect con2;
# restart
# Expect (0, 1)
SET SESSION TRANSACTION ISOLATION LEVEL READ UNCOMMITTED;
SELECT * FROM t1;
id data
0 1
INSERT INTO t1 VALUES(1, 1);
UPDATE t1 SET data = data + 1 WHERE id = 0;
# Expect (0, 2), (1, 1)
SELECT * FROM t1;
id data
0 2
1 1
#
# 2. Test that innodb shutdown as expected if any error happens before
# normal rollback task is started. In the situation, rollback task
# should be started at preshutdown accordingly to rollback or
# deregister all recovered active transactions.
#
INSERT INTO t1 SELECT seq + 2, 1 FROM seq_1_to_1024;
BEGIN;
UPDATE t1 SET data = 10;
SET GLOBAL innodb_log_checkpoint_now = 1;
# Kill the server
# restart: --innodb-read-only
SELECT count(*) FROM information_schema.innodb_trx;
count(*)
1
# Kill the server
# restart: --innodb-read-only
SELECT count(*) FROM information_schema.innodb_trx;
count(*)
1
# restart
DROP TABLE t1;
################################################################################
# Async rollback prepared transactions during binlog crash recovery
#
# It verifies that binlog recovery just set the prepared transactions to
# active and the background recovery rollback thread will rollback the
# transactions asynchronously.
################################################################################
--source include/have_debug.inc
--source include/have_debug_sync.inc
--source include/have_innodb.inc
--source include/have_binlog_format_row.inc
--source include/have_sequence.inc
CREATE TABLE t1 (id int primary key, data int) ENGINE = InnoDB;
INSERT INTO t1 VALUES (0, 1);
--echo #
--echo # 1. Check DML in prepared state can rollback correctly.
--echo #
--connect(con1, localhost, root,,)
SET debug_sync = "ha_commit_trans_after_prepare SIGNAL prepared1 WAIT_FOR continue";
--send INSERT INTO t1 VALUES(1, 1);
--connect(con2, localhost, root,,)
SET debug_sync = "now WAIT_FOR prepared1";
SET debug_sync = "ha_commit_trans_after_prepare SIGNAL prepared2 WAIT_FOR continue";
--send UPDATE t1 SET data = data + 1 WHERE id = 0
--connection default
SET debug_sync = "now WAIT_FOR prepared2";
--source include/kill_mysqld.inc
--disconnect con1
--disconnect con2
# With the debug option, recovery rollback thread just rolls back the
# first prepared transaction and then goes to sleep.
--source include/start_mysqld.inc
--let $wait_condition= SELECT count(*) = 0 FROM information_schema.innodb_trx
--source include/wait_condition.inc
--echo # Expect (0, 1)
SET SESSION TRANSACTION ISOLATION LEVEL READ UNCOMMITTED;
SELECT * FROM t1;
# The previous INSERT is rolled back, so this INSERT will succeed.
INSERT INTO t1 VALUES(1, 1);
# The previous UPDATE is rolled back, so this UPDATE will succeed.
UPDATE t1 SET data = data + 1 WHERE id = 0;
--echo # Expect (0, 2), (1, 1)
SELECT * FROM t1;
--echo #
--echo # 2. Test that innodb shutdown as expected if any error happens before
--echo # normal rollback task is started. In the situation, rollback task
--echo # should be started at preshutdown accordingly to rollback or
--echo # deregister all recovered active transactions.
--echo #
# Generate an large active transaction
INSERT INTO t1 SELECT seq + 2, 1 FROM seq_1_to_1024;
BEGIN;
UPDATE t1 SET data = 10;
# Make sure above update is persisted.
SET GLOBAL innodb_log_checkpoint_now = 1;
--source include/kill_mysqld.inc
# tc-heuristic-recover triggers an error before innodb rollback task start
# Rollback task will not be started at preshutdown of read only mode. Active
# transactions are not expected to rollback.
--error 1
--exec $MYSQLD_LAST_CMD --tc-heuristic-recover=ROLLBACK --innodb-read-only --log-error=$MYSQLTEST_VARDIR/tmp/log.err
# Rollback task will not be started at preshutdown if recovery mode is greater
# to 2. Active transactions are not expected to rollback.
--error 1
--exec $MYSQLD_LAST_CMD --tc-heuristic-recover=ROLLBACK --innodb-force-recovery=3 --log-error=$MYSQLTEST_VARDIR/tmp/log.err
# Rollback task will be started at preshutdown of fast shutdown if force
# recovery is 2. But the transaction is deregistered instead of rollback.
--error 1
--exec $MYSQLD_LAST_CMD --tc-heuristic-recover=ROLLBACK --innodb-fast-shutdown=1 --innodb-force-recovery=2 --log-error=$MYSQLTEST_VARDIR/tmp/log.err
--let $restart_parameters= --innodb-read-only
--source include/start_mysqld.inc
# Verify that the transaction is still there.
SELECT count(*) FROM information_schema.innodb_trx;
--source include/kill_mysqld.inc
# Rollback task will be started at preshutdown of fast shutdown. The
# active transaction is rolled back.
--error 1
--exec $MYSQLD_LAST_CMD --tc-heuristic-recover=ROLLBACK --innodb-fast-shutdown=1 --log-error=$MYSQLTEST_VARDIR/tmp/log.err
--let $restart_parameters= --innodb-read-only
--source include/start_mysqld.inc
# Verify that the transaction is still there.
SELECT count(*) FROM information_schema.innodb_trx;
--remove_file $MYSQLTEST_VARDIR/tmp/log.err
--let $restart_parameters=
--source include/restart_mysqld.inc
# There should be no any transaction
--let $wait_condition= SELECT count(*) = 0 FROM information_schema.innodb_trx
--source include/wait_condition.inc
# Cleanup.
DROP TABLE t1;
......@@ -2682,8 +2682,12 @@ static void xarecover_do_commit_or_rollback(handlerton *hton,
else
x= *member->full_xid;
rc= xarecover_decide_to_commit(member, ptr_commit_max) ?
hton->commit_by_xid(hton, &x) : hton->rollback_by_xid(hton, &x);
if (xarecover_decide_to_commit(member, ptr_commit_max))
rc= hton->commit_by_xid(hton, &x);
else if (hton->recover_rollback_by_xid)
rc= hton->recover_rollback_by_xid(&x);
else
rc= hton->rollback_by_xid(hton, &x);
/*
It's fine to have non-zero rc which would be from transaction
......@@ -2750,6 +2754,21 @@ static my_bool xarecover_complete_and_count(void *member_arg,
return false;
}
static my_bool tc_log_recover_done_handlerton(THD*, plugin_ref plugin, void *arg)
{
handlerton *hton= plugin_hton(plugin);
if (hton->signal_tc_log_recovery_done)
hton->signal_tc_log_recovery_done();
return false;
}
void ha_signal_tc_log_recovery_done()
{
std::ignore = plugin_foreach(nullptr, tc_log_recover_done_handlerton,
MYSQL_STORAGE_ENGINE_PLUGIN, 0);
}
/*
Completes binlog recovery to invoke decider functions for
each xid.
......
......@@ -1427,6 +1427,23 @@ struct handlerton
int (*recover)(handlerton *hton, XID *xid_list, uint len);
int (*commit_by_xid)(handlerton *hton, XID *xid);
int (*rollback_by_xid)(handlerton *hton, XID *xid);
/*
recover_rollback_by_xid is optional. If set, it will be called instead of
rollback_by_xid when transactions should be rolled back at server startup.
This function should just change the transaction's state from prepared to
active before returing. The actual rollback should then happen
asynchroneously (eg. in a background thread). This way, rollbacks that
take a long time to complete will not block server startup, and the
database becomes available sooner to serve user queries.
*/
int (*recover_rollback_by_xid)(const XID *xid);
/*
It is called after binlog recovery has done commit/rollback of
all transactions. It is used together with recover_rollback_by_xid()
together to rollback prepared transactions asynchronously.
*/
void (*signal_tc_log_recovery_done)();
/*
The commit_checkpoint_request() handlerton method is used to checkpoint
the XA recovery process for storage engines that support two-phase
......@@ -5622,6 +5639,7 @@ int ha_panic(enum ha_panic_function flag);
void ha_close_connection(THD* thd);
void ha_kill_query(THD* thd, enum thd_kill_levels level);
void ha_signal_ddl_recovery_done();
void ha_signal_tc_log_recovery_done();
bool ha_flush_logs();
void ha_drop_database(const char* path);
void ha_checkpoint_state(bool disable);
......
......@@ -5602,6 +5602,8 @@ static int init_server_components()
if (ha_recover(0))
unireg_abort(1);
ha_signal_tc_log_recovery_done();
if (opt_bin_log)
{
int error;
......
......@@ -4108,6 +4108,11 @@ static int innodb_init(void* p)
innobase_hton->recover = innobase_xa_recover;
innobase_hton->commit_by_xid = innobase_commit_by_xid;
innobase_hton->rollback_by_xid = innobase_rollback_by_xid;
#ifndef EMBEDDED_LIBRARY
innobase_hton->recover_rollback_by_xid =
innobase_recover_rollback_by_xid;
innobase_hton->signal_tc_log_recovery_done = innobase_tc_log_recovery_done;
#endif
innobase_hton->commit_checkpoint_request = innodb_log_flush_request;
innobase_hton->create = innobase_create_handler;
......@@ -17171,6 +17176,80 @@ int innobase_rollback_by_xid(handlerton* hton, XID* xid)
}
}
#ifndef EMBEDDED_LIBRARY
/**
This function is used to rollback one X/Open XA distributed transaction
which is in the prepared state asynchronously.
It only set the transaction's status to ACTIVE and persist the status.
The transaction will be rolled back by background rollback thread.
@param xid X/Open XA transaction identification
@return 0 or error number
*/
int innobase_recover_rollback_by_xid(const XID *xid)
{
DBUG_EXECUTE_IF("innobase_xa_fail", return XAER_RMFAIL;);
if (high_level_read_only)
return XAER_RMFAIL;
/*
trx_get_trx_by_xid() sets trx's xid to null. Thus only one call for any
given XID can find the transaction. Subsequent calls by other threads
would return nullptr. That is what guarantees that no other thread can be
modifying the state of the transaction at this point.
*/
trx_t *trx= trx_get_trx_by_xid(xid);
if (!trx)
return XAER_RMFAIL;
// ddl should not be rolled back through recovery
ut_ad(!trx->dict_operation);
ut_ad(trx->is_recovered);
ut_ad(trx->state == TRX_STATE_PREPARED);
#ifdef WITH_WSREP
ut_ad(!wsrep_is_wsrep_xid(&trx->xid));
#endif
if (trx->rsegs.m_redo.undo)
{
ut_ad(trx->rsegs.m_redo.undo->rseg == trx->rsegs.m_redo.rseg);
mtr_t mtr;
mtr.start();
trx_undo_set_state_at_prepare(trx, trx->rsegs.m_redo.undo, true, &mtr);
mtr.commit();
ut_ad(mtr.commit_lsn() > 0);
}
/* The above state change from XA PREPARE will be made durable in
innobase_tc_log_recovery_done(), which will also initiate
trx_rollback_recovered() to roll back this transaction. */
trx->state= TRX_STATE_ACTIVE;
return 0;
}
void innobase_tc_log_recovery_done()
{
if (high_level_read_only)
return;
/* Make durable any innobase_recover_rollback_by_xid(). */
log_buffer_flush_to_disk(true);
if (srv_force_recovery < SRV_FORCE_NO_TRX_UNDO)
{
/* Rollback incomplete non-DDL transactions */
trx_rollback_is_active= true;
srv_thread_pool->submit_task(&rollback_all_recovered_task);
}
}
#endif // EMBEDDED_LIBRARY
bool
ha_innobase::check_if_incompatible_data(
/*====================================*/
......
......@@ -943,3 +943,24 @@ which is in the prepared state
@return 0 or error number */
int innobase_rollback_by_xid(handlerton* hton, XID* xid);
/**
This function is used to rollback one X/Open XA distributed transaction
which is in the prepared state asynchronously.
It only set the transaction's status to ACTIVE and persist the status.
The transaction will be rolled back by background rollback thread.
@param xid X/Open XA transaction identification
@return 0 or error number
*/
int innobase_recover_rollback_by_xid(const XID *xid);
/**
This function is called after tc log is opened(typically binlog recovery)
has done. It starts rollback thread to rollback the transactions
have been changed from PREPARED to ACTIVE.
@return 0 or error number
*/
void innobase_tc_log_recovery_done();
......@@ -31,7 +31,8 @@ Created 3/26/1996 Heikki Tuuri
#include "mtr0mtr.h"
#include "trx0sys.h"
extern bool trx_rollback_is_active;
extern tpool::task_group rollback_all_recovered_group;
extern tpool::waitable_task rollback_all_recovered_task;
extern const trx_t* trx_roll_crash_recv_trx;
/** Report progress when rolling back a row of a recovered transaction. */
......
......@@ -42,6 +42,8 @@ Created 3/26/1996 Heikki Tuuri
extern mysql_pfs_key_t trx_sys_mutex_key;
#endif
extern bool trx_rollback_is_active;
/** Checks if a page address is the trx sys header page.
@param[in] page_id page id
@return true if trx sys header page */
......@@ -822,6 +824,21 @@ class thread_safe_trx_ilist_t
mysql_mutex_unlock(&mutex);
}
template <typename Callable> bool find_first(Callable &&callback) const
{
mysql_mutex_lock(&mutex);
for (trx_t &trx : trx_list)
{
if (callback(trx))
{
mysql_mutex_unlock(&mutex);
return true;
}
}
mysql_mutex_unlock(&mutex);
return false;
}
template <typename Callable> void for_each(Callable &&callback) const
{
mysql_mutex_lock(&mutex);
......@@ -1072,6 +1089,10 @@ class trx_sys_t
/** @return total number of active (non-prepared) transactions */
size_t any_active_transactions(size_t *prepared= nullptr);
#ifndef EMBEDDED_LIBRARY
/** @return true if any active (non-prepared) transactions is recovered */
bool any_active_transaction_recovered();
#endif
/**
Determine the rollback segment identifier.
......
......@@ -715,10 +715,10 @@ struct trx_t : ilist_node<>
XA (2PC):
* NOT_STARTED -> ACTIVE -> PREPARED -> COMMITTED -> NOT_STARTED
Recovered XA:
Recovered XA(2PC) followed by XA COMMIT :
* NOT_STARTED -> PREPARED -> COMMITTED -> (freed)
Recovered XA followed by XA ROLLBACK:
Recovered XA followed by XA ROLLBACK or recover_rollback_by_xid:
* NOT_STARTED -> PREPARED -> ACTIVE -> COMMITTED -> (freed)
XA (2PC) (shutdown or disconnect before ROLLBACK or COMMIT):
......
......@@ -1181,9 +1181,7 @@ ATTRIBUTE_COLD void logs_empty_and_mark_files_at_shutdown()
}
/* We need these threads to stop early in shutdown. */
const char* thread_name = srv_fast_shutdown != 2
&& trx_rollback_is_active
? "rollback of recovered transactions" : nullptr;
const char* thread_name= nullptr;
if (thread_name) {
ut_ad(!srv_read_only_mode);
......@@ -1215,6 +1213,7 @@ ATTRIBUTE_COLD void logs_empty_and_mark_files_at_shutdown()
}
buf_load_dump_end();
rollback_all_recovered_task.wait();
if (!buf_pool.is_initialised()) {
ut_ad(!srv_was_started);
......
......@@ -1180,11 +1180,6 @@ ATTRIBUTE_COLD static dberr_t ibuf_log_rebuild_if_needed()
return err;
}
static tpool::task_group rollback_all_recovered_group(1);
static tpool::task rollback_all_recovered_task(trx_rollback_all_recovered,
nullptr,
&rollback_all_recovered_group);
/** Start InnoDB.
@param[in] create_new_db whether to create a new database
@return DB_SUCCESS or error code */
......@@ -1862,9 +1857,11 @@ dberr_t srv_start(bool create_new_db)
&& !srv_read_only_mode) {
/* Drop partially created indexes. */
row_merge_drop_temp_indexes();
#ifdef EMBEDDED_LIBRARY
/* Rollback incomplete non-DDL transactions */
trx_rollback_is_active = true;
srv_thread_pool->submit_task(&rollback_all_recovered_task);
#endif
}
}
......@@ -1987,6 +1984,20 @@ void innodb_preshutdown()
if (srv_read_only_mode)
return;
#ifndef EMBEDDED_LIBRARY
if (srv_force_recovery < SRV_FORCE_NO_TRX_UNDO && srv_was_started &&
!trx_rollback_is_active && trx_sys.any_active_transaction_recovered())
{
/* Trx rollback thread may not be started yet, since error happened before
innobase_tc_log_recovery_done(), thus it is started here to finish the
normal shutdown process.
*/
trx_rollback_is_active= true;
srv_thread_pool->submit_task(&rollback_all_recovered_task);
}
#endif
if (!srv_fast_shutdown && srv_operation <= SRV_OPERATION_EXPORT_RESTORED)
if (srv_force_recovery < SRV_FORCE_NO_TRX_UNDO && srv_was_started)
while (trx_sys.any_active_transactions())
......
......@@ -48,6 +48,11 @@ Created 3/26/1996 Heikki Tuuri
mysql_pfs_key_t trx_rollback_clean_thread_key;
#endif
tpool::task_group rollback_all_recovered_group(1);
tpool::waitable_task rollback_all_recovered_task(trx_rollback_all_recovered,
nullptr,
&rollback_all_recovered_group);
/** true if trx_rollback_all_recovered() thread is active */
bool trx_rollback_is_active;
......
......@@ -362,3 +362,21 @@ size_t trx_sys_t::any_active_transactions(size_t *prepared)
return total_trx;
}
#ifndef EMBEDDED_LIBRARY
/** @return true if any active (non-prepared) transactions is recovered */
bool trx_sys_t::any_active_transaction_recovered()
{
return trx_sys.trx_list.find_first([&](trx_t &trx)
{
if (trx.state != TRX_STATE_ACTIVE)
return false;
bool found= false;
trx.mutex_lock();
found= trx.is_recovered;
trx.mutex_unlock();
return found;
});
}
#endif
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment