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

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().
parent 7c79c127
...@@ -575,9 +575,9 @@ struct xid_t { ...@@ -575,9 +575,9 @@ struct xid_t {
char data[XIDDATASIZE]; // not \0-terminated ! char data[XIDDATASIZE]; // not \0-terminated !
xid_t() {} /* Remove gcc warning */ xid_t() {} /* Remove gcc warning */
bool eq(struct xid_t *xid) bool eq(struct xid_t *xid) const
{ return !xid->is_null() && eq(xid->gtrid_length, xid->bqual_length, xid->data); } { return !xid->is_null() && eq(xid->gtrid_length, xid->bqual_length, xid->data); }
bool eq(long g, long b, const char *d) bool eq(long g, long b, const char *d) const
{ return !is_null() && g == gtrid_length && b == bqual_length && !memcmp(d, data, g+b); } { return !is_null() && g == gtrid_length && b == bqual_length && !memcmp(d, data, g+b); }
void set(struct xid_t *xid) void set(struct xid_t *xid)
{ memcpy(this, xid, xid->length()); } { memcpy(this, xid, xid->length()); }
......
...@@ -499,14 +499,11 @@ lock_rec_unlock( ...@@ -499,14 +499,11 @@ lock_rec_unlock(
const buf_block_t* block, /*!< in: buffer block containing rec */ const buf_block_t* block, /*!< in: buffer block containing rec */
const rec_t* rec, /*!< in: record */ const rec_t* rec, /*!< in: record */
lock_mode lock_mode);/*!< in: LOCK_S or LOCK_X */ lock_mode lock_mode);/*!< in: LOCK_S or LOCK_X */
/*********************************************************************//**
Releases a transaction's locks, and releases possible other transactions /** Release the explicit locks of a committing transaction,
waiting because of these locks. Change the state of the transaction to and release possible other transactions waiting because of these locks. */
TRX_STATE_COMMITTED_IN_MEMORY. */ void lock_trx_release_locks(trx_t* trx);
void
lock_trx_release_locks(
/*===================*/
trx_t* trx); /*!< in/out: transaction */
/*********************************************************************//** /*********************************************************************//**
Removes locks on a table to be dropped or discarded. 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 If remove_also_table_sx_locks is TRUE then table-level S and X locks are
......
...@@ -43,7 +43,8 @@ index record. ...@@ -43,7 +43,8 @@ index record.
@param[in] rec secondary index record @param[in] rec secondary index record
@param[in] index secondary index @param[in] index secondary index
@param[in] offsets rec_get_offsets(rec, 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 */ @retval NULL if the record was committed */
trx_t* trx_t*
row_vers_impl_x_locked( row_vers_impl_x_locked(
......
/***************************************************************************** /*****************************************************************************
Copyright (c) 1996, 2016, Oracle and/or its affiliates. All Rights Reserved. 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 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 the terms of the GNU General Public License as published by the Free Software
...@@ -186,30 +186,21 @@ UNIV_INLINE ...@@ -186,30 +186,21 @@ UNIV_INLINE
trx_id_t trx_id_t
trx_rw_min_trx_id(void); trx_rw_min_trx_id(void);
/*===================*/ /*===================*/
/****************************************************************//** /** Look up a rw transaction with the given id.
Checks if a rw transaction with the given id is active. @param[in] trx_id transaction identifier
@return transaction instance if active, or NULL */ @param[out] corrupt flag that will be set if trx_id is corrupted
UNIV_INLINE @return transaction; its state should be rechecked after acquiring trx_t::mutex
trx_t* @retval NULL if there is no transaction identified by trx_id. */
trx_rw_is_active_low( inline trx_t* trx_rw_is_active_low(trx_id_t trx_id, bool* corrupt);
/*=================*/
trx_id_t trx_id, /*!< in: trx id of the transaction */ /** Look up a rw transaction with the given id.
ibool* corrupt); /*!< in: NULL or pointer to a flag @param[in] trx_id transaction identifier
that will be set if corrupt */ @param[out] corrupt flag that will be set if trx_id is corrupted
/****************************************************************//** @param[in] ref_count whether to increment trx->n_ref
Checks if a rw transaction with the given id is active. If the caller is @return transaction; its state should be rechecked after acquiring trx_t::mutex
not holding trx_sys->mutex, the transaction may already have been @retval NULL if there is no active transaction identified by trx_id. */
committed. inline trx_t* trx_rw_is_active(trx_id_t trx_id, bool* corrupt, bool ref_count);
@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 */
#if defined UNIV_DEBUG || defined UNIV_BLOB_LIGHT_DEBUG #if defined UNIV_DEBUG || defined UNIV_BLOB_LIGHT_DEBUG
/***********************************************************//** /***********************************************************//**
Assert that a transaction has been recovered. Assert that a transaction has been recovered.
......
/***************************************************************************** /*****************************************************************************
Copyright (c) 1996, 2015, Oracle and/or its affiliates. All Rights Reserved. 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 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 the terms of the GNU General Public License as published by the Free Software
...@@ -302,61 +302,38 @@ trx_rw_min_trx_id(void) ...@@ -302,61 +302,38 @@ trx_rw_min_trx_id(void)
return(id); return(id);
} }
/****************************************************************//** /** Look up a rw transaction with the given id.
Checks if a rw transaction with the given id is active. If the caller is @param[in] trx_id transaction identifier
not holding lock_sys->mutex, the transaction may already have been committed. @param[out] corrupt flag that will be set if trx_id is corrupted
@return transaction instance if active, or NULL */ @return transaction; its state should be rechecked after acquiring trx_t::mutex
UNIV_INLINE @retval NULL if there is no transaction identified by trx_id. */
trx_t* inline trx_t* trx_rw_is_active_low(trx_id_t trx_id, bool* corrupt)
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 */
{ {
trx_t* trx;
ut_ad(trx_sys_mutex_own()); ut_ad(trx_sys_mutex_own());
if (trx_id < trx_rw_min_trx_id_low()) { if (trx_id < trx_rw_min_trx_id_low()) {
trx = NULL;
} else if (trx_id >= trx_sys->max_trx_id) { } else if (trx_id >= trx_sys->max_trx_id) {
/* There must be corruption: we let the caller handle the /* There must be corruption: we let the caller handle the
diagnostic prints in this case. */ diagnostic prints in this case. */
trx = NULL;
if (corrupt != NULL) { if (corrupt != NULL) {
*corrupt = TRUE; *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;
} }
} else if (trx_t* trx = trx_get_rw_trx_by_id(trx_id)) {
return trx;
} }
return(trx); return NULL;
} }
/****************************************************************//** /** Look up a rw transaction with the given id.
Checks if a rw transaction with the given id is active. If the caller is @param[in] trx_id transaction identifier
not holding lock_sys->mutex, the transaction may already have been @param[out] corrupt flag that will be set if trx_id is corrupted
committed. @param[in] ref_count whether to increment trx->n_ref
@return transaction instance if active, or NULL; */ @return transaction; its state should be rechecked after acquiring trx_t::mutex
UNIV_INLINE @retval NULL if there is no active transaction identified by trx_id. */
trx_t* inline trx_t* trx_rw_is_active(trx_id_t trx_id, bool* corrupt, bool ref_count)
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 */
{ {
ut_ad(trx_id); ut_ad(trx_id);
...@@ -364,13 +341,25 @@ trx_rw_is_active( ...@@ -364,13 +341,25 @@ trx_rw_is_active(
trx_t* trx = trx_rw_is_active_low(trx_id, corrupt); trx_t* trx = trx_rw_is_active_low(trx_id, corrupt);
if (trx && do_ref_count) { if (trx && ref_count) {
trx_mutex_enter(trx); TrxMutex* trx_mutex = &trx->mutex;
ut_ad(!trx_state_eq(trx, TRX_STATE_COMMITTED_IN_MEMORY)); mutex_enter(trx_mutex);
ut_ad(!trx_state_eq(trx, TRX_STATE_NOT_STARTED));
ut_ad(trx->id == trx_id); ut_ad(trx->id == trx_id);
ut_ad(trx->n_ref >= 0); ut_ad(trx->n_ref >= 0);
++trx->n_ref; if (trx_state_eq(trx, TRX_STATE_COMMITTED_IN_MEMORY)) {
trx_mutex_exit(trx); /* 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(); trx_sys_mutex_exit();
......
...@@ -249,16 +249,13 @@ trx_recover_for_mysql( ...@@ -249,16 +249,13 @@ trx_recover_for_mysql(
/*==================*/ /*==================*/
XID* xid_list, /*!< in/out: prepared transactions */ XID* xid_list, /*!< in/out: prepared transactions */
ulint len); /*!< in: number of slots in xid_list */ ulint len); /*!< in: number of slots in xid_list */
/*******************************************************************//** /** Look up an X/Open distributed transaction in XA PREPARE state.
This function is used to find one X/Open XA distributed transaction @param[in] xid X/Open XA transaction identifier
which is in the prepared state @return transaction on match (the trx_t::xid will be invalidated);
@return trx or NULL; on match, the trx->xid will be invalidated; note that the trx may have been committed before the caller acquires
note that the trx may have been committed, unless the caller is trx_t::mutex
holding lock_sys->mutex */ @retval NULL if no match */
trx_t * trx_t* trx_get_trx_by_xid(const XID* xid);
trx_get_trx_by_xid(
/*===============*/
XID* xid); /*!< in: X/Open XA transaction identifier */
/**********************************************************************//** /**********************************************************************//**
If required, flushes the log to disk if we called trx_commit_for_mysql() If required, flushes the log to disk if we called trx_commit_for_mysql()
with trx->flush_log_later == TRUE. */ with trx->flush_log_later == TRUE. */
...@@ -578,6 +575,9 @@ Check transaction state */ ...@@ -578,6 +575,9 @@ Check transaction state */
ut_ad(!MVCC::is_view_active((t)->read_view)); \ ut_ad(!MVCC::is_view_active((t)->read_view)); \
ut_ad((t)->lock.wait_thr == NULL); \ ut_ad((t)->lock.wait_thr == NULL); \
ut_ad(UT_LIST_GET_LEN((t)->lock.trx_locks) == 0); \ 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); \ ut_ad((t)->dict_operation == TRX_DICT_OP_NONE); \
} while(0) } while(0)
...@@ -754,8 +754,8 @@ so without holding any mutex. The following are exceptions to this: ...@@ -754,8 +754,8 @@ so without holding any mutex. The following are exceptions to this:
* trx_rollback_resurrected() may access resurrected (connectionless) * trx_rollback_resurrected() may access resurrected (connectionless)
transactions while the system is already processing new user transactions while the system is already processing new user
transactions. The trx_sys->mutex prevents a race condition between it transactions. The trx_sys->mutex and trx->is_recovered prevent
and lock_trx_release_locks() [invoked by trx_commit()]. a race condition between it and trx_commit().
* trx_print_low() may access transactions not associated with the current * trx_print_low() may access transactions not associated with the current
thread. The caller must be holding trx_sys->mutex and lock_sys->mutex. thread. The caller must be holding trx_sys->mutex and lock_sys->mutex.
...@@ -767,7 +767,7 @@ holding trx_sys->mutex exclusively. ...@@ -767,7 +767,7 @@ holding trx_sys->mutex exclusively.
* The locking code (in particular, lock_deadlock_recursive() and * The locking code (in particular, lock_deadlock_recursive() and
lock_rec_convert_impl_to_expl()) will access transactions associated lock_rec_convert_impl_to_expl()) will access transactions associated
to other connections. The locks of transactions are protected by 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.*/ /** Represents an instance of rollback segment along with its state variables.*/
struct trx_undo_ptr_t { struct trx_undo_ptr_t {
...@@ -873,14 +873,12 @@ struct trx_t { ...@@ -873,14 +873,12 @@ struct trx_t {
ACTIVE->COMMITTED is possible when the transaction is in ACTIVE->COMMITTED is possible when the transaction is in
rw_trx_list. rw_trx_list.
Transitions to COMMITTED are protected by both lock_sys->mutex Transitions to COMMITTED are protected by trx_t::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. */
trx_state_t state; 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 ReadView* read_view; /*!< consistent read view used in the
transaction, or NULL if not yet set */ transaction, or NULL if not yet set */
...@@ -895,13 +893,8 @@ struct trx_t { ...@@ -895,13 +893,8 @@ struct trx_t {
trx_lock_t lock; /*!< Information about the transaction trx_lock_t lock; /*!< Information about the transaction
locks and state. Protected by locks and state. Protected by
trx->mutex or lock_sys->mutex lock_sys->mutex (insertions also
or both */ by trx_t::mutex). */
bool is_recovered; /*!< 0=normal transaction,
1=recovered, must be rolled back,
protected by trx_sys->mutex when
trx->in_rw_trx_list holds */
/* These fields are not protected by any mutex. */ /* These fields are not protected by any mutex. */
const char* op_info; /*!< English text describing the const char* op_info; /*!< English text describing the
...@@ -1185,6 +1178,12 @@ struct trx_t { ...@@ -1185,6 +1178,12 @@ struct trx_t {
return flush_observer; 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: private:
/** Assign a rollback segment for modifying temporary tables. /** Assign a rollback segment for modifying temporary tables.
@return the assigned rollback segment */ @return the assigned rollback segment */
......
This diff is collapsed.
...@@ -76,7 +76,8 @@ index record. ...@@ -76,7 +76,8 @@ index record.
@param[in] index secondary index @param[in] index secondary index
@param[in] offsets rec_get_offsets(rec, index) @param[in] offsets rec_get_offsets(rec, index)
@param[in,out] mtr mini-transaction @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 */ @retval NULL if the record was committed */
UNIV_INLINE UNIV_INLINE
trx_t* trx_t*
...@@ -88,11 +89,6 @@ row_vers_impl_x_locked_low( ...@@ -88,11 +89,6 @@ row_vers_impl_x_locked_low(
const ulint* offsets, const ulint* offsets,
mtr_t* mtr) mtr_t* mtr)
{ {
trx_id_t trx_id;
ibool corrupt;
ulint comp;
ulint rec_del;
const rec_t* version;
rec_t* prev_version = NULL; rec_t* prev_version = NULL;
ulint* clust_offsets; ulint* clust_offsets;
mem_heap_t* heap; mem_heap_t* heap;
...@@ -109,11 +105,12 @@ row_vers_impl_x_locked_low( ...@@ -109,11 +105,12 @@ row_vers_impl_x_locked_low(
clust_offsets = rec_get_offsets( clust_offsets = rec_get_offsets(
clust_rec, clust_index, NULL, true, ULINT_UNDEFINED, &heap); clust_rec, clust_index, NULL, true, ULINT_UNDEFINED, &heap);
trx_id = row_get_rec_trx_id(clust_rec, clust_index, clust_offsets); const trx_id_t trx_id = row_get_rec_trx_id(
corrupt = FALSE; 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); trx_t* trx = trx_rw_is_active(trx_id, &corrupt, true);
if (trx == 0) { if (trx == 0) {
...@@ -128,12 +125,12 @@ row_vers_impl_x_locked_low( ...@@ -128,12 +125,12 @@ row_vers_impl_x_locked_low(
DBUG_RETURN(0); 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(index->table == clust_index->table);
ut_ad(!!comp == dict_table_is_comp(index->table)); ut_ad(!!comp == dict_table_is_comp(index->table));
ut_ad(!comp == !page_rec_is_comp(clust_rec)); 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)) { if (dict_index_has_virtual(index)) {
ulint n_ext; ulint n_ext;
...@@ -158,7 +155,7 @@ row_vers_impl_x_locked_low( ...@@ -158,7 +155,7 @@ row_vers_impl_x_locked_low(
modify rec, and does not necessarily have an implicit x-lock modify rec, and does not necessarily have an implicit x-lock
on rec. */ on rec. */
for (version = clust_rec;; version = prev_version) { for (const rec_t* version = clust_rec;; version = prev_version) {
row_ext_t* ext; row_ext_t* ext;
dtuple_t* row; dtuple_t* row;
dtuple_t* entry; dtuple_t* entry;
...@@ -178,16 +175,24 @@ row_vers_impl_x_locked_low( ...@@ -178,16 +175,24 @@ row_vers_impl_x_locked_low(
heap, &prev_version, NULL, heap, &prev_version, NULL,
dict_index_has_virtual(index) ? &vrow : NULL, 0); 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 /* The oldest visible clustered index version must not be
delete-marked, because we never start a transaction by delete-marked, because we never start a transaction by
inserting a delete-marked record. */ inserting a delete-marked record. */
ut_ad(prev_version ut_ad(committed || prev_version
|| !rec_get_deleted_flag(version, comp) || !rec_get_deleted_flag(version, comp));
|| !trx_rw_is_active(trx_id, NULL, false));
/* Free version and clust_offsets. */ /* Free version and clust_offsets. */
mem_heap_free(old_heap); mem_heap_free(old_heap);
if (committed) {
goto not_locked;
}
if (prev_version == NULL) { if (prev_version == NULL) {
/* We reached the oldest visible version without /* We reached the oldest visible version without
...@@ -207,6 +212,7 @@ row_vers_impl_x_locked_low( ...@@ -207,6 +212,7 @@ row_vers_impl_x_locked_low(
or updated, the leaf page record always is or updated, the leaf page record always is
created with a clear delete-mark flag. created with a clear delete-mark flag.
(We never insert a delete-marked record.) */ (We never insert a delete-marked record.) */
not_locked:
trx_release_reference(trx); trx_release_reference(trx);
trx = 0; trx = 0;
} }
...@@ -333,14 +339,14 @@ row_vers_impl_x_locked_low( ...@@ -333,14 +339,14 @@ row_vers_impl_x_locked_low(
if (trx->id != prev_trx_id) { if (trx->id != prev_trx_id) {
/* prev_version was the first version modified by /* prev_version was the first version modified by
the trx_id transaction: no implicit x-lock */ the trx_id transaction: no implicit x-lock */
goto not_locked;
trx_release_reference(trx);
trx = 0;
break;
} }
} }
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) { if (v_heap != NULL) {
mem_heap_free(v_heap); mem_heap_free(v_heap);
...@@ -355,7 +361,8 @@ index record. ...@@ -355,7 +361,8 @@ index record.
@param[in] rec secondary index record @param[in] rec secondary index record
@param[in] index secondary index @param[in] index secondary index
@param[in] offsets rec_get_offsets(rec, 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 */ @retval NULL if the record was committed */
trx_t* trx_t*
row_vers_impl_x_locked( row_vers_impl_x_locked(
......
...@@ -697,9 +697,9 @@ trx_rollback_resurrected( ...@@ -697,9 +697,9 @@ trx_rollback_resurrected(
ut_ad(trx_sys_mutex_own()); ut_ad(trx_sys_mutex_own());
/* The trx->is_recovered flag and trx->state are set /* The trx->is_recovered flag and trx->state are set
atomically under the protection of the trx->mutex (and atomically under the protection of the trx->mutex in
lock_sys->mutex) in lock_trx_release_locks(). We do not want trx_t::commit_state(). We do not want to accidentally clean up
to accidentally clean up a non-recovered transaction here. */ a non-recovered transaction here. */
trx_mutex_enter(trx); trx_mutex_enter(trx);
if (!trx->is_recovered) { if (!trx->is_recovered) {
......
...@@ -519,6 +519,52 @@ trx_free_for_background(trx_t* trx) ...@@ -519,6 +519,52 @@ trx_free_for_background(trx_t* trx)
trx_free(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. */ At shutdown, frees a transaction object that is in the PREPARED state. */
void void
...@@ -526,6 +572,7 @@ trx_free_prepared( ...@@ -526,6 +572,7 @@ trx_free_prepared(
/*==============*/ /*==============*/
trx_t* trx) /*!< in, own: trx object */ trx_t* trx) /*!< in, own: trx object */
{ {
trx_mutex_enter(trx);
ut_ad(trx->state == TRX_STATE_PREPARED ut_ad(trx->state == TRX_STATE_PREPARED
|| trx->state == TRX_STATE_PREPARED_RECOVERED || trx->state == TRX_STATE_PREPARED_RECOVERED
|| !srv_was_started || !srv_was_started
...@@ -543,7 +590,9 @@ trx_free_prepared( ...@@ -543,7 +590,9 @@ trx_free_prepared(
|| srv_force_recovery >= SRV_FORCE_NO_TRX_UNDO))); || srv_force_recovery >= SRV_FORCE_NO_TRX_UNDO)));
ut_a(trx->magic_n == TRX_MAGIC_N); 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); trx_undo_free_prepared(trx);
assert_trx_in_rw_list(trx); assert_trx_in_rw_list(trx);
...@@ -556,14 +605,7 @@ trx_free_prepared( ...@@ -556,14 +605,7 @@ trx_free_prepared(
DBUG_LOG("trx", "Free prepared: " << trx); DBUG_LOG("trx", "Free prepared: " << trx);
trx->state = TRX_STATE_NOT_STARTED; trx->state = TRX_STATE_NOT_STARTED;
ut_ad(!UT_LIST_GET_LEN(trx->lock.trx_locks));
/* 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();
trx->id = 0; trx->id = 0;
trx_free(trx); trx_free(trx);
...@@ -1630,8 +1672,8 @@ trx_commit_in_memory( ...@@ -1630,8 +1672,8 @@ trx_commit_in_memory(
/* Note: We are asserting without holding the lock mutex. But /* Note: We are asserting without holding the lock mutex. But
that is OK because this transaction is not waiting and cannot that is OK because this transaction is not waiting and cannot
be rolled back and no new locks can (or should not) be added be rolled back and no new locks can (or should) be added
becuase it is flagged as a non-locking read-only transaction. */ because it is flagged as a non-locking read-only transaction. */
ut_a(UT_LIST_GET_LEN(trx->lock.trx_locks) == 0); ut_a(UT_LIST_GET_LEN(trx->lock.trx_locks) == 0);
...@@ -1653,35 +1695,43 @@ trx_commit_in_memory( ...@@ -1653,35 +1695,43 @@ trx_commit_in_memory(
DBUG_LOG("trx", "Autocommit in memory: " << trx); DBUG_LOG("trx", "Autocommit in memory: " << trx);
trx->state = TRX_STATE_NOT_STARTED; trx->state = TRX_STATE_NOT_STARTED;
} else { } else {
if (trx->id > 0) { trx_mutex_enter(trx);
/* For consistent snapshot, we need to remove current trx->commit_state();
transaction from running transaction id list for mvcc trx_mutex_exit(trx);
before doing commit and releasing locks. */
if (trx->id) {
trx_erase_lists(trx, serialised); trx_erase_lists(trx, serialised);
}
lock_trx_release_locks(trx); /* Wait for any implicit-to-explicit lock
ut_ad(trx->read_only || !trx->rsegs.m_redo.rseg || trx->id); 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 trx->release_locks();
transactions now that it no longer holds any user 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"); 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); MONITOR_INC(MONITOR_TRX_RO_COMMIT);
if (trx->read_view != NULL) { if (trx->read_view) {
trx_sys->mvcc->view_close( trx_sys->mvcc->view_close(
trx->read_view, false); trx->read_view, false);
} }
} else { } else {
MONITOR_INC(MONITOR_TRX_RW_COMMIT); MONITOR_INC(MONITOR_TRX_RW_COMMIT);
} }
trx->id = 0;
} }
ut_ad(!trx->rsegs.m_redo.update_undo); ut_ad(!trx->rsegs.m_redo.update_undo);
...@@ -2723,18 +2773,13 @@ trx_recover_for_mysql( ...@@ -2723,18 +2773,13 @@ trx_recover_for_mysql(
return(int (count)); return(int (count));
} }
/*******************************************************************//** /** Look up an X/Open distributed transaction in XA PREPARE state.
This function is used to find one X/Open XA distributed transaction @param[in] xid X/Open XA transaction identifier
which is in the prepared state
@return trx on match, the trx->xid will be invalidated; @return trx on match, the trx->xid will be invalidated;
note that the trx may have been committed, unless the caller is note that the trx may have been committed before the caller
holding lock_sys->mutex */ acquires trx_t::mutex */
static MY_ATTRIBUTE((warn_unused_result)) static MY_ATTRIBUTE((warn_unused_result))
trx_t* trx_t* trx_get_trx_by_xid_low(const XID* xid)
trx_get_trx_by_xid_low(
/*===================*/
XID* xid) /*!< in: X/Open XA transaction
identifier */
{ {
trx_t* trx; trx_t* trx;
...@@ -2743,7 +2788,7 @@ trx_get_trx_by_xid_low( ...@@ -2743,7 +2788,7 @@ trx_get_trx_by_xid_low(
for (trx = UT_LIST_GET_FIRST(trx_sys->rw_trx_list); for (trx = UT_LIST_GET_FIRST(trx_sys->rw_trx_list);
trx != NULL; trx != NULL;
trx = UT_LIST_GET_NEXT(trx_list, trx)) { trx = UT_LIST_GET_NEXT(trx_list, trx)) {
trx_mutex_enter(trx);
assert_trx_in_rw_list(trx); assert_trx_in_rw_list(trx);
/* Compare two X/Open XA transaction id's: their /* Compare two X/Open XA transaction id's: their
...@@ -2759,28 +2804,28 @@ trx_get_trx_by_xid_low( ...@@ -2759,28 +2804,28 @@ trx_get_trx_by_xid_low(
/* The commit of a prepared recovered Galera /* The commit of a prepared recovered Galera
transaction needs a valid trx->xid for transaction needs a valid trx->xid for
invoking trx_sys_update_wsrep_checkpoint(). */ invoking trx_sys_update_wsrep_checkpoint(). */
if (wsrep_is_wsrep_xid(trx->xid)) break; if (!wsrep_is_wsrep_xid(trx->xid))
#endif #endif
/* Invalidate the XID, so that subsequent calls /* Invalidate the XID, so that subsequent calls
will not find it. */ will not find it. */
trx->xid->null(); trx->xid->null();
trx_mutex_exit(trx);
break; break;
} }
trx_mutex_exit(trx);
} }
return(trx); return(trx);
} }
/*******************************************************************//** /** Look up an X/Open distributed transaction in XA PREPARE state.
This function is used to find one X/Open XA distributed transaction @param[in] xid X/Open XA transaction identifier
which is in the prepared state @return transaction on match (the trx_t::xid will be invalidated);
@return trx or NULL; on match, the trx->xid will be invalidated; note that the trx may have been committed before the caller acquires
note that the trx may have been committed, unless the caller is trx_t::mutex
holding lock_sys->mutex */ @retval NULL if no match */
trx_t* trx_t* trx_get_trx_by_xid(const XID* xid)
trx_get_trx_by_xid(
/*===============*/
XID* xid) /*!< in: X/Open XA transaction identifier */
{ {
trx_t* trx; trx_t* trx;
...@@ -2793,7 +2838,7 @@ trx_get_trx_by_xid( ...@@ -2793,7 +2838,7 @@ trx_get_trx_by_xid(
/* Recovered/Resurrected transactions are always only on the /* Recovered/Resurrected transactions are always only on the
trx_sys_t::rw_trx_list. */ 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(); trx_sys_mutex_exit();
......
...@@ -1820,7 +1820,7 @@ trx_undo_free_prepared( ...@@ -1820,7 +1820,7 @@ trx_undo_free_prepared(
TRX_STATE_COMMITTED_IN_MEMORY)); TRX_STATE_COMMITTED_IN_MEMORY));
/* fall through */ /* fall through */
case TRX_UNDO_ACTIVE: case TRX_UNDO_ACTIVE:
/* lock_trx_release_locks() assigns /* trx_t::commit_state() assigns
trx->is_recovered=false and trx->is_recovered=false and
trx->state = TRX_STATE_COMMITTED_IN_MEMORY, trx->state = TRX_STATE_COMMITTED_IN_MEMORY,
also for transactions that we faked also for transactions that we faked
...@@ -1852,7 +1852,7 @@ trx_undo_free_prepared( ...@@ -1852,7 +1852,7 @@ trx_undo_free_prepared(
TRX_STATE_COMMITTED_IN_MEMORY)); TRX_STATE_COMMITTED_IN_MEMORY));
/* fall through */ /* fall through */
case TRX_UNDO_ACTIVE: case TRX_UNDO_ACTIVE:
/* lock_trx_release_locks() assigns /* trx_t::commit_state() assigns
trx->is_recovered=false and trx->is_recovered=false and
trx->state = TRX_STATE_COMMITTED_IN_MEMORY, trx->state = TRX_STATE_COMMITTED_IN_MEMORY,
also for transactions that we faked also for transactions that we faked
......
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