The Shared/Update/Exclusive locks that were introduced in
commit 03ca6495df (MDEV-24142)
did not work correctly when a futex-like system call interface
was not available.
On all tested implementations (IBM AIX as well as FreeBSD and GNU/Linux
with the futex interface artificially disabled), the old implementation
would cause hangs in some SPATIAL INDEX tests (innodb_gis suite).
On FreeBSD, a hang was also observed in an encryption test.
We will simply emulate the futex system calls with a single mutex
and two condition variables, one for each wait queue. The condition
variables basically shadow the lock words and are used as wait queues,
just like the futex system calls would be.
The storage overhead of ssux_lock_impl will be increased by 32 bits
when using SUX_LOCK_GENERIC. Compared to the futex-based implementation,
the SUX_LOCK_GENERIC implementation has an overhead of
sizeof(pthread_mutex_t)+2*sizeof(pthread_cond_t).
rw_lock: Remove all SUX_LOCK_GENERIC extensions.
pthread_mutex_wrapper: A simple wrapper of pthread_mutex that
implements srw_spin_mutex and srw_mutex for SUX_LOCK_GENERIC.
srw_mutex_impl: Define this also for SUX_LOCK_GENERIC, but in
that case add the fields mutex, cond.
ssux_lock_impl: Define for SUX_LOCK_GENERIC with a minimal difference:
adding readers_cond.
Per https://gcc.gnu.org/onlinedocs/gcc/PowerPC-Hardware-Transactional-Memory-Built-in-Functions.html
The .. high level HTM interface .. is common between PowerPC and S/390
Reimplemented the transactional_lock_enabled() detection mechanism for
s390x and POWER based on SIGILL. This also gives non-Linux based unixes
the ability to use HTM. The implementation is based off openssl.
(ref:
1c0eede982/crypto/s390xcap.c (L104))
The other ppc64{,le} problems with getauxvec based detection:
* Checking PPC_FEATURE2_HTM_NOSC not needed as we do not do syscalls while
in a transactional state.
* As we don't use, and never should use PPC_FEATURE2_HTM_NO_SUSPEND,
or do syscalls while in transactional state, don't test it.
From: https://www.kernel.org/doc/html/v5.4/powerpc/syscall64-abi.html#transactional-memory
S390x high level __builtin_tbegin functions in the htmxlintrin.h are not
inline. This header file can be included once in the entire set of sources for
a linked target, otherwise duplicate symbols occur. While we could use inline
xabort/xend functions using the low level interface, we keep this the same as
ppc64 for simplicity.
SLES-15, gcc-7, appeared to want everything that included the htmlxlintrin to
be compiled with -mhtm otherwise the __builtin_t{func} where not defined
(in addition to a #ifdef __HTM__ #error). Debian sid gcc-11.2 wanted the same
on ppc64le/ppc64. In general we want to avoid a wide spread use of architecture
cflags as it makes justifications for selective optimizations easier.
(ref: https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=1006702)
There is only a very small range of gcc compiler versions
that allow the built_{htm} functions to be defined without -mhtm
being specified as a global C{,XX}FLAGS.
Because the design is centered around enable HTM only in the
functional blocks that use it, this breaks on the inclusion
of the htmxlintrin.h header that includes this.
As a partial mitigation, extented to GNU/clang compilers,
transaction functions gain the attribute "hot".
In general the use of htm is around the optimistic
transaction ability of the function. The key part of using the
hot attribute is to place these functions together so that
a maximization of icache, tlb and OS paging can ensure that
these can be ready to execute by any thread/cpu with the
minimum amount of overhead.
POWER is particularly affected here because the xbegin/xend
functions are not inline.
srw_lock.cc requires the -mhtm cflag, both in the storage
engine and the unit tests.
This implements memory transaction support for:
* Intel Restricted Transactional Memory (RTM), also known as TSX-NI
(Transactional Synchronization Extensions New Instructions)
* POWER v2.09 Hardware Trace Monitor (HTM) on GNU/Linux
transactional_lock_guard, transactional_shared_lock_guard:
RAII lock guards that try to elide the lock acquisition
when transactional memory is available.
buf_pool.page_hash: Try to elide latches whenever feasible.
Related to the InnoDB change buffer and ROW_FORMAT=COMPRESSED
tables, this is not always possible.
In buf_page_get_low(), memory transactions only work reasonably
well for validating a guessed block address.
TMLockGuard, TMLockTrxGuard, TMLockMutexGuard: RAII lock guards
that try to elide lock_sys.latch and related latches.
Microsoft does not define inline assembler syntax for AMD64,
but it defines the intrinsic function _interlockedbittestandset()
that we can use.
ssux_lock_impl<bool>::rd_wait(): Remove a call to yield,
because writer.wr_lock() will take care of context switches
between loop iterations.
This addresses suggestions by Vladislav Vaintroub.
Previously, neither our wrapper of Microsoft Windows SRWLOCK
nor the futex-less implementation SUX_LOCK_GENERIC supported spin loops.
This was suggested by Vladislav Vaintroub.
srw_mutex_impl<bool>::wait_and_lock(): In
commit a73eedbf3f we introduced
an std::atomic::fetch_or() in a loop. Alas, on the IA-32 and AMD64,
that was being translated into a loop around LOCK CMPXCHG.
To avoid a nested loop, it is better to explicitly invoke
std::atomic::compare_exchange_weak() in the loop, but only if
the attempt has a chance to succeed (the HOLDER flag is not set).
It is even more efficient to use LOCK BTS, but contemporary compilers
fail to translate std::atomic::fetch_or(x) & x into that when x is
a single-bit constant. On GCC-compatible compilers, we will use
inline assembler to achieve that.
On other ISA than IA-32 and AMD64, we will continue to use
std::atomic::fetch_or().
ssux_lock_impl<spinloop>::rd_wait(): Use rd_lock_try().
A loop around std::atomic::compare_exchange_weak() should be
cheaper than fetch_add(), fetch_sub() and a wakeup system call.
These deficiencies were pointed out and the use of LOCK BTS was
suggested by Thiago Macieira.
Typically, index_lock and fil_space_t::latch will be held for a longer
time than the spin loop in latch acquisition would be waiting for.
Let us avoid spin loops for those as well as dict_sys.latch, which
could be held in exclusive mode for a longer time (while loading
metadata into the buffer pool and the dictionary cache).
Performance testing on a dual Intel Xeon E5-2630 v4 (2 NUMA nodes)
suggests that the buffer pool page latch (block_lock) benefits from a
spin loop in both read-only and read-write workloads where the working
set is slightly larger than the buffer pool. Presumably, most contention
would occur on leaf page latches. Contention on upper level pages in
the buffer pool should intuitively last longer.
We introduce srw_spin_lock and srw_spin_mutex to allow users of
srw_lock or srw_mutex to opt in for the spin loop.
On Microsoft Windows, a spin loop variant was and will not be available;
srw_mutex and srw_lock will simply wrap SRWLOCK.
That is, on Microsoft Windows, the parameters innodb_sync_spin_loops
and innodb_spin_wait_delay will only affect block_lock.
Invoking ut_delay(srv_wpin_wait_delay) inside a spinloop would
cause a read of 2 global variables as well as multiplication.
Let us loop around MY_RELAX_CPU() using a precomputed loop count
to keep the loops simpler, to help them scale better.
We also tried precomputing the delay into a global variable,
but that appeared to result in slightly worse throughput.
srw_mutex::wait_and_lock(): In the spin loop, we will try to poll
for non-conflicting lock word state by reads, avoiding any writes.
We invoke explicit std::atomic_thread_fence(std::memory_order_acquire)
before returning. The individual operations on the lock word
can use memory_order_relaxed.
srw_mutex:🔒 Document that the value for a single writer is
HOLDER+1 instead of HOLDER.
srw_mutex::wr_lock_try(), srw_mutex::wr_unlock(): Adjust the value
of the lock word of a single writer from HOLDER to HOLDER+1.
In the SUX_LOCK_GENERIC implementation, we can remember at most
one pending exclusive lock request. If multiple exclusive lock
requests are pending, the WRITER_WAITING flag will be cleared when
the first waiting writer acquires the exclusive lock.
ssux_lock_low::update_lock(): If WRITER_WAITING is set, wake up
the writer even if the UPDATER flag is set, because the waiting
writer may be in the process of upgrading its U lock to X.
rw_lock::read_unlock(): Also indicate that an X lock waiter must
be woken up if an U lock exists.
This fix may cause unnecessary wake-ups and system calls, but this
is the best that we can do. Ideally we would use the MDEV-25404
idea of a separate 'writer' mutex, but there is no portable way to
request that a non-recursive mutex be created, and InnoDB requires
the ability to transfer buf_block_t::lock ownership to an I/O thread.
To allow problems like this to be caught more reliably in the future,
we add a unit test for srw_mutex, srw_lock, ssux_lock, sux_lock.
Having both readers and writers use a single lock word in
futex system calls caused performance regression compared to
SRW_LOCK_DUMMY (mutex and 2 condition variables).
A contributing factor is that we did not accurately keep
track of the number of waiting threads and thus had to invoke
system calls to wake up any waiting threads.
SUX_LOCK_GENERIC: Renamed from SRW_LOCK_DUMMY. This is the
original implementation, with rw_lock (std::atomic<uint32_t>),
a mutex and two condition variables. Using a separate writer
mutex (as described below) is not possible, because the mutex ownership
in a buf_block_t::lock must be able to transfer from a write submitter
thread to an I/O completion thread, and pthread_mutex_lock() may assume
that the submitter thread is recursively acquiring the mutex that it
already holds, while in reality the I/O completion thread is the real
owner. POSIX does not define an interface for requesting a mutex to
be non-recursive.
On Microsoft Windows, srw_lock_low will remain a simple wrapper of
SRWLOCK. On 32-bit Microsoft Windows, sizeof(SRWLOCK)=4 while
sizeof(srw_lock_low)=8.
On other platforms, srw_lock_low is an alias of ssux_lock_low,
the Simple (non-recursive) Shared/Update/eXclusive lock.
In the futex-based implementation of ssux_lock_low (Linux, OpenBSD,
Microsoft Windows), we shall use a dedicated mutex for exclusive
requests (writer), and have a WRITER flag in the 'readers' lock word
to inform that a writer is holding the lock or waiting for the lock to
be granted. When the WRITER flag is set, all lock requests must acquire
the writer mutex. Normally, shared (S) lock requests simply perform a
compare-and-swap on the 'readers' word.
Update locks are implemented as a combination of writer mutex
and a normal counter in the 'readers' lock word. The conflict between
U and X locks is guaranteed by the writer mutex.
Unlike SUX_LOCK_GENERIC, wr_u_downgrade() will not wake up any pending
rd_lock() waits. They will wait until u_unlock() releases the writer mutex.
The ssux_lock_low is always wrapped by sux_lock (with a recursion count
of U and X locks), used for dict_index_t::lock and buf_block_t::lock.
Their memory footprint for the futex-based implementation will increase
by sizeof(srw_mutex), or 4 bytes.
This change addresses a performance regression in read-only benchmarks,
such as sysbench oltp_read_only. Also write performance was improved.
On 32-bit Linux and OpenBSD, lock_sys_t::hash_table will allocate
two hash table elements for each srw_lock (14 instead of 15 hash
table cells per 64-byte cache line on IA-32). On Microsoft Windows,
sizeof(SRWLOCK)==sizeof(void*) and there is no change.
Reviewed by: Vladislav Vaintroub
Tested by: Axel Schwenke and Vladislav Vaintroub
On Linux, OpenBSD and Microsoft Windows, srw_mutex was an alias for a
rw-lock while we only need mutex functionality. Let us implement a
futex-based mutex with one bit for HOLDER and 31 bits for counting
waiting requests.
srw_lock::wr_unlock() can avoid waking up a waiter when no waiting
requests exist. (Previously, we only had 1-bit rw_lock::WRITER_WAITING
flag that could be wrongly cleared if multiple waiting wr_lock() exist.
Now we have no problem with up to 2,147,483,648 conflicting threads.)
On 64-bit Microsoft Windows, the advantage is that
sizeof(srw_mutex) is 4, while sizeof(SRWLOCK) would be 8.
Reviewed by: Vladislav Vaintroub
ssux_lock_low::write_lock(): Before invoking writer_wait(), keep
attempting write_lock_wait_try() as long as no conflict exists.
rw_lock::upgrade_trylock(): Relax a bogus assertion and correct
the acquisition operation. Another thread may be executing in
ssux_lock_low::write_lock() on the same latch. Because we are the
only thread that can make progress on that latch, we must become
the writer. Any waiting thread will be eventually woken up by
ssux_lock_low::u_unlock() or ssux_lock_low::wr_unlock(), but not
by wr_u_downgrade() because the upgrade is a very rare operation.
It turns out that the hang that was fixed in
commit 43d3dad114
for the SRW_LOCK_DUMMY implementation is also possible in the futex
implementation. We have observed hangs of ssux_lock_low::u_unlock()
on Windows where the undesirable value is rw_lock::UPDATER, in the
test mariabackup.xb_compressed_encrypted.
The exact sequence of events to the hang is not known, but
it seems that u_unlock() had better always wake up one thread.
Possibly, the case involves multiple blocked u_unlock().
On a busy server, the hang might be 'rescued' by a subsequent
lock acquisition and release that is executed by another thread.
rw_lock::update_unlock(): Change the return type to void.
ssux_lock_low::u_unlock(): Always invoke readers_wake() [sic],
to wake up any pending update_lock() or write_lock().
On futex implementation, this will wake up all waiters.
On SRW_LOCK_DUMMY, writer_wake() and readers_wake() do the same
thing: wake up one write_lock(), or all update_lock() waiters.
SHOW ENGINE INNODB MUTEX functionality is completely removed,
as are the InnoDB latching order checks.
We will enforce innodb_fatal_semaphore_wait_threshold
only for dict_sys.mutex and lock_sys.mutex.
dict_sys_t::mutex_lock(): A single entry point for dict_sys.mutex.
lock_sys_t::mutex_lock(): A single entry point for lock_sys.mutex.
FIXME: srv_sys should be removed altogether; it is duplicating tpool
functionality.
fil_crypt_threads_init(): To prevent SAFE_MUTEX warnings, we must
not hold fil_system.mutex.
fil_close_all_files(): To prevent SAFE_MUTEX warnings for
fil_space_destroy_crypt_data(), we must not hold fil_system.mutex
while invoking fil_space_free_low() on a detached tablespace.
We will default to MUTEXTYPE=sys (using OSTrackMutex) for those
ib_mutex_t that have not been replaced yet.
The view INFORMATION_SCHEMA.INNODB_SYS_SEMAPHORE_WAITS is removed.
The parameter innodb_sync_array_size is removed.
FIXME: innodb_fatal_semaphore_wait_threshold will no longer be enforced.
We should enforce it for lock_sys.mutex and dict_sys.mutex somehow!
innodb_sync_debug=ON might still cover ib_mutex_t.
Let us replace os_event_t with mysql_cond_t, and replace the
necessary ib_mutex_t with mysql_mutex_t so that they can be
used with condition variables.
Also, let us replace polling (os_thread_sleep() or timed waits)
with plain mysql_cond_wait() wherever possible.
Furthermore, we will use the lightweight srw_mutex for trx_t::mutex,
to hopefully reduce contention on lock_sys.mutex.
FIXME: Add test coverage of
mariabackup --backup --kill-long-queries-timeout
This conceptually reverts commit 1fdc161d8f
and reintroduces an option for srw_lock to wrap a native implementation.
The srw_lock and srw_lock_low differ from ssux_lock and ssux_lock_low
in that Slim SUX locks support three modes (Shared, Update, eXclusive)
while Slim RW locks support only two (Read, Write).
On Microsoft Windows, the srw_lock will be implemented by SRWLOCK.
On Linux and OpenBSD, it will be implemented by rw_lock and the
futex system call, just like earlier.
On other systems or if SRW_LOCK_DUMMY is defined on anything else
than Microsoft Windows, rw_lock_t will be used.
ssux_lock_low::read_lock(), ssux_lock_low::update_lock(): Correct
the SRW_LOCK_DUMMY implementation to prevent hangs. The intention of
commit 1fdc161d8f seems to have been
do ... while loops, but the 'do' keyword was missing. This total
breakage was missed in commit 260161fc9f
which did reduce the probability of the hangs.
ssux_lock_low::u_unlock(): In the SRW_LOCK_DUMMY implementation
(based on a mutex and two condition variables), always invoke
writer_wake() in order to ensure that a waiting update_lock()
will be woken up.
ssux_lock_low::writer_wait(), ssux_lock_low::readers_wait():
In the SRW_LOCK_DUMMY implementation, keep waiting for the signal
until the lock word has changed. The "while" had been changed to "if"
in order to avoid hangs.
Let us remove sux_lock::waits and the associated bookkeeping.
Starting with commit 1669c8890c
the PERFORMANCE_SCHEMA instrumentation interface is keeping
track of lock waits.
The view INFORMATION_SCHEMA.INNODB_MUTEXES only exported counts
of rw-lock waits.
Also, SHOW ENGINE INNODB MUTEX will no longer export any information
about rw-locks.
The latching order checks for rw-locks have not caught many bugs
in the past few years and they are greatly complicating the code.
Last time the debug checks were useful was in
commit 59caf2c3c1 (MDEV-13485).
The B-tree hang MDEV-14637 was not caught by LatchDebug,
because the granularity of the checks is not sufficient
to distinguish the levels of non-leaf B-tree pages.
The interface was already made dead code by the grandparent
commit 03ca6495df.
InnoDB buffer pool block and index tree latches depend on a
special kind of read-update-write lock that allows reentrant
(recursive) acquisition of the 'update' and 'write' locks
as well as an upgrade from 'update' lock to 'write' lock.
The 'update' lock allows any number of reader locks from
other threads, but no concurrent 'update' or 'write' lock.
If there were no requirement to support an upgrade from 'update'
to 'write', we could compose the lock out of two srw_lock
(implemented as any type of native rw-lock, such as SRWLOCK on
Microsoft Windows). Removing this requirement is very difficult,
so in commit f7e7f487d4b06695f91f6fbeb0396b9d87fc7bbf we
implemented an 'update' mode to our srw_lock.
Re-entrant or recursive locking is mostly needed when writing or
freeing BLOB pages, but also in crash recovery or when merging
buffered changes to an index page. The re-entrancy allows us to
attach a previously acquired page to a sub-mini-transaction that
will be committed before whatever else is holding the page latch.
The SUX lock supports Shared ('read'), Update, and eXclusive ('write')
locking modes. The S latches are not re-entrant, but a single S latch
may be acquired even if the thread already holds an U latch.
The idea of the U latch is to allow a write of something that concurrent
readers do not care about (such as the contents of BTR_SEG_LEAF,
BTR_SEG_TOP and other page allocation metadata structures, or
the MDEV-6076 PAGE_ROOT_AUTO_INC). (The PAGE_ROOT_AUTO_INC field
is only updated when a dict_table_t for the table exists, and only
read when a dict_table_t for the table is being added to dict_sys.)
block_lock::u_lock_try(bool for_io=true) is used in buf_flush_page()
to allow concurrent readers but no concurrent modifications while the
page is being written to the data file. That latch will be released
by buf_page_write_complete() in a different thread. Hence, we use
the special lock owner value FOR_IO.
The index_lock::u_lock() improves concurrency on operations that
involve non-leaf index pages.
The interface has been cleaned up a little. We will use
x_lock_recursive() instead of x_lock() when we know that a
lock is already held by the current thread. Similarly,
a lock upgrade from U to X is only allowed via u_x_upgrade()
or x_lock_upgraded() but not via x_lock().
We will disable the LatchDebug and sync_array interfaces to
InnoDB rw-locks.
The SEMAPHORES section of SHOW ENGINE INNODB STATUS output
will no longer include any information about InnoDB rw-locks,
only TTASEventMutex (cmake -DMUTEXTYPE=event) waits.
This will make a part of the 'innotop' script dead code.
The block_lock buf_block_t::lock will not be covered by any
PERFORMANCE_SCHEMA instrumentation.
SHOW ENGINE INNODB MUTEX and INFORMATION_SCHEMA.INNODB_MUTEXES
will no longer output source code file names or line numbers.
The dict_index_t::lock will be identified by index and table names,
which should be much more useful. PERFORMANCE_SCHEMA is lumping
information about all dict_index_t::lock together as
event_name='wait/synch/sxlock/innodb/index_tree_rw_lock'.
buf_page_free(): Remove the file,line parameters. The sux_lock will
not store such diagnostic information.
buf_block_dbg_add_level(): Define as empty macro, to be removed
in a subsequent commit.
Unless the build was configured with cmake -DPLUGIN_PERFSCHEMA=NO
the index_lock dict_index_t::lock will be instrumented via
PERFORMANCE_SCHEMA. Similar to
commit 1669c8890c
we will distinguish lock waits by registering shared_lock,exclusive_lock
events instead of try_shared_lock,try_exclusive_lock.
Actual 'try' operations will not be instrumented at all.
rw_lock_list: Remove. After MDEV-24167, this only covered
buf_block_t::lock and dict_index_t::lock. We will output their
information by traversing buf_pool or dict_sys.
The PERFORMANCE_SCHEMA insists on distinguishing read-update-write
locks from read-write locks, so we must add
template<bool support_u_lock> in rd_lock() and wr_lock() operations.
rd_lock::read_trylock(): Add template<bool prioritize_updater=false>
which is used by the srw_lock_low::read_lock() loop. As long as
an UPDATE lock has already been granted to some thread, we will grant
subsequent READ lock requests even if a waiting WRITE lock request
exists. This will be necessary to be compatible with existing usage
pattern of InnoDB rw_lock_t where the holder of SX-latch (which we
will rename to UPDATE latch) may acquire an additional S-latch
on the same object. For normal read-write locks without update operations
this should make no difference at all, because the rw_lock::UPDATER
flag would never be set.
Let us try to avoid code bloat for the common case that
performance_schema is disabled at runtime, and use
ATTRIBUTE_NOINLINE member functions for instrumented latch acquisition.
Also, let us distinguish lock waits from non-contended lock requests
by using write_lock,read_lock for the requests that lead to waits,
and try_write_lock,try_read_lock for the wait-free lock acquisitions.
Actual 'try' operations are not being instrumented at all.
In commit 1fdc161d8f we introduced
a mutex-and-condition-variable based fallback implementation
for platforms that lack a futex system call. That implementation
is prone to hangs.
Let us use separate condition variables for shared and exclusive requests.
Let us always base srw_lock on our own std::atomic<uint32_t>
based rw_lock. In this way, we can extend the locks in a portable
way across all platforms.
We will use futex system calls where available:
Linux, OpenBSD, and Microsoft Windows.
Elsewhere, we will emulate futex with a mutex and a condition variable.
Thanks to Daniel Black for testing this on OpenBSD.
A side effect of MDEV-16264 is that a large number of threads will
be created at server startup, to be destroyed after a minute or two.
One source of such thread creation is srv_start_periodic_timer().
InnoDB is creating 3 periodic tasks: srv_master_callback (1Hz)
srv_error_monitor_task (1Hz), and srv_monitor_task (0.2Hz).
It appears that we can merge srv_error_monitor_task and srv_monitor_task
and have them invoked 4 times per minute (every 15 seconds). This will
affect our ability to enforce innodb_fatal_semaphore_wait_threshold and
some computations around BUF_LRU_STAT_N_INTERVAL.
We could remove srv_master_callback along with the DROP TABLE queue
at some point of time in the future. We must keep it independent
of the innodb_fatal_semaphore_wait_threshold detection, because
the background DROP TABLE queue could get stuck due to dict_sys
being locked by another thread. For now, srv_master_callback
must be invoked once per second, so that
innodb_flush_log_at_timeout=1 can work.
BUF_LRU_STAT_N_INTERVAL: Reduce the precision and extend the time
from 50*1 second to 4*15 seconds.
srv_error_monitor_timer: Remove.
MAX_MUTEX_NOWAIT: Increase from 20*1 second to 2*15 seconds.
srv_refresh_innodb_monitor_stats(): Avoid a repeated call to time(NULL).
Change the interval to less than 60 seconds.
srv_monitor(): Renamed from srv_monitor_task.
srv_monitor_task(): Renamed from srv_error_monitor_task().
Invoked only once in 15 seconds. Invoke also srv_monitor().
Increase the fatal_cnt threshold from 10*1 second to 1*15 seconds.
sync_array_print_long_waits_low(): Invoke time(NULL) only once.
Remove a bogus message about printouts for 30 seconds. Those
printouts were effectively already disabled in MDEV-16264
(commit 5e62b6a5e0).
We must avoid acquiring a latch while we are already holding one.
The tablespace latch was being acquired recursively in some
operations that allocate or free pages.
fts_cache_t::init_lock: Replace with mutex. This was only acquired
in exclusive mode.
fts_cache_t:🔒 Replace with mutex. The only read-lock user was
i_s_fts_index_cache_fill() for producing content for the view
INFORMATION_SCHEMA.INNODB_FT_INDEX_CACHE.
Many InnoDB rw-locks unnecessarily depend on the complex
InnoDB rw_lock_t implementation that support the SX lock mode
as well as recursive acquisition of X or SX locks.
One of them is the bunch of adaptive hash index search latches,
instrumented as btr_search_latch in PERFORMANCE_SCHEMA.
Let us introduce a simpler lock for those in order to
reduce overhead.
srw_lock: A simple read-write lock that does not support recursion.
On Microsoft Windows, this wraps SRWLOCK, only adding
runtime overhead if PERFORMANCE_SCHEMA is enabled.
On Linux (all architectures), this is implemented with
std::atomic<uint32_t> and the futex system call.
On other platforms, we will wrap mysql_rwlock_t with
zero runtime overhead.
The PERFORMANCE_SCHEMA instrumentation differs
from InnoDB rw_lock_t in that we will only invoke
PSI_RWLOCK_CALL(start_rwlock_wrwait) or
PSI_RWLOCK_CALL(start_rwlock_rdwait)
if there is an actual conflict.
We always defined PFS_SKIP_BUFFER_MUTEX_RWLOCK, that is,
the latches of the buffer pool blocks were never instrumented
in PERFORMANCE_SCHEMA.
For some reason, the debug_latch (which enforce proper usage of
buffer-fixing in debug builds) was instrumented.
The fix of MDEV-23456 (commit b1009ae5c1)
introduced a livelock between page flushing and a thread that is
executing buf_page_create().
buf_page_create(): If the current mini-transaction is holding
an exclusive latch on the page, do not attempt to acquire another
one, and do not care about any I/O fix.
mtr_t::have_x_latch(): Replaces mtr_t::get_fix_count().
dyn_buf_t::for_each_block(const Functor&) const: A new variant.
rw_lock_own(): Add a const qualifier.
Reviewed by: Thirunarayanan Balathandayuthapani