1
0
mirror of https://github.com/MariaDB/server.git synced 2025-08-01 03:47:19 +03:00

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.
This commit is contained in:
Marko Mäkelä
2021-02-04 16:38:07 +02:00
parent 43ca6059ca
commit 5f46385764
14 changed files with 453 additions and 571 deletions

View File

@ -305,7 +305,8 @@ INSERT INTO t2 VALUES (1);
COMMIT; COMMIT;
BEGIN; BEGIN;
INSERT INTO t2 VALUES (2); 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; connect con2,localhost,root;
XA START 'xid1'; XA START 'xid1';
INSERT INTO t1 VALUES (1); INSERT INTO t1 VALUES (1);

View File

@ -401,7 +401,8 @@ CREATE TABLE t2 (a INT) ENGINE=InnoDB;
INSERT INTO t2 VALUES (1); COMMIT; INSERT INTO t2 VALUES (1); COMMIT;
BEGIN; BEGIN;
INSERT INTO t2 VALUES (2); 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) --connect (con2,localhost,root)
XA START 'xid1'; XA START 'xid1';

View File

@ -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_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_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 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_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_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 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

View File

@ -1,6 +1,6 @@
# Copyright (c) 2006, 2017, Oracle and/or its affiliates. All rights reserved. # 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 # 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 # it under the terms of the GNU General Public License as published by
@ -278,7 +278,6 @@ SET(INNOBASE_SOURCES
lock/lock0iter.cc lock/lock0iter.cc
lock/lock0prdt.cc lock/lock0prdt.cc
lock/lock0lock.cc lock/lock0lock.cc
lock/lock0wait.cc
log/log0log.cc log/log0log.cc
log/log0recv.cc log/log0recv.cc
log/log0crypt.cc log/log0crypt.cc

View File

@ -961,7 +961,7 @@ static SHOW_VAR innodb_status_variables[]= {
{"data_written", &export_vars.innodb_data_written, SHOW_SIZE_T}, {"data_written", &export_vars.innodb_data_written, SHOW_SIZE_T},
{"dblwr_pages_written", &export_vars.innodb_dblwr_pages_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}, {"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}, {"history_list_length", &export_vars.innodb_history_list_length,SHOW_SIZE_T},
{"ibuf_discarded_delete_marks", &ibuf.n_discarded_ops[IBUF_OP_DELETE_MARK], {"ibuf_discarded_delete_marks", &ibuf.n_discarded_ops[IBUF_OP_DELETE_MARK],
SHOW_SIZE_T}, SHOW_SIZE_T},

View File

@ -642,6 +642,8 @@ public:
hash_table_t prdt_hash; hash_table_t prdt_hash;
/** page locks for SPATIAL INDEX */ /** page locks for SPATIAL INDEX */
hash_table_t prdt_page_hash; 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 */ /** mutex covering lock waits; @see trx_lock_t::wait_lock */
MY_ALIGNED(CACHE_LINE_SIZE) mysql_mutex_t wait_mutex; MY_ALIGNED(CACHE_LINE_SIZE) mysql_mutex_t wait_mutex;
@ -766,6 +768,16 @@ public:
{ return get_first(prdt_page_hash, id); } { 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 Creates a new record lock and inserts it to the lock queue. Does NOT check
for deadlocks or lock compatibility! for deadlocks or lock compatibility!
@ -835,9 +847,7 @@ Check for deadlocks.
@param[in,out] thr query thread @param[in,out] thr query thread
@param[in] prdt minimum bounding box (spatial index) @param[in] prdt minimum bounding box (spatial index)
@retval DB_LOCK_WAIT if the waiting lock was enqueued @retval DB_LOCK_WAIT if the waiting lock was enqueued
@retval DB_DEADLOCK if this transaction was chosen as the victim @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) */
dberr_t dberr_t
lock_rec_enqueue_waiting( lock_rec_enqueue_waiting(
#ifdef WITH_WSREP #ifdef WITH_WSREP
@ -871,9 +881,6 @@ lock_rec_free_all_from_discard_page(
/*================================*/ /*================================*/
const buf_block_t* block); /*!< in: page to be discarded */ 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 */ /** Cancel a waiting lock request and release possibly waiting transactions */
void lock_cancel_waiting_and_release(lock_t *lock); void lock_cancel_waiting_and_release(lock_t *lock);

View File

@ -391,9 +391,6 @@ static const byte lock_strength_matrix[5][5] = {
/* AI */ { FALSE, FALSE, FALSE, FALSE, TRUE} /* 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 #define PRDT_HEAPNO PAGE_HEAP_NO_INFIMUM
/** Record locking request status */ /** Record locking request status */
enum lock_rec_req_status { enum lock_rec_req_status {

View File

@ -187,9 +187,6 @@ struct srv_stats_t
/** Number of temporary tablespace blocks decrypted */ /** Number of temporary tablespace blocks decrypted */
ulint_ctr_64_t n_temp_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 /** We are prepared for a situation that we have this many threads waiting for

View File

@ -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 /** The locks and state of an active transaction. Protected by
lock_sys.mutex, trx->mutex or both. */ lock_sys.mutex, trx->mutex or both. */
struct trx_lock_t { struct trx_lock_t
{
/** Lock request being waited for. /** Lock request being waited for.
Set to nonnull when holding lock_sys.mutex, lock_sys.wait_mutex and Set to nonnull when holding lock_sys.mutex, lock_sys.wait_mutex and
trx->mutex, by the thread that is executing the transaction. 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 ib_uint64_t deadlock_mark; /*!< A mark field that is initialized
to and checked against lock_mark_counter to and checked against lock_mark_counter
by lock_deadlock_recursive(). */ by lock_deadlock_recursive(). */
bool was_chosen_as_deadlock_victim; /** When the transaction decides to wait for a lock, it clears this;
/*!< when the transaction decides to set if another transaction chooses this transaction as a victim in deadlock
wait for a lock, it sets this to false; resolution. Protected by lock_sys.mutex and lock_sys.wait_mutex. */
if another transaction chooses this bool was_chosen_as_deadlock_victim;
transaction as a victim in deadlock
resolution, it sets this to true.
Protected by trx->mutex. */
que_thr_t* wait_thr; /*!< query thread belonging to this que_thr_t* wait_thr; /*!< query thread belonging to this
trx that is in waiting trx that is in waiting
state. For threads suspended in a state. For threads suspended in a
@ -666,88 +664,94 @@ private:
public: 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 /** mutex protecting state and some of lock
(some are protected by lock_sys.mutex) */ (some are protected by lock_sys.mutex) */
srw_mutex 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 Possible states:
and the valid state transitions.
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 Valid state transitions are:
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: Regular transactions:
* NOT_STARTED -> ACTIVE -> COMMITTED -> NOT_STARTED
Regular transactions: Auto-commit non-locking read-only:
* NOT_STARTED -> ACTIVE -> COMMITTED -> NOT_STARTED * NOT_STARTED -> ACTIVE -> NOT_STARTED
Auto-commit non-locking read-only: XA (2PC):
* NOT_STARTED -> ACTIVE -> NOT_STARTED * NOT_STARTED -> ACTIVE -> PREPARED -> COMMITTED -> NOT_STARTED
XA (2PC): Recovered XA:
* NOT_STARTED -> ACTIVE -> PREPARED -> COMMITTED -> NOT_STARTED * NOT_STARTED -> PREPARED -> COMMITTED -> (freed)
Recovered XA: Recovered XA followed by XA ROLLBACK:
* NOT_STARTED -> PREPARED -> COMMITTED -> (freed) * NOT_STARTED -> PREPARED -> ACTIVE -> COMMITTED -> (freed)
Recovered XA followed by XA ROLLBACK: XA (2PC) (shutdown or disconnect before ROLLBACK or COMMIT):
* NOT_STARTED -> PREPARED -> ACTIVE -> COMMITTED -> (freed) * NOT_STARTED -> PREPARED -> (freed)
XA (2PC) (shutdown or disconnect before ROLLBACK or COMMIT): Disconnected XA PREPARE transaction can become recovered:
* NOT_STARTED -> PREPARED -> (freed) * ... -> ACTIVE -> PREPARED (connected) -> PREPARED (disconnected)
Disconnected XA can become recovered: Latching and various transaction lists membership rules:
* ... -> ACTIVE -> PREPARED (connected) -> PREPARED (disconnected)
Disconnected means from mysql e.g due to the mysql client disconnection.
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 Transitions to ACTIVE or NOT_STARTED occur when transaction
is not in rw_trx_hash. is not in rw_trx_hash.
Autocommit non-locking read-only transactions move between states Autocommit non-locking read-only transactions move between states
without holding any mutex. They are not in rw_trx_hash. without holding any mutex. They are not in rw_trx_hash.
All transactions, unless they are determined to be ac-nl-ro, All transactions, unless they are determined to be ac-nl-ro,
explicitly tagged as read-only or read-write, will first be put explicitly tagged as read-only or read-write, will first be put
on the read-only transaction list. Only when a !read-only transaction 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 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 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. 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 When a transaction is NOT_STARTED, it can be in trx_list. It cannot be
in rw_trx_hash. in rw_trx_hash.
ACTIVE->PREPARED->COMMITTED is only possible when trx is 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. The transition ACTIVE->PREPARED is protected by trx->mutex.
ACTIVE->COMMITTED is possible when the transaction is in ACTIVE->COMMITTED is possible when the transaction is in
rw_trx_hash. 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; 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 #ifdef WITH_WSREP
/** whether wsrep_on(mysql_thd) held at the start of transaction */ /** whether wsrep_on(mysql_thd) held at the start of transaction */
bool wsrep; bool wsrep;
bool is_wsrep() const { return UNIV_UNLIKELY(wsrep); } bool is_wsrep() const { return UNIV_UNLIKELY(wsrep); }
#else /* WITH_WSREP */ #else /* WITH_WSREP */
bool is_wsrep() const { return false; } bool is_wsrep() const { return false; }
#endif /* WITH_WSREP */ #endif /* WITH_WSREP */
ReadView read_view; /*!< consistent read view used in the /** Consistent read view of the transaction */
transaction, or NULL if not yet set */ ReadView read_view;
trx_lock_t lock; /*!< Information about the transaction
locks and state. Protected by
lock_sys.mutex (insertions also
by trx_t::mutex). */
/* These fields are not protected by any mutex. */ /* These fields are not protected by any mutex. */

View File

@ -29,6 +29,7 @@ Created 5/7/1996 Heikki Tuuri
#include "univ.i" #include "univ.i"
#include <mysql/service_thd_error_context.h> #include <mysql/service_thd_error_context.h>
#include <mysql/service_thd_wait.h>
#include <sql_class.h> #include <sql_class.h>
#include "lock0lock.h" #include "lock0lock.h"
@ -53,9 +54,11 @@ Created 5/7/1996 Heikki Tuuri
/** The value of innodb_deadlock_detect */ /** The value of innodb_deadlock_detect */
my_bool innobase_deadlock_detect; my_bool innobase_deadlock_detect;
#ifdef HAVE_REPLICATION
extern "C" void thd_rpl_deadlock_check(MYSQL_THD thd, MYSQL_THD other_thd); extern "C" void thd_rpl_deadlock_check(MYSQL_THD thd, MYSQL_THD other_thd);
extern "C" int thd_need_wait_reports(const MYSQL_THD thd); extern "C" int thd_need_wait_reports(const MYSQL_THD thd);
extern "C" int thd_need_ordering_with(const MYSQL_THD thd, const MYSQL_THD other_thd); extern "C" int thd_need_ordering_with(const MYSQL_THD thd, const MYSQL_THD other_thd);
#endif
/** Pretty-print a table lock. /** Pretty-print a table lock.
@param[in,out] file output stream @param[in,out] file output stream
@ -73,16 +76,13 @@ class DeadlockChecker {
public: public:
/** Check if a joining lock request results in a deadlock. /** Check if a joining lock request results in a deadlock.
If a deadlock is found, we will resolve the deadlock by If a deadlock is found, we will resolve the deadlock by
choosing a victim transaction and rolling it back. choosing a victim transaction to be rolled it back.
We will attempt to resolve all deadlocks. We will attempt to resolve all deadlocks.
@param[in] lock the lock request @param[in,out] trx transaction requesting a lock
@param[in,out] trx transaction requesting the lock
@return trx if it was chosen as victim @return whether the transaction was chosen as victim */
@retval NULL if another victim was chosen, static bool check_and_resolve(trx_t *trx);
or there is no deadlock (any more) */
static const trx_t* check_and_resolve(const lock_t* lock, trx_t* trx);
private: private:
/** Do a shallow copy. Default destructor OK. /** Do a shallow copy. Default destructor OK.
@ -91,18 +91,17 @@ private:
@param mark_start visited node counter @param mark_start visited node counter
@param report_waiters whether to call thd_rpl_deadlock_check() */ @param report_waiters whether to call thd_rpl_deadlock_check() */
DeadlockChecker( DeadlockChecker(
const trx_t* trx, trx_t* trx,
const lock_t* wait_lock, const lock_t* wait_lock,
ib_uint64_t mark_start, ib_uint64_t mark_start,
bool report_waiters) bool report_waiters)
: :
m_cost(), #ifdef HAVE_REPLICATION
m_report_waiters(report_waiters),
#endif
m_start(trx), m_start(trx),
m_too_deep(),
m_wait_lock(wait_lock), m_wait_lock(wait_lock),
m_mark_start(mark_start), m_mark_start(mark_start)
m_n_elems(),
m_report_waiters(report_waiters)
{ {
} }
@ -186,18 +185,14 @@ private:
@param lock lock causing deadlock */ @param lock lock causing deadlock */
void notify(const lock_t* lock) const; void notify(const lock_t* lock) const;
/** Select the victim transaction that should be rolledback. /** @return the victim transaction that should be rolled back */
@return victim transaction */ trx_t *select_victim() const;
const trx_t* select_victim() const;
/** Rollback transaction selected as the victim. */
void trx_rollback();
/** Looks iteratively for a deadlock. Note: the joining transaction /** Looks iteratively for a deadlock. Note: the joining transaction
may have been granted its lock by the deadlock checks. may have been granted its lock by the deadlock checks.
@return the victim transaction
@return 0 if no deadlock else the victim transaction.*/ @return nullptr if no deadlock */
const trx_t* search(); inline trx_t *search();
/** Print transaction data to the deadlock file and possibly to stderr. /** Print transaction data to the deadlock file and possibly to stderr.
@param trx transaction @param trx transaction
@ -234,36 +229,38 @@ private:
static ib_uint64_t s_lock_mark_counter; static ib_uint64_t s_lock_mark_counter;
/** Calculation steps thus far. It is the count of the nodes visited. */ /** Calculation steps thus far. It is the count of the nodes visited. */
ulint m_cost; ulint m_cost= 0;
#ifdef HAVE_REPLICATION
/** Set if thd_rpl_deadlock_check() should be called for waits. */
const bool m_report_waiters;
#endif
/** Joining transaction that is requesting a lock in an /** Joining transaction that is requesting a lock in an
incompatible mode */ incompatible mode */
const trx_t* m_start; trx_t* const m_start;
/** TRUE if search was too deep and was aborted */ /** TRUE if search was too deep and was aborted */
bool m_too_deep; bool m_too_deep= false;
/** Lock that trx wants */ /** Lock that trx wants */
const lock_t* m_wait_lock; const lock_t* m_wait_lock;
/** Value of lock_mark_count at the start of the deadlock check. */ /** Value of lock_mark_count at the start of the deadlock check. */
ib_uint64_t m_mark_start; const ib_uint64_t m_mark_start;
/** Number of states pushed onto the stack */ /** Number of states pushed onto the stack */
size_t m_n_elems; size_t m_n_elems= 0;
/** This is to avoid malloc/free calls. */ /** This is to avoid malloc/free calls. */
static state_t s_states[MAX_STACK_SIZE]; static state_t s_states[4096];
/** Set if thd_rpl_deadlock_check() should be called for waits. */
const bool m_report_waiters;
}; };
/** Counter to mark visited nodes during deadlock search. */ /** Counter to mark visited nodes during deadlock search. */
ib_uint64_t DeadlockChecker::s_lock_mark_counter = 0; ib_uint64_t DeadlockChecker::s_lock_mark_counter = 0;
/** The stack used for deadlock searches. */ /** The stack used for deadlock searches. */
DeadlockChecker::state_t DeadlockChecker::s_states[MAX_STACK_SIZE]; DeadlockChecker::state_t DeadlockChecker::s_states[4096];
#ifdef UNIV_DEBUG #ifdef UNIV_DEBUG
/*********************************************************************//** /*********************************************************************//**
@ -285,10 +282,6 @@ static bool lock_rec_validate_page(const buf_block_t *block, bool latched)
/* The lock system */ /* The lock system */
lock_sys_t lock_sys; lock_sys_t lock_sys;
/** We store info on the latest deadlock error to this buffer. InnoDB
Monitor will then fetch it and print */
static bool lock_deadlock_found = false;
/** Only created if !srv_read_only_mode */ /** Only created if !srv_read_only_mode */
static FILE* lock_latest_err_file; static FILE* lock_latest_err_file;
@ -507,6 +500,38 @@ static void wsrep_assert_no_bf_bf_wait(
/* BF-BF wait is a bug */ /* BF-BF wait is a bug */
ut_error; ut_error;
} }
/*********************************************************************//**
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.latch is held
@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) {
LockMutexGuard g;
trx_print_latched(stderr, trx, 3000);
} else {
lock_sys.mutex_assert_locked();
trx_print_latched(stderr, trx, 3000);
}
srv_print_innodb_monitor = TRUE;
srv_print_innodb_lock_monitor = TRUE;
srv_monitor_timer_schedule_now();
return true;
}
return false;
}
#endif /* WITH_WSREP */ #endif /* WITH_WSREP */
/*********************************************************************//** /*********************************************************************//**
@ -588,6 +613,7 @@ lock_rec_has_to_wait(
return false; return false;
} }
#ifdef HAVE_REPLICATION
if ((type_mode & LOCK_GAP || lock2->is_gap()) if ((type_mode & LOCK_GAP || lock2->is_gap())
&& !thd_need_ordering_with(trx->mysql_thd, lock2->trx->mysql_thd)) { && !thd_need_ordering_with(trx->mysql_thd, lock2->trx->mysql_thd)) {
/* If the upper server layer has already decided on the /* If the upper server layer has already decided on the
@ -612,6 +638,7 @@ lock_rec_has_to_wait(
return false; return false;
} }
#endif /* HAVE_REPLICATION */
#ifdef WITH_WSREP #ifdef WITH_WSREP
/* There should not be two conflicting locks that are /* There should not be two conflicting locks that are
@ -1207,10 +1234,11 @@ lock_rec_create_low(
if (holds_trx_mutex) { if (holds_trx_mutex) {
trx->mutex.wr_lock(); trx->mutex.wr_lock();
} }
c_lock->trx->mutex.wr_lock(); trx_t *ctrx = c_lock->trx;
if (c_lock->trx->lock.wait_thr) { ctrx->mutex.wr_lock();
if (ctrx->lock.wait_thr) {
c_lock->trx->lock.was_chosen_as_deadlock_victim = TRUE; ctrx->lock.was_chosen_as_deadlock_victim = true;
ctrx->lock.was_chosen_as_wsrep_victim = true;
if (UNIV_UNLIKELY(wsrep_debug)) { if (UNIV_UNLIKELY(wsrep_debug)) {
wsrep_print_wait_locks(c_lock); wsrep_print_wait_locks(c_lock);
@ -1229,20 +1257,20 @@ lock_rec_create_low(
trx->mutex.wr_unlock(); trx->mutex.wr_unlock();
} }
lock_cancel_waiting_and_release( lock_cancel_waiting_and_release(
c_lock->trx->lock.wait_lock); ctrx->lock.wait_lock);
if (holds_trx_mutex) { if (holds_trx_mutex) {
trx->mutex.wr_lock(); trx->mutex.wr_lock();
} }
c_lock->trx->mutex.wr_unlock(); ctrx->mutex.wr_unlock();
/* have to bail out here to avoid lock_set_lock... */ /* have to bail out here to avoid lock_set_lock... */
mysql_mutex_unlock(&lock_sys.wait_mutex); mysql_mutex_unlock(&lock_sys.wait_mutex);
return(lock); return(lock);
} }
mysql_mutex_unlock(&lock_sys.wait_mutex); mysql_mutex_unlock(&lock_sys.wait_mutex);
c_lock->trx->mutex.wr_unlock(); ctrx->mutex.wr_unlock();
} else } else
#endif /* WITH_WSREP */ #endif /* WITH_WSREP */
HASH_INSERT(lock_t, hash, lock_hash_get(type_mode), HASH_INSERT(lock_t, hash, lock_hash_get(type_mode),
@ -1281,9 +1309,7 @@ Check for deadlocks.
@param[in,out] thr query thread @param[in,out] thr query thread
@param[in] prdt minimum bounding box (spatial index) @param[in] prdt minimum bounding box (spatial index)
@retval DB_LOCK_WAIT if the waiting lock was enqueued @retval DB_LOCK_WAIT if the waiting lock was enqueued
@retval DB_DEADLOCK if this transaction was chosen as the victim @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) */
dberr_t dberr_t
lock_rec_enqueue_waiting( lock_rec_enqueue_waiting(
#ifdef WITH_WSREP #ifdef WITH_WSREP
@ -1333,33 +1359,7 @@ lock_rec_enqueue_waiting(
lock_prdt_set_prdt(lock, prdt); lock_prdt_set_prdt(lock, prdt);
} }
if (ut_d(const trx_t* victim =)
DeadlockChecker::check_and_resolve(lock, trx)) {
ut_ad(victim == trx);
/* There is no need to hold lock_sys.wait_mutex here,
because we are clearing the wait flag on a lock request
that is associated with the current transaction. So,
this is not conflicting with lock_wait(). */
lock_reset_lock_and_trx_wait(lock);
lock_rec_reset_nth_bit(lock, heap_no);
return DB_DEADLOCK;
}
if (!trx->lock.wait_lock) {
/* If there was a deadlock but we chose another
transaction as a victim, it is possible that we
already have the lock now granted! */
#ifdef WITH_WSREP
if (UNIV_UNLIKELY(wsrep_debug)) {
ib::info() << "WSREP: BF thread got lock granted early, ID " << ib::hex(trx->id)
<< " query: " << wsrep_thd_query(trx->mysql_thd);
}
#endif
return DB_SUCCESS_LOCKED_REC;
}
trx->lock.wait_thr = thr; trx->lock.wait_thr = thr;
trx->lock.was_chosen_as_deadlock_victim = false; trx->lock.was_chosen_as_deadlock_victim = false;
DBUG_LOG("ib_lock", "trx " << ib::hex(trx->id) DBUG_LOG("ib_lock", "trx " << ib::hex(trx->id)
@ -1668,6 +1668,178 @@ lock_rec_has_to_wait_in_queue(
return(NULL); return(NULL);
} }
/** 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
IF_WSREP(|| trx->lock.was_chosen_as_wsrep_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;
int wait_for= 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)
{
static_assert(THD_WAIT_TABLE_LOCK != 0, "compatibility");
static_assert(THD_WAIT_ROW_LOCK != 0, "compatibility");
wait_for= wait_lock->is_table() ? THD_WAIT_TABLE_LOCK : THD_WAIT_ROW_LOCK;
mysql_mutex_unlock(&lock_sys.wait_mutex);
if (had_dict_lock) /* Release foreign key check latch */
row_mysql_unfreeze_data_dictionary(trx);
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_for);
const bool deadlock= DeadlockChecker::check_and_resolve(trx);
if (deadlock)
trx->error_state= DB_DEADLOCK;
mysql_mutex_lock(&lock_sys.wait_mutex);
if (deadlock)
goto end_wait;
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;
}
}
else
had_dict_lock= false;
end_wait:
if (row_lock_wait)
lock_sys.wait_resume(trx->mysql_thd, suspend_time, my_hrtime_coarse());
mysql_mutex_unlock(&lock_sys.wait_mutex);
if (wait_for)
thd_wait_end(trx->mysql_thd);
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)
{
{
LockMutexGuard g;
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();
}
}
mysql_mutex_unlock(&lock_sys.wait_mutex);
}
return trx->error_state;
}
/** Resume a lock wait */ /** Resume a lock wait */
static void lock_wait_end(trx_t *trx) static void lock_wait_end(trx_t *trx)
@ -3196,8 +3368,7 @@ lock_table_remove_low(
Enqueues a waiting request for a table lock which cannot be granted Enqueues a waiting request for a table lock which cannot be granted
immediately. Checks for deadlocks. immediately. Checks for deadlocks.
@retval DB_LOCK_WAIT if the waiting lock was enqueued @retval DB_LOCK_WAIT if the waiting lock was enqueued
@retval DB_DEADLOCK if this transaction was chosen as the victim @retval DB_DEADLOCK if this transaction was chosen as the victim */
@retval DB_SUCCESS if the other transaction committed or aborted */
static static
dberr_t dberr_t
lock_table_enqueue_waiting( lock_table_enqueue_waiting(
@ -3212,7 +3383,6 @@ lock_table_enqueue_waiting(
) )
{ {
trx_t* trx; trx_t* trx;
lock_t* lock;
lock_sys.mutex_assert_locked(); lock_sys.mutex_assert_locked();
ut_ad(!srv_read_only_mode); ut_ad(!srv_read_only_mode);
@ -3237,40 +3407,16 @@ lock_table_enqueue_waiting(
#endif /* WITH_WSREP */ #endif /* WITH_WSREP */
/* Enqueue the lock request that will wait to be granted */ /* Enqueue the lock request that will wait to be granted */
lock = lock_table_create(table, mode | LOCK_WAIT, trx lock_table_create(table, mode | LOCK_WAIT, trx
#ifdef WITH_WSREP #ifdef WITH_WSREP
, c_lock , c_lock
#endif #endif
); );
const trx_t* victim_trx =
DeadlockChecker::check_and_resolve(lock, trx);
if (victim_trx) {
ut_ad(victim_trx == trx);
/* The order here is important, we don't want to
lose the state of the lock before calling remove. */
lock_table_remove_low(lock);
/* There is no need to hold lock_sys.wait_mutex here,
because we are clearing the wait flag on a lock request
that is associated with the current transaction. So,
this is not conflicting with lock_wait(). */
lock_reset_lock_and_trx_wait(lock);
return(DB_DEADLOCK);
} else if (trx->lock.wait_lock == NULL) {
/* Deadlock resolution chose another transaction as a victim,
and we accidentally got our lock granted! */
return(DB_SUCCESS);
}
trx->lock.wait_thr = thr; trx->lock.wait_thr = thr;
trx->lock.was_chosen_as_deadlock_victim = false; trx->lock.was_chosen_as_deadlock_victim = false;
MONITOR_INC(MONITOR_TABLELOCK_WAIT); MONITOR_INC(MONITOR_TABLELOCK_WAIT);
return(DB_LOCK_WAIT); return(DB_LOCK_WAIT);
} }
@ -3383,6 +3529,8 @@ lock_table(
trx_set_rw_mode(trx); trx_set_rw_mode(trx);
} }
err = DB_SUCCESS;
lock_sys.mutex_lock(); lock_sys.mutex_lock();
/* We have to check if the new lock is compatible with any locks /* We have to check if the new lock is compatible with any locks
@ -3393,22 +3541,14 @@ lock_table(
trx->mutex.wr_lock(); trx->mutex.wr_lock();
/* Another trx has a request on the table in an incompatible if (wait_for) {
mode: this trx may have to wait */ err = lock_table_enqueue_waiting(mode, table, thr
if (wait_for != NULL) {
err = lock_table_enqueue_waiting(flags | mode, table,
thr
#ifdef WITH_WSREP #ifdef WITH_WSREP
, wait_for , wait_for
#endif #endif
); );
} else { } else {
lock_table_create(table, flags | mode, trx); lock_table_create(table, mode, trx);
ut_a(!flags || mode == LOCK_S || mode == LOCK_X);
err = DB_SUCCESS;
} }
lock_sys.mutex_unlock(); lock_sys.mutex_unlock();
@ -4036,7 +4176,7 @@ lock_print_info_summary(
return(FALSE); return(FALSE);
} }
if (lock_deadlock_found) { if (lock_sys.deadlocks) {
fputs("------------------------\n" fputs("------------------------\n"
"LATEST DETECTED DEADLOCK\n" "LATEST DETECTED DEADLOCK\n"
"------------------------\n", file); "------------------------\n", file);
@ -5918,10 +6058,10 @@ DeadlockChecker::notify(const lock_t* lock) const
/* It is possible that the joining transaction was granted its /* It is possible that the joining transaction was granted its
lock when we rolled back some other waiting transaction. */ lock when we rolled back some other waiting transaction. */
if (m_start->lock.wait_lock != 0) { if (auto wait_lock= m_start->lock.wait_lock) {
print("*** (2) WAITING FOR THIS LOCK TO BE GRANTED:\n"); print("*** (2) WAITING FOR THIS LOCK TO BE GRANTED:\n");
print(m_start->lock.wait_lock); print(wait_lock);
} }
DBUG_PRINT("ib_lock", ("deadlock detected")); DBUG_PRINT("ib_lock", ("deadlock detected"));
@ -5938,43 +6078,39 @@ inline bool trx_weight_ge(const trx_t *a, const trx_t *b)
return a_notrans != b_notrans ? a_notrans : TRX_WEIGHT(a) >= TRX_WEIGHT(b); return a_notrans != b_notrans ? a_notrans : TRX_WEIGHT(a) >= TRX_WEIGHT(b);
} }
/** Select the victim transaction that should be rolledback. /** @return the victim transaction that should be rolled back */
@return victim transaction */ trx_t *DeadlockChecker::select_victim() const
const trx_t*
DeadlockChecker::select_victim() const
{ {
lock_sys.mutex_assert_locked(); lock_sys.mutex_assert_locked();
ut_ad(m_start->lock.wait_lock != 0); trx_t *lock_trx= m_wait_lock->trx;
ut_ad(m_wait_lock->trx != m_start); ut_ad(m_start->lock.wait_lock);
ut_ad(lock_trx != m_start);
if (trx_weight_ge(m_wait_lock->trx, m_start)) { if (trx_weight_ge(lock_trx, m_start))
/* The joining transaction is 'smaller', {
choose it as the victim and roll it back. */
#ifdef WITH_WSREP #ifdef WITH_WSREP
if (wsrep_thd_is_BF(m_start->mysql_thd, FALSE)) { if (m_start->is_wsrep() && wsrep_thd_is_BF(m_start->mysql_thd, FALSE))
return(m_wait_lock->trx); return lock_trx;
}
#endif /* WITH_WSREP */ #endif /* WITH_WSREP */
return(m_start); /* The joining transaction is 'smaller', choose it as the victim */
} return m_start;
}
#ifdef WITH_WSREP #ifdef WITH_WSREP
if (wsrep_thd_is_BF(m_wait_lock->trx->mysql_thd, FALSE)) { if (lock_trx->is_wsrep() && wsrep_thd_is_BF(lock_trx->mysql_thd, FALSE))
return(m_start); return m_start;
}
#endif /* WITH_WSREP */ #endif /* WITH_WSREP */
return(m_wait_lock->trx); return lock_trx;
} }
/** Looks iteratively for a deadlock. Note: the joining transaction may /** Looks iteratively for a deadlock. Note: the joining transaction may
have been granted its lock by the deadlock checks. have been granted its lock by the deadlock checks.
@return 0 if no deadlock else the victim transaction instance.*/ @return the victim transaction
const trx_t* @return nullptr if no deadlock */
DeadlockChecker::search() inline trx_t* DeadlockChecker::search()
{ {
lock_sys.mutex_assert_locked(); lock_sys.mutex_assert_locked();
mysql_mutex_assert_owner(&lock_sys.wait_mutex);
ut_ad(m_start != NULL); ut_ad(m_start != NULL);
ut_ad(m_wait_lock != NULL); ut_ad(m_wait_lock != NULL);
@ -6040,18 +6176,24 @@ DeadlockChecker::search()
return m_start; return m_start;
} }
#ifdef HAVE_REPLICATION
/* We do not need to report autoinc locks to the upper /* We do not need to report autoinc locks to the upper
layer. These locks are released before commit, so they layer. These locks are released before commit, so they
can not cause deadlocks with binlog-fixed commit can not cause deadlocks with binlog-fixed commit
order. */ order. */
if (m_report_waiters && !(lock->type_mode & LOCK_AUTO_INC)) { if (m_report_waiters) {
ut_ad(!(lock->type_mode & LOCK_AUTO_INC));
thd_rpl_deadlock_check(m_start->mysql_thd, thd_rpl_deadlock_check(m_start->mysql_thd,
trx->mysql_thd); trx->mysql_thd);
} }
#endif /* HAVE_REPLICATION */
lock_t* wait_lock = trx->lock.wait_lock; if (!trx->lock.wait_thr) {
} else if (lock_t* wait_lock = trx->lock.wait_lock) {
/* Note: Because we are not holding
lock_sys.wait_mutex here, trx->lock.wait_lock
could be reset soon after we read it. */
if (wait_lock && trx->lock.wait_thr) {
/* Another trx ahead has requested a lock in an /* Another trx ahead has requested a lock in an
incompatible mode, and is itself waiting for a lock. */ incompatible mode, and is itself waiting for a lock. */
@ -6066,12 +6208,12 @@ DeadlockChecker::search()
lock = get_first_lock(&heap_no); lock = get_first_lock(&heap_no);
if (is_visited(lock)) { if (!is_visited(lock)) {
lock = get_next_lock(lock, heap_no); continue;
} }
} else {
lock = get_next_lock(lock, heap_no);
} }
lock = get_next_lock(lock, heap_no);
} }
ut_a(lock == NULL && m_n_elems == 0); ut_a(lock == NULL && m_n_elems == 0);
@ -6105,119 +6247,90 @@ DeadlockChecker::rollback_print(const trx_t* trx, const lock_t* lock)
print(lock); print(lock);
} }
/** Rollback transaction selected as the victim. */
void
DeadlockChecker::trx_rollback()
{
lock_sys.mutex_assert_locked();
mysql_mutex_assert_owner(&lock_sys.wait_mutex);
trx_t* trx = m_wait_lock->trx;
print("*** WE ROLL BACK TRANSACTION (1)\n");
#ifdef WITH_WSREP
if (trx->is_wsrep() && wsrep_thd_is_SR(trx->mysql_thd)) {
wsrep_handle_SR_rollback(m_start->mysql_thd, trx->mysql_thd);
}
#endif
trx->mutex.wr_lock();
trx->lock.was_chosen_as_deadlock_victim = true;
lock_cancel_waiting_and_release(trx->lock.wait_lock);
trx->mutex.wr_unlock();
}
/** Check if a joining lock request results in a deadlock. /** Check if a joining lock request results in a deadlock.
If a deadlock is found, we will resolve the deadlock by If a deadlock is found, we will resolve the deadlock by
choosing a victim transaction and rolling it back. choosing a victim transaction to be rolled it back.
We will attempt to resolve all deadlocks. We will attempt to resolve all deadlocks.
@param[in] lock the lock request @param[in,out] trx transaction requesting a lock
@param[in,out] trx transaction requesting the lock
@return trx if it was chosen as victim @return whether the transaction was chosen as victim */
@retval NULL if another victim was chosen, inline bool DeadlockChecker::check_and_resolve(trx_t *trx)
or there is no deadlock (any more) */
const trx_t*
DeadlockChecker::check_and_resolve(const lock_t* lock, trx_t* trx)
{ {
lock_sys.mutex_assert_locked(); ut_ad(trx->state == TRX_STATE_ACTIVE);
check_trx_state(trx); ut_ad(!srv_read_only_mode);
ut_ad(!srv_read_only_mode);
if (!innobase_deadlock_detect) { if (!innobase_deadlock_detect || !trx->lock.wait_lock)
return(NULL); return false;
}
/* Release the mutex to obey the latching order. #ifdef HAVE_REPLICATION
This is safe, because DeadlockChecker::check_and_resolve() const bool report_waiters= thd_need_wait_reports(trx->mysql_thd);
is invoked when a lock wait is enqueued for the currently #else
running transaction. Because m_trx is a running transaction const bool report_waiters= false;
(it is not currently suspended because of a lock wait),
its state can only be changed by this thread, which is
currently associated with the transaction. */
trx->mutex.wr_unlock();
const trx_t* victim_trx;
const bool report_waiters = trx->mysql_thd
&& thd_need_wait_reports(trx->mysql_thd);
/* Try and resolve as many deadlocks as possible. */
do {
mysql_mutex_lock(&lock_sys.wait_mutex);
DeadlockChecker checker(trx, lock, s_lock_mark_counter,
report_waiters);
victim_trx = checker.search();
/* Search too deep, we rollback the joining transaction only
if it is possible to rollback. Otherwise we rollback the
transaction that is holding the lock that the joining
transaction wants. */
if (checker.is_too_deep()) {
ut_ad(trx == checker.m_start);
ut_ad(trx == victim_trx);
rollback_print(victim_trx, lock);
MONITOR_INC(MONITOR_DEADLOCK);
srv_stats.lock_deadlock_count.inc();
mysql_mutex_unlock(&lock_sys.wait_mutex);
break;
} else if (victim_trx != NULL && victim_trx != trx) {
ut_ad(victim_trx == checker.m_wait_lock->trx);
checker.trx_rollback();
lock_deadlock_found = true;
MONITOR_INC(MONITOR_DEADLOCK);
srv_stats.lock_deadlock_count.inc();
}
mysql_mutex_unlock(&lock_sys.wait_mutex);
} while (victim_trx != NULL && victim_trx != trx);
/* If the joining transaction was selected as the victim. */
if (victim_trx != NULL) {
print("*** WE ROLL BACK TRANSACTION (2)\n");
#ifdef WITH_WSREP
if (trx->is_wsrep() && wsrep_thd_is_SR(trx->mysql_thd)) {
wsrep_handle_SR_rollback(trx->mysql_thd,
victim_trx->mysql_thd);
}
#endif #endif
lock_deadlock_found = true; trx_t *victim_trx;
}
trx->mutex.wr_lock(); for (;;)
{
LockMutexGuard g; // FIXME: only lock_sys.wait_mutex?
lock_t *lock= trx->lock.wait_lock;
return(victim_trx); if (!lock)
return false;
if (trx->lock.was_chosen_as_deadlock_victim)
return true;
DeadlockChecker checker(trx, lock, s_lock_mark_counter, report_waiters &&
!(lock->type_mode & LOCK_AUTO_INC));
victim_trx= checker.search();
/* Search too deep, we rollback the joining transaction only if it
is possible to rollback. Otherwise we rollback the transaction
that is holding the lock that the joining transaction wants. */
if (checker.is_too_deep())
{
ut_ad(trx == checker.m_start);
ut_ad(trx == victim_trx);
rollback_print(trx, lock);
self_victim:
print("*** WE ROLL BACK TRANSACTION (2)\n");
}
else if (victim_trx == trx)
goto self_victim;
else if (!victim_trx)
return false;
else
{
mysql_mutex_lock(&lock_sys.wait_mutex);
victim_trx->mutex.wr_lock();
const auto state= victim_trx->state;
if (state != TRX_STATE_ACTIVE)
{
ut_ad(state == TRX_STATE_COMMITTED_IN_MEMORY);
mysql_mutex_unlock(&lock_sys.wait_mutex);
victim_trx->mutex.wr_unlock();
return false;
}
print("*** WE ROLL BACK TRANSACTION (1)\n");
victim_trx->lock.was_chosen_as_deadlock_victim= true;
lock_cancel_waiting_and_release(victim_trx->lock.wait_lock);
mysql_mutex_unlock(&lock_sys.wait_mutex);
victim_trx->mutex.wr_unlock();
}
lock_sys.deadlocks++;
break;
}
ut_ad(trx->state == TRX_STATE_ACTIVE);
#ifdef WITH_WSREP
if (victim_trx->is_wsrep() && wsrep_thd_is_SR(victim_trx->mysql_thd))
wsrep_handle_SR_rollback(trx->mysql_thd, victim_trx->mysql_thd);
#endif
return victim_trx == trx;
} }
/*************************************************************//** /*************************************************************//**

View File

@ -571,18 +571,11 @@ lock_prdt_insert_check_and_lock(
lock_sys.mutex_unlock(); lock_sys.mutex_unlock();
switch (err) { if (err == DB_SUCCESS) {
case DB_SUCCESS_LOCKED_REC:
err = DB_SUCCESS;
/* fall through */
case DB_SUCCESS:
/* Update the page max trx id field */ /* Update the page max trx id field */
page_update_max_trx_id(block, page_update_max_trx_id(block,
buf_block_get_page_zip(block), buf_block_get_page_zip(block),
trx->id, mtr); trx->id, mtr);
default:
/* We only care about the two return values. */
break;
} }
return(err); return(err);

View File

@ -1,234 +0,0 @@
/*****************************************************************************
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;
}

View File

@ -91,7 +91,8 @@ static monitor_info_t innodb_counter_info[] =
MONITOR_DEFAULT_START, MONITOR_MODULE_LOCK}, MONITOR_DEFAULT_START, MONITOR_MODULE_LOCK},
{"lock_deadlocks", "lock", "Number of deadlocks", {"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}, MONITOR_DEFAULT_START, MONITOR_DEADLOCK},
{"lock_timeouts", "lock", "Number of lock timeouts", {"lock_timeouts", "lock", "Number of lock timeouts",
@ -1879,6 +1880,9 @@ srv_mon_process_existing_counter(
case MONITOR_OVLD_PAGES_DECRYPTED: case MONITOR_OVLD_PAGES_DECRYPTED:
value = srv_stats.pages_decrypted; value = srv_stats.pages_decrypted;
break; break;
case MONITOR_DEADLOCK:
value = lock_sys.deadlocks;
break;
default: default:
ut_error; ut_error;

View File

@ -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_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_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 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_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_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 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