summaryrefslogtreecommitdiff
path: root/storage/innobase
diff options
context:
space:
mode:
authorMarko Mäkelä <marko.makela@mariadb.com>2019-09-03 12:31:37 +0300
committerMarko Mäkelä <marko.makela@mariadb.com>2019-09-04 09:42:38 +0300
commitb07beff8943d083cf900c8f2ea0a386a84d9f8db (patch)
tree388665f25f2c818a4efa057c9d9048558ddfe27d /storage/innobase
parent7c79c127842133d1f85ac2273a229d84007075c9 (diff)
downloadmariadb-git-b07beff8943d083cf900c8f2ea0a386a84d9f8db.tar.gz
MDEV-15326: InnoDB: Failing assertion: !other_lock
MySQL 5.7.9 (and MariaDB 10.2.2) introduced a race condition between InnoDB transaction commit and the conversion of implicit locks into explicit ones. The assertion failure can be triggered with a test that runs 3 concurrent single-statement transactions in a loop on a simple table: CREATE TABLE t (a INT PRIMARY KEY) ENGINE=InnoDB; thread1: INSERT INTO t SET a=1; thread2: DELETE FROM t; thread3: SELECT * FROM t FOR UPDATE; -- or DELETE FROM t; The failure scenarios are like the following: (1) The INSERT statement is being committed, waiting for lock_sys->mutex. (2) At the time of the failure, both the DELETE and SELECT transactions are active but have not logged any changes yet. (3) The transaction where the !other_lock assertion fails started lock_rec_convert_impl_to_expl(). (4) After this point, the commit of the INSERT removed the transaction from trx_sys->rw_trx_set, in trx_erase_lists(). (5) The other transaction consulted trx_sys->rw_trx_set and determined that there is no implicit lock. Hence, it grabbed the lock. (6) The !other_lock assertion fails in lock_rec_add_to_queue() for the lock_rec_convert_impl_to_expl(), because the lock was 'stolen'. This assertion failure looks genuine, because the INSERT transaction is still active (trx->state=TRX_STATE_ACTIVE). The problematic step (4) was introduced in mysql/mysql-server@e27e0e0bb75b4d35e87059816f1cc370c09890ad which fixed something related to MVCC (covered by the test innodb.innodb-read-view). Basically, it reintroduced an error that had been mentioned in an earlier commit mysql/mysql-server@a17be6963fc0d9210fa0642d3985b7219cdaf0c5: "The active transaction was removed from trx_sys->rw_trx_set prematurely." Our fix goes along the following lines: (a) Implicit locks will released by assigning trx->state=TRX_STATE_COMMITTED_IN_MEMORY as the first step. This transition will no longer be protected by lock_sys_t::mutex, only by trx->mutex. This idea is by Sergey Vojtovich. (b) We detach the transaction from trx_sys before starting to release explicit locks. (c) All callers of trx_rw_is_active() and trx_rw_is_active_low() must recheck trx->state after acquiring trx->mutex. (d) Before releasing any explicit locks, we will ensure that any activity by other threads to convert implicit locks into explicit will have ceased, by checking !trx_is_referenced(trx). There was a glitch in this check when it was part of lock_trx_release_locks(); at the end we would release trx->mutex and acquire lock_sys->mutex and trx->mutex, and fail to recheck (trx_is_referenced() is protected by trx_t::mutex). (e) Explicit locks can be released in batches (LOCK_RELEASE_INTERVAL=1000) just like we did before. trx_t::state: Document that the transition to COMMITTED is only protected by trx_t::mutex, no longer by lock_sys_t::mutex. trx_rw_is_active_low(), trx_rw_is_active(): Document that the transaction state should be rechecked after acquiring trx_t::mutex. trx_t::commit_state(): New function to change a transaction to committed state, to release implicit locks. trx_t::release_locks(): New function to release the explicit locks after commit_state(). lock_trx_release_locks(): Move much of the logic to the caller (which must invoke trx_t::commit_state() and trx_t::release_locks() as needed), and assert that the transaction will have locks. trx_get_trx_by_xid(): Make the parameter a pointer to const. lock_rec_other_trx_holds_expl(): Recheck trx->state after acquiring trx->mutex, and avoid a redundant lookup of the transaction. lock_rec_queue_validate(): Recheck impl_trx->state while holding impl_trx->mutex. row_vers_impl_x_locked(), row_vers_impl_x_locked_low(): Document that the transaction state must be rechecked after trx_mutex_enter(). trx_free_prepared(): Adjust for the changes to lock_trx_release_locks().
Diffstat (limited to 'storage/innobase')
-rw-r--r--storage/innobase/include/lock0lock.h13
-rw-r--r--storage/innobase/include/row0vers.h3
-rw-r--r--storage/innobase/include/trx0sys.h41
-rw-r--r--storage/innobase/include/trx0sys.ic81
-rw-r--r--storage/innobase/include/trx0trx.h53
-rw-r--r--storage/innobase/lock/lock0lock.cc229
-rw-r--r--storage/innobase/row/row0vers.cc49
-rw-r--r--storage/innobase/trx/trx0roll.cc6
-rw-r--r--storage/innobase/trx/trx0trx.cc151
-rw-r--r--storage/innobase/trx/trx0undo.cc4
10 files changed, 278 insertions, 352 deletions
diff --git a/storage/innobase/include/lock0lock.h b/storage/innobase/include/lock0lock.h
index 18bc3abfaf1..8369db8b879 100644
--- a/storage/innobase/include/lock0lock.h
+++ b/storage/innobase/include/lock0lock.h
@@ -499,14 +499,11 @@ lock_rec_unlock(
const buf_block_t* block, /*!< in: buffer block containing rec */
const rec_t* rec, /*!< in: record */
lock_mode lock_mode);/*!< in: LOCK_S or LOCK_X */
-/*********************************************************************//**
-Releases a transaction's locks, and releases possible other transactions
-waiting because of these locks. Change the state of the transaction to
-TRX_STATE_COMMITTED_IN_MEMORY. */
-void
-lock_trx_release_locks(
-/*===================*/
- trx_t* trx); /*!< in/out: transaction */
+
+/** Release the explicit locks of a committing transaction,
+and release possible other transactions waiting because of these locks. */
+void lock_trx_release_locks(trx_t* trx);
+
/*********************************************************************//**
Removes locks on a table to be dropped or discarded.
If remove_also_table_sx_locks is TRUE then table-level S and X locks are
diff --git a/storage/innobase/include/row0vers.h b/storage/innobase/include/row0vers.h
index ef60423506e..b5279e5851b 100644
--- a/storage/innobase/include/row0vers.h
+++ b/storage/innobase/include/row0vers.h
@@ -43,7 +43,8 @@ index record.
@param[in] rec secondary index record
@param[in] index secondary index
@param[in] offsets rec_get_offsets(rec, index)
-@return the active transaction; trx_release_reference() must be invoked
+@return the active transaction; state must be rechecked after
+trx_mutex_enter(), and trx_release_reference() must be invoked
@retval NULL if the record was committed */
trx_t*
row_vers_impl_x_locked(
diff --git a/storage/innobase/include/trx0sys.h b/storage/innobase/include/trx0sys.h
index 37ac8975783..c4b1636cfd2 100644
--- a/storage/innobase/include/trx0sys.h
+++ b/storage/innobase/include/trx0sys.h
@@ -1,7 +1,7 @@
/*****************************************************************************
Copyright (c) 1996, 2016, Oracle and/or its affiliates. All Rights Reserved.
-Copyright (c) 2017, MariaDB Corporation.
+Copyright (c) 2017, 2019, 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
@@ -186,30 +186,21 @@ UNIV_INLINE
trx_id_t
trx_rw_min_trx_id(void);
/*===================*/
-/****************************************************************//**
-Checks if a rw transaction with the given id is active.
-@return transaction instance if active, or NULL */
-UNIV_INLINE
-trx_t*
-trx_rw_is_active_low(
-/*=================*/
- trx_id_t trx_id, /*!< in: trx id of the transaction */
- ibool* corrupt); /*!< in: NULL or pointer to a flag
- that will be set if corrupt */
-/****************************************************************//**
-Checks if a rw transaction with the given id is active. If the caller is
-not holding trx_sys->mutex, the transaction may already have been
-committed.
-@return transaction instance if active, or NULL; */
-UNIV_INLINE
-trx_t*
-trx_rw_is_active(
-/*=============*/
- trx_id_t trx_id, /*!< in: trx id of the transaction */
- ibool* corrupt, /*!< in: NULL or pointer to a flag
- that will be set if corrupt */
- bool do_ref_count); /*!< in: if true then increment the
- trx_t::n_ref_count */
+/** Look up a rw transaction with the given id.
+@param[in] trx_id transaction identifier
+@param[out] corrupt flag that will be set if trx_id is corrupted
+@return transaction; its state should be rechecked after acquiring trx_t::mutex
+@retval NULL if there is no transaction identified by trx_id. */
+inline trx_t* trx_rw_is_active_low(trx_id_t trx_id, bool* corrupt);
+
+/** Look up a rw transaction with the given id.
+@param[in] trx_id transaction identifier
+@param[out] corrupt flag that will be set if trx_id is corrupted
+@param[in] ref_count whether to increment trx->n_ref
+@return transaction; its state should be rechecked after acquiring trx_t::mutex
+@retval NULL if there is no active transaction identified by trx_id. */
+inline trx_t* trx_rw_is_active(trx_id_t trx_id, bool* corrupt, bool ref_count);
+
#if defined UNIV_DEBUG || defined UNIV_BLOB_LIGHT_DEBUG
/***********************************************************//**
Assert that a transaction has been recovered.
diff --git a/storage/innobase/include/trx0sys.ic b/storage/innobase/include/trx0sys.ic
index bfd345a27db..ac84404a397 100644
--- a/storage/innobase/include/trx0sys.ic
+++ b/storage/innobase/include/trx0sys.ic
@@ -1,7 +1,7 @@
/*****************************************************************************
Copyright (c) 1996, 2015, Oracle and/or its affiliates. All Rights Reserved.
-Copyright (c) 2018, MariaDB Corporation.
+Copyright (c) 2018, 2019, 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
@@ -302,61 +302,38 @@ trx_rw_min_trx_id(void)
return(id);
}
-/****************************************************************//**
-Checks if a rw transaction with the given id is active. If the caller is
-not holding lock_sys->mutex, the transaction may already have been committed.
-@return transaction instance if active, or NULL */
-UNIV_INLINE
-trx_t*
-trx_rw_is_active_low(
-/*=================*/
- trx_id_t trx_id, /*!< in: trx id of the transaction */
- ibool* corrupt) /*!< in: NULL or pointer to a flag
- that will be set if corrupt */
+/** Look up a rw transaction with the given id.
+@param[in] trx_id transaction identifier
+@param[out] corrupt flag that will be set if trx_id is corrupted
+@return transaction; its state should be rechecked after acquiring trx_t::mutex
+@retval NULL if there is no transaction identified by trx_id. */
+inline trx_t* trx_rw_is_active_low(trx_id_t trx_id, bool* corrupt)
{
- trx_t* trx;
-
ut_ad(trx_sys_mutex_own());
if (trx_id < trx_rw_min_trx_id_low()) {
-
- trx = NULL;
} else if (trx_id >= trx_sys->max_trx_id) {
/* There must be corruption: we let the caller handle the
diagnostic prints in this case. */
- trx = NULL;
if (corrupt != NULL) {
- *corrupt = TRUE;
- }
- } else {
- trx = trx_get_rw_trx_by_id(trx_id);
-
- if (trx != NULL
- && trx_state_eq(trx, TRX_STATE_COMMITTED_IN_MEMORY)) {
-
- trx = NULL;
+ *corrupt = true;
}
+ } else if (trx_t* trx = trx_get_rw_trx_by_id(trx_id)) {
+ return trx;
}
- return(trx);
+ return NULL;
}
-/****************************************************************//**
-Checks if a rw transaction with the given id is active. If the caller is
-not holding lock_sys->mutex, the transaction may already have been
-committed.
-@return transaction instance if active, or NULL; */
-UNIV_INLINE
-trx_t*
-trx_rw_is_active(
-/*=============*/
- trx_id_t trx_id, /*!< in: trx id of the transaction */
- ibool* corrupt, /*!< in: NULL or pointer to a flag
- that will be set if corrupt */
- bool do_ref_count) /*!< in: if true then increment the
- trx_t::n_ref_count */
+/** Look up a rw transaction with the given id.
+@param[in] trx_id transaction identifier
+@param[out] corrupt flag that will be set if trx_id is corrupted
+@param[in] ref_count whether to increment trx->n_ref
+@return transaction; its state should be rechecked after acquiring trx_t::mutex
+@retval NULL if there is no active transaction identified by trx_id. */
+inline trx_t* trx_rw_is_active(trx_id_t trx_id, bool* corrupt, bool ref_count)
{
ut_ad(trx_id);
@@ -364,13 +341,25 @@ trx_rw_is_active(
trx_t* trx = trx_rw_is_active_low(trx_id, corrupt);
- if (trx && do_ref_count) {
- trx_mutex_enter(trx);
- ut_ad(!trx_state_eq(trx, TRX_STATE_COMMITTED_IN_MEMORY));
+ if (trx && ref_count) {
+ TrxMutex* trx_mutex = &trx->mutex;
+ mutex_enter(trx_mutex);
+ ut_ad(!trx_state_eq(trx, TRX_STATE_NOT_STARTED));
ut_ad(trx->id == trx_id);
ut_ad(trx->n_ref >= 0);
- ++trx->n_ref;
- trx_mutex_exit(trx);
+ if (trx_state_eq(trx, TRX_STATE_COMMITTED_IN_MEMORY)) {
+ /* We have an early state check here to avoid
+ committer starvation in a wait loop for
+ transaction references, when there's a stream of
+ trx_rw_is_active() calls from other threads.
+ The trx->state may change to COMMITTED after
+ trx_mutex is released, and it will have to be
+ rechecked by the caller after reacquiring the mutex. */
+ trx = NULL;
+ } else {
+ ++trx->n_ref;
+ }
+ mutex_exit(trx_mutex);
}
trx_sys_mutex_exit();
diff --git a/storage/innobase/include/trx0trx.h b/storage/innobase/include/trx0trx.h
index 62f7f172453..796e6b8bda6 100644
--- a/storage/innobase/include/trx0trx.h
+++ b/storage/innobase/include/trx0trx.h
@@ -249,16 +249,13 @@ trx_recover_for_mysql(
/*==================*/
XID* xid_list, /*!< in/out: prepared transactions */
ulint len); /*!< in: number of slots in xid_list */
-/*******************************************************************//**
-This function is used to find one X/Open XA distributed transaction
-which is in the prepared state
-@return trx or NULL; on match, the trx->xid will be invalidated;
-note that the trx may have been committed, unless the caller is
-holding lock_sys->mutex */
-trx_t *
-trx_get_trx_by_xid(
-/*===============*/
- XID* xid); /*!< in: X/Open XA transaction identifier */
+/** Look up an X/Open distributed transaction in XA PREPARE state.
+@param[in] xid X/Open XA transaction identifier
+@return transaction on match (the trx_t::xid will be invalidated);
+note that the trx may have been committed before the caller acquires
+trx_t::mutex
+@retval NULL if no match */
+trx_t* trx_get_trx_by_xid(const XID* xid);
/**********************************************************************//**
If required, flushes the log to disk if we called trx_commit_for_mysql()
with trx->flush_log_later == TRUE. */
@@ -578,6 +575,9 @@ Check transaction state */
ut_ad(!MVCC::is_view_active((t)->read_view)); \
ut_ad((t)->lock.wait_thr == NULL); \
ut_ad(UT_LIST_GET_LEN((t)->lock.trx_locks) == 0); \
+ ut_ad((t)->lock.table_locks.empty()); \
+ ut_ad(!(t)->autoinc_locks \
+ || ib_vector_is_empty((t)->autoinc_locks)); \
ut_ad((t)->dict_operation == TRX_DICT_OP_NONE); \
} while(0)
@@ -754,8 +754,8 @@ so without holding any mutex. The following are exceptions to this:
* trx_rollback_resurrected() may access resurrected (connectionless)
transactions while the system is already processing new user
-transactions. The trx_sys->mutex prevents a race condition between it
-and lock_trx_release_locks() [invoked by trx_commit()].
+transactions. The trx_sys->mutex and trx->is_recovered prevent
+a race condition between it and trx_commit().
* trx_print_low() may access transactions not associated with the current
thread. The caller must be holding trx_sys->mutex and lock_sys->mutex.
@@ -767,7 +767,7 @@ holding trx_sys->mutex exclusively.
* The locking code (in particular, lock_deadlock_recursive() and
lock_rec_convert_impl_to_expl()) will access transactions associated
to other connections. The locks of transactions are protected by
-lock_sys->mutex and sometimes by trx->mutex. */
+lock_sys->mutex (insertions also by trx->mutex). */
/** Represents an instance of rollback segment along with its state variables.*/
struct trx_undo_ptr_t {
@@ -873,14 +873,12 @@ struct trx_t {
ACTIVE->COMMITTED is possible when the transaction is in
rw_trx_list.
- Transitions to COMMITTED are protected by both lock_sys->mutex
- and trx->mutex.
-
- NOTE: Some of these state change constraints are an overkill,
- currently only required for a consistent view for printing stats.
- This unnecessarily adds a huge cost for the general case. */
-
+ Transitions to COMMITTED are protected by trx_t::mutex. */
trx_state_t state;
+ /** whether this is a recovered transaction that should be
+ rolled back by trx_rollback_or_clean_recovered().
+ Protected by trx_t::mutex for transactions that are in trx_sys. */
+ bool is_recovered;
ReadView* read_view; /*!< consistent read view used in the
transaction, or NULL if not yet set */
@@ -895,13 +893,8 @@ struct trx_t {
trx_lock_t lock; /*!< Information about the transaction
locks and state. Protected by
- trx->mutex or lock_sys->mutex
- or both */
- bool is_recovered; /*!< 0=normal transaction,
- 1=recovered, must be rolled back,
- protected by trx_sys->mutex when
- trx->in_rw_trx_list holds */
-
+ lock_sys->mutex (insertions also
+ by trx_t::mutex). */
/* These fields are not protected by any mutex. */
const char* op_info; /*!< English text describing the
@@ -1185,6 +1178,12 @@ public:
return flush_observer;
}
+ /** Transition to committed state, to release implicit locks. */
+ inline void commit_state();
+
+ /** Release any explicit locks of a committing transaction. */
+ inline void release_locks();
+
private:
/** Assign a rollback segment for modifying temporary tables.
@return the assigned rollback segment */
diff --git a/storage/innobase/lock/lock0lock.cc b/storage/innobase/lock/lock0lock.cc
index 8634f7e73c5..49c0e1da743 100644
--- a/storage/innobase/lock/lock0lock.cc
+++ b/storage/innobase/lock/lock0lock.cc
@@ -1273,30 +1273,31 @@ lock_rec_other_trx_holds_expl(
trx_t* holds = NULL;
lock_mutex_enter();
+ mutex_enter(&trx_sys->mutex);
+ trx_mutex_enter(trx);
- if (trx_t* impl_trx = trx_rw_is_active(trx->id, NULL, false)) {
- ulint heap_no = page_rec_get_heap_no(rec);
- mutex_enter(&trx_sys->mutex);
+ ut_ad(!trx_state_eq(trx, TRX_STATE_NOT_STARTED));
+ if (!trx_state_eq(trx, TRX_STATE_COMMITTED_IN_MEMORY)) {
+ const ulint heap_no = page_rec_get_heap_no(rec);
for (trx_t* t = UT_LIST_GET_FIRST(trx_sys->rw_trx_list);
t != NULL;
t = UT_LIST_GET_NEXT(trx_list, t)) {
lock_t* expl_lock = lock_rec_has_expl(
precise_mode, block, heap_no, t);
-
- if (expl_lock && expl_lock->trx != impl_trx) {
+ if (expl_lock && expl_lock->trx != trx) {
/* An explicit lock is held by trx other than
the trx holding the implicit lock. */
holds = expl_lock->trx;
break;
}
}
-
- mutex_exit(&trx_sys->mutex);
}
lock_mutex_exit();
+ mutex_exit(&trx_sys->mutex);
+ trx_mutex_exit(trx);
return(holds);
}
@@ -5107,7 +5108,7 @@ lock_trx_table_locks_find(
{
bool found = false;
- trx_mutex_enter(trx);
+ ut_ad(trx_mutex_own(trx));
for (lock_list::const_iterator it = trx->lock.table_locks.begin(),
end = trx->lock.table_locks.end(); it != end; ++it) {
@@ -5130,8 +5131,6 @@ lock_trx_table_locks_find(
ut_a(lock->un_member.tab_lock.table != NULL);
}
- trx_mutex_exit(trx);
-
return(found);
}
@@ -5155,21 +5154,20 @@ lock_table_queue_validate(
/* lock->trx->state cannot change from or to NOT_STARTED
while we are holding the trx_sys->mutex. It may change
- from ACTIVE to PREPARED, but it may not change to
- COMMITTED, because we are holding the lock_sys->mutex. */
+ from ACTIVE or PREPARED to PREPARED or COMMITTED. */
+ trx_mutex_enter(lock->trx);
ut_ad(trx_assert_started(lock->trx));
-
- if (!lock_get_wait(lock)) {
-
+ if (trx_state_eq(lock->trx, TRX_STATE_COMMITTED_IN_MEMORY)) {
+ } else if (!lock_get_wait(lock)) {
ut_a(!lock_table_other_has_incompatible(
lock->trx, 0, table,
lock_get_mode(lock)));
} else {
-
ut_a(lock_table_has_to_wait_in_queue(lock));
}
ut_a(lock_trx_table_locks_find(lock->trx, lock));
+ trx_mutex_exit(lock->trx);
}
return(TRUE);
@@ -5191,7 +5189,6 @@ lock_rec_queue_validate(
const dict_index_t* index, /*!< in: index, or NULL if not known */
const ulint* offsets)/*!< in: rec_get_offsets(rec, index) */
{
- const trx_t* impl_trx;
const lock_t* lock;
ulint heap_no;
@@ -5217,40 +5214,34 @@ lock_rec_queue_validate(
lock != NULL;
lock = lock_rec_get_next_const(heap_no, lock)) {
- ut_ad(!trx_is_ac_nl_ro(lock->trx));
+ ut_ad(!index || lock->index == index);
- if (lock_get_wait(lock)) {
- ut_a(lock_rec_has_to_wait_in_queue(lock));
- }
-
- if (index != NULL) {
- ut_a(lock->index == index);
- }
+ trx_mutex_enter(lock->trx);
+ ut_ad(!trx_is_ac_nl_ro(lock->trx));
+ ut_ad(trx_state_eq(lock->trx,
+ TRX_STATE_COMMITTED_IN_MEMORY)
+ || !lock_get_wait(lock)
+ || lock_rec_has_to_wait_in_queue(lock));
+ trx_mutex_exit(lock->trx);
}
goto func_exit;
}
ut_ad(page_rec_is_leaf(rec));
+ ut_ad(lock_mutex_own());
- if (index == NULL) {
-
+ if (!index || !index->is_primary()) {
/* Nothing we can do */
-
- } else if (dict_index_is_clust(index)) {
- trx_id_t trx_id;
-
- /* Unlike the non-debug code, this invariant can only succeed
- if the check and assertion are covered by the lock mutex. */
-
- trx_id = lock_clust_rec_some_has_impl(rec, index, offsets);
- impl_trx = trx_rw_is_active_low(trx_id, NULL);
-
- ut_ad(lock_mutex_own());
- /* impl_trx cannot be committed until lock_mutex_exit()
- because lock_trx_release_locks() acquires lock_sys->mutex */
-
- if (!impl_trx) {
+ } else if (trx_t* impl_trx = trx_rw_is_active_low(
+ lock_clust_rec_some_has_impl(rec, index, offsets),
+ NULL)) {
+ /* impl_trx could have been committed before we
+ acquire its mutex, but not thereafter. */
+
+ mutex_enter(&impl_trx->mutex);
+ ut_ad(impl_trx->state != TRX_STATE_NOT_STARTED);
+ if (impl_trx->state == TRX_STATE_COMMITTED_IN_MEMORY) {
} else if (const lock_t* other_lock
= lock_rec_other_has_expl_req(
LOCK_S, block, true, heap_no,
@@ -5292,6 +5283,8 @@ lock_rec_queue_validate(
ut_ad(lock_rec_has_expl(LOCK_X | LOCK_REC_NOT_GAP,
block, heap_no, impl_trx));
}
+
+ mutex_exit(&impl_trx->mutex);
}
for (lock = lock_rec_get_first(lock_sys->rec_hash, block, heap_no);
@@ -5779,29 +5772,23 @@ lock_rec_convert_impl_to_expl_for_trx(
trx_t* trx, /*!< in/out: active transaction */
ulint heap_no)/*!< in: rec heap number to lock */
{
- ut_ad(trx_is_referenced(trx));
-
DEBUG_SYNC_C("before_lock_rec_convert_impl_to_expl_for_trx");
lock_mutex_enter();
-
+ trx_mutex_enter(trx);
ut_ad(!trx_state_eq(trx, TRX_STATE_NOT_STARTED));
if (!trx_state_eq(trx, TRX_STATE_COMMITTED_IN_MEMORY)
&& !lock_rec_has_expl(LOCK_X | LOCK_REC_NOT_GAP,
block, heap_no, trx)) {
-
- ulint type_mode;
-
- type_mode = (LOCK_REC | LOCK_X | LOCK_REC_NOT_GAP);
-
- lock_rec_add_to_queue(
- type_mode, block, heap_no, index, trx, FALSE);
+ lock_rec_add_to_queue(LOCK_REC | LOCK_X | LOCK_REC_NOT_GAP,
+ block, heap_no, index, trx, true);
}
lock_mutex_exit();
-
- trx_release_reference(trx);
+ ut_ad(trx->n_ref > 0);
+ --trx->n_ref;
+ trx_mutex_exit(trx);
DEBUG_SYNC_C("after_lock_rec_convert_impl_to_expl_for_trx");
}
@@ -6541,118 +6528,24 @@ lock_unlock_table_autoinc(
}
}
-/*********************************************************************//**
-Releases a transaction's locks, and releases possible other transactions
-waiting because of these locks. Change the state of the transaction to
-TRX_STATE_COMMITTED_IN_MEMORY. */
-void
-lock_trx_release_locks(
-/*===================*/
- trx_t* trx) /*!< in/out: transaction */
+/** Release the explicit locks of a committing transaction,
+and release possible other transactions waiting because of these locks. */
+void lock_trx_release_locks(trx_t* trx)
{
- check_trx_state(trx);
-
- ut_ad(trx_state_eq(trx, TRX_STATE_ACTIVE)
- || trx_state_eq(trx, TRX_STATE_PREPARED)
- || trx_state_eq(trx, TRX_STATE_PREPARED_RECOVERED)
- || (trx_state_eq(trx, TRX_STATE_COMMITTED_IN_MEMORY)
- && trx->is_recovered
- && !UT_LIST_GET_LEN(trx->lock.trx_locks)));
-
- bool release_lock;
-
- release_lock = (UT_LIST_GET_LEN(trx->lock.trx_locks) > 0);
-
- /* Don't take lock_sys mutex if trx didn't acquire any lock. */
- if (release_lock) {
-
- /* The transition of trx->state to TRX_STATE_COMMITTED_IN_MEMORY
- is protected by both the lock_sys->mutex and the trx->mutex. */
- lock_mutex_enter();
- }
-
- trx_mutex_enter(trx);
-
- /* The following assignment makes the transaction committed in memory
- and makes its changes to data visible to other transactions.
- NOTE that there is a small discrepancy from the strict formal
- visibility rules here: a human user of the database can see
- modifications made by another transaction T even before the necessary
- log segment has been flushed to the disk. If the database happens to
- crash before the flush, the user has seen modifications from T which
- will never be a committed transaction. However, any transaction T2
- which sees the modifications of the committing transaction T, and
- which also itself makes modifications to the database, will get an lsn
- larger than the committing transaction T. In the case where the log
- flush fails, and T never gets committed, also T2 will never get
- committed. */
-
- /*--------------------------------------*/
- trx->state = TRX_STATE_COMMITTED_IN_MEMORY;
- /*--------------------------------------*/
-
- if (trx_is_referenced(trx)) {
-
- ut_a(release_lock);
-
- lock_mutex_exit();
-
- while (trx_is_referenced(trx)) {
-
- trx_mutex_exit(trx);
-
- DEBUG_SYNC_C("waiting_trx_is_not_referenced");
-
- /** Doing an implicit to explicit conversion
- should not be expensive. */
- ut_delay(ut_rnd_interval(0, srv_spin_wait_delay));
-
- trx_mutex_enter(trx);
- }
-
- trx_mutex_exit(trx);
-
- lock_mutex_enter();
-
- trx_mutex_enter(trx);
- }
-
- ut_ad(!trx_is_referenced(trx));
-
- /* If the background thread trx_rollback_or_clean_recovered()
- is still active then there is a chance that the rollback
- thread may see this trx as COMMITTED_IN_MEMORY and goes ahead
- to clean it up calling trx_cleanup_at_db_startup(). This can
- happen in the case we are committing a trx here that is left
- in PREPARED state during the crash. Note that commit of the
- rollback of a PREPARED trx happens in the recovery thread
- while the rollback of other transactions happen in the
- background thread. To avoid this race we unconditionally unset
- the is_recovered flag. */
-
- trx->is_recovered = false;
-
- trx_mutex_exit(trx);
-
- if (release_lock) {
-
- lock_release(trx);
-
- lock_mutex_exit();
- }
+ ut_ad(UT_LIST_GET_LEN(trx->lock.trx_locks));
+ lock_mutex_enter();
+ lock_release(trx);
trx->lock.n_rec_locks = 0;
-
/* We don't remove the locks one by one from the vector for
efficiency reasons. We simply reset it because we would have
released all the locks anyway. */
trx->lock.table_locks.clear();
- ut_a(UT_LIST_GET_LEN(trx->lock.trx_locks) == 0);
- ut_a(ib_vector_is_empty(trx->autoinc_locks));
- ut_a(trx->lock.table_locks.empty());
-
+ ut_ad(UT_LIST_GET_LEN(trx->lock.trx_locks) == 0);
+ ut_ad(ib_vector_is_empty(trx->autoinc_locks));
+ lock_mutex_exit();
mem_heap_empty(trx->lock.lock_heap);
}
@@ -6724,18 +6617,16 @@ lock_table_locks_lookup(
itself */
const trx_ut_list_t* trx_list) /*!< in: trx list to check */
{
- trx_t* trx;
-
ut_a(table != NULL);
ut_ad(lock_mutex_own());
ut_ad(trx_sys_mutex_own());
- for (trx = UT_LIST_GET_FIRST(*trx_list);
+ for (trx_t* trx = UT_LIST_GET_FIRST(*trx_list);
trx != NULL;
trx = UT_LIST_GET_NEXT(trx_list, trx)) {
+ const lock_t* lock;
- const lock_t* lock;
-
+ trx_mutex_enter(trx);
check_trx_state(trx);
for (lock = UT_LIST_GET_FIRST(trx->lock.trx_locks);
@@ -6748,15 +6639,21 @@ lock_table_locks_lookup(
ut_ad(!dict_index_is_online_ddl(lock->index)
|| dict_index_is_clust(lock->index));
if (lock->index->table == table) {
- return(lock);
+ break;
}
} else if (lock->un_member.tab_lock.table == table) {
- return(lock);
+ break;
}
}
+
+ trx_mutex_exit(trx);
+
+ if (lock) {
+ return lock;
+ }
}
- return(NULL);
+ return NULL;
}
#endif /* UNIV_DEBUG */
@@ -6921,7 +6818,7 @@ DeadlockChecker::start_print()
if (srv_print_all_deadlocks) {
ib::info() << "Transactions deadlock detected, dumping"
- << " detailed information.";
+ " detailed information.";
}
}
diff --git a/storage/innobase/row/row0vers.cc b/storage/innobase/row/row0vers.cc
index 7db1b936348..533d6df1321 100644
--- a/storage/innobase/row/row0vers.cc
+++ b/storage/innobase/row/row0vers.cc
@@ -76,7 +76,8 @@ index record.
@param[in] index secondary index
@param[in] offsets rec_get_offsets(rec, index)
@param[in,out] mtr mini-transaction
-@return the active transaction; trx_release_reference() must be invoked
+@return the active transaction; state must be rechecked after
+trx_mutex_enter(), and trx_release_reference() must be invoked
@retval NULL if the record was committed */
UNIV_INLINE
trx_t*
@@ -88,11 +89,6 @@ row_vers_impl_x_locked_low(
const ulint* offsets,
mtr_t* mtr)
{
- trx_id_t trx_id;
- ibool corrupt;
- ulint comp;
- ulint rec_del;
- const rec_t* version;
rec_t* prev_version = NULL;
ulint* clust_offsets;
mem_heap_t* heap;
@@ -109,11 +105,12 @@ row_vers_impl_x_locked_low(
clust_offsets = rec_get_offsets(
clust_rec, clust_index, NULL, true, ULINT_UNDEFINED, &heap);
- trx_id = row_get_rec_trx_id(clust_rec, clust_index, clust_offsets);
- corrupt = FALSE;
+ const trx_id_t trx_id = row_get_rec_trx_id(
+ clust_rec, clust_index, clust_offsets);
- ut_ad(!dict_table_is_temporary(clust_index->table));
+ ut_ad(!clust_index->table->is_temporary());
+ bool corrupt = false;
trx_t* trx = trx_rw_is_active(trx_id, &corrupt, true);
if (trx == 0) {
@@ -128,12 +125,12 @@ row_vers_impl_x_locked_low(
DBUG_RETURN(0);
}
- comp = page_rec_is_comp(rec);
+ const ulint comp = page_rec_is_comp(rec);
ut_ad(index->table == clust_index->table);
ut_ad(!!comp == dict_table_is_comp(index->table));
ut_ad(!comp == !page_rec_is_comp(clust_rec));
- rec_del = rec_get_deleted_flag(rec, comp);
+ const ulint rec_del = rec_get_deleted_flag(rec, comp);
if (dict_index_has_virtual(index)) {
ulint n_ext;
@@ -158,7 +155,7 @@ row_vers_impl_x_locked_low(
modify rec, and does not necessarily have an implicit x-lock
on rec. */
- for (version = clust_rec;; version = prev_version) {
+ for (const rec_t* version = clust_rec;; version = prev_version) {
row_ext_t* ext;
dtuple_t* row;
dtuple_t* entry;
@@ -178,16 +175,24 @@ row_vers_impl_x_locked_low(
heap, &prev_version, NULL,
dict_index_has_virtual(index) ? &vrow : NULL, 0);
+ trx_mutex_enter(trx);
+ const bool committed = trx_state_eq(
+ trx, TRX_STATE_COMMITTED_IN_MEMORY);
+ trx_mutex_exit(trx);
+
/* The oldest visible clustered index version must not be
delete-marked, because we never start a transaction by
inserting a delete-marked record. */
- ut_ad(prev_version
- || !rec_get_deleted_flag(version, comp)
- || !trx_rw_is_active(trx_id, NULL, false));
+ ut_ad(committed || prev_version
+ || !rec_get_deleted_flag(version, comp));
/* Free version and clust_offsets. */
mem_heap_free(old_heap);
+ if (committed) {
+ goto not_locked;
+ }
+
if (prev_version == NULL) {
/* We reached the oldest visible version without
@@ -207,6 +212,7 @@ row_vers_impl_x_locked_low(
or updated, the leaf page record always is
created with a clear delete-mark flag.
(We never insert a delete-marked record.) */
+not_locked:
trx_release_reference(trx);
trx = 0;
}
@@ -333,14 +339,14 @@ result_check:
if (trx->id != prev_trx_id) {
/* prev_version was the first version modified by
the trx_id transaction: no implicit x-lock */
-
- trx_release_reference(trx);
- trx = 0;
- break;
+ goto not_locked;
}
}
- DBUG_PRINT("info", ("Implicit lock is held by trx:" TRX_ID_FMT, trx_id));
+ if (trx) {
+ DBUG_PRINT("info", ("Implicit lock is held by trx:" TRX_ID_FMT,
+ trx_id));
+ }
if (v_heap != NULL) {
mem_heap_free(v_heap);
@@ -355,7 +361,8 @@ index record.
@param[in] rec secondary index record
@param[in] index secondary index
@param[in] offsets rec_get_offsets(rec, index)
-@return the active transaction; trx_release_reference() must be invoked
+@return the active transaction; state must be rechecked after
+trx_mutex_enter(), and trx_release_reference() must be invoked
@retval NULL if the record was committed */
trx_t*
row_vers_impl_x_locked(
diff --git a/storage/innobase/trx/trx0roll.cc b/storage/innobase/trx/trx0roll.cc
index e827cda46eb..20798323802 100644
--- a/storage/innobase/trx/trx0roll.cc
+++ b/storage/innobase/trx/trx0roll.cc
@@ -697,9 +697,9 @@ trx_rollback_resurrected(
ut_ad(trx_sys_mutex_own());
/* The trx->is_recovered flag and trx->state are set
- atomically under the protection of the trx->mutex (and
- lock_sys->mutex) in lock_trx_release_locks(). We do not want
- to accidentally clean up a non-recovered transaction here. */
+ atomically under the protection of the trx->mutex in
+ trx_t::commit_state(). We do not want to accidentally clean up
+ a non-recovered transaction here. */
trx_mutex_enter(trx);
if (!trx->is_recovered) {
diff --git a/storage/innobase/trx/trx0trx.cc b/storage/innobase/trx/trx0trx.cc
index d6fbc65ea55..40dcc37b30b 100644
--- a/storage/innobase/trx/trx0trx.cc
+++ b/storage/innobase/trx/trx0trx.cc
@@ -519,6 +519,52 @@ trx_free_for_background(trx_t* trx)
trx_free(trx);
}
+/** Transition to committed state, to release implicit locks. */
+inline void trx_t::commit_state()
+{
+ /* This makes the transaction committed in memory and makes its
+ changes to data visible to other transactions. NOTE that there is a
+ small discrepancy from the strict formal visibility rules here: a
+ user of the database can see modifications made by another
+ transaction T even before the necessary redo log segment has been
+ flushed to the disk. If the database happens to crash before the
+ flush, the user has seen modifications from T which will never be a
+ committed transaction. However, any transaction T2 which sees the
+ modifications of the committing transaction T, and which also itself
+ makes modifications to the database, will get an lsn larger than the
+ committing transaction T. In the case where the log flush fails, and
+ T never gets committed, also T2 will never get committed. */
+ ut_ad(trx_mutex_own(this));
+ ut_ad(state != TRX_STATE_NOT_STARTED);
+ ut_ad(state != TRX_STATE_COMMITTED_IN_MEMORY
+ || (is_recovered && !UT_LIST_GET_LEN(lock.trx_locks)));
+ state= TRX_STATE_COMMITTED_IN_MEMORY;
+
+ /* If the background thread trx_rollback_or_clean_recovered()
+ is still active then there is a chance that the rollback
+ thread may see this trx as COMMITTED_IN_MEMORY and goes ahead
+ to clean it up calling trx_cleanup_at_db_startup(). This can
+ happen in the case we are committing a trx here that is left
+ in PREPARED state during the crash. Note that commit of the
+ rollback of a PREPARED trx happens in the recovery thread
+ while the rollback of other transactions happen in the
+ background thread. To avoid this race we unconditionally unset
+ the is_recovered flag. */
+ is_recovered= false;
+ ut_ad(id || !trx_is_referenced(this));
+}
+
+/** Release any explicit locks of a committing transaction. */
+inline void trx_t::release_locks()
+{
+ DBUG_ASSERT(state == TRX_STATE_COMMITTED_IN_MEMORY);
+
+ if (UT_LIST_GET_LEN(lock.trx_locks))
+ lock_trx_release_locks(this);
+ else
+ lock.table_locks.clear(); /* Work around MDEV-20483 */
+}
+
/********************************************************************//**
At shutdown, frees a transaction object that is in the PREPARED state. */
void
@@ -526,6 +572,7 @@ trx_free_prepared(
/*==============*/
trx_t* trx) /*!< in, own: trx object */
{
+ trx_mutex_enter(trx);
ut_ad(trx->state == TRX_STATE_PREPARED
|| trx->state == TRX_STATE_PREPARED_RECOVERED
|| !srv_was_started
@@ -543,7 +590,9 @@ trx_free_prepared(
|| srv_force_recovery >= SRV_FORCE_NO_TRX_UNDO)));
ut_a(trx->magic_n == TRX_MAGIC_N);
- lock_trx_release_locks(trx);
+ trx->commit_state();
+ trx_mutex_exit(trx);
+ trx->release_locks();
trx_undo_free_prepared(trx);
assert_trx_in_rw_list(trx);
@@ -556,14 +605,7 @@ trx_free_prepared(
DBUG_LOG("trx", "Free prepared: " << trx);
trx->state = TRX_STATE_NOT_STARTED;
-
- /* Undo trx_resurrect_table_locks(). */
- lock_trx_lock_list_init(&trx->lock.trx_locks);
-
- /* Note: This vector is not guaranteed to be empty because the
- transaction was never committed and therefore lock_trx_release()
- was not called. */
- trx->lock.table_locks.clear();
+ ut_ad(!UT_LIST_GET_LEN(trx->lock.trx_locks));
trx->id = 0;
trx_free(trx);
@@ -1630,8 +1672,8 @@ trx_commit_in_memory(
/* 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 not) be added
- becuase it is flagged as a non-locking read-only transaction. */
+ 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);
@@ -1653,35 +1695,43 @@ trx_commit_in_memory(
DBUG_LOG("trx", "Autocommit in memory: " << trx);
trx->state = TRX_STATE_NOT_STARTED;
} else {
- if (trx->id > 0) {
- /* For consistent snapshot, we need to remove current
- transaction from running transaction id list for mvcc
- before doing commit and releasing locks. */
+ trx_mutex_enter(trx);
+ trx->commit_state();
+ trx_mutex_exit(trx);
+
+ if (trx->id) {
trx_erase_lists(trx, serialised);
- }
- lock_trx_release_locks(trx);
- ut_ad(trx->read_only || !trx->rsegs.m_redo.rseg || trx->id);
+ /* Wait for any implicit-to-explicit lock
+ conversions to cease, so that there will be no
+ race condition in lock_release(). */
+ trx_mutex_enter(trx);
+ while (UNIV_UNLIKELY(trx_is_referenced(trx))) {
+ trx_mutex_exit(trx);
+ ut_delay(srv_spin_wait_delay);
+ trx_mutex_enter(trx);
+ }
+ trx_mutex_exit(trx);
- /* Remove the transaction from the list of active
- transactions now that it no longer holds any user locks. */
+ trx->release_locks();
+ trx->id = 0;
+ } else {
+ ut_ad(trx->read_only || !trx->rsegs.m_redo.rseg);
+ ut_ad(!trx->in_rw_trx_list);
+ trx->release_locks();
+ }
- ut_ad(trx_state_eq(trx, TRX_STATE_COMMITTED_IN_MEMORY));
DEBUG_SYNC_C("after_trx_committed_in_memory");
- if (trx->read_only || trx->rsegs.m_redo.rseg == NULL) {
-
+ if (trx->read_only || !trx->rsegs.m_redo.rseg) {
MONITOR_INC(MONITOR_TRX_RO_COMMIT);
- if (trx->read_view != NULL) {
+ if (trx->read_view) {
trx_sys->mvcc->view_close(
trx->read_view, false);
}
-
} else {
MONITOR_INC(MONITOR_TRX_RW_COMMIT);
}
-
- trx->id = 0;
}
ut_ad(!trx->rsegs.m_redo.update_undo);
@@ -2723,18 +2773,13 @@ partial:
return(int (count));
}
-/*******************************************************************//**
-This function is used to find one X/Open XA distributed transaction
-which is in the prepared state
+/** Look up an X/Open distributed transaction in XA PREPARE state.
+@param[in] xid X/Open XA transaction identifier
@return trx on match, the trx->xid will be invalidated;
-note that the trx may have been committed, unless the caller is
-holding lock_sys->mutex */
+note that the trx may have been committed before the caller
+acquires trx_t::mutex */
static MY_ATTRIBUTE((warn_unused_result))
-trx_t*
-trx_get_trx_by_xid_low(
-/*===================*/
- XID* xid) /*!< in: X/Open XA transaction
- identifier */
+trx_t* trx_get_trx_by_xid_low(const XID* xid)
{
trx_t* trx;
@@ -2743,7 +2788,7 @@ trx_get_trx_by_xid_low(
for (trx = UT_LIST_GET_FIRST(trx_sys->rw_trx_list);
trx != NULL;
trx = UT_LIST_GET_NEXT(trx_list, trx)) {
-
+ trx_mutex_enter(trx);
assert_trx_in_rw_list(trx);
/* Compare two X/Open XA transaction id's: their
@@ -2759,28 +2804,28 @@ trx_get_trx_by_xid_low(
/* The commit of a prepared recovered Galera
transaction needs a valid trx->xid for
invoking trx_sys_update_wsrep_checkpoint(). */
- if (wsrep_is_wsrep_xid(trx->xid)) break;
+ if (!wsrep_is_wsrep_xid(trx->xid))
#endif
- /* Invalidate the XID, so that subsequent calls
- will not find it. */
- trx->xid->null();
+ /* Invalidate the XID, so that subsequent calls
+ will not find it. */
+ trx->xid->null();
+ trx_mutex_exit(trx);
break;
}
+
+ trx_mutex_exit(trx);
}
return(trx);
}
-/*******************************************************************//**
-This function is used to find one X/Open XA distributed transaction
-which is in the prepared state
-@return trx or NULL; on match, the trx->xid will be invalidated;
-note that the trx may have been committed, unless the caller is
-holding lock_sys->mutex */
-trx_t*
-trx_get_trx_by_xid(
-/*===============*/
- XID* xid) /*!< in: X/Open XA transaction identifier */
+/** Look up an X/Open distributed transaction in XA PREPARE state.
+@param[in] xid X/Open XA transaction identifier
+@return transaction on match (the trx_t::xid will be invalidated);
+note that the trx may have been committed before the caller acquires
+trx_t::mutex
+@retval NULL if no match */
+trx_t* trx_get_trx_by_xid(const XID* xid)
{
trx_t* trx;
@@ -2793,7 +2838,7 @@ trx_get_trx_by_xid(
/* Recovered/Resurrected transactions are always only on the
trx_sys_t::rw_trx_list. */
- trx = trx_get_trx_by_xid_low((XID*)xid);
+ trx = trx_get_trx_by_xid_low(xid);
trx_sys_mutex_exit();
diff --git a/storage/innobase/trx/trx0undo.cc b/storage/innobase/trx/trx0undo.cc
index 7ff6ae8fc3f..336506c7b65 100644
--- a/storage/innobase/trx/trx0undo.cc
+++ b/storage/innobase/trx/trx0undo.cc
@@ -1820,7 +1820,7 @@ trx_undo_free_prepared(
TRX_STATE_COMMITTED_IN_MEMORY));
/* fall through */
case TRX_UNDO_ACTIVE:
- /* lock_trx_release_locks() assigns
+ /* trx_t::commit_state() assigns
trx->is_recovered=false and
trx->state = TRX_STATE_COMMITTED_IN_MEMORY,
also for transactions that we faked
@@ -1852,7 +1852,7 @@ trx_undo_free_prepared(
TRX_STATE_COMMITTED_IN_MEMORY));
/* fall through */
case TRX_UNDO_ACTIVE:
- /* lock_trx_release_locks() assigns
+ /* trx_t::commit_state() assigns
trx->is_recovered=false and
trx->state = TRX_STATE_COMMITTED_IN_MEMORY,
also for transactions that we faked