1
0
mirror of https://github.com/MariaDB/server.git synced 2025-08-08 11:22:35 +03:00

MDEV-20612: Replace lock_sys.mutex with lock_sys.latch

For now, we will acquire the lock_sys.latch only in exclusive mode,
that is, use it as a mutex.

This is preparation for the next commit where we will introduce
a less intrusive alternative, combining a shared lock_sys.latch
with dict_table_t::lock_mutex or a mutex embedded in
lock_sys.rec_hash, lock_sys.prdt_hash, or lock_sys.prdt_page_hash.
This commit is contained in:
Marko Mäkelä
2021-02-11 14:52:10 +02:00
parent 903464929c
commit b01d8e1a33
24 changed files with 344 additions and 281 deletions

View File

@@ -9,6 +9,7 @@ AND name!='wait/synch/rwlock/innodb/btr_search_latch' ORDER BY name;
name name
wait/synch/rwlock/innodb/dict_operation_lock wait/synch/rwlock/innodb/dict_operation_lock
wait/synch/rwlock/innodb/fil_space_latch wait/synch/rwlock/innodb/fil_space_latch
wait/synch/rwlock/innodb/lock_latch
wait/synch/rwlock/innodb/trx_i_s_cache_lock wait/synch/rwlock/innodb/trx_i_s_cache_lock
wait/synch/rwlock/innodb/trx_purge_latch wait/synch/rwlock/innodb/trx_purge_latch
TRUNCATE TABLE performance_schema.events_waits_history_long; TRUNCATE TABLE performance_schema.events_waits_history_long;
@@ -39,6 +40,7 @@ ORDER BY event_name;
event_name event_name
wait/synch/rwlock/innodb/dict_operation_lock wait/synch/rwlock/innodb/dict_operation_lock
wait/synch/rwlock/innodb/fil_space_latch wait/synch/rwlock/innodb/fil_space_latch
wait/synch/rwlock/innodb/lock_latch
SELECT event_name FROM performance_schema.events_waits_history_long SELECT event_name FROM performance_schema.events_waits_history_long
WHERE event_name = 'wait/synch/sxlock/innodb/index_tree_rw_lock' WHERE event_name = 'wait/synch/sxlock/innodb/index_tree_rw_lock'
AND operation IN ('try_shared_lock','shared_lock') LIMIT 1; AND operation IN ('try_shared_lock','shared_lock') LIMIT 1;

View File

@@ -3355,7 +3355,7 @@ btr_lift_page_up(
const page_id_t id{block->page.id()}; const page_id_t id{block->page.id()};
/* Free predicate page locks on the block */ /* Free predicate page locks on the block */
if (index->is_spatial()) { if (index->is_spatial()) {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
lock_prdt_page_free_from_discard( lock_prdt_page_free_from_discard(
id, &lock_sys.prdt_page_hash); id, &lock_sys.prdt_page_hash);
} }
@@ -3609,7 +3609,7 @@ retry:
} }
/* No GAP lock needs to be worrying about */ /* No GAP lock needs to be worrying about */
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
lock_prdt_page_free_from_discard( lock_prdt_page_free_from_discard(
id, &lock_sys.prdt_page_hash); id, &lock_sys.prdt_page_hash);
lock_rec_free_all_from_discard_page(id); lock_rec_free_all_from_discard_page(id);
@@ -3762,7 +3762,7 @@ retry:
merge_page, mtr); merge_page, mtr);
} }
const page_id_t id{block->page.id()}; const page_id_t id{block->page.id()};
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
lock_prdt_page_free_from_discard( lock_prdt_page_free_from_discard(
id, &lock_sys.prdt_page_hash); id, &lock_sys.prdt_page_hash);
lock_rec_free_all_from_discard_page(id); lock_rec_free_all_from_discard_page(id);

View File

@@ -1999,11 +1999,12 @@ retry_page_get:
trx_t* trx = thr_get_trx(cursor->thr); trx_t* trx = thr_get_trx(cursor->thr);
lock_prdt_t prdt; lock_prdt_t prdt;
lock_sys.mutex_lock(); {
lock_init_prdt_from_mbr( LockMutexGuard g{SRW_LOCK_CALL};
&prdt, &cursor->rtr_info->mbr, mode, lock_init_prdt_from_mbr(
trx->lock.lock_heap); &prdt, &cursor->rtr_info->mbr, mode,
lock_sys.mutex_unlock(); trx->lock.lock_heap);
}
if (rw_latch == RW_NO_LATCH && height != 0) { if (rw_latch == RW_NO_LATCH && height != 0) {
block->lock.s_lock(); block->lock.s_lock();

View File

@@ -2038,7 +2038,7 @@ withdraw_retry:
{found, withdraw_started, my_hrtime_coarse()}; {found, withdraw_started, my_hrtime_coarse()};
withdraw_started = current_time; withdraw_started = current_time;
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
trx_sys.trx_list.for_each(f); trx_sys.trx_list.for_each(f);
} }

View File

