mirror of
https://github.com/MariaDB/server.git
synced 2025-07-30 16:24:05 +03:00
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@e27e0e0bb7 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@a17be6963f: "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().
This commit is contained in:
@ -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(
|
|||||||
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.
|
|||||||
@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(
|
||||||
|
@ -1,7 +1,7 @@
|
|||||||
/*****************************************************************************
|
/*****************************************************************************
|
||||||
|
|
||||||
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
|
|||||||
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.
|
||||||
|
@ -1,7 +1,7 @@
|
|||||||
/*****************************************************************************
|
/*****************************************************************************
|
||||||
|
|
||||||
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)
|
|||||||
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(
|
|||||||
|
|
||||||
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);
|
||||||
|
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;
|
++trx->n_ref;
|
||||||
trx_mutex_exit(trx);
|
}
|
||||||
|
mutex_exit(trx_mutex);
|
||||||
}
|
}
|
||||||
|
|
||||||
trx_sys_mutex_exit();
|
trx_sys_mutex_exit();
|
||||||
|
@ -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 */
|
|||||||
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:
|
|||||||
|
|
||||||
* 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.
|
|||||||
* 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 {
|
|||||||
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 {
|
|||||||
|
|
||||||
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 @@ public:
|
|||||||
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 */
|
||||||
|
@ -1273,30 +1273,31 @@ lock_rec_other_trx_holds_expl(
|
|||||||
trx_t* holds = NULL;
|
trx_t* holds = NULL;
|
||||||
|
|
||||||
lock_mutex_enter();
|
lock_mutex_enter();
|
||||||
|
|
||||||
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);
|
mutex_enter(&trx_sys->mutex);
|
||||||
|
trx_mutex_enter(trx);
|
||||||
|
|
||||||
|
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);
|
for (trx_t* t = UT_LIST_GET_FIRST(trx_sys->rw_trx_list);
|
||||||
t != NULL;
|
t != NULL;
|
||||||
t = UT_LIST_GET_NEXT(trx_list, t)) {
|
t = UT_LIST_GET_NEXT(trx_list, t)) {
|
||||||
|
|
||||||
lock_t* expl_lock = lock_rec_has_expl(
|
lock_t* expl_lock = lock_rec_has_expl(
|
||||||
precise_mode, block, heap_no, t);
|
precise_mode, block, heap_no, t);
|
||||||
|
if (expl_lock && expl_lock->trx != trx) {
|
||||||
if (expl_lock && expl_lock->trx != impl_trx) {
|
|
||||||
/* An explicit lock is held by trx other than
|
/* An explicit lock is held by trx other than
|
||||||
the trx holding the implicit lock. */
|
the trx holding the implicit lock. */
|
||||||
holds = expl_lock->trx;
|
holds = expl_lock->trx;
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
mutex_exit(&trx_sys->mutex);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
lock_mutex_exit();
|
lock_mutex_exit();
|
||||||
|
mutex_exit(&trx_sys->mutex);
|
||||||
|
trx_mutex_exit(trx);
|
||||||
|
|
||||||
return(holds);
|
return(holds);
|
||||||
}
|
}
|
||||||
@ -5107,7 +5108,7 @@ lock_trx_table_locks_find(
|
|||||||
{
|
{
|
||||||
bool found = false;
|
bool found = false;
|
||||||
|
|
||||||
trx_mutex_enter(trx);
|
ut_ad(trx_mutex_own(trx));
|
||||||
|
|
||||||
for (lock_list::const_iterator it = trx->lock.table_locks.begin(),
|
for (lock_list::const_iterator it = trx->lock.table_locks.begin(),
|
||||||
end = trx->lock.table_locks.end(); it != end; ++it) {
|
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);
|
ut_a(lock->un_member.tab_lock.table != NULL);
|
||||||
}
|
}
|
||||||
|
|
||||||
trx_mutex_exit(trx);
|
|
||||||
|
|
||||||
return(found);
|
return(found);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -5155,21 +5154,20 @@ lock_table_queue_validate(
|
|||||||
|
|
||||||
/* lock->trx->state cannot change from or to NOT_STARTED
|
/* lock->trx->state cannot change from or to NOT_STARTED
|
||||||
while we are holding the trx_sys->mutex. It may change
|
while we are holding the trx_sys->mutex. It may change
|
||||||
from ACTIVE to PREPARED, but it may not change to
|
from ACTIVE or PREPARED to PREPARED or COMMITTED. */
|
||||||
COMMITTED, because we are holding the lock_sys->mutex. */
|
trx_mutex_enter(lock->trx);
|
||||||
ut_ad(trx_assert_started(lock->trx));
|
ut_ad(trx_assert_started(lock->trx));
|
||||||
|
if (trx_state_eq(lock->trx, TRX_STATE_COMMITTED_IN_MEMORY)) {
|
||||||
if (!lock_get_wait(lock)) {
|
} else if (!lock_get_wait(lock)) {
|
||||||
|
|
||||||
ut_a(!lock_table_other_has_incompatible(
|
ut_a(!lock_table_other_has_incompatible(
|
||||||
lock->trx, 0, table,
|
lock->trx, 0, table,
|
||||||
lock_get_mode(lock)));
|
lock_get_mode(lock)));
|
||||||
} else {
|
} else {
|
||||||
|
|
||||||
ut_a(lock_table_has_to_wait_in_queue(lock));
|
ut_a(lock_table_has_to_wait_in_queue(lock));
|
||||||
}
|
}
|
||||||
|
|
||||||
ut_a(lock_trx_table_locks_find(lock->trx, lock));
|
ut_a(lock_trx_table_locks_find(lock->trx, lock));
|
||||||
|
trx_mutex_exit(lock->trx);
|
||||||
}
|
}
|
||||||
|
|
||||||
return(TRUE);
|
return(TRUE);
|
||||||
@ -5191,7 +5189,6 @@ lock_rec_queue_validate(
|
|||||||
const dict_index_t* index, /*!< in: index, or NULL if not known */
|
const dict_index_t* index, /*!< in: index, or NULL if not known */
|
||||||
const ulint* offsets)/*!< in: rec_get_offsets(rec, index) */
|
const ulint* offsets)/*!< in: rec_get_offsets(rec, index) */
|
||||||
{
|
{
|
||||||
const trx_t* impl_trx;
|
|
||||||
const lock_t* lock;
|
const lock_t* lock;
|
||||||
ulint heap_no;
|
ulint heap_no;
|
||||||
|
|
||||||
@ -5217,40 +5214,34 @@ lock_rec_queue_validate(
|
|||||||
lock != NULL;
|
lock != NULL;
|
||||||
lock = lock_rec_get_next_const(heap_no, lock)) {
|
lock = lock_rec_get_next_const(heap_no, lock)) {
|
||||||
|
|
||||||
|
ut_ad(!index || lock->index == index);
|
||||||
|
|
||||||
|
trx_mutex_enter(lock->trx);
|
||||||
ut_ad(!trx_is_ac_nl_ro(lock->trx));
|
ut_ad(!trx_is_ac_nl_ro(lock->trx));
|
||||||
|
ut_ad(trx_state_eq(lock->trx,
|
||||||
if (lock_get_wait(lock)) {
|
TRX_STATE_COMMITTED_IN_MEMORY)
|
||||||
ut_a(lock_rec_has_to_wait_in_queue(lock));
|
|| !lock_get_wait(lock)
|
||||||
}
|
|| lock_rec_has_to_wait_in_queue(lock));
|
||||||
|
trx_mutex_exit(lock->trx);
|
||||||
if (index != NULL) {
|
|
||||||
ut_a(lock->index == index);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
goto func_exit;
|
goto func_exit;
|
||||||
}
|
}
|
||||||
|
|
||||||
ut_ad(page_rec_is_leaf(rec));
|
ut_ad(page_rec_is_leaf(rec));
|
||||||
|
|
||||||
if (index == NULL) {
|
|
||||||
|
|
||||||
/* 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());
|
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) {
|
if (!index || !index->is_primary()) {
|
||||||
|
/* Nothing we can do */
|
||||||
|
} 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
|
} else if (const lock_t* other_lock
|
||||||
= lock_rec_other_has_expl_req(
|
= lock_rec_other_has_expl_req(
|
||||||
LOCK_S, block, true, heap_no,
|
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,
|
ut_ad(lock_rec_has_expl(LOCK_X | LOCK_REC_NOT_GAP,
|
||||||
block, heap_no, impl_trx));
|
block, heap_no, impl_trx));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
mutex_exit(&impl_trx->mutex);
|
||||||
}
|
}
|
||||||
|
|
||||||
for (lock = lock_rec_get_first(lock_sys->rec_hash, block, heap_no);
|
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 */
|
trx_t* trx, /*!< in/out: active transaction */
|
||||||
ulint heap_no)/*!< in: rec heap number to lock */
|
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");
|
DEBUG_SYNC_C("before_lock_rec_convert_impl_to_expl_for_trx");
|
||||||
|
|
||||||
lock_mutex_enter();
|
lock_mutex_enter();
|
||||||
|
trx_mutex_enter(trx);
|
||||||
ut_ad(!trx_state_eq(trx, TRX_STATE_NOT_STARTED));
|
ut_ad(!trx_state_eq(trx, TRX_STATE_NOT_STARTED));
|
||||||
|
|
||||||
if (!trx_state_eq(trx, TRX_STATE_COMMITTED_IN_MEMORY)
|
if (!trx_state_eq(trx, TRX_STATE_COMMITTED_IN_MEMORY)
|
||||||
&& !lock_rec_has_expl(LOCK_X | LOCK_REC_NOT_GAP,
|
&& !lock_rec_has_expl(LOCK_X | LOCK_REC_NOT_GAP,
|
||||||
block, heap_no, trx)) {
|
block, heap_no, trx)) {
|
||||||
|
lock_rec_add_to_queue(LOCK_REC | LOCK_X | LOCK_REC_NOT_GAP,
|
||||||
ulint type_mode;
|
block, heap_no, index, trx, true);
|
||||||
|
|
||||||
type_mode = (LOCK_REC | LOCK_X | LOCK_REC_NOT_GAP);
|
|
||||||
|
|
||||||
lock_rec_add_to_queue(
|
|
||||||
type_mode, block, heap_no, index, trx, FALSE);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
lock_mutex_exit();
|
lock_mutex_exit();
|
||||||
|
ut_ad(trx->n_ref > 0);
|
||||||
trx_release_reference(trx);
|
--trx->n_ref;
|
||||||
|
trx_mutex_exit(trx);
|
||||||
|
|
||||||
DEBUG_SYNC_C("after_lock_rec_convert_impl_to_expl_for_trx");
|
DEBUG_SYNC_C("after_lock_rec_convert_impl_to_expl_for_trx");
|
||||||
}
|
}
|
||||||
@ -6541,118 +6528,24 @@ lock_unlock_table_autoinc(
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/*********************************************************************//**
|
/** Release the explicit locks of a committing transaction,
|
||||||
Releases a transaction's locks, and releases possible other transactions
|
and release possible other transactions waiting because of these locks. */
|
||||||
waiting because of these locks. Change the state of the transaction to
|
void lock_trx_release_locks(trx_t* trx)
|
||||||
TRX_STATE_COMMITTED_IN_MEMORY. */
|
|
||||||
void
|
|
||||||
lock_trx_release_locks(
|
|
||||||
/*===================*/
|
|
||||||
trx_t* trx) /*!< in/out: transaction */
|
|
||||||
{
|
{
|
||||||
check_trx_state(trx);
|
ut_ad(UT_LIST_GET_LEN(trx->lock.trx_locks));
|
||||||
|
|
||||||
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();
|
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_release(trx);
|
||||||
|
|
||||||
lock_mutex_exit();
|
|
||||||
}
|
|
||||||
|
|
||||||
trx->lock.n_rec_locks = 0;
|
trx->lock.n_rec_locks = 0;
|
||||||
|
|
||||||
/* We don't remove the locks one by one from the vector for
|
/* We don't remove the locks one by one from the vector for
|
||||||
efficiency reasons. We simply reset it because we would have
|
efficiency reasons. We simply reset it because we would have
|
||||||
released all the locks anyway. */
|
released all the locks anyway. */
|
||||||
|
|
||||||
trx->lock.table_locks.clear();
|
trx->lock.table_locks.clear();
|
||||||
|
|
||||||
ut_a(UT_LIST_GET_LEN(trx->lock.trx_locks) == 0);
|
ut_ad(UT_LIST_GET_LEN(trx->lock.trx_locks) == 0);
|
||||||
ut_a(ib_vector_is_empty(trx->autoinc_locks));
|
ut_ad(ib_vector_is_empty(trx->autoinc_locks));
|
||||||
ut_a(trx->lock.table_locks.empty());
|
lock_mutex_exit();
|
||||||
|
|
||||||
mem_heap_empty(trx->lock.lock_heap);
|
mem_heap_empty(trx->lock.lock_heap);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -6724,18 +6617,16 @@ lock_table_locks_lookup(
|
|||||||
itself */
|
itself */
|
||||||
const trx_ut_list_t* trx_list) /*!< in: trx list to check */
|
const trx_ut_list_t* trx_list) /*!< in: trx list to check */
|
||||||
{
|
{
|
||||||
trx_t* trx;
|
|
||||||
|
|
||||||
ut_a(table != NULL);
|
ut_a(table != NULL);
|
||||||
ut_ad(lock_mutex_own());
|
ut_ad(lock_mutex_own());
|
||||||
ut_ad(trx_sys_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 != NULL;
|
||||||
trx = UT_LIST_GET_NEXT(trx_list, trx)) {
|
trx = UT_LIST_GET_NEXT(trx_list, trx)) {
|
||||||
|
|
||||||
const lock_t* lock;
|
const lock_t* lock;
|
||||||
|
|
||||||
|
trx_mutex_enter(trx);
|
||||||
check_trx_state(trx);
|
check_trx_state(trx);
|
||||||
|
|
||||||
for (lock = UT_LIST_GET_FIRST(trx->lock.trx_locks);
|
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)
|
ut_ad(!dict_index_is_online_ddl(lock->index)
|
||||||
|| dict_index_is_clust(lock->index));
|
|| dict_index_is_clust(lock->index));
|
||||||
if (lock->index->table == table) {
|
if (lock->index->table == table) {
|
||||||
return(lock);
|
break;
|
||||||
}
|
}
|
||||||
} else if (lock->un_member.tab_lock.table == table) {
|
} else if (lock->un_member.tab_lock.table == table) {
|
||||||
return(lock);
|
break;
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
return(NULL);
|
trx_mutex_exit(trx);
|
||||||
|
|
||||||
|
if (lock) {
|
||||||
|
return lock;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return NULL;
|
||||||
}
|
}
|
||||||
#endif /* UNIV_DEBUG */
|
#endif /* UNIV_DEBUG */
|
||||||
|
|
||||||
@ -6921,7 +6818,7 @@ DeadlockChecker::start_print()
|
|||||||
|
|
||||||
if (srv_print_all_deadlocks) {
|
if (srv_print_all_deadlocks) {
|
||||||
ib::info() << "Transactions deadlock detected, dumping"
|
ib::info() << "Transactions deadlock detected, dumping"
|
||||||
<< " detailed information.";
|
" detailed information.";
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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(
|
|||||||
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(
|
|||||||
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(
|
|||||||
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(
|
|||||||
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(
|
|||||||
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(
|
|||||||
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 @@ result_check:
|
|||||||
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.
|
|||||||
@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(
|
|||||||
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)
|
|||||||
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(
|
|||||||
/*==============*/
|
/*==============*/
|
||||||
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(
|
|||||||
|| 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(
|
|||||||
|
|
||||||
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(
|
|||||||
|
|
||||||
/* 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(
|
|||||||
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);
|
||||||
|
|
||||||
|
/* 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);
|
||||||
|
|
||||||
|
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();
|
||||||
}
|
}
|
||||||
|
|
||||||
lock_trx_release_locks(trx);
|
|
||||||
ut_ad(trx->read_only || !trx->rsegs.m_redo.rseg || trx->id);
|
|
||||||
|
|
||||||
/* Remove the transaction from the list of active
|
|
||||||
transactions now that it no longer holds any user 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 @@ partial:
|
|||||||
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(
|
|||||||
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(
|
|||||||
/* 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(
|
|||||||
|
|
||||||
/* 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(
|
|||||||
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(
|
|||||||
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
|
||||||
|
Reference in New Issue
Block a user