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:
Marko Mäkelä 2021-02-12 17:35:42 +02:00
parent b01d8e1a33
commit b08448de64
20 changed files with 1109 additions and 831 deletions

View File

@ -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,

View File

@ -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),

View File

@ -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;

View File

@ -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.

View File

@ -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));

View File

@ -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;
}
}

View File

@ -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'. */

View File

@ -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
{

View File

@ -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);

View File

@ -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!

View File

@ -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

View File

@ -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;

View File

@ -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)

View File

@ -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 */

View File

@ -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

View File

@ -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();
}

View File

@ -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 = "";

View File

@ -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;
}

View File

@ -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;
}