@@ -387,7 +387,7 @@ rtr_pcur_getnext_from_path(
trx_t* trx = thr_get_trx( trx_t* trx = thr_get_trx(
btr_cur->rtr_info->thr); btr_cur->rtr_info->thr);
{ {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
lock_init_prdt_from_mbr( lock_init_prdt_from_mbr(
&prdt, &btr_cur->rtr_info->mbr, &prdt, &btr_cur->rtr_info->mbr,
mode, trx->lock.lock_heap); mode, trx->lock.lock_heap);
@@ -1197,7 +1197,7 @@ rtr_check_discard_page(
mysql_mutex_unlock(&index->rtr_track->rtr_active_mutex); mysql_mutex_unlock(&index->rtr_track->rtr_active_mutex);
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
lock_prdt_page_free_from_discard(id, &lock_sys.prdt_hash); lock_prdt_page_free_from_discard(id, &lock_sys.prdt_hash);
lock_prdt_page_free_from_discard(id, &lock_sys.prdt_page_hash); lock_prdt_page_free_from_discard(id, &lock_sys.prdt_page_hash);
} }

View File

@@ -531,7 +531,6 @@ mysql_pfs_key_t srv_monitor_file_mutex_key;
mysql_pfs_key_t buf_dblwr_mutex_key; mysql_pfs_key_t buf_dblwr_mutex_key;
mysql_pfs_key_t trx_pool_mutex_key; mysql_pfs_key_t trx_pool_mutex_key;
mysql_pfs_key_t trx_pool_manager_mutex_key; mysql_pfs_key_t trx_pool_manager_mutex_key;
mysql_pfs_key_t lock_mutex_key;
mysql_pfs_key_t lock_wait_mutex_key; mysql_pfs_key_t lock_wait_mutex_key;
mysql_pfs_key_t trx_sys_mutex_key; mysql_pfs_key_t trx_sys_mutex_key;
mysql_pfs_key_t srv_threads_mutex_key; mysql_pfs_key_t srv_threads_mutex_key;
@@ -571,7 +570,6 @@ static PSI_mutex_info all_innodb_mutexes[] = {
PSI_KEY(buf_dblwr_mutex), PSI_KEY(buf_dblwr_mutex),
PSI_KEY(trx_pool_mutex), PSI_KEY(trx_pool_mutex),
PSI_KEY(trx_pool_manager_mutex), PSI_KEY(trx_pool_manager_mutex),
PSI_KEY(lock_mutex),
PSI_KEY(lock_wait_mutex), PSI_KEY(lock_wait_mutex),
PSI_KEY(srv_threads_mutex), PSI_KEY(srv_threads_mutex),
PSI_KEY(rtr_active_mutex), PSI_KEY(rtr_active_mutex),
@@ -588,6 +586,7 @@ mysql_pfs_key_t index_online_log_key;
mysql_pfs_key_t fil_space_latch_key; mysql_pfs_key_t fil_space_latch_key;
mysql_pfs_key_t trx_i_s_cache_lock_key; mysql_pfs_key_t trx_i_s_cache_lock_key;
mysql_pfs_key_t trx_purge_latch_key; mysql_pfs_key_t trx_purge_latch_key;
mysql_pfs_key_t lock_latch_key;
/* all_innodb_rwlocks array contains rwlocks that are /* all_innodb_rwlocks array contains rwlocks that are
performance schema instrumented if "UNIV_PFS_RWLOCK" performance schema instrumented if "UNIV_PFS_RWLOCK"
@@ -601,6 +600,7 @@ static PSI_rwlock_info all_innodb_rwlocks[] =
{ &fil_space_latch_key, "fil_space_latch", 0 }, { &fil_space_latch_key, "fil_space_latch", 0 },
{ &trx_i_s_cache_lock_key, "trx_i_s_cache_lock", 0 }, { &trx_i_s_cache_lock_key, "trx_i_s_cache_lock", 0 },
{ &trx_purge_latch_key, "trx_purge_latch", 0 }, { &trx_purge_latch_key, "trx_purge_latch", 0 },
{ &lock_latch_key, "lock_latch", 0 },
{ &index_tree_rw_lock_key, "index_tree_rw_lock", PSI_RWLOCK_FLAG_SX } { &index_tree_rw_lock_key, "index_tree_rw_lock", PSI_RWLOCK_FLAG_SX }
}; };
# endif /* UNIV_PFS_RWLOCK */ # endif /* UNIV_PFS_RWLOCK */
@@ -2307,7 +2307,7 @@ innobase_trx_init(
DBUG_ASSERT(thd == trx->mysql_thd); DBUG_ASSERT(thd == trx->mysql_thd);
/* Ensure that thd_lock_wait_timeout(), which may be called /* Ensure that thd_lock_wait_timeout(), which may be called
while holding lock_sys.mutex, by lock_rec_enqueue_waiting(), while holding lock_sys.latch, by lock_rec_enqueue_waiting(),
will not end up acquiring LOCK_global_system_variables in will not end up acquiring LOCK_global_system_variables in
intern_sys_var_ptr(). */ intern_sys_var_ptr(). */
THDVAR(thd, lock_wait_timeout); THDVAR(thd, lock_wait_timeout);
@@ -2695,7 +2695,7 @@ static bool innobase_query_caching_table_check_low(
return false; return false;
} }
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
return UT_LIST_GET_LEN(table->locks) == 0; return UT_LIST_GET_LEN(table->locks) == 0;
} }
@@ -4466,7 +4466,7 @@ static void innobase_kill_query(handlerton*, THD *thd, enum thd_kill_levels)
if (trx->lock.wait_lock) if (trx->lock.wait_lock)
{ {
{ {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
mysql_mutex_lock(&lock_sys.wait_mutex); mysql_mutex_lock(&lock_sys.wait_mutex);
if (lock_t *lock= trx->lock.wait_lock) if (lock_t *lock= trx->lock.wait_lock)
{ {
@@ -17998,7 +17998,7 @@ int wsrep_innobase_kill_one_trx(THD *bf_thd, trx_t *victim_trx, bool signal)
{ {
ut_ad(bf_thd); ut_ad(bf_thd);
ut_ad(victim_trx); ut_ad(victim_trx);
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
ut_ad(victim_trx->mutex_is_owner()); ut_ad(victim_trx->mutex_is_owner());
DBUG_ENTER("wsrep_innobase_kill_one_trx"); DBUG_ENTER("wsrep_innobase_kill_one_trx");
@@ -18102,7 +18102,7 @@ wsrep_abort_transaction(
wsrep_thd_transaction_state_str(victim_thd)); wsrep_thd_transaction_state_str(victim_thd));
if (victim_trx) { if (victim_trx) {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
victim_trx->mutex_lock(); victim_trx->mutex_lock();
int rcode= wsrep_innobase_kill_one_trx(bf_thd, int rcode= wsrep_innobase_kill_one_trx(bf_thd,
victim_trx, signal); victim_trx, signal);

View File

@@ -3280,7 +3280,7 @@ commit_exit:
ibuf_mtr_commit(&bitmap_mtr); ibuf_mtr_commit(&bitmap_mtr);
goto fail_exit; goto fail_exit;
} else { } else {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
if (lock_sys.get_first(page_id)) { if (lock_sys.get_first(page_id)) {
goto commit_exit; goto commit_exit;
} }

View File

@@ -2261,7 +2261,7 @@ public:
kept in trx_t. In order to quickly determine whether a transaction has kept in trx_t. In order to quickly determine whether a transaction has
locked the AUTOINC lock we keep a pointer to the transaction here in locked the AUTOINC lock we keep a pointer to the transaction here in
the 'autoinc_trx' member. This is to avoid acquiring the the 'autoinc_trx' member. This is to avoid acquiring the
lock_sys_t::mutex and scanning the vector in trx_t. lock_sys.latch and scanning the vector in trx_t.
When an AUTOINC lock has to wait, the corresponding lock instance is When an AUTOINC lock has to wait, the corresponding lock instance is
created on the trx lock heap rather than use the pre-allocated instance created on the trx lock heap rather than use the pre-allocated instance
in autoinc_lock below. */ in autoinc_lock below. */
@@ -2280,11 +2280,11 @@ public:
ib_uint64_t autoinc; ib_uint64_t autoinc;
/** The transaction that currently holds the the AUTOINC lock on this /** The transaction that currently holds the the AUTOINC lock on this
table. Protected by lock_sys.mutex. */ table. Protected by lock_sys.latch. */
const trx_t* autoinc_trx; const trx_t* autoinc_trx;
/** Number of granted or pending autoinc_lock on this table. This /** Number of granted or pending autoinc_lock on this table. This
value is set after acquiring lock_sys.mutex but value is set after acquiring lock_sys.latch but
in innodb_autoinc_lock_mode=1 (the default), in innodb_autoinc_lock_mode=1 (the default),
ha_innobase::innobase_lock_autoinc() will perform a dirty read ha_innobase::innobase_lock_autoinc() will perform a dirty read
to determine whether other transactions have acquired the autoinc_lock. */ to determine whether other transactions have acquired the autoinc_lock. */
@@ -2292,7 +2292,8 @@ public:
/* @} */ /* @} */
/** Number of granted or pending LOCK_S or LOCK_X on the table */ /** Number of granted or pending LOCK_S or LOCK_X on the table.
Protected by lock_sys.assert_locked(*this). */
uint32_t n_lock_x_or_s; uint32_t n_lock_x_or_s;
/** FTS specific state variables. */ /** FTS specific state variables. */
@@ -2305,8 +2306,8 @@ public:
/** Count of the number of record locks on this table. We use this to /** Count of the number of record locks on this table. We use this to
determine whether we can evict the table from the dictionary cache. determine whether we can evict the table from the dictionary cache.
It is protected by lock_sys.mutex. */ Protected by LockGuard. */
ulint n_rec_locks; ulint n_rec_locks;
private: private:
/** Count of how many handles are opened to this table. Dropping of the /** Count of how many handles are opened to this table. Dropping of the
@@ -2314,8 +2315,8 @@ private:
itself check the number of open handles at DROP. */ itself check the number of open handles at DROP. */
Atomic_counter<uint32_t> n_ref_count; Atomic_counter<uint32_t> n_ref_count;
public: public:
/** List of locks on the table. Protected by lock_sys.mutex. */ /** List of locks on the table. Protected by lock_sys.assert_locked(lock). */
table_lock_list_t locks; table_lock_list_t locks;
/** Timestamp of the last modification of this table. */ /** Timestamp of the last modification of this table. */
time_t update_time; time_t update_time;

View File

@@ -449,13 +449,12 @@ lock_report_trx_id_insanity(
trx_id_t max_trx_id); /*!< in: trx_sys.get_max_trx_id() */ trx_id_t max_trx_id); /*!< in: trx_sys.get_max_trx_id() */
/*********************************************************************//** /*********************************************************************//**
Prints info of locks for all transactions. Prints info of locks for all transactions.
@return FALSE if not able to obtain lock mutex and exits without @return FALSE if not able to acquire lock_sys.latch (and display info) */
printing info */
ibool ibool
lock_print_info_summary( lock_print_info_summary(
/*====================*/ /*====================*/
FILE* file, /*!< in: file where to print */ FILE* file, /*!< in: file where to print */
ibool nowait) /*!< in: whether to wait for the lock mutex */ ibool nowait) /*!< in: whether to wait for lock_sys.latch */
MY_ATTRIBUTE((warn_unused_result)); MY_ATTRIBUTE((warn_unused_result));
/** Prints transaction lock wait and MVCC state. /** Prints transaction lock wait and MVCC state.
@@ -466,9 +465,8 @@ void lock_trx_print_wait_and_mvcc_state(FILE *file, const trx_t *trx,
my_hrtime_t now); my_hrtime_t now);
/*********************************************************************//** /*********************************************************************//**
Prints info of locks for each transaction. This function assumes that the Prints info of locks for each transaction. This function will release
caller holds the lock mutex and more importantly it will release the lock lock_sys.latch, which the caller must be holding in exclusive mode. */
mutex on behalf of the caller. (This should be fixed in the future). */
void void
lock_print_info_all_transactions( lock_print_info_all_transactions(
/*=============================*/ /*=============================*/
@@ -476,7 +474,7 @@ lock_print_info_all_transactions(
/*********************************************************************//** /*********************************************************************//**
Return the number of table locks for a transaction. Return the number of table locks for a transaction.
The caller must be holding lock_sys.mutex. */ The caller must be holding lock_sys.latch. */
ulint ulint
lock_number_of_tables_locked( lock_number_of_tables_locked(
/*=========================*/ /*=========================*/
@@ -583,7 +581,13 @@ class lock_sys_t
bool m_initialised; bool m_initialised;
/** mutex proteting the locks */ /** mutex proteting the locks */
MY_ALIGNED(CACHE_LINE_SIZE) mysql_mutex_t mutex; MY_ALIGNED(CACHE_LINE_SIZE) srw_lock latch;
#ifdef UNIV_DEBUG
/** The owner of exclusive latch (0 if none); protected by latch */
std::atomic<os_thread_id_t> writer{0};
/** Number of shared latches */
std::atomic<ulint> readers{0};
#endif
public: public:
/** record locks */ /** record locks */
hash_table_t rec_hash; hash_table_t rec_hash;
@@ -615,31 +619,77 @@ public:
lock_sys_t(): m_initialised(false) {} lock_sys_t(): m_initialised(false) {}
bool is_initialised() { return m_initialised; } bool is_initialised() const { return m_initialised; }
#ifdef HAVE_PSI_MUTEX_INTERFACE #ifdef UNIV_PFS_RWLOCK
/** Try to acquire lock_sys.mutex */ /** Acquire exclusive lock_sys.latch */
ATTRIBUTE_NOINLINE int mutex_trylock(); ATTRIBUTE_NOINLINE
/** Acquire lock_sys.mutex */ void wr_lock(const char *file, unsigned line);
ATTRIBUTE_NOINLINE void mutex_lock(); /** Release exclusive lock_sys.latch */
/** Release lock_sys.mutex */ ATTRIBUTE_NOINLINE void wr_unlock();
ATTRIBUTE_NOINLINE void mutex_unlock(); /** Acquire shared lock_sys.latch */
ATTRIBUTE_NOINLINE void rd_lock(const char *file, unsigned line);
/** Release shared lock_sys.latch */
ATTRIBUTE_NOINLINE void rd_unlock();
#else #else
/** Try to acquire lock_sys.mutex */ /** Acquire exclusive lock_sys.latch */
int mutex_trylock() { return mysql_mutex_trylock(&mutex); } void wr_lock()
/** Aqcuire lock_sys.mutex */ {
void mutex_lock() { mysql_mutex_lock(&mutex); } ut_ad(!is_writer());
/** Release lock_sys.mutex */ latch.wr_lock();
void mutex_unlock() { mysql_mutex_unlock(&mutex); } ut_ad(!writer.exchange(os_thread_get_curr_id(),
std::memory_order_relaxed));
}
/** Release exclusive lock_sys.latch */
void wr_unlock()
{
ut_ad(writer.exchange(0, std::memory_order_relaxed) ==
os_thread_get_curr_id());
latch.wr_unlock();
}
/** Acquire shared lock_sys.latch */
void rd_lock()
{
ut_ad(!is_writer());
latch.rd_lock();
ut_ad(!writer.load(std::memory_order_relaxed));
ut_d(readers.fetch_add(1, std::memory_order_relaxed));
}
/** Release shared lock_sys.latch */
void rd_unlock()
{
ut_ad(!is_writer());
ut_ad(readers.fetch_sub(1, std::memory_order_relaxed));
latch.rd_unlock();
}
#endif #endif
/** Assert that mutex_lock() has been invoked */ /** Try to acquire exclusive lock_sys.latch
void mutex_assert_locked() const { mysql_mutex_assert_owner(&mutex); } @return whether the latch was acquired */
/** Assert that mutex_lock() has not been invoked */ bool wr_lock_try()
void mutex_assert_unlocked() const { mysql_mutex_assert_not_owner(&mutex); } {
ut_ad(!is_writer());
if (!latch.wr_lock_try()) return false;
ut_ad(!writer.exchange(os_thread_get_curr_id(),
std::memory_order_relaxed));
return true;
}
/** Wait for a lock to be granted */ /** Assert that wr_lock() has been invoked by this thread */
void wait_lock(lock_t **lock, pthread_cond_t *cond) void assert_locked() const { ut_ad(is_writer()); }
{ while (*lock) my_cond_wait(cond, &mutex.m_mutex); } /** Assert that wr_lock() has not been invoked by this thread */
void assert_unlocked() const { ut_ad(!is_writer()); }
#ifdef UNIV_DEBUG
/** @return whether the current thread is the lock_sys.latch writer */
bool is_writer() const
{ return writer.load(std::memory_order_relaxed) == os_thread_get_curr_id(); }
/** Assert that a lock shard is exclusively latched by this thread */
void assert_locked(const lock_t &lock) const;
/** Assert that a table lock shard is exclusively latched by this thread */
void assert_locked(const dict_table_t &table) const;
#else
void assert_locked(const lock_t &) const {}
void assert_locked(const dict_table_t &) const {}
#endif
/** /**
Creates the lock system at database start. Creates the lock system at database start.
@@ -678,7 +728,7 @@ public:
/** @return the hash value for a page address */ /** @return the hash value for a page address */
ulint hash(const page_id_t id) const ulint hash(const page_id_t id) const
{ mysql_mutex_assert_owner(&mutex); return rec_hash.calc_hash(id.fold()); } { assert_locked(); return rec_hash.calc_hash(id.fold()); }
/** Get the first lock on a page. /** Get the first lock on a page.
@param lock_hash hash table to look at @param lock_hash hash table to look at
@@ -720,11 +770,12 @@ public:
/** The lock system */ /** The lock system */
extern lock_sys_t lock_sys; extern lock_sys_t lock_sys;
/** lock_sys.mutex guard */ /** lock_sys.latch guard */
struct LockMutexGuard struct LockMutexGuard
{ {
LockMutexGuard() { lock_sys.mutex_lock(); } LockMutexGuard(SRW_LOCK_ARGS(const char *file, unsigned line))
~LockMutexGuard() { lock_sys.mutex_unlock(); } { lock_sys.wr_lock(SRW_LOCK_ARGS(file, line)); }
~LockMutexGuard() { lock_sys.wr_unlock(); }
}; };
/*********************************************************************//** /*********************************************************************//**

View File

@@ -481,7 +481,7 @@ lock_rec_set_nth_bit(
inline byte lock_rec_reset_nth_bit(lock_t* lock, ulint i) inline byte lock_rec_reset_nth_bit(lock_t* lock, ulint i)
{ {
ut_ad(!lock->is_table()); ut_ad(!lock->is_table());
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
ut_ad(i < lock->un_member.rec_lock.n_bits); ut_ad(i < lock->un_member.rec_lock.n_bits);
byte* b = reinterpret_cast<byte*>(&lock[1]) + (i >> 3); byte* b = reinterpret_cast<byte*>(&lock[1]) + (i >> 3);

View File

@@ -78,7 +78,7 @@ lock_rec_set_nth_bit(
ulint bit_index; ulint bit_index;
ut_ad(!lock->is_table()); ut_ad(!lock->is_table());
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
ut_ad(i < lock->un_member.rec_lock.n_bits); ut_ad(i < lock->un_member.rec_lock.n_bits);
byte_index = i / 8; byte_index = i / 8;
@@ -117,7 +117,7 @@ lock_rec_get_next(
ulint heap_no,/*!< in: heap number of the record */ ulint heap_no,/*!< in: heap number of the record */
lock_t* lock) /*!< in: lock */ lock_t* lock) /*!< in: lock */
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
do { do {
lock = lock_rec_get_next_on_page(lock); lock = lock_rec_get_next_on_page(lock);
@@ -175,7 +175,7 @@ lock_rec_get_next_on_page_const(
ut_ad(!lock->is_table()); ut_ad(!lock->is_table());
const page_id_t page_id{lock->un_member.rec_lock.page_id}; const page_id_t page_id{lock->un_member.rec_lock.page_id};
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
while (!!(lock= static_cast<const lock_t*>(HASH_GET_NEXT(hash, lock)))) while (!!(lock= static_cast<const lock_t*>(HASH_GET_NEXT(hash, lock))))
if (lock->un_member.rec_lock.page_id == page_id) if (lock->un_member.rec_lock.page_id == page_id)

View File

@@ -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, 2020, MariaDB Corporation. Copyright (c) 2018, 2021, 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
@@ -145,7 +145,7 @@ operator<<(std::ostream& out, const lock_rec_t& lock)
#endif #endif
/* @} */ /* @} */
/** Lock struct; protected by lock_sys.mutex */ /** Lock struct; protected by lock_sys.latch */
struct ib_lock_t struct ib_lock_t
{ {
trx_t* trx; /*!< transaction owning the trx_t* trx; /*!< transaction owning the

View File

@@ -593,10 +593,10 @@ public:
the transaction may get committed before this method returns. the transaction may get committed before this method returns.
With do_ref_count == false the caller may dereference returned trx pointer With do_ref_count == false the caller may dereference returned trx pointer
only if lock_sys.mutex was acquired before calling find(). only if lock_sys.latch was acquired before calling find().
With do_ref_count == true caller may dereference trx even if it is not With do_ref_count == true caller may dereference trx even if it is not
holding lock_sys.mutex. Caller is responsible for calling holding lock_sys.latch. Caller is responsible for calling
trx->release_reference() when it is done playing with trx. trx->release_reference() when it is done playing with trx.
Ideally this method should get caller rw_trx_hash_pins along with trx Ideally this method should get caller rw_trx_hash_pins along with trx

View File

@@ -264,7 +264,7 @@ trx_print_latched(
/**********************************************************************//** /**********************************************************************//**
Prints info about a transaction. Prints info about a transaction.
Acquires and releases lock_sys.mutex. */ Acquires and releases lock_sys.latch. */
void void
trx_print( trx_print(
/*======*/ /*======*/
@@ -417,11 +417,11 @@ The transaction must have mysql_thd assigned. */
typedef std::vector<ib_lock_t*, ut_allocator<ib_lock_t*> > lock_list; 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.latch, 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.latch, 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.
Set to nullptr when holding lock_sys.wait_mutex. */ Set to nullptr when holding lock_sys.wait_mutex. */
Atomic_relaxed<lock_t*> wait_lock; Atomic_relaxed<lock_t*> wait_lock;
@@ -440,14 +440,14 @@ struct trx_lock_t
#else #else
/** When the transaction decides to wait for a lock, it clears this; /** When the transaction decides to wait for a lock, it clears this;
set if another transaction chooses this transaction as a victim in deadlock set if another transaction chooses this transaction as a victim in deadlock
resolution. Protected by lock_sys.mutex and lock_sys.wait_mutex. */ resolution. Protected by lock_sys.latch and lock_sys.wait_mutex. */
bool was_chosen_as_deadlock_victim; bool was_chosen_as_deadlock_victim;
#endif #endif
/** Whether the transaction is being rolled back either via deadlock /** Whether the transaction is being rolled back either via deadlock
detection or timeout. The caller has to acquire the trx_t::mutex in detection or timeout. The caller has to acquire the trx_t::mutex in
order to cancel the locks. In lock_trx_table_locks_remove() we must order to cancel the locks. In lock_trx_table_locks_remove() we must
avoid reacquiring the trx_t::mutex to prevent recursive avoid reacquiring the trx_t::mutex to prevent recursive
deadlocks. Protected by both lock_sys.mutex and trx_t::mutex. */ deadlocks. Protected by both lock_sys.latch and trx_t::mutex. */
bool cancel; bool cancel;
/** Next available rec_pool[] entry */ /** Next available rec_pool[] entry */
@@ -459,7 +459,7 @@ struct trx_lock_t
trx that is in waiting trx that is in waiting
state. For threads suspended in a state. For threads suspended in a
lock wait, this is protected by lock wait, this is protected by
lock_sys.mutex. Otherwise, this may lock_sys.latch. Otherwise, this may
only be modified by the thread that is only be modified by the thread that is
serving the running transaction. */ serving the running transaction. */
@@ -472,12 +472,12 @@ struct trx_lock_t
ib_lock_t table_pool[8]; ib_lock_t table_pool[8];
mem_heap_t* lock_heap; /*!< memory heap for trx_locks; mem_heap_t* lock_heap; /*!< memory heap for trx_locks;
protected by lock_sys.mutex */ protected by lock_sys.latch */
trx_lock_list_t trx_locks; /*!< locks requested by the transaction; trx_lock_list_t trx_locks; /*!< locks requested by the transaction;
insertions are protected by trx->mutex insertions are protected by trx->mutex
and lock_sys.mutex; removals are and lock_sys.latch; removals are
protected by lock_sys.mutex */ protected by lock_sys.latch */
lock_list table_locks; /*!< All table locks requested by this lock_list table_locks; /*!< All table locks requested by this
transaction, including AUTOINC locks */ transaction, including AUTOINC locks */
@@ -603,7 +603,7 @@ transactions (state == TRX_STATE_ACTIVE && is_recovered)
while the system is already processing new user transactions (!is_recovered). while the system is already processing new user transactions (!is_recovered).
* 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 lock_sys.mutex. thread. The caller must be holding lock_sys.latch.
* When a transaction handle is in the trx_sys.trx_list, some of its fields * When a transaction handle is in the trx_sys.trx_list, some of its fields
must not be modified without holding trx->mutex. must not be modified without holding trx->mutex.
@@ -611,7 +611,7 @@ must not be modified without holding trx->mutex.
* 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 (insertions also by trx->mutex). */ lock_sys.latch (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 {
@@ -669,7 +669,7 @@ public:
private: private:
/** 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.latch) */
srw_mutex mutex; srw_mutex mutex;
#ifdef UNIV_DEBUG #ifdef UNIV_DEBUG
/** The owner of mutex (0 if none); protected by mutex */ /** The owner of mutex (0 if none); protected by mutex */
@@ -766,7 +766,7 @@ public:
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 /** The locks of the transaction. Protected by lock_sys.latch
(insertions also by trx_t::mutex). */ (insertions also by trx_t::mutex). */
trx_lock_t lock; trx_lock_t lock;
@@ -918,7 +918,7 @@ public:
also in the lock list trx_locks. This also in the lock list trx_locks. This
vector needs to be freed explicitly vector needs to be freed explicitly
when the trx instance is destroyed. when the trx instance is destroyed.
Protected by lock_sys.mutex. */ Protected by lock_sys.latch. */
/*------------------------------*/ /*------------------------------*/
bool read_only; /*!< true if transaction is flagged bool read_only; /*!< true if transaction is flagged
as a READ-ONLY transaction. as a READ-ONLY transaction.

View File

@@ -1,7 +1,7 @@
/***************************************************************************** /*****************************************************************************
Copyright (c) 1994, 2016, Oracle and/or its affiliates. All Rights Reserved. Copyright (c) 1994, 2016, Oracle and/or its affiliates. All Rights Reserved.
Copyright (c) 2013, 2020, MariaDB Corporation. Copyright (c) 2013, 2021, MariaDB Corporation.
Copyright (c) 2008, Google Inc. Copyright (c) 2008, Google Inc.
Portions of this file contain modifications contributed and copyrighted by Portions of this file contain modifications contributed and copyrighted by
@@ -607,7 +607,6 @@ extern mysql_pfs_key_t srv_monitor_file_mutex_key;
extern mysql_pfs_key_t buf_dblwr_mutex_key; extern mysql_pfs_key_t buf_dblwr_mutex_key;
extern mysql_pfs_key_t trx_pool_mutex_key; extern mysql_pfs_key_t trx_pool_mutex_key;
extern mysql_pfs_key_t trx_pool_manager_mutex_key; extern mysql_pfs_key_t trx_pool_manager_mutex_key;
extern mysql_pfs_key_t lock_mutex_key;
extern mysql_pfs_key_t lock_wait_mutex_key; extern mysql_pfs_key_t lock_wait_mutex_key;
extern mysql_pfs_key_t srv_threads_mutex_key; extern mysql_pfs_key_t srv_threads_mutex_key;
extern mysql_pfs_key_t thread_mutex_key; extern mysql_pfs_key_t thread_mutex_key;
@@ -622,5 +621,6 @@ extern mysql_pfs_key_t trx_purge_latch_key;
extern mysql_pfs_key_t index_tree_rw_lock_key; extern mysql_pfs_key_t index_tree_rw_lock_key;
extern mysql_pfs_key_t index_online_log_key; extern mysql_pfs_key_t index_online_log_key;
extern mysql_pfs_key_t trx_sys_rw_lock_key; extern mysql_pfs_key_t trx_sys_rw_lock_key;
extern mysql_pfs_key_t lock_latch_key;
# endif /* UNIV_PFS_RWLOCK */ # endif /* UNIV_PFS_RWLOCK */
#endif /* HAVE_PSI_INTERFACE */ #endif /* HAVE_PSI_INTERFACE */

View File

@@ -49,7 +49,7 @@ lock_queue_iterator_reset(
ulint bit_no) /*!< in: record number in the ulint bit_no) /*!< in: record number in the
heap */ heap */
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked(*lock);
iter->current_lock = lock; iter->current_lock = lock;
@@ -75,7 +75,7 @@ lock_queue_iterator_get_prev(
/*=========================*/ /*=========================*/
lock_queue_iterator_t* iter) /*!< in/out: iterator */ lock_queue_iterator_t* iter) /*!< in/out: iterator */
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked(*iter->current_lock);
const lock_t *prev_lock= !iter->current_lock->is_table() const lock_t *prev_lock= !iter->current_lock->is_table()
? lock_rec_get_prev(iter->current_lock, iter->bit_no) ? lock_rec_get_prev(iter->current_lock, iter->bit_no)

View File

@@ -225,7 +225,7 @@ private:
ulint m_heap_no; /*!< heap number if rec lock */ ulint m_heap_no; /*!< heap number if rec lock */
}; };
/** Used in deadlock tracking. Protected by lock_sys.mutex. */ /** Used in deadlock tracking. Protected by lock_sys.latch. */
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. */
@@ -342,7 +342,7 @@ void lock_sys_t::create(ulint n_cells)
m_initialised= true; m_initialised= true;
mysql_mutex_init(lock_mutex_key, &mutex, nullptr); latch.SRW_LOCK_INIT(lock_latch_key);
mysql_mutex_init(lock_wait_mutex_key, &wait_mutex, nullptr); mysql_mutex_init(lock_wait_mutex_key, &wait_mutex, nullptr);
rec_hash.create(n_cells); rec_hash.create(n_cells);
@@ -357,15 +357,37 @@ void lock_sys_t::create(ulint n_cells)
} }
#ifdef HAVE_PSI_MUTEX_INTERFACE #ifdef UNIV_PFS_RWLOCK
/** Try to acquire lock_sys.mutex */ /** Acquire exclusive lock_sys.latch */
int lock_sys_t::mutex_trylock() { return mysql_mutex_trylock(&mutex); } void lock_sys_t::wr_lock(const char *file, unsigned line)
/** Acquire lock_sys.mutex */ {
void lock_sys_t::mutex_lock() { mysql_mutex_lock(&mutex); } latch.wr_lock(file, line);
/** Release lock_sys.mutex */ ut_ad(!writer.exchange(os_thread_get_curr_id(), std::memory_order_relaxed));
void lock_sys_t::mutex_unlock() { mysql_mutex_unlock(&mutex); } }
#endif /** Release exclusive lock_sys.latch */
void lock_sys_t::wr_unlock()
{
ut_ad(writer.exchange(0, std::memory_order_relaxed) ==
os_thread_get_curr_id());
latch.wr_unlock();
}
/** Acquire shared lock_sys.latch */
void lock_sys_t::rd_lock(const char *file, unsigned line)
{
latch.rd_lock(file, line);
ut_ad(!writer.load(std::memory_order_relaxed));
ut_d(readers.fetch_add(1, std::memory_order_relaxed));
}
/** Release shared lock_sys.latch */
void lock_sys_t::rd_unlock()
{
ut_ad(!writer.load(std::memory_order_relaxed));
ut_ad(readers.fetch_sub(1, std::memory_order_relaxed));
latch.rd_unlock();
}
#endif
/** Calculates the fold value of a lock: used in migrating the hash table. /** Calculates the fold value of a lock: used in migrating the hash table.
@param[in] lock record lock object @param[in] lock record lock object
@@ -385,7 +407,7 @@ void lock_sys_t::resize(ulint n_cells)
{ {
ut_ad(this == &lock_sys); ut_ad(this == &lock_sys);
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
hash_table_t old_hash(rec_hash); hash_table_t old_hash(rec_hash);
rec_hash.create(n_cells); rec_hash.create(n_cells);
@@ -425,7 +447,7 @@ void lock_sys_t::close()
prdt_hash.free(); prdt_hash.free();
prdt_page_hash.free(); prdt_page_hash.free();
mysql_mutex_destroy(&mutex); latch.destroy();
mysql_mutex_destroy(&wait_mutex); mysql_mutex_destroy(&wait_mutex);
m_initialised= false; m_initialised= false;
@@ -512,10 +534,10 @@ wsrep_is_BF_lock_timeout(
ib::info() << "WSREP: BF lock wait long for trx:" << ib::hex(trx->id) ib::info() << "WSREP: BF lock wait long for trx:" << ib::hex(trx->id)
<< " query: " << wsrep_thd_query(trx->mysql_thd); << " query: " << wsrep_thd_query(trx->mysql_thd);
if (!locked) { if (!locked) {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
trx_print_latched(stderr, trx, 3000); trx_print_latched(stderr, trx, 3000);
} else { } else {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
trx_print_latched(stderr, trx, 3000); trx_print_latched(stderr, trx, 3000);
} }
@@ -759,7 +781,7 @@ lock_rec_get_prev(
ut_ad(!in_lock->is_table()); ut_ad(!in_lock->is_table());
const page_id_t id{in_lock->un_member.rec_lock.page_id}; const page_id_t id{in_lock->un_member.rec_lock.page_id};
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
for (lock = lock_sys.get_first(*lock_hash_get(in_lock->type_mode), id); for (lock = lock_sys.get_first(*lock_hash_get(in_lock->type_mode), id);
lock != in_lock; lock != in_lock;
@@ -855,7 +877,7 @@ lock_rec_other_has_expl_req(
ATTRIBUTE_COLD ATTRIBUTE_COLD
static void wsrep_kill_victim(const trx_t *trx, const lock_t *lock) static void wsrep_kill_victim(const trx_t *trx, const lock_t *lock)
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
ut_ad(lock->trx->mutex_is_owner()); ut_ad(lock->trx->mutex_is_owner());
ut_ad(trx->is_wsrep()); ut_ad(trx->is_wsrep());
@@ -926,7 +948,7 @@ lock_rec_other_has_conflicting(
ulint heap_no,/*!< in: heap number of the record */ ulint heap_no,/*!< in: heap number of the record */
const trx_t* trx) /*!< in: our transaction */ const trx_t* trx) /*!< in: our transaction */
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
bool is_supremum = (heap_no == PAGE_HEAP_NO_SUPREMUM); bool is_supremum = (heap_no == PAGE_HEAP_NO_SUPREMUM);
@@ -973,7 +995,7 @@ lock_sec_rec_some_has_impl(
trx_id_t max_trx_id; trx_id_t max_trx_id;
const page_t* page = page_align(rec); const page_t* page = page_align(rec);
lock_sys.mutex_assert_unlocked(); lock_sys.assert_unlocked();
ut_ad(!dict_index_is_clust(index)); ut_ad(!dict_index_is_clust(index));
ut_ad(page_rec_is_user_rec(rec)); ut_ad(page_rec_is_user_rec(rec));
ut_ad(rec_offs_validate(rec, index, offsets)); ut_ad(rec_offs_validate(rec, index, offsets));
@@ -1006,7 +1028,7 @@ lock_sec_rec_some_has_impl(
/*********************************************************************//** /*********************************************************************//**
Return the number of table locks for a transaction. Return the number of table locks for a transaction.
The caller must be holding lock_sys.mutex. */ The caller must be holding lock_sys.latch. */
ulint ulint
lock_number_of_tables_locked( lock_number_of_tables_locked(
/*=========================*/ /*=========================*/
@@ -1015,7 +1037,7 @@ lock_number_of_tables_locked(
const lock_t* lock; const lock_t* lock;
ulint n_tables = 0; ulint n_tables = 0;
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
for (lock = UT_LIST_GET_FIRST(trx_lock->trx_locks); for (lock = UT_LIST_GET_FIRST(trx_lock->trx_locks);
lock != NULL; lock != NULL;
@@ -1062,14 +1084,30 @@ wsrep_print_wait_locks(
} }
#endif /* WITH_WSREP */ #endif /* WITH_WSREP */
#ifdef UNIV_DEBUG
/** Assert that a lock shard is exclusively latched by this thread */
void lock_sys_t::assert_locked(const lock_t &) const
{
assert_locked();
}
/** Assert that a table lock shard is exclusively latched by this thread */
void lock_sys_t::assert_locked(const dict_table_t &) const
{
assert_locked();
}
#endif
/** Reset the wait status of a lock. /** Reset the wait status of a lock.
@param[in,out] lock lock that was possibly being waited for */ @param[in,out] lock lock that was possibly being waited for */
static void lock_reset_lock_and_trx_wait(lock_t *lock) static void lock_reset_lock_and_trx_wait(lock_t *lock)
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked(*lock);
mysql_mutex_assert_owner(&lock_sys.wait_mutex);
trx_t *trx= lock->trx;
ut_ad(lock->is_waiting()); ut_ad(lock->is_waiting());
ut_ad(!lock->trx->lock.wait_lock || lock->trx->lock.wait_lock == lock); ut_ad(!trx->lock.wait_lock || trx->lock.wait_lock == lock);
lock->trx->lock.wait_lock= nullptr; trx->lock.wait_lock= nullptr;
lock->type_mode&= ~LOCK_WAIT; lock->type_mode&= ~LOCK_WAIT;
} }
@@ -1083,7 +1121,7 @@ static void lock_set_lock_and_trx_wait(lock_t *lock, trx_t *trx)
ut_ad(lock->trx == trx); ut_ad(lock->trx == trx);
ut_ad(!trx->lock.wait_lock || trx->lock.wait_lock != lock); ut_ad(!trx->lock.wait_lock || trx->lock.wait_lock != lock);
ut_ad(!trx->lock.wait_lock || (*trx->lock.wait_lock).trx == trx); ut_ad(!trx->lock.wait_lock || (*trx->lock.wait_lock).trx == trx);
lock_sys.mutex_assert_locked(); lock_sys.assert_locked(*lock);
ut_ad(trx->mutex_is_owner()); ut_ad(trx->mutex_is_owner());
trx->lock.wait_lock= lock; trx->lock.wait_lock= lock;
@@ -1118,7 +1156,7 @@ lock_rec_create_low(
lock_t* lock; lock_t* lock;
ulint n_bytes; ulint n_bytes;
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
ut_ad(holds_trx_mutex == trx->mutex_is_owner()); ut_ad(holds_trx_mutex == trx->mutex_is_owner());
ut_ad(dict_index_is_clust(index) || !dict_index_is_online_ddl(index)); ut_ad(dict_index_is_clust(index) || !dict_index_is_online_ddl(index));
ut_ad(!(type_mode & LOCK_TABLE)); ut_ad(!(type_mode & LOCK_TABLE));
@@ -1307,7 +1345,7 @@ lock_rec_enqueue_waiting(
que_thr_t* thr, que_thr_t* thr,
lock_prdt_t* prdt) lock_prdt_t* prdt)
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
ut_ad(!srv_read_only_mode); ut_ad(!srv_read_only_mode);
ut_ad(dict_index_is_clust(index) || !dict_index_is_online_ddl(index)); ut_ad(dict_index_is_clust(index) || !dict_index_is_online_ddl(index));
@@ -1371,7 +1409,7 @@ lock_rec_find_similar_on_page(
lock_t* lock, /*!< in: lock_sys.get_first() */ lock_t* lock, /*!< in: lock_sys.get_first() */
const trx_t* trx) /*!< in: transaction */ const trx_t* trx) /*!< in: transaction */
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
for (/* No op */; for (/* No op */;
lock != NULL; lock != NULL;
@@ -1412,7 +1450,7 @@ lock_rec_add_to_queue(
/*!< in: TRUE if caller owns the /*!< in: TRUE if caller owns the
transaction mutex */ transaction mutex */
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
ut_ad(caller_owns_trx_mutex == trx->mutex_is_owner()); ut_ad(caller_owns_trx_mutex == trx->mutex_is_owner());
ut_ad(index->is_primary() ut_ad(index->is_primary()
|| dict_index_get_online_status(index) != ONLINE_INDEX_CREATION); || dict_index_get_online_status(index) != ONLINE_INDEX_CREATION);
@@ -1541,7 +1579,7 @@ lock_rec_lock(
MONITOR_ATOMIC_INC(MONITOR_NUM_RECLOCK_REQ); MONITOR_ATOMIC_INC(MONITOR_NUM_RECLOCK_REQ);
const page_id_t id{block->page.id()}; const page_id_t id{block->page.id()};
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
if (lock_t *lock= lock_sys.get_first(id)) if (lock_t *lock= lock_sys.get_first(id))
{ {
@@ -1629,7 +1667,7 @@ lock_rec_has_to_wait_in_queue(
ut_ad(wait_lock->is_waiting()); ut_ad(wait_lock->is_waiting());
ut_ad(!wait_lock->is_table()); ut_ad(!wait_lock->is_table());
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
heap_no = lock_rec_find_set_bit(wait_lock); heap_no = lock_rec_find_set_bit(wait_lock);
@@ -1810,7 +1848,7 @@ end_wait:
if (trx->lock.wait_lock) if (trx->lock.wait_lock)
{ {
{ {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
mysql_mutex_lock(&lock_sys.wait_mutex); mysql_mutex_lock(&lock_sys.wait_mutex);
if (lock_t *lock= trx->lock.wait_lock) if (lock_t *lock= trx->lock.wait_lock)
{ {
@@ -1845,7 +1883,7 @@ static void lock_wait_end(trx_t *trx)
/** Grant a waiting lock request and release the waiting transaction. */ /** Grant a waiting lock request and release the waiting transaction. */
static void lock_grant(lock_t *lock) static void lock_grant(lock_t *lock)
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked(*lock);
mysql_mutex_assert_owner(&lock_sys.wait_mutex); mysql_mutex_assert_owner(&lock_sys.wait_mutex);
lock_reset_lock_and_trx_wait(lock); lock_reset_lock_and_trx_wait(lock);
trx_t *trx= lock->trx; trx_t *trx= lock->trx;
@@ -1909,7 +1947,7 @@ static void lock_rec_dequeue_from_page(lock_t *in_lock, bool owns_wait_mutex)
/* We may or may not be holding in_lock->trx->mutex here. */ /* We may or may not be holding in_lock->trx->mutex here. */
const page_id_t page_id{in_lock->un_member.rec_lock.page_id}; const page_id_t page_id{in_lock->un_member.rec_lock.page_id};
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
in_lock->index->table->n_rec_locks--; in_lock->index->table->n_rec_locks--;
@@ -1968,7 +2006,7 @@ lock_rec_discard(
trx_lock_t* trx_lock; trx_lock_t* trx_lock;
ut_ad(!in_lock->is_table()); ut_ad(!in_lock->is_table());
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
trx_lock = &in_lock->trx->lock; trx_lock = &in_lock->trx->lock;
@@ -2073,7 +2111,7 @@ lock_rec_inherit_to_gap(
ulint heap_no) /*!< in: heap_no of the ulint heap_no) /*!< in: heap_no of the
donating record */ donating record */
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
/* At READ UNCOMMITTED or READ COMMITTED isolation level, /* At READ UNCOMMITTED or READ COMMITTED isolation level,
we do not want locks set we do not want locks set
@@ -2114,7 +2152,7 @@ lock_rec_inherit_to_gap_if_gap_lock(
on this record */ on this record */
{ {
const page_id_t id{block->page.id()}; const page_id_t id{block->page.id()};
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
for (lock_t *lock= lock_rec_get_first(&lock_sys.rec_hash, id, heap_no); for (lock_t *lock= lock_rec_get_first(&lock_sys.rec_hash, id, heap_no);
lock; lock= lock_rec_get_next(heap_no, lock)) lock; lock= lock_rec_get_next(heap_no, lock))
@@ -2146,8 +2184,6 @@ lock_rec_move_low(
{ {
const page_id_t receiver_id{receiver.page.id()}; const page_id_t receiver_id{receiver.page.id()};
lock_sys.mutex_assert_locked();
/* If the lock is predicate lock, it resides on INFIMUM record */ /* If the lock is predicate lock, it resides on INFIMUM record */
ut_ad(!lock_rec_get_first(lock_hash, receiver_id, receiver_heap_no) ut_ad(!lock_rec_get_first(lock_hash, receiver_id, receiver_heap_no)
|| lock_hash == &lock_sys.prdt_hash || lock_hash == &lock_sys.prdt_hash
@@ -2253,7 +2289,7 @@ lock_move_reorganize_page(
UT_LIST_INIT(old_locks, &lock_t::trx_locks); UT_LIST_INIT(old_locks, &lock_t::trx_locks);
const page_id_t id{block->page.id()}; const page_id_t id{block->page.id()};
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
/* FIXME: This needs to deal with predicate lock too */ /* FIXME: This needs to deal with predicate lock too */
lock_t *lock= lock_sys.get_first(id); lock_t *lock= lock_sys.get_first(id);
@@ -2384,7 +2420,7 @@ lock_move_rec_list_end(
const page_id_t id{block->page.id()}; const page_id_t id{block->page.id()};
const page_id_t new_id{new_block->page.id()}; const page_id_t new_id{new_block->page.id()};
{ {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
/* Note: when we move locks from record to record, waiting locks /* Note: when we move locks from record to record, waiting locks
and possible granted gap type locks behind them are enqueued in and possible granted gap type locks behind them are enqueued in
@@ -2503,7 +2539,7 @@ lock_move_rec_list_start(
const page_id_t new_id{new_block->page.id()}; const page_id_t new_id{new_block->page.id()};
{ {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
for (lock_t *lock= lock_sys.get_first(id); lock; for (lock_t *lock= lock_sys.get_first(id); lock;
lock= lock_rec_get_next_on_page(lock)) lock= lock_rec_get_next_on_page(lock))
@@ -2607,7 +2643,7 @@ lock_rtr_move_rec_list(
const page_id_t new_id{new_block->page.id()}; const page_id_t new_id{new_block->page.id()};
{ {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
for (lock_t *lock= lock_sys.get_first(id); lock; for (lock_t *lock= lock_sys.get_first(id); lock;
lock= lock_rec_get_next_on_page(lock)) lock= lock_rec_get_next_on_page(lock))
@@ -2676,7 +2712,7 @@ lock_update_split_right(
const page_id_t l{left_block->page.id()}; const page_id_t l{left_block->page.id()};
const page_id_t r{right_block->page.id()}; const page_id_t r{right_block->page.id()};
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
/* Move the locks on the supremum of the left page to the supremum /* Move the locks on the supremum of the left page to the supremum
of the right page */ of the right page */
@@ -2707,7 +2743,7 @@ lock_update_merge_right(
const page_id_t l{left_block->page.id()}; const page_id_t l{left_block->page.id()};
const page_id_t r{right_block->page.id()}; const page_id_t r{right_block->page.id()};
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
/* Inherit the locks from the supremum of the left page to the /* Inherit the locks from the supremum of the left page to the
original successor of infimum on the right page, to which the left original successor of infimum on the right page, to which the left
@@ -2738,7 +2774,7 @@ of the root page will act as a dummy carrier of the locks of the record
to be updated. */ to be updated. */
void lock_update_root_raise(const buf_block_t &block, const page_id_t root) void lock_update_root_raise(const buf_block_t &block, const page_id_t root)
{ {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
/* Move the locks on the supremum of the root to the supremum of block */ /* Move the locks on the supremum of the root to the supremum of block */
lock_rec_move(block, root, PAGE_HEAP_NO_SUPREMUM, PAGE_HEAP_NO_SUPREMUM); lock_rec_move(block, root, PAGE_HEAP_NO_SUPREMUM, PAGE_HEAP_NO_SUPREMUM);
} }
@@ -2748,7 +2784,7 @@ void lock_update_root_raise(const buf_block_t &block, const page_id_t root)
@param old old page (not index root page) */ @param old old page (not index root page) */
void lock_update_copy_and_discard(const buf_block_t &new_block, page_id_t old) void lock_update_copy_and_discard(const buf_block_t &new_block, page_id_t old)
{ {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
/* Move the locks on the supremum of the old page to the supremum of new */ /* Move the locks on the supremum of the old page to the supremum of new */
lock_rec_move(new_block, old, PAGE_HEAP_NO_SUPREMUM, PAGE_HEAP_NO_SUPREMUM); lock_rec_move(new_block, old, PAGE_HEAP_NO_SUPREMUM, PAGE_HEAP_NO_SUPREMUM);
lock_rec_free_all_from_discard_page(old); lock_rec_free_all_from_discard_page(old);
@@ -2765,7 +2801,7 @@ lock_update_split_left(
ulint h= lock_get_min_heap_no(right_block); ulint h= lock_get_min_heap_no(right_block);
const page_id_t l{left_block->page.id()}; const page_id_t l{left_block->page.id()};
const page_id_t r{right_block->page.id()}; const page_id_t r{right_block->page.id()};
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
/* Inherit the locks to the supremum of the left page from the /* Inherit the locks to the supremum of the left page from the
successor of the infimum on the right page */ successor of the infimum on the right page */
lock_rec_inherit_to_gap(l, r, left_block->frame, PAGE_HEAP_NO_SUPREMUM, h); lock_rec_inherit_to_gap(l, r, left_block->frame, PAGE_HEAP_NO_SUPREMUM, h);
@@ -2782,7 +2818,7 @@ void lock_update_merge_left(const buf_block_t& left, const rec_t *orig_pred,
const page_id_t l{left.page.id()}; const page_id_t l{left.page.id()};
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
const rec_t *left_next_rec= page_rec_get_next_const(orig_pred); const rec_t *left_next_rec= page_rec_get_next_const(orig_pred);
if (!page_rec_is_supremum(left_next_rec)) if (!page_rec_is_supremum(left_next_rec))
@@ -2827,7 +2863,7 @@ lock_rec_reset_and_inherit_gap_locks(
donating record */ donating record */
{ {
const page_id_t heir{heir_block.page.id()}; const page_id_t heir{heir_block.page.id()};
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
lock_rec_reset_and_release_wait(heir, heir_heap_no); lock_rec_reset_and_release_wait(heir, heir_heap_no);
lock_rec_inherit_to_gap(heir, donor, heir_block.frame, heir_heap_no, heap_no); lock_rec_inherit_to_gap(heir, donor, heir_block.frame, heir_heap_no, heap_no);
} }
@@ -2849,7 +2885,7 @@ lock_update_discard(
ulint heap_no; ulint heap_no;
const page_id_t heir(heir_block->page.id()); const page_id_t heir(heir_block->page.id());
const page_id_t page_id(block->page.id()); const page_id_t page_id(block->page.id());
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
if (lock_sys.get_first(page_id)) { if (lock_sys.get_first(page_id)) {
ut_ad(!lock_sys.get_first_prdt(page_id)); ut_ad(!lock_sys.get_first_prdt(page_id));
@@ -2958,7 +2994,7 @@ lock_update_delete(
} }
const page_id_t id{block->page.id()}; const page_id_t id{block->page.id()};
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
/* Let the next record inherit the locks from rec, in gap mode */ /* Let the next record inherit the locks from rec, in gap mode */
@@ -2991,7 +3027,7 @@ lock_rec_store_on_page_infimum(
ut_ad(block->frame == page_align(rec)); ut_ad(block->frame == page_align(rec));
const page_id_t id{block->page.id()}; const page_id_t id{block->page.id()};
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
lock_rec_move(*block, id, PAGE_HEAP_NO_INFIMUM, heap_no); lock_rec_move(*block, id, PAGE_HEAP_NO_INFIMUM, heap_no);
} }
@@ -3005,7 +3041,7 @@ void lock_rec_restore_from_page_infimum(const buf_block_t &block,
const rec_t *rec, page_id_t donator) const rec_t *rec, page_id_t donator)
{ {
const ulint heap_no= page_rec_get_heap_no(rec); const ulint heap_no= page_rec_get_heap_no(rec);
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
lock_rec_move(block, donator, heap_no, PAGE_HEAP_NO_INFIMUM); lock_rec_move(block, donator, heap_no, PAGE_HEAP_NO_INFIMUM);
} }
@@ -3040,7 +3076,7 @@ lock_table_create(
lock_t* lock; lock_t* lock;
ut_ad(table && trx); ut_ad(table && trx);
lock_sys.mutex_assert_locked(); lock_sys.assert_locked(*table);
ut_ad(trx->mutex_is_owner()); ut_ad(trx->mutex_is_owner());
check_trx_state(trx); check_trx_state(trx);
@@ -3146,7 +3182,6 @@ lock_table_pop_autoinc_locks(
/*=========================*/ /*=========================*/
trx_t* trx) /*!< in/out: transaction that owns the AUTOINC locks */ trx_t* trx) /*!< in/out: transaction that owns the AUTOINC locks */
{ {
lock_sys.mutex_assert_locked();
ut_ad(!ib_vector_is_empty(trx->autoinc_locks)); ut_ad(!ib_vector_is_empty(trx->autoinc_locks));
/* Skip any gaps, gaps are NULL lock entries in the /* Skip any gaps, gaps are NULL lock entries in the
@@ -3175,7 +3210,7 @@ lock_table_remove_autoinc_lock(
lint i = ib_vector_size(trx->autoinc_locks) - 1; lint i = ib_vector_size(trx->autoinc_locks) - 1;
ut_ad(lock->type_mode == (LOCK_AUTO_INC | LOCK_TABLE)); ut_ad(lock->type_mode == (LOCK_AUTO_INC | LOCK_TABLE));
lock_sys.mutex_assert_locked(); lock_sys.assert_locked(*lock->un_member.tab_lock.table);
ut_ad(!ib_vector_is_empty(trx->autoinc_locks)); ut_ad(!ib_vector_is_empty(trx->autoinc_locks));
/* With stored functions and procedures the user may drop /* With stored functions and procedures the user may drop
@@ -3227,10 +3262,10 @@ lock_table_remove_low(
trx_t* trx; trx_t* trx;
dict_table_t* table; dict_table_t* table;
lock_sys.mutex_assert_locked(); ut_ad(lock->is_table());
trx = lock->trx; trx = lock->trx;
table = lock->un_member.tab_lock.table; table = lock->un_member.tab_lock.table;
lock_sys.assert_locked(*table);
/* Remove the table from the transaction's AUTOINC vector, if /* Remove the table from the transaction's AUTOINC vector, if
the lock that is being released is an AUTOINC lock. */ the lock that is being released is an AUTOINC lock. */
@@ -3288,7 +3323,7 @@ lock_table_enqueue_waiting(
#endif #endif
) )
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked(*table);
ut_ad(!srv_read_only_mode); ut_ad(!srv_read_only_mode);
trx_t* trx = thr_get_trx(thr); trx_t* trx = thr_get_trx(thr);
@@ -3342,7 +3377,7 @@ lock_table_other_has_incompatible(
const dict_table_t* table, /*!< in: table */ const dict_table_t* table, /*!< in: table */
lock_mode mode) /*!< in: lock mode */ lock_mode mode) /*!< in: lock mode */
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked(*table);
static_assert(LOCK_IS == 0, "compatibility"); static_assert(LOCK_IS == 0, "compatibility");
static_assert(LOCK_IX == 1, "compatibility"); static_assert(LOCK_IX == 1, "compatibility");
@@ -3427,7 +3462,7 @@ lock_table(
err = DB_SUCCESS; err = DB_SUCCESS;
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
/* 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
other transactions have in the table lock queue. */ other transactions have in the table lock queue. */
@@ -3465,7 +3500,7 @@ void lock_table_resurrect(dict_table_t *table, trx_t *trx, lock_mode mode)
return; return;
{ {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
ut_ad(!lock_table_other_has_incompatible(trx, LOCK_WAIT, table, mode)); ut_ad(!lock_table_other_has_incompatible(trx, LOCK_WAIT, table, mode));
trx->mutex_lock(); trx->mutex_lock();
@@ -3474,40 +3509,27 @@ void lock_table_resurrect(dict_table_t *table, trx_t *trx, lock_mode mode)
trx->mutex_unlock(); trx->mutex_unlock();
} }
/*********************************************************************//** /** Find a lock that a waiting table lock request still has to wait for. */
Checks if a waiting table lock request still has to wait in a queue. static const lock_t *lock_table_has_to_wait_in_queue(const lock_t *wait_lock)
@return TRUE if still has to wait */
static
bool
lock_table_has_to_wait_in_queue(
/*============================*/
const lock_t* wait_lock) /*!< in: waiting table lock */
{ {
ut_ad(wait_lock->is_waiting()); ut_ad(wait_lock->is_waiting());
ut_ad(wait_lock->is_table()); ut_ad(wait_lock->is_table());
dict_table_t *table = wait_lock->un_member.tab_lock.table; dict_table_t *table= wait_lock->un_member.tab_lock.table;
lock_sys.mutex_assert_locked(); lock_sys.assert_locked(*table);
static_assert(LOCK_IS == 0, "compatibility"); static_assert(LOCK_IS == 0, "compatibility");
static_assert(LOCK_IX == 1, "compatibility"); static_assert(LOCK_IX == 1, "compatibility");
if (UNIV_LIKELY(wait_lock->mode() <= LOCK_IX if (UNIV_LIKELY(wait_lock->mode() <= LOCK_IX && !table->n_lock_x_or_s))
&& !table->n_lock_x_or_s)) { return nullptr;
return(false);
}
for (const lock_t *lock = UT_LIST_GET_FIRST(table->locks); for (const lock_t *lock= UT_LIST_GET_FIRST(table->locks); lock != wait_lock;
lock != wait_lock; lock= UT_LIST_GET_NEXT(un_member.tab_lock.locks, lock))
lock = UT_LIST_GET_NEXT(un_member.tab_lock.locks, lock)) { if (lock_has_to_wait(wait_lock, lock))
return lock;
if (lock_has_to_wait(wait_lock, lock)) { return nullptr;
return(true);
}
}
return(false);
} }
/*************************************************************//** /*************************************************************//**
@@ -3521,7 +3543,6 @@ static void lock_table_dequeue(lock_t *in_lock, bool owns_wait_mutex)
#ifdef SAFE_MUTEX #ifdef SAFE_MUTEX
ut_ad(owns_wait_mutex == mysql_mutex_is_owner(&lock_sys.wait_mutex)); ut_ad(owns_wait_mutex == mysql_mutex_is_owner(&lock_sys.wait_mutex));
#endif #endif
lock_sys.mutex_assert_locked();
lock_t* lock = UT_LIST_GET_NEXT(un_member.tab_lock.locks, in_lock); lock_t* lock = UT_LIST_GET_NEXT(un_member.tab_lock.locks, in_lock);
const dict_table_t* table = lock_table_remove_low(in_lock); const dict_table_t* table = lock_table_remove_low(in_lock);
@@ -3579,7 +3600,7 @@ void lock_table_x_unlock(dict_table_t *table, trx_t *trx)
if (lock->type_mode != (LOCK_TABLE | LOCK_X)) if (lock->type_mode != (LOCK_TABLE | LOCK_X))
continue; continue;
{ {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
lock_table_dequeue(lock, false); lock_table_dequeue(lock, false);
} }
lock= nullptr; lock= nullptr;
@@ -3663,7 +3684,7 @@ lock_rec_unlock(
heap_no = page_rec_get_heap_no(rec); heap_no = page_rec_get_heap_no(rec);
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
first_lock = lock_rec_get_first(&lock_sys.rec_hash, id, heap_no); first_lock = lock_rec_get_first(&lock_sys.rec_hash, id, heap_no);
@@ -3766,7 +3787,7 @@ void lock_release(trx_t* trx)
trx_id_t max_trx_id = trx_sys.get_max_trx_id(); trx_id_t max_trx_id = trx_sys.get_max_trx_id();
ut_ad(!trx->mutex_is_owner()); ut_ad(!trx->mutex_is_owner());
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
for (lock_t* lock = UT_LIST_GET_LAST(trx->lock.trx_locks); for (lock_t* lock = UT_LIST_GET_LAST(trx->lock.trx_locks);
lock != NULL; lock != NULL;
@@ -3792,9 +3813,9 @@ void lock_release(trx_t* trx)
/* Release the mutex for a while, so that we /* Release the mutex for a while, so that we
do not monopolize it */ do not monopolize it */
lock_sys.mutex_unlock(); lock_sys.wr_unlock();
count = 0; count = 0;
lock_sys.mutex_lock(); lock_sys.wr_lock(SRW_LOCK_CALL);
} }
++count; ++count;
@@ -3814,9 +3835,9 @@ lock_trx_table_locks_remove(
trx_t* trx = lock_to_remove->trx; trx_t* trx = lock_to_remove->trx;
ut_ad(lock_to_remove->is_table()); ut_ad(lock_to_remove->is_table());
lock_sys.mutex_assert_locked(); lock_sys.assert_locked(*lock_to_remove->un_member.tab_lock.table);
/* It is safe to read this because we are holding the lock mutex */ /* It is safe to read this because we are holding lock_sys.latch */
const bool have_mutex = trx->lock.cancel; const bool have_mutex = trx->lock.cancel;
if (!have_mutex) { if (!have_mutex) {
trx->mutex_lock(); trx->mutex_lock();
@@ -3855,7 +3876,7 @@ static
void void
lock_table_print(FILE* file, const lock_t* lock) lock_table_print(FILE* file, const lock_t* lock)
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
ut_a(lock->is_table()); ut_a(lock->is_table());
fputs("TABLE LOCK table ", file); fputs("TABLE LOCK table ", file);
@@ -3901,7 +3922,7 @@ static void lock_rec_print(FILE* file, const lock_t* lock, mtr_t& mtr)
ut_ad(!lock->is_table()); ut_ad(!lock->is_table());
const page_id_t page_id{lock->un_member.rec_lock.page_id}; const page_id_t page_id{lock->un_member.rec_lock.page_id};
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
fprintf(file, "RECORD LOCKS space id %u page no %u n bits " ULINTPF fprintf(file, "RECORD LOCKS space id %u page no %u n bits " ULINTPF
" index %s of table ", " index %s of table ",
@@ -3998,7 +4019,7 @@ static ulint lock_get_n_rec_locks()
ulint n_locks = 0; ulint n_locks = 0;
ulint i; ulint i;
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
for (i = 0; i < lock_sys.rec_hash.n_cells; i++) { for (i = 0; i < lock_sys.rec_hash.n_cells; i++) {
const lock_t* lock; const lock_t* lock;
@@ -4019,20 +4040,16 @@ static ulint lock_get_n_rec_locks()
/*********************************************************************//** /*********************************************************************//**
Prints info of locks for all transactions. Prints info of locks for all transactions.
@return FALSE if not able to obtain lock mutex @return FALSE if not able to acquire lock_sys.latch (and dislay info) */
and exits without printing info */
ibool ibool
lock_print_info_summary( lock_print_info_summary(
/*====================*/ /*====================*/
FILE* file, /*!< in: file where to print */ FILE* file, /*!< in: file where to print */
ibool nowait) /*!< in: whether to wait for the lock mutex */ ibool nowait) /*!< in: whether to wait for lock_sys.latch */
{ {
/* if nowait is FALSE, wait on the lock mutex,
otherwise return immediately if fail to obtain the
mutex. */
if (!nowait) { if (!nowait) {
lock_sys.mutex_lock(); lock_sys.wr_lock(SRW_LOCK_CALL);
} else if (lock_sys.mutex_trylock()) { } else if (!lock_sys.wr_lock_try()) {
fputs("FAIL TO OBTAIN LOCK MUTEX," fputs("FAIL TO OBTAIN LOCK MUTEX,"
" SKIP LOCK INFO PRINTING\n", file); " SKIP LOCK INFO PRINTING\n", file);
return(FALSE); return(FALSE);
@@ -4160,9 +4177,8 @@ struct lock_print_info
}; };
/*********************************************************************//** /*********************************************************************//**
Prints info of locks for each transaction. This function assumes that the Prints info of locks for each transaction. This function will release
caller holds the lock mutex and more importantly it will release the lock lock_sys.latch, which the caller must be holding in exclusive mode. */
mutex on behalf of the caller. (This should be fixed in the future). */
void void
lock_print_info_all_transactions( lock_print_info_all_transactions(
/*=============================*/ /*=============================*/
@@ -4171,7 +4187,7 @@ lock_print_info_all_transactions(
fprintf(file, "LIST OF TRANSACTIONS FOR EACH SESSION:\n"); fprintf(file, "LIST OF TRANSACTIONS FOR EACH SESSION:\n");
trx_sys.trx_list.for_each(lock_print_info(file, my_hrtime_coarse())); trx_sys.trx_list.for_each(lock_print_info(file, my_hrtime_coarse()));
lock_sys.mutex_unlock(); lock_sys.wr_unlock();
ut_d(lock_validate()); ut_d(lock_validate());
} }
@@ -4226,14 +4242,14 @@ lock_table_queue_validate(
{ {
const lock_t* lock; const lock_t* lock;
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
for (lock = UT_LIST_GET_FIRST(table->locks); for (lock = UT_LIST_GET_FIRST(table->locks);
lock != NULL; lock != NULL;
lock = UT_LIST_GET_NEXT(un_member.tab_lock.locks, lock)) { lock = UT_LIST_GET_NEXT(un_member.tab_lock.locks, lock)) {
/* 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 lock_sys.mutex. It may change while we are holding the lock_sys.latch. It may change
from ACTIVE or PREPARED to PREPARED or COMMITTED. */ from ACTIVE or PREPARED to PREPARED or COMMITTED. */
lock->trx->mutex_lock(); lock->trx->mutex_lock();
check_trx_state(lock->trx); check_trx_state(lock->trx);
@@ -4263,7 +4279,7 @@ lock_rec_queue_validate(
/*====================*/ /*====================*/
bool locked_lock_trx_sys, bool locked_lock_trx_sys,
/*!< in: if the caller holds /*!< in: if the caller holds
both the lock mutex and both the lock_sys.latch and
trx_sys_t->lock. */ trx_sys_t->lock. */
const page_id_t id, /*!< in: page identifier */ const page_id_t id, /*!< in: page identifier */
const rec_t* rec, /*!< in: record to look at */ const rec_t* rec, /*!< in: record to look at */
@@ -4283,10 +4299,10 @@ lock_rec_queue_validate(
heap_no = page_rec_get_heap_no(rec); heap_no = page_rec_get_heap_no(rec);
if (!locked_lock_trx_sys) { if (!locked_lock_trx_sys) {
lock_sys.mutex_lock(); lock_sys.wr_lock(SRW_LOCK_CALL);
} }
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
if (!page_rec_is_user_rec(rec)) { if (!page_rec_is_user_rec(rec)) {
@@ -4308,14 +4324,14 @@ lock_rec_queue_validate(
func_exit: func_exit:
if (!locked_lock_trx_sys) { if (!locked_lock_trx_sys) {
lock_sys.mutex_unlock(); lock_sys.wr_unlock();
} }
return true; return true;
} }
ut_ad(page_rec_is_leaf(rec)); ut_ad(page_rec_is_leaf(rec));
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
const trx_id_t impl_trx_id = index && index->is_primary() const trx_id_t impl_trx_id = index && index->is_primary()
? lock_clust_rec_some_has_impl(rec, index, offsets) ? lock_clust_rec_some_has_impl(rec, index, offsets)
@@ -4439,7 +4455,7 @@ static bool lock_rec_validate_page(const buf_block_t *block, bool latched)
rec_offs_init(offsets_); rec_offs_init(offsets_);
const page_id_t id{block->page.id()}; const page_id_t id{block->page.id()};
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
loop: loop:
lock = lock_sys.get_first(id); lock = lock_sys.get_first(id);
@@ -4514,7 +4530,7 @@ lock_rec_validate(
page_id_t* limit) /*!< in/out: upper limit of page_id_t* limit) /*!< in/out: upper limit of
(space, page_no) */ (space, page_no) */
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
for (const lock_t* lock = static_cast<const lock_t*>( for (const lock_t* lock = static_cast<const lock_t*>(
HASH_GET_FIRST(&lock_sys.rec_hash, start)); HASH_GET_FIRST(&lock_sys.rec_hash, start));
@@ -4581,7 +4597,7 @@ static void lock_rec_block_validate(const page_id_t page_id)
static my_bool lock_validate_table_locks(rw_trx_hash_element_t *element, void*) static my_bool lock_validate_table_locks(rw_trx_hash_element_t *element, void*)
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
mysql_mutex_lock(&element->mutex); mysql_mutex_lock(&element->mutex);
if (element->trx) if (element->trx)
{ {
@@ -4602,7 +4618,7 @@ static void lock_validate()
{ {
std::set<page_id_t> pages; std::set<page_id_t> pages;
{ {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
/* Validate table locks */ /* Validate table locks */
trx_sys.rw_trx_hash.iterate(lock_validate_table_locks); trx_sys.rw_trx_hash.iterate(lock_validate_table_locks);
@@ -4659,7 +4675,7 @@ lock_rec_insert_check_and_lock(
ut_ad(!rec_is_metadata(next_rec, *index)); ut_ad(!rec_is_metadata(next_rec, *index));
{ {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
/* Because this code is invoked for a running transaction by /* Because this code is invoked for a running transaction by
the thread that is serving the transaction, it is not necessary the thread that is serving the transaction, it is not necessary
to hold trx->mutex here. */ to hold trx->mutex here. */
@@ -4759,7 +4775,7 @@ lock_rec_convert_impl_to_expl_for_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");
{ {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
trx->mutex_lock(); trx->mutex_lock();
ut_ad(!trx_state_eq(trx, TRX_STATE_NOT_STARTED)); ut_ad(!trx_state_eq(trx, TRX_STATE_NOT_STARTED));
@@ -4831,7 +4847,7 @@ static void lock_rec_other_trx_holds_expl(trx_t *caller_trx, trx_t *trx,
if (trx) if (trx)
{ {
ut_ad(!page_rec_is_metadata(rec)); ut_ad(!page_rec_is_metadata(rec));
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
ut_ad(trx->is_referenced()); ut_ad(trx->is_referenced());
const trx_state_t state{trx->state}; const trx_state_t state{trx->state};
ut_ad(state != TRX_STATE_NOT_STARTED); ut_ad(state != TRX_STATE_NOT_STARTED);
@@ -4876,7 +4892,7 @@ lock_rec_convert_impl_to_expl(
{ {
trx_t* trx; trx_t* trx;
lock_sys.mutex_assert_unlocked(); lock_sys.assert_unlocked();
ut_ad(page_rec_is_user_rec(rec)); ut_ad(page_rec_is_user_rec(rec));
ut_ad(rec_offs_validate(rec, index, offsets)); ut_ad(rec_offs_validate(rec, index, offsets));
ut_ad(!page_rec_is_comp(rec) == !rec_offs_comp(offsets)); ut_ad(!page_rec_is_comp(rec) == !rec_offs_comp(offsets));
@@ -5265,7 +5281,7 @@ lock_trx_holds_autoinc_locks(
Release all the transaction's autoinc locks. */ Release all the transaction's autoinc locks. */
static void lock_release_autoinc_locks(trx_t *trx, bool owns_wait_mutex) static void lock_release_autoinc_locks(trx_t *trx, bool owns_wait_mutex)
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
#ifdef SAFE_MUTEX #ifdef SAFE_MUTEX
ut_ad(owns_wait_mutex == mysql_mutex_is_owner(&lock_sys.wait_mutex)); ut_ad(owns_wait_mutex == mysql_mutex_is_owner(&lock_sys.wait_mutex));
#endif /* SAFE_MUTEX */ #endif /* SAFE_MUTEX */
@@ -5345,7 +5361,7 @@ lock_rec_get_index(
/** 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)
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked(*lock);
mysql_mutex_assert_owner(&lock_sys.wait_mutex); mysql_mutex_assert_owner(&lock_sys.wait_mutex);
trx_t *trx= lock->trx; trx_t *trx= lock->trx;
ut_ad(trx->state == TRX_STATE_ACTIVE); ut_ad(trx->state == TRX_STATE_ACTIVE);
@@ -5381,7 +5397,7 @@ lock_unlock_table_autoinc(
/*======================*/ /*======================*/
trx_t* trx) /*!< in/out: transaction */ trx_t* trx) /*!< in/out: transaction */
{ {
lock_sys.mutex_assert_unlocked(); lock_sys.assert_unlocked();
ut_ad(!trx->mutex_is_owner()); ut_ad(!trx->mutex_is_owner());
ut_ad(!trx->lock.wait_lock); ut_ad(!trx->lock.wait_lock);
@@ -5396,14 +5412,14 @@ lock_unlock_table_autoinc(
necessary to hold trx->mutex here. */ necessary to hold trx->mutex here. */
if (lock_trx_holds_autoinc_locks(trx)) { if (lock_trx_holds_autoinc_locks(trx)) {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
lock_release_autoinc_locks(trx, false); lock_release_autoinc_locks(trx, false);
} }
} }
static inline dberr_t lock_trx_handle_wait_low(trx_t* trx) static inline dberr_t lock_trx_handle_wait_low(trx_t* trx)
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
mysql_mutex_assert_owner(&lock_sys.wait_mutex); mysql_mutex_assert_owner(&lock_sys.wait_mutex);
ut_ad(trx->mutex_is_owner()); ut_ad(trx->mutex_is_owner());
@@ -5433,7 +5449,7 @@ lock_trx_handle_wait(
#endif /* WITH_WSREP */ #endif /* WITH_WSREP */
dberr_t err; dberr_t err;
{ {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
mysql_mutex_lock(&lock_sys.wait_mutex); mysql_mutex_lock(&lock_sys.wait_mutex);
trx->mutex_lock(); trx->mutex_lock();
err= lock_trx_handle_wait_low(trx); err= lock_trx_handle_wait_low(trx);
@@ -5454,7 +5470,7 @@ lock_trx_handle_wait(
static my_bool lock_table_locks_lookup(rw_trx_hash_element_t *element, static my_bool lock_table_locks_lookup(rw_trx_hash_element_t *element,
const dict_table_t *table) const dict_table_t *table)
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
mysql_mutex_lock(&element->mutex); mysql_mutex_lock(&element->mutex);
if (element->trx) if (element->trx)
{ {
@@ -5494,7 +5510,7 @@ lock_table_has_locks(
held on records in this table or on the held on records in this table or on the
table itself */ table itself */
{ {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
bool has_locks= UT_LIST_GET_LEN(table->locks) > 0 || table->n_rec_locks > 0; bool has_locks= UT_LIST_GET_LEN(table->locks) > 0 || table->n_rec_locks > 0;
#ifdef UNIV_DEBUG #ifdef UNIV_DEBUG
if (!has_locks) if (!has_locks)
@@ -5526,7 +5542,7 @@ lock_trx_has_sys_table_locks(
const lock_t* strongest_lock = 0; const lock_t* strongest_lock = 0;
lock_mode strongest = LOCK_NONE; lock_mode strongest = LOCK_NONE;
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
const lock_list::const_iterator end = trx->lock.table_locks.end(); const lock_list::const_iterator end = trx->lock.table_locks.end();
lock_list::const_iterator it = trx->lock.table_locks.begin(); lock_list::const_iterator it = trx->lock.table_locks.begin();
@@ -5587,7 +5603,7 @@ bool lock_trx_has_expl_x_lock(const trx_t &trx, const dict_table_t &table,
ut_ad(lock_table_has(&trx, &table, LOCK_IX)); ut_ad(lock_table_has(&trx, &table, LOCK_IX));
if (!lock_table_has(&trx, &table, LOCK_X)) if (!lock_table_has(&trx, &table, LOCK_X))
{ {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
ut_ad(lock_rec_has_expl(LOCK_X | LOCK_REC_NOT_GAP, id, heap_no, &trx)); ut_ad(lock_rec_has_expl(LOCK_X | LOCK_REC_NOT_GAP, id, heap_no, &trx));
} }
return true; return true;
@@ -5600,7 +5616,7 @@ is enabled. */
void void
DeadlockChecker::start_print() DeadlockChecker::start_print()
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
rewind(lock_latest_err_file); rewind(lock_latest_err_file);
ut_print_timestamp(lock_latest_err_file); ut_print_timestamp(lock_latest_err_file);
@@ -5629,7 +5645,7 @@ DeadlockChecker::print(const char* msg)
void void
DeadlockChecker::print(const trx_t* trx, ulint max_query_len) DeadlockChecker::print(const trx_t* trx, ulint max_query_len)
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
ulint n_rec_locks = trx->lock.n_rec_locks; ulint n_rec_locks = trx->lock.n_rec_locks;
ulint n_trx_locks = UT_LIST_GET_LEN(trx->lock.trx_locks); ulint n_trx_locks = UT_LIST_GET_LEN(trx->lock.trx_locks);
@@ -5649,7 +5665,7 @@ DeadlockChecker::print(const trx_t* trx, ulint max_query_len)
void void
DeadlockChecker::print(const lock_t* lock) DeadlockChecker::print(const lock_t* lock)
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
if (!lock->is_table()) { if (!lock->is_table()) {
mtr_t mtr; mtr_t mtr;
@@ -5678,7 +5694,7 @@ Note: "next" here means PREV for table locks.
const lock_t* const lock_t*
DeadlockChecker::get_next_lock(const lock_t* lock, ulint heap_no) const DeadlockChecker::get_next_lock(const lock_t* lock, ulint heap_no) const
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
do { do {
if (!lock->is_table()) { if (!lock->is_table()) {
@@ -5716,7 +5732,7 @@ due to the way the record lock has is implemented.
const lock_t* const lock_t*
DeadlockChecker::get_first_lock(ulint* heap_no) const DeadlockChecker::get_first_lock(ulint* heap_no) const
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
const lock_t* lock = m_wait_lock; const lock_t* lock = m_wait_lock;
@@ -5764,7 +5780,7 @@ transaction info.
void void
DeadlockChecker::notify(const lock_t* lock) const DeadlockChecker::notify(const lock_t* lock) const
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
start_print(); start_print();
@@ -5810,7 +5826,7 @@ inline bool trx_weight_ge(const trx_t *a, const trx_t *b)
/** @return the victim transaction that should be rolled back */ /** @return the victim transaction that should be rolled back */
trx_t *DeadlockChecker::select_victim() const trx_t *DeadlockChecker::select_victim() const
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
trx_t *lock_trx= m_wait_lock->trx; trx_t *lock_trx= m_wait_lock->trx;
ut_ad(m_start->lock.wait_lock); ut_ad(m_start->lock.wait_lock);
ut_ad(lock_trx != m_start); ut_ad(lock_trx != m_start);
@@ -5839,7 +5855,7 @@ have been granted its lock by the deadlock checks.
@return nullptr if no deadlock */ @return nullptr if no deadlock */
inline trx_t* DeadlockChecker::search() inline trx_t* DeadlockChecker::search()
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
ut_ad(!m_start->mutex_is_owner()); ut_ad(!m_start->mutex_is_owner());
check_trx_state(m_wait_lock->trx); check_trx_state(m_wait_lock->trx);
ut_ad(m_mark_start <= s_lock_mark_counter); ut_ad(m_mark_start <= s_lock_mark_counter);
@@ -5954,7 +5970,7 @@ inline trx_t* DeadlockChecker::search()
void void
DeadlockChecker::rollback_print(const trx_t* trx, const lock_t* lock) DeadlockChecker::rollback_print(const trx_t* trx, const lock_t* lock)
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
/* If the lock search exceeds the max step /* If the lock search exceeds the max step
or the max depth, the current trx will be or the max depth, the current trx will be
@@ -6000,7 +6016,7 @@ inline bool DeadlockChecker::check_and_resolve(trx_t *trx)
for (;;) for (;;)
{ {
LockMutexGuard g; // FIXME: only lock_sys.wait_mutex? LockMutexGuard g{SRW_LOCK_CALL}; // FIXME: only lock_sys.wait_mutex?
lock_t *lock= trx->lock.wait_lock; lock_t *lock= trx->lock.wait_lock;
if (!lock) if (!lock)
@@ -6080,7 +6096,7 @@ lock_update_split_and_merge(
const page_id_t l{left_block->page.id()}; const page_id_t l{left_block->page.id()};
const page_id_t r{right_block->page.id()}; const page_id_t r{right_block->page.id()};
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
left_next_rec = page_rec_get_next_const(orig_pred); left_next_rec = page_rec_get_next_const(orig_pred);
ut_ad(!page_rec_is_metadata(left_next_rec)); ut_ad(!page_rec_is_metadata(left_next_rec));

View File

@@ -234,7 +234,7 @@ lock_prdt_has_lock(
attached to the new lock */ attached to the new lock */
const trx_t* trx) /*!< in: transaction */ const trx_t* trx) /*!< in: transaction */
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
ut_ad((precise_mode & LOCK_MODE_MASK) == LOCK_S ut_ad((precise_mode & LOCK_MODE_MASK) == LOCK_S
|| (precise_mode & LOCK_MODE_MASK) == LOCK_X); || (precise_mode & LOCK_MODE_MASK) == LOCK_X);
ut_ad(!(precise_mode & LOCK_INSERT_INTENTION)); ut_ad(!(precise_mode & LOCK_INSERT_INTENTION));
@@ -426,7 +426,7 @@ lock_prdt_add_to_queue(
transaction mutex */ transaction mutex */
{ {
const page_id_t id{block->page.id()}; const page_id_t id{block->page.id()};
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
ut_ad(caller_owns_trx_mutex == trx->mutex_is_owner()); ut_ad(caller_owns_trx_mutex == trx->mutex_is_owner());
ut_ad(index->is_spatial()); ut_ad(index->is_spatial());
ut_ad(!dict_index_is_online_ddl(index)); ut_ad(!dict_index_is_online_ddl(index));
@@ -504,7 +504,7 @@ lock_prdt_insert_check_and_lock(
dberr_t err= DB_SUCCESS; dberr_t err= DB_SUCCESS;
{ {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
/* Because this code is invoked for a running transaction by /* Because this code is invoked for a running transaction by
the thread that is serving the transaction, it is not necessary the thread that is serving the transaction, it is not necessary
to hold trx->mutex here. */ to hold trx->mutex here. */
@@ -561,7 +561,7 @@ lock_prdt_update_parent(
lock_prdt_t* right_prdt, /*!< in: MBR on the new page */ lock_prdt_t* right_prdt, /*!< in: MBR on the new page */
const page_id_t page_id) /*!< in: parent page */ const page_id_t page_id) /*!< in: parent page */
{ {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
/* Get all locks in parent */ /* Get all locks in parent */
for (lock_t *lock = lock_sys.get_first_prdt(page_id); for (lock_t *lock = lock_sys.get_first_prdt(page_id);
@@ -660,7 +660,7 @@ lock_prdt_update_split(
lock_prdt_t* new_prdt, /*!< in: MBR on the new page */ lock_prdt_t* new_prdt, /*!< in: MBR on the new page */
const page_id_t page_id) /*!< in: page number */ const page_id_t page_id) /*!< in: page number */
{ {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
lock_prdt_update_split_low(new_block, prdt, new_prdt, lock_prdt_update_split_low(new_block, prdt, new_prdt,
page_id, LOCK_PREDICATE); page_id, LOCK_PREDICATE);
@@ -732,7 +732,7 @@ lock_prdt_lock(
index record, and this would not have been possible if another active index record, and this would not have been possible if another active
transaction had modified this secondary index record. */ transaction had modified this secondary index record. */
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
const unsigned prdt_mode = type_mode | mode; const unsigned prdt_mode = type_mode | mode;
lock_t* lock = lock_sys.get_first(hash, id); lock_t* lock = lock_sys.get_first(hash, id);
@@ -823,7 +823,7 @@ lock_place_prdt_page_lock(
index record, and this would not have been possible if another active index record, and this would not have been possible if another active
transaction had modified this secondary index record. */ transaction had modified this secondary index record. */
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
const lock_t* lock = lock_sys.get_first_prdt_page(page_id); const lock_t* lock = lock_sys.get_first_prdt_page(page_id);
const ulint mode = LOCK_S | LOCK_PRDT_PAGE; const ulint mode = LOCK_S | LOCK_PRDT_PAGE;
@@ -862,7 +862,7 @@ lock_place_prdt_page_lock(
@return true if there is none */ @return true if there is none */
bool lock_test_prdt_page_lock(const trx_t *trx, const page_id_t page_id) bool lock_test_prdt_page_lock(const trx_t *trx, const page_id_t page_id)
{ {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
lock_t *lock= lock_sys.get_first_prdt_page(page_id); lock_t *lock= lock_sys.get_first_prdt_page(page_id);
return !lock || trx == lock->trx; return !lock || trx == lock->trx;
} }
@@ -877,7 +877,7 @@ lock_prdt_rec_move(
the receiving record */ the receiving record */
const page_id_t donator) /*!< in: target page */ const page_id_t donator) /*!< in: target page */
{ {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
for (lock_t *lock = lock_rec_get_first(&lock_sys.prdt_hash, for (lock_t *lock = lock_rec_get_first(&lock_sys.prdt_hash,
donator, PRDT_HEAPNO); donator, PRDT_HEAPNO);
@@ -904,7 +904,7 @@ lock_prdt_rec_move(
void void
lock_prdt_page_free_from_discard(const page_id_t id, hash_table_t *lock_hash) lock_prdt_page_free_from_discard(const page_id_t id, hash_table_t *lock_hash)
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
for (lock_t *lock= lock_sys.get_first(*lock_hash, id), *next; lock; for (lock_t *lock= lock_sys.get_first(*lock_hash, id), *next; lock;
lock= next) lock= next)

View File

@@ -708,7 +708,7 @@ row_ins_foreign_trx_print(
ut_ad(!srv_read_only_mode); ut_ad(!srv_read_only_mode);
{ {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
n_rec_locks = trx->lock.n_rec_locks; n_rec_locks = trx->lock.n_rec_locks;
n_trx_locks = UT_LIST_GET_LEN(trx->lock.trx_locks); n_trx_locks = UT_LIST_GET_LEN(trx->lock.trx_locks);
heap_size = mem_heap_get_size(trx->lock.lock_heap); heap_size = mem_heap_get_size(trx->lock.lock_heap);
@@ -1019,8 +1019,8 @@ row_ins_foreign_check_on_constraint(
/* Since we are going to delete or update a row, we have to invalidate /* Since we are going to delete or update a row, we have to invalidate
the MySQL query cache for table. A deadlock of threads is not possible the MySQL query cache for table. A deadlock of threads is not possible
here because the caller of this function does not hold any latches with here because the caller of this function does not hold any latches with
the mutex rank above the lock_sys_t::mutex. The query cache mutex the mutex rank above the lock_sys.latch. The query cache mutex
has a rank just above the lock_sys_t::mutex. */ has a rank just above the lock_sys.latch. */
row_ins_invalidate_query_cache(thr, table->name.m_name); row_ins_invalidate_query_cache(thr, table->name.m_name);

View File

@@ -1170,7 +1170,7 @@ row_lock_table_autoinc_for_mysql(
/* If we already hold an AUTOINC lock on the table then do nothing. /* If we already hold an AUTOINC lock on the table then do nothing.
Note: We peek at the value of the current owner without acquiring Note: We peek at the value of the current owner without acquiring
the lock mutex. */ lock_sys.latch. */
if (trx == table->autoinc_trx) { if (trx == table->autoinc_trx) {
return(DB_SUCCESS); return(DB_SUCCESS);
@@ -2614,7 +2614,7 @@ skip:
if (!srv_fast_shutdown && !trx_sys.any_active_transactions()) { if (!srv_fast_shutdown && !trx_sys.any_active_transactions()) {
{ {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
skip = UT_LIST_GET_LEN(table->locks) != 0; skip = UT_LIST_GET_LEN(table->locks) != 0;
} }
if (skip) { if (skip) {

View File

@@ -1,7 +1,7 @@
/***************************************************************************** /*****************************************************************************
Copyright (c) 1997, 2017, Oracle and/or its affiliates. All Rights Reserved. Copyright (c) 1997, 2017, Oracle and/or its affiliates. All Rights Reserved.
Copyright (c) 2017, 2020, MariaDB Corporation. Copyright (c) 2017, 2021, 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
@@ -395,7 +395,7 @@ row_vers_impl_x_locked(
const rec_t* clust_rec; const rec_t* clust_rec;
dict_index_t* clust_index; dict_index_t* clust_index;
lock_sys.mutex_assert_unlocked(); lock_sys.assert_unlocked();
mtr_start(&mtr); mtr_start(&mtr);

View File

@@ -456,19 +456,13 @@ lock -- semaphore;
kernel -- kernel; kernel -- kernel;
query thread execution: query thread execution:
(a) without lock mutex (a) without lock_sys.latch
reserved -- process executing in user mode; reserved -- process executing in user mode;
(b) with lock mutex reserved (b) with lock_sys.latch reserved
-- process executing in kernel mode; -- process executing in kernel mode;
The server has several backgroind threads all running at the same The server has several background threads all running at the same
priority as user threads. It periodically checks if here is anything priority as user threads.
happening in the server which requires intervention of the master
thread. Such situations may be, for example, when flushing of dirty
blocks is needed in the buffer pool or old version of database rows
have to be cleaned away (purged). The user can configure a separate
dedicated purge thread(s) too, in which case the master thread does not
do any purging.
The threads which we call user threads serve the queries of the MySQL The threads which we call user threads serve the queries of the MySQL
server. They run at normal priority. server. They run at normal priority.
@@ -800,7 +794,7 @@ srv_printf_innodb_monitor(
/* Only if lock_print_info_summary proceeds correctly, /* Only if lock_print_info_summary proceeds correctly,
before we call the lock_print_info_all_transactions before we call the lock_print_info_all_transactions
to print all the lock information. IMPORTANT NOTE: This to print all the lock information. IMPORTANT NOTE: This
function acquires the lock mutex on success. */ function acquires exclusive lock_sys.latch on success. */
ret = lock_print_info_summary(file, nowait); ret = lock_print_info_summary(file, nowait);
if (ret) { if (ret) {
@@ -813,9 +807,8 @@ srv_printf_innodb_monitor(
} }
} }
/* NOTE: If we get here then we have the lock mutex. This /* NOTE: The following function will release the lock_sys.latch
function will release the lock mutex that we acquired when that lock_print_info_summary() acquired. */
we called the lock_print_info_summary() function earlier. */
lock_print_info_all_transactions(file); lock_print_info_all_transactions(file);
@@ -1235,7 +1228,7 @@ static void srv_monitor()
if (srv_print_innodb_monitor) { if (srv_print_innodb_monitor) {
/* Reset mutex_skipped counter everytime /* Reset mutex_skipped counter everytime
srv_print_innodb_monitor changes. This is to srv_print_innodb_monitor changes. This is to
ensure we will not be blocked by lock_sys.mutex ensure we will not be blocked by lock_sys.latch
for short duration information printing */ for short duration information printing */
if (!monitor_state.last_srv_print_monitor) { if (!monitor_state.last_srv_print_monitor) {
monitor_state.mutex_skipped = 0; monitor_state.mutex_skipped = 0;

View File

@@ -156,7 +156,7 @@ struct trx_i_s_cache_t {
ha_storage_t* storage; /*!< storage for external volatile ha_storage_t* storage; /*!< storage for external volatile
data that may become unavailable data that may become unavailable
when we release when we release
lock_sys.mutex */ lock_sys.latch */
ulint mem_allocd; /*!< the amount of memory ulint mem_allocd; /*!< the amount of memory
allocated with mem_alloc*() */ allocated with mem_alloc*() */
bool is_truncated; /*!< this is true if the memory bool is_truncated; /*!< this is true if the memory
@@ -422,7 +422,7 @@ fill_trx_row(
{ {
const char* s; const char* s;
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
const lock_t* wait_lock = trx->lock.wait_lock; const lock_t* wait_lock = trx->lock.wait_lock;
@@ -487,9 +487,8 @@ thd_done:
row->trx_tables_locked = lock_number_of_tables_locked(&trx->lock); row->trx_tables_locked = lock_number_of_tables_locked(&trx->lock);
/* These are protected by both trx->mutex or lock_sys.mutex, /* These are protected by lock_sys.latch (which we are holding)
or just lock_sys.mutex. For reading, it suffices to hold and sometimes also trx->mutex. */
lock_sys.mutex. */
row->trx_lock_structs = UT_LIST_GET_LEN(trx->lock.trx_locks); row->trx_lock_structs = UT_LIST_GET_LEN(trx->lock.trx_locks);
@@ -1038,7 +1037,7 @@ add_trx_relevant_locks_to_cache(
requested lock row, or NULL or requested lock row, or NULL or
undefined */ undefined */
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
/* If transaction is waiting we add the wait lock and all locks /* If transaction is waiting we add the wait lock and all locks
from another transactions that are blocking the wait lock. */ from another transactions that are blocking the wait lock. */
@@ -1181,7 +1180,7 @@ static void fetch_data_into_cache_low(trx_i_s_cache_t *cache, const trx_t *trx)
static void fetch_data_into_cache(trx_i_s_cache_t *cache) static void fetch_data_into_cache(trx_i_s_cache_t *cache)
{ {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
trx_i_s_cache_clear(cache); trx_i_s_cache_clear(cache);
/* Capture the state of transactions */ /* Capture the state of transactions */
@@ -1240,8 +1239,8 @@ trx_i_s_cache_init(
{ {
/* The latching is done in the following order: /* The latching is done in the following order:
acquire trx_i_s_cache_t::rw_lock, rwlock acquire trx_i_s_cache_t::rw_lock, rwlock
acquire lock mutex acquire exclusive lock_sys.latch
release lock mutex release exclusive lock_sys.latch
release trx_i_s_cache_t::rw_lock release trx_i_s_cache_t::rw_lock
acquire trx_i_s_cache_t::rw_lock, rdlock acquire trx_i_s_cache_t::rw_lock, rdlock
release trx_i_s_cache_t::rw_lock */ release trx_i_s_cache_t::rw_lock */

View File

@@ -1258,7 +1258,7 @@ trx_update_mod_tables_timestamp(
table->acquire() */ table->acquire() */
dict_sys.mutex_lock(); dict_sys.mutex_lock();
{ {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
if (!table->get_ref_count() if (!table->get_ref_count()
&& !UT_LIST_GET_LEN(table->locks)) { && !UT_LIST_GET_LEN(table->locks)) {
dict_sys.remove(table, true); dict_sys.remove(table, true);
@@ -1833,7 +1833,7 @@ state_ok:
/**********************************************************************//** /**********************************************************************//**
Prints info about a transaction. Prints info about a transaction.
The caller must hold lock_sys.mutex. The caller must hold lock_sys.latch.
When possible, use trx_print() instead. */ When possible, use trx_print() instead. */
void void
trx_print_latched( trx_print_latched(
@@ -1843,7 +1843,7 @@ trx_print_latched(
ulint max_query_len) /*!< in: max query length to print, ulint max_query_len) /*!< in: max query length to print,
or 0 to use the default max length */ or 0 to use the default max length */
{ {
lock_sys.mutex_assert_locked(); lock_sys.assert_locked();
trx_print_low(f, trx, max_query_len, trx_print_low(f, trx, max_query_len,
trx->lock.n_rec_locks, trx->lock.n_rec_locks,
@@ -1853,7 +1853,7 @@ trx_print_latched(
/**********************************************************************//** /**********************************************************************//**
Prints info about a transaction. Prints info about a transaction.
Acquires and releases lock_sys.mutex. */ Acquires and releases lock_sys.latch. */
void void
trx_print( trx_print(
/*======*/ /*======*/
@@ -1864,7 +1864,7 @@ trx_print(
{ {
ulint n_rec_locks, n_trx_locks, heap_size; ulint n_rec_locks, n_trx_locks, heap_size;
{ {
LockMutexGuard g; LockMutexGuard g{SRW_LOCK_CALL};
n_rec_locks= trx->lock.n_rec_locks; n_rec_locks= trx->lock.n_rec_locks;
n_trx_locks= UT_LIST_GET_LEN(trx->lock.trx_locks); n_trx_locks= UT_LIST_GET_LEN(trx->lock.trx_locks);
heap_size= mem_heap_get_size(trx->lock.lock_heap); heap_size= mem_heap_get_size(trx->lock.lock_heap);