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().
This commit is contained in:
parent
0632b8034b
commit
cfbbf5424b
@ -7079,7 +7079,7 @@ op_ok:
|
||||
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 @@ public:
|
||||
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 @@ public:
|
||||
@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();
|
||||
must_flush_log_later= false;
|
||||
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);
|
||||
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. */
|
||||
/* 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);
|
||||
|
||||
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(this, TRX_STATE_ACTIVE));
|
||||
|
||||
/* 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. */
|
||||
MONITOR_INC(MONITOR_TRX_NL_RO_COMMIT);
|
||||
|
||||
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 {
|
||||
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();
|
||||
commit_state();
|
||||
|
||||
if (trx->id) {
|
||||
trx_sys.deregister_rw(trx);
|
||||
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(trx->is_referenced())) {
|
||||
ut_delay(srv_spin_wait_delay);
|
||||
}
|
||||
/* 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();
|
||||
}
|
||||
|
||||
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");
|
||||
|
||||
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;
|
||||
}
|
||||
|
||||
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(lock.evicted_tables))
|
||||
{
|
||||
UT_LIST_REMOVE(lock.evicted_tables, table);
|
||||
dict_mem_table_free(table);
|
||||
}
|
||||
}
|
||||
|
||||
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(!rsegs.m_redo.undo);
|
||||
ut_ad(UT_LIST_GET_LEN(lock.evicted_tables) == 0);
|
||||
|
||||
ut_ad(!trx->rsegs.m_redo.undo);
|
||||
ut_ad(UT_LIST_GET_LEN(trx->lock.evicted_tables) == 0);
|
||||
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);
|
||||
|
||||
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= rsegs.m_redo.old_insert)
|
||||
{
|
||||
ut_ad(insert->rseg == rseg);
|
||||
trx_undo_commit_cleanup(insert, false);
|
||||
insert= nullptr;
|
||||
}
|
||||
}
|
||||
|
||||
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(!rsegs.m_redo.old_insert);
|
||||
|
||||
ut_ad(!trx->rsegs.m_redo.old_insert);
|
||||
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;
|
||||
}
|
||||
|
||||
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;
|
||||
}
|
||||
/* 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! */
|
||||
|
||||
/* 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.
|
||||
|
||||
/* 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.
|
||||
|
||||
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. */
|
||||
|
||||
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. */
|
||||
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);
|
||||
|
||||
lsn_t lsn = mtr->commit_lsn();
|
||||
/* 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();
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
ut_ad(!rsegs.m_noredo.undo);
|
||||
|
||||
trx->commit_lsn = lsn;
|
||||
/* Free all savepoints, starting from the first. */
|
||||
trx_named_savept_t *savep= UT_LIST_GET_FIRST(trx_savepoints);
|
||||
|
||||
/* 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(this, savep);
|
||||
|
||||
ut_ad(!trx->rsegs.m_noredo.undo);
|
||||
if (fts_trx)
|
||||
trx_finalize_for_fts(this, undo_no != 0);
|
||||
|
||||
/* Free all savepoints, starting from the first. */
|
||||
trx_named_savept_t* savep = UT_LIST_GET_FIRST(trx->trx_savepoints);
|
||||
trx_mutex_enter(this);
|
||||
dict_operation= TRX_DICT_OP_NONE;
|
||||
lock.was_chosen_as_deadlock_victim= false;
|
||||
|
||||
trx_roll_savepoints_free(trx, savep);
|
||||
DBUG_LOG("trx", "Commit in memory: " << this);
|
||||
state= TRX_STATE_NOT_STARTED;
|
||||
|
||||
if (trx->fts_trx != NULL) {
|
||||
trx_finalize_for_fts(trx, trx->undo_no != 0);
|
||||
}
|
||||
assert_trx_is_free(this);
|
||||
trx_init(this);
|
||||
trx_mutex_exit(this);
|
||||
|
||||
trx_mutex_enter(trx);
|
||||
trx->dict_operation = TRX_DICT_OP_NONE;
|
||||
trx->lock.was_chosen_as_deadlock_victim = false;
|
||||
|
||||
DBUG_LOG("trx", "Commit in memory: " << trx);
|
||||
trx->state = TRX_STATE_NOT_STARTED;
|
||||
|
||||
assert_trx_is_free(trx);
|
||||
|
||||
trx_init(trx);
|
||||
|
||||
trx_mutex_exit(trx);
|
||||
|
||||
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);
|
||||
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 (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;
|
||||
}
|
||||
}
|
||||
/* 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);
|
||||
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.
|
||||
/* 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. */
|
||||
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();
|
||||
});
|
||||
/*--------------*/
|
||||
}
|
||||
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 */
|
||||
|
||||
void trx_t::commit()
|
||||
{
|
||||
mtr_t* mtr;
|
||||
mtr_t local_mtr;
|
||||
mtr_t *mtr= nullptr;
|
||||
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;
|
||||
}
|
||||
|
||||
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);
|
||||
|
Loading…
x
Reference in New Issue
Block a user