mirror of
https://github.com/MariaDB/server.git
synced 2025-02-05 13:22:17 +01:00
622 lines
17 KiB
C++
622 lines
17 KiB
C++
/*****************************************************************************
|
|
|
|
Copyright (c) 1996, 2017, Oracle and/or its affiliates. All Rights Reserved.
|
|
Copyright (c) 2016, 2022, MariaDB Corporation.
|
|
|
|
This program is free software; you can redistribute it and/or modify it under
|
|
the terms of the GNU General Public License as published by the Free Software
|
|
Foundation; version 2 of the License.
|
|
|
|
This program is distributed in the hope that it will be useful, but WITHOUT
|
|
ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or FITNESS
|
|
FOR A PARTICULAR PURPOSE. See the GNU General Public License for more details.
|
|
|
|
You should have received a copy of the GNU General Public License along with
|
|
this program; if not, write to the Free Software Foundation, Inc.,
|
|
51 Franklin Street, Fifth Floor, Boston, MA 02110-1335 USA
|
|
|
|
*****************************************************************************/
|
|
|
|
/**************************************************//**
|
|
@file trx/trx0roll.cc
|
|
Transaction rollback
|
|
|
|
Created 3/26/1996 Heikki Tuuri
|
|
*******************************************************/
|
|
|
|
#include "trx0roll.h"
|
|
|
|
#include <my_service_manager.h>
|
|
#include <mysql/service_wsrep.h>
|
|
|
|
#include "fsp0fsp.h"
|
|
#include "lock0lock.h"
|
|
#include "mach0data.h"
|
|
#include "pars0pars.h"
|
|
#include "que0que.h"
|
|
#include "row0mysql.h"
|
|
#include "row0undo.h"
|
|
#include "srv0mon.h"
|
|
#include "srv0start.h"
|
|
#include "trx0rec.h"
|
|
#include "trx0rseg.h"
|
|
#include "trx0sys.h"
|
|
#include "trx0trx.h"
|
|
#include "trx0undo.h"
|
|
#include "log.h"
|
|
|
|
#ifdef UNIV_PFS_THREAD
|
|
mysql_pfs_key_t trx_rollback_clean_thread_key;
|
|
#endif
|
|
|
|
/** true if trx_rollback_all_recovered() thread is active */
|
|
bool trx_rollback_is_active;
|
|
|
|
/** In crash recovery, the current trx to be rolled back; NULL otherwise */
|
|
const trx_t* trx_roll_crash_recv_trx;
|
|
|
|
bool trx_t::rollback_finish() noexcept
|
|
{
|
|
apply_online_log= false;
|
|
if (UNIV_LIKELY(error_state == DB_SUCCESS))
|
|
{
|
|
commit();
|
|
return true;
|
|
}
|
|
|
|
ut_a(error_state == DB_INTERRUPTED);
|
|
ut_ad(srv_shutdown_state != SRV_SHUTDOWN_NONE);
|
|
ut_a(!srv_undo_sources);
|
|
ut_ad(srv_fast_shutdown);
|
|
ut_d(in_rollback= false);
|
|
if (trx_undo_t *&undo= rsegs.m_redo.undo)
|
|
{
|
|
UT_LIST_REMOVE(rsegs.m_redo.rseg->undo_list, undo);
|
|
ut_free(undo);
|
|
undo= nullptr;
|
|
}
|
|
if (trx_undo_t *&undo= rsegs.m_noredo.undo)
|
|
{
|
|
UT_LIST_REMOVE(rsegs.m_noredo.rseg->undo_list, undo);
|
|
ut_free(undo);
|
|
undo= nullptr;
|
|
}
|
|
commit_low();
|
|
return commit_cleanup();
|
|
}
|
|
|
|
dberr_t trx_t::rollback_low(const undo_no_t *savept) noexcept
|
|
{
|
|
op_info= "rollback";
|
|
mem_heap_t *heap= mem_heap_create(512);
|
|
roll_node_t *roll_node= roll_node_create(heap);
|
|
|
|
roll_node->savept= savept ? *savept : 0;
|
|
|
|
ut_ad(!in_rollback);
|
|
#ifdef UNIV_DEBUG
|
|
if (savept)
|
|
{
|
|
ut_ad(state == TRX_STATE_ACTIVE);
|
|
ut_ad(mysql_thd);
|
|
ut_ad(!is_recovered);
|
|
}
|
|
#endif
|
|
|
|
error_state= DB_SUCCESS;
|
|
|
|
if (has_logged())
|
|
{
|
|
ut_ad(rsegs.m_redo.rseg || rsegs.m_noredo.rseg);
|
|
que_thr_t *thr= pars_complete_graph_for_exec(roll_node, this, heap,
|
|
nullptr);
|
|
ut_a(thr == que_fork_start_command(static_cast<que_fork_t*>
|
|
(que_node_get_parent(thr))));
|
|
que_run_threads(thr);
|
|
que_run_threads(roll_node->undo_thr);
|
|
|
|
/* Free the memory reserved by the undo graph. */
|
|
que_graph_free(static_cast<que_t*>(roll_node->undo_thr->common.parent));
|
|
}
|
|
|
|
if (!savept)
|
|
{
|
|
rollback_finish();
|
|
MONITOR_INC(MONITOR_TRX_ROLLBACK);
|
|
}
|
|
else
|
|
{
|
|
/* There must not be partial rollback if transaction was chosen as deadlock
|
|
victim. Galera transaction abort can be invoked during partial rollback. */
|
|
ut_ad(!(lock.was_chosen_as_deadlock_victim & 1));
|
|
ut_a(error_state == DB_SUCCESS);
|
|
const undo_no_t limit{*savept};
|
|
apply_online_log= false;
|
|
for (trx_mod_tables_t::iterator i= mod_tables.begin();
|
|
i != mod_tables.end(); )
|
|
{
|
|
trx_mod_tables_t::iterator j= i++;
|
|
ut_ad(j->second.valid());
|
|
if (j->second.rollback(limit))
|
|
{
|
|
j->second.clear_bulk_buffer();
|
|
mod_tables.erase(j);
|
|
}
|
|
else if (!apply_online_log)
|
|
apply_online_log= j->first->is_native_online_ddl();
|
|
}
|
|
MONITOR_INC(MONITOR_TRX_ROLLBACK_SAVEPOINT);
|
|
}
|
|
|
|
mem_heap_free(heap);
|
|
op_info= "";
|
|
return error_state;
|
|
}
|
|
|
|
dberr_t trx_t::rollback(const undo_no_t *savept) noexcept
|
|
{
|
|
ut_ad(!mutex_is_owner());
|
|
switch (state) {
|
|
case TRX_STATE_ABORTED:
|
|
ut_ad(!savept);
|
|
state= TRX_STATE_NOT_STARTED;
|
|
/* fall through */
|
|
case TRX_STATE_NOT_STARTED:
|
|
error_state= DB_SUCCESS;
|
|
return DB_SUCCESS;
|
|
case TRX_STATE_PREPARED:
|
|
case TRX_STATE_PREPARED_RECOVERED:
|
|
case TRX_STATE_COMMITTED_IN_MEMORY:
|
|
ut_ad("invalid state" == 0);
|
|
/* fall through */
|
|
case TRX_STATE_ACTIVE:
|
|
break;
|
|
}
|
|
#ifdef WITH_WSREP
|
|
if (!savept && is_wsrep() && wsrep_thd_is_SR(mysql_thd))
|
|
wsrep_handle_SR_rollback(nullptr, mysql_thd);
|
|
#endif /* WITH_WSREP */
|
|
return rollback_low(savept);
|
|
}
|
|
|
|
/** Rollback a transaction used in MySQL
|
|
@param[in, out] trx transaction
|
|
@return error code or DB_SUCCESS */
|
|
dberr_t trx_rollback_for_mysql(trx_t* trx)
|
|
{
|
|
/* We are reading trx->state without holding trx->mutex
|
|
here, because the rollback should be invoked for a running
|
|
active MySQL transaction (or recovered prepared transaction)
|
|
that is associated with the current thread. */
|
|
|
|
switch (trx->state) {
|
|
case TRX_STATE_ABORTED:
|
|
trx->state = TRX_STATE_NOT_STARTED;
|
|
/* fall through */
|
|
case TRX_STATE_NOT_STARTED:
|
|
trx->will_lock = false;
|
|
ut_ad(trx->mysql_thd);
|
|
/* Galera transaction abort can be invoked from MDL acquision
|
|
code, so trx->lock.was_chosen_as_deadlock_victim can be set
|
|
even if trx->state is TRX_STATE_NOT_STARTED. */
|
|
ut_ad(!(trx->lock.was_chosen_as_deadlock_victim & 1));
|
|
#ifdef WITH_WSREP
|
|
ut_ad(!trx->is_wsrep());
|
|
trx->lock.was_chosen_as_deadlock_victim= false;
|
|
#endif
|
|
ut_a(UT_LIST_GET_LEN(trx->lock.trx_locks) == 0);
|
|
return(DB_SUCCESS);
|
|
case TRX_STATE_ACTIVE:
|
|
ut_ad(trx->mysql_thd);
|
|
ut_ad(!trx->is_recovered);
|
|
ut_ad(!trx->is_autocommit_non_locking() || trx->read_only);
|
|
return trx->rollback_low();
|
|
|
|
case TRX_STATE_PREPARED:
|
|
case TRX_STATE_PREPARED_RECOVERED:
|
|
ut_ad(!trx->is_autocommit_non_locking());
|
|
if (trx->rsegs.m_redo.undo) {
|
|
/* The XA ROLLBACK of a XA PREPARE transaction
|
|
will consist of multiple mini-transactions.
|
|
|
|
As the very first step of XA ROLLBACK, we must
|
|
change the undo log state back from
|
|
TRX_UNDO_PREPARED to TRX_UNDO_ACTIVE, in order
|
|
to ensure that recovery will complete the
|
|
rollback.
|
|
|
|
Failure to perform this step could cause a
|
|
situation where we would roll back part of
|
|
a XA PREPARE transaction, the server would be
|
|
killed, and finally, the transaction would be
|
|
recovered in XA PREPARE state, with some of
|
|
the actions already having been rolled back. */
|
|
ut_ad(trx->rsegs.m_redo.undo->rseg
|
|
== trx->rsegs.m_redo.rseg);
|
|
mtr_t mtr;
|
|
mtr.start();
|
|
if (trx_undo_t* undo = trx->rsegs.m_redo.undo) {
|
|
trx_undo_set_state_at_prepare(trx, undo, true,
|
|
&mtr);
|
|
}
|
|
/* Write the redo log for the XA ROLLBACK
|
|
state change to the global buffer. It is
|
|
not necessary to flush the redo log. If
|
|
a durable log write of a later mini-transaction
|
|
takes place for whatever reason, then this state
|
|
change will be durable as well. */
|
|
mtr.commit();
|
|
ut_ad(mtr.commit_lsn() > 0);
|
|
}
|
|
return trx->rollback_low();
|
|
|
|
case TRX_STATE_COMMITTED_IN_MEMORY:
|
|
ut_ad(!trx->is_autocommit_non_locking());
|
|
break;
|
|
}
|
|
|
|
ut_error;
|
|
return(DB_CORRUPTION);
|
|
}
|
|
|
|
/*******************************************************************//**
|
|
Roll back an active transaction. */
|
|
static
|
|
void
|
|
trx_rollback_active(
|
|
/*================*/
|
|
trx_t* trx) /*!< in/out: transaction */
|
|
{
|
|
mem_heap_t* heap;
|
|
que_fork_t* fork;
|
|
que_thr_t* thr;
|
|
roll_node_t* roll_node;
|
|
const trx_id_t trx_id = trx->id;
|
|
|
|
ut_ad(trx_id);
|
|
|
|
heap = mem_heap_create(512);
|
|
|
|
fork = que_fork_create(heap);
|
|
fork->trx = trx;
|
|
|
|
thr = que_thr_create(fork, heap, NULL);
|
|
|
|
roll_node = roll_node_create(heap);
|
|
|
|
thr->child = roll_node;
|
|
roll_node->common.parent = thr;
|
|
|
|
trx->graph = fork;
|
|
|
|
ut_a(thr == que_fork_start_command(fork));
|
|
|
|
trx_roll_crash_recv_trx = trx;
|
|
|
|
const bool dictionary_locked = trx->dict_operation;
|
|
|
|
if (dictionary_locked) {
|
|
row_mysql_lock_data_dictionary(trx);
|
|
}
|
|
|
|
que_run_threads(thr);
|
|
ut_a(roll_node->undo_thr != NULL);
|
|
|
|
que_run_threads(roll_node->undo_thr);
|
|
|
|
que_graph_free(
|
|
static_cast<que_t*>(roll_node->undo_thr->common.parent));
|
|
|
|
if (UNIV_UNLIKELY(!trx->rollback_finish())) {
|
|
ut_ad(!dictionary_locked);
|
|
} else {
|
|
sql_print_information(
|
|
"InnoDB: Rolled back recovered transaction "
|
|
TRX_ID_FMT, trx_id);
|
|
}
|
|
|
|
if (dictionary_locked) {
|
|
row_mysql_unlock_data_dictionary(trx);
|
|
}
|
|
|
|
mem_heap_free(heap);
|
|
|
|
trx_roll_crash_recv_trx = NULL;
|
|
}
|
|
|
|
|
|
struct trx_roll_count_callback_arg
|
|
{
|
|
uint32_t n_trx;
|
|
uint64_t n_rows;
|
|
trx_roll_count_callback_arg(): n_trx(0), n_rows(0) {}
|
|
};
|
|
|
|
|
|
static my_bool trx_roll_count_callback(rw_trx_hash_element_t *element,
|
|
trx_roll_count_callback_arg *arg)
|
|
{
|
|
element->mutex.wr_lock();
|
|
if (trx_t *trx= element->trx)
|
|
{
|
|
if (trx->is_recovered && trx_state_eq(trx, TRX_STATE_ACTIVE))
|
|
{
|
|
arg->n_trx++;
|
|
arg->n_rows+= trx->undo_no;
|
|
}
|
|
}
|
|
element->mutex.wr_unlock();
|
|
return 0;
|
|
}
|
|
|
|
/** Report progress when rolling back a row of a recovered transaction. */
|
|
void trx_roll_report_progress()
|
|
{
|
|
time_t now = time(NULL);
|
|
mysql_mutex_lock(&recv_sys.mutex);
|
|
bool report = recv_sys.report(now);
|
|
mysql_mutex_unlock(&recv_sys.mutex);
|
|
|
|
if (report) {
|
|
trx_roll_count_callback_arg arg;
|
|
|
|
/* Get number of recovered active transactions and number of
|
|
rows they modified. Numbers must be accurate, because only this
|
|
thread is allowed to touch recovered transactions. */
|
|
trx_sys.rw_trx_hash.iterate_no_dups(
|
|
trx_roll_count_callback, &arg);
|
|
|
|
if (arg.n_rows > 0) {
|
|
service_manager_extend_timeout(
|
|
INNODB_EXTEND_TIMEOUT_INTERVAL,
|
|
"To roll back: " UINT32PF " transactions, "
|
|
UINT64PF " rows", arg.n_trx, arg.n_rows);
|
|
}
|
|
|
|
ib::info() << "To roll back: " << arg.n_trx
|
|
<< " transactions, " << arg.n_rows << " rows";
|
|
|
|
}
|
|
}
|
|
|
|
|
|
static my_bool trx_rollback_recovered_callback(rw_trx_hash_element_t *element,
|
|
std::vector<trx_t*> *trx_list)
|
|
{
|
|
element->mutex.wr_lock();
|
|
if (trx_t *trx= element->trx)
|
|
{
|
|
trx->mutex_lock();
|
|
if (trx_state_eq(trx, TRX_STATE_ACTIVE) && trx->is_recovered)
|
|
trx_list->push_back(trx);
|
|
trx->mutex_unlock();
|
|
}
|
|
element->mutex.wr_unlock();
|
|
return 0;
|
|
}
|
|
|
|
/**
|
|
Rollback any incomplete transactions which were encountered in crash recovery.
|
|
|
|
If the transaction already was committed, then we clean up a possible insert
|
|
undo log. If the transaction was not yet committed, then we roll it back.
|
|
|
|
Note: For XA recovered transactions, we rely on MySQL to
|
|
do rollback. They will be in TRX_STATE_PREPARED state. If the server
|
|
is shutdown and they are still lingering in trx_sys_t::trx_list
|
|
then the shutdown will hang.
|
|
|
|
@param[in] all true=roll back all recovered active transactions;
|
|
false=roll back any incomplete dictionary transaction
|
|
*/
|
|
|
|
void trx_rollback_recovered(bool all)
|
|
{
|
|
std::vector<trx_t*> trx_list;
|
|
|
|
ut_a(srv_force_recovery <
|
|
ulong(all ? SRV_FORCE_NO_TRX_UNDO : SRV_FORCE_NO_DDL_UNDO));
|
|
|
|
/*
|
|
Collect list of recovered ACTIVE transaction ids first. Once collected, no
|
|
other thread is allowed to modify or remove these transactions from
|
|
rw_trx_hash.
|
|
*/
|
|
trx_sys.rw_trx_hash.iterate_no_dups(trx_rollback_recovered_callback,
|
|
&trx_list);
|
|
|
|
while (!trx_list.empty())
|
|
{
|
|
trx_t *trx= trx_list.back();
|
|
trx_list.pop_back();
|
|
|
|
ut_ad(trx);
|
|
ut_d(trx->mutex_lock());
|
|
ut_ad(trx->is_recovered);
|
|
ut_ad(trx_state_eq(trx, TRX_STATE_ACTIVE));
|
|
ut_d(trx->mutex_unlock());
|
|
|
|
if (srv_shutdown_state != SRV_SHUTDOWN_NONE && !srv_undo_sources &&
|
|
srv_fast_shutdown)
|
|
goto discard;
|
|
|
|
if (all || trx->dict_operation || trx->has_stats_table_lock())
|
|
{
|
|
trx_rollback_active(trx);
|
|
if (trx->error_state != DB_SUCCESS)
|
|
{
|
|
ut_ad(trx->error_state == DB_INTERRUPTED);
|
|
trx->error_state= DB_SUCCESS;
|
|
ut_ad(!srv_undo_sources);
|
|
ut_ad(srv_fast_shutdown);
|
|
discard:
|
|
/* Note: before kill_server() invoked innobase_end() via
|
|
unireg_end(), it invoked close_connections(), which should initiate
|
|
the rollback of any user transactions via THD::cleanup() in the
|
|
connection threads, and wait for all THD::cleanup() to complete.
|
|
So, no active user transactions should exist at this point.
|
|
|
|
srv_undo_sources=false was cleared early in innobase_end().
|
|
|
|
Generally, the server guarantees that all connections using
|
|
InnoDB must be disconnected by the time we are reaching this code,
|
|
be it during shutdown or UNINSTALL PLUGIN.
|
|
|
|
Because there is no possible race condition with any
|
|
concurrent user transaction, we do not have to invoke
|
|
trx->commit_state() or wait for !trx->is_referenced()
|
|
before trx_sys.deregister_rw(trx). */
|
|
trx_sys.deregister_rw(trx);
|
|
trx_free_at_shutdown(trx);
|
|
}
|
|
else
|
|
trx->free();
|
|
}
|
|
}
|
|
}
|
|
|
|
/*******************************************************************//**
|
|
Rollback or clean up any incomplete transactions which were
|
|
encountered in crash recovery. If the transaction already was
|
|
committed, then we clean up a possible insert undo log. If the
|
|
transaction was not yet committed, then we roll it back.
|
|
Note: this is done in a background thread. */
|
|
void trx_rollback_all_recovered(void*)
|
|
{
|
|
ut_ad(!srv_read_only_mode);
|
|
|
|
if (trx_sys.rw_trx_hash.size()) {
|
|
ib::info() << "Starting in background the rollback of"
|
|
" recovered transactions";
|
|
trx_rollback_recovered(true);
|
|
ib::info() << "Rollback of non-prepared transactions"
|
|
" completed";
|
|
}
|
|
|
|
trx_rollback_is_active = false;
|
|
}
|
|
|
|
/****************************************************************//**
|
|
Builds an undo 'query' graph for a transaction. The actual rollback is
|
|
performed by executing this query graph like a query subprocedure call.
|
|
The reply about the completion of the rollback will be sent by this
|
|
graph.
|
|
@return own: the query graph */
|
|
static
|
|
que_t*
|
|
trx_roll_graph_build(
|
|
/*=================*/
|
|
trx_t* trx) /*!< in/out: transaction */
|
|
{
|
|
mem_heap_t* heap;
|
|
que_fork_t* fork;
|
|
que_thr_t* thr;
|
|
|
|
ut_ad(trx->mutex_is_owner());
|
|
heap = mem_heap_create(512);
|
|
fork = que_fork_create(heap);
|
|
fork->trx = trx;
|
|
|
|
thr = que_thr_create(fork, heap, NULL);
|
|
|
|
thr->child = row_undo_node_create(trx, thr, heap);
|
|
|
|
return(fork);
|
|
}
|
|
|
|
/*********************************************************************//**
|
|
Starts a rollback operation, creates the UNDO graph that will do the
|
|
actual undo operation.
|
|
@return query graph thread that will perform the UNDO operations. */
|
|
static
|
|
que_thr_t*
|
|
trx_rollback_start(
|
|
/*===============*/
|
|
trx_t* trx, /*!< in: transaction */
|
|
undo_no_t roll_limit) /*!< in: rollback to undo no (for
|
|
partial undo), 0 if we are rolling back
|
|
the entire transaction */
|
|
{
|
|
/* Initialize the rollback field in the transaction */
|
|
|
|
ut_ad(trx->mutex_is_owner());
|
|
ut_ad(!trx->roll_limit);
|
|
ut_ad(!trx->in_rollback);
|
|
|
|
trx->roll_limit = roll_limit;
|
|
trx->in_rollback = true;
|
|
|
|
ut_a(trx->roll_limit <= trx->undo_no);
|
|
|
|
trx->pages_undone = 0;
|
|
|
|
/* Build a 'query' graph which will perform the undo operations */
|
|
|
|
que_t* roll_graph = trx_roll_graph_build(trx);
|
|
|
|
trx->graph = roll_graph;
|
|
|
|
return(que_fork_start_command(roll_graph));
|
|
}
|
|
|
|
/*********************************************************************//**
|
|
Creates a rollback command node struct.
|
|
@return own: rollback node struct */
|
|
roll_node_t*
|
|
roll_node_create(
|
|
/*=============*/
|
|
mem_heap_t* heap) /*!< in: mem heap where created */
|
|
{
|
|
roll_node_t* node;
|
|
|
|
node = static_cast<roll_node_t*>(mem_heap_zalloc(heap, sizeof(*node)));
|
|
|
|
node->state = ROLL_NODE_SEND;
|
|
|
|
node->common.type = QUE_NODE_ROLLBACK;
|
|
|
|
return(node);
|
|
}
|
|
|
|
/***********************************************************//**
|
|
Performs an execution step for a rollback command node in a query graph.
|
|
@return query thread to run next, or NULL */
|
|
que_thr_t*
|
|
trx_rollback_step(
|
|
/*==============*/
|
|
que_thr_t* thr) /*!< in: query thread */
|
|
{
|
|
roll_node_t* node;
|
|
|
|
node = static_cast<roll_node_t*>(thr->run_node);
|
|
|
|
ut_ad(que_node_get_type(node) == QUE_NODE_ROLLBACK);
|
|
|
|
if (thr->prev_node == que_node_get_parent(node)) {
|
|
node->state = ROLL_NODE_SEND;
|
|
}
|
|
|
|
if (node->state == ROLL_NODE_SEND) {
|
|
trx_t* trx;
|
|
|
|
trx = thr_get_trx(thr);
|
|
|
|
node->state = ROLL_NODE_WAIT;
|
|
|
|
ut_a(node->undo_thr == NULL);
|
|
|
|
trx->mutex_lock();
|
|
|
|
trx_commit_or_rollback_prepare(trx);
|
|
|
|
node->undo_thr = trx_rollback_start(trx, node->savept);
|
|
|
|
trx->mutex_unlock();
|
|
} else {
|
|
ut_ad(node->state == ROLL_NODE_WAIT);
|
|
|
|
thr->run_node = que_node_get_parent(node);
|
|
}
|
|
|
|
return(thr);
|
|
}
|