Commit 5f463857 authored by Marko Mäkelä's avatar Marko Mäkelä

MDEV-24731 Excessive mutex contention in DeadlockChecker::check_and_resolve()

The DeadlockChecker expects to be able to freeze the waits-for graph.
Hence, it is best executed somewhere where we are not holding any
additional mutexes.

lock_wait(): Defer the deadlock check to this function, instead
of executing it in lock_rec_enqueue_waiting(), lock_table_enqueue_waiting().

DeadlockChecker::trx_rollback(): Merge with the only caller,
check_and_resolve().

LockMutexGuard: RAII accessor for lock_sys.mutex.

lock_sys.deadlocks: Replaces lock_deadlock_found.

trx_t: Clean up some comments.
parent 43ca6059
......@@ -305,7 +305,8 @@ INSERT INTO t2 VALUES (1);
COMMIT;
BEGIN;
INSERT INTO t2 VALUES (2);
UPDATE t2 SET a=a+1;
UPDATE t2 SET a=a+2;
UPDATE t2 SET a=a-1;
connect con2,localhost,root;
XA START 'xid1';
INSERT INTO t1 VALUES (1);
......
......@@ -401,7 +401,8 @@ CREATE TABLE t2 (a INT) ENGINE=InnoDB;
INSERT INTO t2 VALUES (1); COMMIT;
BEGIN;
INSERT INTO t2 VALUES (2);
UPDATE t2 SET a=a+1;
UPDATE t2 SET a=a+2;
UPDATE t2 SET a=a-1;
--connect (con2,localhost,root)
XA START 'xid1';
......
......@@ -39,7 +39,7 @@ NAME SUBSYSTEM COUNT MAX_COUNT MIN_COUNT AVG_COUNT COUNT_RESET MAX_COUNT_RESET M
metadata_table_handles_opened metadata 0 NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL 0 counter Number of table handles opened
metadata_table_handles_closed metadata 0 NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL 0 counter Number of table handles closed
metadata_table_reference_count metadata 0 NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL 0 counter Table reference counter
lock_deadlocks lock 0 NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL 0 counter Number of deadlocks
lock_deadlocks lock 0 NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL 0 value Number of deadlocks
lock_timeouts lock 0 NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL 0 counter Number of lock timeouts
lock_rec_lock_waits lock 0 NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL 0 counter Number of times enqueued into record lock wait queue
lock_table_lock_waits lock 0 NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL 0 counter Number of times enqueued into table lock wait queue
......
# Copyright (c) 2006, 2017, Oracle and/or its affiliates. All rights reserved.
# Copyright (c) 2014, 2020, MariaDB Corporation.
# Copyright (c) 2014, 2021, MariaDB Corporation.
#
# This program is free software; you can redistribute it and/or modify
# it under the terms of the GNU General Public License as published by
......@@ -278,7 +278,6 @@ SET(INNOBASE_SOURCES
lock/lock0iter.cc
lock/lock0prdt.cc
lock/lock0lock.cc
lock/lock0wait.cc
log/log0log.cc
log/log0recv.cc
log/log0crypt.cc
......
......@@ -961,7 +961,7 @@ static SHOW_VAR innodb_status_variables[]= {
{"data_written", &export_vars.innodb_data_written, SHOW_SIZE_T},
{"dblwr_pages_written", &export_vars.innodb_dblwr_pages_written,SHOW_SIZE_T},
{"dblwr_writes", &export_vars.innodb_dblwr_writes, SHOW_SIZE_T},
{"deadlocks", &srv_stats.lock_deadlock_count, SHOW_SIZE_T},
{"deadlocks", &lock_sys.deadlocks, SHOW_SIZE_T},
{"history_list_length", &export_vars.innodb_history_list_length,SHOW_SIZE_T},
{"ibuf_discarded_delete_marks", &ibuf.n_discarded_ops[IBUF_OP_DELETE_MARK],
SHOW_SIZE_T},
......
......@@ -642,6 +642,8 @@ class lock_sys_t
hash_table_t prdt_hash;
/** page locks for SPATIAL INDEX */
hash_table_t prdt_page_hash;
/** number of deadlocks detected; protected by mutex */
ulint deadlocks;
/** mutex covering lock waits; @see trx_lock_t::wait_lock */
MY_ALIGNED(CACHE_LINE_SIZE) mysql_mutex_t wait_mutex;
......@@ -766,6 +768,16 @@ class lock_sys_t
{ return get_first(prdt_page_hash, id); }
};
/** The lock system */
extern lock_sys_t lock_sys;
/** lock_sys.mutex guard */
struct LockMutexGuard
{
LockMutexGuard() { lock_sys.mutex_lock(); }
~LockMutexGuard() { lock_sys.mutex_unlock(); }
};
/*********************************************************************//**
Creates a new record lock and inserts it to the lock queue. Does NOT check
for deadlocks or lock compatibility!
......@@ -835,9 +847,7 @@ Check for deadlocks.
@param[in,out] thr query thread
@param[in] prdt minimum bounding box (spatial index)
@retval DB_LOCK_WAIT if the waiting lock was enqueued
@retval DB_DEADLOCK if this transaction was chosen as the victim
@retval DB_SUCCESS_LOCKED_REC if the other transaction was chosen as a victim
(or it happened to commit) */
@retval DB_DEADLOCK if this transaction was chosen as the victim */
dberr_t
lock_rec_enqueue_waiting(
#ifdef WITH_WSREP
......@@ -871,9 +881,6 @@ lock_rec_free_all_from_discard_page(
/*================================*/
const buf_block_t* block); /*!< in: page to be discarded */
/** The lock system */
extern lock_sys_t lock_sys;
/** Cancel a waiting lock request and release possibly waiting transactions */
void lock_cancel_waiting_and_release(lock_t *lock);
......
......@@ -391,9 +391,6 @@ static const byte lock_strength_matrix[5][5] = {
/* AI */ { FALSE, FALSE, FALSE, FALSE, TRUE}
};
/** Maximum depth of the DFS stack. */
static const ulint MAX_STACK_SIZE = 4096;
#define PRDT_HEAPNO PAGE_HEAP_NO_INFIMUM
/** Record locking request status */
enum lock_rec_req_status {
......
......@@ -187,9 +187,6 @@ struct srv_stats_t
/** Number of temporary tablespace blocks decrypted */
ulint_ctr_64_t n_temp_blocks_decrypted;
/** Number of lock deadlocks */
ulint_ctr_1_t lock_deadlock_count;
};
/** We are prepared for a situation that we have this many threads waiting for
......
......@@ -418,7 +418,8 @@ typedef std::vector<ib_lock_t*, ut_allocator<ib_lock_t*> > lock_list;
/** The locks and state of an active transaction. Protected by
lock_sys.mutex, trx->mutex or both. */
struct trx_lock_t {
struct trx_lock_t
{
/** Lock request being waited for.
Set to nonnull when holding lock_sys.mutex, lock_sys.wait_mutex and
trx->mutex, by the thread that is executing the transaction.
......@@ -432,13 +433,10 @@ struct trx_lock_t {
ib_uint64_t deadlock_mark; /*!< A mark field that is initialized
to and checked against lock_mark_counter
by lock_deadlock_recursive(). */
bool was_chosen_as_deadlock_victim;
/*!< when the transaction decides to
wait for a lock, it sets this to false;
if another transaction chooses this
transaction as a victim in deadlock
resolution, it sets this to true.
Protected by trx->mutex. */
/** When the transaction decides to wait for a lock, it clears this;
set if another transaction chooses this transaction as a victim in deadlock
resolution. Protected by lock_sys.mutex and lock_sys.wait_mutex. */
bool was_chosen_as_deadlock_victim;
que_thr_t* wait_thr; /*!< query thread belonging to this
trx that is in waiting
state. For threads suspended in a
......@@ -666,88 +664,94 @@ struct trx_t : ilist_node<> {
public:
/** Transaction identifier (0 if no locks were acquired).
Set by trx_sys_t::register_rw() or trx_resurrect() before
the transaction is added to trx_sys.rw_trx_hash.
Cleared in commit_in_memory() after commit_state(),
trx_sys_t::deregister_rw(), release_locks(). */
trx_id_t id;
/** mutex protecting state and some of lock
(some are protected by lock_sys.mutex) */
srw_mutex mutex;
trx_id_t id; /*!< transaction id */
/** State of the trx from the point of view of concurrency control
and the valid state transitions.
/** State of the trx from the point of view of concurrency control
and the valid state transitions.
Possible states:
Possible states:
TRX_STATE_NOT_STARTED
TRX_STATE_ACTIVE
TRX_STATE_PREPARED
TRX_STATE_PREPARED_RECOVERED (special case of TRX_STATE_PREPARED)
TRX_STATE_COMMITTED_IN_MEMORY (alias below COMMITTED)
TRX_STATE_NOT_STARTED
TRX_STATE_ACTIVE
TRX_STATE_PREPARED
TRX_STATE_PREPARED_RECOVERED (special case of TRX_STATE_PREPARED)
TRX_STATE_COMMITTED_IN_MEMORY (alias below COMMITTED)
Valid state transitions are:
Valid state transitions are:
Regular transactions:
* NOT_STARTED -> ACTIVE -> COMMITTED -> NOT_STARTED
Regular transactions:
* NOT_STARTED -> ACTIVE -> COMMITTED -> NOT_STARTED
Auto-commit non-locking read-only:
* NOT_STARTED -> ACTIVE -> NOT_STARTED
Auto-commit non-locking read-only:
* NOT_STARTED -> ACTIVE -> NOT_STARTED
XA (2PC):
* NOT_STARTED -> ACTIVE -> PREPARED -> COMMITTED -> NOT_STARTED
XA (2PC):
* NOT_STARTED -> ACTIVE -> PREPARED -> COMMITTED -> NOT_STARTED
Recovered XA:
* NOT_STARTED -> PREPARED -> COMMITTED -> (freed)
Recovered XA:
* NOT_STARTED -> PREPARED -> COMMITTED -> (freed)
Recovered XA followed by XA ROLLBACK:
* NOT_STARTED -> PREPARED -> ACTIVE -> COMMITTED -> (freed)
Recovered XA followed by XA ROLLBACK:
* NOT_STARTED -> PREPARED -> ACTIVE -> COMMITTED -> (freed)
XA (2PC) (shutdown or disconnect before ROLLBACK or COMMIT):
* NOT_STARTED -> PREPARED -> (freed)
XA (2PC) (shutdown or disconnect before ROLLBACK or COMMIT):
* NOT_STARTED -> PREPARED -> (freed)
Disconnected XA PREPARE transaction can become recovered:
* ... -> ACTIVE -> PREPARED (connected) -> PREPARED (disconnected)
Disconnected XA can become recovered:
* ... -> ACTIVE -> PREPARED (connected) -> PREPARED (disconnected)
Disconnected means from mysql e.g due to the mysql client disconnection.
Latching and various transaction lists membership rules:
Latching and various transaction lists membership rules:
XA (2PC) transactions are always treated as non-autocommit.
XA (2PC) transactions are always treated as non-autocommit.
Transitions to ACTIVE or NOT_STARTED occur when transaction
is not in rw_trx_hash.
Transitions to ACTIVE or NOT_STARTED occur when transaction
is not in rw_trx_hash.
Autocommit non-locking read-only transactions move between states
without holding any mutex. They are not in rw_trx_hash.
Autocommit non-locking read-only transactions move between states
without holding any mutex. They are not in rw_trx_hash.
All transactions, unless they are determined to be ac-nl-ro,
explicitly tagged as read-only or read-write, will first be put
on the read-only transaction list. Only when a !read-only transaction
in the read-only list tries to acquire an X or IX lock on a table
do we remove it from the read-only list and put it on the read-write
list. During this switch we assign it a rollback segment.
All transactions, unless they are determined to be ac-nl-ro,
explicitly tagged as read-only or read-write, will first be put
on the read-only transaction list. Only when a !read-only transaction
in the read-only list tries to acquire an X or IX lock on a table
do we remove it from the read-only list and put it on the read-write
list. During this switch we assign it a rollback segment.
When a transaction is NOT_STARTED, it can be in trx_list. It cannot be
in rw_trx_hash.
When a transaction is NOT_STARTED, it can be in trx_list. It cannot be
in rw_trx_hash.
ACTIVE->PREPARED->COMMITTED is only possible when trx is in rw_trx_hash.
The transition ACTIVE->PREPARED is protected by trx->mutex.
ACTIVE->PREPARED->COMMITTED is only possible when trx is in rw_trx_hash.
The transition ACTIVE->PREPARED is protected by trx->mutex.
ACTIVE->COMMITTED is possible when the transaction is in
rw_trx_hash.
ACTIVE->COMMITTED is possible when the transaction is in
rw_trx_hash.
Transitions to COMMITTED are protected by trx_t::mutex. */
Transitions to COMMITTED are protected by trx_t::mutex. */
Atomic_relaxed<trx_state_t> state;
/** The locks of the transaction. Protected by lock_sys.mutex
(insertions also by trx_t::mutex). */
trx_lock_t lock;
#ifdef WITH_WSREP
/** whether wsrep_on(mysql_thd) held at the start of transaction */
bool wsrep;
bool is_wsrep() const { return UNIV_UNLIKELY(wsrep); }
/** whether wsrep_on(mysql_thd) held at the start of transaction */
bool wsrep;
bool is_wsrep() const { return UNIV_UNLIKELY(wsrep); }
#else /* WITH_WSREP */
bool is_wsrep() const { return false; }
bool is_wsrep() const { return false; }
#endif /* WITH_WSREP */
ReadView read_view; /*!< consistent read view used in the
transaction, or NULL if not yet set */
trx_lock_t lock; /*!< Information about the transaction
locks and state. Protected by
lock_sys.mutex (insertions also
by trx_t::mutex). */
/** Consistent read view of the transaction */
ReadView read_view;
/* These fields are not protected by any mutex. */
......
This diff is collapsed.
......@@ -571,18 +571,11 @@ lock_prdt_insert_check_and_lock(
lock_sys.mutex_unlock();
switch (err) {
case DB_SUCCESS_LOCKED_REC:
err = DB_SUCCESS;
/* fall through */
case DB_SUCCESS:
if (err == DB_SUCCESS) {
/* Update the page max trx id field */
page_update_max_trx_id(block,
buf_block_get_page_zip(block),
trx->id, mtr);
default:
/* We only care about the two return values. */
break;
}
return(err);
......
/*****************************************************************************
Copyright (c) 1996, 2016, Oracle and/or its affiliates. All Rights Reserved.
Copyright (c) 2014, 2021, MariaDB Corporation.
This program is free software; you can redistribute it and/or modify it under
the terms of the GNU General Public License as published by the Free Software
Foundation; version 2 of the License.
This program is distributed in the hope that it will be useful, but WITHOUT
ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or FITNESS
FOR A PARTICULAR PURPOSE. See the GNU General Public License for more details.
You should have received a copy of the GNU General Public License along with
this program; if not, write to the Free Software Foundation, Inc.,
51 Franklin Street, Fifth Floor, Boston, MA 02110-1335 USA
*****************************************************************************/
/**************************************************//**
@file lock/lock0wait.cc
The transaction lock system
Created 25/5/2010 Sunny Bains
*******************************************************/
#define LOCK_MODULE_IMPLEMENTATION
#include "univ.i"
#include <mysql/service_thd_wait.h>
#include <mysql/service_wsrep.h>
#include "srv0mon.h"
#include "que0que.h"
#include "lock0lock.h"
#include "row0mysql.h"
#include "srv0start.h"
#include "lock0priv.h"
#ifdef WITH_WSREP
/*********************************************************************//**
check if lock timeout was for priority thread,
as a side effect trigger lock monitor
@param[in] trx transaction owning the lock
@param[in] locked true if trx and lock_sys.mutex is ownd
@return false for regular lock timeout */
static
bool
wsrep_is_BF_lock_timeout(
const trx_t* trx,
bool locked = true)
{
if (trx->error_state != DB_DEADLOCK && trx->is_wsrep() &&
srv_monitor_timer && wsrep_thd_is_BF(trx->mysql_thd, FALSE)) {
ib::info() << "WSREP: BF lock wait long for trx:" << ib::hex(trx->id)
<< " query: " << wsrep_thd_query(trx->mysql_thd);
if (!locked) {
lock_sys.mutex_lock();
}
lock_sys.mutex_assert_locked();
trx_print_latched(stderr, trx, 3000);
if (!locked) {
lock_sys.mutex_unlock();
}
srv_print_innodb_monitor = TRUE;
srv_print_innodb_lock_monitor = TRUE;
srv_monitor_timer_schedule_now();
return true;
}
return false;
}
#endif /* WITH_WSREP */
/** Note that a record lock wait started */
inline void lock_sys_t::wait_start()
{
mysql_mutex_assert_owner(&wait_mutex);
wait_pending++;
wait_count++;
}
/** Note that a record lock wait resumed */
inline
void lock_sys_t::wait_resume(THD *thd, my_hrtime_t start, my_hrtime_t now)
{
mysql_mutex_assert_owner(&wait_mutex);
wait_pending--;
if (now.val >= start.val)
{
const ulint diff_time= static_cast<ulint>((now.val - start.val) / 1000);
wait_time+= diff_time;
if (diff_time > wait_time_max)
wait_time_max= diff_time;
thd_storage_lock_wait(thd, diff_time);
}
}
/** Wait for a lock to be released.
@retval DB_DEADLOCK if this transaction was chosen as the deadlock victim
@retval DB_INTERRUPTED if the execution was interrupted by the user
@retval DB_LOCK_WAIT_TIMEOUT if the lock wait timed out
@retval DB_SUCCESS if the lock was granted */
dberr_t lock_wait(que_thr_t *thr)
{
trx_t *trx= thr_get_trx(thr);
if (trx->mysql_thd)
DEBUG_SYNC_C("lock_wait_suspend_thread_enter");
/* InnoDB system transactions may use the global value of
innodb_lock_wait_timeout, because trx->mysql_thd == NULL. */
const ulong innodb_lock_wait_timeout= trx_lock_wait_timeout_get(trx);
const bool no_timeout= innodb_lock_wait_timeout > 100000000;
const my_hrtime_t suspend_time= my_hrtime_coarse();
ut_ad(!trx->dict_operation_lock_mode ||
trx->dict_operation_lock_mode == RW_S_LATCH);
const bool row_lock_wait= thr->lock_state == QUE_THR_LOCK_ROW;
bool had_dict_lock= trx->dict_operation_lock_mode != 0;
mysql_mutex_lock(&lock_sys.wait_mutex);
trx->mutex.wr_lock();
trx->error_state= DB_SUCCESS;
if (!trx->lock.wait_lock)
{
/* The lock has already been released or this transaction
was chosen as a deadlock victim: no need to suspend */
if (trx->lock.was_chosen_as_deadlock_victim)
{
trx->error_state= DB_DEADLOCK;
trx->lock.was_chosen_as_deadlock_victim= false;
}
mysql_mutex_unlock(&lock_sys.wait_mutex);
trx->mutex.wr_unlock();
return trx->error_state;
}
trx->lock.suspend_time= suspend_time;
trx->mutex.wr_unlock();
if (row_lock_wait)
lock_sys.wait_start();
int err= 0;
/* The wait_lock can be cleared by another thread in lock_grant(),
lock_rec_cancel(), or lock_cancel_waiting_and_release(). But, a wait
can only be initiated by the current thread which owns the transaction. */
if (const lock_t *wait_lock= trx->lock.wait_lock)
{
if (had_dict_lock) /* Release foreign key check latch */
{
mysql_mutex_unlock(&lock_sys.wait_mutex);
row_mysql_unfreeze_data_dictionary(trx);
mysql_mutex_lock(&lock_sys.wait_mutex);
}
timespec abstime;
set_timespec_time_nsec(abstime, suspend_time.val * 1000);
abstime.MY_tv_sec+= innodb_lock_wait_timeout;
thd_wait_begin(trx->mysql_thd, wait_lock->is_table()
? THD_WAIT_TABLE_LOCK : THD_WAIT_ROW_LOCK);
while (trx->lock.wait_lock)
{
if (no_timeout)
mysql_cond_wait(&trx->lock.cond, &lock_sys.wait_mutex);
else
err= mysql_cond_timedwait(&trx->lock.cond, &lock_sys.wait_mutex,
&abstime);
switch (trx->error_state) {
default:
if (trx_is_interrupted(trx))
/* innobase_kill_query() can only set trx->error_state=DB_INTERRUPTED
for any transaction that is attached to a connection. */
trx->error_state= DB_INTERRUPTED;
else if (!err)
continue;
else
break;
/* fall through */
case DB_DEADLOCK:
case DB_INTERRUPTED:
err= 0;
}
break;
}
thd_wait_end(trx->mysql_thd);
}
else
had_dict_lock= false;
if (row_lock_wait)
lock_sys.wait_resume(trx->mysql_thd, suspend_time, my_hrtime_coarse());
mysql_mutex_unlock(&lock_sys.wait_mutex);
if (had_dict_lock)
row_mysql_freeze_data_dictionary(trx);
if (!err);
#ifdef WITH_WSREP
else if (trx->is_wsrep() && wsrep_is_BF_lock_timeout(trx, false));
#endif
else
{
trx->error_state= DB_LOCK_WAIT_TIMEOUT;
MONITOR_INC(MONITOR_TIMEOUT);
}
if (trx->lock.wait_lock)
{
{
lock_sys.mutex_lock();
mysql_mutex_lock(&lock_sys.wait_mutex);
if (lock_t *lock= trx->lock.wait_lock)
{
trx->mutex.wr_lock();
lock_cancel_waiting_and_release(lock);
trx->mutex.wr_unlock();
}
lock_sys.mutex_unlock();
mysql_mutex_unlock(&lock_sys.wait_mutex);
}
}
return trx->error_state;
}
......@@ -91,7 +91,8 @@ static monitor_info_t innodb_counter_info[] =
MONITOR_DEFAULT_START, MONITOR_MODULE_LOCK},
{"lock_deadlocks", "lock", "Number of deadlocks",
MONITOR_DEFAULT_ON,
static_cast<monitor_type_t>(
MONITOR_EXISTING | MONITOR_DEFAULT_ON | MONITOR_DISPLAY_CURRENT),
MONITOR_DEFAULT_START, MONITOR_DEADLOCK},
{"lock_timeouts", "lock", "Number of lock timeouts",
......@@ -1879,6 +1880,9 @@ srv_mon_process_existing_counter(
case MONITOR_OVLD_PAGES_DECRYPTED:
value = srv_stats.pages_decrypted;
break;
case MONITOR_DEADLOCK:
value = lock_sys.deadlocks;
break;
default:
ut_error;
......
......@@ -21,7 +21,7 @@ NAME SUBSYSTEM COUNT MAX_COUNT MIN_COUNT AVG_COUNT COUNT_RESET MAX_COUNT_RESET M
metadata_table_handles_opened metadata 0 NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL 0 counter Number of table handles opened
metadata_table_handles_closed metadata 0 NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL 0 counter Number of table handles closed
metadata_table_reference_count metadata 0 NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL 0 counter Table reference counter
lock_deadlocks lock 0 NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL 0 counter Number of deadlocks
lock_deadlocks lock 0 NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL 0 value Number of deadlocks
lock_timeouts lock 0 NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL 0 counter Number of lock timeouts
lock_rec_lock_waits lock 0 NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL 0 counter Number of times enqueued into record lock wait queue
lock_table_lock_waits lock 0 NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL 0 counter Number of times enqueued into table lock wait queue
......
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