summaryrefslogtreecommitdiff
path: root/storage/innobase/include
diff options
context:
space:
mode:
authorMarko Mäkelä <marko.makela@mariadb.com>2020-06-18 13:38:30 +0300
committerMarko Mäkelä <marko.makela@mariadb.com>2020-06-18 14:16:01 +0300
commit5155a300fab85e97217c75e3ba3c3ce78082dd8a (patch)
tree7180cc802900bbf2c5328835d3de39e78c469615 /storage/innobase/include
parentcfd3d70ccbbfcf3fdec034be446317741dfae824 (diff)
downloadmariadb-git-5155a300fab85e97217c75e3ba3c3ce78082dd8a.tar.gz
MDEV-22871: Reduce InnoDB buf_pool.page_hash contention
The rw_lock_s_lock() calls for the buf_pool.page_hash became a clear bottleneck after MDEV-15053 reduced the contention on buf_pool.mutex. We will replace that use of rw_lock_t with a special implementation that is optimized for memory bus traffic. The hash_table_locks instrumentation will be removed. buf_pool_t::page_hash: Use a special implementation whose API is compatible with hash_table_t, and store the custom rw-locks directly in buf_pool.page_hash.array, intentionally sharing cache lines with the hash table pointers. rw_lock: A low-level rw-lock implementation based on std::atomic<uint32_t> where read_trylock() becomes a simple fetch_add(1). buf_pool_t::page_hash_latch: The special of rw_lock for the page_hash. buf_pool_t::page_hash_latch::read_lock(): Assert that buf_pool.mutex is not being held by the caller. buf_pool_t::page_hash_latch::write_lock() may be called while not holding buf_pool.mutex. buf_pool_t::watch_set() is such a caller. buf_pool_t::page_hash_latch::read_lock_wait(), page_hash_latch::write_lock_wait(): The spin loops. These will obey the global parameters innodb_sync_spin_loops and innodb_sync_spin_wait_delay. buf_pool_t::freed_page_hash: A singly linked list of copies of buf_pool.page_hash that ever existed. The fact that we never free any buf_pool.page_hash.array guarantees that all page_hash_latch that ever existed will remain valid until shutdown. buf_pool_t::resize_hash(): Replaces buf_pool_resize_hash(). Prepend a shallow copy of the old page_hash to freed_page_hash. buf_pool_t::page_hash_table::n_cells: Declare as Atomic_relaxed. buf_pool_t::page_hash_table::lock(): Explain what prevents a race condition with buf_pool_t::resize_hash().
Diffstat (limited to 'storage/innobase/include')
-rw-r--r--storage/innobase/include/buf0buf.h193
-rw-r--r--storage/innobase/include/buf0lru.h2
-rw-r--r--storage/innobase/include/buf0types.h33
-rw-r--r--storage/innobase/include/hash0hash.h4
-rw-r--r--storage/innobase/include/rw_lock.h106
-rw-r--r--storage/innobase/include/srv0srv.h2
-rw-r--r--storage/innobase/include/sync0rw.ic18
-rw-r--r--storage/innobase/include/sync0sync.h1
8 files changed, 262 insertions, 97 deletions
diff --git a/storage/innobase/include/buf0buf.h b/storage/innobase/include/buf0buf.h
index 96c96113e85..4e0b25c52dd 100644
--- a/storage/innobase/include/buf0buf.h
+++ b/storage/innobase/include/buf0buf.h
@@ -70,9 +70,6 @@ struct fil_addr_t;
#define BUF_EVICT_IF_IN_POOL 20 /*!< evict a clean block if found */
/* @} */
-#define MAX_PAGE_HASH_LOCKS 1024 /*!< The maximum number of
- page_hash locks */
-
# ifdef UNIV_DEBUG
extern my_bool buf_disable_resize_buffer_pool_debug; /*!< if TRUE, resizing
buffer pool is not allowed. */
@@ -1605,47 +1602,9 @@ public:
}
/** Get the page_hash latch for a page */
- rw_lock_t *hash_lock_get(const page_id_t id) const
- {
- return hash_lock_get_low(id.fold());
- }
- /** Get a page_hash latch. */
- rw_lock_t *hash_lock_get_low(ulint fold) const
- {
- return page_hash_latches +
- ut_2pow_remainder(page_hash.calc_hash(fold),
- ulint{srv_n_page_hash_locks});
- }
-private:
- /** Get a page_hash latch. */
- rw_lock_t *hash_lock_get_low(ulint fold, ulint n_cells) const
- {
- return page_hash_latches +
- ut_2pow_remainder(ut_hash_ulint(fold, n_cells),
- ulint{srv_n_page_hash_locks});
- }
-public:
-
- /** Acquire a page_hash bucket latch, tolerating concurrent resize()
- @tparam exclusive whether the latch is to be acquired exclusively
- @param fold hash bucket key */
- template<bool exclusive> rw_lock_t *page_hash_lock(ulint fold)
+ page_hash_latch *hash_lock_get(const page_id_t id) const
{
- for (;;)
- {
- auto n_cells= page_hash.n_cells;
- rw_lock_t *latch= hash_lock_get_low(fold, n_cells);
- if (exclusive)
- rw_lock_x_lock(latch);
- else
- rw_lock_s_lock(latch);
- if (UNIV_LIKELY(n_cells == page_hash.n_cells))
- return latch;
- if (exclusive)
- rw_lock_x_unlock(latch);
- else
- rw_lock_s_unlock(latch);
- }
+ return page_hash.lock_get(id.fold());
}
/** Look up a block descriptor.
@@ -1656,9 +1615,7 @@ public:
buf_page_t *page_hash_get_low(const page_id_t id, const ulint fold)
{
ut_ad(id.fold() == fold);
- ut_ad(mutex_own(&mutex) ||
- rw_lock_own_flagged(hash_lock_get_low(fold),
- RW_LOCK_FLAG_X | RW_LOCK_FLAG_S));
+ ut_ad(mutex_own(&mutex) || page_hash.lock_get(fold)->is_locked());
buf_page_t *bpage;
/* Look for the page in the hash table */
HASH_SEARCH(hash, &page_hash, fold, buf_page_t*, bpage,
@@ -1676,17 +1633,14 @@ private:
@retval nullptr if no block was found; !lock || !*lock will also hold */
template<bool exclusive,bool watch>
buf_page_t *page_hash_get_locked(const page_id_t page_id, ulint fold,
- rw_lock_t **hash_lock)
+ page_hash_latch **hash_lock)
{
ut_ad(hash_lock || !exclusive);
- rw_lock_t *latch= page_hash_lock<exclusive>(fold);
+ page_hash_latch *latch= page_hash.lock<exclusive>(fold);
buf_page_t *bpage= page_hash_get_low(page_id, fold);
if (!bpage || watch_is_sentinel(*bpage))
{
- if (exclusive)
- rw_lock_x_unlock(latch);
- else
- rw_lock_s_unlock(latch);
+ latch->release<exclusive>();
if (hash_lock)
*hash_lock= nullptr;
return watch ? bpage : nullptr;
@@ -1697,10 +1651,8 @@ private:
if (hash_lock)
*hash_lock= latch; /* to be released by the caller */
- else if (exclusive)
- rw_lock_x_unlock(latch);
else
- rw_lock_s_unlock(latch);
+ latch->release<exclusive>();
return bpage;
}
public:
@@ -1713,7 +1665,7 @@ public:
@retval nullptr if no block was found; !lock || !*lock will also hold */
template<bool exclusive>
buf_page_t *page_hash_get_locked(const page_id_t page_id, ulint fold,
- rw_lock_t **hash_lock)
+ page_hash_latch **hash_lock)
{ return page_hash_get_locked<exclusive,false>(page_id, fold, hash_lock); }
/** @return whether the buffer pool contains a page
@@ -1730,9 +1682,7 @@ public:
@return whether bpage a sentinel for a buffer pool watch */
bool watch_is_sentinel(const buf_page_t &bpage)
{
- ut_ad(mutex_own(&mutex) ||
- rw_lock_own_flagged(hash_lock_get(bpage.id()),
- RW_LOCK_FLAG_X | RW_LOCK_FLAG_S));
+ ut_ad(mutex_own(&mutex) || hash_lock_get(bpage.id())->is_locked());
ut_ad(bpage.in_file());
if (&bpage < &watch[0] || &bpage >= &watch[UT_ARR_SIZE(watch)])
@@ -1754,11 +1704,11 @@ public:
bool watch_occurred(const page_id_t id)
{
const ulint fold= id.fold();
- rw_lock_t *hash_lock= page_hash_lock<false>(fold);
+ page_hash_latch *hash_lock= page_hash.lock<false>(fold);
/* The page must exist because watch_set() increments buf_fix_count. */
buf_page_t *bpage= page_hash_get_low(id, fold);
const bool is_sentinel= watch_is_sentinel(*bpage);
- rw_lock_s_unlock(hash_lock);
+ hash_lock->read_unlock();
return !is_sentinel;
}
@@ -1769,7 +1719,8 @@ public:
@param hash_lock exclusively held page_hash latch
@return a buffer pool block corresponding to id
@retval nullptr if the block was not present, and a watch was installed */
- inline buf_page_t *watch_set(const page_id_t id, rw_lock_t **hash_lock);
+ inline buf_page_t *watch_set(const page_id_t id,
+ page_hash_latch **hash_lock);
/** Stop watching whether a page has been read in.
watch_set(id) must have returned nullptr before.
@@ -1777,7 +1728,7 @@ public:
void watch_unset(const page_id_t id)
{
const ulint fold= id.fold();
- rw_lock_t *hash_lock= page_hash_lock<true>(fold);
+ page_hash_latch *hash_lock= page_hash.lock<true>(fold);
/* The page must exist because watch_set() increments buf_fix_count. */
buf_page_t *watch= page_hash_get_low(id, fold);
if (watch->unfix() == 0 && watch_is_sentinel(*watch))
@@ -1786,7 +1737,7 @@ public:
ut_ad(watch->in_page_hash);
ut_d(watch->in_page_hash= false);
HASH_DELETE(buf_page_t, hash, &page_hash, fold, watch);
- rw_lock_x_unlock(hash_lock);
+ hash_lock->write_unlock();
// Now that the watch is detached from page_hash, release it to watch[].
mutex_enter(&mutex);
/* It is possible that watch_remove() already removed the watch. */
@@ -1799,7 +1750,7 @@ public:
mutex_exit(&mutex);
}
else
- rw_lock_x_unlock(hash_lock);
+ hash_lock->write_unlock();
}
/** Remove the sentinel block for the watch before replacing it with a
@@ -1872,11 +1823,92 @@ public:
/** read-ahead request size in pages */
Atomic_counter<uint32_t> read_ahead_area;
+ /** Hash table with singly-linked overflow lists. @see hash_table_t */
+ struct page_hash_table
+ {
+ /** Number of array[] elements per page_hash_latch.
+ Must be one less than a power of 2. */
+ static constexpr size_t ELEMENTS_PER_LATCH= 1023;
+
+ /** number of payload elements in array[] */
+ Atomic_relaxed<ulint> n_cells;
+ /** the hash array, with pad(n_cells) elements */
+ hash_cell_t *array;
+
+ /** Create the hash table.
+ @param n the lower bound of n_cells */
+ void create(ulint n);
+
+ /** Free the hash table. */
+ void free() { ut_free(array); array= nullptr; }
+
+ /** @return the index of an array element */
+ ulint calc_hash(ulint fold) const { return calc_hash(fold, n_cells); }
+ /** @return raw array index converted to padded index */
+ static ulint pad(ulint h) { return 1 + (h / ELEMENTS_PER_LATCH) + h; }
+ private:
+ /** @return the hash value before any ELEMENTS_PER_LATCH padding */
+ static ulint hash(ulint fold, ulint n) { return ut_hash_ulint(fold, n); }
+
+ /** @return the index of an array element */
+ static ulint calc_hash(ulint fold, ulint n_cells)
+ {
+ return pad(hash(fold, n_cells));
+ }
+ /** Get a page_hash latch. */
+ page_hash_latch *lock_get(ulint fold, ulint n) const
+ {
+ static_assert(!((ELEMENTS_PER_LATCH + 1) & ELEMENTS_PER_LATCH),
+ "must be one less than a power of 2");
+ return reinterpret_cast<page_hash_latch*>
+ (&array[calc_hash(fold, n) & ~ELEMENTS_PER_LATCH]);
+ }
+ public:
+ /** Get a page_hash latch. */
+ page_hash_latch *lock_get(ulint fold) const
+ { return lock_get(fold, n_cells); }
+
+ /** Acquire an array latch, tolerating concurrent buf_pool_t::resize()
+ @tparam exclusive whether the latch is to be acquired exclusively
+ @param fold hash bucket key */
+ template<bool exclusive> page_hash_latch *lock(ulint fold)
+ {
+ for (;;)
+ {
+ auto n= n_cells;
+ page_hash_latch *latch= lock_get(fold, n);
+ latch->acquire<exclusive>();
+ /* Our latch prevents n_cells from changing. */
+ if (UNIV_LIKELY(n == n_cells))
+ return latch;
+ /* Retry, because buf_pool_t::resize_hash() affected us. */
+ latch->release<exclusive>();
+ }
+ }
+
+ /** Exclusively aqcuire all latches */
+ inline void write_lock_all();
+
+ /** Release all latches */
+ inline void write_unlock_all();
+ };
+
+private:
+ /** Former page_hash that has been deleted during resize();
+ singly-linked list via freed_page_hash->array[1] */
+ page_hash_table *freed_page_hash;
+
+ /** Lock all page_hash, also freed_page_hash. */
+ inline void write_lock_all_page_hash();
+ /** Release all page_hash, also freed_page_hash. */
+ inline void write_unlock_all_page_hash();
+ /** Resize page_hash and zip_hash. */
+ inline void resize_hash();
+
+public:
/** Hash table of file pages (buf_page_t::in_file() holds),
- indexed by page_id_t. Protected by both mutex and page_hash_latches[]. */
- hash_table_t page_hash;
- /** Latches protecting page_hash */
- mutable rw_lock_t page_hash_latches[MAX_PAGE_HASH_LOCKS];
+ indexed by page_id_t. Protected by both mutex and page_hash.lock_get(). */
+ page_hash_table page_hash;
/** map of block->frame to buf_block_t blocks that belong
to buf_buddy_alloc(); protected by buf_pool.mutex */
@@ -2103,6 +2135,19 @@ private:
/** The InnoDB buffer pool */
extern buf_pool_t buf_pool;
+inline void page_hash_latch::read_lock()
+{
+ ut_ad(!mutex_own(&buf_pool.mutex));
+ if (!read_trylock())
+ read_lock_wait();
+}
+
+inline void page_hash_latch::write_lock()
+{
+ if (!write_trylock())
+ write_lock_wait();
+}
+
inline void buf_page_t::add_buf_fix_count(uint32_t count)
{
ut_ad(mutex_own(&buf_pool.mutex));
@@ -2129,15 +2174,15 @@ inline void buf_page_t::set_state(buf_page_state state)
if (!in_file()) break;
/* fall through */
case BUF_BLOCK_FILE_PAGE:
- ut_ad(rw_lock_own(buf_pool.hash_lock_get(id_), RW_LOCK_X));
+ ut_ad(buf_pool.hash_lock_get(id_)->is_write_locked());
break;
case BUF_BLOCK_NOT_USED:
if (!in_file()) break;
/* fall through */
case BUF_BLOCK_ZIP_PAGE:
- ut_ad((this >= &buf_pool.watch[0] &&
- this <= &buf_pool.watch[UT_ARR_SIZE(buf_pool.watch)]) ||
- rw_lock_own(buf_pool.hash_lock_get(id_), RW_LOCK_X));
+ ut_ad(buf_pool.hash_lock_get(id_)->is_write_locked() ||
+ (this >= &buf_pool.watch[0] &&
+ this <= &buf_pool.watch[UT_ARR_SIZE(buf_pool.watch)]));
break;
}
#endif
@@ -2159,7 +2204,7 @@ inline void buf_page_t::set_corrupt_id()
break;
case BUF_BLOCK_ZIP_PAGE:
case BUF_BLOCK_FILE_PAGE:
- ut_ad(rw_lock_own(buf_pool.hash_lock_get(id_), RW_LOCK_X));
+ ut_ad(buf_pool.hash_lock_get(id_)->is_write_locked());
break;
case BUF_BLOCK_NOT_USED:
case BUF_BLOCK_MEMORY:
diff --git a/storage/innobase/include/buf0lru.h b/storage/innobase/include/buf0lru.h
index ed3a6cabdb3..937cb427a47 100644
--- a/storage/innobase/include/buf0lru.h
+++ b/storage/innobase/include/buf0lru.h
@@ -153,7 +153,7 @@ buf_LRU_stat_update();
@param id page identifier
@param hash_lock buf_pool.page_hash latch (will be released here) */
void buf_LRU_free_one_page(buf_page_t *bpage, const page_id_t id,
- rw_lock_t *hash_lock)
+ page_hash_latch *hash_lock)
MY_ATTRIBUTE((nonnull));
#ifdef UNIV_DEBUG
diff --git a/storage/innobase/include/buf0types.h b/storage/innobase/include/buf0types.h
index 1802bd57ddd..55bd2ac3a5a 100644
--- a/storage/innobase/include/buf0types.h
+++ b/storage/innobase/include/buf0types.h
@@ -192,10 +192,43 @@ extern const byte field_ref_zero[UNIV_PAGE_SIZE_MAX];
#include "ut0mutex.h"
#include "sync0rw.h"
+#include "rw_lock.h"
typedef ib_mutex_t BufPoolMutex;
typedef ib_mutex_t FlushListMutex;
typedef rw_lock_t BPageLock;
+
+class page_hash_latch : public rw_lock
+{
+public:
+ /** Wait for a shared lock */
+ void read_lock_wait();
+ /** Wait for an exclusive lock */
+ void write_lock_wait();
+
+ /** Acquire a shared lock */
+ inline void read_lock();
+ /** Acquire an exclusive lock */
+ inline void write_lock();
+
+ /** Acquire a lock */
+ template<bool exclusive> void acquire()
+ {
+ if (exclusive)
+ write_lock();
+ else
+ read_lock();
+ }
+ /** Release a lock */
+ template<bool exclusive> void release()
+ {
+ if (exclusive)
+ write_unlock();
+ else
+ read_unlock();
+ }
+};
+
#endif /* !UNIV_INNOCHECKSUM */
#endif /* buf0types.h */
diff --git a/storage/innobase/include/hash0hash.h b/storage/innobase/include/hash0hash.h
index 58da36fee5e..981ff5a0814 100644
--- a/storage/innobase/include/hash0hash.h
+++ b/storage/innobase/include/hash0hash.h
@@ -33,8 +33,6 @@ struct hash_cell_t{
};
typedef void* hash_node_t;
-#define hash_calc_hash(FOLD, TABLE) (TABLE)->calc_hash(FOLD)
-
/*******************************************************************//**
Inserts a struct to a hash table. */
@@ -145,7 +143,7 @@ Gets the next struct in a hash chain, NULL if none. */
Looks for a struct in a hash table. */
#define HASH_SEARCH(NAME, TABLE, FOLD, TYPE, DATA, ASSERTION, TEST)\
{\
- (DATA) = (TYPE) HASH_GET_FIRST(TABLE, hash_calc_hash(FOLD, TABLE));\
+ (DATA) = (TYPE) HASH_GET_FIRST(TABLE, (TABLE)->calc_hash(FOLD)); \
HASH_ASSERT_VALID(DATA);\
\
while ((DATA) != NULL) {\
diff --git a/storage/innobase/include/rw_lock.h b/storage/innobase/include/rw_lock.h
new file mode 100644
index 00000000000..613adfef3f5
--- /dev/null
+++ b/storage/innobase/include/rw_lock.h
@@ -0,0 +1,106 @@
+/*****************************************************************************
+
+Copyright (c) 2020, MariaDB Corporation.
+
+This program is free software; you can redistribute it and/or modify it under
+the terms of the GNU General Public License as published by the Free Software
+Foundation; version 2 of the License.
+
+This program is distributed in the hope that it will be useful, but WITHOUT
+ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or FITNESS
+FOR A PARTICULAR PURPOSE. See the GNU General Public License for more details.
+
+You should have received a copy of the GNU General Public License along with
+this program; if not, write to the Free Software Foundation, Inc.,
+51 Franklin Street, Fifth Floor, Boston, MA 02110-1335 USA
+
+*****************************************************************************/
+
+#pragma once
+#include <atomic>
+#include "my_dbug.h"
+
+/** Simple read-write lock based on std::atomic */
+class rw_lock
+{
+ /** The lock word */
+ std::atomic<uint32_t> lock;
+
+protected:
+ /** Available lock */
+ static constexpr uint32_t UNLOCKED= 0;
+ /** Flag to indicate that write_lock() is being held */
+ static constexpr uint32_t WRITER= 1 << 31;
+ /** Flag to indicate that write_lock_wait() is pending */
+ static constexpr uint32_t WRITER_WAITING= 1 << 30;
+ /** Flag to indicate that write_lock() or write_lock_wait() is pending */
+ static constexpr uint32_t WRITER_PENDING= WRITER | WRITER_WAITING;
+
+ /** Yield a read lock request due to a conflict with a write lock.
+ @return the lock value */
+ uint32_t read_lock_yield()
+ {
+ uint32_t l= lock.fetch_sub(1, std::memory_order_relaxed);
+ DBUG_ASSERT(l & ~WRITER_PENDING);
+ return l;
+ }
+ /** Start waiting for an exclusive lock. */
+ void write_lock_wait_start()
+ { lock.fetch_or(WRITER_WAITING, std::memory_order_relaxed); }
+ /** Wait for an exclusive lock.
+ @return whether the exclusive lock was acquired */
+ bool write_lock_poll()
+ {
+ auto l= WRITER_WAITING;
+ if (lock.compare_exchange_strong(l, WRITER, std::memory_order_acquire,
+ std::memory_order_relaxed))
+ return true;
+ if (!(l & WRITER_WAITING))
+ /* write_lock() must have succeeded for another thread */
+ write_lock_wait_start();
+ return false;
+ }
+
+public:
+ /** Default constructor */
+ rw_lock() : lock(UNLOCKED) {}
+
+ /** Release a shared lock */
+ void read_unlock()
+ {
+ IF_DBUG_ASSERT(auto l=,) lock.fetch_sub(1, std::memory_order_release);
+ DBUG_ASSERT(l & ~WRITER_PENDING); /* at least one read lock */
+ DBUG_ASSERT(!(l & WRITER)); /* no write lock must have existed */
+ }
+ /** Release an exclusive lock */
+ void write_unlock()
+ {
+ IF_DBUG_ASSERT(auto l=,) lock.fetch_sub(WRITER, std::memory_order_release);
+ DBUG_ASSERT(l & WRITER); /* the write lock must have existed */
+ }
+ /** Try to acquire a shared lock.
+ @return whether the lock was acquired */
+ bool read_trylock()
+ { return !(lock.fetch_add(1, std::memory_order_acquire) & WRITER_PENDING); }
+ /** Try to acquire an exclusive lock.
+ @return whether the lock was acquired */
+ bool write_trylock()
+ {
+ auto l= UNLOCKED;
+ return lock.compare_exchange_strong(l, WRITER, std::memory_order_acquire,
+ std::memory_order_relaxed);
+ }
+
+ /** @return whether an exclusive lock is being held by any thread */
+ bool is_write_locked() const
+ { return !!(lock.load(std::memory_order_relaxed) & WRITER); }
+ /** @return whether a shared lock is being held by any thread */
+ bool is_read_locked() const
+ {
+ auto l= lock.load(std::memory_order_relaxed);
+ return (l & ~WRITER_PENDING) && !(l & WRITER);
+ }
+ /** @return whether any lock is being held by any thread */
+ bool is_locked() const
+ { return (lock.load(std::memory_order_relaxed) & ~WRITER_WAITING) != 0; }
+};
diff --git a/storage/innobase/include/srv0srv.h b/storage/innobase/include/srv0srv.h
index 2aa874edfad..d4b6425c44b 100644
--- a/storage/innobase/include/srv0srv.h
+++ b/storage/innobase/include/srv0srv.h
@@ -333,8 +333,6 @@ extern const ulint srv_buf_pool_min_size;
extern const ulint srv_buf_pool_def_size;
/** Requested buffer pool chunk size */
extern ulong srv_buf_pool_chunk_unit;
-/** Number of locks to protect buf_pool.page_hash */
-extern ulong srv_n_page_hash_locks;
/** Scan depth for LRU flush batch i.e.: number of blocks scanned*/
extern ulong srv_LRU_scan_depth;
/** Whether or not to flush neighbors of a block */
diff --git a/storage/innobase/include/sync0rw.ic b/storage/innobase/include/sync0rw.ic
index 7fcac01e5ba..169cbdd9aa5 100644
--- a/storage/innobase/include/sync0rw.ic
+++ b/storage/innobase/include/sync0rw.ic
@@ -226,22 +226,8 @@ rw_lock_lock_word_decr(
caused by concurrent executions of
rw_lock_s_lock(). */
-#if 1 /* FIXME: MDEV-22871 Spurious contention between rw_lock_s_lock() */
-
- /* When the number of concurrently executing threads
- exceeds the number of available processor cores,
- multiple buf_pool.page_hash S-latch requests would
- conflict here, mostly in buf_page_get_low(). We should
- implement a simpler rw-lock where the S-latch
- acquisition would be a simple fetch_add(1) followed by
- either an optional load() loop to wait for the X-latch
- to be released, or a fetch_sub(1) and a retry.
-
- For now, we work around the problem with a delay in
- this loop. It helped a little on some systems and was
- reducing performance on others. */
- (void) LF_BACKOFF();
-#endif
+ /* Note: unlike this implementation, rw_lock::read_lock()
+ allows concurrent calls without a spin loop */
}
/* A real conflict was detected. */
diff --git a/storage/innobase/include/sync0sync.h b/storage/innobase/include/sync0sync.h
index 7eb8250b63d..c63fedb43ee 100644
--- a/storage/innobase/include/sync0sync.h
+++ b/storage/innobase/include/sync0sync.h
@@ -126,7 +126,6 @@ extern mysql_pfs_key_t index_tree_rw_lock_key;
extern mysql_pfs_key_t index_online_log_key;
extern mysql_pfs_key_t dict_table_stats_key;
extern mysql_pfs_key_t trx_sys_rw_lock_key;
-extern mysql_pfs_key_t hash_table_locks_key;
#endif /* UNIV_PFS_RWLOCK */
/** Prints info of the sync system.