summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorMarko Mäkelä <marko.makela@mariadb.com>2021-02-11 14:52:10 +0200
committerMarko Mäkelä <marko.makela@mariadb.com>2021-02-11 14:52:10 +0200
commitb01d8e1a336cd216fd87867cc7f6160c932ae8c0 (patch)
tree892eec6ff986b4ffd15a862ac731a741771d4f06
parent903464929ca2cd5a3fd994dad90d7bf78d653def (diff)
downloadmariadb-git-b01d8e1a336cd216fd87867cc7f6160c932ae8c0.tar.gz
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.
-rw-r--r--mysql-test/suite/perfschema/r/sxlock_func.result2
-rw-r--r--storage/innobase/btr/btr0btr.cc6
-rw-r--r--storage/innobase/btr/btr0cur.cc11
-rw-r--r--storage/innobase/buf/buf0buf.cc2
-rw-r--r--storage/innobase/gis/gis0sea.cc4
-rw-r--r--storage/innobase/handler/ha_innodb.cc14
-rw-r--r--storage/innobase/ibuf/ibuf0ibuf.cc2
-rw-r--r--storage/innobase/include/dict0mem.h17
-rw-r--r--storage/innobase/include/lock0lock.h117
-rw-r--r--storage/innobase/include/lock0priv.h2
-rw-r--r--storage/innobase/include/lock0priv.ic6
-rw-r--r--storage/innobase/include/lock0types.h4
-rw-r--r--storage/innobase/include/trx0sys.h4
-rw-r--r--storage/innobase/include/trx0trx.h28
-rw-r--r--storage/innobase/include/univ.i4
-rw-r--r--storage/innobase/lock/lock0iter.cc4
-rw-r--r--storage/innobase/lock/lock0lock.cc314
-rw-r--r--storage/innobase/lock/lock0prdt.cc20
-rw-r--r--storage/innobase/row/row0ins.cc6
-rw-r--r--storage/innobase/row/row0mysql.cc4
-rw-r--r--storage/innobase/row/row0vers.cc4
-rw-r--r--storage/innobase/srv/srv0srv.cc23
-rw-r--r--storage/innobase/trx/trx0i_s.cc17
-rw-r--r--storage/innobase/trx/trx0trx.cc10
24 files changed, 344 insertions, 281 deletions
diff --git a/mysql-test/suite/perfschema/r/sxlock_func.result b/mysql-test/suite/perfschema/r/sxlock_func.result
index a56d542bf77..943f2c9301b 100644
--- a/mysql-test/suite/perfschema/r/sxlock_func.result
+++ b/mysql-test/suite/perfschema/r/sxlock_func.result
@@ -9,6 +9,7 @@ AND name!='wait/synch/rwlock/innodb/btr_search_latch' ORDER BY name;
name
wait/synch/rwlock/innodb/dict_operation_lock
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_purge_latch
TRUNCATE TABLE performance_schema.events_waits_history_long;
@@ -39,6 +40,7 @@ ORDER BY event_name;
event_name
wait/synch/rwlock/innodb/dict_operation_lock
wait/synch/rwlock/innodb/fil_space_latch
+wait/synch/rwlock/innodb/lock_latch
SELECT event_name FROM performance_schema.events_waits_history_long
WHERE event_name = 'wait/synch/sxlock/innodb/index_tree_rw_lock'
AND operation IN ('try_shared_lock','shared_lock') LIMIT 1;
diff --git a/storage/innobase/btr/btr0btr.cc b/storage/innobase/btr/btr0btr.cc
index e0b1fdddf9e..44d120f9881 100644
--- a/storage/innobase/btr/btr0btr.cc
+++ b/storage/innobase/btr/btr0btr.cc
@@ -3355,7 +3355,7 @@ btr_lift_page_up(
const page_id_t id{block->page.id()};
/* Free predicate page locks on the block */
if (index->is_spatial()) {
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
lock_prdt_page_free_from_discard(
id, &lock_sys.prdt_page_hash);
}
@@ -3609,7 +3609,7 @@ retry:
}
/* No GAP lock needs to be worrying about */
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
lock_prdt_page_free_from_discard(
id, &lock_sys.prdt_page_hash);
lock_rec_free_all_from_discard_page(id);
@@ -3762,7 +3762,7 @@ retry:
merge_page, mtr);
}
const page_id_t id{block->page.id()};
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
lock_prdt_page_free_from_discard(
id, &lock_sys.prdt_page_hash);
lock_rec_free_all_from_discard_page(id);
diff --git a/storage/innobase/btr/btr0cur.cc b/storage/innobase/btr/btr0cur.cc
index f4ecdf31625..4fea505ed28 100644
--- a/storage/innobase/btr/btr0cur.cc
+++ b/storage/innobase/btr/btr0cur.cc
@@ -1999,11 +1999,12 @@ retry_page_get:
trx_t* trx = thr_get_trx(cursor->thr);
lock_prdt_t prdt;
- lock_sys.mutex_lock();
- lock_init_prdt_from_mbr(
- &prdt, &cursor->rtr_info->mbr, mode,
- trx->lock.lock_heap);
- lock_sys.mutex_unlock();
+ {
+ LockMutexGuard g{SRW_LOCK_CALL};
+ lock_init_prdt_from_mbr(
+ &prdt, &cursor->rtr_info->mbr, mode,
+ trx->lock.lock_heap);
+ }
if (rw_latch == RW_NO_LATCH && height != 0) {
block->lock.s_lock();
diff --git a/storage/innobase/buf/buf0buf.cc b/storage/innobase/buf/buf0buf.cc
index 4fcbc61e4f0..273178c164f 100644
--- a/storage/innobase/buf/buf0buf.cc
+++ b/storage/innobase/buf/buf0buf.cc
@@ -2038,7 +2038,7 @@ withdraw_retry:
{found, withdraw_started, my_hrtime_coarse()};
withdraw_started = current_time;
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
trx_sys.trx_list.for_each(f);
}
diff --git a/storage/innobase/gis/gis0sea.cc b/storage/innobase/gis/gis0sea.cc
index 52bea50edcf..e2bb40c2155 100644
--- a/storage/innobase/gis/gis0sea.cc
+++ b/storage/innobase/gis/gis0sea.cc
@@ -387,7 +387,7 @@ rtr_pcur_getnext_from_path(
trx_t* trx = thr_get_trx(
btr_cur->rtr_info->thr);
{
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
lock_init_prdt_from_mbr(
&prdt, &btr_cur->rtr_info->mbr,
mode, trx->lock.lock_heap);
@@ -1197,7 +1197,7 @@ rtr_check_discard_page(
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_page_hash);
}
diff --git a/storage/innobase/handler/ha_innodb.cc b/storage/innobase/handler/ha_innodb.cc
index 050aa7e9f5e..1dc1af1e4ec 100644
--- a/storage/innobase/handler/ha_innodb.cc
+++ b/storage/innobase/handler/ha_innodb.cc
@@ -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 trx_pool_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 trx_sys_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(trx_pool_mutex),
PSI_KEY(trx_pool_manager_mutex),
- PSI_KEY(lock_mutex),
PSI_KEY(lock_wait_mutex),
PSI_KEY(srv_threads_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 trx_i_s_cache_lock_key;
mysql_pfs_key_t trx_purge_latch_key;
+mysql_pfs_key_t lock_latch_key;
/* all_innodb_rwlocks array contains rwlocks that are
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 },
{ &trx_i_s_cache_lock_key, "trx_i_s_cache_lock", 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 }
};
# endif /* UNIV_PFS_RWLOCK */
@@ -2307,7 +2307,7 @@ innobase_trx_init(
DBUG_ASSERT(thd == trx->mysql_thd);
/* 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
intern_sys_var_ptr(). */
THDVAR(thd, lock_wait_timeout);
@@ -2695,7 +2695,7 @@ static bool innobase_query_caching_table_check_low(
return false;
}
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
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)
{
{
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
mysql_mutex_lock(&lock_sys.wait_mutex);
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(victim_trx);
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
ut_ad(victim_trx->mutex_is_owner());
DBUG_ENTER("wsrep_innobase_kill_one_trx");
@@ -18102,7 +18102,7 @@ wsrep_abort_transaction(
wsrep_thd_transaction_state_str(victim_thd));
if (victim_trx) {
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
victim_trx->mutex_lock();
int rcode= wsrep_innobase_kill_one_trx(bf_thd,
victim_trx, signal);
diff --git a/storage/innobase/ibuf/ibuf0ibuf.cc b/storage/innobase/ibuf/ibuf0ibuf.cc
index 7f288d0b30a..a240a180b34 100644
--- a/storage/innobase/ibuf/ibuf0ibuf.cc
+++ b/storage/innobase/ibuf/ibuf0ibuf.cc
@@ -3280,7 +3280,7 @@ commit_exit:
ibuf_mtr_commit(&bitmap_mtr);
goto fail_exit;
} else {
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
if (lock_sys.get_first(page_id)) {
goto commit_exit;
}
diff --git a/storage/innobase/include/dict0mem.h b/storage/innobase/include/dict0mem.h
index 11611b1110c..32498c832e0 100644
--- a/storage/innobase/include/dict0mem.h
+++ b/storage/innobase/include/dict0mem.h
@@ -2261,7 +2261,7 @@ public:
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
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
created on the trx lock heap rather than use the pre-allocated instance
in autoinc_lock below. */
@@ -2280,11 +2280,11 @@ public:
ib_uint64_t autoinc;
/** 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;
/** 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),
ha_innobase::innobase_lock_autoinc() will perform a dirty read
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;
/** FTS specific state variables. */
@@ -2305,8 +2306,8 @@ public:
/** 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.
- It is protected by lock_sys.mutex. */
- ulint n_rec_locks;
+ Protected by LockGuard. */
+ ulint n_rec_locks;
private:
/** 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. */
Atomic_counter<uint32_t> n_ref_count;
public:
- /** List of locks on the table. Protected by lock_sys.mutex. */
- table_lock_list_t locks;
+ /** List of locks on the table. Protected by lock_sys.assert_locked(lock). */
+ table_lock_list_t locks;
/** Timestamp of the last modification of this table. */
time_t update_time;
diff --git a/storage/innobase/include/lock0lock.h b/storage/innobase/include/lock0lock.h
index 0caf8925b4c..b2238ddd47d 100644
--- a/storage/innobase/include/lock0lock.h
+++ b/storage/innobase/include/lock0lock.h
@@ -449,13 +449,12 @@ lock_report_trx_id_insanity(
trx_id_t max_trx_id); /*!< in: trx_sys.get_max_trx_id() */
/*********************************************************************//**
Prints info of locks for all transactions.
-@return FALSE if not able to obtain lock mutex and exits without
-printing info */
+@return FALSE if not able to acquire lock_sys.latch (and display info) */
ibool
lock_print_info_summary(
/*====================*/
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));
/** 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);
/*********************************************************************//**
-Prints info of locks for each transaction. This function assumes that the
-caller holds the lock mutex and more importantly it will release the lock
-mutex on behalf of the caller. (This should be fixed in the future). */
+Prints info of locks for each transaction. This function will release
+lock_sys.latch, which the caller must be holding in exclusive mode. */
void
lock_print_info_all_transactions(
/*=============================*/
@@ -476,7 +474,7 @@ lock_print_info_all_transactions(
/*********************************************************************//**
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
lock_number_of_tables_locked(
/*=========================*/
@@ -583,7 +581,13 @@ class lock_sys_t
bool m_initialised;
/** 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:
/** record locks */
hash_table_t rec_hash;
@@ -615,31 +619,77 @@ public:
lock_sys_t(): m_initialised(false) {}
- bool is_initialised() { return m_initialised; }
+ bool is_initialised() const { return m_initialised; }
-#ifdef HAVE_PSI_MUTEX_INTERFACE
- /** Try to acquire lock_sys.mutex */
- ATTRIBUTE_NOINLINE int mutex_trylock();
- /** Acquire lock_sys.mutex */
- ATTRIBUTE_NOINLINE void mutex_lock();
- /** Release lock_sys.mutex */
- ATTRIBUTE_NOINLINE void mutex_unlock();
+#ifdef UNIV_PFS_RWLOCK
+ /** Acquire exclusive lock_sys.latch */
+ ATTRIBUTE_NOINLINE
+ void wr_lock(const char *file, unsigned line);
+ /** Release exclusive lock_sys.latch */
+ ATTRIBUTE_NOINLINE void wr_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
- /** Try to acquire lock_sys.mutex */
- int mutex_trylock() { return mysql_mutex_trylock(&mutex); }
- /** Aqcuire lock_sys.mutex */
- void mutex_lock() { mysql_mutex_lock(&mutex); }
- /** Release lock_sys.mutex */
- void mutex_unlock() { mysql_mutex_unlock(&mutex); }
+ /** Acquire exclusive lock_sys.latch */
+ void wr_lock()
+ {
+ ut_ad(!is_writer());
+ latch.wr_lock();
+ 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
- /** Assert that mutex_lock() has been invoked */
- void mutex_assert_locked() const { mysql_mutex_assert_owner(&mutex); }
- /** Assert that mutex_lock() has not been invoked */
- void mutex_assert_unlocked() const { mysql_mutex_assert_not_owner(&mutex); }
+ /** Try to acquire exclusive lock_sys.latch
+ @return whether the latch was acquired */
+ bool wr_lock_try()
+ {
+ 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 */
- void wait_lock(lock_t **lock, pthread_cond_t *cond)
- { while (*lock) my_cond_wait(cond, &mutex.m_mutex); }
+ /** Assert that wr_lock() has been invoked by this thread */
+ void assert_locked() const { ut_ad(is_writer()); }
+ /** 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.
@@ -678,7 +728,7 @@ public:
/** @return the hash value for a page address */
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.
@param lock_hash hash table to look at
@@ -720,11 +770,12 @@ public:
/** The lock system */
extern lock_sys_t lock_sys;
-/** lock_sys.mutex guard */
+/** lock_sys.latch guard */
struct LockMutexGuard
{
- LockMutexGuard() { lock_sys.mutex_lock(); }
- ~LockMutexGuard() { lock_sys.mutex_unlock(); }
+ LockMutexGuard(SRW_LOCK_ARGS(const char *file, unsigned line))
+ { lock_sys.wr_lock(SRW_LOCK_ARGS(file, line)); }
+ ~LockMutexGuard() { lock_sys.wr_unlock(); }
};
/*********************************************************************//**
diff --git a/storage/innobase/include/lock0priv.h b/storage/innobase/include/lock0priv.h
index d9dbbf5dbc4..33a9c51e50f 100644
--- a/storage/innobase/include/lock0priv.h
+++ b/storage/innobase/include/lock0priv.h
@@ -481,7 +481,7 @@ lock_rec_set_nth_bit(
inline byte lock_rec_reset_nth_bit(lock_t* lock, ulint i)
{
ut_ad(!lock->is_table());
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
ut_ad(i < lock->un_member.rec_lock.n_bits);
byte* b = reinterpret_cast<byte*>(&lock[1]) + (i >> 3);
diff --git a/storage/innobase/include/lock0priv.ic b/storage/innobase/include/lock0priv.ic
index 65d0cd04610..45c91ba20d8 100644
--- a/storage/innobase/include/lock0priv.ic
+++ b/storage/innobase/include/lock0priv.ic
@@ -78,7 +78,7 @@ lock_rec_set_nth_bit(
ulint bit_index;
ut_ad(!lock->is_table());
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
ut_ad(i < lock->un_member.rec_lock.n_bits);
byte_index = i / 8;
@@ -117,7 +117,7 @@ lock_rec_get_next(
ulint heap_no,/*!< in: heap number of the record */
lock_t* lock) /*!< in: lock */
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
do {
lock = lock_rec_get_next_on_page(lock);
@@ -175,7 +175,7 @@ lock_rec_get_next_on_page_const(
ut_ad(!lock->is_table());
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))))
if (lock->un_member.rec_lock.page_id == page_id)
diff --git a/storage/innobase/include/lock0types.h b/storage/innobase/include/lock0types.h
index 3f3f7a3a9a5..d694c079327 100644
--- a/storage/innobase/include/lock0types.h
+++ b/storage/innobase/include/lock0types.h
@@ -1,7 +1,7 @@
/*****************************************************************************
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
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
/* @} */
-/** Lock struct; protected by lock_sys.mutex */
+/** Lock struct; protected by lock_sys.latch */
struct ib_lock_t
{
trx_t* trx; /*!< transaction owning the
diff --git a/storage/innobase/include/trx0sys.h b/storage/innobase/include/trx0sys.h
index fdd30223794..d3741940d76 100644
--- a/storage/innobase/include/trx0sys.h
+++ b/storage/innobase/include/trx0sys.h
@@ -593,10 +593,10 @@ public:
the transaction may get committed before this method returns.
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
- 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.
Ideally this method should get caller rw_trx_hash_pins along with trx
diff --git a/storage/innobase/include/trx0trx.h b/storage/innobase/include/trx0trx.h
index 9ab7a37d636..48908576eb3 100644
--- a/storage/innobase/include/trx0trx.h
+++ b/storage/innobase/include/trx0trx.h
@@ -264,7 +264,7 @@ trx_print_latched(
/**********************************************************************//**
Prints info about a transaction.
-Acquires and releases lock_sys.mutex. */
+Acquires and releases lock_sys.latch. */
void
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;
/** 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
{
/** 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.
Set to nullptr when holding lock_sys.wait_mutex. */
Atomic_relaxed<lock_t*> wait_lock;
@@ -440,14 +440,14 @@ struct trx_lock_t
#else
/** When the transaction decides to wait for a lock, it clears this;
set if another transaction chooses this transaction as a victim in deadlock
- resolution. Protected by lock_sys.mutex and lock_sys.wait_mutex. */
+ resolution. Protected by lock_sys.latch and lock_sys.wait_mutex. */
bool was_chosen_as_deadlock_victim;
#endif
/** Whether the transaction is being rolled back either via deadlock
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
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;
/** Next available rec_pool[] entry */
@@ -459,7 +459,7 @@ struct trx_lock_t
trx that is in waiting
state. For threads suspended in a
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
serving the running transaction. */
@@ -472,12 +472,12 @@ struct trx_lock_t
ib_lock_t table_pool[8];
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;
insertions are protected by trx->mutex
- and lock_sys.mutex; removals are
- protected by lock_sys.mutex */
+ and lock_sys.latch; removals are
+ protected by lock_sys.latch */
lock_list table_locks; /*!< All table locks requested by this
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).
* 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
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
lock_rec_convert_impl_to_expl()) will access transactions associated
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.*/
struct trx_undo_ptr_t {
@@ -669,7 +669,7 @@ public:
private:
/** mutex protecting state and some of lock
- (some are protected by lock_sys.mutex) */
+ (some are protected by lock_sys.latch) */
srw_mutex mutex;
#ifdef UNIV_DEBUG
/** The owner of mutex (0 if none); protected by mutex */
@@ -766,7 +766,7 @@ public:
Transitions to COMMITTED are protected by trx_t::mutex. */
Atomic_relaxed<trx_state_t> state;
- /** The locks of the transaction. Protected by lock_sys.mutex
+ /** The locks of the transaction. Protected by lock_sys.latch
(insertions also by trx_t::mutex). */
trx_lock_t lock;
@@ -918,7 +918,7 @@ public:
also in the lock list trx_locks. This
vector needs to be freed explicitly
when the trx instance is destroyed.
- Protected by lock_sys.mutex. */
+ Protected by lock_sys.latch. */
/*------------------------------*/
bool read_only; /*!< true if transaction is flagged
as a READ-ONLY transaction.
diff --git a/storage/innobase/include/univ.i b/storage/innobase/include/univ.i
index 44d76be0f5f..9808e34c860 100644
--- a/storage/innobase/include/univ.i
+++ b/storage/innobase/include/univ.i
@@ -1,7 +1,7 @@
/*****************************************************************************
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.
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 trx_pool_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 srv_threads_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_online_log_key;
extern mysql_pfs_key_t trx_sys_rw_lock_key;
+extern mysql_pfs_key_t lock_latch_key;
# endif /* UNIV_PFS_RWLOCK */
#endif /* HAVE_PSI_INTERFACE */
diff --git a/storage/innobase/lock/lock0iter.cc b/storage/innobase/lock/lock0iter.cc
index eb0c2c1d005..0cd271bfd8d 100644
--- a/storage/innobase/lock/lock0iter.cc
+++ b/storage/innobase/lock/lock0iter.cc
@@ -49,7 +49,7 @@ lock_queue_iterator_reset(
ulint bit_no) /*!< in: record number in the
heap */
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked(*lock);
iter->current_lock = lock;
@@ -75,7 +75,7 @@ lock_queue_iterator_get_prev(
/*=========================*/
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()
? lock_rec_get_prev(iter->current_lock, iter->bit_no)
diff --git a/storage/innobase/lock/lock0lock.cc b/storage/innobase/lock/lock0lock.cc
index ef38bdd57c2..bff2954c1eb 100644
--- a/storage/innobase/lock/lock0lock.cc
+++ b/storage/innobase/lock/lock0lock.cc
@@ -225,7 +225,7 @@ private:
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;
/** 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;
- 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);
rec_hash.create(n_cells);
@@ -357,15 +357,37 @@ void lock_sys_t::create(ulint n_cells)
}
-#ifdef HAVE_PSI_MUTEX_INTERFACE
-/** Try to acquire lock_sys.mutex */
-int lock_sys_t::mutex_trylock() { return mysql_mutex_trylock(&mutex); }
-/** Acquire lock_sys.mutex */
-void lock_sys_t::mutex_lock() { mysql_mutex_lock(&mutex); }
-/** Release lock_sys.mutex */
-void lock_sys_t::mutex_unlock() { mysql_mutex_unlock(&mutex); }
-#endif
+#ifdef UNIV_PFS_RWLOCK
+/** Acquire exclusive lock_sys.latch */
+void lock_sys_t::wr_lock(const char *file, unsigned line)
+{
+ latch.wr_lock(file, line);
+ ut_ad(!writer.exchange(os_thread_get_curr_id(), std::memory_order_relaxed));
+}
+/** 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.
@param[in] lock record lock object
@@ -385,7 +407,7 @@ void lock_sys_t::resize(ulint n_cells)
{
ut_ad(this == &lock_sys);
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
hash_table_t old_hash(rec_hash);
rec_hash.create(n_cells);
@@ -425,7 +447,7 @@ void lock_sys_t::close()
prdt_hash.free();
prdt_page_hash.free();
- mysql_mutex_destroy(&mutex);
+ latch.destroy();
mysql_mutex_destroy(&wait_mutex);
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)
<< " query: " << wsrep_thd_query(trx->mysql_thd);
if (!locked) {
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
trx_print_latched(stderr, trx, 3000);
} else {
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
trx_print_latched(stderr, trx, 3000);
}
@@ -759,7 +781,7 @@ lock_rec_get_prev(
ut_ad(!in_lock->is_table());
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);
lock != in_lock;
@@ -855,7 +877,7 @@ lock_rec_other_has_expl_req(
ATTRIBUTE_COLD
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(trx->is_wsrep());
@@ -926,7 +948,7 @@ lock_rec_other_has_conflicting(
ulint heap_no,/*!< in: heap number of the record */
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);
@@ -973,7 +995,7 @@ lock_sec_rec_some_has_impl(
trx_id_t max_trx_id;
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(page_rec_is_user_rec(rec));
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.
-The caller must be holding lock_sys.mutex. */
+The caller must be holding lock_sys.latch. */
ulint
lock_number_of_tables_locked(
/*=========================*/
@@ -1015,7 +1037,7 @@ lock_number_of_tables_locked(
const lock_t* lock;
ulint n_tables = 0;
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
for (lock = UT_LIST_GET_FIRST(trx_lock->trx_locks);
lock != NULL;
@@ -1062,14 +1084,30 @@ wsrep_print_wait_locks(
}
#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.
@param[in,out] lock lock that was possibly being waited for */
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->trx->lock.wait_lock || lock->trx->lock.wait_lock == lock);
- lock->trx->lock.wait_lock= nullptr;
+ ut_ad(!trx->lock.wait_lock || trx->lock.wait_lock == lock);
+ trx->lock.wait_lock= nullptr;
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(!trx->lock.wait_lock || trx->lock.wait_lock != lock);
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());
trx->lock.wait_lock= lock;
@@ -1118,7 +1156,7 @@ lock_rec_create_low(
lock_t* lock;
ulint n_bytes;
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
ut_ad(holds_trx_mutex == trx->mutex_is_owner());
ut_ad(dict_index_is_clust(index) || !dict_index_is_online_ddl(index));
ut_ad(!(type_mode & LOCK_TABLE));
@@ -1307,7 +1345,7 @@ lock_rec_enqueue_waiting(
que_thr_t* thr,
lock_prdt_t* prdt)
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
ut_ad(!srv_read_only_mode);
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() */
const trx_t* trx) /*!< in: transaction */
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
for (/* No op */;
lock != NULL;
@@ -1412,7 +1450,7 @@ lock_rec_add_to_queue(
/*!< in: TRUE if caller owns the
transaction mutex */
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
ut_ad(caller_owns_trx_mutex == trx->mutex_is_owner());
ut_ad(index->is_primary()
|| dict_index_get_online_status(index) != ONLINE_INDEX_CREATION);
@@ -1541,7 +1579,7 @@ lock_rec_lock(
MONITOR_ATOMIC_INC(MONITOR_NUM_RECLOCK_REQ);
const page_id_t id{block->page.id()};
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
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_table());
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
heap_no = lock_rec_find_set_bit(wait_lock);
@@ -1810,7 +1848,7 @@ end_wait:
if (trx->lock.wait_lock)
{
{
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
mysql_mutex_lock(&lock_sys.wait_mutex);
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. */
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);
lock_reset_lock_and_trx_wait(lock);
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. */
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--;
@@ -1968,7 +2006,7 @@ lock_rec_discard(
trx_lock_t* trx_lock;
ut_ad(!in_lock->is_table());
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
trx_lock = &in_lock->trx->lock;
@@ -2073,7 +2111,7 @@ lock_rec_inherit_to_gap(
ulint heap_no) /*!< in: heap_no of the
donating record */
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
/* At READ UNCOMMITTED or READ COMMITTED isolation level,
we do not want locks set
@@ -2114,7 +2152,7 @@ lock_rec_inherit_to_gap_if_gap_lock(
on this record */
{
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);
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()};
- lock_sys.mutex_assert_locked();
-
/* If the lock is predicate lock, it resides on INFIMUM record */
ut_ad(!lock_rec_get_first(lock_hash, receiver_id, receiver_heap_no)
|| lock_hash == &lock_sys.prdt_hash
@@ -2253,7 +2289,7 @@ lock_move_reorganize_page(
UT_LIST_INIT(old_locks, &lock_t::trx_locks);
const page_id_t id{block->page.id()};
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
/* FIXME: This needs to deal with predicate lock too */
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 new_id{new_block->page.id()};
{
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
/* Note: when we move locks from record to record, waiting locks
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()};
{
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
for (lock_t *lock= lock_sys.get_first(id); 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()};
{
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
for (lock_t *lock= lock_sys.get_first(id); 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 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
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 r{right_block->page.id()};
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
/* Inherit the locks from the supremum of the left page to the
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. */
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 */
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) */
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 */
lock_rec_move(new_block, old, PAGE_HEAP_NO_SUPREMUM, PAGE_HEAP_NO_SUPREMUM);
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);
const page_id_t l{left_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
successor of the infimum on the right page */
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()};
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
const rec_t *left_next_rec= page_rec_get_next_const(orig_pred);
if (!page_rec_is_supremum(left_next_rec))
@@ -2827,7 +2863,7 @@ lock_rec_reset_and_inherit_gap_locks(
donating record */
{
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_inherit_to_gap(heir, donor, heir_block.frame, heir_heap_no, heap_no);
}
@@ -2849,7 +2885,7 @@ lock_update_discard(
ulint heap_no;
const page_id_t heir(heir_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)) {
ut_ad(!lock_sys.get_first_prdt(page_id));
@@ -2958,7 +2994,7 @@ lock_update_delete(
}
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 */
@@ -2991,7 +3027,7 @@ lock_rec_store_on_page_infimum(
ut_ad(block->frame == page_align(rec));
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);
}
@@ -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 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);
}
@@ -3040,7 +3076,7 @@ lock_table_create(
lock_t* lock;
ut_ad(table && trx);
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked(*table);
ut_ad(trx->mutex_is_owner());
check_trx_state(trx);
@@ -3146,7 +3182,6 @@ lock_table_pop_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));
/* 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;
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));
/* With stored functions and procedures the user may drop
@@ -3227,10 +3262,10 @@ lock_table_remove_low(
trx_t* trx;
dict_table_t* table;
- lock_sys.mutex_assert_locked();
-
+ ut_ad(lock->is_table());
trx = lock->trx;
table = lock->un_member.tab_lock.table;
+ lock_sys.assert_locked(*table);
/* Remove the table from the transaction's AUTOINC vector, if
the lock that is being released is an AUTOINC lock. */
@@ -3288,7 +3323,7 @@ lock_table_enqueue_waiting(
#endif
)
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked(*table);
ut_ad(!srv_read_only_mode);
trx_t* trx = thr_get_trx(thr);
@@ -3342,7 +3377,7 @@ lock_table_other_has_incompatible(
const dict_table_t* table, /*!< in: table */
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_IX == 1, "compatibility");
@@ -3427,7 +3462,7 @@ lock_table(
err = DB_SUCCESS;
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
/* We have to check if the new lock is compatible with any locks
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;
{
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
ut_ad(!lock_table_other_has_incompatible(trx, LOCK_WAIT, table, mode));
trx->mutex_lock();
@@ -3474,40 +3509,27 @@ void lock_table_resurrect(dict_table_t *table, trx_t *trx, lock_mode mode)
trx->mutex_unlock();
}
-/*********************************************************************//**
-Checks if a waiting table lock request still has to wait in a queue.
-@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 */
+/** Find a lock that a waiting table lock request still has to wait for. */
+static const lock_t *lock_table_has_to_wait_in_queue(const lock_t *wait_lock)
{
- ut_ad(wait_lock->is_waiting());
- ut_ad(wait_lock->is_table());
-
- dict_table_t *table = wait_lock->un_member.tab_lock.table;
- lock_sys.mutex_assert_locked();
+ ut_ad(wait_lock->is_waiting());
+ ut_ad(wait_lock->is_table());
- static_assert(LOCK_IS == 0, "compatibility");
- static_assert(LOCK_IX == 1, "compatibility");
+ dict_table_t *table= wait_lock->un_member.tab_lock.table;
+ lock_sys.assert_locked(*table);
- if (UNIV_LIKELY(wait_lock->mode() <= LOCK_IX
- && !table->n_lock_x_or_s)) {
- return(false);
- }
-
- for (const lock_t *lock = UT_LIST_GET_FIRST(table->locks);
- lock != wait_lock;
- lock = UT_LIST_GET_NEXT(un_member.tab_lock.locks, lock)) {
+ static_assert(LOCK_IS == 0, "compatibility");
+ static_assert(LOCK_IX == 1, "compatibility");
- if (lock_has_to_wait(wait_lock, lock)) {
+ if (UNIV_LIKELY(wait_lock->mode() <= LOCK_IX && !table->n_lock_x_or_s))
+ return nullptr;
- return(true);
- }
- }
+ for (const lock_t *lock= UT_LIST_GET_FIRST(table->locks); lock != wait_lock;
+ lock= UT_LIST_GET_NEXT(un_member.tab_lock.locks, lock))
+ if (lock_has_to_wait(wait_lock, lock))
+ return lock;
- return(false);
+ return nullptr;
}
/*************************************************************//**
@@ -3521,7 +3543,6 @@ static void lock_table_dequeue(lock_t *in_lock, bool owns_wait_mutex)
#ifdef SAFE_MUTEX
ut_ad(owns_wait_mutex == mysql_mutex_is_owner(&lock_sys.wait_mutex));
#endif
- lock_sys.mutex_assert_locked();
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);
@@ -3579,7 +3600,7 @@ void lock_table_x_unlock(dict_table_t *table, trx_t *trx)
if (lock->type_mode != (LOCK_TABLE | LOCK_X))
continue;
{
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
lock_table_dequeue(lock, false);
}
lock= nullptr;
@@ -3663,7 +3684,7 @@ lock_rec_unlock(
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);
@@ -3766,7 +3787,7 @@ void lock_release(trx_t* trx)
trx_id_t max_trx_id = trx_sys.get_max_trx_id();
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);
lock != NULL;
@@ -3792,9 +3813,9 @@ void lock_release(trx_t* trx)
/* Release the mutex for a while, so that we
do not monopolize it */
- lock_sys.mutex_unlock();
+ lock_sys.wr_unlock();
count = 0;
- lock_sys.mutex_lock();
+ lock_sys.wr_lock(SRW_LOCK_CALL);
}
++count;
@@ -3814,9 +3835,9 @@ lock_trx_table_locks_remove(
trx_t* trx = lock_to_remove->trx;
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;
if (!have_mutex) {
trx->mutex_lock();
@@ -3855,7 +3876,7 @@ static
void
lock_table_print(FILE* file, const lock_t* lock)
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
ut_a(lock->is_table());
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());
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
" index %s of table ",
@@ -3998,7 +4019,7 @@ static ulint lock_get_n_rec_locks()
ulint n_locks = 0;
ulint i;
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
for (i = 0; i < lock_sys.rec_hash.n_cells; i++) {
const lock_t* lock;
@@ -4019,20 +4040,16 @@ static ulint lock_get_n_rec_locks()
/*********************************************************************//**
Prints info of locks for all transactions.
-@return FALSE if not able to obtain lock mutex
-and exits without printing info */
+@return FALSE if not able to acquire lock_sys.latch (and dislay info) */
ibool
lock_print_info_summary(
/*====================*/
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) {
- lock_sys.mutex_lock();
- } else if (lock_sys.mutex_trylock()) {
+ lock_sys.wr_lock(SRW_LOCK_CALL);
+ } else if (!lock_sys.wr_lock_try()) {
fputs("FAIL TO OBTAIN LOCK MUTEX,"
" SKIP LOCK INFO PRINTING\n", file);
return(FALSE);
@@ -4160,9 +4177,8 @@ struct lock_print_info
};
/*********************************************************************//**
-Prints info of locks for each transaction. This function assumes that the
-caller holds the lock mutex and more importantly it will release the lock
-mutex on behalf of the caller. (This should be fixed in the future). */
+Prints info of locks for each transaction. This function will release
+lock_sys.latch, which the caller must be holding in exclusive mode. */
void
lock_print_info_all_transactions(
/*=============================*/
@@ -4171,7 +4187,7 @@ lock_print_info_all_transactions(
fprintf(file, "LIST OF TRANSACTIONS FOR EACH SESSION:\n");
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());
}
@@ -4226,14 +4242,14 @@ lock_table_queue_validate(
{
const lock_t* lock;
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
for (lock = UT_LIST_GET_FIRST(table->locks);
lock != NULL;
lock = UT_LIST_GET_NEXT(un_member.tab_lock.locks, lock)) {
/* 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. */
lock->trx->mutex_lock();
check_trx_state(lock->trx);
@@ -4263,7 +4279,7 @@ lock_rec_queue_validate(
/*====================*/
bool locked_lock_trx_sys,
/*!< in: if the caller holds
- both the lock mutex and
+ both the lock_sys.latch and
trx_sys_t->lock. */
const page_id_t id, /*!< in: page identifier */
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);
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)) {
@@ -4308,14 +4324,14 @@ lock_rec_queue_validate(
func_exit:
if (!locked_lock_trx_sys) {
- lock_sys.mutex_unlock();
+ lock_sys.wr_unlock();
}
return true;
}
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()
? 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_);
const page_id_t id{block->page.id()};
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
loop:
lock = lock_sys.get_first(id);
@@ -4514,7 +4530,7 @@ lock_rec_validate(
page_id_t* limit) /*!< in/out: upper limit of
(space, page_no) */
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
for (const lock_t* lock = static_cast<const lock_t*>(
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*)
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
mysql_mutex_lock(&element->mutex);
if (element->trx)
{
@@ -4602,7 +4618,7 @@ static void lock_validate()
{
std::set<page_id_t> pages;
{
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
/* 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));
{
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
/* Because this code is invoked for a running transaction by
the thread that is serving the transaction, it is not necessary
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");
{
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
trx->mutex_lock();
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)
{
ut_ad(!page_rec_is_metadata(rec));
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
ut_ad(trx->is_referenced());
const trx_state_t state{trx->state};
ut_ad(state != TRX_STATE_NOT_STARTED);
@@ -4876,7 +4892,7 @@ lock_rec_convert_impl_to_expl(
{
trx_t* trx;
- lock_sys.mutex_assert_unlocked();
+ lock_sys.assert_unlocked();
ut_ad(page_rec_is_user_rec(rec));
ut_ad(rec_offs_validate(rec, index, 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. */
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
ut_ad(owns_wait_mutex == mysql_mutex_is_owner(&lock_sys.wait_mutex));
#endif /* SAFE_MUTEX */
@@ -5345,7 +5361,7 @@ lock_rec_get_index(
/** Cancel a waiting lock request and release possibly waiting transactions */
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);
trx_t *trx= lock->trx;
ut_ad(trx->state == TRX_STATE_ACTIVE);
@@ -5381,7 +5397,7 @@ lock_unlock_table_autoinc(
/*======================*/
trx_t* trx) /*!< in/out: transaction */
{
- lock_sys.mutex_assert_unlocked();
+ lock_sys.assert_unlocked();
ut_ad(!trx->mutex_is_owner());
ut_ad(!trx->lock.wait_lock);
@@ -5396,14 +5412,14 @@ lock_unlock_table_autoinc(
necessary to hold trx->mutex here. */
if (lock_trx_holds_autoinc_locks(trx)) {
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
lock_release_autoinc_locks(trx, false);
}
}
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);
ut_ad(trx->mutex_is_owner());
@@ -5433,7 +5449,7 @@ lock_trx_handle_wait(
#endif /* WITH_WSREP */
dberr_t err;
{
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
mysql_mutex_lock(&lock_sys.wait_mutex);
trx->mutex_lock();
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,
const dict_table_t *table)
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
mysql_mutex_lock(&element->mutex);
if (element->trx)
{
@@ -5494,7 +5510,7 @@ lock_table_has_locks(
held on records in this table or on the
table itself */
{
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
bool has_locks= UT_LIST_GET_LEN(table->locks) > 0 || table->n_rec_locks > 0;
#ifdef UNIV_DEBUG
if (!has_locks)
@@ -5526,7 +5542,7 @@ lock_trx_has_sys_table_locks(
const lock_t* strongest_lock = 0;
lock_mode strongest = LOCK_NONE;
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
const lock_list::const_iterator end = trx->lock.table_locks.end();
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));
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));
}
return true;
@@ -5600,7 +5616,7 @@ is enabled. */
void
DeadlockChecker::start_print()
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
rewind(lock_latest_err_file);
ut_print_timestamp(lock_latest_err_file);
@@ -5629,7 +5645,7 @@ DeadlockChecker::print(const char* msg)
void
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_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
DeadlockChecker::print(const lock_t* lock)
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
if (!lock->is_table()) {
mtr_t mtr;
@@ -5678,7 +5694,7 @@ Note: "next" here means PREV for table locks.
const lock_t*
DeadlockChecker::get_next_lock(const lock_t* lock, ulint heap_no) const
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
do {
if (!lock->is_table()) {
@@ -5716,7 +5732,7 @@ due to the way the record lock has is implemented.
const lock_t*
DeadlockChecker::get_first_lock(ulint* heap_no) const
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
const lock_t* lock = m_wait_lock;
@@ -5764,7 +5780,7 @@ transaction info.
void
DeadlockChecker::notify(const lock_t* lock) const
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
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 */
trx_t *DeadlockChecker::select_victim() const
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
trx_t *lock_trx= m_wait_lock->trx;
ut_ad(m_start->lock.wait_lock);
ut_ad(lock_trx != m_start);
@@ -5839,7 +5855,7 @@ have been granted its lock by the deadlock checks.
@return nullptr if no deadlock */
inline trx_t* DeadlockChecker::search()
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
ut_ad(!m_start->mutex_is_owner());
check_trx_state(m_wait_lock->trx);
ut_ad(m_mark_start <= s_lock_mark_counter);
@@ -5954,7 +5970,7 @@ inline trx_t* DeadlockChecker::search()
void
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
or the max depth, the current trx will be
@@ -6000,7 +6016,7 @@ inline bool DeadlockChecker::check_and_resolve(trx_t *trx)
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;
if (!lock)
@@ -6080,7 +6096,7 @@ lock_update_split_and_merge(
const page_id_t l{left_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);
ut_ad(!page_rec_is_metadata(left_next_rec));
diff --git a/storage/innobase/lock/lock0prdt.cc b/storage/innobase/lock/lock0prdt.cc
index 5fdbf9c8a1b..06e42e82fcf 100644
--- a/storage/innobase/lock/lock0prdt.cc
+++ b/storage/innobase/lock/lock0prdt.cc
@@ -234,7 +234,7 @@ lock_prdt_has_lock(
attached to the new lock */
const trx_t* trx) /*!< in: transaction */
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
ut_ad((precise_mode & LOCK_MODE_MASK) == LOCK_S
|| (precise_mode & LOCK_MODE_MASK) == LOCK_X);
ut_ad(!(precise_mode & LOCK_INSERT_INTENTION));
@@ -426,7 +426,7 @@ lock_prdt_add_to_queue(
transaction mutex */
{
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(index->is_spatial());
ut_ad(!dict_index_is_online_ddl(index));
@@ -504,7 +504,7 @@ lock_prdt_insert_check_and_lock(
dberr_t err= DB_SUCCESS;
{
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
/* Because this code is invoked for a running transaction by
the thread that is serving the transaction, it is not necessary
to hold trx->mutex here. */
@@ -561,7 +561,7 @@ lock_prdt_update_parent(
lock_prdt_t* right_prdt, /*!< in: MBR on the new page */
const page_id_t page_id) /*!< in: parent page */
{
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
/* Get all locks in parent */
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 */
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,
page_id, LOCK_PREDICATE);
@@ -732,7 +732,7 @@ lock_prdt_lock(
index record, and this would not have been possible if another active
transaction had modified this secondary index record. */
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
const unsigned prdt_mode = type_mode | mode;
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
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 ulint mode = LOCK_S | LOCK_PRDT_PAGE;
@@ -862,7 +862,7 @@ lock_place_prdt_page_lock(
@return true if there is none */
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);
return !lock || trx == lock->trx;
}
@@ -877,7 +877,7 @@ lock_prdt_rec_move(
the receiving record */
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,
donator, PRDT_HEAPNO);
@@ -904,7 +904,7 @@ lock_prdt_rec_move(
void
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;
lock= next)
diff --git a/storage/innobase/row/row0ins.cc b/storage/innobase/row/row0ins.cc
index 2ba367c1752..65cb8321500 100644
--- a/storage/innobase/row/row0ins.cc
+++ b/storage/innobase/row/row0ins.cc
@@ -708,7 +708,7 @@ row_ins_foreign_trx_print(
ut_ad(!srv_read_only_mode);
{
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
n_rec_locks = trx->lock.n_rec_locks;
n_trx_locks = UT_LIST_GET_LEN(trx->lock.trx_locks);
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
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
- the mutex rank above the lock_sys_t::mutex. The query cache mutex
- has a rank just above the lock_sys_t::mutex. */
+ the mutex rank above the lock_sys.latch. The query cache mutex
+ has a rank just above the lock_sys.latch. */
row_ins_invalidate_query_cache(thr, table->name.m_name);
diff --git a/storage/innobase/row/row0mysql.cc b/storage/innobase/row/row0mysql.cc
index 13bcc2a0a09..c088d3f0994 100644
--- a/storage/innobase/row/row0mysql.cc
+++ b/storage/innobase/row/row0mysql.cc
@@ -1170,7 +1170,7 @@ row_lock_table_autoinc_for_mysql(
/* 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
- the lock mutex. */
+ lock_sys.latch. */
if (trx == table->autoinc_trx) {
return(DB_SUCCESS);
@@ -2614,7 +2614,7 @@ skip:
if (!srv_fast_shutdown && !trx_sys.any_active_transactions()) {
{
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
skip = UT_LIST_GET_LEN(table->locks) != 0;
}
if (skip) {
diff --git a/storage/innobase/row/row0vers.cc b/storage/innobase/row/row0vers.cc
index f387e72b1bb..c64a9d07131 100644
--- a/storage/innobase/row/row0vers.cc
+++ b/storage/innobase/row/row0vers.cc
@@ -1,7 +1,7 @@
/*****************************************************************************
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
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;
dict_index_t* clust_index;
- lock_sys.mutex_assert_unlocked();
+ lock_sys.assert_unlocked();
mtr_start(&mtr);
diff --git a/storage/innobase/srv/srv0srv.cc b/storage/innobase/srv/srv0srv.cc
index 39d358a7165..d75451ce791 100644
--- a/storage/innobase/srv/srv0srv.cc
+++ b/storage/innobase/srv/srv0srv.cc
@@ -456,19 +456,13 @@ lock -- semaphore;
kernel -- kernel;
query thread execution:
-(a) without lock mutex
+(a) without lock_sys.latch
reserved -- process executing in user mode;
-(b) with lock mutex reserved
+(b) with lock_sys.latch reserved
-- process executing in kernel mode;
-The server has several backgroind threads all running at the same
-priority as user threads. It periodically checks if here is anything
-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 server has several background threads all running at the same
+priority as user threads.
The threads which we call user threads serve the queries of the MySQL
server. They run at normal priority.
@@ -800,7 +794,7 @@ srv_printf_innodb_monitor(
/* Only if lock_print_info_summary proceeds correctly,
before we call the lock_print_info_all_transactions
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);
if (ret) {
@@ -813,9 +807,8 @@ srv_printf_innodb_monitor(
}
}
- /* NOTE: If we get here then we have the lock mutex. This
- function will release the lock mutex that we acquired when
- we called the lock_print_info_summary() function earlier. */
+ /* NOTE: The following function will release the lock_sys.latch
+ that lock_print_info_summary() acquired. */
lock_print_info_all_transactions(file);
@@ -1235,7 +1228,7 @@ static void srv_monitor()
if (srv_print_innodb_monitor) {
/* Reset mutex_skipped counter everytime
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 */
if (!monitor_state.last_srv_print_monitor) {
monitor_state.mutex_skipped = 0;
diff --git a/storage/innobase/trx/trx0i_s.cc b/storage/innobase/trx/trx0i_s.cc
index 4a73b02df0c..4cfc58c7ca7 100644
--- a/storage/innobase/trx/trx0i_s.cc
+++ b/storage/innobase/trx/trx0i_s.cc
@@ -156,7 +156,7 @@ struct trx_i_s_cache_t {
ha_storage_t* storage; /*!< storage for external volatile
data that may become unavailable
when we release
- lock_sys.mutex */
+ lock_sys.latch */
ulint mem_allocd; /*!< the amount of memory
allocated with mem_alloc*() */
bool is_truncated; /*!< this is true if the memory
@@ -422,7 +422,7 @@ fill_trx_row(
{
const char* s;
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
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);
- /* These are protected by both trx->mutex or lock_sys.mutex,
- or just lock_sys.mutex. For reading, it suffices to hold
- lock_sys.mutex. */
+ /* These are protected by lock_sys.latch (which we are holding)
+ and sometimes also trx->mutex. */
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
undefined */
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
/* If transaction is waiting we add the wait lock and all locks
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)
{
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
trx_i_s_cache_clear(cache);
/* Capture the state of transactions */
@@ -1240,8 +1239,8 @@ trx_i_s_cache_init(
{
/* The latching is done in the following order:
acquire trx_i_s_cache_t::rw_lock, rwlock
- acquire lock mutex
- release lock mutex
+ acquire exclusive lock_sys.latch
+ release exclusive lock_sys.latch
release trx_i_s_cache_t::rw_lock
acquire trx_i_s_cache_t::rw_lock, rdlock
release trx_i_s_cache_t::rw_lock */
diff --git a/storage/innobase/trx/trx0trx.cc b/storage/innobase/trx/trx0trx.cc
index 88b61fc9118..887c4b2206f 100644
--- a/storage/innobase/trx/trx0trx.cc
+++ b/storage/innobase/trx/trx0trx.cc
@@ -1258,7 +1258,7 @@ trx_update_mod_tables_timestamp(
table->acquire() */
dict_sys.mutex_lock();
{
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
if (!table->get_ref_count()
&& !UT_LIST_GET_LEN(table->locks)) {
dict_sys.remove(table, true);
@@ -1833,7 +1833,7 @@ state_ok:
/**********************************************************************//**
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. */
void
trx_print_latched(
@@ -1843,7 +1843,7 @@ trx_print_latched(
ulint max_query_len) /*!< in: max query length to print,
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->lock.n_rec_locks,
@@ -1853,7 +1853,7 @@ trx_print_latched(
/**********************************************************************//**
Prints info about a transaction.
-Acquires and releases lock_sys.mutex. */
+Acquires and releases lock_sys.latch. */
void
trx_print(
/*======*/
@@ -1864,7 +1864,7 @@ trx_print(
{
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_trx_locks= UT_LIST_GET_LEN(trx->lock.trx_locks);
heap_size= mem_heap_get_size(trx->lock.lock_heap);