Commit cfbbf542 authored by Marko Mäkelä's avatar Marko Mäkelä

MDEV-7962: Follow-up fix for 10.4

Replace wsrep_on() with trx_t::is_wsrep() where possible.

Also, rename some functions to member functions and
remove unused DBUG_EXECUTE_IF instrumentation:

trx_t::commit(): Renamed from trx_commit().

trx_t::commit_low(): Renamed from trx_commit_low().

trx_t::commit_in_memory(): Renamed from trx_commit_in_memory().
parent 0632b803
......@@ -7079,7 +7079,7 @@ prepare_inplace_alter_table_dict(
goto error_handling;
}
trx_commit(ctx->trx);
ctx->trx->commit();
trx_start_for_ddl(ctx->trx, op);
if (!ctx->new_table->fts
......@@ -11101,7 +11101,7 @@ ha_innobase::commit_inplace_alter_table(
logical sense the commit in the file-based
data structures happens here. */
trx_commit_low(trx, &mtr);
trx->commit_low(&mtr);
}
/* If server crashes here, the dictionary in
......
......@@ -181,17 +181,6 @@ trx_start_for_ddl_low(
trx_start_for_ddl_low((t), (o))
#endif /* UNIV_DEBUG */
/****************************************************************//**
Commits a transaction. */
void
trx_commit(
/*=======*/
trx_t* trx); /*!< in/out: transaction */
/** Commit a transaction and a mini-transaction.
@param[in,out] trx transaction
@param[in,out] mtr mini-transaction (NULL if no modifications) */
void trx_commit_low(trx_t* trx, mtr_t* mtr);
/**********************************************************************//**
Does the transaction commit for MySQL.
@return DB_SUCCESS or error number */
......@@ -898,10 +887,10 @@ struct trx_t {
defer flush of the logs to disk
until after we release the
mutex. */
bool must_flush_log_later;/*!< this flag is set to TRUE in
trx_commit() if flush_log_later was
TRUE, and there were modifications by
the transaction; in that case we must
bool must_flush_log_later;/*!< set in commit()
if flush_log_later was
set and redo log was written;
in that case we will
flush the log in
trx_commit_complete_for_mysql() */
ulint duplicates; /*!< TRX_DUP_IGNORE | TRX_DUP_REPLACE */
......@@ -1132,11 +1121,20 @@ struct trx_t {
@param[in] table_id table identifier */
void evict_table(table_id_t table_id);
private:
/** Mark a transaction committed in the main memory data structures. */
inline void commit_in_memory(const mtr_t *mtr);
public:
/** Commit the transaction. */
void commit();
bool is_referenced()
{
return n_ref > 0;
}
/** Commit the transaction in a mini-transaction.
@param mtr mini-transaction (if there are any persistent modifications) */
void commit_low(mtr_t *mtr= nullptr);
bool is_referenced() const { return n_ref > 0; }
void reference()
......
......@@ -6865,7 +6865,7 @@ DeadlockChecker::trx_rollback()
print("*** WE ROLL BACK TRANSACTION (1)\n");
#ifdef WITH_WSREP
if (wsrep_on(trx->mysql_thd) && wsrep_thd_is_SR(trx->mysql_thd)) {
if (trx->is_wsrep() && wsrep_thd_is_SR(trx->mysql_thd)) {
wsrep_handle_SR_rollback(m_start->mysql_thd, trx->mysql_thd);
}
#endif
......@@ -6956,8 +6956,7 @@ DeadlockChecker::check_and_resolve(const lock_t* lock, trx_t* trx)
print("*** WE ROLL BACK TRANSACTION (2)\n");
#ifdef WITH_WSREP
if (wsrep_on(trx->mysql_thd)
&& wsrep_thd_is_SR(trx->mysql_thd)) {
if (trx->is_wsrep() && wsrep_thd_is_SR(trx->mysql_thd)) {
wsrep_handle_SR_rollback(trx->mysql_thd,
victim_trx->mysql_thd);
}
......
......@@ -59,7 +59,7 @@ static bool trx_rollback_finish(trx_t* trx)
trx->mod_tables.clear();
bool finished = trx->error_state == DB_SUCCESS;
if (UNIV_LIKELY(finished)) {
trx_commit(trx);
trx->commit();
} else {
ut_a(trx->error_state == DB_INTERRUPTED);
ut_ad(!srv_is_being_started);
......@@ -84,7 +84,7 @@ static bool trx_rollback_finish(trx_t* trx)
ut_free(undo);
undo = NULL;
}
trx_commit_low(trx, NULL);
trx->commit_low();
}
trx->lock.que_state = TRX_QUE_RUNNING;
......
......@@ -1338,316 +1338,255 @@ void trx_t::evict_table(table_id_t table_id)
}
}
/****************************************************************//**
Commits a transaction in memory. */
static
void
trx_commit_in_memory(
/*=================*/
trx_t* trx, /*!< in/out: transaction */
const mtr_t* mtr) /*!< in: mini-transaction of
trx_write_serialisation_history(), or NULL if
the transaction did not modify anything */
/** Mark a transaction committed in the main memory data structures. */
inline void trx_t::commit_in_memory(const mtr_t *mtr)
{
trx->must_flush_log_later = false;
trx->read_view.close();
if (trx_is_autocommit_non_locking(trx)) {
ut_ad(trx->id == 0);
ut_ad(trx->read_only);
ut_a(!trx->is_recovered);
ut_ad(trx->rsegs.m_redo.rseg == NULL);
must_flush_log_later= false;
read_view.close();
/* Note: We are asserting without holding the lock mutex. But
that is OK because this transaction is not waiting and cannot
be rolled back and no new locks can (or should) be added
because it is flagged as a non-locking read-only transaction. */
ut_a(UT_LIST_GET_LEN(trx->lock.trx_locks) == 0);
/* This state change is not protected by any mutex, therefore
there is an inherent race here around state transition during
printouts. We ignore this race for the sake of efficiency.
However, the trx_sys_t::mutex will protect the trx_t instance
and it cannot be removed from the trx_list and freed
without first acquiring the trx_sys_t::mutex. */
ut_ad(trx_state_eq(trx, TRX_STATE_ACTIVE));
MONITOR_INC(MONITOR_TRX_NL_RO_COMMIT);
DBUG_LOG("trx", "Autocommit in memory: " << trx);
trx->state = TRX_STATE_NOT_STARTED;
} else {
if (trx_is_autocommit_non_locking(this))
{
ut_ad(id == 0);
ut_ad(read_only);
ut_a(!is_recovered);
ut_ad(!rsegs.m_redo.rseg);
/* Note: We are asserting without holding the lock mutex. But
that is OK because this transaction is not waiting and cannot
be rolled back and no new locks can (or should) be added
because it is flagged as a non-locking read-only transaction. */
ut_a(UT_LIST_GET_LEN(lock.trx_locks) == 0);
/* This state change is not protected by any mutex, therefore
there is an inherent race here around state transition during
printouts. We ignore this race for the sake of efficiency.
However, the trx_sys_t::mutex will protect the trx_t instance
and it cannot be removed from the trx_list and freed
without first acquiring the trx_sys_t::mutex. */
ut_ad(trx_state_eq(this, TRX_STATE_ACTIVE));
MONITOR_INC(MONITOR_TRX_NL_RO_COMMIT);
DBUG_LOG("trx", "Autocommit in memory: " << this);
state= TRX_STATE_NOT_STARTED;
}
else
{
#ifdef UNIV_DEBUG
if (!UT_LIST_GET_LEN(trx->lock.trx_locks)) {
for (lock_list::iterator it
= trx->lock.table_locks.begin();
it != trx->lock.table_locks.end();
it++) {
ut_ad(!*it);
}
}
if (!UT_LIST_GET_LEN(lock.trx_locks))
for (auto l : lock.table_locks)
ut_ad(!l);
#endif /* UNIV_DEBUG */
trx->commit_state();
if (trx->id) {
trx_sys.deregister_rw(trx);
/* Wait for any implicit-to-explicit lock
conversions to cease, so that there will be no
race condition in lock_release(). */
while (UNIV_UNLIKELY(trx->is_referenced())) {
ut_delay(srv_spin_wait_delay);
}
trx->release_locks();
trx->id = 0;
} else {
ut_ad(trx->read_only || !trx->rsegs.m_redo.rseg);
trx->release_locks();
}
DEBUG_SYNC_C("after_trx_committed_in_memory");
commit_state();
if (trx->read_only || !trx->rsegs.m_redo.rseg) {
MONITOR_INC(MONITOR_TRX_RO_COMMIT);
} else {
trx_update_mod_tables_timestamp(trx);
MONITOR_INC(MONITOR_TRX_RW_COMMIT);
trx->is_recovered = false;
}
while (dict_table_t* table = UT_LIST_GET_FIRST(
trx->lock.evicted_tables)) {
UT_LIST_REMOVE(trx->lock.evicted_tables, table);
dict_mem_table_free(table);
}
}
ut_ad(!trx->rsegs.m_redo.undo);
ut_ad(UT_LIST_GET_LEN(trx->lock.evicted_tables) == 0);
if (trx_rseg_t* rseg = trx->rsegs.m_redo.rseg) {
mutex_enter(&rseg->mutex);
ut_ad(rseg->trx_ref_count > 0);
--rseg->trx_ref_count;
mutex_exit(&rseg->mutex);
if (trx_undo_t*& insert = trx->rsegs.m_redo.old_insert) {
ut_ad(insert->rseg == rseg);
trx_undo_commit_cleanup(insert, false);
insert = NULL;
}
}
ut_ad(!trx->rsegs.m_redo.old_insert);
if (mtr != NULL) {
if (trx_undo_t*& undo = trx->rsegs.m_noredo.undo) {
ut_ad(undo->rseg == trx->rsegs.m_noredo.rseg);
trx_undo_commit_cleanup(undo, true);
undo = NULL;
}
if (id)
{
trx_sys.deregister_rw(this);
/* Wait for any implicit-to-explicit lock conversions to cease,
so that there will be no race condition in lock_release(). */
while (UNIV_UNLIKELY(is_referenced()))
ut_delay(srv_spin_wait_delay);
release_locks();
id= 0;
}
else
{
ut_ad(read_only || !rsegs.m_redo.rseg);
release_locks();
}
/* NOTE that we could possibly make a group commit more
efficient here: call os_thread_yield here to allow also other
trxs to come to commit! */
DEBUG_SYNC_C("after_trx_committed_in_memory");
/*-------------------------------------*/
if (read_only || !rsegs.m_redo.rseg)
{
MONITOR_INC(MONITOR_TRX_RO_COMMIT);
}
else
{
trx_update_mod_tables_timestamp(this);
MONITOR_INC(MONITOR_TRX_RW_COMMIT);
is_recovered= false;
}
/* Depending on the my.cnf options, we may now write the log
buffer to the log files, making the transaction durable if
the OS does not crash. We may also flush the log files to
disk, making the transaction durable also at an OS crash or a
power outage.
while (dict_table_t *table= UT_LIST_GET_FIRST(lock.evicted_tables))
{
UT_LIST_REMOVE(lock.evicted_tables, table);
dict_mem_table_free(table);
}
}
The idea in InnoDB's group commit is that a group of
transactions gather behind a trx doing a physical disk write
to log files, and when that physical write has been completed,
one of those transactions does a write which commits the whole
group. Note that this group commit will only bring benefit if
there are > 2 users in the database. Then at least 2 users can
gather behind one doing the physical log write to disk.
ut_ad(!rsegs.m_redo.undo);
ut_ad(UT_LIST_GET_LEN(lock.evicted_tables) == 0);
If we are calling trx_commit() under prepare_commit_mutex, we
will delay possible log write and flush to a separate function
trx_commit_complete_for_mysql(), which is only called when the
thread has released the mutex. This is to make the
group commit algorithm to work. Otherwise, the prepare_commit
mutex would serialize all commits and prevent a group of
transactions from gathering. */
lsn_t lsn = mtr->commit_lsn();
if (lsn == 0) {
/* Nothing to be done. */
} else if (trx->flush_log_later) {
/* Do nothing yet */
trx->must_flush_log_later = true;
} else if (srv_flush_log_at_trx_commit == 0) {
/* Do nothing */
} else {
trx_flush_log_if_needed(lsn, trx);
}
if (trx_rseg_t *rseg= rsegs.m_redo.rseg)
{
mutex_enter(&rseg->mutex);
ut_ad(rseg->trx_ref_count > 0);
--rseg->trx_ref_count;
mutex_exit(&rseg->mutex);
trx->commit_lsn = lsn;
if (trx_undo_t *&insert= rsegs.m_redo.old_insert)
{
ut_ad(insert->rseg == rseg);
trx_undo_commit_cleanup(insert, false);
insert= nullptr;
}
}
/* Tell server some activity has happened, since the trx
does changes something. Background utility threads like
master thread, purge thread or page_cleaner thread might
have some work to do. */
srv_active_wake_master_thread();
}
ut_ad(!rsegs.m_redo.old_insert);
ut_ad(!trx->rsegs.m_noredo.undo);
if (mtr)
{
if (trx_undo_t *&undo= rsegs.m_noredo.undo)
{
ut_ad(undo->rseg == rsegs.m_noredo.rseg);
trx_undo_commit_cleanup(undo, true);
undo= nullptr;
}
/* Free all savepoints, starting from the first. */
trx_named_savept_t* savep = UT_LIST_GET_FIRST(trx->trx_savepoints);
/* NOTE that we could possibly make a group commit more efficient
here: call os_thread_yield here to allow also other trxs to come
to commit! */
/*-------------------------------------*/
/* Depending on the my.cnf options, we may now write the log
buffer to the log files, making the transaction durable if the OS
does not crash. We may also flush the log files to disk, making
the transaction durable also at an OS crash or a power outage.
The idea in InnoDB's group commit is that a group of transactions
gather behind a trx doing a physical disk write to log files, and
when that physical write has been completed, one of those
transactions does a write which commits the whole group. Note that
this group commit will only bring benefit if there are > 2 users
in the database. Then at least 2 users can gather behind one doing
the physical log write to disk.
If we are calling trx_t::commit() under prepare_commit_mutex, we
will delay possible log write and flush to a separate function
trx_commit_complete_for_mysql(), which is only called when the
thread has released the mutex. This is to make the group commit
algorithm to work. Otherwise, the prepare_commit mutex would
serialize all commits and prevent a group of transactions from
gathering. */
commit_lsn= mtr->commit_lsn();
if (!commit_lsn)
/* Nothing to be done. */;
else if (flush_log_later)
/* Do nothing yet */
must_flush_log_later= true;
else if (srv_flush_log_at_trx_commit)
trx_flush_log_if_needed(commit_lsn, this);
/* Tell server some activity has happened, since the trx does
changes something. Background utility threads like master thread,
purge thread or page_cleaner thread might have some work to do. */
srv_active_wake_master_thread();
}
trx_roll_savepoints_free(trx, savep);
ut_ad(!rsegs.m_noredo.undo);
if (trx->fts_trx != NULL) {
trx_finalize_for_fts(trx, trx->undo_no != 0);
}
/* Free all savepoints, starting from the first. */
trx_named_savept_t *savep= UT_LIST_GET_FIRST(trx_savepoints);
trx_mutex_enter(trx);
trx->dict_operation = TRX_DICT_OP_NONE;
trx->lock.was_chosen_as_deadlock_victim = false;
trx_roll_savepoints_free(this, savep);
DBUG_LOG("trx", "Commit in memory: " << trx);
trx->state = TRX_STATE_NOT_STARTED;
if (fts_trx)
trx_finalize_for_fts(this, undo_no != 0);
assert_trx_is_free(trx);
trx_mutex_enter(this);
dict_operation= TRX_DICT_OP_NONE;
lock.was_chosen_as_deadlock_victim= false;
trx_init(trx);
DBUG_LOG("trx", "Commit in memory: " << this);
state= TRX_STATE_NOT_STARTED;
trx_mutex_exit(trx);
assert_trx_is_free(this);
trx_init(this);
trx_mutex_exit(this);
ut_a(trx->error_state == DB_SUCCESS);
if (!srv_read_only_mode) {
srv_wake_purge_thread_if_not_active();
}
ut_a(error_state == DB_SUCCESS);
if (!srv_read_only_mode)
srv_wake_purge_thread_if_not_active();
}
/** Commit a transaction and a mini-transaction.
@param[in,out] trx transaction
@param[in,out] mtr mini-transaction (NULL if no modifications) */
void trx_commit_low(trx_t* trx, mtr_t* mtr)
/** Commit the transaction in a mini-transaction.
@param mtr mini-transaction (if there are any persistent modifications) */
void trx_t::commit_low(mtr_t *mtr)
{
assert_trx_nonlocking_or_in_list(trx);
ut_ad(!trx_state_eq(trx, TRX_STATE_COMMITTED_IN_MEMORY));
ut_ad(!mtr || mtr->is_active());
ut_d(bool aborted = trx->in_rollback
&& trx->error_state == DB_DEADLOCK);
ut_ad(!mtr == (aborted || !trx->has_logged_or_recovered()));
ut_ad(!mtr || !aborted);
/* undo_no is non-zero if we're doing the final commit. */
if (trx->fts_trx != NULL && trx->undo_no != 0) {
dberr_t error;
ut_a(!trx_is_autocommit_non_locking(trx));
error = fts_commit(trx);
/* FTS-FIXME: Temporarily tolerate DB_DUPLICATE_KEY
instead of dying. This is a possible scenario if there
is a crash between insert to DELETED table committing
and transaction committing. The fix would be able to
return error from this function */
if (error != DB_SUCCESS && error != DB_DUPLICATE_KEY) {
/* FTS-FIXME: once we can return values from this
function, we should do so and signal an error
instead of just dying. */
ut_error;
}
}
assert_trx_nonlocking_or_in_list(this);
ut_ad(!trx_state_eq(this, TRX_STATE_COMMITTED_IN_MEMORY));
ut_ad(!mtr || mtr->is_active());
ut_d(bool aborted = in_rollback && error_state == DB_DEADLOCK);
ut_ad(!mtr == (aborted || !has_logged_or_recovered()));
ut_ad(!mtr || !aborted);
/* undo_no is non-zero if we're doing the final commit. */
if (fts_trx && undo_no)
{
ut_a(!trx_is_autocommit_non_locking(this));
dberr_t error= fts_commit(this);
/* FTS-FIXME: Temporarily tolerate DB_DUPLICATE_KEY instead of
dying. This is a possible scenario if there is a crash between
insert to DELETED table committing and transaction committing. The
fix would be able to return error from this function */
ut_a(error == DB_SUCCESS || error == DB_DUPLICATE_KEY);
}
#ifndef DBUG_OFF
const bool debug_sync = trx->mysql_thd && trx->has_logged_persistent();
const bool debug_sync= mysql_thd && has_logged_persistent();
#endif
if (mtr != NULL) {
trx_write_serialisation_history(trx, mtr);
/* The following call commits the mini-transaction, making the
whole transaction committed in the file-based world, at this
log sequence number. The transaction becomes 'durable' when
we write the log to disk, but in the logical sense the commit
in the file-based data structures (undo logs etc.) happens
here.
NOTE that transaction numbers, which are assigned only to
transactions with an update undo log, do not necessarily come
in exactly the same order as commit lsn's, if the transactions
have different rollback segments. To get exactly the same
order we should hold the kernel mutex up to this point,
adding to the contention of the kernel mutex. However, if
a transaction T2 is able to see modifications made by
a transaction T1, T2 will always get a bigger transaction
number and a bigger commit lsn than T1. */
/*--------------*/
mtr_commit(mtr);
DBUG_EXECUTE_IF("ib_crash_during_trx_commit_in_mem",
if (trx->has_logged()) {
log_write_up_to(mtr->commit_lsn(),
true);
DBUG_SUICIDE();
});
/*--------------*/
}
if (mtr)
{
trx_write_serialisation_history(this, mtr);
/* The following call commits the mini-transaction, making the
whole transaction committed in the file-based world, at this log
sequence number. The transaction becomes 'durable' when we write
the log to disk, but in the logical sense the commit in the
file-based data structures (undo logs etc.) happens here.
NOTE that transaction numbers, which are assigned only to
transactions with an update undo log, do not necessarily come in
exactly the same order as commit lsn's, if the transactions have
different rollback segments. To get exactly the same order we
should hold the kernel mutex up to this point, adding to the
contention of the kernel mutex. However, if a transaction T2 is
able to see modifications made by a transaction T1, T2 will always
get a bigger transaction number and a bigger commit lsn than T1. */
mtr->commit();
}
#ifndef DBUG_OFF
/* In case of this function is called from a stack executing
THD::release_resources -> ...
innobase_connection_close() ->
trx_rollback_for_mysql... -> .
mysql's thd does not seem to have
thd->debug_sync_control defined any longer. However the stack
is possible only with a prepared trx not updating any data.
*/
if (debug_sync) {
DEBUG_SYNC_C("before_trx_state_committed_in_memory");
}
if (debug_sync)
DEBUG_SYNC_C("before_trx_state_committed_in_memory");
#endif
trx_commit_in_memory(trx, mtr);
commit_in_memory(mtr);
}
/****************************************************************//**
Commits a transaction. */
void
trx_commit(
/*=======*/
trx_t* trx) /*!< in/out: transaction */
{
mtr_t* mtr;
mtr_t local_mtr;
DBUG_EXECUTE_IF("ib_trx_commit_crash_before_trx_commit_start",
DBUG_SUICIDE(););
if (trx->has_logged_or_recovered()) {
mtr = &local_mtr;
mtr->start();
} else {
mtr = NULL;
}
void trx_t::commit()
{
mtr_t *mtr= nullptr;
mtr_t local_mtr;
trx_commit_low(trx, mtr);
if (has_logged_or_recovered())
{
mtr= &local_mtr;
local_mtr.start();
}
commit_low(mtr);
#ifdef WITH_WSREP
/* Serialization history has been written and the
transaction is committed in memory, which makes
this commit ordered. Release commit order critical
section. */
if (wsrep_on(trx->mysql_thd))
{
wsrep_commit_ordered(trx->mysql_thd);
}
/* Serialization history has been written and the transaction is
committed in memory, which makes this commit ordered. Release commit
order critical section. */
if (mtr && is_wsrep())
wsrep_commit_ordered(mysql_thd);
#endif /* WITH_WSREP */
}
......@@ -1741,11 +1680,8 @@ trx_commit_step(
trx_commit_or_rollback_prepare(trx);
trx->lock.que_state = TRX_QUE_COMMITTING;
trx_commit(trx);
trx->commit();
ut_ad(trx->lock.wait_thr == NULL);
trx->lock.que_state = TRX_QUE_RUNNING;
thr = NULL;
......@@ -1783,9 +1719,7 @@ trx_commit_for_mysql(
case TRX_STATE_PREPARED:
case TRX_STATE_PREPARED_RECOVERED:
trx->op_info = "committing";
trx_commit(trx);
trx->commit();
MONITOR_DEC(MONITOR_TRX_ACTIVE);
trx->op_info = "";
return(DB_SUCCESS);
......
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