MDEV-20612: Partition lock_sys.latch
We replace the old lock_sys.mutex (which was renamed to lock_sys.latch) with a combination of a global lock_sys.latch and table or page hash lock mutexes. The global lock_sys.latch can be acquired in exclusive mode, or it can be acquired in shared mode and another mutex will be acquired to protect the locks for a particular page or a table. This is inspired by mysql/mysql-server@1d259b87a6 but the optimization of lock_release() will be done in the next commit. Also, we will interleave mutexes with the hash table elements, similar to how buf_pool.page_hash was optimized in commit 5155a300fab85e97217c75e3ba3c3ce78082dd8a (MDEV-22871). dict_table_t::autoinc_trx: Use Atomic_relaxed. dict_table_t::autoinc_mutex: Use srw_mutex in order to reduce the memory footprint. On 64-bit Linux or OpenBSD, both this and the new dict_table_t::lock_mutex should be 32 bits and be stored in the same 64-bit word. On Microsoft Windows, the underlying SRWLOCK is 32 or 64 bits, and on other systems, sizeof(pthread_mutex_t) can be much larger. ib_lock_t::trx_locks, trx_lock_t::trx_locks: Document the new rules. Writers must assert lock_sys.is_writer() || trx->mutex_is_owner(). LockGuard: A RAII wrapper for acquiring a page hash table lock. LockGGuard: Like LockGuard, but when Galera Write-Set Replication is enabled, we must acquire all shards, for updating arbitrary trx_locks. LockMultiGuard: A RAII wrapper for acquiring two page hash table locks. lock_rec_create_wsrep(), lock_table_create_wsrep(): Special Galera conflict resolution in non-inlined functions in order to keep the common code paths shorter. lock_sys_t::prdt_page_free_from_discard(): Refactored from lock_prdt_page_free_from_discard() and lock_rec_free_all_from_discard_page(). trx_t::commit_tables(): Replaces trx_update_mod_tables_timestamp(). lock_release(): Let trx_t::commit_tables() invalidate the query cache for those tables that were actually modified by the transaction. Merge lock_check_dict_lock() to lock_release(). We must never release lock_sys.latch while holding any lock_sys_t::hash_latch. Failure to do that could lead to memory corruption if the buffer pool is resized between the time lock_sys.latch is released and the hash_latch is released.
This commit is contained in:
parent
b01d8e1a33
commit
b08448de64
@ -3355,11 +3355,10 @@ btr_lift_page_up(
|
||||
const page_id_t id{block->page.id()};
|
||||
/* Free predicate page locks on the block */
|
||||
if (index->is_spatial()) {
|
||||
LockMutexGuard g{SRW_LOCK_CALL};
|
||||
lock_prdt_page_free_from_discard(
|
||||
id, &lock_sys.prdt_page_hash);
|
||||
lock_sys.prdt_page_free_from_discard(id);
|
||||
} else {
|
||||
lock_update_copy_and_discard(*father_block, id);
|
||||
}
|
||||
lock_update_copy_and_discard(*father_block, id);
|
||||
}
|
||||
|
||||
/* Go upward to root page, decrementing levels by one. */
|
||||
@ -3609,10 +3608,7 @@ retry:
|
||||
}
|
||||
|
||||
/* No GAP lock needs to be worrying about */
|
||||
LockMutexGuard g{SRW_LOCK_CALL};
|
||||
lock_prdt_page_free_from_discard(
|
||||
id, &lock_sys.prdt_page_hash);
|
||||
lock_rec_free_all_from_discard_page(id);
|
||||
lock_sys.prdt_page_free_from_discard(id);
|
||||
} else {
|
||||
btr_cur_node_ptr_delete(&father_cursor, mtr);
|
||||
if (!dict_table_is_locking_disabled(index->table)) {
|
||||
@ -3762,10 +3758,7 @@ retry:
|
||||
merge_page, mtr);
|
||||
}
|
||||
const page_id_t id{block->page.id()};
|
||||
LockMutexGuard g{SRW_LOCK_CALL};
|
||||
lock_prdt_page_free_from_discard(
|
||||
id, &lock_sys.prdt_page_hash);
|
||||
lock_rec_free_all_from_discard_page(id);
|
||||
lock_sys.prdt_page_free_from_discard(id);
|
||||
} else {
|
||||
|
||||
compressed = btr_cur_pessimistic_delete(&err, TRUE,
|
||||
|
@ -1999,12 +1999,13 @@ retry_page_get:
|
||||
trx_t* trx = thr_get_trx(cursor->thr);
|
||||
lock_prdt_t prdt;
|
||||
|
||||
{
|
||||
LockMutexGuard g{SRW_LOCK_CALL};
|
||||
lock_init_prdt_from_mbr(
|
||||
&prdt, &cursor->rtr_info->mbr, mode,
|
||||
trx->lock.lock_heap);
|
||||
}
|
||||
lock_sys.rd_lock(SRW_LOCK_CALL);
|
||||
trx->mutex_lock();
|
||||
lock_init_prdt_from_mbr(
|
||||
&prdt, &cursor->rtr_info->mbr, mode,
|
||||
trx->lock.lock_heap);
|
||||
lock_sys.rd_unlock();
|
||||
trx->mutex_unlock();
|
||||
|
||||
if (rw_latch == RW_NO_LATCH && height != 0) {
|
||||
block->lock.s_lock();
|
||||
@ -3242,8 +3243,7 @@ btr_cur_ins_lock_and_undo(
|
||||
/* Use on stack MBR variable to test if a lock is
|
||||
needed. If so, the predicate (MBR) will be allocated
|
||||
from lock heap in lock_prdt_insert_check_and_lock() */
|
||||
lock_init_prdt_from_mbr(
|
||||
&prdt, &mbr, 0, NULL);
|
||||
lock_init_prdt_from_mbr(&prdt, &mbr, 0, nullptr);
|
||||
|
||||
if (dberr_t err = lock_prdt_insert_check_and_lock(
|
||||
rec, btr_cur_get_block(cursor),
|
||||
|
@ -1247,7 +1247,8 @@ inline void dict_sys_t::add(dict_table_t* table)
|
||||
|
||||
ulint fold = ut_fold_string(table->name.m_name);
|
||||
|
||||
new (&table->autoinc_mutex) std::mutex();
|
||||
table->autoinc_mutex.init();
|
||||
table->lock_mutex_init();
|
||||
|
||||
/* Look for a table with the same name: error if such exists */
|
||||
{
|
||||
@ -2038,7 +2039,8 @@ void dict_sys_t::remove(dict_table_t* table, bool lru, bool keep)
|
||||
UT_DELETE(table->vc_templ);
|
||||
}
|
||||
|
||||
table->autoinc_mutex.~mutex();
|
||||
table->autoinc_mutex.destroy();
|
||||
table->lock_mutex_destroy();
|
||||
|
||||
if (keep) {
|
||||
return;
|
||||
|
@ -1197,9 +1197,7 @@ rtr_check_discard_page(
|
||||
|
||||
mysql_mutex_unlock(&index->rtr_track->rtr_active_mutex);
|
||||
|
||||
LockMutexGuard g{SRW_LOCK_CALL};
|
||||
lock_prdt_page_free_from_discard(id, &lock_sys.prdt_hash);
|
||||
lock_prdt_page_free_from_discard(id, &lock_sys.prdt_page_hash);
|
||||
lock_sys.prdt_page_free_from_discard(id, true);
|
||||
}
|
||||
|
||||
/** Structure acts as functor to get the optimistic access of the page.
|
||||
|
@ -2253,7 +2253,7 @@ ha_innobase::innobase_reset_autoinc(
|
||||
if (error == DB_SUCCESS) {
|
||||
|
||||
dict_table_autoinc_initialize(m_prebuilt->table, autoinc);
|
||||
m_prebuilt->table->autoinc_mutex.unlock();
|
||||
m_prebuilt->table->autoinc_mutex.wr_unlock();
|
||||
}
|
||||
|
||||
return(error);
|
||||
@ -2685,13 +2685,13 @@ static bool innobase_query_caching_table_check_low(
|
||||
For read-only transaction: should satisfy (1) and (3)
|
||||
For read-write transaction: should satisfy (1), (2), (3) */
|
||||
|
||||
if (trx->id && trx->id < table->query_cache_inv_trx_id) {
|
||||
const trx_id_t inv = table->query_cache_inv_trx_id;
|
||||
|
||||
if (trx->id && trx->id < inv) {
|
||||
return false;
|
||||
}
|
||||
|
||||
if (trx->read_view.is_open()
|
||||
&& trx->read_view.low_limit_id()
|
||||
< table->query_cache_inv_trx_id) {
|
||||
if (trx->read_view.is_open() && trx->read_view.low_limit_id() < inv) {
|
||||
return false;
|
||||
}
|
||||
|
||||
@ -5359,7 +5359,7 @@ initialize_auto_increment(dict_table_t* table, const Field* field)
|
||||
|
||||
const unsigned col_no = innodb_col_no(field);
|
||||
|
||||
table->autoinc_mutex.lock();
|
||||
table->autoinc_mutex.wr_lock();
|
||||
|
||||
table->persistent_autoinc = static_cast<uint16_t>(
|
||||
dict_table_get_nth_col_pos(table, col_no, NULL) + 1)
|
||||
@ -5390,7 +5390,7 @@ initialize_auto_increment(dict_table_t* table, const Field* field)
|
||||
innobase_get_int_col_max_value(field));
|
||||
}
|
||||
|
||||
table->autoinc_mutex.unlock();
|
||||
table->autoinc_mutex.wr_unlock();
|
||||
}
|
||||
|
||||
/** Open an InnoDB table
|
||||
@ -7192,7 +7192,7 @@ ha_innobase::innobase_lock_autoinc(void)
|
||||
switch (innobase_autoinc_lock_mode) {
|
||||
case AUTOINC_NO_LOCKING:
|
||||
/* Acquire only the AUTOINC mutex. */
|
||||
m_prebuilt->table->autoinc_mutex.lock();
|
||||
m_prebuilt->table->autoinc_mutex.wr_lock();
|
||||
break;
|
||||
|
||||
case AUTOINC_NEW_STYLE_LOCKING:
|
||||
@ -7206,14 +7206,14 @@ ha_innobase::innobase_lock_autoinc(void)
|
||||
case SQLCOM_REPLACE:
|
||||
case SQLCOM_END: // RBR event
|
||||
/* Acquire the AUTOINC mutex. */
|
||||
m_prebuilt->table->autoinc_mutex.lock();
|
||||
m_prebuilt->table->autoinc_mutex.wr_lock();
|
||||
/* We need to check that another transaction isn't
|
||||
already holding the AUTOINC lock on the table. */
|
||||
if (!m_prebuilt->table->n_waiting_or_granted_auto_inc_locks) {
|
||||
/* Do not fall back to old style locking. */
|
||||
DBUG_RETURN(error);
|
||||
}
|
||||
m_prebuilt->table->autoinc_mutex.unlock();
|
||||
m_prebuilt->table->autoinc_mutex.wr_unlock();
|
||||
}
|
||||
/* Use old style locking. */
|
||||
/* fall through */
|
||||
@ -7225,7 +7225,7 @@ ha_innobase::innobase_lock_autoinc(void)
|
||||
if (error == DB_SUCCESS) {
|
||||
|
||||
/* Acquire the AUTOINC mutex. */
|
||||
m_prebuilt->table->autoinc_mutex.lock();
|
||||
m_prebuilt->table->autoinc_mutex.wr_lock();
|
||||
}
|
||||
break;
|
||||
|
||||
@ -7253,7 +7253,7 @@ ha_innobase::innobase_set_max_autoinc(
|
||||
if (error == DB_SUCCESS) {
|
||||
|
||||
dict_table_autoinc_update_if_greater(m_prebuilt->table, auto_inc);
|
||||
m_prebuilt->table->autoinc_mutex.unlock();
|
||||
m_prebuilt->table->autoinc_mutex.wr_unlock();
|
||||
}
|
||||
|
||||
return(error);
|
||||
@ -12634,7 +12634,7 @@ create_table_info_t::create_table_update_dict()
|
||||
autoinc = 1;
|
||||
}
|
||||
|
||||
innobase_table->autoinc_mutex.lock();
|
||||
innobase_table->autoinc_mutex.wr_lock();
|
||||
dict_table_autoinc_initialize(innobase_table, autoinc);
|
||||
|
||||
if (innobase_table->is_temporary()) {
|
||||
@ -12662,7 +12662,7 @@ create_table_info_t::create_table_update_dict()
|
||||
}
|
||||
}
|
||||
|
||||
innobase_table->autoinc_mutex.unlock();
|
||||
innobase_table->autoinc_mutex.wr_unlock();
|
||||
}
|
||||
|
||||
innobase_parse_hint_from_comment(m_thd, innobase_table, m_form->s);
|
||||
@ -15910,7 +15910,7 @@ ha_innobase::innobase_get_autoinc(
|
||||
/* It should have been initialized during open. */
|
||||
if (*value == 0) {
|
||||
m_prebuilt->autoinc_error = DB_UNSUPPORTED;
|
||||
m_prebuilt->table->autoinc_mutex.unlock();
|
||||
m_prebuilt->table->autoinc_mutex.wr_unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@ -15934,7 +15934,7 @@ ha_innobase::innobase_peek_autoinc(void)
|
||||
|
||||
innodb_table = m_prebuilt->table;
|
||||
|
||||
innodb_table->autoinc_mutex.lock();
|
||||
innodb_table->autoinc_mutex.wr_lock();
|
||||
|
||||
auto_inc = dict_table_autoinc_read(innodb_table);
|
||||
|
||||
@ -15943,7 +15943,7 @@ ha_innobase::innobase_peek_autoinc(void)
|
||||
" '" << innodb_table->name << "'";
|
||||
}
|
||||
|
||||
innodb_table->autoinc_mutex.unlock();
|
||||
innodb_table->autoinc_mutex.wr_unlock();
|
||||
|
||||
return(auto_inc);
|
||||
}
|
||||
@ -16050,7 +16050,7 @@ ha_innobase::get_auto_increment(
|
||||
/* Out of range number. Let handler::update_auto_increment()
|
||||
take care of this */
|
||||
m_prebuilt->autoinc_last_value = 0;
|
||||
m_prebuilt->table->autoinc_mutex.unlock();
|
||||
m_prebuilt->table->autoinc_mutex.wr_unlock();
|
||||
*nb_reserved_values= 0;
|
||||
return;
|
||||
}
|
||||
@ -16093,7 +16093,7 @@ ha_innobase::get_auto_increment(
|
||||
m_prebuilt->autoinc_offset = offset;
|
||||
m_prebuilt->autoinc_increment = increment;
|
||||
|
||||
m_prebuilt->table->autoinc_mutex.unlock();
|
||||
m_prebuilt->table->autoinc_mutex.wr_unlock();
|
||||
}
|
||||
|
||||
/*******************************************************************//**
|
||||
@ -17998,7 +17998,6 @@ 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.assert_locked();
|
||||
ut_ad(victim_trx->mutex_is_owner());
|
||||
|
||||
DBUG_ENTER("wsrep_innobase_kill_one_trx");
|
||||
@ -18059,6 +18058,7 @@ int wsrep_innobase_kill_one_trx(THD *bf_thd, trx_t *victim_trx, bool signal)
|
||||
} else if (victim_trx->lock.wait_lock) {
|
||||
mysql_mutex_lock(&lock_sys.wait_mutex);
|
||||
if (lock_t* wait_lock = victim_trx->lock.wait_lock) {
|
||||
lock_sys.assert_locked(*wait_lock);
|
||||
DBUG_ASSERT(victim_trx->is_wsrep());
|
||||
WSREP_DEBUG("victim has wait flag: %lu",
|
||||
thd_get_thread_id(thd));
|
||||
|
@ -3280,8 +3280,8 @@ commit_exit:
|
||||
ibuf_mtr_commit(&bitmap_mtr);
|
||||
goto fail_exit;
|
||||
} else {
|
||||
LockMutexGuard g{SRW_LOCK_CALL};
|
||||
if (lock_sys.get_first(page_id)) {
|
||||
LockGuard g{lock_sys.rec_hash, page_id};
|
||||
if (lock_sys.rec_hash.get_first(page_id)) {
|
||||
goto commit_exit;
|
||||
}
|
||||
}
|
||||
|
@ -1958,6 +1958,26 @@ struct dict_table_t {
|
||||
/** Clear the table when rolling back TRX_UNDO_EMPTY */
|
||||
void clear(que_thr_t *thr);
|
||||
|
||||
#ifdef UNIV_DEBUG
|
||||
/** @return whether the current thread holds the lock_mutex */
|
||||
bool lock_mutex_is_owner() const
|
||||
{ return lock_mutex_owner == os_thread_get_curr_id(); }
|
||||
#endif /* UNIV_DEBUG */
|
||||
void lock_mutex_init() { lock_mutex.init(); }
|
||||
void lock_mutex_destroy() { lock_mutex.destroy(); }
|
||||
/** Acquire lock_mutex */
|
||||
void lock_mutex_lock()
|
||||
{
|
||||
ut_ad(!lock_mutex_is_owner());
|
||||
lock_mutex.wr_lock();
|
||||
ut_ad(!lock_mutex_owner.exchange(os_thread_get_curr_id()));
|
||||
}
|
||||
/** Release lock_mutex */
|
||||
void lock_mutex_unlock()
|
||||
{
|
||||
ut_ad(lock_mutex_owner.exchange(0) == os_thread_get_curr_id());
|
||||
lock_mutex.wr_unlock();
|
||||
}
|
||||
private:
|
||||
/** Initialize instant->field_map.
|
||||
@param[in] table table definition to copy from */
|
||||
@ -2118,19 +2138,13 @@ public:
|
||||
/** Maximum recursive level we support when loading tables chained
|
||||
together with FK constraints. If exceeds this level, we will stop
|
||||
loading child table into memory along with its parent table. */
|
||||
unsigned fk_max_recusive_level:8;
|
||||
byte fk_max_recusive_level;
|
||||
|
||||
/** Count of how many foreign key check operations are currently being
|
||||
performed on the table. We cannot drop the table while there are
|
||||
foreign key checks running on it. */
|
||||
Atomic_counter<int32_t> n_foreign_key_checks_running;
|
||||
|
||||
/** Transactions whose view low limit is greater than this number are
|
||||
not allowed to store to the MySQL query cache or retrieve from it.
|
||||
When a trx with undo logs commits, it sets this to the value of the
|
||||
transaction id. */
|
||||
trx_id_t query_cache_inv_trx_id;
|
||||
|
||||
/** Transaction id that last touched the table definition. Either when
|
||||
loading the definition or CREATE TABLE, or ALTER TABLE (prepare,
|
||||
commit, and rollback phases). */
|
||||
@ -2273,15 +2287,27 @@ public:
|
||||
from a select. */
|
||||
lock_t* autoinc_lock;
|
||||
|
||||
/** Mutex protecting the autoincrement counter. */
|
||||
std::mutex autoinc_mutex;
|
||||
/** Mutex protecting autoinc. */
|
||||
srw_mutex autoinc_mutex;
|
||||
private:
|
||||
/** Mutex protecting locks on this table. */
|
||||
srw_mutex lock_mutex;
|
||||
#ifdef UNIV_DEBUG
|
||||
/** The owner of lock_mutex (0 if none) */
|
||||
Atomic_relaxed<os_thread_id_t> lock_mutex_owner{0};
|
||||
#endif
|
||||
public:
|
||||
/** Autoinc counter value to give to the next inserted row. */
|
||||
uint64_t autoinc;
|
||||
|
||||
/** Autoinc counter value to give to the next inserted row. */
|
||||
ib_uint64_t autoinc;
|
||||
|
||||
/** The transaction that currently holds the the AUTOINC lock on this
|
||||
table. Protected by lock_sys.latch. */
|
||||
const trx_t* autoinc_trx;
|
||||
/** The transaction that currently holds the the AUTOINC lock on this table.
|
||||
Protected by lock_mutex.
|
||||
The thread that is executing autoinc_trx may read this field without
|
||||
holding a latch, in row_lock_table_autoinc_for_mysql().
|
||||
Only the autoinc_trx thread may clear this field; it cannot be
|
||||
modified on the behalf of a transaction that is being handled by a
|
||||
different thread. */
|
||||
Atomic_relaxed<const trx_t*> autoinc_trx;
|
||||
|
||||
/** Number of granted or pending autoinc_lock on this table. This
|
||||
value is set after acquiring lock_sys.latch but
|
||||
@ -2293,7 +2319,7 @@ public:
|
||||
/* @} */
|
||||
|
||||
/** Number of granted or pending LOCK_S or LOCK_X on the table.
|
||||
Protected by lock_sys.assert_locked(*this). */
|
||||
Protected by lock_mutex. */
|
||||
uint32_t n_lock_x_or_s;
|
||||
|
||||
/** FTS specific state variables. */
|
||||
@ -2304,22 +2330,29 @@ public:
|
||||
in X mode of this table's indexes. */
|
||||
ib_quiesce_t quiesce;
|
||||
|
||||
/** 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.
|
||||
Protected by LockGuard. */
|
||||
ulint n_rec_locks;
|
||||
/** 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.
|
||||
Modified when lock_sys.is_writer(), or
|
||||
lock_sys.assert_locked(page_id) and trx->mutex_is_owner() hold.
|
||||
@see trx_lock_t::trx_locks */
|
||||
Atomic_counter<uint32_t> n_rec_locks;
|
||||
|
||||
private:
|
||||
/** Count of how many handles are opened to this table. Dropping of the
|
||||
table is NOT allowed until this count gets to zero. MySQL does NOT
|
||||
itself check the number of open handles at DROP. */
|
||||
Atomic_counter<uint32_t> n_ref_count;
|
||||
/** Count of how many handles are opened to this table. Dropping of the
|
||||
table is NOT allowed until this count gets to zero. MySQL does NOT
|
||||
itself check the number of open handles at DROP. */
|
||||
Atomic_counter<uint32_t> n_ref_count;
|
||||
public:
|
||||
/** List of locks on the table. Protected by lock_sys.assert_locked(lock). */
|
||||
table_lock_list_t locks;
|
||||
|
||||
/** Timestamp of the last modification of this table. */
|
||||
time_t update_time;
|
||||
/** Timestamp of the last modification of this table. */
|
||||
Atomic_relaxed<time_t> update_time;
|
||||
/** Transactions whose view low limit is greater than this number are
|
||||
not allowed to store to the query cache or retrieve from it.
|
||||
When a trx with undo logs commits, it sets this to the value of the
|
||||
transaction id. */
|
||||
Atomic_relaxed<trx_id_t> query_cache_inv_trx_id;
|
||||
|
||||
#ifdef UNIV_DEBUG
|
||||
/** Value of 'magic_n'. */
|
||||
|
@ -1,7 +1,7 @@
|
||||
/*****************************************************************************
|
||||
|
||||
Copyright (c) 1997, 2016, 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
|
||||
@ -184,33 +184,6 @@ do { \
|
||||
} \
|
||||
} while (0)
|
||||
|
||||
/****************************************************************//**
|
||||
Move all hash table entries from OLD_TABLE to NEW_TABLE. */
|
||||
|
||||
#define HASH_MIGRATE(OLD_TABLE, NEW_TABLE, NODE_TYPE, PTR_NAME, FOLD_FUNC) \
|
||||
do {\
|
||||
ulint i2222;\
|
||||
ulint cell_count2222;\
|
||||
\
|
||||
cell_count2222 = (OLD_TABLE)->n_cells; \
|
||||
\
|
||||
for (i2222 = 0; i2222 < cell_count2222; i2222++) {\
|
||||
NODE_TYPE* node2222 = static_cast<NODE_TYPE*>(\
|
||||
HASH_GET_FIRST((OLD_TABLE), i2222));\
|
||||
\
|
||||
while (node2222) {\
|
||||
NODE_TYPE* next2222 = static_cast<NODE_TYPE*>(\
|
||||
node2222->PTR_NAME);\
|
||||
ulint fold2222 = FOLD_FUNC(node2222);\
|
||||
\
|
||||
HASH_INSERT(NODE_TYPE, PTR_NAME, (NEW_TABLE),\
|
||||
fold2222, node2222);\
|
||||
\
|
||||
node2222 = next2222;\
|
||||
}\
|
||||
}\
|
||||
} while (0)
|
||||
|
||||
/** Hash table with singly-linked overflow lists */
|
||||
struct hash_table_t
|
||||
{
|
||||
|
@ -406,14 +406,6 @@ lock_rec_unlock(
|
||||
and release possible other transactions waiting because of these locks. */
|
||||
void lock_release(trx_t* trx);
|
||||
|
||||
/*************************************************************//**
|
||||
Get the lock hash table */
|
||||
UNIV_INLINE
|
||||
hash_table_t*
|
||||
lock_hash_get(
|
||||
/*==========*/
|
||||
ulint mode); /*!< in: lock mode */
|
||||
|
||||
/**********************************************************************//**
|
||||
Looks for a set bit in a record lock bitmap. Returns ULINT_UNDEFINED,
|
||||
if none found.
|
||||
@ -578,28 +570,141 @@ struct lock_op_t{
|
||||
/** The lock system struct */
|
||||
class lock_sys_t
|
||||
{
|
||||
friend struct LockGuard;
|
||||
friend struct LockMultiGuard;
|
||||
friend struct LockGGuard;
|
||||
|
||||
/** Hash table latch */
|
||||
struct hash_latch
|
||||
#if defined SRW_LOCK_DUMMY && !defined _WIN32
|
||||
: private rw_lock
|
||||
{
|
||||
/** Wait for an exclusive lock */
|
||||
void wait();
|
||||
/** Acquire a lock */
|
||||
void acquire() { if (!write_trylock()) wait(); }
|
||||
/** Release a lock */
|
||||
void release();
|
||||
#else
|
||||
{
|
||||
private:
|
||||
srw_lock_low lock;
|
||||
public:
|
||||
/** Acquire a lock */
|
||||
void acquire() { lock.wr_lock(); }
|
||||
/** Release a lock */
|
||||
void release() { lock.wr_unlock(); }
|
||||
#endif
|
||||
#ifdef UNIV_DEBUG
|
||||
/** @return whether this latch is possibly held by any thread */
|
||||
bool is_locked() const
|
||||
{ return memcmp(this, field_ref_zero, sizeof *this); }
|
||||
#endif
|
||||
};
|
||||
static_assert(sizeof(hash_latch) <= sizeof(void*), "compatibility");
|
||||
|
||||
public:
|
||||
struct hash_table
|
||||
{
|
||||
/** Number of array[] elements per hash_latch.
|
||||
Must be one less than a power of 2. */
|
||||
static constexpr size_t ELEMENTS_PER_LATCH= CPU_LEVEL1_DCACHE_LINESIZE /
|
||||
sizeof(void*) - 1;
|
||||
|
||||
/** number of payload elements in array[]. Protected by lock_sys.latch. */
|
||||
ulint n_cells;
|
||||
/** the hash table, with pad(n_cells) elements, aligned to L1 cache size;
|
||||
in any hash chain, lock_t::is_waiting() entries must not precede
|
||||
granted locks */
|
||||
hash_cell_t *array;
|
||||
|
||||
/** Create the hash table.
|
||||
@param n the lower bound of n_cells */
|
||||
void create(ulint n);
|
||||
|
||||
/** Resize the hash table.
|
||||
@param n the lower bound of n_cells */
|
||||
void resize(ulint n);
|
||||
|
||||
/** Free the hash table. */
|
||||
void free() { aligned_free(array); array= nullptr; }
|
||||
|
||||
/** @return the index of an array element */
|
||||
inline ulint calc_hash(ulint fold) const;
|
||||
/** @return raw array index converted to padded index */
|
||||
static ulint pad(ulint h) { return 1 + (h / ELEMENTS_PER_LATCH) + h; }
|
||||
/** Get a latch. */
|
||||
inline hash_latch *lock_get(ulint fold) const;
|
||||
|
||||
#ifdef UNIV_DEBUG
|
||||
void assert_locked(const page_id_t id) const;
|
||||
#else
|
||||
void assert_locked(const page_id_t) const {}
|
||||
#endif
|
||||
|
||||
/** Get the first lock on a page.
|
||||
@param id page number
|
||||
@return first lock
|
||||
@retval nullptr if none exists */
|
||||
lock_t *get_first(const page_id_t id) const
|
||||
{
|
||||
assert_locked(id);
|
||||
for (auto lock= static_cast<lock_t*>(array[calc_hash(id.fold())].node);
|
||||
lock; lock= lock->hash)
|
||||
if (lock->un_member.rec_lock.page_id == id)
|
||||
return lock;
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
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. */
|
||||
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<hash_latch*>
|
||||
(&array[calc_hash(fold, n) & ~ELEMENTS_PER_LATCH]);
|
||||
}
|
||||
};
|
||||
|
||||
private:
|
||||
bool m_initialised;
|
||||
|
||||
/** mutex proteting the locks */
|
||||
MY_ALIGNED(CACHE_LINE_SIZE) srw_lock latch;
|
||||
MY_ALIGNED(CPU_LEVEL1_DCACHE_LINESIZE) 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
|
||||
#if defined SRW_LOCK_DUMMY && !defined _WIN32
|
||||
protected:
|
||||
/** mutex for hash_latch::wait() */
|
||||
pthread_mutex_t hash_mutex;
|
||||
/** condition variable for hash_latch::wait() */
|
||||
pthread_cond_t hash_cond;
|
||||
#endif
|
||||
public:
|
||||
/** record locks */
|
||||
hash_table_t rec_hash;
|
||||
hash_table rec_hash;
|
||||
/** predicate locks for SPATIAL INDEX */
|
||||
hash_table_t prdt_hash;
|
||||
hash_table prdt_hash;
|
||||
/** page locks for SPATIAL INDEX */
|
||||
hash_table_t prdt_page_hash;
|
||||
hash_table prdt_page_hash;
|
||||
/** number of deadlocks detected; protected by mutex */
|
||||
ulint deadlocks;
|
||||
|
||||
/** mutex covering lock waits; @see trx_lock_t::wait_lock */
|
||||
MY_ALIGNED(CACHE_LINE_SIZE) mysql_mutex_t wait_mutex;
|
||||
MY_ALIGNED(CPU_LEVEL1_DCACHE_LINESIZE) mysql_mutex_t wait_mutex;
|
||||
private:
|
||||
/** Pending number of lock waits; protected by wait_mutex */
|
||||
ulint wait_pending;
|
||||
@ -609,6 +714,7 @@ private:
|
||||
ulint wait_time;
|
||||
/** Longest wait time; protected by wait_mutex */
|
||||
ulint wait_time_max;
|
||||
|
||||
public:
|
||||
/**
|
||||
Constructor.
|
||||
@ -726,50 +832,56 @@ public:
|
||||
/** Longest wait time; protected by wait_mutex */
|
||||
ulint get_wait_time_max() const { return wait_time_max; }
|
||||
|
||||
/** @return the hash value for a page address */
|
||||
ulint hash(const page_id_t id) const
|
||||
{ assert_locked(); return rec_hash.calc_hash(id.fold()); }
|
||||
|
||||
/** Get the first lock on a page.
|
||||
@param lock_hash hash table to look at
|
||||
@param id page number
|
||||
@return first lock
|
||||
@retval nullptr if none exists */
|
||||
lock_t *get_first(const hash_table_t &lock_hash, const page_id_t id) const
|
||||
/** Get the lock hash table for a mode */
|
||||
hash_table &hash_get(ulint mode)
|
||||
{
|
||||
ut_ad(&lock_hash == &rec_hash || &lock_hash == &prdt_hash ||
|
||||
&lock_hash == &prdt_page_hash);
|
||||
for (lock_t *lock= static_cast<lock_t*>
|
||||
(HASH_GET_FIRST(&lock_hash, hash(id)));
|
||||
lock; lock= static_cast<lock_t*>(HASH_GET_NEXT(hash, lock)))
|
||||
if (lock->un_member.rec_lock.page_id == id)
|
||||
return lock;
|
||||
return nullptr;
|
||||
if (UNIV_LIKELY(!(mode & (LOCK_PREDICATE | LOCK_PRDT_PAGE))))
|
||||
return rec_hash;
|
||||
return (mode & LOCK_PREDICATE) ? prdt_hash : prdt_page_hash;
|
||||
}
|
||||
|
||||
/** Get the first record lock on a page.
|
||||
@param id page number
|
||||
/** Get the lock hash table for predicate a mode */
|
||||
hash_table &prdt_hash_get(bool page)
|
||||
{ return page ? prdt_page_hash : prdt_hash; }
|
||||
|
||||
lock_t *get_first(ulint type_mode, const page_id_t id)
|
||||
{
|
||||
return hash_get(type_mode).get_first(id);
|
||||
}
|
||||
|
||||
/** Get the first explicit lock request on a record.
|
||||
@param hash lock hash table
|
||||
@param id page identifier
|
||||
@param heap_no record identifier in page
|
||||
@return first lock
|
||||
@retval nullptr if none exists */
|
||||
lock_t *get_first(const page_id_t id) const
|
||||
{ return get_first(rec_hash, id); }
|
||||
/** Get the first predicate lock on a SPATIAL INDEX page.
|
||||
@param id page number
|
||||
@return first lock
|
||||
@retval nullptr if none exists */
|
||||
lock_t *get_first_prdt(const page_id_t id) const
|
||||
{ return get_first(prdt_hash, id); }
|
||||
/** Get the first predicate lock on a SPATIAL INDEX page.
|
||||
@param id page number
|
||||
@return first lock
|
||||
@retval nullptr if none exists */
|
||||
lock_t *get_first_prdt_page(const page_id_t id) const
|
||||
{ return get_first(prdt_page_hash, id); }
|
||||
inline lock_t *get_first(hash_table &hash, const page_id_t id,
|
||||
ulint heap_no);
|
||||
|
||||
/** Remove locks on a discarded SPATIAL INDEX page.
|
||||
@param id page to be discarded
|
||||
@param page whether to discard also from lock_sys.prdt_hash */
|
||||
void prdt_page_free_from_discard(const page_id_t id, bool all= false);
|
||||
};
|
||||
|
||||
/** The lock system */
|
||||
extern lock_sys_t lock_sys;
|
||||
|
||||
/** @return the index of an array element */
|
||||
inline ulint lock_sys_t::hash_table::calc_hash(ulint fold) const
|
||||
{
|
||||
ut_ad(lock_sys.is_writer() || lock_sys.readers);
|
||||
return calc_hash(fold, n_cells);
|
||||
}
|
||||
|
||||
/** Get a latch. */
|
||||
inline
|
||||
lock_sys_t::hash_latch *lock_sys_t::hash_table::lock_get(ulint fold) const
|
||||
{
|
||||
ut_ad(lock_sys.is_writer() || lock_sys.readers);
|
||||
return lock_get(fold, n_cells);
|
||||
}
|
||||
|
||||
/** lock_sys.latch guard */
|
||||
struct LockMutexGuard
|
||||
{
|
||||
@ -778,6 +890,44 @@ struct LockMutexGuard
|
||||
~LockMutexGuard() { lock_sys.wr_unlock(); }
|
||||
};
|
||||
|
||||
/** lock_sys.latch guard for a page_id_t shard */
|
||||
struct LockGuard
|
||||
{
|
||||
LockGuard(lock_sys_t::hash_table &hash, const page_id_t id);
|
||||
~LockGuard()
|
||||
{
|
||||
latch->release();
|
||||
/* Must be last, to avoid a race with lock_sys_t::hash_table::resize() */
|
||||
lock_sys.rd_unlock();
|
||||
}
|
||||
private:
|
||||
/** The hash bucket */
|
||||
lock_sys_t::hash_latch *latch;
|
||||
};
|
||||
|
||||
#ifdef WITH_WSREP
|
||||
/** lock_sys.latch guard for a page_id_t shard */
|
||||
struct LockGGuard
|
||||
{
|
||||
LockGGuard(lock_sys_t::hash_table &hash, const page_id_t id, bool all);
|
||||
~LockGGuard();
|
||||
private:
|
||||
/** The hash bucket (nullptr if all of them) */
|
||||
lock_sys_t::hash_latch *latch;
|
||||
};
|
||||
#endif
|
||||
|
||||
/** lock_sys.latch guard for 2 page_id_t shards */
|
||||
struct LockMultiGuard
|
||||
{
|
||||
LockMultiGuard(lock_sys_t::hash_table &hash,
|
||||
const page_id_t id1, const page_id_t id2);
|
||||
~LockMultiGuard();
|
||||
private:
|
||||
/** The hash buckets */
|
||||
lock_sys_t::hash_latch *latch1, *latch2;
|
||||
};
|
||||
|
||||
/*********************************************************************//**
|
||||
Creates a new record lock and inserts it to the lock queue. Does NOT check
|
||||
for deadlocks or lock compatibility!
|
||||
@ -800,14 +950,10 @@ lock_rec_create(
|
||||
/*!< in: true if caller owns
|
||||
trx mutex */
|
||||
|
||||
/*************************************************************//**
|
||||
Removes a record lock request, waiting or granted, from the queue. */
|
||||
void
|
||||
lock_rec_discard(
|
||||
/*=============*/
|
||||
lock_t* in_lock); /*!< in: record lock object: all
|
||||
record locks which are contained
|
||||
in this lock object are removed */
|
||||
/** Remove a record lock request, waiting or granted, on a discarded page
|
||||
@param hash hash table
|
||||
@param in_lock lock object */
|
||||
void lock_rec_discard(lock_sys_t::hash_table &lock_hash, lock_t *in_lock);
|
||||
|
||||
/** Create a new record lock and inserts it to the lock queue,
|
||||
without checking for deadlocks or conflicts.
|
||||
@ -875,11 +1021,6 @@ lock_rtr_move_rec_list(
|
||||
moved */
|
||||
ulint num_move); /*!< in: num of rec to move */
|
||||
|
||||
/** Remove record locks for an index page which is discarded. This
|
||||
function does not move locks, or check for waiting locks, therefore the
|
||||
lock bitmaps must already be reset when this function is called. */
|
||||
void lock_rec_free_all_from_discard_page(const page_id_t page_id);
|
||||
|
||||
/** Cancel a waiting lock request and release possibly waiting transactions */
|
||||
void lock_cancel_waiting_and_release(lock_t *lock);
|
||||
|
||||
|
@ -52,23 +52,6 @@ lock_get_min_heap_no(
|
||||
}
|
||||
}
|
||||
|
||||
/*************************************************************//**
|
||||
Get the lock hash table */
|
||||
UNIV_INLINE
|
||||
hash_table_t*
|
||||
lock_hash_get(
|
||||
/*==========*/
|
||||
ulint mode) /*!< in: lock mode */
|
||||
{
|
||||
if (mode & LOCK_PREDICATE) {
|
||||
return &lock_sys.prdt_hash;
|
||||
} else if (mode & LOCK_PRDT_PAGE) {
|
||||
return &lock_sys.prdt_page_hash;
|
||||
} else {
|
||||
return &lock_sys.rec_hash;
|
||||
}
|
||||
}
|
||||
|
||||
/*********************************************************************//**
|
||||
Creates a new record lock and inserts it to the lock queue. Does NOT check
|
||||
for deadlocks or lock compatibility!
|
||||
|
@ -189,10 +189,4 @@ lock_prdt_rec_move(
|
||||
@return true if there is none */
|
||||
bool lock_test_prdt_page_lock(const trx_t *trx, const page_id_t page_id);
|
||||
|
||||
/** Removes predicate lock objects set on an index page which is discarded.
|
||||
@param[in] id page to be discarded
|
||||
@param[in] lock_hash lock hash */
|
||||
void
|
||||
lock_prdt_page_free_from_discard(const page_id_t id, hash_table_t *lock_hash);
|
||||
|
||||
#endif
|
||||
|
@ -481,7 +481,7 @@ lock_rec_set_nth_bit(
|
||||
inline byte lock_rec_reset_nth_bit(lock_t* lock, ulint i)
|
||||
{
|
||||
ut_ad(!lock->is_table());
|
||||
lock_sys.assert_locked();
|
||||
ut_ad(lock_sys.is_writer() || lock->trx->mutex_is_owner());
|
||||
ut_ad(i < lock->un_member.rec_lock.n_bits);
|
||||
|
||||
byte* b = reinterpret_cast<byte*>(&lock[1]) + (i >> 3);
|
||||
@ -533,10 +533,10 @@ lock_rec_get_next_const(
|
||||
@param heap_no record identifier in page
|
||||
@return first lock
|
||||
@retval nullptr if none exists */
|
||||
inline lock_t*
|
||||
lock_rec_get_first(hash_table_t *hash, const page_id_t id, ulint heap_no)
|
||||
inline lock_t *lock_sys_t::get_first(lock_sys_t::hash_table &hash,
|
||||
const page_id_t id, ulint heap_no)
|
||||
{
|
||||
for (lock_t *lock= lock_sys.get_first(*hash, id);
|
||||
for (lock_t *lock= hash.get_first(id);
|
||||
lock; lock= lock_rec_get_next_on_page(lock))
|
||||
if (lock_rec_get_nth_bit(lock, heap_no))
|
||||
return lock;
|
||||
|
@ -78,7 +78,6 @@ lock_rec_set_nth_bit(
|
||||
ulint bit_index;
|
||||
|
||||
ut_ad(!lock->is_table());
|
||||
lock_sys.assert_locked();
|
||||
ut_ad(i < lock->un_member.rec_lock.n_bits);
|
||||
|
||||
byte_index = i / 8;
|
||||
@ -92,6 +91,7 @@ lock_rec_set_nth_bit(
|
||||
#if defined __GNUC__ && !defined __clang__ && __GNUC__ < 6
|
||||
# pragma GCC diagnostic pop
|
||||
#endif
|
||||
ut_ad(lock_sys.is_writer() || lock->trx->mutex_is_owner());
|
||||
lock->trx->lock.n_rec_locks++;
|
||||
}
|
||||
|
||||
@ -117,8 +117,6 @@ lock_rec_get_next(
|
||||
ulint heap_no,/*!< in: heap number of the record */
|
||||
lock_t* lock) /*!< in: lock */
|
||||
{
|
||||
lock_sys.assert_locked();
|
||||
|
||||
do {
|
||||
lock = lock_rec_get_next_on_page(lock);
|
||||
} while (lock && !lock_rec_get_nth_bit(lock, heap_no));
|
||||
@ -175,7 +173,6 @@ lock_rec_get_next_on_page_const(
|
||||
ut_ad(!lock->is_table());
|
||||
|
||||
const page_id_t page_id{lock->un_member.rec_lock.page_id};
|
||||
lock_sys.assert_locked();
|
||||
|
||||
while (!!(lock= static_cast<const lock_t*>(HASH_GET_NEXT(hash, lock))))
|
||||
if (lock->un_member.rec_lock.page_id == page_id)
|
||||
|
@ -148,11 +148,11 @@ operator<<(std::ostream& out, const lock_rec_t& lock)
|
||||
/** Lock struct; protected by lock_sys.latch */
|
||||
struct ib_lock_t
|
||||
{
|
||||
trx_t* trx; /*!< transaction owning the
|
||||
lock */
|
||||
UT_LIST_NODE_T(ib_lock_t)
|
||||
trx_locks; /*!< list of the locks of the
|
||||
transaction */
|
||||
/** the owner of the lock */
|
||||
trx_t *trx;
|
||||
/** other locks of the transaction; protected by
|
||||
lock_sys.is_writer() and trx->mutex_is_owner(); @see trx_lock_t::trx_locks */
|
||||
UT_LIST_NODE_T(ib_lock_t) trx_locks;
|
||||
|
||||
dict_index_t* index; /*!< index for a record lock */
|
||||
|
||||
|
@ -430,9 +430,10 @@ struct trx_lock_t
|
||||
/** lock wait start time, protected only by lock_sys.wait_mutex */
|
||||
my_hrtime_t suspend_time;
|
||||
|
||||
ib_uint64_t deadlock_mark; /*!< A mark field that is initialized
|
||||
to and checked against lock_mark_counter
|
||||
by lock_deadlock_recursive(). */
|
||||
/** DeadlockChecker::search() uses this to keep track of visited locks.
|
||||
Protected by lock_sys.is_writer(). */
|
||||
uint64_t deadlock_mark;
|
||||
|
||||
#ifdef WITH_WSREP
|
||||
/** 2=high priority wsrep thread has marked this trx to abort;
|
||||
1=another transaction chose this as a victim in deadlock resolution. */
|
||||
@ -443,12 +444,6 @@ struct trx_lock_t
|
||||
resolution. Protected by lock_sys.latch and lock_sys.wait_mutex. */
|
||||
bool was_chosen_as_deadlock_victim;
|
||||
#endif
|
||||
/** Whether the transaction is being rolled back either via deadlock
|
||||
detection or timeout. The caller has to acquire the trx_t::mutex in
|
||||
order to cancel the locks. In lock_trx_table_locks_remove() we must
|
||||
avoid reacquiring the trx_t::mutex to prevent recursive
|
||||
deadlocks. Protected by both lock_sys.latch and trx_t::mutex. */
|
||||
bool cancel;
|
||||
|
||||
/** Next available rec_pool[] entry */
|
||||
byte rec_cached;
|
||||
@ -471,13 +466,15 @@ struct trx_lock_t
|
||||
/** Pre-allocated table locks */
|
||||
ib_lock_t table_pool[8];
|
||||
|
||||
mem_heap_t* lock_heap; /*!< memory heap for trx_locks;
|
||||
protected by lock_sys.latch */
|
||||
/** Memory heap for trx_locks. Protected by lock_sys.assert_locked()
|
||||
and lock_sys.is_writer() || trx->mutex_is_owner(). */
|
||||
mem_heap_t *lock_heap;
|
||||
|
||||
trx_lock_list_t trx_locks; /*!< locks requested by the transaction;
|
||||
insertions are protected by trx->mutex
|
||||
and lock_sys.latch; removals are
|
||||
protected by lock_sys.latch */
|
||||
/** Locks held by the transaction. Protected by lock_sys.assert_locked()
|
||||
and lock_sys.is_writer() || trx->mutex_is_owner().
|
||||
(If lock_sys.latch is only held in shared mode, then the modification
|
||||
must be protected by trx->mutex.) */
|
||||
trx_lock_list_t trx_locks;
|
||||
|
||||
lock_list table_locks; /*!< All table locks requested by this
|
||||
transaction, including AUTOINC locks */
|
||||
@ -485,7 +482,7 @@ struct trx_lock_t
|
||||
/** List of pending trx_t::evict_table() */
|
||||
UT_LIST_BASE_NODE_T(dict_table_t) evicted_tables;
|
||||
|
||||
/** number of record locks; writers use LockGuard or LockMutexGuard */
|
||||
/** number of record locks; protected by lock_sys.assert_locked(page_id) */
|
||||
ulint n_rec_locks;
|
||||
};
|
||||
|
||||
@ -1019,6 +1016,8 @@ public:
|
||||
@retval false if the rollback was aborted by shutdown */
|
||||
inline bool rollback_finish();
|
||||
private:
|
||||
/** Process tables that were modified by the committing transaction. */
|
||||
inline void commit_tables();
|
||||
/** Mark a transaction committed in the main memory data structures. */
|
||||
inline void commit_in_memory(const mtr_t *mtr);
|
||||
/** Commit the transaction in a mini-transaction.
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -234,13 +234,13 @@ lock_prdt_has_lock(
|
||||
attached to the new lock */
|
||||
const trx_t* trx) /*!< in: transaction */
|
||||
{
|
||||
lock_sys.assert_locked();
|
||||
ut_ad((precise_mode & LOCK_MODE_MASK) == LOCK_S
|
||||
|| (precise_mode & LOCK_MODE_MASK) == LOCK_X);
|
||||
ut_ad(!(precise_mode & LOCK_INSERT_INTENTION));
|
||||
|
||||
for (lock_t* lock = lock_rec_get_first(lock_hash_get(type_mode), id,
|
||||
PRDT_HEAPNO); lock;
|
||||
for (lock_t*lock= lock_sys.get_first(lock_sys.hash_get(type_mode),
|
||||
id, PRDT_HEAPNO);
|
||||
lock;
|
||||
lock = lock_rec_get_next(PRDT_HEAPNO, lock)) {
|
||||
ut_ad(lock->type_mode & (LOCK_PREDICATE | LOCK_PRDT_PAGE));
|
||||
|
||||
@ -288,8 +288,8 @@ lock_prdt_other_has_conflicting(
|
||||
the new lock will be on */
|
||||
const trx_t* trx) /*!< in: our transaction */
|
||||
{
|
||||
for (lock_t* lock = lock_rec_get_first(lock_hash_get(mode), id,
|
||||
PRDT_HEAPNO);
|
||||
for (lock_t* lock = lock_sys.get_first(lock_sys.hash_get(mode),
|
||||
id, PRDT_HEAPNO);
|
||||
lock != NULL;
|
||||
lock = lock_rec_get_next(PRDT_HEAPNO, lock)) {
|
||||
|
||||
@ -383,8 +383,7 @@ lock_prdt_find_on_page(
|
||||
{
|
||||
lock_t* lock;
|
||||
|
||||
for (lock = lock_sys.get_first(*lock_hash_get(type_mode),
|
||||
block->page.id());
|
||||
for (lock = lock_sys.get_first(type_mode, block->page.id());
|
||||
lock != NULL;
|
||||
lock = lock_rec_get_next_on_page(lock)) {
|
||||
|
||||
@ -425,8 +424,6 @@ lock_prdt_add_to_queue(
|
||||
/*!< in: TRUE if caller owns the
|
||||
transaction mutex */
|
||||
{
|
||||
const page_id_t id{block->page.id()};
|
||||
lock_sys.assert_locked();
|
||||
ut_ad(caller_owns_trx_mutex == trx->mutex_is_owner());
|
||||
ut_ad(index->is_spatial());
|
||||
ut_ad(!dict_index_is_online_ddl(index));
|
||||
@ -447,7 +444,7 @@ lock_prdt_add_to_queue(
|
||||
goto create;
|
||||
}
|
||||
|
||||
for (lock_t* lock = lock_sys.get_first(*lock_hash_get(type_mode), id);
|
||||
for (lock_t* lock = lock_sys.get_first(type_mode, block->page.id());
|
||||
lock; lock = lock_rec_get_next_on_page(lock)) {
|
||||
if (lock->is_waiting()
|
||||
&& lock->type_mode & (LOCK_PREDICATE | LOCK_PRDT_PAGE)
|
||||
@ -504,7 +501,7 @@ lock_prdt_insert_check_and_lock(
|
||||
dberr_t err= DB_SUCCESS;
|
||||
|
||||
{
|
||||
LockMutexGuard g{SRW_LOCK_CALL};
|
||||
LockGuard g{lock_sys.prdt_hash, id};
|
||||
/* Because this code is invoked for a running transaction by
|
||||
the thread that is serving the transaction, it is not necessary
|
||||
to hold trx->mutex here. */
|
||||
@ -512,7 +509,7 @@ lock_prdt_insert_check_and_lock(
|
||||
|
||||
/* Only need to check locks on prdt_hash */
|
||||
if (ut_d(lock_t *lock=)
|
||||
lock_rec_get_first(&lock_sys.prdt_hash, id, PRDT_HEAPNO))
|
||||
lock_sys.get_first(lock_sys.prdt_hash, id, PRDT_HEAPNO))
|
||||
{
|
||||
ut_ad(lock->type_mode & LOCK_PREDICATE);
|
||||
|
||||
@ -529,9 +526,9 @@ lock_prdt_insert_check_and_lock(
|
||||
if (c_lock)
|
||||
{
|
||||
rtr_mbr_t *mbr= prdt_get_mbr_from_prdt(prdt);
|
||||
trx->mutex_lock();
|
||||
/* Allocate MBR on the lock heap */
|
||||
lock_init_prdt_from_mbr(prdt, mbr, 0, trx->lock.lock_heap);
|
||||
trx->mutex_lock();
|
||||
err= lock_rec_enqueue_waiting(
|
||||
#ifdef WITH_WSREP
|
||||
c_lock,
|
||||
@ -564,7 +561,7 @@ lock_prdt_update_parent(
|
||||
LockMutexGuard g{SRW_LOCK_CALL};
|
||||
|
||||
/* Get all locks in parent */
|
||||
for (lock_t *lock = lock_sys.get_first_prdt(page_id);
|
||||
for (lock_t *lock = lock_sys.prdt_hash.get_first(page_id);
|
||||
lock;
|
||||
lock = lock_rec_get_next_on_page(lock)) {
|
||||
lock_prdt_t* lock_prdt;
|
||||
@ -615,7 +612,7 @@ lock_prdt_update_split_low(
|
||||
{
|
||||
lock_t* lock;
|
||||
|
||||
for (lock = lock_sys.get_first(*lock_hash_get(type_mode), page_id);
|
||||
for (lock = lock_sys.get_first(type_mode, page_id);
|
||||
lock;
|
||||
lock = lock_rec_get_next_on_page(lock)) {
|
||||
/* First dealing with Page Lock */
|
||||
@ -661,7 +658,6 @@ lock_prdt_update_split(
|
||||
const page_id_t page_id) /*!< in: page number */
|
||||
{
|
||||
LockMutexGuard g{SRW_LOCK_CALL};
|
||||
|
||||
lock_prdt_update_split_low(new_block, prdt, new_prdt,
|
||||
page_id, LOCK_PREDICATE);
|
||||
|
||||
@ -682,8 +678,7 @@ lock_init_prdt_from_mbr(
|
||||
memset(prdt, 0, sizeof(*prdt));
|
||||
|
||||
if (heap != NULL) {
|
||||
prdt->data = mem_heap_alloc(heap, sizeof(*mbr));
|
||||
memcpy(prdt->data, mbr, sizeof(*mbr));
|
||||
prdt->data = mem_heap_dup(heap, mbr, sizeof *mbr);
|
||||
} else {
|
||||
prdt->data = static_cast<void*>(mbr);
|
||||
}
|
||||
@ -722,9 +717,7 @@ lock_prdt_lock(
|
||||
ut_ad(!dict_index_is_online_ddl(index));
|
||||
ut_ad(type_mode & (LOCK_PREDICATE | LOCK_PRDT_PAGE));
|
||||
|
||||
const hash_table_t& hash = type_mode == LOCK_PREDICATE
|
||||
? lock_sys.prdt_hash
|
||||
: lock_sys.prdt_page_hash;
|
||||
auto& hash = lock_sys.prdt_hash_get(type_mode != LOCK_PREDICATE);
|
||||
const page_id_t id{block->page.id()};
|
||||
|
||||
/* Another transaction cannot have an implicit lock on the record,
|
||||
@ -732,10 +725,10 @@ lock_prdt_lock(
|
||||
index record, and this would not have been possible if another active
|
||||
transaction had modified this secondary index record. */
|
||||
|
||||
LockMutexGuard g{SRW_LOCK_CALL};
|
||||
LockGuard g{hash, id};
|
||||
|
||||
const unsigned prdt_mode = type_mode | mode;
|
||||
lock_t* lock = lock_sys.get_first(hash, id);
|
||||
lock_t* lock = hash.get_first(id);
|
||||
|
||||
if (lock == NULL) {
|
||||
lock = lock_rec_create(
|
||||
@ -823,9 +816,9 @@ lock_place_prdt_page_lock(
|
||||
index record, and this would not have been possible if another active
|
||||
transaction had modified this secondary index record. */
|
||||
|
||||
LockMutexGuard g{SRW_LOCK_CALL};
|
||||
LockGuard g{lock_sys.prdt_page_hash, page_id};
|
||||
|
||||
const lock_t* lock = lock_sys.get_first_prdt_page(page_id);
|
||||
const lock_t* lock = lock_sys.prdt_page_hash.get_first(page_id);
|
||||
const ulint mode = LOCK_S | LOCK_PRDT_PAGE;
|
||||
trx_t* trx = thr_get_trx(thr);
|
||||
|
||||
@ -862,8 +855,8 @@ lock_place_prdt_page_lock(
|
||||
@return true if there is none */
|
||||
bool lock_test_prdt_page_lock(const trx_t *trx, const page_id_t page_id)
|
||||
{
|
||||
LockMutexGuard g{SRW_LOCK_CALL};
|
||||
lock_t *lock= lock_sys.get_first_prdt_page(page_id);
|
||||
LockGuard g{lock_sys.prdt_page_hash, page_id};
|
||||
lock_t *lock= lock_sys.prdt_page_hash.get_first(page_id);
|
||||
return !lock || trx == lock->trx;
|
||||
}
|
||||
|
||||
@ -877,9 +870,9 @@ lock_prdt_rec_move(
|
||||
the receiving record */
|
||||
const page_id_t donator) /*!< in: target page */
|
||||
{
|
||||
LockMutexGuard g{SRW_LOCK_CALL};
|
||||
LockMultiGuard g{lock_sys.prdt_hash, receiver->page.id(), donator};
|
||||
|
||||
for (lock_t *lock = lock_rec_get_first(&lock_sys.prdt_hash,
|
||||
for (lock_t *lock = lock_sys.get_first(lock_sys.prdt_hash,
|
||||
donator, PRDT_HEAPNO);
|
||||
lock != NULL;
|
||||
lock = lock_rec_get_next(PRDT_HEAPNO, lock)) {
|
||||
@ -898,18 +891,48 @@ lock_prdt_rec_move(
|
||||
}
|
||||
}
|
||||
|
||||
/** Removes predicate lock objects set on an index page which is discarded.
|
||||
@param[in] id page to be discarded
|
||||
@param[in] lock_hash lock hash */
|
||||
void
|
||||
lock_prdt_page_free_from_discard(const page_id_t id, hash_table_t *lock_hash)
|
||||
/** Remove locks on a discarded SPATIAL INDEX page.
|
||||
@param id page to be discarded
|
||||
@param page whether to discard also from lock_sys.prdt_hash */
|
||||
void lock_sys_t::prdt_page_free_from_discard(const page_id_t id, bool all)
|
||||
{
|
||||
lock_sys.assert_locked();
|
||||
const auto id_fold= id.fold();
|
||||
rd_lock(SRW_LOCK_CALL);
|
||||
auto latch= prdt_page_hash.lock_get(id_fold);
|
||||
latch->acquire();
|
||||
|
||||
for (lock_t *lock= lock_sys.get_first(*lock_hash, id), *next; lock;
|
||||
for (lock_t *lock= lock_sys.prdt_page_hash.get_first(id), *next; lock;
|
||||
lock= next)
|
||||
{
|
||||
next= lock_rec_get_next_on_page(lock);
|
||||
lock_rec_discard(lock);
|
||||
lock_rec_discard(prdt_page_hash, lock);
|
||||
}
|
||||
|
||||
if (all)
|
||||
{
|
||||
latch->release();
|
||||
latch= prdt_hash.lock_get(id_fold);
|
||||
latch->acquire();
|
||||
for (lock_t *lock= lock_sys.prdt_hash.get_first(id), *next; lock;
|
||||
lock= next)
|
||||
{
|
||||
next= lock_rec_get_next_on_page(lock);
|
||||
lock_rec_discard(prdt_hash, lock);
|
||||
}
|
||||
}
|
||||
|
||||
latch->release();
|
||||
latch= rec_hash.lock_get(id_fold);
|
||||
latch->acquire();
|
||||
|
||||
for (lock_t *lock= lock_sys.rec_hash.get_first(id), *next; lock;
|
||||
lock= next)
|
||||
{
|
||||
next= lock_rec_get_next_on_page(lock);
|
||||
lock_rec_discard(rec_hash, lock);
|
||||
}
|
||||
|
||||
latch->release();
|
||||
/* Must be last, to avoid a race with lock_sys_t::hash_table::resize() */
|
||||
rd_unlock();
|
||||
}
|
||||
|
@ -2838,12 +2838,8 @@ wait_again:
|
||||
}
|
||||
|
||||
if (vers_update_trt) {
|
||||
trx_mod_table_time_t& time =
|
||||
trx->mod_tables
|
||||
.insert(trx_mod_tables_t::value_type(
|
||||
const_cast<dict_table_t*>(new_table), 0))
|
||||
.first->second;
|
||||
time.set_versioned(0);
|
||||
trx->mod_tables.emplace(new_table, 0)
|
||||
.first->second.set_versioned(0);
|
||||
}
|
||||
|
||||
trx->op_info = "";
|
||||
|
@ -3479,8 +3479,7 @@ defer:
|
||||
shouldn't have to. There should never be record locks on a table
|
||||
that is going to be dropped. */
|
||||
|
||||
if (table->get_ref_count() > 0 || table->n_rec_locks > 0
|
||||
|| lock_table_has_locks(table)) {
|
||||
if (table->get_ref_count() > 0 || lock_table_has_locks(table)) {
|
||||
goto defer;
|
||||
}
|
||||
|
||||
|
@ -491,7 +491,7 @@ inline void trx_t::release_locks()
|
||||
if (UT_LIST_GET_LEN(lock.trx_locks))
|
||||
{
|
||||
lock_release(this);
|
||||
lock.n_rec_locks = 0;
|
||||
ut_ad(!lock.n_rec_locks);
|
||||
ut_ad(UT_LIST_GET_LEN(lock.trx_locks) == 0);
|
||||
ut_ad(ib_vector_is_empty(autoinc_locks));
|
||||
mem_heap_empty(lock.lock_heap);
|
||||
@ -1209,66 +1209,50 @@ trx_flush_log_if_needed(
|
||||
trx->op_info = "";
|
||||
}
|
||||
|
||||
/**********************************************************************//**
|
||||
For each table that has been modified by the given transaction: update
|
||||
its dict_table_t::update_time with the current timestamp. Clear the list
|
||||
of the modified tables at the end. */
|
||||
static
|
||||
void
|
||||
trx_update_mod_tables_timestamp(
|
||||
/*============================*/
|
||||
trx_t* trx) /*!< in: transaction */
|
||||
/** Process tables that were modified by the committing transaction. */
|
||||
inline void trx_t::commit_tables()
|
||||
{
|
||||
/* consider using trx->start_time if calling time() is too
|
||||
expensive here */
|
||||
const time_t now = time(NULL);
|
||||
if (!undo_no || mod_tables.empty())
|
||||
return;
|
||||
|
||||
#if defined SAFE_MUTEX && defined UNIV_DEBUG
|
||||
const bool preserve_tables = !innodb_evict_tables_on_commit_debug
|
||||
|| trx->is_recovered /* avoid trouble with XA recovery */
|
||||
const bool preserve_tables= !innodb_evict_tables_on_commit_debug ||
|
||||
is_recovered || /* avoid trouble with XA recovery */
|
||||
# if 1 /* if dict_stats_exec_sql() were not playing dirty tricks */
|
||||
|| dict_sys.mutex_is_locked()
|
||||
dict_sys.mutex_is_locked();
|
||||
# else /* this would be more proper way to do it */
|
||||
|| trx->dict_operation_lock_mode || trx->dict_operation
|
||||
dict_operation_lock_mode || dict_operation;
|
||||
# endif
|
||||
;
|
||||
#endif
|
||||
|
||||
for (const auto& p : trx->mod_tables) {
|
||||
/* This could be executed by multiple threads concurrently
|
||||
on the same table object. This is fine because time_t is
|
||||
word size or less. And _purely_ _theoretically_, even if
|
||||
time_t write is not atomic, likely the value of 'now' is
|
||||
the same in all threads and even if it is not, getting a
|
||||
"garbage" in table->update_time is justified because
|
||||
protecting it with a latch here would be too performance
|
||||
intrusive. */
|
||||
dict_table_t* table = p.first;
|
||||
table->update_time = now;
|
||||
const trx_id_t max_trx_id= trx_sys.get_max_trx_id();
|
||||
const auto now= start_time;
|
||||
|
||||
for (const auto& p : mod_tables)
|
||||
{
|
||||
dict_table_t *table= p.first;
|
||||
table->update_time= now;
|
||||
table->query_cache_inv_trx_id= max_trx_id;
|
||||
|
||||
#if defined SAFE_MUTEX && defined UNIV_DEBUG
|
||||
if (preserve_tables || table->get_ref_count()
|
||||
|| table->is_temporary()
|
||||
|| UT_LIST_GET_LEN(table->locks)) {
|
||||
/* do not evict when committing DDL operations
|
||||
or if some other transaction is holding the
|
||||
table handle */
|
||||
continue;
|
||||
}
|
||||
/* recheck while holding the mutex that blocks
|
||||
table->acquire() */
|
||||
dict_sys.mutex_lock();
|
||||
{
|
||||
LockMutexGuard g{SRW_LOCK_CALL};
|
||||
if (!table->get_ref_count()
|
||||
&& !UT_LIST_GET_LEN(table->locks)) {
|
||||
dict_sys.remove(table, true);
|
||||
}
|
||||
}
|
||||
dict_sys.mutex_unlock();
|
||||
if (preserve_tables || table->get_ref_count() || table->is_temporary() ||
|
||||
UT_LIST_GET_LEN(table->locks))
|
||||
/* do not evict when committing DDL operations or if some other
|
||||
transaction is holding the table handle */
|
||||
continue;
|
||||
/* recheck while holding the mutex that blocks
|
||||
table->acquire() */
|
||||
dict_sys.mutex_lock();
|
||||
{
|
||||
LockMutexGuard g{SRW_LOCK_CALL};
|
||||
if (!table->get_ref_count() && !UT_LIST_GET_LEN(table->locks))
|
||||
dict_sys.remove(table, true);
|
||||
}
|
||||
dict_sys.mutex_unlock();
|
||||
#endif
|
||||
}
|
||||
}
|
||||
|
||||
trx->mod_tables.clear();
|
||||
mod_tables.clear();
|
||||
}
|
||||
|
||||
/** Evict a table definition due to the rollback of ALTER TABLE.
|
||||
@ -1366,7 +1350,7 @@ inline void trx_t::commit_in_memory(const mtr_t *mtr)
|
||||
}
|
||||
else
|
||||
{
|
||||
trx_update_mod_tables_timestamp(this);
|
||||
commit_tables();
|
||||
MONITOR_INC(MONITOR_TRX_RW_COMMIT);
|
||||
is_recovered= false;
|
||||
}
|
||||
|
Loading…
x
Reference in New Issue
Block a user