summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-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.cc2
-rw-r--r--storage/innobase/buf/buf0buf.cc2
-rw-r--r--storage/innobase/gis/gis0sea.cc2
-rw-r--r--storage/innobase/handler/ha_innodb.cc14
-rw-r--r--storage/innobase/include/dict0mem.h14
-rw-r--r--storage/innobase/include/lock0lock.h104
-rw-r--r--storage/innobase/include/lock0priv.ic4
-rw-r--r--storage/innobase/include/lock0types.h4
-rw-r--r--storage/innobase/include/que0que.h4
-rw-r--r--storage/innobase/include/trx0sys.h6
-rw-r--r--storage/innobase/include/trx0trx.h32
-rw-r--r--storage/innobase/include/univ.i4
-rw-r--r--storage/innobase/lock/lock0iter.cc6
-rw-r--r--storage/innobase/lock/lock0lock.cc227
-rw-r--r--storage/innobase/lock/lock0prdt.cc6
-rw-r--r--storage/innobase/lock/lock0wait.cc20
-rw-r--r--storage/innobase/que/que0que.cc2
-rw-r--r--storage/innobase/row/row0ins.cc6
-rw-r--r--storage/innobase/row/row0mysql.cc2
-rw-r--r--storage/innobase/row/row0vers.cc4
-rw-r--r--storage/innobase/trx/trx0i_s.cc13
-rw-r--r--storage/innobase/trx/trx0trx.cc10
24 files changed, 277 insertions, 219 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 0ad00bfef11..5cbc0988c3a 100644
--- a/storage/innobase/btr/btr0btr.cc
+++ b/storage/innobase/btr/btr0btr.cc
@@ -3295,7 +3295,7 @@ btr_lift_page_up(
if (!dict_table_is_locking_disabled(index->table)) {
/* Free predicate page locks on the block */
if (dict_index_is_spatial(index)) {
- LockMutexGuard g;
+ LockGuard guard(block->page.id());
lock_prdt_page_free_from_discard(
block, &lock_sys.prdt_page_hash);
}
@@ -3547,7 +3547,7 @@ retry:
}
/* No GAP lock needs to be worrying about */
- LockMutexGuard g;
+ LockGuard g(block->page.id());
lock_prdt_page_free_from_discard(
block, &lock_sys.prdt_page_hash);
lock_rec_free_all_from_discard_page(block);
@@ -3699,7 +3699,7 @@ retry:
offsets2, offsets,
merge_page, mtr);
}
- LockMutexGuard g;
+ LockGuard g(block->page.id());
lock_prdt_page_free_from_discard(
block, &lock_sys.prdt_page_hash);
lock_rec_free_all_from_discard_page(block);
diff --git a/storage/innobase/btr/btr0cur.cc b/storage/innobase/btr/btr0cur.cc
index 47196e1228d..8471eadf295 100644
--- a/storage/innobase/btr/btr0cur.cc
+++ b/storage/innobase/btr/btr0cur.cc
@@ -1998,7 +1998,7 @@ retry_page_get:
lock_prdt_t prdt;
{
- LockMutexGuard g;
+ LockGuard g(block->page.id());
lock_init_prdt_from_mbr(
&prdt, &cursor->rtr_info->mbr, mode,
trx->lock.lock_heap);
diff --git a/storage/innobase/buf/buf0buf.cc b/storage/innobase/buf/buf0buf.cc
index b1561174cd8..65171f7403f 100644
--- a/storage/innobase/buf/buf0buf.cc
+++ b/storage/innobase/buf/buf0buf.cc
@@ -2039,7 +2039,7 @@ withdraw_retry:
find_interesting_trx f
{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 9320fcf88fa..59f8105fcfc 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);
diff --git a/storage/innobase/handler/ha_innodb.cc b/storage/innobase/handler/ha_innodb.cc
index 619273f1737..8bf10638726 100644
--- a/storage/innobase/handler/ha_innodb.cc
+++ b/storage/innobase/handler/ha_innodb.cc
@@ -538,7 +538,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;
@@ -578,7 +577,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),
@@ -595,6 +593,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"
@@ -608,6 +607,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 */
@@ -2291,7 +2291,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);
@@ -2680,7 +2680,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;
}
@@ -4450,7 +4450,7 @@ static void innobase_kill_query(handlerton*, THD *thd, enum thd_kill_levels)
Also, BF thread should own trx mutex for the victim. */
DBUG_VOID_RETURN;
#endif /* WITH_WSREP */
- LockMutexGuard g;
+ LockMutexGuard g(SRW_LOCK_CALL);
trx_sys.trx_list.freeze();
mysql_mutex_lock(&lock_sys.wait_mutex);
trx->mutex.wr_lock();
@@ -17991,7 +17991,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();
DBUG_ENTER("wsrep_innobase_kill_one_trx");
@@ -18094,7 +18094,7 @@ wsrep_abort_transaction(
wsrep_thd_transaction_state_str(victim_thd));
if (victim_trx) {
- LockMutexGuard g;
+ LockMutexGuard g(SRW_LOCK_CALL);
victim_trx->mutex.wr_lock();
int rcode= wsrep_innobase_kill_one_trx(bf_thd,
victim_trx, signal);
diff --git a/storage/innobase/include/dict0mem.h b/storage/innobase/include/dict0mem.h
index 8ad293f6a9a..b3d782e3caa 100644
--- a/storage/innobase/include/dict0mem.h
+++ b/storage/innobase/include/dict0mem.h
@@ -2,7 +2,7 @@
Copyright (c) 1996, 2017, Oracle and/or its affiliates. All Rights Reserved.
Copyright (c) 2012, Facebook Inc.
-Copyright (c) 2013, 2020, MariaDB Corporation.
+Copyright (c) 2013, 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
@@ -2248,7 +2248,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. */
@@ -2268,14 +2268,14 @@ public:
/** This counter is used to track the number of granted and pending
autoinc locks on this table. This value is set after acquiring the
- lock_sys_t::mutex but we peek the contents to determine whether other
+ lock_sys.latch but we peek the contents to determine whether other
transactions have acquired the AUTOINC lock or not. Of course only one
transaction can be granted the lock but there can be multiple
waiters. */
ulong n_waiting_or_granted_auto_inc_locks;
/** 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;
/* @} */
@@ -2290,8 +2290,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. */
+ Atomic_counter<ulint> n_rec_locks;
private:
/** Count of how many handles are opened to this table. Dropping of the
@@ -2300,7 +2300,7 @@ private:
Atomic_counter<uint32_t> n_ref_count;
public:
- /** List of locks on the table. Protected by lock_sys.mutex. */
+ /** List of locks on the table. Protected by lock_sys.latch. */
table_lock_list_t locks;
/** Timestamp of the last modification of this table. */
diff --git a/storage/innobase/include/lock0lock.h b/storage/innobase/include/lock0lock.h
index cc7657fbcbe..80f18790547 100644
--- a/storage/innobase/include/lock0lock.h
+++ b/storage/innobase/include/lock0lock.h
@@ -537,7 +537,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(
/*=========================*/
@@ -682,7 +682,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;
@@ -705,33 +711,56 @@ 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()
+ {
+ 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()
+ {
+ 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(!writer.load(std::memory_order_relaxed));
+ 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); }
- /** Assert that a page shared is exclusively latched */
- void assert_locked(const page_id_t) { mutex_assert_locked(); }
-
- /** Wait for a lock to be granted */
- void wait_lock(lock_t **lock, mysql_cond_t *cond)
- { while (*lock) mysql_cond_wait(cond, &mutex); }
+ /** Assert that wr_lock() has been invoked by this thread */
+ void assert_locked() const
+ { ut_ad(writer.load(std::memory_order_relaxed) == os_thread_get_curr_id()); }
+ /** Assert that wr_lock() has not been invoked by this thread */
+ void assert_unlocked() const
+ { ut_ad(writer.load(std::memory_order_relaxed) != os_thread_get_curr_id()); }
+ /** Assert that a page shard is exclusively latched by this thread */
+ void assert_locked(const page_id_t) { assert_locked(); }
/**
Creates the lock system at database start.
@@ -754,7 +783,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
@@ -796,18 +825,27 @@ 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(); }
};
-/** lock_sys.mutex guard for a page_id_t shard */
+/** lock_sys.latch guard for a page_id_t shard */
struct LockGuard
{
- LockGuard(const page_id_t) { lock_sys.mutex_lock(); }
- ~LockGuard() { lock_sys.mutex_unlock(); }
+ LockGuard(const page_id_t) { lock_sys.wr_lock(SRW_LOCK_CALL); }
+ ~LockGuard() { lock_sys.wr_unlock(); }
+};
+
+/** lock_sys.latch guard for 2 page_id_t shards */
+struct LockMultiGuard
+{
+ LockMultiGuard(const page_id_t, const page_id_t)
+ { lock_sys.wr_lock(SRW_LOCK_CALL); }
+ ~LockMultiGuard() { lock_sys.wr_unlock(); }
};
/*********************************************************************//**
diff --git a/storage/innobase/include/lock0priv.ic b/storage/innobase/include/lock0priv.ic
index b2ee4d1121c..d33ab6edcd4 100644
--- a/storage/innobase/include/lock0priv.ic
+++ b/storage/innobase/include/lock0priv.ic
@@ -91,7 +91,7 @@ lock_rec_set_nth_bit(
ulint byte_index;
ulint bit_index;
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
ut_ad(lock);
ut_ad(lock_get_type_low(lock) == LOCK_REC);
ut_ad(i < lock->un_member.rec_lock.n_bits);
@@ -189,7 +189,7 @@ lock_rec_get_next_on_page_const(
/*============================*/
const lock_t* lock) /*!< in: a record lock */
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
ut_ad(lock_get_type_low(lock) == LOCK_REC);
const page_id_t page_id(lock->un_member.rec_lock.page_id);
diff --git a/storage/innobase/include/lock0types.h b/storage/innobase/include/lock0types.h
index 23307375426..a0473c18e30 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
@@ -176,7 +176,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/que0que.h b/storage/innobase/include/que0que.h
index 5386facb793..eb44b131989 100644
--- a/storage/innobase/include/que0que.h
+++ b/storage/innobase/include/que0que.h
@@ -1,7 +1,7 @@
/*****************************************************************************
Copyright (c) 1996, 2016, 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
@@ -98,7 +98,7 @@ que_graph_free(
afterwards! */
/**********************************************************************//**
Stops a query thread if graph or trx is in a state requiring it. The
-conditions are tested in the order (1) graph, (2) trx. The lock_sys_t::mutex
+conditions are tested in the order (1) graph, (2) trx. The lock_sys.latch
has to be reserved.
@return TRUE if stopped */
ibool
diff --git a/storage/innobase/include/trx0sys.h b/storage/innobase/include/trx0sys.h
index f99ef4eb7ce..0fb73f39e10 100644
--- a/storage/innobase/include/trx0sys.h
+++ b/storage/innobase/include/trx0sys.h
@@ -1,7 +1,7 @@
/*****************************************************************************
Copyright (c) 1996, 2016, 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
@@ -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 3eaf131d9f4..3f69a5fb204 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(
/*======*/
@@ -456,7 +456,7 @@ granted the lock it was waiting for, which implies that the state can be changed
asynchronously.
All these operations take place within the context of locking. Therefore state
-changes within the locking code must acquire both the lock mutex and the
+changes within the locking code must acquire both lock_sys.latch and the
trx->mutex when changing trx->lock.que_state to TRX_QUE_LOCK_WAIT or
trx->lock.wait_lock to non-NULL but when the lock wait ends it is sufficient
to only acquire the trx->mutex.
@@ -464,7 +464,7 @@ To query the state either of the mutexes is sufficient within the locking
code and no mutex is required when the query thread is no longer waiting. */
/** 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 {
#ifdef UNIV_DEBUG
/** number of active query threads; at most 1, except for the
@@ -476,7 +476,7 @@ struct trx_lock_t {
TRX_QUE_LOCK_WAIT, ... */
/** Lock request being waited for, in que_state==TRX_QUE_LOCK_WAIT.
- Set to nonnull when holding both lock_sys.mutex and trx->mutex,
+ Set to nonnull when holding both lock_sys.latch 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;
@@ -500,7 +500,7 @@ struct trx_lock_t {
trx that is in QUE_THR_LOCK_WAIT
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. */
#ifdef WITH_WSREP
@@ -524,12 +524,12 @@ struct trx_lock_t {
unsigned table_cached;
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 */
@@ -546,10 +546,10 @@ struct trx_lock_t {
check for this cancel of a transaction's
locks and avoid reacquiring the trx
mutex to prevent recursive deadlocks.
- Protected by both the lock sys mutex
+ Protected by both lock_sys.latch
and the trx_t::mutex. */
- /** number of record locks; writes are protected by lock_sys.mutex */
- ulint n_rec_locks;
+ /** number of record locks; writers use LockGuard or LockMultiGuard */
+ Atomic_counter<ulint> n_rec_locks;
};
/** Logical first modification time of a table in a transaction */
@@ -651,7 +651,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.
@@ -659,7 +659,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 {
@@ -709,7 +709,7 @@ private:
public:
/** mutex protecting state and some of lock
- (some are protected by lock_sys.mutex) */
+ (some are protected by lock_sys.latch) */
srw_mutex mutex;
trx_id_t id; /*!< transaction id */
@@ -788,7 +788,7 @@ public:
transaction, or NULL if not yet set */
trx_lock_t lock; /*!< Information about the transaction
locks and state. Protected by
- lock_sys.mutex (insertions also
+ lock_sys.latch (insertions also
by trx_t::mutex). */
/* These fields are not protected by any mutex. */
@@ -928,7 +928,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 d082aef07c9..e634233435d 100644
--- a/storage/innobase/lock/lock0iter.cc
+++ b/storage/innobase/lock/lock0iter.cc
@@ -1,7 +1,7 @@
/*****************************************************************************
Copyright (c) 2007, 2014, Oracle and/or its affiliates. All Rights Reserved.
-Copyright (c) 2020, MariaDB Corporation.
+Copyright (c) 2020, 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
@@ -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();
iter->current_lock = lock;
@@ -84,7 +84,7 @@ lock_queue_iterator_get_prev(
{
const lock_t* prev_lock;
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
switch (lock_get_type_low(iter->current_lock)) {
case LOCK_REC:
diff --git a/storage/innobase/lock/lock0lock.cc b/storage/innobase/lock/lock0lock.cc
index e0f1a1d7998..cbeb754e65f 100644
--- a/storage/innobase/lock/lock0lock.cc
+++ b/storage/innobase/lock/lock0lock.cc
@@ -245,7 +245,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. */
@@ -440,7 +440,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);
@@ -454,15 +454,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
@@ -482,7 +504,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);
@@ -520,7 +542,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;
@@ -861,7 +883,7 @@ lock_rec_get_prev(
lock_t* lock;
lock_t* found_lock = NULL;
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
ut_ad(lock_get_type_low(in_lock) == LOCK_REC);
for (lock = lock_sys.get_first(*lock_hash_get(in_lock->type_mode),
@@ -899,7 +921,7 @@ lock_rec_has_expl(
lock_t* lock;
const page_id_t id{block->page.id()};
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked(id);
ut_ad((precise_mode & LOCK_MODE_MASK) == LOCK_S
|| (precise_mode & LOCK_MODE_MASK) == LOCK_X);
ut_ad(!(precise_mode & LOCK_INSERT_INTENTION));
@@ -957,7 +979,7 @@ lock_rec_other_has_expl_req(
const page_id_t id{block->page.id()};
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked(id);
for (lock_t* lock = lock_rec_get_first(&lock_sys.rec_hash,
id, heap_no);
@@ -985,7 +1007,7 @@ wsrep_kill_victim(
const trx_t * const trx,
const lock_t *lock)
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
/* quit for native mysql */
if (!trx->is_wsrep()) return;
@@ -1059,7 +1081,7 @@ lock_rec_other_has_conflicting(
const trx_t* trx) /*!< in: our transaction */
{
const page_id_t id{block->page.id()};
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked(id);
bool is_supremum = (heap_no == PAGE_HEAP_NO_SUPREMUM);
@@ -1105,7 +1127,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));
@@ -1138,7 +1160,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(
/*=========================*/
@@ -1147,7 +1169,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;
@@ -1197,7 +1219,7 @@ wsrep_print_wait_locks(
@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();
ut_ad(lock_get_wait(lock));
ut_ad(!lock->trx->lock.wait_lock || lock->trx->lock.wait_lock == lock);
lock->trx->lock.wait_lock= nullptr;
@@ -1214,7 +1236,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();
trx->lock.wait_lock= lock;
lock->type_mode|= LOCK_WAIT;
@@ -1250,7 +1272,7 @@ lock_rec_create_low(
ulint n_bits;
ulint n_bytes;
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
ut_ad(dict_index_is_clust(index) || !dict_index_is_online_ddl(index));
#ifdef UNIV_DEBUG
@@ -1445,7 +1467,7 @@ lock_rec_enqueue_waiting(
que_thr_t* thr,
lock_prdt_t* prdt)
{
- lock_sys.mutex_assert_locked();
+ ut_d(lock_sys.assert_locked(block->page.id()));
ut_ad(!srv_read_only_mode);
ut_ad(dict_index_is_clust(index) || !dict_index_is_online_ddl(index));
@@ -1537,7 +1559,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;
@@ -1578,7 +1600,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(index->is_primary()
|| dict_index_get_online_status(index) != ONLINE_INDEX_CREATION);
#ifdef UNIV_DEBUG
@@ -1707,7 +1729,7 @@ lock_rec_lock(
ut_ad(dict_index_is_clust(index) || !dict_index_is_online_ddl(index));
DBUG_EXECUTE_IF("innodb_report_deadlock", return DB_DEADLOCK;);
MONITOR_ATOMIC_INC(MONITOR_NUM_RECLOCK_REQ);
- LockMutexGuard g;
+ LockMutexGuard g(SRW_LOCK_CALL);
ut_ad((LOCK_MODE_MASK & mode) != LOCK_S ||
lock_table_has(trx, index->table, LOCK_IS));
ut_ad((LOCK_MODE_MASK & mode) != LOCK_X ||
@@ -1799,7 +1821,7 @@ lock_rec_has_to_wait_in_queue(
ulint bit_offset;
ut_ad(wait_lock);
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
ut_ad(lock_get_wait(wait_lock));
ut_ad(lock_get_type_low(wait_lock) == LOCK_REC);
@@ -1827,7 +1849,7 @@ lock_rec_has_to_wait_in_queue(
/** 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();
mysql_mutex_assert_owner(&lock_sys.wait_mutex);
lock_reset_lock_and_trx_wait(lock);
auto mutex= &lock->trx->mutex;
@@ -1898,7 +1920,7 @@ static void lock_rec_dequeue_from_page(lock_t* in_lock)
{
hash_table_t* lock_hash;
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
mysql_mutex_assert_owner(&lock_sys.wait_mutex);
ut_ad(lock_get_type_low(in_lock) == LOCK_REC);
/* We may or may not be holding in_lock->trx->mutex here. */
@@ -1952,7 +1974,7 @@ lock_rec_discard(
{
trx_lock_t* trx_lock;
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
ut_ad(lock_get_type_low(in_lock) == LOCK_REC);
trx_lock = &in_lock->trx->lock;
@@ -2017,7 +2039,7 @@ lock_rec_reset_and_release_wait_low(
ulint heap_no)/*!< in: heap number of record */
{
const page_id_t page_id{block->page.id()};
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked(page_id);
for (lock_t *lock= lock_rec_get_first(hash, page_id, heap_no); lock;
lock= lock_rec_get_next(heap_no, lock))
@@ -2068,7 +2090,7 @@ lock_rec_inherit_to_gap(
donating record */
{
const page_id_t id{block->page.id()};
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked(id);
/* At READ UNCOMMITTED or READ COMMITTED isolation level,
we do not want locks set
@@ -2148,7 +2170,8 @@ lock_rec_move_low(
{
const page_id_t donator_id{donator->page.id()};
- lock_sys.mutex_assert_locked();
+ ut_d(lock_sys.assert_locked(receiver->page.id()));
+ lock_sys.assert_locked(donator_id);
/* If the lock is predicate lock, it resides on INFIMUM record */
ut_ad(!lock_rec_get_first(lock_hash, receiver->page.id(),
@@ -2386,7 +2409,7 @@ lock_move_rec_list_end(
{
const page_id_t id{block->page.id()};
- LockMutexGuard g;
+ LockMultiGuard g{id, new_block->page.id()};
/* Note: when we move locks from record to record, waiting locks
and possible granted gap type locks behind them are enqueued in
@@ -2504,7 +2527,7 @@ lock_move_rec_list_start(
{
const page_id_t id{block->page.id()};
- LockMutexGuard g;
+ LockMultiGuard g{id, new_block->page.id()};
for (lock_t *lock= lock_sys.get_first(id); lock;
lock= lock_rec_get_next_on_page(lock))
@@ -2673,7 +2696,7 @@ lock_update_split_right(
{
ulint heap_no = lock_get_min_heap_no(right_block);
- LockMutexGuard g;
+ LockMutexGuard g(SRW_LOCK_CALL);
/* Move the locks on the supremum of the left page to the supremum
of the right page */
@@ -2705,7 +2728,7 @@ lock_update_merge_right(
{
ut_ad(!page_rec_is_metadata(orig_succ));
- 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
@@ -2741,7 +2764,7 @@ lock_update_root_raise(
const buf_block_t* block, /*!< in: index page to which copied */
const buf_block_t* root) /*!< in: root page */
{
- LockMutexGuard g;
+ LockMutexGuard g(SRW_LOCK_CALL);
/* Move the locks on the supremum of the root to the supremum
of block */
@@ -2761,7 +2784,7 @@ lock_update_copy_and_discard(
const buf_block_t* block) /*!< in: index page;
NOT the root! */
{
- LockMutexGuard g;
+ LockMutexGuard g(SRW_LOCK_CALL);
/* Move the locks on the supremum of the old page to the supremum
of new_page */
@@ -2781,7 +2804,7 @@ lock_update_split_left(
{
ulint heap_no = lock_get_min_heap_no(right_block);
- 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 */
@@ -2807,7 +2830,7 @@ lock_update_merge_left(
ut_ad(left_block->frame == page_align(orig_pred));
- LockMutexGuard g;
+ LockMutexGuard g(SRW_LOCK_CALL);
left_next_rec = page_rec_get_next_const(orig_pred);
@@ -2857,7 +2880,7 @@ lock_rec_reset_and_inherit_gap_locks(
ulint heap_no) /*!< in: heap_no of the
donating record */
{
- LockMutexGuard g;
+ LockMutexGuard g(SRW_LOCK_CALL);
lock_rec_reset_and_release_wait(heir_block, heir_heap_no);
@@ -2881,7 +2904,7 @@ lock_update_discard(
ulint heap_no;
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));
@@ -2987,7 +3010,7 @@ lock_update_delete(
FALSE));
}
- LockMutexGuard g;
+ LockMutexGuard g(SRW_LOCK_CALL);
/* Let the next record inherit the locks from rec, in gap mode */
@@ -3019,7 +3042,7 @@ lock_rec_store_on_page_infimum(
ut_ad(block->frame == page_align(rec));
- LockMutexGuard g;
+ LockMutexGuard g(SRW_LOCK_CALL);
lock_rec_move(block, block, PAGE_HEAP_NO_INFIMUM, heap_no);
}
@@ -3041,7 +3064,7 @@ lock_rec_restore_from_page_infimum(
{
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);
}
@@ -3077,7 +3100,7 @@ lock_table_create(
lock_t* lock;
ut_ad(table && trx);
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
check_trx_state(trx);
@@ -3178,7 +3201,7 @@ lock_table_pop_autoinc_locks(
/*=========================*/
trx_t* trx) /*!< in/out: transaction that owns the AUTOINC locks */
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
ut_ad(!ib_vector_is_empty(trx->autoinc_locks));
/* Skip any gaps, gaps are NULL lock entries in the
@@ -3206,7 +3229,7 @@ lock_table_remove_autoinc_lock(
lock_t* autoinc_lock;
lint i = ib_vector_size(trx->autoinc_locks) - 1;
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
ut_ad(lock_get_mode(lock) == LOCK_AUTO_INC);
ut_ad(lock_get_type_low(lock) & LOCK_TABLE);
ut_ad(!ib_vector_is_empty(trx->autoinc_locks));
@@ -3258,7 +3281,7 @@ lock_table_remove_low(
trx_t* trx;
dict_table_t* table;
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
trx = lock->trx;
table = lock->un_member.tab_lock.table;
@@ -3321,7 +3344,7 @@ lock_table_enqueue_waiting(
trx_t* trx;
lock_t* lock;
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
ut_ad(!srv_read_only_mode);
trx = thr_get_trx(thr);
@@ -3402,7 +3425,7 @@ lock_table_other_has_incompatible(
{
lock_t* lock;
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
for (lock = UT_LIST_GET_LAST(table->locks);
lock != NULL;
@@ -3488,7 +3511,7 @@ lock_table(
trx_set_rw_mode(trx);
}
- 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. */
@@ -3537,7 +3560,7 @@ lock_table_ix_resurrect(
auto mutex= &trx->mutex;
- 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. */
@@ -3562,7 +3585,7 @@ lock_table_has_to_wait_in_queue(
const dict_table_t* table;
const lock_t* lock;
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
ut_ad(lock_get_wait(wait_lock));
table = wait_lock->un_member.tab_lock.table;
@@ -3592,7 +3615,7 @@ lock_table_dequeue(
behind will get their lock requests granted, if
they are now qualified to it */
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
mysql_mutex_assert_owner(&lock_sys.wait_mutex);
ut_a(lock_get_type_low(in_lock) == LOCK_TABLE);
@@ -3802,7 +3825,7 @@ void lock_release(trx_t* trx)
ulint count = 0;
trx_id_t max_trx_id = trx_sys.get_max_trx_id();
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
mysql_mutex_lock(&lock_sys.wait_mutex);
for (lock_t* lock = UT_LIST_GET_LAST(trx->lock.trx_locks);
@@ -3836,10 +3859,10 @@ 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();
mysql_mutex_unlock(&lock_sys.wait_mutex);
count = 0;
- lock_sys.mutex_lock();
+ lock_sys.wr_lock(SRW_LOCK_CALL);
mysql_mutex_lock(&lock_sys.wait_mutex);
}
@@ -3864,7 +3887,7 @@ lock_trx_table_locks_remove(
{
trx_t* trx = lock_to_remove->trx;
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
/* It is safe to read this because we are holding the lock mutex */
const bool have_mutex = trx->lock.cancel;
@@ -3904,7 +3927,7 @@ static
void
lock_table_print(FILE* file, const lock_t* lock)
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
ut_a(lock_get_type_low(lock) == LOCK_TABLE);
fputs("TABLE LOCK table ", file);
@@ -3942,7 +3965,7 @@ lock_table_print(FILE* file, const lock_t* lock)
@param[in,out] mtr mini-transaction for accessing the record */
static void lock_rec_print(FILE* file, const lock_t* lock, mtr_t& mtr)
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
ut_a(lock_get_type_low(lock) == LOCK_REC);
const page_id_t page_id(lock->un_member.rec_lock.page_id);
@@ -4039,7 +4062,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;
@@ -4072,8 +4095,8 @@ lock_print_info_summary(
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.mutex_trylock()) */ {
fputs("FAIL TO OBTAIN LOCK MUTEX,"
" SKIP LOCK INFO PRINTING\n", file);
return(FALSE);
@@ -4216,7 +4239,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());
}
@@ -4269,14 +4292,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.wr_lock();
check_trx_state(lock->trx);
@@ -4329,10 +4352,10 @@ lock_rec_queue_validate(
const page_id_t id{block->page.id()};
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)) {
@@ -4354,14 +4377,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)
@@ -4568,7 +4591,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));
@@ -4635,7 +4658,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)
{
@@ -4658,7 +4681,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);
@@ -4827,7 +4850,7 @@ lock_rec_convert_impl_to_expl_for_trx(
DEBUG_SYNC_C("before_lock_rec_convert_impl_to_expl_for_trx");
{
- LockMutexGuard g;
+ LockGuard g(block->page.id());
trx->mutex.wr_lock();
ut_ad(!trx_state_eq(trx, TRX_STATE_NOT_STARTED));
@@ -4898,7 +4921,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);
@@ -4942,7 +4965,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));
@@ -5326,7 +5349,7 @@ lock_release_autoinc_last_lock(
ulint last;
lock_t* lock;
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
ut_a(!ib_vector_is_empty(autoinc_locks));
/* The lock to be release must be the last lock acquired. */
@@ -5368,7 +5391,7 @@ lock_release_autoinc_locks(
/*=======================*/
trx_t* trx) /*!< in/out: transaction */
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
mysql_mutex_assert_owner(&lock_sys.wait_mutex);
/* If this is invoked for a running transaction by the thread
that is serving the transaction, then it is not necessary to
@@ -5499,7 +5522,7 @@ lock_cancel_waiting_and_release(
{
que_thr_t* thr;
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
mysql_mutex_assert_owner(&lock_sys.wait_mutex);
lock->trx->lock.cancel = true;
@@ -5544,7 +5567,7 @@ lock_unlock_table_autoinc(
/*======================*/
trx_t* trx) /*!< in/out: transaction */
{
- lock_sys.mutex_assert_unlocked();
+ lock_sys.assert_unlocked();
ut_ad(!trx->lock.wait_lock);
/* This can be invoked on NOT_STARTED, ACTIVE, PREPARED,
@@ -5558,7 +5581,7 @@ lock_unlock_table_autoinc(
necessary to hold trx->mutex here. */
if (lock_trx_holds_autoinc_locks(trx)) {
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
mysql_mutex_lock(&lock_sys.wait_mutex);
lock_release_autoinc_locks(trx);
mysql_mutex_unlock(&lock_sys.wait_mutex);
@@ -5567,7 +5590,7 @@ lock_unlock_table_autoinc(
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);
if (trx->lock.was_chosen_as_deadlock_victim)
@@ -5597,7 +5620,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.wr_lock();
err= lock_trx_handle_wait_low(trx);
@@ -5618,7 +5641,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)
{
@@ -5658,7 +5681,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)
@@ -5701,7 +5724,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();
@@ -5764,7 +5787,7 @@ lock_trx_has_expl_x_lock(
{
ut_ad(heap_no > PAGE_HEAP_NO_SUPREMUM);
- LockMutexGuard g;
+ LockMutexGuard g{SRW_LOCK_CALL};
ut_ad(lock_table_has(trx, table, LOCK_IX));
ut_ad(lock_table_has(trx, table, LOCK_X)
|| lock_rec_has_expl(LOCK_X | LOCK_REC_NOT_GAP, block, heap_no,
@@ -5779,7 +5802,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);
@@ -5808,7 +5831,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);
@@ -5828,7 +5851,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_get_type_low(lock) == LOCK_REC) {
mtr_t mtr;
@@ -5857,7 +5880,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_get_type_low(lock) == LOCK_REC) {
@@ -5897,7 +5920,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;
@@ -5946,7 +5969,7 @@ transaction info.
void
DeadlockChecker::notify(const lock_t* lock) const
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
start_print();
@@ -5994,7 +6017,7 @@ inline bool trx_weight_ge(const trx_t *a, const trx_t *b)
const trx_t*
DeadlockChecker::select_victim() const
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
ut_ad(m_start->lock.wait_lock != 0);
ut_ad(m_wait_lock->trx != m_start);
@@ -6024,7 +6047,7 @@ have been granted its lock by the deadlock checks.
const trx_t*
DeadlockChecker::search()
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
ut_ad(m_start != NULL);
ut_ad(m_wait_lock != NULL);
@@ -6134,7 +6157,7 @@ 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
@@ -6157,7 +6180,7 @@ DeadlockChecker::rollback_print(const trx_t* trx, const lock_t* lock)
void
DeadlockChecker::trx_rollback()
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
trx_t* trx = m_wait_lock->trx;
@@ -6193,7 +6216,7 @@ or there is no deadlock (any more) */
const trx_t*
DeadlockChecker::check_and_resolve(const lock_t* lock, trx_t* trx)
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
check_trx_state(trx);
ut_ad(!srv_read_only_mode);
@@ -6288,7 +6311,7 @@ lock_update_split_and_merge(
ut_ad(page_is_leaf(right_block->frame));
ut_ad(page_align(orig_pred) == left_block->frame);
- LockMutexGuard g;
+ LockMultiGuard g(left_block->page.id(), right_block->page.id());
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 905283798d5..99c3ae35f80 100644
--- a/storage/innobase/lock/lock0prdt.cc
+++ b/storage/innobase/lock/lock0prdt.cc
@@ -582,7 +582,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);
@@ -681,7 +681,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;
+ LockMultiGuard g(page_id, new_block->page.id());
lock_prdt_update_split_low(new_block, prdt, new_prdt,
page_id, LOCK_PREDICATE);
@@ -901,7 +901,7 @@ lock_prdt_rec_move(
the donating record */
{
const page_id_t donator_id{donator->page.id()};
- LockMutexGuard g;
+ LockMultiGuard g(receiver->page.id(), donator_id);
for (lock_t *lock = lock_rec_get_first(&lock_sys.prdt_hash,
donator_id, PRDT_HEAPNO);
diff --git a/storage/innobase/lock/lock0wait.cc b/storage/innobase/lock/lock0wait.cc
index 2ec411e70c7..8925fb8e2a2 100644
--- a/storage/innobase/lock/lock0wait.cc
+++ b/storage/innobase/lock/lock0wait.cc
@@ -42,7 +42,7 @@ Created 25/5/2010 Sunny Bains
check if lock timeout was for priority thread,
as a side effect trigger lock monitor
@param[in] trx transaction owning the lock
-@param[in] locked true if trx and lock_sys.mutex is ownd
+@param[in] locked true if trx and lock_sys.latch is held
@return false for regular lock timeout */
static
bool
@@ -55,15 +55,11 @@ 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) {
- lock_sys.mutex_lock();
- }
-
- lock_sys.mutex_assert_locked();
-
- trx_print_latched(stderr, trx, 3000);
-
- if (!locked) {
- lock_sys.mutex_unlock();
+ LockMutexGuard{SRW_LOCK_CALL};
+ trx_print_latched(stderr, trx, 3000);
+ } else {
+ lock_sys.assert_locked();
+ trx_print_latched(stderr, trx, 3000);
}
srv_print_innodb_monitor = TRUE;
@@ -211,7 +207,7 @@ dberr_t lock_wait(que_thr_t *thr)
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)
{
@@ -235,7 +231,7 @@ lock_wait_release_thread_if_suspended(
que_thr_t* thr) /*!< in: query thread associated with the
user OS thread */
{
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
mysql_mutex_assert_owner(&lock_sys.wait_mutex);
trx_t *trx= thr_get_trx(thr);
if (trx->lock.was_chosen_as_deadlock_victim)
diff --git a/storage/innobase/que/que0que.cc b/storage/innobase/que/que0que.cc
index 183288b2b72..b9566774af2 100644
--- a/storage/innobase/que/que0que.cc
+++ b/storage/innobase/que/que0que.cc
@@ -182,7 +182,7 @@ que_thr_end_lock_wait(
{
que_thr_t* thr;
- lock_sys.mutex_assert_locked();
+ lock_sys.assert_locked();
mysql_mutex_assert_owner(&lock_sys.wait_mutex);
thr = trx->lock.wait_thr;
diff --git a/storage/innobase/row/row0ins.cc b/storage/innobase/row/row0ins.cc
index 7e0a209a4ed..7a6026a78c9 100644
--- a/storage/innobase/row/row0ins.cc
+++ b/storage/innobase/row/row0ins.cc
@@ -705,7 +705,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);
@@ -1016,8 +1016,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 82b6e3f0644..1a7d25a9dea 100644
--- a/storage/innobase/row/row0mysql.cc
+++ b/storage/innobase/row/row0mysql.cc
@@ -2657,7 +2657,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 b0523fa141d..2d47fe2d8a4 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/trx/trx0i_s.cc b/storage/innobase/trx/trx0i_s.cc
index 590c031ed58..893f0e06b6d 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();
row->trx_id = trx_get_id_for_print(trx);
row->trx_started = trx->start_time;
@@ -483,9 +483,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);
@@ -1049,7 +1048,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. */
@@ -1197,7 +1196,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 */
diff --git a/storage/innobase/trx/trx0trx.cc b/storage/innobase/trx/trx0trx.cc
index 1a363c94654..7ff7215bd12 100644
--- a/storage/innobase/trx/trx0trx.cc
+++ b/storage/innobase/trx/trx0trx.cc
@@ -1269,7 +1269,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);
@@ -1862,7 +1862,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(
@@ -1872,7 +1872,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,
@@ -1882,7 +1882,7 @@ trx_print_latched(
/**********************************************************************//**
Prints info about a transaction.
-Acquires and releases lock_sys.mutex. */
+Acquires and releases lock_sys.latch. */
void
trx_print(
/*======*/
@@ -1893,7 +1893,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);