Commit 5155a300 authored by Marko Mäkelä's avatar Marko Mäkelä

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().
parent cfd3d70c
......@@ -17,7 +17,6 @@ wait/synch/sxlock/innodb/dict_table_stats
wait/synch/sxlock/innodb/fil_space_latch
wait/synch/sxlock/innodb/fts_cache_init_rw_lock
wait/synch/sxlock/innodb/fts_cache_rw_lock
wait/synch/sxlock/innodb/hash_table_locks
wait/synch/sxlock/innodb/index_online_log
wait/synch/sxlock/innodb/index_tree_rw_lock
wait/synch/sxlock/innodb/trx_i_s_cache_lock
......
......@@ -486,7 +486,6 @@ insert into test.sanity values
("JUNK: GLOBAL-ONLY", "I_S.SESSION_VARIABLES", "INNODB_OPTIMIZE_FULLTEXT_ONLY"),
("JUNK: GLOBAL-ONLY", "I_S.SESSION_VARIABLES", "INNODB_PAGE_CLEANERS"),
("JUNK: GLOBAL-ONLY", "I_S.SESSION_VARIABLES", "INNODB_PAGE_CLEANER_DISABLED_DEBUG"),
("JUNK: GLOBAL-ONLY", "I_S.SESSION_VARIABLES", "INNODB_PAGE_HASH_LOCKS"),
("JUNK: GLOBAL-ONLY", "I_S.SESSION_VARIABLES", "INNODB_PAGE_SIZE"),
("JUNK: GLOBAL-ONLY", "I_S.SESSION_VARIABLES", "INNODB_PRINT_ALL_DEADLOCKS"),
("JUNK: GLOBAL-ONLY", "I_S.SESSION_VARIABLES", "INNODB_PURGE_BATCH_SIZE"),
......
select @@global.innodb_page_hash_locks between 1 and 1024;
@@global.innodb_page_hash_locks between 1 and 1024
1
select @@global.innodb_page_hash_locks;
@@global.innodb_page_hash_locks
64
select @@session.innodb_page_hash_locks;
ERROR HY000: Variable 'innodb_page_hash_locks' is a GLOBAL variable
show global variables like 'innodb_page_hash_locks';
Variable_name Value
innodb_page_hash_locks 64
show session variables like 'innodb_page_hash_locks';
Variable_name Value
innodb_page_hash_locks 64
select * from information_schema.global_variables where variable_name='innodb_page_hash_locks';
VARIABLE_NAME VARIABLE_VALUE
INNODB_PAGE_HASH_LOCKS 64
select * from information_schema.session_variables where variable_name='innodb_page_hash_locks';
VARIABLE_NAME VARIABLE_VALUE
INNODB_PAGE_HASH_LOCKS 64
set global innodb_page_hash_locks=1;
ERROR HY000: Variable 'innodb_page_hash_locks' is a read only variable
set @@session.innodb_page_hash_locks='some';
ERROR HY000: Variable 'innodb_page_hash_locks' is a read only variable
......@@ -380,15 +380,6 @@
VARIABLE_COMMENT Deprecated parameter with no effect.
NUMERIC_MIN_VALUE 0
NUMERIC_MAX_VALUE 64
@@ -1513,7 +1513,7 @@
SESSION_VALUE NULL
DEFAULT_VALUE 16
VARIABLE_SCOPE GLOBAL
-VARIABLE_TYPE BIGINT UNSIGNED
+VARIABLE_TYPE INT UNSIGNED
VARIABLE_COMMENT Number of rw_locks protecting buffer pool page_hash. Rounded up to the next power of 2
NUMERIC_MIN_VALUE 1
NUMERIC_MAX_VALUE 1024
@@ -1525,7 +1525,7 @@
SESSION_VALUE NULL
DEFAULT_VALUE 16384
......
......@@ -1509,18 +1509,6 @@ NUMERIC_BLOCK_SIZE NULL
ENUM_VALUE_LIST OFF,ON
READ_ONLY NO
COMMAND_LINE_ARGUMENT OPTIONAL
VARIABLE_NAME INNODB_PAGE_HASH_LOCKS
SESSION_VALUE NULL
DEFAULT_VALUE 64
VARIABLE_SCOPE GLOBAL
VARIABLE_TYPE BIGINT UNSIGNED
VARIABLE_COMMENT Number of rw_locks protecting buffer pool page_hash. Rounded up to the next power of 2
NUMERIC_MIN_VALUE 1
NUMERIC_MAX_VALUE 1024
NUMERIC_BLOCK_SIZE 0
ENUM_VALUE_LIST NULL
READ_ONLY YES
COMMAND_LINE_ARGUMENT OPTIONAL
VARIABLE_NAME INNODB_PAGE_SIZE
SESSION_VALUE NULL
DEFAULT_VALUE 16384
......
--source include/have_innodb.inc
--source include/have_debug.inc
#
# exists as global only
#
select @@global.innodb_page_hash_locks between 1 and 1024;
select @@global.innodb_page_hash_locks;
--error ER_INCORRECT_GLOBAL_LOCAL_VAR
select @@session.innodb_page_hash_locks;
show global variables like 'innodb_page_hash_locks';
show session variables like 'innodb_page_hash_locks';
--disable_warnings
select * from information_schema.global_variables where variable_name='innodb_page_hash_locks';
select * from information_schema.session_variables where variable_name='innodb_page_hash_locks';
--enable_warnings
#
# show that it's read-only
#
--error ER_INCORRECT_GLOBAL_LOCAL_VAR
set global innodb_page_hash_locks=1;
--error ER_INCORRECT_GLOBAL_LOCAL_VAR
set @@session.innodb_page_hash_locks='some';
......@@ -1085,15 +1085,15 @@ btr_search_guess_on_hash(
buf_block_t* block = buf_pool.block_from_ahi(rec);
if (!ahi_latch) {
rw_lock_t* hash_lock = buf_pool.hash_lock_get(
page_hash_latch* hash_lock = buf_pool.hash_lock_get(
block->page.id());
rw_lock_s_lock(hash_lock);
hash_lock->read_lock();
if (block->page.state() == BUF_BLOCK_REMOVE_HASH) {
/* Another thread is just freeing the block
from the LRU list of the buffer pool: do not
try to access this page. */
rw_lock_s_unlock(hash_lock);
hash_lock->read_unlock();
goto fail;
}
......@@ -1104,7 +1104,7 @@ btr_search_guess_on_hash(
DBUG_ASSERT(fail || block->page.status != buf_page_t::FREED);
buf_block_buf_fix_inc(block, __FILE__, __LINE__);
rw_lock_s_unlock(hash_lock);
hash_lock->read_unlock();
block->page.set_accessed();
buf_page_make_young_if_needed(&block->page);
......
......@@ -555,8 +555,8 @@ static bool buf_buddy_relocate(void* src, void* dst, ulint i, bool force)
return false;
}
rw_lock_t * hash_lock = buf_pool.hash_lock_get_low(fold);
rw_lock_x_lock(hash_lock);
page_hash_latch *hash_lock = buf_pool.page_hash.lock_get(fold);
hash_lock->write_lock();
if (bpage->can_relocate()) {
/* Relocate the compressed page. */
......@@ -567,7 +567,7 @@ static bool buf_buddy_relocate(void* src, void* dst, ulint i, bool force)
memcpy(dst, src, size);
bpage->zip.data = reinterpret_cast<page_zip_t*>(dst);
rw_lock_x_unlock(hash_lock);
hash_lock->write_unlock();
buf_buddy_mem_invalid(
reinterpret_cast<buf_buddy_free_t*>(src), i);
......@@ -578,7 +578,7 @@ static bool buf_buddy_relocate(void* src, void* dst, ulint i, bool force)
return(true);
}
rw_lock_x_unlock(hash_lock);
hash_lock->write_unlock();
return(false);
}
......
This diff is collapsed.
......@@ -141,7 +141,7 @@ caller needs to free the page to the free list
@retval false if BUF_BLOCK_ZIP_PAGE was removed from page_hash. In
this case the block is already returned to the buddy allocator. */
static bool buf_LRU_block_remove_hashed(buf_page_t *bpage, const page_id_t id,
rw_lock_t *hash_lock, bool zip);
page_hash_latch *hash_lock, bool zip);
/** Free a block to buf_pool */
static void buf_LRU_block_free_hashed_page(buf_block_t *block)
......@@ -1160,8 +1160,8 @@ bool buf_LRU_free_page(buf_page_t *bpage, bool zip)
bpage->can_relocate() from changing due to a concurrent
execution of buf_page_get_low(). */
const ulint fold = id.fold();
rw_lock_t* hash_lock = buf_pool.hash_lock_get_low(fold);
rw_lock_x_lock(hash_lock);
page_hash_latch* hash_lock = buf_pool.page_hash.lock_get(fold);
hash_lock->write_lock();
if (UNIV_UNLIKELY(!bpage->can_relocate())) {
/* Do not free buffer fixed and I/O-fixed blocks. */
......@@ -1178,7 +1178,7 @@ bool buf_LRU_free_page(buf_page_t *bpage, bool zip)
} else if (bpage->oldest_modification()
&& bpage->state() != BUF_BLOCK_FILE_PAGE) {
func_exit:
rw_lock_x_unlock(hash_lock);
hash_lock->write_unlock();
return(false);
} else if (bpage->state() == BUF_BLOCK_FILE_PAGE) {
......@@ -1201,10 +1201,6 @@ bool buf_LRU_free_page(buf_page_t *bpage, bool zip)
return(true);
}
/* buf_LRU_block_remove_hashed() releases the hash_lock */
ut_ad(!rw_lock_own_flagged(hash_lock,
RW_LOCK_FLAG_X | RW_LOCK_FLAG_S));
/* We have just freed a BUF_BLOCK_FILE_PAGE. If b != nullptr
then it was a compressed page with an uncompressed frame and
we are interested in freeing only the uncompressed frame.
......@@ -1215,7 +1211,7 @@ bool buf_LRU_free_page(buf_page_t *bpage, bool zip)
if (UNIV_LIKELY_NULL(b)) {
buf_page_t* prev_b = UT_LIST_GET_PREV(LRU, b);
rw_lock_x_lock(hash_lock);
hash_lock->write_lock();
ut_ad(!buf_pool.page_hash_get_low(id, fold));
ut_ad(b->zip_size());
......@@ -1301,7 +1297,7 @@ bool buf_LRU_free_page(buf_page_t *bpage, bool zip)
decompressing the block while we release
hash_lock. */
b->set_io_fix(BUF_IO_PIN);
rw_lock_x_unlock(hash_lock);
hash_lock->write_unlock();
}
mutex_exit(&buf_pool.mutex);
......@@ -1405,10 +1401,10 @@ caller needs to free the page to the free list
@retval false if BUF_BLOCK_ZIP_PAGE was removed from page_hash. In
this case the block is already returned to the buddy allocator. */
static bool buf_LRU_block_remove_hashed(buf_page_t *bpage, const page_id_t id,
rw_lock_t *hash_lock, bool zip)
page_hash_latch *hash_lock, bool zip)
{
ut_ad(mutex_own(&buf_pool.mutex));
ut_ad(rw_lock_own(hash_lock, RW_LOCK_X));
ut_ad(hash_lock->is_write_locked());
ut_a(bpage->io_fix() == BUF_IO_NONE);
ut_a(!bpage->buf_fix_count());
......@@ -1501,7 +1497,7 @@ static bool buf_LRU_block_remove_hashed(buf_page_t *bpage, const page_id_t id,
#ifdef UNIV_DEBUG
UT_LIST_REMOVE(buf_pool.zip_clean, bpage);
#endif /* UNIV_DEBUG */
rw_lock_x_unlock(hash_lock);
hash_lock->write_unlock();
buf_pool_mutex_exit_forbid();
buf_buddy_free(bpage->zip.data, bpage->zip_size());
......@@ -1542,7 +1538,7 @@ static bool buf_LRU_block_remove_hashed(buf_page_t *bpage, const page_id_t id,
and by the time we'll release it in the caller we'd
have inserted the compressed only descriptor in the
page_hash. */
rw_lock_x_unlock(hash_lock);
hash_lock->write_unlock();
if (zip && bpage->zip.data) {
/* Free the compressed page. */
......@@ -1578,20 +1574,15 @@ static bool buf_LRU_block_remove_hashed(buf_page_t *bpage, const page_id_t id,
@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)
{
while (bpage->buf_fix_count())
{
/* Wait for other threads to release the fix count
before releasing the bpage from LRU list. */
ut_delay(1);
}
(void) LF_BACKOFF();
if (buf_LRU_block_remove_hashed(bpage, id, hash_lock, true))
buf_LRU_block_free_hashed_page(reinterpret_cast<buf_block_t*>(bpage));
/* buf_LRU_block_remove_hashed() releases hash_lock */
ut_ad(!rw_lock_own_flagged(hash_lock, RW_LOCK_FLAG_X | RW_LOCK_FLAG_S));
}
/** Update buf_pool.LRU_old_ratio.
......
......@@ -53,7 +53,7 @@ that the block has been replaced with the real block.
@param watch sentinel */
inline void buf_pool_t::watch_remove(buf_page_t *watch)
{
ut_ad(rw_lock_own(hash_lock_get(watch->id()), RW_LOCK_X));
ut_ad(hash_lock_get(watch->id())->is_write_locked());
ut_a(watch_is_sentinel(*watch));
if (watch->buf_fix_count())
{
......@@ -125,14 +125,14 @@ static buf_page_t* buf_page_init_for_read(ulint mode, const page_id_t page_id,
/* We must acquire hash_lock this early to prevent
a race condition with buf_pool_t::watch_remove() */
rw_lock_t *hash_lock= buf_pool.hash_lock_get_low(fold);
rw_lock_x_lock(hash_lock);
page_hash_latch *hash_lock= buf_pool.page_hash.lock_get(fold);
hash_lock->write_lock();
buf_page_t *hash_page= buf_pool.page_hash_get_low(page_id, fold);
if (hash_page && !buf_pool.watch_is_sentinel(*hash_page))
{
/* The page is already in the buffer pool. */
rw_lock_x_unlock(hash_lock);
hash_lock->write_unlock();
if (block)
{
rw_lock_x_unlock_gen(&block->lock, BUF_IO_READ);
......@@ -160,7 +160,7 @@ static buf_page_t* buf_page_init_for_read(ulint mode, const page_id_t page_id,
ut_ad(!block->page.in_page_hash);
ut_d(block->page.in_page_hash= true);
HASH_INSERT(buf_page_t, hash, &buf_pool.page_hash, fold, bpage);
rw_lock_x_unlock(hash_lock);
hash_lock->write_unlock();
/* The block must be put to the LRU list, to the old blocks */
buf_LRU_add_block(bpage, true/* to old blocks */);
......@@ -184,7 +184,7 @@ static buf_page_t* buf_page_init_for_read(ulint mode, const page_id_t page_id,
}
else
{
rw_lock_x_unlock(hash_lock);
hash_lock->write_unlock();
/* The compressed page must be allocated before the
control block (bpage), in order to avoid the
......@@ -193,7 +193,7 @@ static buf_page_t* buf_page_init_for_read(ulint mode, const page_id_t page_id,
bool lru= false;
void *data= buf_buddy_alloc(zip_size, &lru);
rw_lock_x_lock(hash_lock);
hash_lock->write_lock();
/* If buf_buddy_alloc() allocated storage from the LRU list,
it released and reacquired buf_pool.mutex. Thus, we must
......@@ -205,7 +205,7 @@ static buf_page_t* buf_page_init_for_read(ulint mode, const page_id_t page_id,
if (UNIV_UNLIKELY(hash_page && !buf_pool.watch_is_sentinel(*hash_page)))
{
/* The block was added by some other thread. */
rw_lock_x_unlock(hash_lock);
hash_lock->write_unlock();
buf_buddy_free(data, zip_size);
goto func_exit;
}
......@@ -234,7 +234,7 @@ static buf_page_t* buf_page_init_for_read(ulint mode, const page_id_t page_id,
ut_d(bpage->in_page_hash= true);
HASH_INSERT(buf_page_t, hash, &buf_pool.page_hash, fold, bpage);
bpage->set_io_fix(BUF_IO_READ);
rw_lock_x_unlock(hash_lock);
hash_lock->write_unlock();
/* The block must be put to the LRU list, to the old blocks.
The zip size is already set into the page zip */
......@@ -253,7 +253,6 @@ static buf_page_t* buf_page_init_for_read(ulint mode, const page_id_t page_id,
if (mode == BUF_READ_IBUF_PAGES_ONLY)
ibuf_mtr_commit(&mtr);
ut_ad(!rw_lock_own_flagged(hash_lock, RW_LOCK_FLAG_X | RW_LOCK_FLAG_S));
ut_ad(!bpage || bpage->in_file());
return bpage;
......@@ -426,10 +425,10 @@ buf_read_ahead_random(const page_id_t page_id, ulint zip_size, bool ibuf)
for (page_id_t i= low; i < high; ++i)
{
const ulint fold= i.fold();
rw_lock_t *hash_lock= buf_pool.page_hash_lock<false>(fold);
const buf_page_t* bpage= buf_pool.page_hash_get_low(i, fold);
page_hash_latch *hash_lock= buf_pool.page_hash.lock<false>(fold);
const buf_page_t *bpage= buf_pool.page_hash_get_low(i, fold);
bool found= bpage && bpage->is_accessed() && buf_page_peek_if_young(bpage);
rw_lock_s_unlock(hash_lock);
hash_lock->read_unlock();
if (found && !--count)
goto read_ahead;
}
......@@ -620,7 +619,7 @@ buf_read_ahead_linear(const page_id_t page_id, ulint zip_size, bool ibuf)
for (page_id_t i= low; i != high_1; ++i)
{
const ulint fold= i.fold();
rw_lock_t *hash_lock= buf_pool.page_hash_lock<false>(fold);
page_hash_latch *hash_lock= buf_pool.page_hash.lock<false>(fold);
const buf_page_t* bpage= buf_pool.page_hash_get_low(i, fold);
if (i == page_id)
{
......@@ -632,7 +631,7 @@ buf_read_ahead_linear(const page_id_t page_id, ulint zip_size, bool ibuf)
if (!bpage)
{
hard_fail:
rw_lock_s_unlock(hash_lock);
hash_lock->read_unlock();
space->release();
return 0;
}
......@@ -673,7 +672,7 @@ buf_read_ahead_linear(const page_id_t page_id, ulint zip_size, bool ibuf)
else if (!bpage)
{
failed:
rw_lock_s_unlock(hash_lock);
hash_lock->read_unlock();
if (--count)
continue;
space->release();
......@@ -694,7 +693,7 @@ buf_read_ahead_linear(const page_id_t page_id, ulint zip_size, bool ibuf)
prev_accessed= accessed;
if (fail)
goto failed;
rw_lock_s_unlock(hash_lock);
hash_lock->read_unlock();
}
/* If we got this far, read-ahead can be sensible: do it */
......
......@@ -590,8 +590,7 @@ static PSI_rwlock_info all_innodb_rwlocks[] = {
PSI_RWLOCK_KEY(trx_purge_latch),
PSI_RWLOCK_KEY(index_tree_rw_lock),
PSI_RWLOCK_KEY(index_online_log),
PSI_RWLOCK_KEY(dict_table_stats),
PSI_RWLOCK_KEY(hash_table_locks)
PSI_RWLOCK_KEY(dict_table_stats)
};
# endif /* UNIV_PFS_RWLOCK */
......@@ -19500,11 +19499,6 @@ static MYSQL_SYSVAR_ULONG(buffer_pool_chunk_size, srv_buf_pool_chunk_unit,
128 * 1024 * 1024, 1024 * 1024, LONG_MAX, 1024 * 1024);
#if defined UNIV_DEBUG || defined UNIV_PERF_DEBUG
static MYSQL_SYSVAR_ULONG(page_hash_locks, srv_n_page_hash_locks,
PLUGIN_VAR_OPCMDARG | PLUGIN_VAR_READONLY,
"Number of rw_locks protecting buffer pool page_hash. Rounded up to the next power of 2",
NULL, NULL, 64, 1, MAX_PAGE_HASH_LOCKS, 0);
static MYSQL_SYSVAR_ULONG(doublewrite_batch_size, srv_doublewrite_batch_size,
PLUGIN_VAR_OPCMDARG | PLUGIN_VAR_READONLY,
"Number of pages reserved in doublewrite buffer for batch flushing",
......@@ -20393,7 +20387,6 @@ static struct st_mysql_sys_var* innobase_system_variables[]= {
MYSQL_SYSVAR(merge_threshold_set_all_debug),
#endif /* UNIV_DEBUG */
#if defined UNIV_DEBUG || defined UNIV_PERF_DEBUG
MYSQL_SYSVAR(page_hash_locks),
MYSQL_SYSVAR(doublewrite_batch_size),
#endif /* defined UNIV_DEBUG || defined UNIV_PERF_DEBUG */
MYSQL_SYSVAR(status_output),
......
This diff is collapsed.
......@@ -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
......
......@@ -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 */
......@@ -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) {\
......
/*****************************************************************************
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; }
};
......@@ -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 */
......
......@@ -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. */
......
......@@ -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.
......
......@@ -207,9 +207,6 @@ const ulint srv_buf_pool_min_size = 5 * 1024 * 1024;
const ulint srv_buf_pool_def_size = 128 * 1024 * 1024;
/** Requested buffer pool chunk size */
ulong srv_buf_pool_chunk_unit;
/** innodb_page_hash_locks (a debug-only parameter);
number of locks to protect buf_pool.page_hash */
ulong srv_n_page_hash_locks = 64;
/** innodb_lru_scan_depth; number of blocks scanned in LRU flush batch */
ulong srv_LRU_scan_depth;
/** innodb_flush_neighbors; whether or not to flush neighbors of a block */
......
......@@ -777,7 +777,7 @@ LatchDebug::check_order(
case SYNC_POOL:
case SYNC_POOL_MANAGER:
case SYNC_RECV_WRITER:
case SYNC_BUF_PAGE_HASH:
basic_check(latches, level, level);
break;
......@@ -825,14 +825,6 @@ LatchDebug::check_order(
basic_check(latches, level, level - 1);
break;
case SYNC_BUF_PAGE_HASH:
/* Multiple page_hash locks are only allowed during
buf_pool.validate() and that is where buf_pool mutex is already
held. */
/* Fall through */
case SYNC_REC_LOCK:
if (find(latches, SYNC_LOCK_SYS) != 0) {
......@@ -1453,9 +1445,6 @@ sync_latch_meta_init()
LATCH_ADD_RWLOCK(DICT_TABLE_STATS, SYNC_INDEX_TREE,
dict_table_stats_key);
LATCH_ADD_RWLOCK(HASH_TABLE_RW_LOCK, SYNC_BUF_PAGE_HASH,
hash_table_locks_key);
LATCH_ADD_MUTEX(SYNC_DEBUG_MUTEX, SYNC_NO_ORDER_CHECK,
PFS_NOT_INSTRUMENTED);
......
......@@ -102,7 +102,6 @@ mysql_pfs_key_t buf_block_debug_latch_key;
# endif /* UNIV_DEBUG */
mysql_pfs_key_t dict_operation_lock_key;
mysql_pfs_key_t dict_table_stats_key;
mysql_pfs_key_t hash_table_locks_key;
mysql_pfs_key_t index_tree_rw_lock_key;
mysql_pfs_key_t index_online_log_key;
mysql_pfs_key_t fil_space_latch_key;
......
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment