summaryrefslogtreecommitdiff
path: root/storage/innobase/trx/trx0trx.cc
diff options
context:
space:
mode:
Diffstat (limited to 'storage/innobase/trx/trx0trx.cc')
-rw-r--r--storage/innobase/trx/trx0trx.cc499
1 files changed, 217 insertions, 282 deletions
diff --git a/storage/innobase/trx/trx0trx.cc b/storage/innobase/trx/trx0trx.cc
index 71a0c1befb4..b6b376215c4 100644
--- a/storage/innobase/trx/trx0trx.cc
+++ b/storage/innobase/trx/trx0trx.cc
@@ -109,6 +109,9 @@ trx_init(
trx->state = TRX_STATE_NOT_STARTED;
trx->is_recovered = false;
+#ifdef WITH_WSREP
+ trx->wsrep = false;
+#endif /* WITH_WSREP */
trx->op_info = "";
@@ -1320,318 +1323,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);
-
- /* 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));
+ must_flush_log_later= false;
+ read_view.close();
- 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();
- }
+ commit_state();
- DEBUG_SYNC_C("after_trx_committed_in_memory");
-
- 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;
-#ifdef WITH_WSREP
- trx->lock.was_chosen_as_wsrep_victim = FALSE;
-#endif
+ 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 */
}
@@ -1725,11 +1665,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;
@@ -1767,9 +1704,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);