summaryrefslogtreecommitdiffstats
path: root/storage/innobase/lock
diff options
context:
space:
mode:
Diffstat (limited to 'storage/innobase/lock')
-rw-r--r--storage/innobase/lock/lock0iter.cc107
-rw-r--r--storage/innobase/lock/lock0lock.cc6818
-rw-r--r--storage/innobase/lock/lock0prdt.cc1028
-rw-r--r--storage/innobase/lock/lock0wait.cc515
4 files changed, 8468 insertions, 0 deletions
diff --git a/storage/innobase/lock/lock0iter.cc b/storage/innobase/lock/lock0iter.cc
new file mode 100644
index 00000000..7a7130ed
--- /dev/null
+++ b/storage/innobase/lock/lock0iter.cc
@@ -0,0 +1,107 @@
+/*****************************************************************************
+
+Copyright (c) 2007, 2014, Oracle and/or its affiliates. All Rights Reserved.
+
+This program is free software; you can redistribute it and/or modify it under
+the terms of the GNU General Public License as published by the Free Software
+Foundation; version 2 of the License.
+
+This program is distributed in the hope that it will be useful, but WITHOUT
+ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or FITNESS
+FOR A PARTICULAR PURPOSE. See the GNU General Public License for more details.
+
+You should have received a copy of the GNU General Public License along with
+this program; if not, write to the Free Software Foundation, Inc.,
+51 Franklin Street, Fifth Floor, Boston, MA 02110-1335 USA
+
+*****************************************************************************/
+
+/**************************************************//**
+@file lock/lock0iter.cc
+Lock queue iterator. Can iterate over table and record
+lock queues.
+
+Created July 16, 2007 Vasil Dimov
+*******************************************************/
+
+#define LOCK_MODULE_IMPLEMENTATION
+
+#include "dict0mem.h"
+#include "lock0iter.h"
+#include "lock0lock.h"
+#include "lock0priv.h"
+
+/*******************************************************************//**
+Initialize lock queue iterator so that it starts to iterate from
+"lock". bit_no specifies the record number within the heap where the
+record is stored. It can be undefined (ULINT_UNDEFINED) in two cases:
+1. If the lock is a table lock, thus we have a table lock queue;
+2. If the lock is a record lock and it is a wait lock. In this case
+ bit_no is calculated in this function by using
+ lock_rec_find_set_bit(). There is exactly one bit set in the bitmap
+ of a wait lock. */
+void
+lock_queue_iterator_reset(
+/*======================*/
+ lock_queue_iterator_t* iter, /*!< out: iterator */
+ const lock_t* lock, /*!< in: lock to start from */
+ ulint bit_no) /*!< in: record number in the
+ heap */
+{
+ ut_ad(lock_mutex_own());
+
+ iter->current_lock = lock;
+
+ if (bit_no != ULINT_UNDEFINED) {
+
+ iter->bit_no = bit_no;
+ } else {
+
+ switch (lock_get_type_low(lock)) {
+ case LOCK_TABLE:
+ iter->bit_no = ULINT_UNDEFINED;
+ break;
+ case LOCK_REC:
+ iter->bit_no = lock_rec_find_set_bit(lock);
+ ut_a(iter->bit_no != ULINT_UNDEFINED);
+ break;
+ default:
+ ut_error;
+ }
+ }
+}
+
+/*******************************************************************//**
+Gets the previous lock in the lock queue, returns NULL if there are no
+more locks (i.e. the current lock is the first one). The iterator is
+receded (if not-NULL is returned).
+@return previous lock or NULL */
+const lock_t*
+lock_queue_iterator_get_prev(
+/*=========================*/
+ lock_queue_iterator_t* iter) /*!< in/out: iterator */
+{
+ const lock_t* prev_lock;
+
+ ut_ad(lock_mutex_own());
+
+ switch (lock_get_type_low(iter->current_lock)) {
+ case LOCK_REC:
+ prev_lock = lock_rec_get_prev(
+ iter->current_lock, iter->bit_no);
+ break;
+ case LOCK_TABLE:
+ prev_lock = UT_LIST_GET_PREV(
+ un_member.tab_lock.locks, iter->current_lock);
+ break;
+ default:
+ ut_error;
+ }
+
+ if (prev_lock != NULL) {
+
+ iter->current_lock = prev_lock;
+ }
+
+ return(prev_lock);
+}
diff --git a/storage/innobase/lock/lock0lock.cc b/storage/innobase/lock/lock0lock.cc
new file mode 100644
index 00000000..8dc2d7c5
--- /dev/null
+++ b/storage/innobase/lock/lock0lock.cc
@@ -0,0 +1,6818 @@
+/*****************************************************************************
+
+Copyright (c) 1996, 2017, Oracle and/or its affiliates. All Rights Reserved.
+Copyright (c) 2014, 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
+Foundation; version 2 of the License.
+
+This program is distributed in the hope that it will be useful, but WITHOUT
+ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or FITNESS
+FOR A PARTICULAR PURPOSE. See the GNU General Public License for more details.
+
+You should have received a copy of the GNU General Public License along with
+this program; if not, write to the Free Software Foundation, Inc.,
+51 Franklin Street, Fifth Floor, Boston, MA 02110-1335 USA
+
+*****************************************************************************/
+
+/**************************************************//**
+@file lock/lock0lock.cc
+The transaction lock system
+
+Created 5/7/1996 Heikki Tuuri
+*******************************************************/
+
+#define LOCK_MODULE_IMPLEMENTATION
+
+#include "univ.i"
+
+#include <mysql/service_thd_error_context.h>
+#include <sql_class.h>
+
+#include "lock0lock.h"
+#include "lock0priv.h"
+#include "dict0mem.h"
+#include "trx0purge.h"
+#include "trx0sys.h"
+#include "ut0vec.h"
+#include "btr0cur.h"
+#include "row0sel.h"
+#include "row0mysql.h"
+#include "row0vers.h"
+#include "pars0pars.h"
+
+#include <set>
+
+#ifdef WITH_WSREP
+#include <mysql/service_wsrep.h>
+#endif /* WITH_WSREP */
+
+/** Lock scheduling algorithm */
+ulong innodb_lock_schedule_algorithm;
+
+/** The value of innodb_deadlock_detect */
+my_bool innobase_deadlock_detect;
+
+/*********************************************************************//**
+Checks if a waiting record lock request still has to wait in a queue.
+@return lock that is causing the wait */
+static
+const lock_t*
+lock_rec_has_to_wait_in_queue(
+/*==========================*/
+ const lock_t* wait_lock); /*!< in: waiting record lock */
+
+/** Grant a lock to a waiting lock request and release the waiting transaction
+after lock_reset_lock_and_trx_wait() has been called. */
+static void lock_grant_after_reset(lock_t* lock);
+
+extern "C" void thd_rpl_deadlock_check(MYSQL_THD thd, MYSQL_THD other_thd);
+extern "C" int thd_need_wait_reports(const MYSQL_THD thd);
+extern "C" int thd_need_ordering_with(const MYSQL_THD thd, const MYSQL_THD other_thd);
+
+/** Pretty-print a table lock.
+@param[in,out] file output stream
+@param[in] lock table lock */
+static void lock_table_print(FILE* file, const lock_t* lock);
+
+/** Pretty-print a record lock.
+@param[in,out] file output stream
+@param[in] lock record lock
+@param[in,out] mtr mini-transaction for accessing the record */
+static void lock_rec_print(FILE* file, const lock_t* lock, mtr_t& mtr);
+
+/** Deadlock checker. */
+class DeadlockChecker {
+public:
+ /** Check if a joining lock request results in a deadlock.
+ If a deadlock is found, we will resolve the deadlock by
+ choosing a victim transaction and rolling it back.
+ We will attempt to resolve all deadlocks.
+
+ @param[in] lock the lock request
+ @param[in,out] trx transaction requesting the lock
+
+ @return trx if it was chosen as victim
+ @retval NULL if another victim was chosen,
+ or there is no deadlock (any more) */
+ static const trx_t* check_and_resolve(const lock_t* lock, trx_t* trx);
+
+private:
+ /** Do a shallow copy. Default destructor OK.
+ @param trx the start transaction (start node)
+ @param wait_lock lock that a transaction wants
+ @param mark_start visited node counter
+ @param report_waiters whether to call thd_rpl_deadlock_check() */
+ DeadlockChecker(
+ const trx_t* trx,
+ const lock_t* wait_lock,
+ ib_uint64_t mark_start,
+ bool report_waiters)
+ :
+ m_cost(),
+ m_start(trx),
+ m_too_deep(),
+ m_wait_lock(wait_lock),
+ m_mark_start(mark_start),
+ m_n_elems(),
+ m_report_waiters(report_waiters)
+ {
+ }
+
+ /** Check if the search is too deep. */
+ bool is_too_deep() const
+ {
+ return(m_n_elems > LOCK_MAX_DEPTH_IN_DEADLOCK_CHECK
+ || m_cost > LOCK_MAX_N_STEPS_IN_DEADLOCK_CHECK);
+ }
+
+ /** Save current state.
+ @param lock lock to push on the stack.
+ @param heap_no the heap number to push on the stack.
+ @return false if stack is full. */
+ bool push(const lock_t* lock, ulint heap_no)
+ {
+ ut_ad((lock_get_type_low(lock) & LOCK_REC)
+ || (lock_get_type_low(lock) & LOCK_TABLE));
+
+ ut_ad(((lock_get_type_low(lock) & LOCK_TABLE) != 0)
+ == (heap_no == ULINT_UNDEFINED));
+
+ /* Ensure that the stack is bounded. */
+ if (m_n_elems >= UT_ARR_SIZE(s_states)) {
+ return(false);
+ }
+
+ state_t& state = s_states[m_n_elems++];
+
+ state.m_lock = lock;
+ state.m_wait_lock = m_wait_lock;
+ state.m_heap_no =heap_no;
+
+ return(true);
+ }
+
+ /** Restore state.
+ @param[out] lock current lock
+ @param[out] heap_no current heap_no */
+ void pop(const lock_t*& lock, ulint& heap_no)
+ {
+ ut_a(m_n_elems > 0);
+
+ const state_t& state = s_states[--m_n_elems];
+
+ lock = state.m_lock;
+ heap_no = state.m_heap_no;
+ m_wait_lock = state.m_wait_lock;
+ }
+
+ /** Check whether the node has been visited.
+ @param lock lock to check
+ @return true if the node has been visited */
+ bool is_visited(const lock_t* lock) const
+ {
+ return(lock->trx->lock.deadlock_mark > m_mark_start);
+ }
+
+ /** Get the next lock in the queue that is owned by a transaction
+ whose sub-tree has not already been searched.
+ Note: "next" here means PREV for table locks.
+ @param lock Lock in queue
+ @param heap_no heap_no if lock is a record lock else ULINT_UNDEFINED
+ @return next lock or NULL if at end of queue */
+ const lock_t* get_next_lock(const lock_t* lock, ulint heap_no) const;
+
+ /** Get the first lock to search. The search starts from the current
+ wait_lock. What we are really interested in is an edge from the
+ current wait_lock's owning transaction to another transaction that has
+ a lock ahead in the queue. We skip locks where the owning transaction's
+ sub-tree has already been searched.
+
+ Note: The record locks are traversed from the oldest lock to the
+ latest. For table locks we go from latest to oldest.
+
+ For record locks, we first position the iterator on first lock on
+ the page and then reposition on the actual heap_no. This is required
+ due to the way the record lock has is implemented.
+
+ @param[out] heap_no if rec lock, else ULINT_UNDEFINED.
+
+ @return first lock or NULL */
+ const lock_t* get_first_lock(ulint* heap_no) const;
+
+ /** Notify that a deadlock has been detected and print the conflicting
+ transaction info.
+ @param lock lock causing deadlock */
+ void notify(const lock_t* lock) const;
+
+ /** Select the victim transaction that should be rolledback.
+ @return victim transaction */
+ const trx_t* select_victim() const;
+
+ /** Rollback transaction selected as the victim. */
+ void trx_rollback();
+
+ /** Looks iteratively for a deadlock. Note: the joining transaction
+ may have been granted its lock by the deadlock checks.
+
+ @return 0 if no deadlock else the victim transaction.*/
+ const trx_t* search();
+
+ /** Print transaction data to the deadlock file and possibly to stderr.
+ @param trx transaction
+ @param max_query_len max query length to print */
+ static void print(const trx_t* trx, ulint max_query_len);
+
+ /** rewind(3) the file used for storing the latest detected deadlock
+ and print a heading message to stderr if printing of all deadlocks to
+ stderr is enabled. */
+ static void start_print();
+
+ /** Print lock data to the deadlock file and possibly to stderr.
+ @param lock record or table type lock */
+ static void print(const lock_t* lock);
+
+ /** Print a message to the deadlock file and possibly to stderr.
+ @param msg message to print */
+ static void print(const char* msg);
+
+ /** Print info about transaction that was rolled back.
+ @param trx transaction rolled back
+ @param lock lock trx wants */
+ static void rollback_print(const trx_t* trx, const lock_t* lock);
+
+private:
+ /** DFS state information, used during deadlock checking. */
+ struct state_t {
+ const lock_t* m_lock; /*!< Current lock */
+ const lock_t* m_wait_lock; /*!< Waiting for lock */
+ ulint m_heap_no; /*!< heap number if rec lock */
+ };
+
+ /** Used in deadlock tracking. Protected by lock_sys.mutex. */
+ static ib_uint64_t s_lock_mark_counter;
+
+ /** Calculation steps thus far. It is the count of the nodes visited. */
+ ulint m_cost;
+
+ /** Joining transaction that is requesting a lock in an
+ incompatible mode */
+ const trx_t* m_start;
+
+ /** TRUE if search was too deep and was aborted */
+ bool m_too_deep;
+
+ /** Lock that trx wants */
+ const lock_t* m_wait_lock;
+
+ /** Value of lock_mark_count at the start of the deadlock check. */
+ ib_uint64_t m_mark_start;
+
+ /** Number of states pushed onto the stack */
+ size_t m_n_elems;
+
+ /** This is to avoid malloc/free calls. */
+ static state_t s_states[MAX_STACK_SIZE];
+
+ /** Set if thd_rpl_deadlock_check() should be called for waits. */
+ const bool m_report_waiters;
+};
+
+/** Counter to mark visited nodes during deadlock search. */
+ib_uint64_t DeadlockChecker::s_lock_mark_counter = 0;
+
+/** The stack used for deadlock searches. */
+DeadlockChecker::state_t DeadlockChecker::s_states[MAX_STACK_SIZE];
+
+#ifdef UNIV_DEBUG
+/*********************************************************************//**
+Validates the lock system.
+@return TRUE if ok */
+static
+bool
+lock_validate();
+/*============*/
+
+/*********************************************************************//**
+Validates the record lock queues on a page.
+@return TRUE if ok */
+static
+ibool
+lock_rec_validate_page(
+/*===================*/
+ const buf_block_t* block) /*!< in: buffer block */
+ MY_ATTRIBUTE((warn_unused_result));
+#endif /* UNIV_DEBUG */
+
+/* The lock system */
+lock_sys_t lock_sys;
+
+/** We store info on the latest deadlock error to this buffer. InnoDB
+Monitor will then fetch it and print */
+static bool lock_deadlock_found = false;
+
+/** Only created if !srv_read_only_mode */
+static FILE* lock_latest_err_file;
+
+/*********************************************************************//**
+Reports that a transaction id is insensible, i.e., in the future. */
+ATTRIBUTE_COLD
+void
+lock_report_trx_id_insanity(
+/*========================*/
+ trx_id_t trx_id, /*!< in: trx id */
+ const rec_t* rec, /*!< in: user record */
+ dict_index_t* index, /*!< in: index */
+ const rec_offs* offsets, /*!< in: rec_get_offsets(rec, index) */
+ trx_id_t max_trx_id) /*!< in: trx_sys.get_max_trx_id() */
+{
+ ut_ad(rec_offs_validate(rec, index, offsets));
+ ut_ad(!rec_is_metadata(rec, *index));
+
+ ib::error()
+ << "Transaction id " << ib::hex(trx_id)
+ << " associated with record" << rec_offsets_print(rec, offsets)
+ << " in index " << index->name
+ << " of table " << index->table->name
+ << " is greater than the global counter " << max_trx_id
+ << "! The table is corrupted.";
+}
+
+/*********************************************************************//**
+Checks that a transaction id is sensible, i.e., not in the future.
+@return true if ok */
+bool
+lock_check_trx_id_sanity(
+/*=====================*/
+ trx_id_t trx_id, /*!< in: trx id */
+ const rec_t* rec, /*!< in: user record */
+ dict_index_t* index, /*!< in: index */
+ const rec_offs* offsets) /*!< in: rec_get_offsets(rec, index) */
+{
+ ut_ad(rec_offs_validate(rec, index, offsets));
+ ut_ad(!rec_is_metadata(rec, *index));
+
+ trx_id_t max_trx_id= trx_sys.get_max_trx_id();
+ ut_ad(max_trx_id || srv_force_recovery >= SRV_FORCE_NO_UNDO_LOG_SCAN);
+
+ if (UNIV_LIKELY(max_trx_id != 0) && UNIV_UNLIKELY(trx_id >= max_trx_id))
+ {
+ lock_report_trx_id_insanity(trx_id, rec, index, offsets, max_trx_id);
+ return false;
+ }
+ return true;
+}
+
+/*********************************************************************//**
+Checks that a record is seen in a consistent read.
+@return true if sees, or false if an earlier version of the record
+should be retrieved */
+bool
+lock_clust_rec_cons_read_sees(
+/*==========================*/
+ const rec_t* rec, /*!< in: user record which should be read or
+ passed over by a read cursor */
+ dict_index_t* index, /*!< in: clustered index */
+ const rec_offs* offsets,/*!< in: rec_get_offsets(rec, index) */
+ ReadView* view) /*!< in: consistent read view */
+{
+ ut_ad(dict_index_is_clust(index));
+ ut_ad(page_rec_is_user_rec(rec));
+ ut_ad(rec_offs_validate(rec, index, offsets));
+ ut_ad(!rec_is_metadata(rec, *index));
+
+ /* Temp-tables are not shared across connections and multiple
+ transactions from different connections cannot simultaneously
+ operate on same temp-table and so read of temp-table is
+ always consistent read. */
+ if (index->table->is_temporary()) {
+ return(true);
+ }
+
+ /* NOTE that we call this function while holding the search
+ system latch. */
+
+ trx_id_t trx_id = row_get_rec_trx_id(rec, index, offsets);
+
+ return(view->changes_visible(trx_id, index->table->name));
+}
+
+/*********************************************************************//**
+Checks that a non-clustered index record is seen in a consistent read.
+
+NOTE that a non-clustered index page contains so little information on
+its modifications that also in the case false, the present version of
+rec may be the right, but we must check this from the clustered index
+record.
+
+@return true if certainly sees, or false if an earlier version of the
+clustered index record might be needed */
+bool
+lock_sec_rec_cons_read_sees(
+/*========================*/
+ const rec_t* rec, /*!< in: user record which
+ should be read or passed over
+ by a read cursor */
+ const dict_index_t* index, /*!< in: index */
+ const ReadView* view) /*!< in: consistent read view */
+{
+ ut_ad(page_rec_is_user_rec(rec));
+ ut_ad(!index->is_primary());
+ ut_ad(!rec_is_metadata(rec, *index));
+
+ /* NOTE that we might call this function while holding the search
+ system latch. */
+
+ if (index->table->is_temporary()) {
+
+ /* Temp-tables are not shared across connections and multiple
+ transactions from different connections cannot simultaneously
+ operate on same temp-table and so read of temp-table is
+ always consistent read. */
+
+ return(true);
+ }
+
+ trx_id_t max_trx_id = page_get_max_trx_id(page_align(rec));
+
+ ut_ad(max_trx_id > 0);
+
+ return(view->sees(max_trx_id));
+}
+
+
+/**
+ Creates the lock system at database start.
+
+ @param[in] n_cells number of slots in lock hash table
+*/
+void lock_sys_t::create(ulint n_cells)
+{
+ ut_ad(this == &lock_sys);
+
+ m_initialised= true;
+
+ waiting_threads = static_cast<srv_slot_t*>
+ (ut_zalloc_nokey(srv_max_n_threads * sizeof *waiting_threads));
+ last_slot = waiting_threads;
+
+ mutex_create(LATCH_ID_LOCK_SYS, &mutex);
+
+ mutex_create(LATCH_ID_LOCK_SYS_WAIT, &wait_mutex);
+
+
+ rec_hash.create(n_cells);
+ prdt_hash.create(n_cells);
+ prdt_page_hash.create(n_cells);
+
+ if (!srv_read_only_mode) {
+ lock_latest_err_file = os_file_create_tmpfile();
+ ut_a(lock_latest_err_file);
+ }
+ timeout_timer_active = false;
+}
+
+/** Calculates the fold value of a lock: used in migrating the hash table.
+@param[in] lock record lock object
+@return folded value */
+static ulint lock_rec_lock_fold(const lock_t *lock)
+{
+ return lock->un_member.rec_lock.page_id.fold();
+}
+
+
+/**
+ Resize the lock hash table.
+
+ @param[in] n_cells number of slots in lock hash table
+*/
+void lock_sys_t::resize(ulint n_cells)
+{
+ ut_ad(this == &lock_sys);
+
+ mutex_enter(&mutex);
+
+ hash_table_t old_hash(rec_hash);
+ rec_hash.create(n_cells);
+ HASH_MIGRATE(&old_hash, &rec_hash, lock_t, hash,
+ lock_rec_lock_fold);
+ old_hash.free();
+
+ old_hash = prdt_hash;
+ prdt_hash.create(n_cells);
+ HASH_MIGRATE(&old_hash, &prdt_hash, lock_t, hash,
+ lock_rec_lock_fold);
+ old_hash.free();
+
+ old_hash = prdt_page_hash;
+ prdt_page_hash.create(n_cells);
+ HASH_MIGRATE(&old_hash, &prdt_page_hash, lock_t, hash,
+ lock_rec_lock_fold);
+ old_hash.free();
+ mutex_exit(&mutex);
+}
+
+
+/** Closes the lock system at database shutdown. */
+void lock_sys_t::close()
+{
+ ut_ad(this == &lock_sys);
+
+ if (!m_initialised) return;
+
+ if (lock_latest_err_file != NULL) {
+ my_fclose(lock_latest_err_file, MYF(MY_WME));
+ lock_latest_err_file = NULL;
+ }
+
+ rec_hash.free();
+ prdt_hash.free();
+ prdt_page_hash.free();
+
+ mutex_destroy(&mutex);
+ mutex_destroy(&wait_mutex);
+
+ for (ulint i = srv_max_n_threads; i--; ) {
+ if (os_event_t& event = waiting_threads[i].event) {
+ os_event_destroy(event);
+ }
+ }
+
+ ut_free(waiting_threads);
+ m_initialised= false;
+}
+
+/*********************************************************************//**
+Gets the size of a lock struct.
+@return size in bytes */
+ulint
+lock_get_size(void)
+/*===============*/
+{
+ return((ulint) sizeof(lock_t));
+}
+
+static inline void lock_grant_have_trx_mutex(lock_t* lock)
+{
+ lock_reset_lock_and_trx_wait(lock);
+ lock_grant_after_reset(lock);
+}
+
+/*********************************************************************//**
+Gets the gap flag of a record lock.
+@return LOCK_GAP or 0 */
+UNIV_INLINE
+ulint
+lock_rec_get_gap(
+/*=============*/
+ const lock_t* lock) /*!< in: record lock */
+{
+ ut_ad(lock);
+ ut_ad(lock_get_type_low(lock) == LOCK_REC);
+
+ return(lock->type_mode & LOCK_GAP);
+}
+
+/*********************************************************************//**
+Gets the LOCK_REC_NOT_GAP flag of a record lock.
+@return LOCK_REC_NOT_GAP or 0 */
+UNIV_INLINE
+ulint
+lock_rec_get_rec_not_gap(
+/*=====================*/
+ const lock_t* lock) /*!< in: record lock */
+{
+ ut_ad(lock);
+ ut_ad(lock_get_type_low(lock) == LOCK_REC);
+
+ return(lock->type_mode & LOCK_REC_NOT_GAP);
+}
+
+/*********************************************************************//**
+Gets the waiting insert flag of a record lock.
+@return LOCK_INSERT_INTENTION or 0 */
+UNIV_INLINE
+ulint
+lock_rec_get_insert_intention(
+/*==========================*/
+ const lock_t* lock) /*!< in: record lock */
+{
+ ut_ad(lock);
+ ut_ad(lock_get_type_low(lock) == LOCK_REC);
+
+ return(lock->type_mode & LOCK_INSERT_INTENTION);
+}
+
+#ifdef UNIV_DEBUG
+#ifdef WITH_WSREP
+/** Check if both conflicting lock transaction and other transaction
+requesting record lock are brute force (BF). If they are check is
+this BF-BF wait correct and if not report BF wait and assert.
+
+@param[in] lock_rec other waiting record lock
+@param[in] trx trx requesting conflicting record lock
+*/
+static void wsrep_assert_no_bf_bf_wait(const lock_t *lock, const trx_t *trx)
+{
+ ut_ad(lock_get_type_low(lock) == LOCK_REC);
+ ut_ad(lock_mutex_own());
+ trx_t* lock_trx= lock->trx;
+
+ /* Note that we are holding lock_sys->mutex, thus we should
+ not acquire THD::LOCK_thd_data mutex below to avoid mutexing
+ order violation. */
+
+ if (!trx->is_wsrep() || !lock_trx->is_wsrep())
+ return;
+ if (UNIV_LIKELY(!wsrep_thd_is_BF(trx->mysql_thd, FALSE))
+ || UNIV_LIKELY(!wsrep_thd_is_BF(lock_trx->mysql_thd, FALSE)))
+ return;
+
+ ut_ad(trx->state == TRX_STATE_ACTIVE);
+
+ trx_mutex_enter(lock_trx);
+ const trx_state_t trx2_state= lock_trx->state;
+ trx_mutex_exit(lock_trx);
+
+ /* If transaction is already committed in memory or
+ prepared we should wait. When transaction is committed in
+ memory we held trx mutex, but not lock_sys->mutex. Therefore,
+ we could end here before transaction has time to do
+ lock_release() that is protected with lock_sys->mutex. */
+ switch (trx2_state) {
+ case TRX_STATE_COMMITTED_IN_MEMORY:
+ case TRX_STATE_PREPARED:
+ return;
+ case TRX_STATE_ACTIVE:
+ break;
+ default:
+ ut_ad("invalid state" == 0);
+ }
+
+ /* If BF - BF order is honored, i.e. trx already holding
+ record lock should be ordered before this new lock request
+ we can keep trx waiting for the lock. If conflicting
+ transaction is already aborting or rolling back for replaying
+ we can also let new transaction waiting. */
+ if (wsrep_thd_order_before(lock_trx->mysql_thd, trx->mysql_thd)
+ || wsrep_thd_is_aborting(lock_trx->mysql_thd)) {
+ return;
+ }
+
+ mtr_t mtr;
+
+ ib::error() << "Conflicting lock on table: "
+ << lock->index->table->name
+ << " index: "
+ << lock->index->name()
+ << " that has lock ";
+ lock_rec_print(stderr, lock, mtr);
+
+ ib::error() << "WSREP state: ";
+
+ wsrep_report_bf_lock_wait(trx->mysql_thd,
+ trx->id);
+ wsrep_report_bf_lock_wait(lock_trx->mysql_thd,
+ lock_trx->id);
+ /* BF-BF wait is a bug */
+ ut_error;
+}
+#endif /* WITH_WSREP */
+#endif /* UNIV_DEBUG */
+
+/*********************************************************************//**
+Checks if a lock request for a new lock has to wait for request lock2.
+@return TRUE if new lock has to wait for lock2 to be removed */
+UNIV_INLINE
+bool
+lock_rec_has_to_wait(
+/*=================*/
+ bool for_locking,
+ /*!< in is called locking or releasing */
+ const trx_t* trx, /*!< in: trx of new lock */
+ unsigned type_mode,/*!< in: precise mode of the new lock
+ to set: LOCK_S or LOCK_X, possibly
+ ORed to LOCK_GAP or LOCK_REC_NOT_GAP,
+ LOCK_INSERT_INTENTION */
+ const lock_t* lock2, /*!< in: another record lock; NOTE that
+ it is assumed that this has a lock bit
+ set on the same record as in the new
+ lock we are setting */
+ bool lock_is_on_supremum)
+ /*!< in: TRUE if we are setting the
+ lock on the 'supremum' record of an
+ index page: we know then that the lock
+ request is really for a 'gap' type lock */
+{
+ ut_ad(trx && lock2);
+ ut_ad(lock_get_type_low(lock2) == LOCK_REC);
+ ut_ad(lock_mutex_own());
+
+ if (trx == lock2->trx
+ || lock_mode_compatible(
+ static_cast<lock_mode>(LOCK_MODE_MASK & type_mode),
+ lock_get_mode(lock2))) {
+ return false;
+ }
+
+ /* We have somewhat complex rules when gap type record locks
+ cause waits */
+
+ if ((lock_is_on_supremum || (type_mode & LOCK_GAP))
+ && !(type_mode & LOCK_INSERT_INTENTION)) {
+
+ /* Gap type locks without LOCK_INSERT_INTENTION flag
+ do not need to wait for anything. This is because
+ different users can have conflicting lock types
+ on gaps. */
+
+ return false;
+ }
+
+ if (!(type_mode & LOCK_INSERT_INTENTION) && lock_rec_get_gap(lock2)) {
+
+ /* Record lock (LOCK_ORDINARY or LOCK_REC_NOT_GAP
+ does not need to wait for a gap type lock */
+
+ return false;
+ }
+
+ if ((type_mode & LOCK_GAP) && lock_rec_get_rec_not_gap(lock2)) {
+
+ /* Lock on gap does not need to wait for
+ a LOCK_REC_NOT_GAP type lock */
+
+ return false;
+ }
+
+ if (lock_rec_get_insert_intention(lock2)) {
+
+ /* No lock request needs to wait for an insert
+ intention lock to be removed. This is ok since our
+ rules allow conflicting locks on gaps. This eliminates
+ a spurious deadlock caused by a next-key lock waiting
+ for an insert intention lock; when the insert
+ intention lock was granted, the insert deadlocked on
+ the waiting next-key lock.
+
+ Also, insert intention locks do not disturb each
+ other. */
+
+ return false;
+ }
+
+ if ((type_mode & LOCK_GAP || lock_rec_get_gap(lock2))
+ && !thd_need_ordering_with(trx->mysql_thd, lock2->trx->mysql_thd)) {
+ /* If the upper server layer has already decided on the
+ commit order between the transaction requesting the
+ lock and the transaction owning the lock, we do not
+ need to wait for gap locks. Such ordeering by the upper
+ server layer happens in parallel replication, where the
+ commit order is fixed to match the original order on the
+ master.
+
+ Such gap locks are mainly needed to get serialisability
+ between transactions so that they will be binlogged in
+ the correct order so that statement-based replication
+ will give the correct results. Since the right order
+ was already determined on the master, we do not need
+ to enforce it again here.
+
+ Skipping the locks is not essential for correctness,
+ since in case of deadlock we will just kill the later
+ transaction and retry it. But it can save some
+ unnecessary rollbacks and retries. */
+
+ return false;
+ }
+
+#ifdef WITH_WSREP
+ /* New lock request from a transaction is using unique key
+ scan and this transaction is a wsrep high priority transaction
+ (brute force). If conflicting transaction is also wsrep high
+ priority transaction we should avoid lock conflict because
+ ordering of these transactions is already decided and
+ conflicting transaction will be later replayed. Note
+ that thread holding conflicting lock can't be
+ committed or rolled back while we hold
+ lock_sys->mutex. */
+ if (trx->is_wsrep_UK_scan()
+ && wsrep_thd_is_BF(lock2->trx->mysql_thd, false)) {
+ return false;
+ }
+
+ /* We very well can let bf to wait normally as other
+ BF will be replayed in case of conflict. For debug
+ builds we will do additional sanity checks to catch
+ unsupported bf wait if any. */
+ ut_d(wsrep_assert_no_bf_bf_wait(lock2, trx));
+#endif /* WITH_WSREP */
+
+ return true;
+}
+
+/*********************************************************************//**
+Checks if a lock request lock1 has to wait for request lock2.
+@return TRUE if lock1 has to wait for lock2 to be removed */
+bool
+lock_has_to_wait(
+/*=============*/
+ const lock_t* lock1, /*!< in: waiting lock */
+ const lock_t* lock2) /*!< in: another lock; NOTE that it is
+ assumed that this has a lock bit set
+ on the same record as in lock1 if the
+ locks are record locks */
+{
+ ut_ad(lock1 && lock2);
+
+ if (lock1->trx == lock2->trx
+ || lock_mode_compatible(lock_get_mode(lock1),
+ lock_get_mode(lock2))) {
+ return false;
+ }
+
+ if (lock_get_type_low(lock1) != LOCK_REC) {
+ return true;
+ }
+
+ ut_ad(lock_get_type_low(lock2) == LOCK_REC);
+
+ if (lock1->type_mode & (LOCK_PREDICATE | LOCK_PRDT_PAGE)) {
+ return lock_prdt_has_to_wait(lock1->trx, lock1->type_mode,
+ lock_get_prdt_from_lock(lock1),
+ lock2);
+ }
+
+ return lock_rec_has_to_wait(
+ false, lock1->trx, lock1->type_mode, lock2,
+ lock_rec_get_nth_bit(lock1, PAGE_HEAP_NO_SUPREMUM));
+}
+
+/*============== RECORD LOCK BASIC FUNCTIONS ============================*/
+
+/**********************************************************************//**
+Looks for a set bit in a record lock bitmap. Returns ULINT_UNDEFINED,
+if none found.
+@return bit index == heap number of the record, or ULINT_UNDEFINED if
+none found */
+ulint
+lock_rec_find_set_bit(
+/*==================*/
+ const lock_t* lock) /*!< in: record lock with at least one bit set */
+{
+ for (ulint i = 0; i < lock_rec_get_n_bits(lock); ++i) {
+
+ if (lock_rec_get_nth_bit(lock, i)) {
+
+ return(i);
+ }
+ }
+
+ return(ULINT_UNDEFINED);
+}
+
+/*********************************************************************//**
+Resets the record lock bitmap to zero. NOTE: does not touch the wait_lock
+pointer in the transaction! This function is used in lock object creation
+and resetting. */
+static
+void
+lock_rec_bitmap_reset(
+/*==================*/
+ lock_t* lock) /*!< in: record lock */
+{
+ ulint n_bytes;
+
+ ut_ad(lock_get_type_low(lock) == LOCK_REC);
+
+ /* Reset to zero the bitmap which resides immediately after the lock
+ struct */
+
+ n_bytes = lock_rec_get_n_bits(lock) / 8;
+
+ ut_ad((lock_rec_get_n_bits(lock) % 8) == 0);
+
+ memset(reinterpret_cast<void*>(&lock[1]), 0, n_bytes);
+}
+
+/*********************************************************************//**
+Copies a record lock to heap.
+@return copy of lock */
+static
+lock_t*
+lock_rec_copy(
+/*==========*/
+ const lock_t* lock, /*!< in: record lock */
+ mem_heap_t* heap) /*!< in: memory heap */
+{
+ ulint size;
+
+ ut_ad(lock_get_type_low(lock) == LOCK_REC);
+
+ size = sizeof(lock_t) + lock_rec_get_n_bits(lock) / 8;
+
+ return(static_cast<lock_t*>(mem_heap_dup(heap, lock, size)));
+}
+
+/*********************************************************************//**
+Gets the previous record lock set on a record.
+@return previous lock on the same record, NULL if none exists */
+const lock_t*
+lock_rec_get_prev(
+/*==============*/
+ const lock_t* in_lock,/*!< in: record lock */
+ ulint heap_no)/*!< in: heap number of the record */
+{
+ lock_t* lock;
+ lock_t* found_lock = NULL;
+
+ ut_ad(lock_mutex_own());
+ ut_ad(lock_get_type_low(in_lock) == LOCK_REC);
+
+ for (lock = lock_sys.get_first(*lock_hash_get(in_lock->type_mode),
+ in_lock->un_member.rec_lock.page_id);
+ lock != in_lock;
+ lock = lock_rec_get_next_on_page(lock)) {
+ if (lock_rec_get_nth_bit(lock, heap_no)) {
+ found_lock = lock;
+ }
+ }
+
+ return found_lock;
+}
+
+/*============= FUNCTIONS FOR ANALYZING RECORD LOCK QUEUE ================*/
+
+/*********************************************************************//**
+Checks if a transaction has a GRANTED explicit lock on rec stronger or equal
+to precise_mode.
+@return lock or NULL */
+UNIV_INLINE
+lock_t*
+lock_rec_has_expl(
+/*==============*/
+ ulint precise_mode,/*!< in: LOCK_S or LOCK_X
+ possibly ORed to LOCK_GAP or
+ LOCK_REC_NOT_GAP, for a
+ supremum record we regard this
+ always a gap type request */
+ const buf_block_t* block, /*!< in: buffer block containing
+ the record */
+ ulint heap_no,/*!< in: heap number of the record */
+ const trx_t* trx) /*!< in: transaction */
+{
+ lock_t* lock;
+
+ ut_ad(lock_mutex_own());
+ 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 = lock_rec_get_first(&lock_sys.rec_hash, block, heap_no);
+ lock != NULL;
+ lock = lock_rec_get_next(heap_no, lock)) {
+
+ if (lock->trx == trx
+ && !lock_rec_get_insert_intention(lock)
+ && lock_mode_stronger_or_eq(
+ lock_get_mode(lock),
+ static_cast<lock_mode>(
+ precise_mode & LOCK_MODE_MASK))
+ && !lock_get_wait(lock)
+ && (!lock_rec_get_rec_not_gap(lock)
+ || (precise_mode & LOCK_REC_NOT_GAP)
+ || heap_no == PAGE_HEAP_NO_SUPREMUM)
+ && (!lock_rec_get_gap(lock)
+ || (precise_mode & LOCK_GAP)
+ || heap_no == PAGE_HEAP_NO_SUPREMUM)) {
+
+ return(lock);
+ }
+ }
+
+ return(NULL);
+}
+
+#ifdef UNIV_DEBUG
+/*********************************************************************//**
+Checks if some other transaction has a lock request in the queue.
+@return lock or NULL */
+static
+lock_t*
+lock_rec_other_has_expl_req(
+/*========================*/
+ lock_mode mode, /*!< in: LOCK_S or LOCK_X */
+ const buf_block_t* block, /*!< in: buffer block containing
+ the record */
+ bool wait, /*!< in: whether also waiting locks
+ are taken into account */
+ ulint heap_no,/*!< in: heap number of the record */
+ const trx_t* trx) /*!< in: transaction, or NULL if
+ requests by all transactions
+ are taken into account */
+{
+
+ ut_ad(lock_mutex_own());
+ ut_ad(mode == LOCK_X || mode == LOCK_S);
+
+ /* Only GAP lock can be on SUPREMUM, and we are not looking for
+ GAP lock */
+ if (heap_no == PAGE_HEAP_NO_SUPREMUM) {
+ return(NULL);
+ }
+
+ for (lock_t* lock = lock_rec_get_first(&lock_sys.rec_hash,
+ block, heap_no);
+ lock != NULL;
+ lock = lock_rec_get_next(heap_no, lock)) {
+
+ if (lock->trx != trx
+ && !lock_rec_get_gap(lock)
+ && (wait || !lock_get_wait(lock))
+ && lock_mode_stronger_or_eq(lock_get_mode(lock), mode)) {
+
+ return(lock);
+ }
+ }
+
+ return(NULL);
+}
+#endif /* UNIV_DEBUG */
+
+#ifdef WITH_WSREP
+static void wsrep_kill_victim(const trx_t * const trx, const lock_t *lock)
+{
+ ut_ad(lock_mutex_own());
+ ut_ad(trx->is_wsrep());
+ trx_t* lock_trx = lock->trx;
+ ut_ad(trx_mutex_own(lock_trx));
+ ut_ad(lock_trx != trx);
+
+ if (!wsrep_thd_is_BF(trx->mysql_thd, FALSE))
+ return;
+
+ if (lock_trx->state == TRX_STATE_COMMITTED_IN_MEMORY
+ || lock_trx->lock.was_chosen_as_deadlock_victim)
+ return;
+
+ if (!wsrep_thd_is_BF(lock_trx->mysql_thd, FALSE)
+ || wsrep_thd_order_before(trx->mysql_thd, lock_trx->mysql_thd)) {
+ if (lock_trx->lock.que_state == TRX_QUE_LOCK_WAIT) {
+ if (UNIV_UNLIKELY(wsrep_debug))
+ WSREP_INFO("BF victim waiting");
+ /* cannot release lock, until our lock
+ is in the queue*/
+ } else {
+ wsrep_innobase_kill_one_trx(trx->mysql_thd,
+ lock_trx, true);
+ }
+ }
+}
+#endif /* WITH_WSREP */
+
+/*********************************************************************//**
+Checks if some other transaction has a conflicting explicit lock request
+in the queue, so that we have to wait.
+@return lock or NULL */
+static
+lock_t*
+lock_rec_other_has_conflicting(
+/*===========================*/
+ unsigned mode, /*!< in: LOCK_S or LOCK_X,
+ possibly ORed to LOCK_GAP or
+ LOC_REC_NOT_GAP,
+ LOCK_INSERT_INTENTION */
+ const buf_block_t* block, /*!< in: buffer block containing
+ the record */
+ ulint heap_no,/*!< in: heap number of the record */
+ const trx_t* trx) /*!< in: our transaction */
+{
+ lock_t* lock;
+
+ ut_ad(lock_mutex_own());
+
+ bool is_supremum = (heap_no == PAGE_HEAP_NO_SUPREMUM);
+
+ for (lock = lock_rec_get_first(&lock_sys.rec_hash, block, heap_no);
+ lock != NULL;
+ lock = lock_rec_get_next(heap_no, lock)) {
+
+ if (lock_rec_has_to_wait(true, trx, mode, lock, is_supremum)) {
+#ifdef WITH_WSREP
+ if (trx->is_wsrep()) {
+ trx_mutex_enter(lock->trx);
+ /* Below function will roll back either trx
+ or lock->trx depending on priority of the
+ transaction. */
+ wsrep_kill_victim(const_cast<trx_t*>(trx), lock);
+ trx_mutex_exit(lock->trx);
+ }
+#endif /* WITH_WSREP */
+ return(lock);
+ }
+ }
+
+ return(NULL);
+}
+
+/*********************************************************************//**
+Checks if some transaction has an implicit x-lock on a record in a secondary
+index.
+@return transaction id of the transaction which has the x-lock, or 0;
+NOTE that this function can return false positives but never false
+negatives. The caller must confirm all positive results by calling
+trx_is_active(). */
+static
+trx_t*
+lock_sec_rec_some_has_impl(
+/*=======================*/
+ trx_t* caller_trx,/*!<in/out: trx of current thread */
+ const rec_t* rec, /*!< in: user record */
+ dict_index_t* index, /*!< in: secondary index */
+ const rec_offs* offsets)/*!< in: rec_get_offsets(rec, index) */
+{
+ trx_t* trx;
+ trx_id_t max_trx_id;
+ const page_t* page = page_align(rec);
+
+ ut_ad(!lock_mutex_own());
+ ut_ad(!dict_index_is_clust(index));
+ ut_ad(page_rec_is_user_rec(rec));
+ ut_ad(rec_offs_validate(rec, index, offsets));
+ ut_ad(!rec_is_metadata(rec, *index));
+
+ max_trx_id = page_get_max_trx_id(page);
+
+ /* Some transaction may have an implicit x-lock on the record only
+ if the max trx id for the page >= min trx id for the trx list, or
+ database recovery is running. */
+
+ if (max_trx_id < trx_sys.get_min_trx_id()) {
+
+ trx = 0;
+
+ } else if (!lock_check_trx_id_sanity(max_trx_id, rec, index, offsets)) {
+
+ /* The page is corrupt: try to avoid a crash by returning 0 */
+ trx = 0;
+
+ /* In this case it is possible that some transaction has an implicit
+ x-lock. We have to look in the clustered index. */
+
+ } else {
+ trx = row_vers_impl_x_locked(caller_trx, rec, index, offsets);
+ }
+
+ return(trx);
+}
+
+/*********************************************************************//**
+Return approximate number or record locks (bits set in the bitmap) for
+this transaction. Since delete-marked records may be removed, the
+record count will not be precise.
+The caller must be holding lock_sys.mutex. */
+ulint
+lock_number_of_rows_locked(
+/*=======================*/
+ const trx_lock_t* trx_lock) /*!< in: transaction locks */
+{
+ ut_ad(lock_mutex_own());
+
+ return(trx_lock->n_rec_locks);
+}
+
+/*********************************************************************//**
+Return the number of table locks for a transaction.
+The caller must be holding lock_sys.mutex. */
+ulint
+lock_number_of_tables_locked(
+/*=========================*/
+ const trx_lock_t* trx_lock) /*!< in: transaction locks */
+{
+ const lock_t* lock;
+ ulint n_tables = 0;
+
+ ut_ad(lock_mutex_own());
+
+ for (lock = UT_LIST_GET_FIRST(trx_lock->trx_locks);
+ lock != NULL;
+ lock = UT_LIST_GET_NEXT(trx_locks, lock)) {
+
+ if (lock_get_type_low(lock) == LOCK_TABLE) {
+ n_tables++;
+ }
+ }
+
+ return(n_tables);
+}
+
+/*============== RECORD LOCK CREATION AND QUEUE MANAGEMENT =============*/
+
+#ifdef WITH_WSREP
+ATTRIBUTE_COLD
+static
+void
+wsrep_print_wait_locks(
+/*===================*/
+ lock_t* c_lock) /* conflicting lock to print */
+{
+ if (c_lock->trx->lock.wait_lock != c_lock) {
+ mtr_t mtr;
+ ib::info() << "WSREP: c_lock != wait lock";
+ ib::info() << " SQL: "
+ << wsrep_thd_query(c_lock->trx->mysql_thd);
+
+ if (lock_get_type_low(c_lock) & LOCK_TABLE) {
+ lock_table_print(stderr, c_lock);
+ } else {
+ lock_rec_print(stderr, c_lock, mtr);
+ }
+
+ if (lock_get_type_low(c_lock->trx->lock.wait_lock) & LOCK_TABLE) {
+ lock_table_print(stderr, c_lock->trx->lock.wait_lock);
+ } else {
+ lock_rec_print(stderr, c_lock->trx->lock.wait_lock,
+ mtr);
+ }
+ }
+}
+#endif /* WITH_WSREP */
+
+#ifdef UNIV_DEBUG
+/** Check transaction state */
+static void check_trx_state(const trx_t *trx)
+{
+ ut_ad(!trx->auto_commit || trx->will_lock);
+ const auto state= trx->state;
+ ut_ad(state == TRX_STATE_ACTIVE ||
+ state == TRX_STATE_PREPARED_RECOVERED ||
+ state == TRX_STATE_PREPARED ||
+ state == TRX_STATE_COMMITTED_IN_MEMORY);
+}
+#endif
+
+/** Create a new record lock and inserts it to the lock queue,
+without checking for deadlocks or conflicts.
+@param[in] type_mode lock mode and wait flag; type will be replaced
+ with LOCK_REC
+@param[in] page_id index page number
+@param[in] page R-tree index page, or NULL
+@param[in] heap_no record heap number in the index page
+@param[in] index the index tree
+@param[in,out] trx transaction
+@param[in] holds_trx_mutex whether the caller holds trx->mutex
+@return created lock */
+lock_t*
+lock_rec_create_low(
+#ifdef WITH_WSREP
+ lock_t* c_lock, /*!< conflicting lock */
+ que_thr_t* thr, /*!< thread owning trx */
+#endif
+ unsigned type_mode,
+ const page_id_t page_id,
+ const page_t* page,
+ ulint heap_no,
+ dict_index_t* index,
+ trx_t* trx,
+ bool holds_trx_mutex)
+{
+ lock_t* lock;
+ ulint n_bits;
+ ulint n_bytes;
+
+ ut_ad(lock_mutex_own());
+ ut_ad(holds_trx_mutex == trx_mutex_own(trx));
+ ut_ad(dict_index_is_clust(index) || !dict_index_is_online_ddl(index));
+
+#ifdef UNIV_DEBUG
+ /* Non-locking autocommit read-only transactions should not set
+ any locks. See comment in trx_set_rw_mode explaining why this
+ conditional check is required in debug code. */
+ if (holds_trx_mutex) {
+ check_trx_state(trx);
+ }
+#endif /* UNIV_DEBUG */
+
+ /* If rec is the supremum record, then we reset the gap and
+ LOCK_REC_NOT_GAP bits, as all locks on the supremum are
+ automatically of the gap type */
+
+ if (UNIV_UNLIKELY(heap_no == PAGE_HEAP_NO_SUPREMUM)) {
+ ut_ad(!(type_mode & LOCK_REC_NOT_GAP));
+ type_mode = type_mode & ~(LOCK_GAP | LOCK_REC_NOT_GAP);
+ }
+
+ if (UNIV_LIKELY(!(type_mode & (LOCK_PREDICATE | LOCK_PRDT_PAGE)))) {
+ /* Make lock bitmap bigger by a safety margin */
+ n_bits = page_dir_get_n_heap(page) + LOCK_PAGE_BITMAP_MARGIN;
+ n_bytes = 1 + n_bits / 8;
+ } else {
+ ut_ad(heap_no == PRDT_HEAPNO);
+
+ /* The lock is always on PAGE_HEAP_NO_INFIMUM (0), so
+ we only need 1 bit (which round up to 1 byte) for
+ lock bit setting */
+ n_bytes = 1;
+
+ if (type_mode & LOCK_PREDICATE) {
+ ulint tmp = UNIV_WORD_SIZE - 1;
+
+ /* We will attach predicate structure after lock.
+ Make sure the memory is aligned on 8 bytes,
+ the mem_heap_alloc will align it with
+ MEM_SPACE_NEEDED anyway. */
+ n_bytes = (n_bytes + sizeof(lock_prdt_t) + tmp) & ~tmp;
+ ut_ad(n_bytes == sizeof(lock_prdt_t) + UNIV_WORD_SIZE);
+ }
+ }
+
+ if (trx->lock.rec_cached >= UT_ARR_SIZE(trx->lock.rec_pool)
+ || sizeof *lock + n_bytes > sizeof *trx->lock.rec_pool) {
+ lock = static_cast<lock_t*>(
+ mem_heap_alloc(trx->lock.lock_heap,
+ sizeof *lock + n_bytes));
+ } else {
+ lock = &trx->lock.rec_pool[trx->lock.rec_cached++].lock;
+ }
+
+ lock->trx = trx;
+ lock->type_mode = (type_mode & unsigned(~LOCK_TYPE_MASK)) | LOCK_REC;
+ lock->index = index;
+ lock->un_member.rec_lock.page_id = page_id;
+
+ if (UNIV_LIKELY(!(type_mode & (LOCK_PREDICATE | LOCK_PRDT_PAGE)))) {
+ lock->un_member.rec_lock.n_bits = uint32_t(n_bytes * 8);
+ } else {
+ /* Predicate lock always on INFIMUM (0) */
+ lock->un_member.rec_lock.n_bits = 8;
+ }
+ lock_rec_bitmap_reset(lock);
+ lock_rec_set_nth_bit(lock, heap_no);
+ index->table->n_rec_locks++;
+ ut_ad(index->table->get_ref_count() > 0 || !index->table->can_be_evicted);
+
+#ifdef WITH_WSREP
+ if (c_lock && trx->is_wsrep()
+ && wsrep_thd_is_BF(trx->mysql_thd, FALSE)) {
+ lock_t *hash = (lock_t *)c_lock->hash;
+ lock_t *prev = NULL;
+
+ while (hash && wsrep_thd_is_BF(hash->trx->mysql_thd, FALSE)
+ && wsrep_thd_order_before(hash->trx->mysql_thd,
+ trx->mysql_thd)) {
+ prev = hash;
+ hash = (lock_t *)hash->hash;
+ }
+ lock->hash = hash;
+ if (prev) {
+ prev->hash = lock;
+ } else {
+ c_lock->hash = lock;
+ }
+ /*
+ * delayed conflict resolution '...kill_one_trx' was not called,
+ * if victim was waiting for some other lock
+ */
+ trx_mutex_enter(c_lock->trx);
+ if (c_lock->trx->lock.que_state == TRX_QUE_LOCK_WAIT) {
+
+ c_lock->trx->lock.was_chosen_as_deadlock_victim = TRUE;
+
+ if (UNIV_UNLIKELY(wsrep_debug)) {
+ wsrep_print_wait_locks(c_lock);
+ }
+
+ trx->lock.que_state = TRX_QUE_LOCK_WAIT;
+ lock_set_lock_and_trx_wait(lock, trx);
+ UT_LIST_ADD_LAST(trx->lock.trx_locks, lock);
+
+ trx->lock.wait_thr = thr;
+ thr->state = QUE_THR_LOCK_WAIT;
+
+ /* have to release trx mutex for the duration of
+ victim lock release. This will eventually call
+ lock_grant, which wants to grant trx mutex again
+ */
+ if (holds_trx_mutex) {
+ trx_mutex_exit(trx);
+ }
+ lock_cancel_waiting_and_release(
+ c_lock->trx->lock.wait_lock);
+
+ if (holds_trx_mutex) {
+ trx_mutex_enter(trx);
+ }
+
+ trx_mutex_exit(c_lock->trx);
+
+ /* have to bail out here to avoid lock_set_lock... */
+ return(lock);
+ }
+ trx_mutex_exit(c_lock->trx);
+ } else
+#endif /* WITH_WSREP */
+ if (!(type_mode & (LOCK_WAIT | LOCK_PREDICATE | LOCK_PRDT_PAGE))
+ && innodb_lock_schedule_algorithm
+ == INNODB_LOCK_SCHEDULE_ALGORITHM_VATS
+ && !thd_is_replication_slave_thread(trx->mysql_thd)) {
+ HASH_PREPEND(lock_t, hash, &lock_sys.rec_hash,
+ page_id.fold(), lock);
+ } else {
+ HASH_INSERT(lock_t, hash, lock_hash_get(type_mode),
+ page_id.fold(), lock);
+ }
+
+ if (!holds_trx_mutex) {
+ trx_mutex_enter(trx);
+ }
+ ut_ad(trx_mutex_own(trx));
+ if (type_mode & LOCK_WAIT) {
+ lock_set_lock_and_trx_wait(lock, trx);
+ }
+ UT_LIST_ADD_LAST(trx->lock.trx_locks, lock);
+ if (!holds_trx_mutex) {
+ trx_mutex_exit(trx);
+ }
+ MONITOR_INC(MONITOR_RECLOCK_CREATED);
+ MONITOR_INC(MONITOR_NUM_RECLOCK);
+
+ return lock;
+}
+
+/*********************************************************************//**
+Check if lock1 has higher priority than lock2.
+NULL has lowest priority.
+If neither of them is wait lock, the first one has higher priority.
+If only one of them is a wait lock, it has lower priority.
+If either is a high priority transaction, the lock has higher priority.
+Otherwise, the one with an older transaction has higher priority.
+@returns true if lock1 has higher priority, false otherwise. */
+static bool has_higher_priority(lock_t *lock1, lock_t *lock2)
+{
+ if (lock1 == NULL) {
+ return false;
+ } else if (lock2 == NULL) {
+ return true;
+ }
+ // Granted locks has higher priority.
+ if (!lock_get_wait(lock1)) {
+ return true;
+ } else if (!lock_get_wait(lock2)) {
+ return false;
+ }
+ return lock1->trx->start_time_micro <= lock2->trx->start_time_micro;
+}
+
+/*********************************************************************//**
+Insert a lock to the hash list according to the mode (whether it is a wait
+lock) and the age of the transaction the it is associated with.
+If the lock is not a wait lock, insert it to the head of the hash list.
+Otherwise, insert it to the middle of the wait locks according to the age of
+the transaciton. */
+static
+dberr_t
+lock_rec_insert_by_trx_age(
+ lock_t *in_lock) /*!< in: lock to be insert */{
+ lock_t* node;
+ lock_t* next;
+ hash_table_t* hash;
+ hash_cell_t* cell;
+
+ ut_ad(!in_lock->trx->is_wsrep());
+ const page_id_t page_id(in_lock->un_member.rec_lock.page_id);
+ hash = lock_hash_get(in_lock->type_mode);
+ cell = &hash->array[hash->calc_hash(page_id.fold())];
+
+ node = (lock_t *) cell->node;
+ // If in_lock is not a wait lock, we insert it to the head of the list.
+ if (node == NULL || !lock_get_wait(in_lock) || has_higher_priority(in_lock, node)) {
+ cell->node = in_lock;
+ in_lock->hash = node;
+ if (lock_get_wait(in_lock)) {
+ lock_grant_have_trx_mutex(in_lock);
+ return DB_SUCCESS_LOCKED_REC;
+ }
+ return DB_SUCCESS;
+ }
+ while (node != NULL && has_higher_priority((lock_t *) node->hash,
+ in_lock)) {
+ node = (lock_t *) node->hash;
+ }
+ next = (lock_t *) node->hash;
+ node->hash = in_lock;
+ in_lock->hash = next;
+
+ if (lock_get_wait(in_lock) && !lock_rec_has_to_wait_in_queue(in_lock)) {
+ lock_grant_have_trx_mutex(in_lock);
+ if (cell->node != in_lock) {
+ // Move it to the front of the queue
+ node->hash = in_lock->hash;
+ next = (lock_t *) cell->node;
+ cell->node = in_lock;
+ in_lock->hash = next;
+ }
+ return DB_SUCCESS_LOCKED_REC;
+ }
+
+ return DB_SUCCESS;
+}
+
+#ifdef UNIV_DEBUG
+static
+bool
+lock_queue_validate(
+ const lock_t *in_lock) /*!< in: lock whose hash list is to be validated */
+{
+ hash_table_t* hash;
+ hash_cell_t* cell;
+ lock_t* next;
+ bool wait_lock __attribute__((unused))= false;
+
+ if (in_lock == NULL) {
+ return true;
+ }
+
+ const page_id_t page_id(in_lock->un_member.rec_lock.page_id);
+ hash = lock_hash_get(in_lock->type_mode);
+ cell = &hash->array[hash->calc_hash(page_id.fold())];
+ next = (lock_t *) cell->node;
+ while (next != NULL) {
+ // If this is a granted lock, check that there's no wait lock before it.
+ if (!lock_get_wait(next)) {
+ ut_ad(!wait_lock);
+ } else {
+ wait_lock = true;
+ }
+ next = next->hash;
+ }
+ return true;
+}
+#endif /* UNIV_DEBUG */
+
+static
+void
+lock_rec_insert_to_head(
+ lock_t *in_lock, /*!< in: lock to be insert */
+ ulint rec_fold) /*!< in: rec_fold of the page */
+{
+ hash_table_t* hash;
+ hash_cell_t* cell;
+ lock_t* node;
+
+ if (in_lock == NULL) {
+ return;
+ }
+
+ hash = lock_hash_get(in_lock->type_mode);
+ cell = &hash->array[hash->calc_hash(rec_fold)];
+ node = (lock_t *) cell->node;
+ if (node != in_lock) {
+ cell->node = in_lock;
+ in_lock->hash = node;
+ }
+}
+
+/** Enqueue a waiting request for a lock which cannot be granted immediately.
+Check for deadlocks.
+@param[in] type_mode the requested lock mode (LOCK_S or LOCK_X)
+ possibly ORed with LOCK_GAP or
+ LOCK_REC_NOT_GAP, ORed with
+ LOCK_INSERT_INTENTION if this
+ waiting lock request is set
+ when performing an insert of
+ an index record
+@param[in] block leaf page in the index
+@param[in] heap_no record heap number in the block
+@param[in] index index tree
+@param[in,out] thr query thread
+@param[in] prdt minimum bounding box (spatial index)
+@retval DB_LOCK_WAIT if the waiting lock was enqueued
+@retval DB_DEADLOCK if this transaction was chosen as the victim
+@retval DB_SUCCESS_LOCKED_REC if the other transaction was chosen as a victim
+ (or it happened to commit) */
+dberr_t
+lock_rec_enqueue_waiting(
+#ifdef WITH_WSREP
+ lock_t* c_lock, /*!< conflicting lock */
+#endif
+ unsigned type_mode,
+ const buf_block_t* block,
+ ulint heap_no,
+ dict_index_t* index,
+ que_thr_t* thr,
+ lock_prdt_t* prdt)
+{
+ ut_ad(lock_mutex_own());
+ ut_ad(!srv_read_only_mode);
+ ut_ad(dict_index_is_clust(index) || !dict_index_is_online_ddl(index));
+
+ trx_t* trx = thr_get_trx(thr);
+
+ ut_ad(trx_mutex_own(trx));
+ ut_a(!que_thr_stop(thr));
+
+ switch (trx_get_dict_operation(trx)) {
+ case TRX_DICT_OP_NONE:
+ break;
+ case TRX_DICT_OP_TABLE:
+ case TRX_DICT_OP_INDEX:
+ ib::error() << "A record lock wait happens in a dictionary"
+ " operation. index "
+ << index->name
+ << " of table "
+ << index->table->name
+ << ". " << BUG_REPORT_MSG;
+ ut_ad(0);
+ }
+
+ if (trx->mysql_thd && thd_lock_wait_timeout(trx->mysql_thd) == 0) {
+ trx->error_state = DB_LOCK_WAIT_TIMEOUT;
+ return DB_LOCK_WAIT_TIMEOUT;
+ }
+
+ /* Enqueue the lock request that will wait to be granted, note that
+ we already own the trx mutex. */
+ lock_t* lock = lock_rec_create(
+#ifdef WITH_WSREP
+ c_lock, thr,
+#endif
+ type_mode | LOCK_WAIT, block, heap_no, index, trx, TRUE);
+
+ if (prdt && type_mode & LOCK_PREDICATE) {
+ lock_prdt_set_prdt(lock, prdt);
+ }
+
+ if (ut_d(const trx_t* victim =)
+ DeadlockChecker::check_and_resolve(lock, trx)) {
+ ut_ad(victim == trx);
+ lock_reset_lock_and_trx_wait(lock);
+ lock_rec_reset_nth_bit(lock, heap_no);
+ return DB_DEADLOCK;
+ }
+
+ if (!trx->lock.wait_lock) {
+ /* If there was a deadlock but we chose another
+ transaction as a victim, it is possible that we
+ already have the lock now granted! */
+#ifdef WITH_WSREP
+ if (UNIV_UNLIKELY(wsrep_debug)) {
+ ib::info() << "WSREP: BF thread got lock granted early, ID " << ib::hex(trx->id)
+ << " query: " << wsrep_thd_query(trx->mysql_thd);
+ }
+#endif
+ return DB_SUCCESS_LOCKED_REC;
+ }
+
+ trx->lock.que_state = TRX_QUE_LOCK_WAIT;
+
+ trx->lock.was_chosen_as_deadlock_victim = false;
+ trx->lock.wait_started = time(NULL);
+
+ ut_a(que_thr_stop(thr));
+
+ DBUG_LOG("ib_lock", "trx " << ib::hex(trx->id)
+ << " waits for lock in index " << index->name
+ << " of table " << index->table->name);
+
+ MONITOR_INC(MONITOR_LOCKREC_WAIT);
+
+ if (innodb_lock_schedule_algorithm
+ == INNODB_LOCK_SCHEDULE_ALGORITHM_VATS
+ && !prdt
+ && !thd_is_replication_slave_thread(lock->trx->mysql_thd)) {
+ HASH_DELETE(lock_t, hash, &lock_sys.rec_hash,
+ lock_rec_lock_fold(lock), lock);
+ dberr_t res = lock_rec_insert_by_trx_age(lock);
+ if (res != DB_SUCCESS) {
+ return res;
+ }
+ }
+
+ return DB_LOCK_WAIT;
+}
+
+/*********************************************************************//**
+Looks for a suitable type record lock struct by the same trx on the same page.
+This can be used to save space when a new record lock should be set on a page:
+no new struct is needed, if a suitable old is found.
+@return lock or NULL */
+static inline
+lock_t*
+lock_rec_find_similar_on_page(
+ ulint type_mode, /*!< in: lock type_mode field */
+ ulint heap_no, /*!< in: heap number of the record */
+ lock_t* lock, /*!< in: lock_sys.get_first() */
+ const trx_t* trx) /*!< in: transaction */
+{
+ ut_ad(lock_mutex_own());
+
+ for (/* No op */;
+ lock != NULL;
+ lock = lock_rec_get_next_on_page(lock)) {
+
+ if (lock->trx == trx
+ && lock->type_mode == type_mode
+ && lock_rec_get_n_bits(lock) > heap_no) {
+
+ return(lock);
+ }
+ }
+
+ return(NULL);
+}
+
+/*********************************************************************//**
+Adds a record lock request in the record queue. The request is normally
+added as the last in the queue, but if there are no waiting lock requests
+on the record, and the request to be added is not a waiting request, we
+can reuse a suitable record lock object already existing on the same page,
+just setting the appropriate bit in its bitmap. This is a low-level function
+which does NOT check for deadlocks or lock compatibility!
+@return lock where the bit was set */
+static
+void
+lock_rec_add_to_queue(
+/*==================*/
+ unsigned type_mode,/*!< in: lock mode, wait, gap
+ etc. flags; type is ignored
+ and replaced by LOCK_REC */
+ const buf_block_t* block, /*!< in: buffer block containing
+ the record */
+ ulint heap_no,/*!< in: heap number of the record */
+ dict_index_t* index, /*!< in: index of record */
+ trx_t* trx, /*!< in/out: transaction */
+ bool caller_owns_trx_mutex)
+ /*!< in: TRUE if caller owns the
+ transaction mutex */
+{
+#ifdef UNIV_DEBUG
+ ut_ad(lock_mutex_own());
+ ut_ad(caller_owns_trx_mutex == trx_mutex_own(trx));
+ ut_ad(dict_index_is_clust(index)
+ || dict_index_get_online_status(index) != ONLINE_INDEX_CREATION);
+ switch (type_mode & LOCK_MODE_MASK) {
+ case LOCK_X:
+ case LOCK_S:
+ break;
+ default:
+ ut_error;
+ }
+
+ if (!(type_mode & (LOCK_WAIT | LOCK_GAP))) {
+ lock_mode mode = (type_mode & LOCK_MODE_MASK) == LOCK_S
+ ? LOCK_X
+ : LOCK_S;
+ const lock_t* other_lock
+ = lock_rec_other_has_expl_req(
+ mode, block, false, heap_no, trx);
+#ifdef WITH_WSREP
+ if (UNIV_LIKELY_NULL(other_lock) && trx->is_wsrep()) {
+ /* Only BF transaction may be granted lock
+ before other conflicting lock request. */
+ if (!wsrep_thd_is_BF(trx->mysql_thd, FALSE)
+ && !wsrep_thd_is_BF(other_lock->trx->mysql_thd, FALSE)) {
+ /* If it is not BF, this case is a bug. */
+ wsrep_report_bf_lock_wait(trx->mysql_thd, trx->id);
+ wsrep_report_bf_lock_wait(other_lock->trx->mysql_thd, other_lock->trx->id);
+ ut_error;
+ }
+ } else
+#endif /* WITH_WSREP */
+ ut_ad(!other_lock);
+ }
+#endif /* UNIV_DEBUG */
+
+ type_mode |= LOCK_REC;
+
+ /* If rec is the supremum record, then we can reset the gap bit, as
+ all locks on the supremum are automatically of the gap type, and we
+ try to avoid unnecessary memory consumption of a new record lock
+ struct for a gap type lock */
+
+ if (heap_no == PAGE_HEAP_NO_SUPREMUM) {
+ ut_ad(!(type_mode & LOCK_REC_NOT_GAP));
+
+ /* There should never be LOCK_REC_NOT_GAP on a supremum
+ record, but let us play safe */
+
+ type_mode &= ~(LOCK_GAP | LOCK_REC_NOT_GAP);
+ }
+
+ lock_t* lock;
+ lock_t* first_lock;
+
+ /* Look for a waiting lock request on the same record or on a gap */
+
+ for (first_lock = lock = lock_sys.get_first(*lock_hash_get(type_mode),
+ block->page.id());
+ lock != NULL;
+ lock = lock_rec_get_next_on_page(lock)) {
+
+ if (lock_get_wait(lock)
+ && lock_rec_get_nth_bit(lock, heap_no)) {
+
+ break;
+ }
+ }
+
+ if (lock == NULL && !(type_mode & LOCK_WAIT)) {
+
+ /* Look for a similar record lock on the same page:
+ if one is found and there are no waiting lock requests,
+ we can just set the bit */
+
+ lock = lock_rec_find_similar_on_page(
+ type_mode, heap_no, first_lock, trx);
+
+ if (lock != NULL) {
+
+ lock_rec_set_nth_bit(lock, heap_no);
+
+ return;
+ }
+ }
+
+ lock_rec_create(
+#ifdef WITH_WSREP
+ NULL, NULL,
+#endif
+ type_mode, block, heap_no, index, trx, caller_owns_trx_mutex);
+}
+
+/*********************************************************************//**
+Tries to lock the specified record in the mode requested. If not immediately
+possible, enqueues a waiting lock request. This is a low-level function
+which does NOT look at implicit locks! Checks lock compatibility within
+explicit locks. This function sets a normal next-key lock, or in the case
+of a page supremum record, a gap type lock.
+@return DB_SUCCESS, DB_SUCCESS_LOCKED_REC, DB_LOCK_WAIT, or DB_DEADLOCK */
+static
+dberr_t
+lock_rec_lock(
+/*==========*/
+ bool impl, /*!< in: if true, no lock is set
+ if no wait is necessary: we
+ assume that the caller will
+ set an implicit lock */
+ unsigned mode, /*!< in: lock mode: LOCK_X or
+ LOCK_S possibly ORed to either
+ LOCK_GAP or LOCK_REC_NOT_GAP */
+ const buf_block_t* block, /*!< in: buffer block containing
+ the record */
+ ulint heap_no,/*!< in: heap number of record */
+ dict_index_t* index, /*!< in: index of record */
+ que_thr_t* thr) /*!< in: query thread */
+{
+ trx_t *trx= thr_get_trx(thr);
+ dberr_t err= DB_SUCCESS;
+
+ ut_ad(!srv_read_only_mode);
+ ut_ad((LOCK_MODE_MASK & mode) == LOCK_S ||
+ (LOCK_MODE_MASK & mode) == LOCK_X);
+ ut_ad((mode & LOCK_TYPE_MASK) == LOCK_GAP ||
+ (mode & LOCK_TYPE_MASK) == LOCK_REC_NOT_GAP ||
+ (mode & LOCK_TYPE_MASK) == 0);
+ ut_ad(dict_index_is_clust(index) || !dict_index_is_online_ddl(index));
+ DBUG_EXECUTE_IF("innodb_report_deadlock", return DB_DEADLOCK;);
+
+ lock_mutex_enter();
+ ut_ad((LOCK_MODE_MASK & mode) != LOCK_S ||
+ lock_table_has(trx, index->table, LOCK_IS));
+ ut_ad((LOCK_MODE_MASK & mode) != LOCK_X ||
+ lock_table_has(trx, index->table, LOCK_IX));
+
+ if (lock_table_has(trx, index->table,
+ static_cast<lock_mode>(LOCK_MODE_MASK & mode)));
+ else if (lock_t *lock= lock_sys.get_first(block->page.id()))
+ {
+ trx_mutex_enter(trx);
+ if (lock_rec_get_next_on_page(lock) ||
+ lock->trx != trx ||
+ lock->type_mode != (ulint(mode) | LOCK_REC) ||
+ lock_rec_get_n_bits(lock) <= heap_no)
+ {
+ /* Do nothing if the trx already has a strong enough lock on rec */
+ if (!lock_rec_has_expl(mode, block, heap_no, trx))
+ {
+ if (
+#ifdef WITH_WSREP
+ lock_t *c_lock=
+#endif
+ lock_rec_other_has_conflicting(mode, block, heap_no, trx))
+ {
+ /*
+ If another transaction has a non-gap conflicting
+ request in the queue, as this transaction does not
+ have a lock strong enough already granted on the
+ record, we have to wait. */
+ err = lock_rec_enqueue_waiting(
+#ifdef WITH_WSREP
+ c_lock,
+#endif /* WITH_WSREP */
+ mode, block, heap_no, index, thr, NULL);
+ }
+ else if (!impl)
+ {
+ /* Set the requested lock on the record. */
+ lock_rec_add_to_queue(LOCK_REC | mode, block, heap_no, index, trx,
+ true);
+ err= DB_SUCCESS_LOCKED_REC;
+ }
+ }
+ }
+ else if (!impl)
+ {
+ /*
+ If the nth bit of the record lock is already set then we do not set
+ a new lock bit, otherwise we do set
+ */
+ if (!lock_rec_get_nth_bit(lock, heap_no))
+ {
+ lock_rec_set_nth_bit(lock, heap_no);
+ err= DB_SUCCESS_LOCKED_REC;
+ }
+ }
+ trx_mutex_exit(trx);
+ }
+ else
+ {
+ /*
+ Simplified and faster path for the most common cases
+ Note that we don't own the trx mutex.
+ */
+ if (!impl)
+ lock_rec_create(
+#ifdef WITH_WSREP
+ NULL, NULL,
+#endif
+ mode, block, heap_no, index, trx, false);
+
+ err= DB_SUCCESS_LOCKED_REC;
+ }
+ lock_mutex_exit();
+ MONITOR_ATOMIC_INC(MONITOR_NUM_RECLOCK_REQ);
+ return err;
+}
+
+/*********************************************************************//**
+Checks if a waiting record lock request still has to wait in a queue.
+@return lock that is causing the wait */
+static
+const lock_t*
+lock_rec_has_to_wait_in_queue(
+/*==========================*/
+ const lock_t* wait_lock) /*!< in: waiting record lock */
+{
+ const lock_t* lock;
+ ulint heap_no;
+ ulint bit_mask;
+ ulint bit_offset;
+
+ ut_ad(wait_lock);
+ ut_ad(lock_mutex_own());
+ ut_ad(lock_get_wait(wait_lock));
+ ut_ad(lock_get_type_low(wait_lock) == LOCK_REC);
+
+ heap_no = lock_rec_find_set_bit(wait_lock);
+
+ bit_offset = heap_no / 8;
+ bit_mask = static_cast<ulint>(1) << (heap_no % 8);
+
+ for (lock = lock_sys.get_first(*lock_hash_get(wait_lock->type_mode),
+ wait_lock->un_member.rec_lock.page_id);
+ lock != wait_lock;
+ lock = lock_rec_get_next_on_page_const(lock)) {
+ const byte* p = (const byte*) &lock[1];
+
+ if (heap_no < lock_rec_get_n_bits(lock)
+ && (p[bit_offset] & bit_mask)
+ && lock_has_to_wait(wait_lock, lock)) {
+ return(lock);
+ }
+ }
+
+ return(NULL);
+}
+
+/** Grant a lock to a waiting lock request and release the waiting transaction
+after lock_reset_lock_and_trx_wait() has been called. */
+static void lock_grant_after_reset(lock_t* lock)
+{
+ ut_ad(lock_mutex_own());
+ ut_ad(trx_mutex_own(lock->trx));
+
+ if (lock_get_mode(lock) == LOCK_AUTO_INC) {
+ dict_table_t* table = lock->un_member.tab_lock.table;
+
+ if (table->autoinc_trx == lock->trx) {
+ ib::error() << "Transaction already had an"
+ << " AUTO-INC lock!";
+ } else {
+ table->autoinc_trx = lock->trx;
+
+ ib_vector_push(lock->trx->autoinc_locks, &lock);
+ }
+ }
+
+ DBUG_PRINT("ib_lock", ("wait for trx " TRX_ID_FMT " ends",
+ trx_get_id_for_print(lock->trx)));
+
+ /* If we are resolving a deadlock by choosing another transaction
+ as a victim, then our original transaction may not be in the
+ TRX_QUE_LOCK_WAIT state, and there is no need to end the lock wait
+ for it */
+
+ if (lock->trx->lock.que_state == TRX_QUE_LOCK_WAIT) {
+ que_thr_t* thr;
+
+ thr = que_thr_end_lock_wait(lock->trx);
+
+ if (thr != NULL) {
+ lock_wait_release_thread_if_suspended(thr);
+ }
+ }
+}
+
+/** Grant a lock to a waiting lock request and release the waiting transaction. */
+static void lock_grant(lock_t* lock)
+{
+ lock_reset_lock_and_trx_wait(lock);
+ trx_mutex_enter(lock->trx);
+ lock_grant_after_reset(lock);
+ trx_mutex_exit(lock->trx);
+}
+
+/*************************************************************//**
+Cancels a waiting record lock request and releases the waiting transaction
+that requested it. NOTE: does NOT check if waiting lock requests behind this
+one can now be granted! */
+static
+void
+lock_rec_cancel(
+/*============*/
+ lock_t* lock) /*!< in: waiting record lock request */
+{
+ que_thr_t* thr;
+
+ ut_ad(lock_mutex_own());
+ ut_ad(lock_get_type_low(lock) == LOCK_REC);
+
+ /* Reset the bit (there can be only one set bit) in the lock bitmap */
+ lock_rec_reset_nth_bit(lock, lock_rec_find_set_bit(lock));
+
+ /* Reset the wait flag and the back pointer to lock in trx */
+
+ lock_reset_lock_and_trx_wait(lock);
+
+ /* The following function releases the trx from lock wait */
+
+ trx_mutex_enter(lock->trx);
+
+ thr = que_thr_end_lock_wait(lock->trx);
+
+ if (thr != NULL) {
+ lock_wait_release_thread_if_suspended(thr);
+ }
+
+ trx_mutex_exit(lock->trx);
+}
+
+static void lock_grant_and_move_on_page(ulint rec_fold, const page_id_t id)
+{
+ lock_t* lock;
+ lock_t* previous = static_cast<lock_t*>(
+ lock_sys.rec_hash.array[lock_sys.rec_hash.calc_hash(rec_fold)].
+ node);
+ if (previous == NULL) {
+ return;
+ }
+ if (previous->un_member.rec_lock.page_id == id) {
+ lock = previous;
+ }
+ else {
+ while (previous->hash &&
+ (previous->hash->un_member.rec_lock.page_id != id)) {
+ previous = previous->hash;
+ }
+ lock = previous->hash;
+ }
+
+ ut_ad(previous->hash == lock || previous == lock);
+ /* Grant locks if there are no conflicting locks ahead.
+ Move granted locks to the head of the list. */
+ while (lock) {
+ /* If the lock is a wait lock on this page, and it does not need to wait. */
+ ut_ad(!lock->trx->is_wsrep());
+ if (lock_get_wait(lock)
+ && lock->un_member.rec_lock.page_id == id
+ && !lock_rec_has_to_wait_in_queue(lock)) {
+ lock_grant(lock);
+
+ if (previous != NULL) {
+ /* Move the lock to the head of the list. */
+ HASH_GET_NEXT(hash, previous) = HASH_GET_NEXT(hash, lock);
+ lock_rec_insert_to_head(lock, rec_fold);
+ } else {
+ /* Already at the head of the list. */
+ previous = lock;
+ }
+ /* Move on to the next lock. */
+ lock = static_cast<lock_t *>(HASH_GET_NEXT(hash, previous));
+ } else {
+ previous = lock;
+ lock = static_cast<lock_t *>(HASH_GET_NEXT(hash, lock));
+ }
+ }
+}
+
+/** Remove a record lock request, waiting or granted, from the queue and
+grant locks to other transactions in the queue if they now are entitled
+to a lock. NOTE: all record locks contained in in_lock are removed.
+@param[in,out] in_lock record lock */
+static void lock_rec_dequeue_from_page(lock_t* in_lock)
+{
+ hash_table_t* lock_hash;
+
+ ut_ad(lock_mutex_own());
+ ut_ad(lock_get_type_low(in_lock) == LOCK_REC);
+ /* We may or may not be holding in_lock->trx->mutex here. */
+
+ const page_id_t page_id(in_lock->un_member.rec_lock.page_id);
+
+ in_lock->index->table->n_rec_locks--;
+
+ lock_hash = lock_hash_get(in_lock->type_mode);
+
+ const ulint rec_fold = page_id.fold();
+
+ HASH_DELETE(lock_t, hash, lock_hash, rec_fold, in_lock);
+ UT_LIST_REMOVE(in_lock->trx->lock.trx_locks, in_lock);
+
+ MONITOR_INC(MONITOR_RECLOCK_REMOVED);
+ MONITOR_DEC(MONITOR_NUM_RECLOCK);
+
+ if (innodb_lock_schedule_algorithm
+ == INNODB_LOCK_SCHEDULE_ALGORITHM_FCFS
+ || lock_hash != &lock_sys.rec_hash
+ || thd_is_replication_slave_thread(in_lock->trx->mysql_thd)) {
+ /* Check if waiting locks in the queue can now be granted:
+ grant locks if there are no conflicting locks ahead. Stop at
+ the first X lock that is waiting or has been granted. */
+
+ for (lock_t* lock = lock_sys.get_first(*lock_hash, page_id);
+ lock != NULL;
+ lock = lock_rec_get_next_on_page(lock)) {
+
+ if (!lock_get_wait(lock)) {
+ continue;
+ }
+ const lock_t* c = lock_rec_has_to_wait_in_queue(lock);
+ if (!c) {
+ /* Grant the lock */
+ ut_ad(lock->trx != in_lock->trx);
+ lock_grant(lock);
+ }
+ }
+ } else {
+ lock_grant_and_move_on_page(rec_fold, page_id);
+ }
+}
+
+/*************************************************************//**
+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 */
+{
+ trx_lock_t* trx_lock;
+
+ ut_ad(lock_mutex_own());
+ ut_ad(lock_get_type_low(in_lock) == LOCK_REC);
+
+ trx_lock = &in_lock->trx->lock;
+
+ in_lock->index->table->n_rec_locks--;
+
+ HASH_DELETE(lock_t, hash, lock_hash_get(in_lock->type_mode),
+ in_lock->un_member.rec_lock.page_id.fold(), in_lock);
+
+ UT_LIST_REMOVE(trx_lock->trx_locks, in_lock);
+
+ MONITOR_INC(MONITOR_RECLOCK_REMOVED);
+ MONITOR_DEC(MONITOR_NUM_RECLOCK);
+}
+
+/*************************************************************//**
+Removes record lock objects set on 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. */
+static void lock_rec_free_all_from_discard_page_low(const page_id_t id,
+ hash_table_t *lock_hash)
+{
+ lock_t *lock= lock_sys.get_first(*lock_hash, id);
+
+ while (lock)
+ {
+ ut_ad(lock_rec_find_set_bit(lock) == ULINT_UNDEFINED);
+ ut_ad(!lock_get_wait(lock));
+ lock_t *next_lock= lock_rec_get_next_on_page(lock);
+ lock_rec_discard(lock);
+ lock= next_lock;
+ }
+}
+
+/*************************************************************//**
+Removes record lock objects set on 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 buf_block_t* block) /*!< in: page to be discarded */
+{
+ const page_id_t page_id(block->page.id());
+ lock_rec_free_all_from_discard_page_low(page_id, &lock_sys.rec_hash);
+ lock_rec_free_all_from_discard_page_low(page_id, &lock_sys.prdt_hash);
+ lock_rec_free_all_from_discard_page_low(page_id, &lock_sys.prdt_page_hash);
+}
+
+/*============= RECORD LOCK MOVING AND INHERITING ===================*/
+
+/*************************************************************//**
+Resets the lock bits for a single record. Releases transactions waiting for
+lock requests here. */
+static
+void
+lock_rec_reset_and_release_wait_low(
+/*================================*/
+ hash_table_t* hash, /*!< in: hash table */
+ const buf_block_t* block, /*!< in: buffer block containing
+ the record */
+ ulint heap_no)/*!< in: heap number of record */
+{
+ lock_t* lock;
+
+ ut_ad(lock_mutex_own());
+
+ for (lock = lock_rec_get_first(hash, block, heap_no);
+ lock != NULL;
+ lock = lock_rec_get_next(heap_no, lock)) {
+
+ if (lock_get_wait(lock)) {
+ lock_rec_cancel(lock);
+ } else {
+ lock_rec_reset_nth_bit(lock, heap_no);
+ }
+ }
+}
+
+/*************************************************************//**
+Resets the lock bits for a single record. Releases transactions waiting for
+lock requests here. */
+static
+void
+lock_rec_reset_and_release_wait(
+/*============================*/
+ const buf_block_t* block, /*!< in: buffer block containing
+ the record */
+ ulint heap_no)/*!< in: heap number of record */
+{
+ lock_rec_reset_and_release_wait_low(
+ &lock_sys.rec_hash, block, heap_no);
+
+ lock_rec_reset_and_release_wait_low(
+ &lock_sys.prdt_hash, block, PAGE_HEAP_NO_INFIMUM);
+ lock_rec_reset_and_release_wait_low(
+ &lock_sys.prdt_page_hash, block, PAGE_HEAP_NO_INFIMUM);
+}
+
+/*************************************************************//**
+Makes a record to inherit the locks (except LOCK_INSERT_INTENTION type)
+of another record as gap type locks, but does not reset the lock bits of
+the other record. Also waiting lock requests on rec are inherited as
+GRANTED gap locks. */
+static
+void
+lock_rec_inherit_to_gap(
+/*====================*/
+ const buf_block_t* heir_block, /*!< in: block containing the
+ record which inherits */
+ const buf_block_t* block, /*!< in: block containing the
+ record from which inherited;
+ does NOT reset the locks on
+ this record */
+ ulint heir_heap_no, /*!< in: heap_no of the
+ inheriting record */
+ ulint heap_no) /*!< in: heap_no of the
+ donating record */
+{
+ lock_t* lock;
+
+ ut_ad(lock_mutex_own());
+
+ /* At READ UNCOMMITTED or READ COMMITTED isolation level,
+ we do not want locks set
+ by an UPDATE or a DELETE to be inherited as gap type locks. But we
+ DO want S-locks/X-locks(taken for replace) set by a consistency
+ constraint to be inherited also then. */
+
+ for (lock = lock_rec_get_first(&lock_sys.rec_hash, block, heap_no);
+ lock != NULL;
+ lock = lock_rec_get_next(heap_no, lock)) {
+
+ if (!lock_rec_get_insert_intention(lock)
+ && (lock->trx->isolation_level > TRX_ISO_READ_COMMITTED
+ || lock_get_mode(lock) !=
+ (lock->trx->duplicates ? LOCK_S : LOCK_X))) {
+ lock_rec_add_to_queue(
+ LOCK_REC | LOCK_GAP | lock_get_mode(lock),
+ heir_block, heir_heap_no, lock->index,
+ lock->trx, FALSE);
+ }
+ }
+}
+
+/*************************************************************//**
+Makes a record to inherit the gap locks (except LOCK_INSERT_INTENTION type)
+of another record as gap type locks, but does not reset the lock bits of the
+other record. Also waiting lock requests are inherited as GRANTED gap locks. */
+static
+void
+lock_rec_inherit_to_gap_if_gap_lock(
+/*================================*/
+ const buf_block_t* block, /*!< in: buffer block */
+ ulint heir_heap_no, /*!< in: heap_no of
+ record which inherits */
+ ulint heap_no) /*!< in: heap_no of record
+ from which inherited;
+ does NOT reset the locks
+ on this record */
+{
+ lock_t* lock;
+
+ lock_mutex_enter();
+
+ for (lock = lock_rec_get_first(&lock_sys.rec_hash, block, heap_no);
+ lock != NULL;
+ lock = lock_rec_get_next(heap_no, lock)) {
+
+ if (!lock_rec_get_insert_intention(lock)
+ && (heap_no == PAGE_HEAP_NO_SUPREMUM
+ || !lock_rec_get_rec_not_gap(lock))) {
+
+ lock_rec_add_to_queue(
+ LOCK_REC | LOCK_GAP | lock_get_mode(lock),
+ block, heir_heap_no, lock->index,
+ lock->trx, FALSE);
+ }
+ }
+
+ lock_mutex_exit();
+}
+
+/*************************************************************//**
+Moves the locks of a record to another record and resets the lock bits of
+the donating record. */
+static
+void
+lock_rec_move_low(
+/*==============*/
+ hash_table_t* lock_hash, /*!< in: hash table to use */
+ const buf_block_t* receiver, /*!< in: buffer block containing
+ the receiving record */
+ const buf_block_t* donator, /*!< in: buffer block containing
+ the donating record */
+ ulint receiver_heap_no,/*!< in: heap_no of the record
+ which gets the locks; there
+ must be no lock requests
+ on it! */
+ ulint donator_heap_no)/*!< in: heap_no of the record
+ which gives the locks */
+{
+ lock_t* lock;
+
+ ut_ad(lock_mutex_own());
+
+ /* If the lock is predicate lock, it resides on INFIMUM record */
+ ut_ad(lock_rec_get_first(
+ lock_hash, receiver, receiver_heap_no) == NULL
+ || lock_hash == &lock_sys.prdt_hash
+ || lock_hash == &lock_sys.prdt_page_hash);
+
+ for (lock = lock_rec_get_first(lock_hash,
+ donator, donator_heap_no);
+ lock != NULL;
+ lock = lock_rec_get_next(donator_heap_no, lock)) {
+
+ const auto type_mode = lock->type_mode;
+
+ lock_rec_reset_nth_bit(lock, donator_heap_no);
+
+ if (type_mode & LOCK_WAIT) {
+ lock_reset_lock_and_trx_wait(lock);
+ }
+
+ /* Note that we FIRST reset the bit, and then set the lock:
+ the function works also if donator == receiver */
+
+ lock_rec_add_to_queue(
+ type_mode, receiver, receiver_heap_no,
+ lock->index, lock->trx, FALSE);
+ }
+
+ ut_ad(!lock_rec_get_first(&lock_sys.rec_hash,
+ donator, donator_heap_no));
+}
+
+/** Move all the granted locks to the front of the given lock list.
+All the waiting locks will be at the end of the list.
+@param[in,out] lock_list the given lock list. */
+static
+void
+lock_move_granted_locks_to_front(
+ UT_LIST_BASE_NODE_T(lock_t)& lock_list)
+{
+ lock_t* lock;
+
+ bool seen_waiting_lock = false;
+
+ for (lock = UT_LIST_GET_FIRST(lock_list); lock;
+ lock = UT_LIST_GET_NEXT(trx_locks, lock)) {
+
+ if (!seen_waiting_lock) {
+ if (lock->is_waiting()) {
+ seen_waiting_lock = true;
+ }
+ continue;
+ }
+
+ ut_ad(seen_waiting_lock);
+
+ if (!lock->is_waiting()) {
+ lock_t* prev = UT_LIST_GET_PREV(trx_locks, lock);
+ ut_a(prev);
+ ut_list_move_to_front(lock_list, lock);
+ lock = prev;
+ }
+ }
+}
+
+/*************************************************************//**
+Moves the locks of a record to another record and resets the lock bits of
+the donating record. */
+UNIV_INLINE
+void
+lock_rec_move(
+/*==========*/
+ const buf_block_t* receiver, /*!< in: buffer block containing
+ the receiving record */
+ const buf_block_t* donator, /*!< in: buffer block containing
+ the donating record */
+ ulint receiver_heap_no,/*!< in: heap_no of the record
+ which gets the locks; there
+ must be no lock requests
+ on it! */
+ ulint donator_heap_no)/*!< in: heap_no of the record
+ which gives the locks */
+{
+ lock_rec_move_low(&lock_sys.rec_hash, receiver, donator,
+ receiver_heap_no, donator_heap_no);
+}
+
+/*************************************************************//**
+Updates the lock table when we have reorganized a page. NOTE: we copy
+also the locks set on the infimum of the page; the infimum may carry
+locks if an update of a record is occurring on the page, and its locks
+were temporarily stored on the infimum. */
+void
+lock_move_reorganize_page(
+/*======================*/
+ const buf_block_t* block, /*!< in: old index page, now
+ reorganized */
+ const buf_block_t* oblock) /*!< in: copy of the old, not
+ reorganized page */
+{
+ lock_t* lock;
+ UT_LIST_BASE_NODE_T(lock_t) old_locks;
+ mem_heap_t* heap = NULL;
+ ulint comp;
+
+ lock_mutex_enter();
+
+ /* FIXME: This needs to deal with predicate lock too */
+ lock = lock_sys.get_first(block->page.id());
+
+ if (lock == NULL) {
+ lock_mutex_exit();
+
+ return;
+ }
+
+ heap = mem_heap_create(256);
+
+ /* Copy first all the locks on the page to heap and reset the
+ bitmaps in the original locks; chain the copies of the locks
+ using the trx_locks field in them. */
+
+ UT_LIST_INIT(old_locks, &lock_t::trx_locks);
+
+ do {
+ /* Make a copy of the lock */
+ lock_t* old_lock = lock_rec_copy(lock, heap);
+
+ UT_LIST_ADD_LAST(old_locks, old_lock);
+
+ /* Reset bitmap of lock */
+ lock_rec_bitmap_reset(lock);
+
+ if (lock_get_wait(lock)) {
+
+ lock_reset_lock_and_trx_wait(lock);
+ }
+
+ lock = lock_rec_get_next_on_page(lock);
+ } while (lock != NULL);
+
+ comp = page_is_comp(block->frame);
+ ut_ad(comp == page_is_comp(oblock->frame));
+
+ lock_move_granted_locks_to_front(old_locks);
+
+ DBUG_EXECUTE_IF("do_lock_reverse_page_reorganize",
+ ut_list_reverse(old_locks););
+
+ for (lock = UT_LIST_GET_FIRST(old_locks); lock;
+ lock = UT_LIST_GET_NEXT(trx_locks, lock)) {
+
+ /* NOTE: we copy also the locks set on the infimum and
+ supremum of the page; the infimum may carry locks if an
+ update of a record is occurring on the page, and its locks
+ were temporarily stored on the infimum */
+ const rec_t* rec1 = page_get_infimum_rec(
+ buf_block_get_frame(block));
+ const rec_t* rec2 = page_get_infimum_rec(
+ buf_block_get_frame(oblock));
+
+ /* Set locks according to old locks */
+ for (;;) {
+ ulint old_heap_no;
+ ulint new_heap_no;
+ ut_d(const rec_t* const orec = rec1);
+ ut_ad(page_rec_is_metadata(rec1)
+ == page_rec_is_metadata(rec2));
+
+ if (comp) {
+ old_heap_no = rec_get_heap_no_new(rec2);
+ new_heap_no = rec_get_heap_no_new(rec1);
+
+ rec1 = page_rec_get_next_low(rec1, TRUE);
+ rec2 = page_rec_get_next_low(rec2, TRUE);
+ } else {
+ old_heap_no = rec_get_heap_no_old(rec2);
+ new_heap_no = rec_get_heap_no_old(rec1);
+ ut_ad(!memcmp(rec1, rec2,
+ rec_get_data_size_old(rec2)));
+
+ rec1 = page_rec_get_next_low(rec1, FALSE);
+ rec2 = page_rec_get_next_low(rec2, FALSE);
+ }
+
+ /* Clear the bit in old_lock. */
+ if (old_heap_no < lock->un_member.rec_lock.n_bits
+ && lock_rec_reset_nth_bit(lock, old_heap_no)) {
+ ut_ad(!page_rec_is_metadata(orec));
+
+ /* NOTE that the old lock bitmap could be too
+ small for the new heap number! */
+
+ lock_rec_add_to_queue(
+ lock->type_mode, block, new_heap_no,
+ lock->index, lock->trx, FALSE);
+ }
+
+ if (new_heap_no == PAGE_HEAP_NO_SUPREMUM) {
+ ut_ad(old_heap_no == PAGE_HEAP_NO_SUPREMUM);
+ break;
+ }
+ }
+
+ ut_ad(lock_rec_find_set_bit(lock) == ULINT_UNDEFINED);
+ }
+
+ lock_mutex_exit();
+
+ mem_heap_free(heap);
+
+#ifdef UNIV_DEBUG_LOCK_VALIDATE
+ ut_ad(lock_rec_validate_page(block));
+#endif
+}
+
+/*************************************************************//**
+Moves the explicit locks on user records to another page if a record
+list end is moved to another page. */
+void
+lock_move_rec_list_end(
+/*===================*/
+ const buf_block_t* new_block, /*!< in: index page to move to */
+ const buf_block_t* block, /*!< in: index page */
+ const rec_t* rec) /*!< in: record on page: this
+ is the first record moved */
+{
+ lock_t* lock;
+ const ulint comp = page_rec_is_comp(rec);
+
+ ut_ad(buf_block_get_frame(block) == page_align(rec));
+ ut_ad(comp == page_is_comp(buf_block_get_frame(new_block)));
+
+ lock_mutex_enter();
+
+ /* Note: when we move locks from record to record, waiting locks
+ and possible granted gap type locks behind them are enqueued in
+ the original order, because new elements are inserted to a hash
+ table to the end of the hash chain, and lock_rec_add_to_queue
+ does not reuse locks if there are waiters in the queue. */
+
+ for (lock = lock_sys.get_first(block->page.id());
+ lock;
+ lock = lock_rec_get_next_on_page(lock)) {
+ const rec_t* rec1 = rec;
+ const rec_t* rec2;
+ const auto type_mode = lock->type_mode;
+
+ if (comp) {
+ if (page_offset(rec1) == PAGE_NEW_INFIMUM) {
+ rec1 = page_rec_get_next_low(rec1, TRUE);
+ }
+
+ rec2 = page_rec_get_next_low(
+ buf_block_get_frame(new_block)
+ + PAGE_NEW_INFIMUM, TRUE);
+ } else {
+ if (page_offset(rec1) == PAGE_OLD_INFIMUM) {
+ rec1 = page_rec_get_next_low(rec1, FALSE);
+ }
+
+ rec2 = page_rec_get_next_low(
+ buf_block_get_frame(new_block)
+ + PAGE_OLD_INFIMUM, FALSE);
+ }
+
+ /* Copy lock requests on user records to new page and
+ reset the lock bits on the old */
+
+ for (;;) {
+ ut_ad(page_rec_is_metadata(rec1)
+ == page_rec_is_metadata(rec2));
+ ut_d(const rec_t* const orec = rec1);
+
+ ulint rec1_heap_no;
+ ulint rec2_heap_no;
+
+ if (comp) {
+ rec1_heap_no = rec_get_heap_no_new(rec1);
+
+ if (rec1_heap_no == PAGE_HEAP_NO_SUPREMUM) {
+ break;
+ }
+
+ rec2_heap_no = rec_get_heap_no_new(rec2);
+ rec1 = page_rec_get_next_low(rec1, TRUE);
+ rec2 = page_rec_get_next_low(rec2, TRUE);
+ } else {
+ rec1_heap_no = rec_get_heap_no_old(rec1);
+
+ if (rec1_heap_no == PAGE_HEAP_NO_SUPREMUM) {
+ break;
+ }
+
+ rec2_heap_no = rec_get_heap_no_old(rec2);
+
+ ut_ad(rec_get_data_size_old(rec1)
+ == rec_get_data_size_old(rec2));
+
+ ut_ad(!memcmp(rec1, rec2,
+ rec_get_data_size_old(rec1)));
+
+ rec1 = page_rec_get_next_low(rec1, FALSE);
+ rec2 = page_rec_get_next_low(rec2, FALSE);
+ }
+
+ if (rec1_heap_no < lock->un_member.rec_lock.n_bits
+ && lock_rec_reset_nth_bit(lock, rec1_heap_no)) {
+ ut_ad(!page_rec_is_metadata(orec));
+
+ if (type_mode & LOCK_WAIT) {
+ lock_reset_lock_and_trx_wait(lock);
+ }
+
+ lock_rec_add_to_queue(
+ type_mode, new_block, rec2_heap_no,
+ lock->index, lock->trx, FALSE);
+ }
+ }
+ }
+
+ lock_mutex_exit();
+
+#ifdef UNIV_DEBUG_LOCK_VALIDATE
+ ut_ad(lock_rec_validate_page(block));
+ ut_ad(lock_rec_validate_page(new_block));
+#endif
+}
+
+/*************************************************************//**
+Moves the explicit locks on user records to another page if a record
+list start is moved to another page. */
+void
+lock_move_rec_list_start(
+/*=====================*/
+ const buf_block_t* new_block, /*!< in: index page to
+ move to */
+ const buf_block_t* block, /*!< in: index page */
+ const rec_t* rec, /*!< in: record on page:
+ this is the first
+ record NOT copied */
+ const rec_t* old_end) /*!< in: old
+ previous-to-last
+ record on new_page
+ before the records
+ were copied */
+{
+ lock_t* lock;
+ const ulint comp = page_rec_is_comp(rec);
+
+ ut_ad(block->frame == page_align(rec));
+ ut_ad(new_block->frame == page_align(old_end));
+ ut_ad(comp == page_rec_is_comp(old_end));
+ ut_ad(!page_rec_is_metadata(rec));
+
+ lock_mutex_enter();
+
+ for (lock = lock_sys.get_first(block->page.id());
+ lock;
+ lock = lock_rec_get_next_on_page(lock)) {
+ const rec_t* rec1;
+ const rec_t* rec2;
+ const auto type_mode = lock->type_mode;
+
+ if (comp) {
+ rec1 = page_rec_get_next_low(
+ buf_block_get_frame(block)
+ + PAGE_NEW_INFIMUM, TRUE);
+ rec2 = page_rec_get_next_low(old_end, TRUE);
+ } else {
+ rec1 = page_rec_get_next_low(
+ buf_block_get_frame(block)
+ + PAGE_OLD_INFIMUM, FALSE);
+ rec2 = page_rec_get_next_low(old_end, FALSE);
+ }
+
+ /* Copy lock requests on user records to new page and
+ reset the lock bits on the old */
+
+ while (rec1 != rec) {
+ ut_ad(page_rec_is_metadata(rec1)
+ == page_rec_is_metadata(rec2));
+ ut_d(const rec_t* const prev = rec1);
+
+ ulint rec1_heap_no;
+ ulint rec2_heap_no;
+
+ if (comp) {
+ rec1_heap_no = rec_get_heap_no_new(rec1);
+ rec2_heap_no = rec_get_heap_no_new(rec2);
+
+ rec1 = page_rec_get_next_low(rec1, TRUE);
+ rec2 = page_rec_get_next_low(rec2, TRUE);
+ } else {
+ rec1_heap_no = rec_get_heap_no_old(rec1);
+ rec2_heap_no = rec_get_heap_no_old(rec2);
+
+ ut_ad(!memcmp(rec1, rec2,
+ rec_get_data_size_old(rec2)));
+
+ rec1 = page_rec_get_next_low(rec1, FALSE);
+ rec2 = page_rec_get_next_low(rec2, FALSE);
+ }
+
+ if (rec1_heap_no < lock->un_member.rec_lock.n_bits
+ && lock_rec_reset_nth_bit(lock, rec1_heap_no)) {
+ ut_ad(!page_rec_is_metadata(prev));
+
+ if (type_mode & LOCK_WAIT) {
+ lock_reset_lock_and_trx_wait(lock);
+ }
+
+ lock_rec_add_to_queue(
+ type_mode, new_block, rec2_heap_no,
+ lock->index, lock->trx, FALSE);
+ }
+ }
+
+#ifdef UNIV_DEBUG
+ if (page_rec_is_supremum(rec)) {
+ ulint i;
+
+ for (i = PAGE_HEAP_NO_USER_LOW;
+ i < lock_rec_get_n_bits(lock); i++) {
+ if (lock_rec_get_nth_bit(lock, i)) {
+ ib::fatal()
+ << "lock_move_rec_list_start():"
+ << i << " not moved in "
+ << (void*) lock;
+ }
+ }
+ }
+#endif /* UNIV_DEBUG */
+ }
+
+ lock_mutex_exit();
+
+#ifdef UNIV_DEBUG_LOCK_VALIDATE
+ ut_ad(lock_rec_validate_page(block));
+#endif
+}
+
+/*************************************************************//**
+Moves the explicit locks on user records to another page if a record
+list start is moved to another page. */
+void
+lock_rtr_move_rec_list(
+/*===================*/
+ const buf_block_t* new_block, /*!< in: index page to
+ move to */
+ const buf_block_t* block, /*!< in: index page */
+ rtr_rec_move_t* rec_move, /*!< in: recording records
+ moved */
+ ulint num_move) /*!< in: num of rec to move */
+{
+ lock_t* lock;
+ ulint comp;
+
+ if (!num_move) {
+ return;
+ }
+
+ comp = page_rec_is_comp(rec_move[0].old_rec);
+
+ ut_ad(block->frame == page_align(rec_move[0].old_rec));
+ ut_ad(new_block->frame == page_align(rec_move[0].new_rec));
+ ut_ad(comp == page_rec_is_comp(rec_move[0].new_rec));
+
+ lock_mutex_enter();
+
+ for (lock = lock_sys.get_first(block->page.id());
+ lock;
+ lock = lock_rec_get_next_on_page(lock)) {
+ ulint moved = 0;
+ const rec_t* rec1;
+ const rec_t* rec2;
+ const auto type_mode = lock->type_mode;
+
+ /* Copy lock requests on user records to new page and
+ reset the lock bits on the old */
+
+ while (moved < num_move) {
+ ulint rec1_heap_no;
+ ulint rec2_heap_no;
+
+ rec1 = rec_move[moved].old_rec;
+ rec2 = rec_move[moved].new_rec;
+ ut_ad(!page_rec_is_metadata(rec1));
+ ut_ad(!page_rec_is_metadata(rec2));
+
+ if (comp) {
+ rec1_heap_no = rec_get_heap_no_new(rec1);
+ rec2_heap_no = rec_get_heap_no_new(rec2);
+
+ } else {
+ rec1_heap_no = rec_get_heap_no_old(rec1);
+ rec2_heap_no = rec_get_heap_no_old(rec2);
+
+ ut_ad(!memcmp(rec1, rec2,
+ rec_get_data_size_old(rec2)));
+ }
+
+ if (rec1_heap_no < lock->un_member.rec_lock.n_bits
+ && lock_rec_reset_nth_bit(lock, rec1_heap_no)) {
+ if (type_mode & LOCK_WAIT) {
+ lock_reset_lock_and_trx_wait(lock);
+ }
+
+ lock_rec_add_to_queue(
+ type_mode, new_block, rec2_heap_no,
+ lock->index, lock->trx, FALSE);
+
+ rec_move[moved].moved = true;
+ }
+
+ moved++;
+ }
+ }
+
+ lock_mutex_exit();
+
+#ifdef UNIV_DEBUG_LOCK_VALIDATE
+ ut_ad(lock_rec_validate_page(block));
+#endif
+}
+/*************************************************************//**
+Updates the lock table when a page is split to the right. */
+void
+lock_update_split_right(
+/*====================*/
+ const buf_block_t* right_block, /*!< in: right page */
+ const buf_block_t* left_block) /*!< in: left page */
+{
+ ulint heap_no = lock_get_min_heap_no(right_block);
+
+ lock_mutex_enter();
+
+ /* Move the locks on the supremum of the left page to the supremum
+ of the right page */
+
+ lock_rec_move(right_block, left_block,
+ PAGE_HEAP_NO_SUPREMUM, PAGE_HEAP_NO_SUPREMUM);
+
+ /* Inherit the locks to the supremum of left page from the successor
+ of the infimum on right page */
+
+ lock_rec_inherit_to_gap(left_block, right_block,
+ PAGE_HEAP_NO_SUPREMUM, heap_no);
+
+ lock_mutex_exit();
+}
+
+/*************************************************************//**
+Updates the lock table when a page is merged to the right. */
+void
+lock_update_merge_right(
+/*====================*/
+ const buf_block_t* right_block, /*!< in: right page to
+ which merged */
+ const rec_t* orig_succ, /*!< in: original
+ successor of infimum
+ on the right page
+ before merge */
+ const buf_block_t* left_block) /*!< in: merged index
+ page which will be
+ discarded */
+{
+ ut_ad(!page_rec_is_metadata(orig_succ));
+
+ lock_mutex_enter();
+
+ /* Inherit the locks from the supremum of the left page to the
+ original successor of infimum on the right page, to which the left
+ page was merged */
+
+ lock_rec_inherit_to_gap(right_block, left_block,
+ page_rec_get_heap_no(orig_succ),
+ PAGE_HEAP_NO_SUPREMUM);
+
+ /* Reset the locks on the supremum of the left page, releasing
+ waiting transactions */
+
+ lock_rec_reset_and_release_wait_low(
+ &lock_sys.rec_hash, left_block, PAGE_HEAP_NO_SUPREMUM);
+
+ /* there should exist no page lock on the left page,
+ otherwise, it will be blocked from merge */
+ ut_ad(!lock_sys.get_first_prdt_page(left_block->page.id()));
+
+ lock_rec_free_all_from_discard_page(left_block);
+
+ lock_mutex_exit();
+}
+
+/*************************************************************//**
+Updates the lock table when the root page is copied to another in
+btr_root_raise_and_insert. Note that we leave lock structs on the
+root page, even though they do not make sense on other than leaf
+pages: the reason is that in a pessimistic update the infimum record
+of the root page will act as a dummy carrier of the locks of the record
+to be updated. */
+void
+lock_update_root_raise(
+/*===================*/
+ const buf_block_t* block, /*!< in: index page to which copied */
+ const buf_block_t* root) /*!< in: root page */
+{
+ lock_mutex_enter();
+
+ /* Move the locks on the supremum of the root to the supremum
+ of block */
+
+ lock_rec_move(block, root,
+ PAGE_HEAP_NO_SUPREMUM, PAGE_HEAP_NO_SUPREMUM);
+ lock_mutex_exit();
+}
+
+/*************************************************************//**
+Updates the lock table when a page is copied to another and the original page
+is removed from the chain of leaf pages, except if page is the root! */
+void
+lock_update_copy_and_discard(
+/*=========================*/
+ const buf_block_t* new_block, /*!< in: index page to
+ which copied */
+ const buf_block_t* block) /*!< in: index page;
+ NOT the root! */
+{
+ lock_mutex_enter();
+
+ /* Move the locks on the supremum of the old page to the supremum
+ of new_page */
+
+ lock_rec_move(new_block, block,
+ PAGE_HEAP_NO_SUPREMUM, PAGE_HEAP_NO_SUPREMUM);
+ lock_rec_free_all_from_discard_page(block);
+
+ lock_mutex_exit();
+}
+
+/*************************************************************//**
+Updates the lock table when a page is split to the left. */
+void
+lock_update_split_left(
+/*===================*/
+ const buf_block_t* right_block, /*!< in: right page */
+ const buf_block_t* left_block) /*!< in: left page */
+{
+ ulint heap_no = lock_get_min_heap_no(right_block);
+
+ lock_mutex_enter();
+
+ /* Inherit the locks to the supremum of the left page from the
+ successor of the infimum on the right page */
+
+ lock_rec_inherit_to_gap(left_block, right_block,
+ PAGE_HEAP_NO_SUPREMUM, heap_no);
+
+ lock_mutex_exit();
+}
+
+/*************************************************************//**
+Updates the lock table when a page is merged to the left. */
+void
+lock_update_merge_left(
+/*===================*/
+ const buf_block_t* left_block, /*!< in: left page to
+ which merged */
+ const rec_t* orig_pred, /*!< in: original predecessor
+ of supremum on the left page
+ before merge */
+ const buf_block_t* right_block) /*!< in: merged index page
+ which will be discarded */
+{
+ const rec_t* left_next_rec;
+
+ ut_ad(left_block->frame == page_align(orig_pred));
+
+ lock_mutex_enter();
+
+ left_next_rec = page_rec_get_next_const(orig_pred);
+
+ if (!page_rec_is_supremum(left_next_rec)) {
+
+ /* Inherit the locks on the supremum of the left page to the
+ first record which was moved from the right page */
+
+ lock_rec_inherit_to_gap(left_block, left_block,
+ page_rec_get_heap_no(left_next_rec),
+ PAGE_HEAP_NO_SUPREMUM);
+
+ /* Reset the locks on the supremum of the left page,
+ releasing waiting transactions */
+
+ lock_rec_reset_and_release_wait_low(
+ &lock_sys.rec_hash, left_block, PAGE_HEAP_NO_SUPREMUM);
+ }
+
+ /* Move the locks from the supremum of right page to the supremum
+ of the left page */
+
+ lock_rec_move(left_block, right_block,
+ PAGE_HEAP_NO_SUPREMUM, PAGE_HEAP_NO_SUPREMUM);
+
+ /* there should exist no page lock on the right page,
+ otherwise, it will be blocked from merge */
+ ut_ad(!lock_sys.get_first_prdt_page(right_block->page.id()));
+
+ lock_rec_free_all_from_discard_page(right_block);
+
+ lock_mutex_exit();
+}
+
+/*************************************************************//**
+Resets the original locks on heir and replaces them with gap type locks
+inherited from rec. */
+void
+lock_rec_reset_and_inherit_gap_locks(
+/*=================================*/
+ const buf_block_t* heir_block, /*!< in: block containing the
+ record which inherits */
+ const buf_block_t* block, /*!< in: block containing the
+ record from which inherited;
+ does NOT reset the locks on
+ this record */
+ ulint heir_heap_no, /*!< in: heap_no of the
+ inheriting record */
+ ulint heap_no) /*!< in: heap_no of the
+ donating record */
+{
+ lock_mutex_enter();
+
+ lock_rec_reset_and_release_wait(heir_block, heir_heap_no);
+
+ lock_rec_inherit_to_gap(heir_block, block, heir_heap_no, heap_no);
+
+ lock_mutex_exit();
+}
+
+/*************************************************************//**
+Updates the lock table when a page is discarded. */
+void
+lock_update_discard(
+/*================*/
+ const buf_block_t* heir_block, /*!< in: index page
+ which will inherit the locks */
+ ulint heir_heap_no, /*!< in: heap_no of the record
+ which will inherit the locks */
+ const buf_block_t* block) /*!< in: index page
+ which will be discarded */
+{
+ const page_t* page = block->frame;
+ const rec_t* rec;
+ ulint heap_no;
+ const page_id_t page_id(block->page.id());
+
+ lock_mutex_enter();
+
+ if (lock_sys.get_first(page_id)) {
+ ut_ad(!lock_sys.get_first_prdt(page_id));
+ ut_ad(!lock_sys.get_first_prdt_page(page_id));
+ /* Inherit all the locks on the page to the record and
+ reset all the locks on the page */
+
+ if (page_is_comp(page)) {
+ rec = page + PAGE_NEW_INFIMUM;
+
+ do {
+ heap_no = rec_get_heap_no_new(rec);
+
+ lock_rec_inherit_to_gap(heir_block, block,
+ heir_heap_no, heap_no);
+
+ lock_rec_reset_and_release_wait(
+ block, heap_no);
+
+ rec = page + rec_get_next_offs(rec, TRUE);
+ } while (heap_no != PAGE_HEAP_NO_SUPREMUM);
+ } else {
+ rec = page + PAGE_OLD_INFIMUM;
+
+ do {
+ heap_no = rec_get_heap_no_old(rec);
+
+ lock_rec_inherit_to_gap(heir_block, block,
+ heir_heap_no, heap_no);
+
+ lock_rec_reset_and_release_wait(
+ block, heap_no);
+
+ rec = page + rec_get_next_offs(rec, FALSE);
+ } while (heap_no != PAGE_HEAP_NO_SUPREMUM);
+ }
+
+ lock_rec_free_all_from_discard_page_low(page_id,
+ &lock_sys.rec_hash);
+ } else {
+ lock_rec_free_all_from_discard_page_low(page_id,
+ &lock_sys.prdt_hash);
+ lock_rec_free_all_from_discard_page_low(
+ page_id, &lock_sys.prdt_page_hash);
+ }
+
+ lock_mutex_exit();
+}
+
+/*************************************************************//**
+Updates the lock table when a new user record is inserted. */
+void
+lock_update_insert(
+/*===============*/
+ const buf_block_t* block, /*!< in: buffer block containing rec */
+ const rec_t* rec) /*!< in: the inserted record */
+{
+ ulint receiver_heap_no;
+ ulint donator_heap_no;
+
+ ut_ad(block->frame == page_align(rec));
+ ut_ad(!page_rec_is_metadata(rec));
+
+ /* Inherit the gap-locking locks for rec, in gap mode, from the next
+ record */
+
+ if (page_rec_is_comp(rec)) {
+ receiver_heap_no = rec_get_heap_no_new(rec);
+ donator_heap_no = rec_get_heap_no_new(
+ page_rec_get_next_low(rec, TRUE));
+ } else {
+ receiver_heap_no = rec_get_heap_no_old(rec);
+ donator_heap_no = rec_get_heap_no_old(
+ page_rec_get_next_low(rec, FALSE));
+ }
+
+ lock_rec_inherit_to_gap_if_gap_lock(
+ block, receiver_heap_no, donator_heap_no);
+}
+
+/*************************************************************//**
+Updates the lock table when a record is removed. */
+void
+lock_update_delete(
+/*===============*/
+ const buf_block_t* block, /*!< in: buffer block containing rec */
+ const rec_t* rec) /*!< in: the record to be removed */
+{
+ const page_t* page = block->frame;
+ ulint heap_no;
+ ulint next_heap_no;
+
+ ut_ad(page == page_align(rec));
+ ut_ad(!page_rec_is_metadata(rec));
+
+ if (page_is_comp(page)) {
+ heap_no = rec_get_heap_no_new(rec);
+ next_heap_no = rec_get_heap_no_new(page
+ + rec_get_next_offs(rec,
+ TRUE));
+ } else {
+ heap_no = rec_get_heap_no_old(rec);
+ next_heap_no = rec_get_heap_no_old(page
+ + rec_get_next_offs(rec,
+ FALSE));
+ }
+
+ lock_mutex_enter();
+
+ /* Let the next record inherit the locks from rec, in gap mode */
+
+ lock_rec_inherit_to_gap(block, block, next_heap_no, heap_no);
+
+ /* Reset the lock bits on rec and release waiting transactions */
+
+ lock_rec_reset_and_release_wait(block, heap_no);
+
+ lock_mutex_exit();
+}
+
+/*********************************************************************//**
+Stores on the page infimum record the explicit locks of another record.
+This function is used to store the lock state of a record when it is
+updated and the size of the record changes in the update. The record
+is moved in such an update, perhaps to another page. The infimum record
+acts as a dummy carrier record, taking care of lock releases while the
+actual record is being moved. */
+void
+lock_rec_store_on_page_infimum(
+/*===========================*/
+ const buf_block_t* block, /*!< in: buffer block containing rec */
+ const rec_t* rec) /*!< in: record whose lock state
+ is stored on the infimum
+ record of the same page; lock
+ bits are reset on the
+ record */
+{
+ ulint heap_no = page_rec_get_heap_no(rec);
+
+ ut_ad(block->frame == page_align(rec));
+
+ lock_mutex_enter();
+
+ lock_rec_move(block, block, PAGE_HEAP_NO_INFIMUM, heap_no);
+
+ lock_mutex_exit();
+}
+
+/*********************************************************************//**
+Restores the state of explicit lock requests on a single record, where the
+state was stored on the infimum of the page. */
+void
+lock_rec_restore_from_page_infimum(
+/*===============================*/
+ const buf_block_t* block, /*!< in: buffer block containing rec */
+ const rec_t* rec, /*!< in: record whose lock state
+ is restored */
+ const buf_block_t* donator)/*!< in: page (rec is not
+ necessarily on this page)
+ whose infimum stored the lock
+ state; lock bits are reset on
+ the infimum */
+{
+ ulint heap_no = page_rec_get_heap_no(rec);
+
+ lock_mutex_enter();
+
+ lock_rec_move(block, donator, heap_no, PAGE_HEAP_NO_INFIMUM);
+
+ lock_mutex_exit();
+}
+
+/*========================= TABLE LOCKS ==============================*/
+
+/** Functor for accessing the embedded node within a table lock. */
+struct TableLockGetNode {
+ ut_list_node<lock_t>& operator() (lock_t& elem)
+ {
+ return(elem.un_member.tab_lock.locks);
+ }
+};
+
+/*********************************************************************//**
+Creates a table lock object and adds it as the last in the lock queue
+of the table. Does NOT check for deadlocks or lock compatibility.
+@return own: new lock object */
+UNIV_INLINE
+lock_t*
+lock_table_create(
+/*==============*/
+ dict_table_t* table, /*!< in/out: database table
+ in dictionary cache */
+ unsigned type_mode,/*!< in: lock mode possibly ORed with
+ LOCK_WAIT */
+ trx_t* trx /*!< in: trx */
+#ifdef WITH_WSREP
+ , lock_t* c_lock = NULL /*!< in: conflicting lock */
+#endif
+ )
+{
+ lock_t* lock;
+
+ ut_ad(table && trx);
+ ut_ad(lock_mutex_own());
+ ut_ad(trx_mutex_own(trx));
+ ut_ad(trx->is_recovered || trx->state == TRX_STATE_ACTIVE);
+ ut_ad(!trx->auto_commit || trx->will_lock);
+
+ if ((type_mode & LOCK_MODE_MASK) == LOCK_AUTO_INC) {
+ ++table->n_waiting_or_granted_auto_inc_locks;
+ }
+
+ /* For AUTOINC locking we reuse the lock instance only if
+ there is no wait involved else we allocate the waiting lock
+ from the transaction lock heap. */
+ if (type_mode == LOCK_AUTO_INC) {
+
+ lock = table->autoinc_lock;
+
+ table->autoinc_trx = trx;
+
+ ib_vector_push(trx->autoinc_locks, &lock);
+
+ } else if (trx->lock.table_cached
+ < UT_ARR_SIZE(trx->lock.table_pool)) {
+ lock = &trx->lock.table_pool[trx->lock.table_cached++];
+ } else {
+
+ lock = static_cast<lock_t*>(
+ mem_heap_alloc(trx->lock.lock_heap, sizeof(*lock)));
+
+ }
+
+ lock->type_mode = ib_uint32_t(type_mode | LOCK_TABLE);
+ lock->trx = trx;
+
+ lock->un_member.tab_lock.table = table;
+
+ ut_ad(table->get_ref_count() > 0 || !table->can_be_evicted);
+
+ UT_LIST_ADD_LAST(trx->lock.trx_locks, lock);
+
+#ifdef WITH_WSREP
+ if (c_lock && trx->is_wsrep()) {
+ if (wsrep_thd_is_BF(trx->mysql_thd, FALSE)) {
+ ut_list_insert(table->locks, c_lock, lock,
+ TableLockGetNode());
+ if (UNIV_UNLIKELY(wsrep_debug)) {
+ wsrep_report_bf_lock_wait(trx->mysql_thd, trx->id);
+ wsrep_report_bf_lock_wait(c_lock->trx->mysql_thd, c_lock->trx->id);
+ }
+ } else {
+ ut_list_append(table->locks, lock, TableLockGetNode());
+ }
+
+ trx_mutex_enter(c_lock->trx);
+
+ if (c_lock->trx->lock.que_state == TRX_QUE_LOCK_WAIT) {
+ c_lock->trx->lock.was_chosen_as_deadlock_victim = TRUE;
+
+ if (UNIV_UNLIKELY(wsrep_debug)) {
+ wsrep_report_bf_lock_wait(trx->mysql_thd, trx->id);
+ wsrep_report_bf_lock_wait(c_lock->trx->mysql_thd, c_lock->trx->id);
+ wsrep_print_wait_locks(c_lock);
+ }
+
+ /* The lock release will call lock_grant(),
+ which would acquire trx->mutex again. */
+ trx_mutex_exit(trx);
+ lock_cancel_waiting_and_release(
+ c_lock->trx->lock.wait_lock);
+ trx_mutex_enter(trx);
+ }
+
+ trx_mutex_exit(c_lock->trx);
+ } else
+#endif /* WITH_WSREP */
+ ut_list_append(table->locks, lock, TableLockGetNode());
+
+ if (type_mode & LOCK_WAIT) {
+
+ lock_set_lock_and_trx_wait(lock, trx);
+ }
+
+ lock->trx->lock.table_locks.push_back(lock);
+
+ MONITOR_INC(MONITOR_TABLELOCK_CREATED);
+ MONITOR_INC(MONITOR_NUM_TABLELOCK);
+
+ return(lock);
+}
+
+/*************************************************************//**
+Pops autoinc lock requests from the transaction's autoinc_locks. We
+handle the case where there are gaps in the array and they need to
+be popped off the stack. */
+UNIV_INLINE
+void
+lock_table_pop_autoinc_locks(
+/*=========================*/
+ trx_t* trx) /*!< in/out: transaction that owns the AUTOINC locks */
+{
+ ut_ad(lock_mutex_own());
+ ut_ad(!ib_vector_is_empty(trx->autoinc_locks));
+
+ /* Skip any gaps, gaps are NULL lock entries in the
+ trx->autoinc_locks vector. */
+
+ do {
+ ib_vector_pop(trx->autoinc_locks);
+
+ if (ib_vector_is_empty(trx->autoinc_locks)) {
+ return;
+ }
+
+ } while (*(lock_t**) ib_vector_get_last(trx->autoinc_locks) == NULL);
+}
+
+/*************************************************************//**
+Removes an autoinc lock request from the transaction's autoinc_locks. */
+UNIV_INLINE
+void
+lock_table_remove_autoinc_lock(
+/*===========================*/
+ lock_t* lock, /*!< in: table lock */
+ trx_t* trx) /*!< in/out: transaction that owns the lock */
+{
+ lock_t* autoinc_lock;
+ lint i = ib_vector_size(trx->autoinc_locks) - 1;
+
+ ut_ad(lock_mutex_own());
+ ut_ad(lock_get_mode(lock) == LOCK_AUTO_INC);
+ ut_ad(lock_get_type_low(lock) & LOCK_TABLE);
+ ut_ad(!ib_vector_is_empty(trx->autoinc_locks));
+
+ /* With stored functions and procedures the user may drop
+ a table within the same "statement". This special case has
+ to be handled by deleting only those AUTOINC locks that were
+ held by the table being dropped. */
+
+ autoinc_lock = *static_cast<lock_t**>(
+ ib_vector_get(trx->autoinc_locks, i));
+
+ /* This is the default fast case. */
+
+ if (autoinc_lock == lock) {
+ lock_table_pop_autoinc_locks(trx);
+ } else {
+ /* The last element should never be NULL */
+ ut_a(autoinc_lock != NULL);
+
+ /* Handle freeing the locks from within the stack. */
+
+ while (--i >= 0) {
+ autoinc_lock = *static_cast<lock_t**>(
+ ib_vector_get(trx->autoinc_locks, i));
+
+ if (autoinc_lock == lock) {
+ void* null_var = NULL;
+ ib_vector_set(trx->autoinc_locks, i, &null_var);
+ return;
+ }
+ }
+
+ /* Must find the autoinc lock. */
+ ut_error;
+ }
+}
+
+/*************************************************************//**
+Removes a table lock request from the queue and the trx list of locks;
+this is a low-level function which does NOT check if waiting requests
+can now be granted. */
+UNIV_INLINE
+void
+lock_table_remove_low(
+/*==================*/
+ lock_t* lock) /*!< in/out: table lock */
+{
+ trx_t* trx;
+ dict_table_t* table;
+
+ ut_ad(lock_mutex_own());
+
+ trx = lock->trx;
+ table = lock->un_member.tab_lock.table;
+
+ /* Remove the table from the transaction's AUTOINC vector, if
+ the lock that is being released is an AUTOINC lock. */
+ if (lock_get_mode(lock) == LOCK_AUTO_INC) {
+
+ /* The table's AUTOINC lock can get transferred to
+ another transaction before we get here. */
+ if (table->autoinc_trx == trx) {
+ table->autoinc_trx = NULL;
+ }
+
+ /* The locks must be freed in the reverse order from
+ the one in which they were acquired. This is to avoid
+ traversing the AUTOINC lock vector unnecessarily.
+
+ We only store locks that were granted in the
+ trx->autoinc_locks vector (see lock_table_create()
+ and lock_grant()). Therefore it can be empty and we
+ need to check for that. */
+
+ if (!lock_get_wait(lock)
+ && !ib_vector_is_empty(trx->autoinc_locks)) {
+
+ lock_table_remove_autoinc_lock(lock, trx);
+ }
+
+ ut_a(table->n_waiting_or_granted_auto_inc_locks > 0);
+ table->n_waiting_or_granted_auto_inc_locks--;
+ }
+
+ UT_LIST_REMOVE(trx->lock.trx_locks, lock);
+ ut_list_remove(table->locks, lock, TableLockGetNode());
+
+ MONITOR_INC(MONITOR_TABLELOCK_REMOVED);
+ MONITOR_DEC(MONITOR_NUM_TABLELOCK);
+}
+
+/*********************************************************************//**
+Enqueues a waiting request for a table lock which cannot be granted
+immediately. Checks for deadlocks.
+@retval DB_LOCK_WAIT if the waiting lock was enqueued
+@retval DB_DEADLOCK if this transaction was chosen as the victim
+@retval DB_SUCCESS if the other transaction committed or aborted */
+static
+dberr_t
+lock_table_enqueue_waiting(
+/*=======================*/
+ unsigned mode, /*!< in: lock mode this transaction is
+ requesting */
+ dict_table_t* table, /*!< in/out: table */
+ que_thr_t* thr /*!< in: query thread */
+#ifdef WITH_WSREP
+ , lock_t* c_lock /*!< in: conflicting lock or NULL */
+#endif
+)
+{
+ trx_t* trx;
+ lock_t* lock;
+
+ ut_ad(lock_mutex_own());
+ ut_ad(!srv_read_only_mode);
+
+ trx = thr_get_trx(thr);
+ ut_ad(trx_mutex_own(trx));
+ ut_a(!que_thr_stop(thr));
+
+ switch (trx_get_dict_operation(trx)) {
+ case TRX_DICT_OP_NONE:
+ break;
+ case TRX_DICT_OP_TABLE:
+ case TRX_DICT_OP_INDEX:
+ ib::error() << "A table lock wait happens in a dictionary"
+ " operation. Table " << table->name
+ << ". " << BUG_REPORT_MSG;
+ ut_ad(0);
+ }
+
+#ifdef WITH_WSREP
+ if (trx->is_wsrep() && trx->lock.was_chosen_as_deadlock_victim) {
+ return(DB_DEADLOCK);
+ }
+#endif /* WITH_WSREP */
+
+ /* Enqueue the lock request that will wait to be granted */
+ lock = lock_table_create(table, mode | LOCK_WAIT, trx
+#ifdef WITH_WSREP
+ , c_lock
+#endif
+ );
+
+ const trx_t* victim_trx =
+ DeadlockChecker::check_and_resolve(lock, trx);
+
+ if (victim_trx != 0) {
+ ut_ad(victim_trx == trx);
+
+ /* The order here is important, we don't want to
+ lose the state of the lock before calling remove. */
+ lock_table_remove_low(lock);
+ lock_reset_lock_and_trx_wait(lock);
+
+ return(DB_DEADLOCK);
+
+ } else if (trx->lock.wait_lock == NULL) {
+ /* Deadlock resolution chose another transaction as a victim,
+ and we accidentally got our lock granted! */
+
+ return(DB_SUCCESS);
+ }
+
+ trx->lock.que_state = TRX_QUE_LOCK_WAIT;
+
+ trx->lock.wait_started = time(NULL);
+ trx->lock.was_chosen_as_deadlock_victim = false;
+
+ ut_a(que_thr_stop(thr));
+
+ MONITOR_INC(MONITOR_TABLELOCK_WAIT);
+
+ return(DB_LOCK_WAIT);
+}
+
+/*********************************************************************//**
+Checks if other transactions have an incompatible mode lock request in
+the lock queue.
+@return lock or NULL */
+UNIV_INLINE
+lock_t*
+lock_table_other_has_incompatible(
+/*==============================*/
+ const trx_t* trx, /*!< in: transaction, or NULL if all
+ transactions should be included */
+ ulint wait, /*!< in: LOCK_WAIT if also
+ waiting locks are taken into
+ account, or 0 if not */
+ const dict_table_t* table, /*!< in: table */
+ lock_mode mode) /*!< in: lock mode */
+{
+ lock_t* lock;
+
+ ut_ad(lock_mutex_own());
+
+ for (lock = UT_LIST_GET_LAST(table->locks);
+ lock != NULL;
+ lock = UT_LIST_GET_PREV(un_member.tab_lock.locks, lock)) {
+
+ if (lock->trx != trx
+ && !lock_mode_compatible(lock_get_mode(lock), mode)
+ && (wait || !lock_get_wait(lock))) {
+
+#ifdef WITH_WSREP
+ if (lock->trx->is_wsrep()) {
+ if (UNIV_UNLIKELY(wsrep_debug)) {
+ ib::info() << "WSREP: table lock abort for table:"
+ << table->name;
+ ib::info() << " SQL: "
+ << wsrep_thd_query(lock->trx->mysql_thd);
+ }
+ trx_mutex_enter(lock->trx);
+ wsrep_kill_victim((trx_t *)trx, (lock_t *)lock);
+ trx_mutex_exit(lock->trx);
+ }
+#endif /* WITH_WSREP */
+
+ return(lock);
+ }
+ }
+
+ return(NULL);
+}
+
+/*********************************************************************//**
+Locks the specified database table in the mode given. If the lock cannot
+be granted immediately, the query thread is put to wait.
+@return DB_SUCCESS, DB_LOCK_WAIT, or DB_DEADLOCK */
+dberr_t
+lock_table(
+/*=======*/
+ unsigned flags, /*!< in: if BTR_NO_LOCKING_FLAG bit is set,
+ does nothing */
+ dict_table_t* table, /*!< in/out: database table
+ in dictionary cache */
+ lock_mode mode, /*!< in: lock mode */
+ que_thr_t* thr) /*!< in: query thread */
+{
+ trx_t* trx;
+ dberr_t err;
+ lock_t* wait_for;
+
+ ut_ad(table && thr);
+
+ /* Given limited visibility of temp-table we can avoid
+ locking overhead */
+ if ((flags & BTR_NO_LOCKING_FLAG)
+ || srv_read_only_mode
+ || table->is_temporary()) {
+
+ return(DB_SUCCESS);
+ }
+
+ ut_a(flags == 0);
+
+ trx = thr_get_trx(thr);
+
+ /* Look for equal or stronger locks the same trx already
+ has on the table. No need to acquire the lock mutex here
+ because only this transacton can add/access table locks
+ to/from trx_t::table_locks. */
+
+ if (lock_table_has(trx, table, mode)) {
+
+ return(DB_SUCCESS);
+ }
+
+ /* Read only transactions can write to temp tables, we don't want
+ to promote them to RW transactions. Their updates cannot be visible
+ to other transactions. Therefore we can keep them out
+ of the read views. */
+
+ if ((mode == LOCK_IX || mode == LOCK_X)
+ && !trx->read_only
+ && trx->rsegs.m_redo.rseg == 0) {
+
+ trx_set_rw_mode(trx);
+ }
+
+ lock_mutex_enter();
+
+ DBUG_EXECUTE_IF("fatal-semaphore-timeout",
+ { os_thread_sleep(3600000000LL); });
+
+ /* We have to check if the new lock is compatible with any locks
+ other transactions have in the table lock queue. */
+
+ wait_for = lock_table_other_has_incompatible(
+ trx, LOCK_WAIT, table, mode);
+
+ trx_mutex_enter(trx);
+
+ /* Another trx has a request on the table in an incompatible
+ mode: this trx may have to wait */
+
+ if (wait_for != NULL) {
+ err = lock_table_enqueue_waiting(flags | mode, table,
+ thr
+#ifdef WITH_WSREP
+ , wait_for
+#endif
+ );
+ } else {
+ lock_table_create(table, flags | mode, trx);
+
+ ut_a(!flags || mode == LOCK_S || mode == LOCK_X);
+
+ err = DB_SUCCESS;
+ }
+
+ lock_mutex_exit();
+
+ trx_mutex_exit(trx);
+
+ return(err);
+}
+
+/*********************************************************************//**
+Creates a table IX lock object for a resurrected transaction. */
+void
+lock_table_ix_resurrect(
+/*====================*/
+ dict_table_t* table, /*!< in/out: table */
+ trx_t* trx) /*!< in/out: transaction */
+{
+ ut_ad(trx->is_recovered);
+
+ if (lock_table_has(trx, table, LOCK_IX)) {
+ return;
+ }
+
+ lock_mutex_enter();
+
+ /* We have to check if the new lock is compatible with any locks
+ other transactions have in the table lock queue. */
+
+ ut_ad(!lock_table_other_has_incompatible(
+ trx, LOCK_WAIT, table, LOCK_IX));
+
+ trx_mutex_enter(trx);
+ lock_table_create(table, LOCK_IX, trx);
+ lock_mutex_exit();
+ trx_mutex_exit(trx);
+}
+
+/*********************************************************************//**
+Checks if a waiting table lock request still has to wait in a queue.
+@return TRUE if still has to wait */
+static
+bool
+lock_table_has_to_wait_in_queue(
+/*============================*/
+ const lock_t* wait_lock) /*!< in: waiting table lock */
+{
+ const dict_table_t* table;
+ const lock_t* lock;
+
+ ut_ad(lock_mutex_own());
+ ut_ad(lock_get_wait(wait_lock));
+
+ table = wait_lock->un_member.tab_lock.table;
+
+ for (lock = UT_LIST_GET_FIRST(table->locks);
+ lock != wait_lock;
+ lock = UT_LIST_GET_NEXT(un_member.tab_lock.locks, lock)) {
+
+ if (lock_has_to_wait(wait_lock, lock)) {
+
+ return(true);
+ }
+ }
+
+ return(false);
+}
+
+/*************************************************************//**
+Removes a table lock request, waiting or granted, from the queue and grants
+locks to other transactions in the queue, if they now are entitled to a
+lock. */
+static
+void
+lock_table_dequeue(
+/*===============*/
+ lock_t* in_lock)/*!< in/out: table lock object; transactions waiting
+ behind will get their lock requests granted, if
+ they are now qualified to it */
+{
+ ut_ad(lock_mutex_own());
+ ut_a(lock_get_type_low(in_lock) == LOCK_TABLE);
+
+ lock_t* lock = UT_LIST_GET_NEXT(un_member.tab_lock.locks, in_lock);
+
+ lock_table_remove_low(in_lock);
+
+ /* Check if waiting locks in the queue can now be granted: grant
+ locks if there are no conflicting locks ahead. */
+
+ for (/* No op */;
+ lock != NULL;
+ lock = UT_LIST_GET_NEXT(un_member.tab_lock.locks, lock)) {
+
+ if (lock_get_wait(lock)
+ && !lock_table_has_to_wait_in_queue(lock)) {
+
+ /* Grant the lock */
+ ut_ad(in_lock->trx != lock->trx);
+ lock_grant(lock);
+ }
+ }
+}
+
+/** Sets a lock on a table based on the given mode.
+@param[in] table table to lock
+@param[in,out] trx transaction
+@param[in] mode LOCK_X or LOCK_S
+@return error code or DB_SUCCESS. */
+dberr_t
+lock_table_for_trx(
+ dict_table_t* table,
+ trx_t* trx,
+ enum lock_mode mode)
+{
+ mem_heap_t* heap;
+ que_thr_t* thr;
+ dberr_t err;
+ sel_node_t* node;
+ heap = mem_heap_create(512);
+
+ node = sel_node_create(heap);
+ thr = pars_complete_graph_for_exec(node, trx, heap, NULL);
+ thr->graph->state = QUE_FORK_ACTIVE;
+
+ /* We use the select query graph as the dummy graph needed
+ in the lock module call */
+
+ thr = static_cast<que_thr_t*>(
+ que_fork_get_first_thr(
+ static_cast<que_fork_t*>(que_node_get_parent(thr))));
+
+ thr->start_running();
+
+run_again:
+ thr->run_node = thr;
+ thr->prev_node = thr->common.parent;
+
+ err = lock_table(0, table, mode, thr);
+
+ trx->error_state = err;
+
+ if (UNIV_LIKELY(err == DB_SUCCESS)) {
+ thr->stop_no_error();
+ } else {
+ que_thr_stop_for_mysql(thr);
+
+ if (row_mysql_handle_errors(&err, trx, thr, NULL)) {
+ goto run_again;
+ }
+ }
+
+ que_graph_free(thr->graph);
+ trx->op_info = "";
+
+ return(err);
+}
+
+/*=========================== LOCK RELEASE ==============================*/
+static
+void
+lock_grant_and_move_on_rec(
+ lock_t* first_lock,
+ ulint heap_no)
+{
+ lock_t* lock;
+ const page_id_t page_id(first_lock->un_member.rec_lock.page_id);
+ const ulint rec_fold= page_id.fold();
+ lock_t* previous = static_cast<lock_t*>(
+ lock_sys.rec_hash.array[lock_sys.hash(page_id)]
+ .node);
+ if (previous == NULL) {
+ return;
+ }
+ if (previous == first_lock) {
+ lock = previous;
+ } else {
+ while (previous->hash &&
+ previous->hash != first_lock) {
+ previous = previous->hash;
+ }
+ lock = previous->hash;
+ }
+ /* Grant locks if there are no conflicting locks ahead.
+ Move granted locks to the head of the list. */
+ while (lock) {
+ ut_ad(!lock->trx->is_wsrep());
+ /* If the lock is a wait lock on this page, and it does not need to wait. */
+ if (lock->un_member.rec_lock.page_id == page_id
+ && lock_rec_get_nth_bit(lock, heap_no)
+ && lock_get_wait(lock)
+ && !lock_rec_has_to_wait_in_queue(lock)) {
+
+ lock_grant(lock);
+
+ if (previous != NULL) {
+ /* Move the lock to the head of the list. */
+ HASH_GET_NEXT(hash, previous) = HASH_GET_NEXT(hash, lock);
+ lock_rec_insert_to_head(lock, rec_fold);
+ } else {
+ /* Already at the head of the list. */
+ previous = lock;
+ }
+ /* Move on to the next lock. */
+ lock = static_cast<lock_t *>(HASH_GET_NEXT(hash, previous));
+ } else {
+ previous = lock;
+ lock = static_cast<lock_t *>(HASH_GET_NEXT(hash, lock));
+ }
+ }
+}
+
+/*************************************************************//**
+Removes a granted record lock of a transaction from the queue and grants
+locks to other transactions waiting in the queue if they now are entitled
+to a lock. */
+void
+lock_rec_unlock(
+/*============*/
+ trx_t* trx, /*!< in/out: transaction that has
+ set a record lock */
+ const buf_block_t* block, /*!< in: buffer block containing rec */
+ const rec_t* rec, /*!< in: record */
+ lock_mode lock_mode)/*!< in: LOCK_S or LOCK_X */
+{
+ lock_t* first_lock;
+ lock_t* lock;
+ ulint heap_no;
+
+ ut_ad(trx);
+ ut_ad(rec);
+ ut_ad(block->frame == page_align(rec));
+ ut_ad(!trx->lock.wait_lock);
+ ut_ad(trx_state_eq(trx, TRX_STATE_ACTIVE));
+ ut_ad(!page_rec_is_metadata(rec));
+
+ heap_no = page_rec_get_heap_no(rec);
+
+ lock_mutex_enter();
+ trx_mutex_enter(trx);
+
+ first_lock = lock_rec_get_first(&lock_sys.rec_hash, block, heap_no);
+
+ /* Find the last lock with the same lock_mode and transaction
+ on the record. */
+
+ for (lock = first_lock; lock != NULL;
+ lock = lock_rec_get_next(heap_no, lock)) {
+ if (lock->trx == trx && lock_get_mode(lock) == lock_mode) {
+ goto released;
+ }
+ }
+
+ lock_mutex_exit();
+ trx_mutex_exit(trx);
+
+ {
+ ib::error err;
+ err << "Unlock row could not find a " << lock_mode
+ << " mode lock on the record. Current statement: ";
+ size_t stmt_len;
+ if (const char* stmt = innobase_get_stmt_unsafe(
+ trx->mysql_thd, &stmt_len)) {
+ err.write(stmt, stmt_len);
+ }
+ }
+
+ return;
+
+released:
+ ut_a(!lock_get_wait(lock));
+ lock_rec_reset_nth_bit(lock, heap_no);
+
+ if (innodb_lock_schedule_algorithm
+ == INNODB_LOCK_SCHEDULE_ALGORITHM_FCFS ||
+ thd_is_replication_slave_thread(lock->trx->mysql_thd)) {
+
+ /* Check if we can now grant waiting lock requests */
+
+ for (lock = first_lock; lock != NULL;
+ lock = lock_rec_get_next(heap_no, lock)) {
+ if (!lock_get_wait(lock)) {
+ continue;
+ }
+ const lock_t* c = lock_rec_has_to_wait_in_queue(lock);
+ if (!c) {
+ /* Grant the lock */
+ ut_ad(trx != lock->trx);
+ lock_grant(lock);
+ }
+ }
+ } else {
+ lock_grant_and_move_on_rec(first_lock, heap_no);
+ }
+
+ lock_mutex_exit();
+ trx_mutex_exit(trx);
+}
+
+#ifdef UNIV_DEBUG
+/*********************************************************************//**
+Check if a transaction that has X or IX locks has set the dict_op
+code correctly. */
+static
+void
+lock_check_dict_lock(
+/*==================*/
+ const lock_t* lock) /*!< in: lock to check */
+{
+ if (lock_get_type_low(lock) == LOCK_REC) {
+ ut_ad(!lock->index->table->is_temporary());
+
+ /* Check if the transcation locked a record
+ in a system table in X mode. It should have set
+ the dict_op code correctly if it did. */
+ if (lock->index->table->id < DICT_HDR_FIRST_ID
+ && lock_get_mode(lock) == LOCK_X) {
+
+ ut_ad(lock_get_mode(lock) != LOCK_IX);
+ ut_ad(lock->trx->dict_operation != TRX_DICT_OP_NONE);
+ }
+ } else {
+ ut_ad(lock_get_type_low(lock) & LOCK_TABLE);
+
+ const dict_table_t* table = lock->un_member.tab_lock.table;
+ ut_ad(!table->is_temporary());
+
+ /* Check if the transcation locked a system table
+ in IX mode. It should have set the dict_op code
+ correctly if it did. */
+ if (table->id < DICT_HDR_FIRST_ID
+ && (lock_get_mode(lock) == LOCK_X
+ || lock_get_mode(lock) == LOCK_IX)) {
+
+ ut_ad(lock->trx->dict_operation != TRX_DICT_OP_NONE);
+ }
+ }
+}
+#endif /* UNIV_DEBUG */
+
+/** Release the explicit locks of a committing transaction,
+and release possible other transactions waiting because of these locks. */
+void lock_release(trx_t* trx)
+{
+#ifdef UNIV_DEBUG
+ std::set<table_id_t> to_evict;
+ if (innodb_evict_tables_on_commit_debug && !trx->is_recovered)
+# if 1 /* if dict_stats_exec_sql() were not playing dirty tricks */
+ if (!mutex_own(&dict_sys.mutex))
+# else /* this would be more proper way to do it */
+ if (!trx->dict_operation_lock_mode && !trx->dict_operation)
+# endif
+ for (const auto& p : trx->mod_tables)
+ if (!p.first->is_temporary())
+ to_evict.emplace(p.first->id);
+#endif
+ ulint count = 0;
+ trx_id_t max_trx_id = trx_sys.get_max_trx_id();
+
+ lock_mutex_enter();
+ ut_ad(!trx_mutex_own(trx));
+
+ for (lock_t* lock = UT_LIST_GET_LAST(trx->lock.trx_locks);
+ lock != NULL;
+ lock = UT_LIST_GET_LAST(trx->lock.trx_locks)) {
+
+ ut_d(lock_check_dict_lock(lock));
+
+ if (lock_get_type_low(lock) == LOCK_REC) {
+
+ lock_rec_dequeue_from_page(lock);
+ } else {
+ dict_table_t* table;
+
+ table = lock->un_member.tab_lock.table;
+
+ if (lock_get_mode(lock) != LOCK_IS
+ && trx->undo_no != 0) {
+
+ /* The trx may have modified the table. We
+ block the use of the MySQL query cache for
+ all currently active transactions. */
+
+ table->query_cache_inv_trx_id = max_trx_id;
+ }
+
+ lock_table_dequeue(lock);
+ }
+
+ if (count == LOCK_RELEASE_INTERVAL) {
+ /* Release the mutex for a while, so that we
+ do not monopolize it */
+
+ lock_mutex_exit();
+
+ lock_mutex_enter();
+
+ count = 0;
+ }
+
+ ++count;
+ }
+
+ lock_mutex_exit();
+
+#ifdef UNIV_DEBUG
+ if (to_evict.empty()) {
+ return;
+ }
+ mutex_enter(&dict_sys.mutex);
+ lock_mutex_enter();
+ for (table_id_t id : to_evict) {
+ if (dict_table_t *table = dict_table_open_on_id(
+ id, TRUE, DICT_TABLE_OP_OPEN_ONLY_IF_CACHED)) {
+ if (!table->get_ref_count()
+ && !UT_LIST_GET_LEN(table->locks)) {
+ dict_sys.remove(table, true);
+ }
+ }
+ }
+ lock_mutex_exit();
+ mutex_exit(&dict_sys.mutex);
+#endif
+}
+
+/* True if a lock mode is S or X */
+#define IS_LOCK_S_OR_X(lock) \
+ (lock_get_mode(lock) == LOCK_S \
+ || lock_get_mode(lock) == LOCK_X)
+
+/*********************************************************************//**
+Removes table locks of the transaction on a table to be dropped. */
+static
+void
+lock_trx_table_locks_remove(
+/*========================*/
+ const lock_t* lock_to_remove) /*!< in: lock to remove */
+{
+ trx_t* trx = lock_to_remove->trx;
+
+ ut_ad(lock_mutex_own());
+
+ /* It is safe to read this because we are holding the lock mutex */
+ if (!trx->lock.cancel) {
+ trx_mutex_enter(trx);
+ } else {
+ ut_ad(trx_mutex_own(trx));
+ }
+
+ for (lock_list::iterator it = trx->lock.table_locks.begin(),
+ end = trx->lock.table_locks.end(); it != end; ++it) {
+ const lock_t* lock = *it;
+
+ ut_ad(!lock || trx == lock->trx);
+ ut_ad(!lock || lock_get_type_low(lock) & LOCK_TABLE);
+ ut_ad(!lock || lock->un_member.tab_lock.table);
+
+ if (lock == lock_to_remove) {
+ *it = NULL;
+
+ if (!trx->lock.cancel) {
+ trx_mutex_exit(trx);
+ }
+
+ return;
+ }
+ }
+
+ if (!trx->lock.cancel) {
+ trx_mutex_exit(trx);
+ }
+
+ /* Lock must exist in the vector. */
+ ut_error;
+}
+
+/*===================== VALIDATION AND DEBUGGING ====================*/
+
+/** Print info of a table lock.
+@param[in,out] file output stream
+@param[in] lock table lock */
+static
+void
+lock_table_print(FILE* file, const lock_t* lock)
+{
+ ut_ad(lock_mutex_own());
+ ut_a(lock_get_type_low(lock) == LOCK_TABLE);
+
+ fputs("TABLE LOCK table ", file);
+ ut_print_name(file, lock->trx,
+ lock->un_member.tab_lock.table->name.m_name);
+ fprintf(file, " trx id " TRX_ID_FMT, trx_get_id_for_print(lock->trx));
+
+ if (lock_get_mode(lock) == LOCK_S) {
+ fputs(" lock mode S", file);
+ } else if (lock_get_mode(lock) == LOCK_X) {
+ ut_ad(lock->trx->id != 0);
+ fputs(" lock mode X", file);
+ } else if (lock_get_mode(lock) == LOCK_IS) {
+ fputs(" lock mode IS", file);
+ } else if (lock_get_mode(lock) == LOCK_IX) {
+ ut_ad(lock->trx->id != 0);
+ fputs(" lock mode IX", file);
+ } else if (lock_get_mode(lock) == LOCK_AUTO_INC) {
+ fputs(" lock mode AUTO-INC", file);
+ } else {
+ fprintf(file, " unknown lock mode %lu",
+ (ulong) lock_get_mode(lock));
+ }
+
+ if (lock_get_wait(lock)) {
+ fputs(" waiting", file);
+ }
+
+ putc('\n', file);
+}
+
+/** Pretty-print a record lock.
+@param[in,out] file output stream
+@param[in] lock record lock
+@param[in,out] mtr mini-transaction for accessing the record */
+static void lock_rec_print(FILE* file, const lock_t* lock, mtr_t& mtr)
+{
+ ut_ad(lock_mutex_own());
+ ut_a(lock_get_type_low(lock) == LOCK_REC);
+
+ const page_id_t page_id(lock->un_member.rec_lock.page_id);
+
+ fprintf(file, "RECORD LOCKS space id %u page no %u n bits " ULINTPF
+ " index %s of table ",
+ page_id.space(), page_id.page_no(),
+ lock_rec_get_n_bits(lock),
+ lock->index->name());
+ ut_print_name(file, lock->trx, lock->index->table->name.m_name);
+ fprintf(file, " trx id " TRX_ID_FMT, trx_get_id_for_print(lock->trx));
+
+ if (lock_get_mode(lock) == LOCK_S) {
+ fputs(" lock mode S", file);
+ } else if (lock_get_mode(lock) == LOCK_X) {
+ fputs(" lock_mode X", file);
+ } else {
+ ut_error;
+ }
+
+ if (lock_rec_get_gap(lock)) {
+ fputs(" locks gap before rec", file);
+ }
+
+ if (lock_rec_get_rec_not_gap(lock)) {
+ fputs(" locks rec but not gap", file);
+ }
+
+ if (lock_rec_get_insert_intention(lock)) {
+ fputs(" insert intention", file);
+ }
+
+ if (lock_get_wait(lock)) {
+ fputs(" waiting", file);
+ }
+
+ putc('\n', file);
+
+ mem_heap_t* heap = NULL;
+ rec_offs offsets_[REC_OFFS_NORMAL_SIZE];
+ rec_offs* offsets = offsets_;
+ rec_offs_init(offsets_);
+
+ mtr.start();
+ const buf_block_t* block = buf_page_try_get(page_id, &mtr);
+
+ for (ulint i = 0; i < lock_rec_get_n_bits(lock); ++i) {
+
+ if (!lock_rec_get_nth_bit(lock, i)) {
+ continue;
+ }
+
+ fprintf(file, "Record lock, heap no %lu", (ulong) i);
+
+ if (block) {
+ ut_ad(page_is_leaf(block->frame));
+ const rec_t* rec;
+
+ rec = page_find_rec_with_heap_no(
+ buf_block_get_frame(block), i);
+ ut_ad(!page_rec_is_metadata(rec));
+
+ offsets = rec_get_offsets(
+ rec, lock->index, offsets,
+ lock->index->n_core_fields,
+ ULINT_UNDEFINED, &heap);
+
+ putc(' ', file);
+ rec_print_new(file, rec, offsets);
+ }
+
+ putc('\n', file);
+ }
+
+ mtr.commit();
+
+ if (UNIV_LIKELY_NULL(heap)) {
+ mem_heap_free(heap);
+ }
+}
+
+#ifdef UNIV_DEBUG
+/* Print the number of lock structs from lock_print_info_summary() only
+in non-production builds for performance reasons, see
+http://bugs.mysql.com/36942 */
+#define PRINT_NUM_OF_LOCK_STRUCTS
+#endif /* UNIV_DEBUG */
+
+#ifdef PRINT_NUM_OF_LOCK_STRUCTS
+/*********************************************************************//**
+Calculates the number of record lock structs in the record lock hash table.
+@return number of record locks */
+static ulint lock_get_n_rec_locks()
+{
+ ulint n_locks = 0;
+ ulint i;
+
+ ut_ad(lock_mutex_own());
+
+ for (i = 0; i < lock_sys.rec_hash.n_cells; i++) {
+ const lock_t* lock;
+
+ for (lock = static_cast<const lock_t*>(
+ HASH_GET_FIRST(&lock_sys.rec_hash, i));
+ lock != 0;
+ lock = static_cast<const lock_t*>(
+ HASH_GET_NEXT(hash, lock))) {
+
+ n_locks++;
+ }
+ }
+
+ return(n_locks);
+}
+#endif /* PRINT_NUM_OF_LOCK_STRUCTS */
+
+/*********************************************************************//**
+Prints info of locks for all transactions.
+@return FALSE if not able to obtain lock mutex
+and exits without printing info */
+ibool
+lock_print_info_summary(
+/*====================*/
+ FILE* file, /*!< in: file where to print */
+ ibool nowait) /*!< in: whether to wait for the lock mutex */
+{
+ /* if nowait is FALSE, wait on the lock mutex,
+ otherwise return immediately if fail to obtain the
+ mutex. */
+ if (!nowait) {
+ lock_mutex_enter();
+ } else if (lock_mutex_enter_nowait()) {
+ fputs("FAIL TO OBTAIN LOCK MUTEX,"
+ " SKIP LOCK INFO PRINTING\n", file);
+ return(FALSE);
+ }
+
+ if (lock_deadlock_found) {
+ fputs("------------------------\n"
+ "LATEST DETECTED DEADLOCK\n"
+ "------------------------\n", file);
+
+ if (!srv_read_only_mode) {
+ ut_copy_file(file, lock_latest_err_file);
+ }
+ }
+
+ fputs("------------\n"
+ "TRANSACTIONS\n"
+ "------------\n", file);
+
+ fprintf(file, "Trx id counter " TRX_ID_FMT "\n",
+ trx_sys.get_max_trx_id());
+
+ fprintf(file,
+ "Purge done for trx's n:o < " TRX_ID_FMT
+ " undo n:o < " TRX_ID_FMT " state: %s\n"
+ "History list length %u\n",
+ purge_sys.tail.trx_no,
+ purge_sys.tail.undo_no,
+ purge_sys.enabled()
+ ? (purge_sys.running() ? "running"
+ : purge_sys.paused() ? "stopped" : "running but idle")
+ : "disabled",
+ uint32_t{trx_sys.rseg_history_len});
+
+#ifdef PRINT_NUM_OF_LOCK_STRUCTS
+ fprintf(file,
+ "Total number of lock structs in row lock hash table %lu\n",
+ (ulong) lock_get_n_rec_locks());
+#endif /* PRINT_NUM_OF_LOCK_STRUCTS */
+ return(TRUE);
+}
+
+/** Prints transaction lock wait and MVCC state.
+@param[in,out] file file where to print
+@param[in] trx transaction
+@param[in] now current time */
+void
+lock_trx_print_wait_and_mvcc_state(FILE* file, const trx_t* trx, time_t now)
+{
+ fprintf(file, "---");
+
+ trx_print_latched(file, trx, 600);
+ trx->read_view.print_limits(file);
+
+ if (trx->lock.que_state == TRX_QUE_LOCK_WAIT) {
+
+ fprintf(file,
+ "------- TRX HAS BEEN WAITING %lu SEC"
+ " FOR THIS LOCK TO BE GRANTED:\n",
+ (ulong) difftime(now, trx->lock.wait_started));
+
+ if (lock_get_type_low(trx->lock.wait_lock) == LOCK_REC) {
+ mtr_t mtr;
+ lock_rec_print(file, trx->lock.wait_lock, mtr);
+ } else {
+ lock_table_print(file, trx->lock.wait_lock);
+ }
+
+ fprintf(file, "------------------\n");
+ }
+}
+
+/*********************************************************************//**
+Prints info of locks for a transaction. */
+static
+void
+lock_trx_print_locks(
+/*=================*/
+ FILE* file, /*!< in/out: File to write */
+ const trx_t* trx) /*!< in: current transaction */
+{
+ mtr_t mtr;
+ uint32_t i= 0;
+ /* Iterate over the transaction's locks. */
+ for (lock_t *lock = UT_LIST_GET_FIRST(trx->lock.trx_locks);
+ lock != NULL;
+ lock = UT_LIST_GET_NEXT(trx_locks, lock)) {
+ if (lock_get_type_low(lock) == LOCK_REC) {
+
+ lock_rec_print(file, lock, mtr);
+ } else {
+ ut_ad(lock_get_type_low(lock) & LOCK_TABLE);
+
+ lock_table_print(file, lock);
+ }
+
+ if (++i == 10) {
+
+ fprintf(file,
+ "10 LOCKS PRINTED FOR THIS TRX:"
+ " SUPPRESSING FURTHER PRINTS\n");
+
+ break;
+ }
+ }
+}
+
+/** Functor to display all transactions */
+struct lock_print_info
+{
+ lock_print_info(FILE* file, time_t now) :
+ file(file), now(now),
+ purge_trx(purge_sys.query ? purge_sys.query->trx : NULL)
+ {}
+
+ void operator()(const trx_t &trx) const
+ {
+ if (UNIV_UNLIKELY(&trx == purge_trx))
+ return;
+ lock_trx_print_wait_and_mvcc_state(file, &trx, now);
+
+ if (trx.will_lock && srv_print_innodb_lock_monitor)
+ lock_trx_print_locks(file, &trx);
+ }
+
+ FILE* const file;
+ const time_t now;
+ const trx_t* const purge_trx;
+};
+
+/*********************************************************************//**
+Prints info of locks for each transaction. This function assumes that the
+caller holds the lock mutex and more importantly it will release the lock
+mutex on behalf of the caller. (This should be fixed in the future). */
+void
+lock_print_info_all_transactions(
+/*=============================*/
+ FILE* file) /*!< in/out: file where to print */
+{
+ ut_ad(lock_mutex_own());
+
+ fprintf(file, "LIST OF TRANSACTIONS FOR EACH SESSION:\n");
+
+ trx_sys.trx_list.for_each(lock_print_info(file, time(nullptr)));
+ lock_mutex_exit();
+
+ ut_ad(lock_validate());
+}
+
+#ifdef UNIV_DEBUG
+/*********************************************************************//**
+Find the the lock in the trx_t::trx_lock_t::table_locks vector.
+@return true if found */
+static
+bool
+lock_trx_table_locks_find(
+/*======================*/
+ trx_t* trx, /*!< in: trx to validate */
+ const lock_t* find_lock) /*!< in: lock to find */
+{
+ bool found = false;
+
+ ut_ad(trx_mutex_own(trx));
+
+ for (lock_list::const_iterator it = trx->lock.table_locks.begin(),
+ end = trx->lock.table_locks.end(); it != end; ++it) {
+
+ const lock_t* lock = *it;
+
+ if (lock == NULL) {
+
+ continue;
+
+ } else if (lock == find_lock) {
+
+ /* Can't be duplicates. */
+ ut_a(!found);
+ found = true;
+ }
+
+ ut_a(trx == lock->trx);
+ ut_a(lock_get_type_low(lock) & LOCK_TABLE);
+ ut_a(lock->un_member.tab_lock.table != NULL);
+ }
+
+ return(found);
+}
+
+/*********************************************************************//**
+Validates the lock queue on a table.
+@return TRUE if ok */
+static
+ibool
+lock_table_queue_validate(
+/*======================*/
+ const dict_table_t* table) /*!< in: table */
+{
+ const lock_t* lock;
+
+ ut_ad(lock_mutex_own());
+
+ for (lock = UT_LIST_GET_FIRST(table->locks);
+ lock != NULL;
+ lock = UT_LIST_GET_NEXT(un_member.tab_lock.locks, lock)) {
+
+ /* lock->trx->state cannot change from or to NOT_STARTED
+ while we are holding the lock_sys.mutex. It may change
+ from ACTIVE or PREPARED to PREPARED or COMMITTED. */
+ trx_mutex_enter(lock->trx);
+ check_trx_state(lock->trx);
+
+ if (lock->trx->state == TRX_STATE_COMMITTED_IN_MEMORY) {
+ } else if (!lock_get_wait(lock)) {
+ ut_a(!lock_table_other_has_incompatible(
+ lock->trx, 0, table,
+ lock_get_mode(lock)));
+ } else {
+ ut_a(lock_table_has_to_wait_in_queue(lock));
+ }
+
+ ut_a(lock_trx_table_locks_find(lock->trx, lock));
+ trx_mutex_exit(lock->trx);
+ }
+
+ return(TRUE);
+}
+
+/*********************************************************************//**
+Validates the lock queue on a single record.
+@return TRUE if ok */
+static
+bool
+lock_rec_queue_validate(
+/*====================*/
+ bool locked_lock_trx_sys,
+ /*!< in: if the caller holds
+ both the lock mutex and
+ trx_sys_t->lock. */
+ const buf_block_t* block, /*!< in: buffer block containing rec */
+ const rec_t* rec, /*!< in: record to look at */
+ const dict_index_t* index, /*!< in: index, or NULL if not known */
+ const rec_offs* offsets)/*!< in: rec_get_offsets(rec, index) */
+{
+ const lock_t* lock;
+ ulint heap_no;
+
+ ut_a(rec);
+ ut_a(block->frame == page_align(rec));
+ ut_ad(rec_offs_validate(rec, index, offsets));
+ ut_ad(!page_rec_is_comp(rec) == !rec_offs_comp(offsets));
+ ut_ad(page_rec_is_leaf(rec));
+ ut_ad(lock_mutex_own() == locked_lock_trx_sys);
+ ut_ad(!index || dict_index_is_clust(index)
+ || !dict_index_is_online_ddl(index));
+
+ heap_no = page_rec_get_heap_no(rec);
+
+ if (!locked_lock_trx_sys) {
+ lock_mutex_enter();
+ }
+
+ if (!page_rec_is_user_rec(rec)) {
+
+ for (lock = lock_rec_get_first(&lock_sys.rec_hash,
+ block, heap_no);
+ lock != NULL;
+ lock = lock_rec_get_next_const(heap_no, lock)) {
+
+ ut_ad(!index || lock->index == index);
+
+ trx_mutex_enter(lock->trx);
+ ut_ad(!lock->trx->read_only
+ || !lock->trx->is_autocommit_non_locking());
+ ut_ad(trx_state_eq(lock->trx,
+ TRX_STATE_COMMITTED_IN_MEMORY)
+ || !lock_get_wait(lock)
+ || lock_rec_has_to_wait_in_queue(lock));
+ trx_mutex_exit(lock->trx);
+ }
+
+func_exit:
+ if (!locked_lock_trx_sys) {
+ lock_mutex_exit();
+ }
+
+ return true;
+ }
+
+ ut_ad(page_rec_is_leaf(rec));
+ ut_ad(lock_mutex_own());
+
+ const trx_id_t impl_trx_id = index && index->is_primary()
+ ? lock_clust_rec_some_has_impl(rec, index, offsets)
+ : 0;
+
+ if (trx_t *impl_trx = impl_trx_id
+ ? trx_sys.find(current_trx(), impl_trx_id, false)
+ : 0) {
+ /* impl_trx could have been committed before we
+ acquire its mutex, but not thereafter. */
+
+ mutex_enter(&impl_trx->mutex);
+ ut_ad(impl_trx->state != TRX_STATE_NOT_STARTED);
+ if (impl_trx->state == TRX_STATE_COMMITTED_IN_MEMORY) {
+ } else if (const lock_t* other_lock
+ = lock_rec_other_has_expl_req(
+ LOCK_S, block, true, heap_no,
+ impl_trx)) {
+ /* The impl_trx is holding an implicit lock on the
+ given record 'rec'. So there cannot be another
+ explicit granted lock. Also, there can be another
+ explicit waiting lock only if the impl_trx has an
+ explicit granted lock. */
+
+#ifdef WITH_WSREP
+ /** Galera record locking rules:
+ * If there is no other record lock to the same record, we may grant
+ the lock request.
+ * If there is other record lock but this requested record lock is
+ compatible, we may grant the lock request.
+ * If there is other record lock and it is not compatible with
+ requested lock, all normal transactions must wait.
+ * BF (brute force) additional exceptions :
+ ** If BF already holds record lock for requested record, we may
+ grant new record lock even if there is conflicting record lock(s)
+ waiting on a queue.
+ ** If conflicting transaction holds requested record lock,
+ we will cancel this record lock and select conflicting transaction
+ for BF abort or kill victim.
+ ** If conflicting transaction is waiting for requested record lock
+ we will cancel this wait and select conflicting transaction
+ for BF abort or kill victim.
+ ** There should not be two BF transactions waiting for same record lock
+ */
+ if (other_lock->trx->is_wsrep() && !lock_get_wait(other_lock)) {
+ wsrep_report_bf_lock_wait(impl_trx->mysql_thd, impl_trx->id);
+ wsrep_report_bf_lock_wait(other_lock->trx->mysql_thd, other_lock->trx->id);
+
+ if (!lock_rec_has_expl(LOCK_X | LOCK_REC_NOT_GAP,
+ block, heap_no,
+ impl_trx)) {
+ ib::info() << "WSREP impl BF lock conflict";
+ }
+ } else
+#endif /* WITH_WSREP */
+ {
+ ut_ad(lock_get_wait(other_lock));
+ ut_ad(lock_rec_has_expl(LOCK_X | LOCK_REC_NOT_GAP,
+ block, heap_no, impl_trx));
+ }
+ }
+
+ mutex_exit(&impl_trx->mutex);
+ }
+
+ for (lock = lock_rec_get_first(&lock_sys.rec_hash, block, heap_no);
+ lock != NULL;
+ lock = lock_rec_get_next_const(heap_no, lock)) {
+ ut_ad(!lock->trx->read_only
+ || !lock->trx->is_autocommit_non_locking());
+ ut_ad(!page_rec_is_metadata(rec));
+
+ if (index) {
+ ut_a(lock->index == index);
+ }
+
+ if (!lock_rec_get_gap(lock) && !lock_get_wait(lock)) {
+
+ lock_mode mode;
+
+ if (lock_get_mode(lock) == LOCK_S) {
+ mode = LOCK_X;
+ } else {
+ mode = LOCK_S;
+ }
+
+ const lock_t* other_lock
+ = lock_rec_other_has_expl_req(
+ mode, block, false, heap_no,
+ lock->trx);
+#ifdef WITH_WSREP
+ if (UNIV_UNLIKELY(other_lock && lock->trx->is_wsrep())) {
+ /* Only BF transaction may be granted
+ lock before other conflicting lock
+ request. */
+ if (!wsrep_thd_is_BF(lock->trx->mysql_thd, FALSE)
+ && !wsrep_thd_is_BF(other_lock->trx->mysql_thd, FALSE)) {
+ /* If no BF, this case is a bug. */
+ wsrep_report_bf_lock_wait(lock->trx->mysql_thd, lock->trx->id);
+ wsrep_report_bf_lock_wait(other_lock->trx->mysql_thd, other_lock->trx->id);
+ ut_error;
+ }
+ } else
+#endif /* WITH_WSREP */
+ ut_ad(!other_lock);
+ } else if (lock_get_wait(lock) && !lock_rec_get_gap(lock)) {
+
+ ut_a(lock_rec_has_to_wait_in_queue(lock));
+ }
+ }
+
+ ut_ad(innodb_lock_schedule_algorithm == INNODB_LOCK_SCHEDULE_ALGORITHM_FCFS ||
+ lock_queue_validate(lock));
+
+ goto func_exit;
+}
+
+/*********************************************************************//**
+Validates the record lock queues on a page.
+@return TRUE if ok */
+static
+ibool
+lock_rec_validate_page(
+/*===================*/
+ const buf_block_t* block) /*!< in: buffer block */
+{
+ const lock_t* lock;
+ const rec_t* rec;
+ ulint nth_lock = 0;
+ ulint nth_bit = 0;
+ ulint i;
+ mem_heap_t* heap = NULL;
+ rec_offs offsets_[REC_OFFS_NORMAL_SIZE];
+ rec_offs* offsets = offsets_;
+ rec_offs_init(offsets_);
+
+ lock_mutex_enter();
+loop:
+ lock = lock_sys.get_first(block->page.id());
+
+ if (!lock) {
+ goto function_exit;
+ }
+
+ DBUG_ASSERT(block->page.status != buf_page_t::FREED);
+
+ for (i = 0; i < nth_lock; i++) {
+
+ lock = lock_rec_get_next_on_page_const(lock);
+
+ if (!lock) {
+ goto function_exit;
+ }
+ }
+
+ ut_ad(!lock->trx->read_only
+ || !lock->trx->is_autocommit_non_locking());
+
+ /* Only validate the record queues when this thread is not
+ holding a space->latch. */
+ if (!sync_check_find(SYNC_FSP))
+ for (i = nth_bit; i < lock_rec_get_n_bits(lock); i++) {
+
+ if (i == PAGE_HEAP_NO_SUPREMUM
+ || lock_rec_get_nth_bit(lock, i)) {
+
+ rec = page_find_rec_with_heap_no(block->frame, i);
+ ut_a(rec);
+ ut_ad(!lock_rec_get_nth_bit(lock, i)
+ || page_rec_is_leaf(rec));
+ offsets = rec_get_offsets(rec, lock->index, offsets,
+ lock->index->n_core_fields,
+ ULINT_UNDEFINED, &heap);
+
+ /* If this thread is holding the file space
+ latch (fil_space_t::latch), the following
+ check WILL break the latching order and may
+ cause a deadlock of threads. */
+
+ lock_rec_queue_validate(
+ TRUE, block, rec, lock->index, offsets);
+
+ nth_bit = i + 1;
+
+ goto loop;
+ }
+ }
+
+ nth_bit = 0;
+ nth_lock++;
+
+ goto loop;
+
+function_exit:
+ lock_mutex_exit();
+
+ if (heap != NULL) {
+ mem_heap_free(heap);
+ }
+ return(TRUE);
+}
+
+/*********************************************************************//**
+Validate record locks up to a limit.
+@return lock at limit or NULL if no more locks in the hash bucket */
+static MY_ATTRIBUTE((warn_unused_result))
+const lock_t*
+lock_rec_validate(
+/*==============*/
+ ulint start, /*!< in: lock_sys.rec_hash
+ bucket */
+ page_id_t* limit) /*!< in/out: upper limit of
+ (space, page_no) */
+{
+ ut_ad(lock_mutex_own());
+
+ for (const lock_t* lock = static_cast<const lock_t*>(
+ HASH_GET_FIRST(&lock_sys.rec_hash, start));
+ lock != NULL;
+ lock = static_cast<const lock_t*>(HASH_GET_NEXT(hash, lock))) {
+
+ ut_ad(!lock->trx->read_only
+ || !lock->trx->is_autocommit_non_locking());
+ ut_ad(lock_get_type(lock) == LOCK_REC);
+
+ page_id_t current(lock->un_member.rec_lock.page_id);
+
+ if (current > *limit) {
+ *limit = current + 1;
+ return(lock);
+ }
+ }
+
+ return(0);
+}
+
+/*********************************************************************//**
+Validate a record lock's block */
+static void lock_rec_block_validate(const page_id_t page_id)
+{
+ /* The lock and the block that it is referring to may be freed at
+ this point. We pass BUF_GET_POSSIBLY_FREED to skip a debug check.
+ If the lock exists in lock_rec_validate_page() we assert
+ block->page.status != FREED. */
+
+ buf_block_t* block;
+ mtr_t mtr;
+
+ /* Transactional locks should never refer to dropped
+ tablespaces, because all DDL operations that would drop or
+ discard or rebuild a tablespace do hold an exclusive table
+ lock, which would conflict with any locks referring to the
+ tablespace from other transactions. */
+ if (fil_space_t* space = fil_space_t::get(page_id.space())) {
+ dberr_t err = DB_SUCCESS;
+ mtr_start(&mtr);
+
+ block = buf_page_get_gen(
+ page_id,
+ space->zip_size(),
+ RW_X_LATCH, NULL,
+ BUF_GET_POSSIBLY_FREED,
+ __FILE__, __LINE__, &mtr, &err);
+
+ if (err != DB_SUCCESS) {
+ ib::error() << "Lock rec block validate failed for tablespace "
+ << space->name
+ << page_id << " err " << err;
+ }
+
+ if (block && block->page.status != buf_page_t::FREED) {
+ buf_block_dbg_add_level(block, SYNC_NO_ORDER_CHECK);
+
+ ut_ad(lock_rec_validate_page(block));
+ }
+
+ mtr_commit(&mtr);
+
+ space->release();
+ }
+}
+
+
+static my_bool lock_validate_table_locks(rw_trx_hash_element_t *element, void*)
+{
+ ut_ad(lock_mutex_own());
+ mutex_enter(&element->mutex);
+ if (element->trx)
+ {
+ check_trx_state(element->trx);
+ for (const lock_t *lock= UT_LIST_GET_FIRST(element->trx->lock.trx_locks);
+ lock != NULL;
+ lock= UT_LIST_GET_NEXT(trx_locks, lock))
+ {
+ if (lock_get_type_low(lock) & LOCK_TABLE)
+ lock_table_queue_validate(lock->un_member.tab_lock.table);
+ }
+ }
+ mutex_exit(&element->mutex);
+ return 0;
+}
+
+
+/*********************************************************************//**
+Validates the lock system.
+@return TRUE if ok */
+static
+bool
+lock_validate()
+/*===========*/
+{
+ std::set<page_id_t> pages;
+
+ lock_mutex_enter();
+
+ /* Validate table locks */
+ trx_sys.rw_trx_hash.iterate(lock_validate_table_locks);
+
+ /* Iterate over all the record locks and validate the locks. We
+ don't want to hog the lock_sys_t::mutex. Release it during the
+ validation check. */
+
+ for (ulint i = 0; i < lock_sys.rec_hash.n_cells; i++) {
+ page_id_t limit(0, 0);
+
+ while (const lock_t* lock = lock_rec_validate(i, &limit)) {
+ if (lock_rec_find_set_bit(lock) == ULINT_UNDEFINED) {
+ /* The lock bitmap is empty; ignore it. */
+ continue;
+ }
+ pages.insert(lock->un_member.rec_lock.page_id);
+ }
+ }
+
+ lock_mutex_exit();
+
+ for (page_id_t page_id : pages) {
+ lock_rec_block_validate(page_id);
+ }
+
+ return(true);
+}
+#endif /* UNIV_DEBUG */
+/*============ RECORD LOCK CHECKS FOR ROW OPERATIONS ====================*/
+
+/*********************************************************************//**
+Checks if locks of other transactions prevent an immediate insert of
+a record. If they do, first tests if the query thread should anyway
+be suspended for some reason; if not, then puts the transaction and
+the query thread to the lock wait state and inserts a waiting request
+for a gap x-lock to the lock queue.
+@return DB_SUCCESS, DB_LOCK_WAIT, or DB_DEADLOCK */
+dberr_t
+lock_rec_insert_check_and_lock(
+/*===========================*/
+ ulint flags, /*!< in: if BTR_NO_LOCKING_FLAG bit is
+ set, does nothing */
+ const rec_t* rec, /*!< in: record after which to insert */
+ buf_block_t* block, /*!< in/out: buffer block of rec */
+ dict_index_t* index, /*!< in: index */
+ que_thr_t* thr, /*!< in: query thread */
+ mtr_t* mtr, /*!< in/out: mini-transaction */
+ bool* inherit)/*!< out: set to true if the new
+ inserted record maybe should inherit
+ LOCK_GAP type locks from the successor
+ record */
+{
+ ut_ad(block->frame == page_align(rec));
+ ut_ad(!dict_index_is_online_ddl(index)
+ || index->is_primary()
+ || (flags & BTR_CREATE_FLAG));
+ ut_ad(mtr->is_named_space(index->table->space));
+ ut_ad(page_rec_is_leaf(rec));
+
+ if (flags & BTR_NO_LOCKING_FLAG) {
+
+ return(DB_SUCCESS);
+ }
+
+ ut_ad(!index->table->is_temporary());
+ ut_ad(page_is_leaf(block->frame));
+
+ dberr_t err;
+ lock_t* lock;
+ bool inherit_in = *inherit;
+ trx_t* trx = thr_get_trx(thr);
+ const rec_t* next_rec = page_rec_get_next_const(rec);
+ ulint heap_no = page_rec_get_heap_no(next_rec);
+ ut_ad(!rec_is_metadata(next_rec, *index));
+
+ lock_mutex_enter();
+ /* 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. */
+
+ /* When inserting a record into an index, the table must be at
+ least IX-locked. When we are building an index, we would pass
+ BTR_NO_LOCKING_FLAG and skip the locking altogether. */
+ ut_ad(lock_table_has(trx, index->table, LOCK_IX));
+
+ lock = lock_rec_get_first(&lock_sys.rec_hash, block, heap_no);
+
+ if (lock == NULL) {
+ /* We optimize CPU time usage in the simplest case */
+
+ lock_mutex_exit();
+
+ if (inherit_in && !dict_index_is_clust(index)) {
+ /* Update the page max trx id field */
+ page_update_max_trx_id(block,
+ buf_block_get_page_zip(block),
+ trx->id, mtr);
+ }
+
+ *inherit = false;
+
+ return(DB_SUCCESS);
+ }
+
+ /* Spatial index does not use GAP lock protection. It uses
+ "predicate lock" to protect the "range" */
+ if (dict_index_is_spatial(index)) {
+ return(DB_SUCCESS);
+ }
+
+ *inherit = true;
+
+ /* If another transaction has an explicit lock request which locks
+ the gap, waiting or granted, on the successor, the insert has to wait.
+
+ An exception is the case where the lock by the another transaction
+ is a gap type lock which it placed to wait for its turn to insert. We
+ do not consider that kind of a lock conflicting with our insert. This
+ eliminates an unnecessary deadlock which resulted when 2 transactions
+ had to wait for their insert. Both had waiting gap type lock requests
+ on the successor, which produced an unnecessary deadlock. */
+
+ const unsigned type_mode = LOCK_X | LOCK_GAP | LOCK_INSERT_INTENTION;
+
+ if (
+#ifdef WITH_WSREP
+ lock_t* c_lock =
+#endif /* WITH_WSREP */
+ lock_rec_other_has_conflicting(type_mode, block, heap_no, trx)) {
+ /* Note that we may get DB_SUCCESS also here! */
+ trx_mutex_enter(trx);
+
+ err = lock_rec_enqueue_waiting(
+#ifdef WITH_WSREP
+ c_lock,
+#endif /* WITH_WSREP */
+ type_mode, block, heap_no, index, thr, NULL);
+
+ trx_mutex_exit(trx);
+ } else {
+ err = DB_SUCCESS;
+ }
+
+ lock_mutex_exit();
+
+ switch (err) {
+ case DB_SUCCESS_LOCKED_REC:
+ err = DB_SUCCESS;
+ /* fall through */
+ case DB_SUCCESS:
+ if (!inherit_in || dict_index_is_clust(index)) {
+ break;
+ }
+
+ /* Update the page max trx id field */
+ page_update_max_trx_id(
+ block, buf_block_get_page_zip(block), trx->id, mtr);
+ default:
+ /* We only care about the two return values. */
+ break;
+ }
+
+#ifdef UNIV_DEBUG
+ {
+ mem_heap_t* heap = NULL;
+ rec_offs offsets_[REC_OFFS_NORMAL_SIZE];
+ const rec_offs* offsets;
+ rec_offs_init(offsets_);
+
+ offsets = rec_get_offsets(next_rec, index, offsets_,
+ index->n_core_fields,
+ ULINT_UNDEFINED, &heap);
+
+ ut_ad(lock_rec_queue_validate(
+ FALSE, block, next_rec, index, offsets));
+
+ if (heap != NULL) {
+ mem_heap_free(heap);
+ }
+ }
+#endif /* UNIV_DEBUG */
+
+ return(err);
+}
+
+/*********************************************************************//**
+Creates an explicit record lock for a running transaction that currently only
+has an implicit lock on the record. The transaction instance must have a
+reference count > 0 so that it can't be committed and freed before this
+function has completed. */
+static
+void
+lock_rec_convert_impl_to_expl_for_trx(
+/*==================================*/
+ const buf_block_t* block, /*!< in: buffer block of rec */
+ const rec_t* rec, /*!< in: user record on page */
+ dict_index_t* index, /*!< in: index of record */
+ trx_t* trx, /*!< in/out: active transaction */
+ ulint heap_no)/*!< in: rec heap number to lock */
+{
+ ut_ad(trx->is_referenced());
+ ut_ad(page_rec_is_leaf(rec));
+ ut_ad(!rec_is_metadata(rec, *index));
+
+ DEBUG_SYNC_C("before_lock_rec_convert_impl_to_expl_for_trx");
+ lock_mutex_enter();
+ trx_mutex_enter(trx);
+ ut_ad(!trx_state_eq(trx, TRX_STATE_NOT_STARTED));
+
+ if (!trx_state_eq(trx, TRX_STATE_COMMITTED_IN_MEMORY)
+ && !lock_rec_has_expl(LOCK_X | LOCK_REC_NOT_GAP,
+ block, heap_no, trx)) {
+ lock_rec_add_to_queue(LOCK_REC | LOCK_X | LOCK_REC_NOT_GAP,
+ block, heap_no, index, trx, true);
+ }
+
+ lock_mutex_exit();
+ trx_mutex_exit(trx);
+ trx->release_reference();
+
+ DEBUG_SYNC_C("after_lock_rec_convert_impl_to_expl_for_trx");
+}
+
+
+#ifdef UNIV_DEBUG
+struct lock_rec_other_trx_holds_expl_arg
+{
+ const ulint heap_no;
+ const buf_block_t * const block;
+ const trx_t *impl_trx;
+};
+
+
+static my_bool lock_rec_other_trx_holds_expl_callback(
+ rw_trx_hash_element_t *element,
+ lock_rec_other_trx_holds_expl_arg *arg)
+{
+ mutex_enter(&element->mutex);
+ if (element->trx)
+ {
+ trx_mutex_enter(element->trx);
+ ut_ad(element->trx->state != TRX_STATE_NOT_STARTED);
+ lock_t *expl_lock= element->trx->state == TRX_STATE_COMMITTED_IN_MEMORY
+ ? NULL : lock_rec_has_expl(LOCK_S | LOCK_REC_NOT_GAP, arg->block,
+ arg->heap_no, element->trx);
+ /*
+ An explicit lock is held by trx other than the trx holding the implicit
+ lock.
+ */
+ ut_ad(!expl_lock || expl_lock->trx == arg->impl_trx);
+ trx_mutex_exit(element->trx);
+ }
+ mutex_exit(&element->mutex);
+ return 0;
+}
+
+
+/**
+ Checks if some transaction, other than given trx_id, has an explicit
+ lock on the given rec.
+
+ FIXME: if the current transaction holds implicit lock from INSERT, a
+ subsequent locking read should not convert it to explicit. See also
+ MDEV-11215.
+
+ @param caller_trx trx of current thread
+ @param[in] trx trx holding implicit lock on rec
+ @param[in] rec user record
+ @param[in] block buffer block containing the record
+*/
+
+static void lock_rec_other_trx_holds_expl(trx_t *caller_trx, trx_t *trx,
+ const rec_t *rec,
+ const buf_block_t *block)
+{
+ if (trx)
+ {
+ ut_ad(!page_rec_is_metadata(rec));
+ lock_mutex_enter();
+ ut_ad(trx->is_referenced());
+ trx_mutex_enter(trx);
+ const trx_state_t state = trx->state;
+ trx_mutex_exit(trx);
+ ut_ad(state != TRX_STATE_NOT_STARTED);
+ if (state == TRX_STATE_COMMITTED_IN_MEMORY)
+ {
+ /* The transaction was committed before our lock_mutex_enter(). */
+ lock_mutex_exit();
+ return;
+ }
+ lock_rec_other_trx_holds_expl_arg arg= { page_rec_get_heap_no(rec), block,
+ trx };
+ trx_sys.rw_trx_hash.iterate(caller_trx,
+ lock_rec_other_trx_holds_expl_callback, &arg);
+ lock_mutex_exit();
+ }
+}
+#endif /* UNIV_DEBUG */
+
+
+/** If an implicit x-lock exists on a record, convert it to an explicit one.
+
+Often, this is called by a transaction that is about to enter a lock wait
+due to the lock conflict. Two explicit locks would be created: first the
+exclusive lock on behalf of the lock-holder transaction in this function,
+and then a wait request on behalf of caller_trx, in the calling function.
+
+This may also be called by the same transaction that is already holding
+an implicit exclusive lock on the record. In this case, no explicit lock
+should be created.
+
+@param[in,out] caller_trx current transaction
+@param[in] block index tree leaf page
+@param[in] rec record on the leaf page
+@param[in] index the index of the record
+@param[in] offsets rec_get_offsets(rec,index)
+@return whether caller_trx already holds an exclusive lock on rec */
+static
+bool
+lock_rec_convert_impl_to_expl(
+ trx_t* caller_trx,
+ const buf_block_t* block,
+ const rec_t* rec,
+ dict_index_t* index,
+ const rec_offs* offsets)
+{
+ trx_t* trx;
+
+ ut_ad(!lock_mutex_own());
+ ut_ad(page_rec_is_user_rec(rec));
+ ut_ad(rec_offs_validate(rec, index, offsets));
+ ut_ad(!page_rec_is_comp(rec) == !rec_offs_comp(offsets));
+ ut_ad(page_rec_is_leaf(rec));
+ ut_ad(!rec_is_metadata(rec, *index));
+
+ if (dict_index_is_clust(index)) {
+ trx_id_t trx_id;
+
+ trx_id = lock_clust_rec_some_has_impl(rec, index, offsets);
+
+ if (trx_id == 0) {
+ return false;
+ }
+ if (UNIV_UNLIKELY(trx_id == caller_trx->id)) {
+ return true;
+ }
+
+ trx = trx_sys.find(caller_trx, trx_id);
+ } else {
+ ut_ad(!dict_index_is_online_ddl(index));
+
+ trx = lock_sec_rec_some_has_impl(caller_trx, rec, index,
+ offsets);
+ if (trx == caller_trx) {
+ trx->release_reference();
+ return true;
+ }
+
+ ut_d(lock_rec_other_trx_holds_expl(caller_trx, trx, rec,
+ block));
+ }
+
+ if (trx != 0) {
+ ulint heap_no = page_rec_get_heap_no(rec);
+
+ ut_ad(trx->is_referenced());
+
+ /* If the transaction is still active and has no
+ explicit x-lock set on the record, set one for it.
+ trx cannot be committed until the ref count is zero. */
+
+ lock_rec_convert_impl_to_expl_for_trx(
+ block, rec, index, trx, heap_no);
+ }
+
+ return false;
+}
+
+/*********************************************************************//**
+Checks if locks of other transactions prevent an immediate modify (update,
+delete mark, or delete unmark) of a clustered index record. If they do,
+first tests if the query thread should anyway be suspended for some
+reason; if not, then puts the transaction and the query thread to the
+lock wait state and inserts a waiting request for a record x-lock to the
+lock queue.
+@return DB_SUCCESS, DB_LOCK_WAIT, or DB_DEADLOCK */
+dberr_t
+lock_clust_rec_modify_check_and_lock(
+/*=================================*/
+ ulint flags, /*!< in: if BTR_NO_LOCKING_FLAG
+ bit is set, does nothing */
+ const buf_block_t* block, /*!< in: buffer block of rec */
+ const rec_t* rec, /*!< in: record which should be
+ modified */
+ dict_index_t* index, /*!< in: clustered index */
+ const rec_offs* offsets,/*!< in: rec_get_offsets(rec, index) */
+ que_thr_t* thr) /*!< in: query thread */
+{
+ dberr_t err;
+ ulint heap_no;
+
+ ut_ad(rec_offs_validate(rec, index, offsets));
+ ut_ad(page_rec_is_leaf(rec));
+ ut_ad(dict_index_is_clust(index));
+ ut_ad(block->frame == page_align(rec));
+
+ if (flags & BTR_NO_LOCKING_FLAG) {
+
+ return(DB_SUCCESS);
+ }
+ ut_ad(!rec_is_metadata(rec, *index));
+ ut_ad(!index->table->is_temporary());
+
+ heap_no = rec_offs_comp(offsets)
+ ? rec_get_heap_no_new(rec)
+ : rec_get_heap_no_old(rec);
+
+ /* If a transaction has no explicit x-lock set on the record, set one
+ for it */
+
+ if (lock_rec_convert_impl_to_expl(thr_get_trx(thr), block, rec, index,
+ offsets)) {
+ /* We already hold an implicit exclusive lock. */
+ return DB_SUCCESS;
+ }
+
+ err = lock_rec_lock(TRUE, LOCK_X | LOCK_REC_NOT_GAP,
+ block, heap_no, index, thr);
+
+ ut_ad(lock_rec_queue_validate(FALSE, block, rec, index, offsets));
+
+ if (err == DB_SUCCESS_LOCKED_REC) {
+ err = DB_SUCCESS;
+ }
+
+ return(err);
+}
+
+/*********************************************************************//**
+Checks if locks of other transactions prevent an immediate modify (delete
+mark or delete unmark) of a secondary index record.
+@return DB_SUCCESS, DB_LOCK_WAIT, or DB_DEADLOCK */
+dberr_t
+lock_sec_rec_modify_check_and_lock(
+/*===============================*/
+ ulint flags, /*!< in: if BTR_NO_LOCKING_FLAG
+ bit is set, does nothing */
+ buf_block_t* block, /*!< in/out: buffer block of rec */
+ const rec_t* rec, /*!< in: record which should be
+ modified; NOTE: as this is a secondary
+ index, we always have to modify the
+ clustered index record first: see the
+ comment below */
+ dict_index_t* index, /*!< in: secondary index */
+ que_thr_t* thr, /*!< in: query thread
+ (can be NULL if BTR_NO_LOCKING_FLAG) */
+ mtr_t* mtr) /*!< in/out: mini-transaction */
+{
+ dberr_t err;
+ ulint heap_no;
+
+ ut_ad(!dict_index_is_clust(index));
+ ut_ad(!dict_index_is_online_ddl(index) || (flags & BTR_CREATE_FLAG));
+ ut_ad(block->frame == page_align(rec));
+ ut_ad(mtr->is_named_space(index->table->space));
+ ut_ad(page_rec_is_leaf(rec));
+ ut_ad(!rec_is_metadata(rec, *index));
+
+ if (flags & BTR_NO_LOCKING_FLAG) {
+
+ return(DB_SUCCESS);
+ }
+ ut_ad(!index->table->is_temporary());
+
+ heap_no = page_rec_get_heap_no(rec);
+
+#ifdef WITH_WSREP
+ trx_t *trx= thr_get_trx(thr);
+ /* If transaction scanning an unique secondary key is wsrep
+ high priority thread (brute force) this scanning may involve
+ GAP-locking in the index. As this locking happens also when
+ applying replication events in high priority applier threads,
+ there is a probability for lock conflicts between two wsrep
+ high priority threads. To avoid this GAP-locking we mark that
+ this transaction is using unique key scan here. */
+ if (trx->is_wsrep() && wsrep_thd_is_BF(trx->mysql_thd, false))
+ trx->wsrep_UK_scan= true;
+#endif /* WITH_WSREP */
+
+ /* Another transaction cannot have an implicit lock on the record,
+ because when we come here, we already have modified the clustered
+ index record, and this would not have been possible if another active
+ transaction had modified this secondary index record. */
+
+ err = lock_rec_lock(TRUE, LOCK_X | LOCK_REC_NOT_GAP,
+ block, heap_no, index, thr);
+
+#ifdef WITH_WSREP
+ trx->wsrep_UK_scan= false;
+#endif /* WITH_WSREP */
+
+#ifdef UNIV_DEBUG
+ {
+ mem_heap_t* heap = NULL;
+ rec_offs offsets_[REC_OFFS_NORMAL_SIZE];
+ const rec_offs* offsets;
+ rec_offs_init(offsets_);
+
+ offsets = rec_get_offsets(rec, index, offsets_,
+ index->n_core_fields,
+ ULINT_UNDEFINED, &heap);
+
+ ut_ad(lock_rec_queue_validate(
+ FALSE, block, rec, index, offsets));
+
+ if (heap != NULL) {
+ mem_heap_free(heap);
+ }
+ }
+#endif /* UNIV_DEBUG */
+
+ if (err == DB_SUCCESS || err == DB_SUCCESS_LOCKED_REC) {
+ /* Update the page max trx id field */
+ /* It might not be necessary to do this if
+ err == DB_SUCCESS (no new lock created),
+ but it should not cost too much performance. */
+ page_update_max_trx_id(block,
+ buf_block_get_page_zip(block),
+ thr_get_trx(thr)->id, mtr);
+ err = DB_SUCCESS;
+ }
+
+ return(err);
+}
+
+/*********************************************************************//**
+Like lock_clust_rec_read_check_and_lock(), but reads a
+secondary index record.
+@return DB_SUCCESS, DB_SUCCESS_LOCKED_REC, DB_LOCK_WAIT, or DB_DEADLOCK */
+dberr_t
+lock_sec_rec_read_check_and_lock(
+/*=============================*/
+ ulint flags, /*!< in: if BTR_NO_LOCKING_FLAG
+ bit is set, does nothing */
+ const buf_block_t* block, /*!< in: buffer block of rec */
+ const rec_t* rec, /*!< in: user record or page
+ supremum record which should
+ be read or passed over by a
+ read cursor */
+ dict_index_t* index, /*!< in: secondary index */
+ const rec_offs* offsets,/*!< in: rec_get_offsets(rec, index) */
+ lock_mode mode, /*!< in: mode of the lock which
+ the read cursor should set on
+ records: LOCK_S or LOCK_X; the
+ latter is possible in
+ SELECT FOR UPDATE */
+ unsigned gap_mode,/*!< in: LOCK_ORDINARY, LOCK_GAP, or
+ LOCK_REC_NOT_GAP */
+ que_thr_t* thr) /*!< in: query thread */
+{
+ dberr_t err;
+ ulint heap_no;
+
+ ut_ad(!dict_index_is_clust(index));
+ ut_ad(!dict_index_is_online_ddl(index));
+ ut_ad(block->frame == page_align(rec));
+ ut_ad(page_rec_is_user_rec(rec) || page_rec_is_supremum(rec));
+ ut_ad(rec_offs_validate(rec, index, offsets));
+ ut_ad(page_rec_is_leaf(rec));
+ ut_ad(mode == LOCK_X || mode == LOCK_S);
+
+ if ((flags & BTR_NO_LOCKING_FLAG)
+ || srv_read_only_mode
+ || index->table->is_temporary()) {
+
+ return(DB_SUCCESS);
+ }
+
+ ut_ad(!rec_is_metadata(rec, *index));
+ heap_no = page_rec_get_heap_no(rec);
+
+ /* Some transaction may have an implicit x-lock on the record only
+ if the max trx id for the page >= min trx id for the trx list or a
+ database recovery is running. */
+
+ if (!page_rec_is_supremum(rec)
+ && page_get_max_trx_id(block->frame) >= trx_sys.get_min_trx_id()
+ && lock_rec_convert_impl_to_expl(thr_get_trx(thr), block, rec,
+ index, offsets)) {
+ /* We already hold an implicit exclusive lock. */
+ return DB_SUCCESS;
+ }
+
+#ifdef WITH_WSREP
+ trx_t *trx= thr_get_trx(thr);
+ /* If transaction scanning an unique secondary key is wsrep
+ high priority thread (brute force) this scanning may involve
+ GAP-locking in the index. As this locking happens also when
+ applying replication events in high priority applier threads,
+ there is a probability for lock conflicts between two wsrep
+ high priority threads. To avoid this GAP-locking we mark that
+ this transaction is using unique key scan here. */
+ if (trx->is_wsrep() && wsrep_thd_is_BF(trx->mysql_thd, false))
+ trx->wsrep_UK_scan= true;
+#endif /* WITH_WSREP */
+
+ err = lock_rec_lock(FALSE, gap_mode | mode,
+ block, heap_no, index, thr);
+
+#ifdef WITH_WSREP
+ trx->wsrep_UK_scan= false;
+#endif /* WITH_WSREP */
+
+ ut_ad(lock_rec_queue_validate(FALSE, block, rec, index, offsets));
+
+ return(err);
+}
+
+/*********************************************************************//**
+Checks if locks of other transactions prevent an immediate read, or passing
+over by a read cursor, of a clustered index record. If they do, first tests
+if the query thread should anyway be suspended for some reason; if not, then
+puts the transaction and the query thread to the lock wait state and inserts a
+waiting request for a record lock to the lock queue. Sets the requested mode
+lock on the record.
+@return DB_SUCCESS, DB_SUCCESS_LOCKED_REC, DB_LOCK_WAIT, or DB_DEADLOCK */
+dberr_t
+lock_clust_rec_read_check_and_lock(
+/*===============================*/
+ ulint flags, /*!< in: if BTR_NO_LOCKING_FLAG
+ bit is set, does nothing */
+ const buf_block_t* block, /*!< in: buffer block of rec */
+ const rec_t* rec, /*!< in: user record or page
+ supremum record which should
+ be read or passed over by a
+ read cursor */
+ dict_index_t* index, /*!< in: clustered index */
+ const rec_offs* offsets,/*!< in: rec_get_offsets(rec, index) */
+ lock_mode mode, /*!< in: mode of the lock which
+ the read cursor should set on
+ records: LOCK_S or LOCK_X; the
+ latter is possible in
+ SELECT FOR UPDATE */
+ unsigned gap_mode,/*!< in: LOCK_ORDINARY, LOCK_GAP, or
+ LOCK_REC_NOT_GAP */
+ que_thr_t* thr) /*!< in: query thread */
+{
+ dberr_t err;
+ ulint heap_no;
+
+ ut_ad(dict_index_is_clust(index));
+ ut_ad(block->frame == page_align(rec));
+ ut_ad(page_rec_is_user_rec(rec) || page_rec_is_supremum(rec));
+ ut_ad(gap_mode == LOCK_ORDINARY || gap_mode == LOCK_GAP
+ || gap_mode == LOCK_REC_NOT_GAP);
+ ut_ad(rec_offs_validate(rec, index, offsets));
+ ut_ad(page_rec_is_leaf(rec));
+ ut_ad(!rec_is_metadata(rec, *index));
+
+ if ((flags & BTR_NO_LOCKING_FLAG)
+ || srv_read_only_mode
+ || index->table->is_temporary()) {
+
+ return(DB_SUCCESS);
+ }
+
+ heap_no = page_rec_get_heap_no(rec);
+
+ if (heap_no != PAGE_HEAP_NO_SUPREMUM
+ && lock_rec_convert_impl_to_expl(thr_get_trx(thr), block, rec,
+ index, offsets)) {
+ /* We already hold an implicit exclusive lock. */
+ return DB_SUCCESS;
+ }
+
+ err = lock_rec_lock(FALSE, gap_mode | mode,
+ block, heap_no, index, thr);
+
+ ut_ad(lock_rec_queue_validate(FALSE, block, rec, index, offsets));
+
+ DEBUG_SYNC_C("after_lock_clust_rec_read_check_and_lock");
+
+ return(err);
+}
+/*********************************************************************//**
+Checks if locks of other transactions prevent an immediate read, or passing
+over by a read cursor, of a clustered index record. If they do, first tests
+if the query thread should anyway be suspended for some reason; if not, then
+puts the transaction and the query thread to the lock wait state and inserts a
+waiting request for a record lock to the lock queue. Sets the requested mode
+lock on the record. This is an alternative version of
+lock_clust_rec_read_check_and_lock() that does not require the parameter
+"offsets".
+@return DB_SUCCESS, DB_LOCK_WAIT, or DB_DEADLOCK */
+dberr_t
+lock_clust_rec_read_check_and_lock_alt(
+/*===================================*/
+ ulint flags, /*!< in: if BTR_NO_LOCKING_FLAG
+ bit is set, does nothing */
+ const buf_block_t* block, /*!< in: buffer block of rec */
+ const rec_t* rec, /*!< in: user record or page
+ supremum record which should
+ be read or passed over by a
+ read cursor */
+ dict_index_t* index, /*!< in: clustered index */
+ lock_mode mode, /*!< in: mode of the lock which
+ the read cursor should set on
+ records: LOCK_S or LOCK_X; the
+ latter is possible in
+ SELECT FOR UPDATE */
+ unsigned gap_mode,/*!< in: LOCK_ORDINARY, LOCK_GAP, or
+ LOCK_REC_NOT_GAP */
+ que_thr_t* thr) /*!< in: query thread */
+{
+ mem_heap_t* tmp_heap = NULL;
+ rec_offs offsets_[REC_OFFS_NORMAL_SIZE];
+ rec_offs* offsets = offsets_;
+ dberr_t err;
+ rec_offs_init(offsets_);
+
+ ut_ad(page_rec_is_leaf(rec));
+ offsets = rec_get_offsets(rec, index, offsets, index->n_core_fields,
+ ULINT_UNDEFINED, &tmp_heap);
+ err = lock_clust_rec_read_check_and_lock(flags, block, rec, index,
+ offsets, mode, gap_mode, thr);
+ if (tmp_heap) {
+ mem_heap_free(tmp_heap);
+ }
+
+ if (err == DB_SUCCESS_LOCKED_REC) {
+ err = DB_SUCCESS;
+ }
+
+ return(err);
+}
+
+/*******************************************************************//**
+Release the last lock from the transaction's autoinc locks. */
+UNIV_INLINE
+void
+lock_release_autoinc_last_lock(
+/*===========================*/
+ ib_vector_t* autoinc_locks) /*!< in/out: vector of AUTOINC locks */
+{
+ ulint last;
+ lock_t* lock;
+
+ ut_ad(lock_mutex_own());
+ ut_a(!ib_vector_is_empty(autoinc_locks));
+
+ /* The lock to be release must be the last lock acquired. */
+ last = ib_vector_size(autoinc_locks) - 1;
+ lock = *static_cast<lock_t**>(ib_vector_get(autoinc_locks, last));
+
+ /* Should have only AUTOINC locks in the vector. */
+ ut_a(lock_get_mode(lock) == LOCK_AUTO_INC);
+ ut_a(lock_get_type(lock) == LOCK_TABLE);
+
+ ut_a(lock->un_member.tab_lock.table != NULL);
+
+ /* This will remove the lock from the trx autoinc_locks too. */
+ lock_table_dequeue(lock);
+
+ /* Remove from the table vector too. */
+ lock_trx_table_locks_remove(lock);
+}
+
+/*******************************************************************//**
+Check if a transaction holds any autoinc locks.
+@return TRUE if the transaction holds any AUTOINC locks. */
+static
+ibool
+lock_trx_holds_autoinc_locks(
+/*=========================*/
+ const trx_t* trx) /*!< in: transaction */
+{
+ ut_a(trx->autoinc_locks != NULL);
+
+ return(!ib_vector_is_empty(trx->autoinc_locks));
+}
+
+/*******************************************************************//**
+Release all the transaction's autoinc locks. */
+static
+void
+lock_release_autoinc_locks(
+/*=======================*/
+ trx_t* trx) /*!< in/out: transaction */
+{
+ ut_ad(lock_mutex_own());
+ /* If this is invoked for a running transaction by the thread
+ that is serving the transaction, then it is not necessary to
+ hold trx->mutex here. */
+
+ ut_a(trx->autoinc_locks != NULL);
+
+ /* We release the locks in the reverse order. This is to
+ avoid searching the vector for the element to delete at
+ the lower level. See (lock_table_remove_low()) for details. */
+ while (!ib_vector_is_empty(trx->autoinc_locks)) {
+
+ /* lock_table_remove_low() will also remove the lock from
+ the transaction's autoinc_locks vector. */
+ lock_release_autoinc_last_lock(trx->autoinc_locks);
+ }
+
+ /* Should release all locks. */
+ ut_a(ib_vector_is_empty(trx->autoinc_locks));
+}
+
+/*******************************************************************//**
+Gets the type of a lock. Non-inline version for using outside of the
+lock module.
+@return LOCK_TABLE or LOCK_REC */
+ulint
+lock_get_type(
+/*==========*/
+ const lock_t* lock) /*!< in: lock */
+{
+ return(lock_get_type_low(lock));
+}
+
+/*******************************************************************//**
+Gets the id of the transaction owning a lock.
+@return transaction id */
+trx_id_t
+lock_get_trx_id(
+/*============*/
+ const lock_t* lock) /*!< in: lock */
+{
+ return(trx_get_id_for_print(lock->trx));
+}
+
+/*******************************************************************//**
+Gets the table on which the lock is.
+@return table */
+UNIV_INLINE
+dict_table_t*
+lock_get_table(
+/*===========*/
+ const lock_t* lock) /*!< in: lock */
+{
+ switch (lock_get_type_low(lock)) {
+ case LOCK_REC:
+ ut_ad(dict_index_is_clust(lock->index)
+ || !dict_index_is_online_ddl(lock->index));
+ return(lock->index->table);
+ case LOCK_TABLE:
+ return(lock->un_member.tab_lock.table);
+ default:
+ ut_error;
+ return(NULL);
+ }
+}
+
+/*******************************************************************//**
+Gets the id of the table on which the lock is.
+@return id of the table */
+table_id_t
+lock_get_table_id(
+/*==============*/
+ const lock_t* lock) /*!< in: lock */
+{
+ dict_table_t* table = lock_get_table(lock);
+ ut_ad(!table->is_temporary());
+ return(table->id);
+}
+
+/** Determine which table a lock is associated with.
+@param[in] lock the lock
+@return name of the table */
+const table_name_t&
+lock_get_table_name(
+ const lock_t* lock)
+{
+ return(lock_get_table(lock)->name);
+}
+
+/*******************************************************************//**
+For a record lock, gets the index on which the lock is.
+@return index */
+const dict_index_t*
+lock_rec_get_index(
+/*===============*/
+ const lock_t* lock) /*!< in: lock */
+{
+ ut_a(lock_get_type_low(lock) == LOCK_REC);
+ ut_ad(dict_index_is_clust(lock->index)
+ || !dict_index_is_online_ddl(lock->index));
+
+ return(lock->index);
+}
+
+/*******************************************************************//**
+For a record lock, gets the name of the index on which the lock is.
+The string should not be free()'d or modified.
+@return name of the index */
+const char*
+lock_rec_get_index_name(
+/*====================*/
+ const lock_t* lock) /*!< in: lock */
+{
+ ut_a(lock_get_type_low(lock) == LOCK_REC);
+ ut_ad(dict_index_is_clust(lock->index)
+ || !dict_index_is_online_ddl(lock->index));
+
+ return(lock->index->name);
+}
+
+/*********************************************************************//**
+Cancels a waiting lock request and releases possible other transactions
+waiting behind it. */
+void
+lock_cancel_waiting_and_release(
+/*============================*/
+ lock_t* lock) /*!< in/out: waiting lock request */
+{
+ que_thr_t* thr;
+
+ ut_ad(lock_mutex_own());
+ ut_ad(trx_mutex_own(lock->trx));
+ ut_ad(lock->trx->state == TRX_STATE_ACTIVE);
+
+ lock->trx->lock.cancel = true;
+
+ if (lock_get_type_low(lock) == LOCK_REC) {
+
+ lock_rec_dequeue_from_page(lock);
+ } else {
+ ut_ad(lock_get_type_low(lock) & LOCK_TABLE);
+
+ if (lock->trx->autoinc_locks != NULL) {
+ /* Release the transaction's AUTOINC locks. */
+ lock_release_autoinc_locks(lock->trx);
+ }
+
+ lock_table_dequeue(lock);
+ /* Remove the lock from table lock vector too. */
+ lock_trx_table_locks_remove(lock);
+ }
+
+ /* Reset the wait flag and the back pointer to lock in trx. */
+
+ lock_reset_lock_and_trx_wait(lock);
+
+ /* The following function releases the trx from lock wait. */
+
+ thr = que_thr_end_lock_wait(lock->trx);
+
+ if (thr != NULL) {
+ lock_wait_release_thread_if_suspended(thr);
+ }
+
+ lock->trx->lock.cancel = false;
+}
+
+/*********************************************************************//**
+Unlocks AUTO_INC type locks that were possibly reserved by a trx. This
+function should be called at the the end of an SQL statement, by the
+connection thread that owns the transaction (trx->mysql_thd). */
+void
+lock_unlock_table_autoinc(
+/*======================*/
+ trx_t* trx) /*!< in/out: transaction */
+{
+ ut_ad(!lock_mutex_own());
+ ut_ad(!trx_mutex_own(trx));
+ ut_ad(!trx->lock.wait_lock);
+
+ /* This can be invoked on NOT_STARTED, ACTIVE, PREPARED,
+ but not COMMITTED transactions. */
+
+ ut_ad(trx_state_eq(trx, TRX_STATE_NOT_STARTED)
+ || !trx_state_eq(trx, TRX_STATE_COMMITTED_IN_MEMORY));
+
+ /* This function is invoked for a running transaction by the
+ thread that is serving the transaction. Therefore it is not
+ necessary to hold trx->mutex here. */
+
+ if (lock_trx_holds_autoinc_locks(trx)) {
+ lock_mutex_enter();
+
+ lock_release_autoinc_locks(trx);
+
+ lock_mutex_exit();
+ }
+}
+
+static inline dberr_t lock_trx_handle_wait_low(trx_t* trx)
+{
+ ut_ad(lock_mutex_own());
+ ut_ad(trx_mutex_own(trx));
+
+ if (trx->lock.was_chosen_as_deadlock_victim) {
+ return DB_DEADLOCK;
+ }
+ if (!trx->lock.wait_lock) {
+ /* The lock was probably granted before we got here. */
+ return DB_SUCCESS;
+ }
+
+ lock_cancel_waiting_and_release(trx->lock.wait_lock);
+ return DB_LOCK_WAIT;
+}
+
+/*********************************************************************//**
+Check whether the transaction has already been rolled back because it
+was selected as a deadlock victim, or if it has to wait then cancel
+the wait lock.
+@return DB_DEADLOCK, DB_LOCK_WAIT or DB_SUCCESS */
+dberr_t
+lock_trx_handle_wait(
+/*=================*/
+ trx_t* trx) /*!< in/out: trx lock state */
+{
+#ifdef WITH_WSREP
+ /* We already own mutexes */
+ if (trx->lock.was_chosen_as_wsrep_victim) {
+ return lock_trx_handle_wait_low(trx);
+ }
+#endif /* WITH_WSREP */
+ lock_mutex_enter();
+ trx_mutex_enter(trx);
+ dberr_t err = lock_trx_handle_wait_low(trx);
+ lock_mutex_exit();
+ trx_mutex_exit(trx);
+ return err;
+}
+
+/*********************************************************************//**
+Get the number of locks on a table.
+@return number of locks */
+ulint
+lock_table_get_n_locks(
+/*===================*/
+ const dict_table_t* table) /*!< in: table */
+{
+ ulint n_table_locks;
+
+ lock_mutex_enter();
+
+ n_table_locks = UT_LIST_GET_LEN(table->locks);
+
+ lock_mutex_exit();
+
+ return(n_table_locks);
+}
+
+#ifdef UNIV_DEBUG
+/**
+ Do an exhaustive check for any locks (table or rec) against the table.
+
+ @param[in] table check if there are any locks held on records in this table
+ or on the table itself
+*/
+
+static my_bool lock_table_locks_lookup(rw_trx_hash_element_t *element,
+ const dict_table_t *table)
+{
+ ut_ad(lock_mutex_own());
+ mutex_enter(&element->mutex);
+ if (element->trx)
+ {
+ trx_mutex_enter(element->trx);
+ check_trx_state(element->trx);
+ if (element->trx->state != TRX_STATE_COMMITTED_IN_MEMORY)
+ {
+ for (const lock_t *lock= UT_LIST_GET_FIRST(element->trx->lock.trx_locks);
+ lock != NULL;
+ lock= UT_LIST_GET_NEXT(trx_locks, lock))
+ {
+ ut_ad(lock->trx == element->trx);
+ if (lock_get_type_low(lock) == LOCK_REC)
+ {
+ ut_ad(lock->index->online_status != ONLINE_INDEX_CREATION ||
+ lock->index->is_primary());
+ ut_ad(lock->index->table != table);
+ }
+ else
+ ut_ad(lock->un_member.tab_lock.table != table);
+ }
+ }
+ trx_mutex_exit(element->trx);
+ }
+ mutex_exit(&element->mutex);
+ return 0;
+}
+#endif /* UNIV_DEBUG */
+
+/*******************************************************************//**
+Check if there are any locks (table or rec) against table.
+@return true if table has either table or record locks. */
+bool
+lock_table_has_locks(
+/*=================*/
+ const dict_table_t* table) /*!< in: check if there are any locks
+ held on records in this table or on the
+ table itself */
+{
+ ibool has_locks;
+
+ ut_ad(table != NULL);
+ lock_mutex_enter();
+
+ has_locks = UT_LIST_GET_LEN(table->locks) > 0 || table->n_rec_locks > 0;
+
+#ifdef UNIV_DEBUG
+ if (!has_locks) {
+ trx_sys.rw_trx_hash.iterate(lock_table_locks_lookup, table);
+ }
+#endif /* UNIV_DEBUG */
+
+ lock_mutex_exit();
+
+ return(has_locks);
+}
+
+/*******************************************************************//**
+Initialise the table lock list. */
+void
+lock_table_lock_list_init(
+/*======================*/
+ table_lock_list_t* lock_list) /*!< List to initialise */
+{
+ UT_LIST_INIT(*lock_list, &lock_table_t::locks);
+}
+
+/*******************************************************************//**
+Initialise the trx lock list. */
+void
+lock_trx_lock_list_init(
+/*====================*/
+ trx_lock_list_t* lock_list) /*!< List to initialise */
+{
+ UT_LIST_INIT(*lock_list, &lock_t::trx_locks);
+}
+
+
+#ifdef UNIV_DEBUG
+/*******************************************************************//**
+Check if the transaction holds any locks on the sys tables
+or its records.
+@return the strongest lock found on any sys table or 0 for none */
+const lock_t*
+lock_trx_has_sys_table_locks(
+/*=========================*/
+ const trx_t* trx) /*!< in: transaction to check */
+{
+ const lock_t* strongest_lock = 0;
+ lock_mode strongest = LOCK_NONE;
+
+ lock_mutex_enter();
+
+ const lock_list::const_iterator end = trx->lock.table_locks.end();
+ lock_list::const_iterator it = trx->lock.table_locks.begin();
+
+ /* Find a valid mode. Note: ib_vector_size() can be 0. */
+
+ for (/* No op */; it != end; ++it) {
+ const lock_t* lock = *it;
+
+ if (lock != NULL
+ && dict_is_sys_table(lock->un_member.tab_lock.table->id)) {
+
+ strongest = lock_get_mode(lock);
+ ut_ad(strongest != LOCK_NONE);
+ strongest_lock = lock;
+ break;
+ }
+ }
+
+ if (strongest == LOCK_NONE) {
+ lock_mutex_exit();
+ return(NULL);
+ }
+
+ for (/* No op */; it != end; ++it) {
+ const lock_t* lock = *it;
+
+ if (lock == NULL) {
+ continue;
+ }
+
+ ut_ad(trx == lock->trx);
+ ut_ad(lock_get_type_low(lock) & LOCK_TABLE);
+ ut_ad(lock->un_member.tab_lock.table != NULL);
+
+ lock_mode mode = lock_get_mode(lock);
+
+ if (dict_is_sys_table(lock->un_member.tab_lock.table->id)
+ && lock_mode_stronger_or_eq(mode, strongest)) {
+
+ strongest = mode;
+ strongest_lock = lock;
+ }
+ }
+
+ lock_mutex_exit();
+
+ return(strongest_lock);
+}
+
+/** Check if the transaction holds an explicit exclusive lock on a record.
+@param[in] trx transaction
+@param[in] table table
+@param[in] block leaf page
+@param[in] heap_no heap number identifying the record
+@return whether an explicit X-lock is held */
+bool
+lock_trx_has_expl_x_lock(
+ const trx_t* trx, /*!< in: transaction to check */
+ const dict_table_t* table, /*!< in: table to check */
+ const buf_block_t* block, /*!< in: buffer block of the record */
+ ulint heap_no)/*!< in: record heap number */
+{
+ ut_ad(heap_no > PAGE_HEAP_NO_SUPREMUM);
+
+ lock_mutex_enter();
+ ut_ad(lock_table_has(trx, table, LOCK_IX));
+ ut_ad(lock_table_has(trx, table, LOCK_X)
+ || lock_rec_has_expl(LOCK_X | LOCK_REC_NOT_GAP, block, heap_no,
+ trx));
+ lock_mutex_exit();
+ return(true);
+}
+#endif /* UNIV_DEBUG */
+
+/** rewind(3) the file used for storing the latest detected deadlock and
+print a heading message to stderr if printing of all deadlocks to stderr
+is enabled. */
+void
+DeadlockChecker::start_print()
+{
+ ut_ad(lock_mutex_own());
+
+ rewind(lock_latest_err_file);
+ ut_print_timestamp(lock_latest_err_file);
+
+ if (srv_print_all_deadlocks) {
+ ib::info() << "Transactions deadlock detected, dumping"
+ " detailed information.";
+ }
+}
+
+/** Print a message to the deadlock file and possibly to stderr.
+@param msg message to print */
+void
+DeadlockChecker::print(const char* msg)
+{
+ fputs(msg, lock_latest_err_file);
+
+ if (srv_print_all_deadlocks) {
+ ib::info() << msg;
+ }
+}
+
+/** Print transaction data to the deadlock file and possibly to stderr.
+@param trx transaction
+@param max_query_len max query length to print */
+void
+DeadlockChecker::print(const trx_t* trx, ulint max_query_len)
+{
+ ut_ad(lock_mutex_own());
+
+ ulint n_rec_locks = lock_number_of_rows_locked(&trx->lock);
+ ulint n_trx_locks = UT_LIST_GET_LEN(trx->lock.trx_locks);
+ ulint heap_size = mem_heap_get_size(trx->lock.lock_heap);
+
+ trx_print_low(lock_latest_err_file, trx, max_query_len,
+ n_rec_locks, n_trx_locks, heap_size);
+
+ if (srv_print_all_deadlocks) {
+ trx_print_low(stderr, trx, max_query_len,
+ n_rec_locks, n_trx_locks, heap_size);
+ }
+}
+
+/** Print lock data to the deadlock file and possibly to stderr.
+@param lock record or table type lock */
+void
+DeadlockChecker::print(const lock_t* lock)
+{
+ ut_ad(lock_mutex_own());
+
+ if (lock_get_type_low(lock) == LOCK_REC) {
+ mtr_t mtr;
+ lock_rec_print(lock_latest_err_file, lock, mtr);
+
+ if (srv_print_all_deadlocks) {
+ lock_rec_print(stderr, lock, mtr);
+ }
+ } else {
+ lock_table_print(lock_latest_err_file, lock);
+
+ if (srv_print_all_deadlocks) {
+ lock_table_print(stderr, lock);
+ }
+ }
+}
+
+/** Get the next lock in the queue that is owned by a transaction whose
+sub-tree has not already been searched.
+Note: "next" here means PREV for table locks.
+
+@param lock Lock in queue
+@param heap_no heap_no if lock is a record lock else ULINT_UNDEFINED
+
+@return next lock or NULL if at end of queue */
+const lock_t*
+DeadlockChecker::get_next_lock(const lock_t* lock, ulint heap_no) const
+{
+ ut_ad(lock_mutex_own());
+
+ do {
+ if (lock_get_type_low(lock) == LOCK_REC) {
+ ut_ad(heap_no != ULINT_UNDEFINED);
+ lock = lock_rec_get_next_const(heap_no, lock);
+ } else {
+ ut_ad(heap_no == ULINT_UNDEFINED);
+ ut_ad(lock_get_type_low(lock) == LOCK_TABLE);
+
+ lock = UT_LIST_GET_NEXT(
+ un_member.tab_lock.locks, lock);
+ }
+
+ } while (lock != NULL && is_visited(lock));
+
+ ut_ad(lock == NULL
+ || lock_get_type_low(lock) == lock_get_type_low(m_wait_lock));
+
+ return(lock);
+}
+
+/** Get the first lock to search. The search starts from the current
+wait_lock. What we are really interested in is an edge from the
+current wait_lock's owning transaction to another transaction that has
+a lock ahead in the queue. We skip locks where the owning transaction's
+sub-tree has already been searched.
+
+Note: The record locks are traversed from the oldest lock to the
+latest. For table locks we go from latest to oldest.
+
+For record locks, we first position the "iterator" on the first lock on
+the page and then reposition on the actual heap_no. This is required
+due to the way the record lock has is implemented.
+
+@param[out] heap_no if rec lock, else ULINT_UNDEFINED.
+@return first lock or NULL */
+const lock_t*
+DeadlockChecker::get_first_lock(ulint* heap_no) const
+{
+ ut_ad(lock_mutex_own());
+
+ const lock_t* lock = m_wait_lock;
+
+ if (lock_get_type_low(lock) == LOCK_REC) {
+ /* We are only interested in records that match the heap_no. */
+ *heap_no = lock_rec_find_set_bit(lock);
+
+ ut_ad(*heap_no <= 0xffff);
+ ut_ad(*heap_no != ULINT_UNDEFINED);
+
+ /* Find the locks on the page. */
+ lock = lock_sys.get_first(
+ lock->type_mode & LOCK_PREDICATE
+ ? lock_sys.prdt_hash
+ : lock_sys.rec_hash,
+ lock->un_member.rec_lock.page_id);
+
+ /* Position on the first lock on the physical record.*/
+ if (!lock_rec_get_nth_bit(lock, *heap_no)) {
+ lock = lock_rec_get_next_const(*heap_no, lock);
+ }
+
+ ut_a(!lock_get_wait(lock));
+ } else {
+ /* Table locks don't care about the heap_no. */
+ *heap_no = ULINT_UNDEFINED;
+ ut_ad(lock_get_type_low(lock) == LOCK_TABLE);
+ dict_table_t* table = lock->un_member.tab_lock.table;
+ lock = UT_LIST_GET_FIRST(table->locks);
+ }
+
+ /* Must find at least two locks, otherwise there cannot be a
+ waiting lock, secondly the first lock cannot be the wait_lock. */
+ ut_a(lock != NULL);
+ ut_a(lock != m_wait_lock ||
+ (innodb_lock_schedule_algorithm
+ == INNODB_LOCK_SCHEDULE_ALGORITHM_VATS
+ && !thd_is_replication_slave_thread(lock->trx->mysql_thd)));
+
+ /* Check that the lock type doesn't change. */
+ ut_ad(lock_get_type_low(lock) == lock_get_type_low(m_wait_lock));
+
+ return(lock);
+}
+
+/** Notify that a deadlock has been detected and print the conflicting
+transaction info.
+@param lock lock causing deadlock */
+void
+DeadlockChecker::notify(const lock_t* lock) const
+{
+ ut_ad(lock_mutex_own());
+
+ start_print();
+
+ print("\n*** (1) TRANSACTION:\n");
+
+ print(m_wait_lock->trx, 3000);
+
+ print("*** (1) WAITING FOR THIS LOCK TO BE GRANTED:\n");
+
+ print(m_wait_lock);
+
+ print("*** (2) TRANSACTION:\n");
+
+ print(lock->trx, 3000);
+
+ print("*** (2) HOLDS THE LOCK(S):\n");
+
+ print(lock);
+
+ /* It is possible that the joining transaction was granted its
+ lock when we rolled back some other waiting transaction. */
+
+ if (m_start->lock.wait_lock != 0) {
+ print("*** (2) WAITING FOR THIS LOCK TO BE GRANTED:\n");
+
+ print(m_start->lock.wait_lock);
+ }
+
+ DBUG_PRINT("ib_lock", ("deadlock detected"));
+}
+
+/** Select the victim transaction that should be rolledback.
+@return victim transaction */
+const trx_t*
+DeadlockChecker::select_victim() const
+{
+ ut_ad(lock_mutex_own());
+ ut_ad(m_start->lock.wait_lock != 0);
+ ut_ad(m_wait_lock->trx != m_start);
+
+ if (trx_weight_ge(m_wait_lock->trx, m_start)) {
+ /* The joining transaction is 'smaller',
+ choose it as the victim and roll it back. */
+#ifdef WITH_WSREP
+ if (wsrep_thd_is_BF(m_start->mysql_thd, FALSE)) {
+ return(m_wait_lock->trx);
+ }
+#endif /* WITH_WSREP */
+ return(m_start);
+ }
+
+#ifdef WITH_WSREP
+ if (wsrep_thd_is_BF(m_wait_lock->trx->mysql_thd, FALSE)) {
+ return(m_start);
+ }
+#endif /* WITH_WSREP */
+
+ return(m_wait_lock->trx);
+}
+
+/** Looks iteratively for a deadlock. Note: the joining transaction may
+have been granted its lock by the deadlock checks.
+@return 0 if no deadlock else the victim transaction instance.*/
+const trx_t*
+DeadlockChecker::search()
+{
+ ut_ad(lock_mutex_own());
+ ut_ad(!trx_mutex_own(m_start));
+
+ ut_ad(m_start != NULL);
+ ut_ad(m_wait_lock != NULL);
+ ut_ad(!m_wait_lock->trx->auto_commit || m_wait_lock->trx->will_lock);
+ ut_d(check_trx_state(m_wait_lock->trx));
+ ut_ad(m_mark_start <= s_lock_mark_counter);
+
+ /* Look at the locks ahead of wait_lock in the lock queue. */
+ ulint heap_no;
+ const lock_t* lock = get_first_lock(&heap_no);
+
+ for (;;) {
+ /* We should never visit the same sub-tree more than once. */
+ ut_ad(lock == NULL || !is_visited(lock));
+
+ while (m_n_elems > 0 && lock == NULL) {
+
+ /* Restore previous search state. */
+
+ pop(lock, heap_no);
+
+ lock = get_next_lock(lock, heap_no);
+ }
+
+ if (lock == NULL) {
+ break;
+ }
+
+ if (lock == m_wait_lock) {
+
+ /* We can mark this subtree as searched */
+ ut_ad(lock->trx->lock.deadlock_mark <= m_mark_start);
+
+ lock->trx->lock.deadlock_mark = ++s_lock_mark_counter;
+
+ /* We are not prepared for an overflow. This 64-bit
+ counter should never wrap around. At 10^9 increments
+ per second, it would take 10^3 years of uptime. */
+
+ ut_ad(s_lock_mark_counter > 0);
+
+ /* Backtrack */
+ lock = NULL;
+ continue;
+ }
+
+ if (!lock_has_to_wait(m_wait_lock, lock)) {
+ /* No conflict, next lock */
+ lock = get_next_lock(lock, heap_no);
+ continue;
+ }
+
+ if (lock->trx == m_start) {
+ /* Found a cycle. */
+ notify(lock);
+ return select_victim();
+ }
+
+ if (is_too_deep()) {
+ /* Search too deep to continue. */
+ m_too_deep = true;
+ return m_start;
+ }
+
+ /* We do not need to report autoinc locks to the upper
+ layer. These locks are released before commit, so they
+ can not cause deadlocks with binlog-fixed commit
+ order. */
+ if (m_report_waiters
+ && (lock_get_type_low(lock) != LOCK_TABLE
+ || lock_get_mode(lock) != LOCK_AUTO_INC)) {
+ thd_rpl_deadlock_check(m_start->mysql_thd,
+ lock->trx->mysql_thd);
+ }
+
+ if (lock->trx->lock.que_state == TRX_QUE_LOCK_WAIT) {
+ /* Another trx ahead has requested a lock in an
+ incompatible mode, and is itself waiting for a lock. */
+
+ ++m_cost;
+
+ if (!push(lock, heap_no)) {
+ m_too_deep = true;
+ return m_start;
+ }
+
+ m_wait_lock = lock->trx->lock.wait_lock;
+
+ lock = get_first_lock(&heap_no);
+
+ if (is_visited(lock)) {
+ lock = get_next_lock(lock, heap_no);
+ }
+ } else {
+ lock = get_next_lock(lock, heap_no);
+ }
+ }
+
+ ut_a(lock == NULL && m_n_elems == 0);
+
+ /* No deadlock found. */
+ return(0);
+}
+
+/** Print info about transaction that was rolled back.
+@param trx transaction rolled back
+@param lock lock trx wants */
+void
+DeadlockChecker::rollback_print(const trx_t* trx, const lock_t* lock)
+{
+ ut_ad(lock_mutex_own());
+
+ /* If the lock search exceeds the max step
+ or the max depth, the current trx will be
+ the victim. Print its information. */
+ start_print();
+
+ print("TOO DEEP OR LONG SEARCH IN THE LOCK TABLE"
+ " WAITS-FOR GRAPH, WE WILL ROLL BACK"
+ " FOLLOWING TRANSACTION \n\n"
+ "*** TRANSACTION:\n");
+
+ print(trx, 3000);
+
+ print("*** WAITING FOR THIS LOCK TO BE GRANTED:\n");
+
+ print(lock);
+}
+
+/** Rollback transaction selected as the victim. */
+void
+DeadlockChecker::trx_rollback()
+{
+ ut_ad(lock_mutex_own());
+
+ trx_t* trx = m_wait_lock->trx;
+
+ print("*** WE ROLL BACK TRANSACTION (1)\n");
+#ifdef WITH_WSREP
+ if (trx->is_wsrep() && wsrep_thd_is_SR(trx->mysql_thd)) {
+ wsrep_handle_SR_rollback(m_start->mysql_thd, trx->mysql_thd);
+ }
+#endif
+
+ trx_mutex_enter(trx);
+
+ trx->lock.was_chosen_as_deadlock_victim = true;
+
+ lock_cancel_waiting_and_release(trx->lock.wait_lock);
+
+ trx_mutex_exit(trx);
+}
+
+/** Check if a joining lock request results in a deadlock.
+If a deadlock is found, we will resolve the deadlock by
+choosing a victim transaction and rolling it back.
+We will attempt to resolve all deadlocks.
+
+@param[in] lock the lock request
+@param[in,out] trx transaction requesting the lock
+
+@return trx if it was chosen as victim
+@retval NULL if another victim was chosen,
+or there is no deadlock (any more) */
+const trx_t*
+DeadlockChecker::check_and_resolve(const lock_t* lock, trx_t* trx)
+{
+ ut_ad(lock_mutex_own());
+ ut_ad(trx_mutex_own(trx));
+ ut_ad(trx->state == TRX_STATE_ACTIVE);
+ ut_ad(!trx->auto_commit || trx->will_lock);
+ ut_ad(!srv_read_only_mode);
+
+ if (!innobase_deadlock_detect) {
+ return(NULL);
+ }
+
+ /* Release the mutex to obey the latching order.
+ This is safe, because DeadlockChecker::check_and_resolve()
+ is invoked when a lock wait is enqueued for the currently
+ running transaction. Because m_trx is a running transaction
+ (it is not currently suspended because of a lock wait),
+ its state can only be changed by this thread, which is
+ currently associated with the transaction. */
+
+ trx_mutex_exit(trx);
+
+ const trx_t* victim_trx;
+ const bool report_waiters = trx->mysql_thd
+ && thd_need_wait_reports(trx->mysql_thd);
+
+ /* Try and resolve as many deadlocks as possible. */
+ do {
+ DeadlockChecker checker(trx, lock, s_lock_mark_counter,
+ report_waiters);
+
+ victim_trx = checker.search();
+
+ /* Search too deep, we rollback the joining transaction only
+ if it is possible to rollback. Otherwise we rollback the
+ transaction that is holding the lock that the joining
+ transaction wants. */
+ if (checker.is_too_deep()) {
+
+ ut_ad(trx == checker.m_start);
+ ut_ad(trx == victim_trx);
+
+ rollback_print(victim_trx, lock);
+
+ MONITOR_INC(MONITOR_DEADLOCK);
+ srv_stats.lock_deadlock_count.inc();
+
+ break;
+
+ } else if (victim_trx != NULL && victim_trx != trx) {
+
+ ut_ad(victim_trx == checker.m_wait_lock->trx);
+
+ checker.trx_rollback();
+
+ lock_deadlock_found = true;
+
+ MONITOR_INC(MONITOR_DEADLOCK);
+ srv_stats.lock_deadlock_count.inc();
+ }
+
+ } while (victim_trx != NULL && victim_trx != trx);
+
+ /* If the joining transaction was selected as the victim. */
+ if (victim_trx != NULL) {
+
+ print("*** WE ROLL BACK TRANSACTION (2)\n");
+#ifdef WITH_WSREP
+ if (trx->is_wsrep() && wsrep_thd_is_SR(trx->mysql_thd)) {
+ wsrep_handle_SR_rollback(trx->mysql_thd,
+ victim_trx->mysql_thd);
+ }
+#endif
+
+ lock_deadlock_found = true;
+ }
+
+ trx_mutex_enter(trx);
+
+ return(victim_trx);
+}
+
+/*************************************************************//**
+Updates the lock table when a page is split and merged to
+two pages. */
+UNIV_INTERN
+void
+lock_update_split_and_merge(
+ const buf_block_t* left_block, /*!< in: left page to which merged */
+ const rec_t* orig_pred, /*!< in: original predecessor of
+ supremum on the left page before merge*/
+ const buf_block_t* right_block) /*!< in: right page from which merged */
+{
+ const rec_t* left_next_rec;
+
+ ut_ad(page_is_leaf(left_block->frame));
+ ut_ad(page_is_leaf(right_block->frame));
+ ut_ad(page_align(orig_pred) == left_block->frame);
+
+ lock_mutex_enter();
+
+ left_next_rec = page_rec_get_next_const(orig_pred);
+ ut_ad(!page_rec_is_metadata(left_next_rec));
+
+ /* Inherit the locks on the supremum of the left page to the
+ first record which was moved from the right page */
+ lock_rec_inherit_to_gap(
+ left_block, left_block,
+ page_rec_get_heap_no(left_next_rec),
+ PAGE_HEAP_NO_SUPREMUM);
+
+ /* Reset the locks on the supremum of the left page,
+ releasing waiting transactions */
+ lock_rec_reset_and_release_wait(left_block,
+ PAGE_HEAP_NO_SUPREMUM);
+
+ /* Inherit the locks to the supremum of the left page from the
+ successor of the infimum on the right page */
+ lock_rec_inherit_to_gap(left_block, right_block,
+ PAGE_HEAP_NO_SUPREMUM,
+ lock_get_min_heap_no(right_block));
+
+ lock_mutex_exit();
+}
diff --git a/storage/innobase/lock/lock0prdt.cc b/storage/innobase/lock/lock0prdt.cc
new file mode 100644
index 00000000..1eb96a0d
--- /dev/null
+++ b/storage/innobase/lock/lock0prdt.cc
@@ -0,0 +1,1028 @@
+/*****************************************************************************
+
+Copyright (c) 2014, 2016, Oracle and/or its affiliates. All Rights Reserved.
+Copyright (c) 2018, 2020, MariaDB Corporation.
+
+This program is free software; you can redistribute it and/or modify it under
+the terms of the GNU General Public License as published by the Free Software
+Foundation; version 2 of the License.
+
+This program is distributed in the hope that it will be useful, but WITHOUT
+ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or FITNESS
+FOR A PARTICULAR PURPOSE. See the GNU General Public License for more details.
+
+You should have received a copy of the GNU General Public License along with
+this program; if not, write to the Free Software Foundation, Inc.,
+51 Franklin Street, Fifth Floor, Boston, MA 02110-1335 USA
+
+*****************************************************************************/
+
+/**************************************************//**
+@file lock/lock0prdt.cc
+The transaction lock system
+
+Created 9/7/2013 Jimmy Yang
+*******************************************************/
+
+#define LOCK_MODULE_IMPLEMENTATION
+
+#include "lock0lock.h"
+#include "lock0priv.h"
+#include "lock0prdt.h"
+#include "dict0mem.h"
+#include "que0que.h"
+
+/*********************************************************************//**
+Get a minimum bounding box from a Predicate
+@return the minimum bounding box */
+UNIV_INLINE
+rtr_mbr_t*
+prdt_get_mbr_from_prdt(
+/*===================*/
+ const lock_prdt_t* prdt) /*!< in: the lock predicate */
+{
+ rtr_mbr_t* mbr_loc = reinterpret_cast<rtr_mbr_t*>(prdt->data);
+
+ return(mbr_loc);
+}
+
+/*********************************************************************//**
+Get a predicate from a lock
+@return the predicate */
+lock_prdt_t*
+lock_get_prdt_from_lock(
+/*====================*/
+ const lock_t* lock) /*!< in: the lock */
+{
+ lock_prdt_t* prdt = reinterpret_cast<lock_prdt_t*>(
+ &((reinterpret_cast<byte*>(
+ const_cast<lock_t*>(&lock[1])))[
+ UNIV_WORD_SIZE]));
+
+ return(prdt);
+}
+
+/*********************************************************************//**
+Get a minimum bounding box directly from a lock
+@return the minimum bounding box*/
+UNIV_INLINE
+rtr_mbr_t*
+lock_prdt_get_mbr_from_lock(
+/*========================*/
+ const lock_t* lock) /*!< in: the lock */
+{
+ ut_ad(lock->type_mode & LOCK_PREDICATE);
+
+ lock_prdt_t* prdt = lock_get_prdt_from_lock(lock);
+
+ rtr_mbr_t* mbr_loc = prdt_get_mbr_from_prdt(prdt);
+
+ return(mbr_loc);
+}
+
+/*********************************************************************//**
+Append a predicate to the lock */
+void
+lock_prdt_set_prdt(
+/*===============*/
+ lock_t* lock, /*!< in: lock */
+ const lock_prdt_t* prdt) /*!< in: Predicate */
+{
+ ut_ad(lock->type_mode & LOCK_PREDICATE);
+
+ memcpy(&(((byte*) &lock[1])[UNIV_WORD_SIZE]), prdt, sizeof *prdt);
+}
+
+
+/** Check whether two predicate locks are compatible with each other
+@param[in] prdt1 first predicate lock
+@param[in] prdt2 second predicate lock
+@param[in] op predicate comparison operator
+@return true if consistent */
+static
+bool
+lock_prdt_consistent(
+ lock_prdt_t* prdt1,
+ lock_prdt_t* prdt2,
+ ulint op)
+{
+ bool ret = false;
+ rtr_mbr_t* mbr1 = prdt_get_mbr_from_prdt(prdt1);
+ rtr_mbr_t* mbr2 = prdt_get_mbr_from_prdt(prdt2);
+ ulint action;
+
+ if (op) {
+ action = op;
+ } else {
+ if (prdt2->op != 0 && (prdt1->op != prdt2->op)) {
+ return(false);
+ }
+
+ action = prdt1->op;
+ }
+
+ switch (action) {
+ case PAGE_CUR_CONTAIN:
+ ret = MBR_CONTAIN_CMP(mbr1, mbr2);
+ break;
+ case PAGE_CUR_DISJOINT:
+ ret = MBR_DISJOINT_CMP(mbr1, mbr2);
+ break;
+ case PAGE_CUR_MBR_EQUAL:
+ ret = MBR_EQUAL_CMP(mbr1, mbr2);
+ break;
+ case PAGE_CUR_INTERSECT:
+ ret = MBR_INTERSECT_CMP(mbr1, mbr2);
+ break;
+ case PAGE_CUR_WITHIN:
+ ret = MBR_WITHIN_CMP(mbr1, mbr2);
+ break;
+ default:
+ ib::error() << "invalid operator " << action;
+ ut_error;
+ }
+
+ return(ret);
+}
+
+/*********************************************************************//**
+Checks if a predicate lock request for a new lock has to wait for
+another lock.
+@return true if new lock has to wait for lock2 to be released */
+bool
+lock_prdt_has_to_wait(
+/*==================*/
+ const trx_t* trx, /*!< in: trx of new lock */
+ unsigned type_mode,/*!< in: precise mode of the new lock
+ to set: LOCK_S or LOCK_X, possibly
+ ORed to LOCK_PREDICATE or LOCK_PRDT_PAGE,
+ LOCK_INSERT_INTENTION */
+ lock_prdt_t* prdt, /*!< in: lock predicate to check */
+ const lock_t* lock2) /*!< in: another record lock; NOTE that
+ it is assumed that this has a lock bit
+ set on the same record as in the new
+ lock we are setting */
+{
+ lock_prdt_t* cur_prdt = lock_get_prdt_from_lock(lock2);
+
+ ut_ad(trx && lock2);
+ ut_ad((lock2->type_mode & LOCK_PREDICATE && type_mode & LOCK_PREDICATE)
+ || (lock2->type_mode & LOCK_PRDT_PAGE
+ && type_mode & LOCK_PRDT_PAGE));
+
+ ut_ad(type_mode & (LOCK_PREDICATE | LOCK_PRDT_PAGE));
+
+ if (trx != lock2->trx
+ && !lock_mode_compatible(static_cast<lock_mode>(
+ LOCK_MODE_MASK & type_mode),
+ lock_get_mode(lock2))) {
+
+ /* If it is a page lock, then return true (conflict) */
+ if (type_mode & LOCK_PRDT_PAGE) {
+ ut_ad(lock2->type_mode & LOCK_PRDT_PAGE);
+
+ return(true);
+ }
+
+ /* Predicate lock does not conflicts with non-predicate lock */
+ if (!(lock2->type_mode & LOCK_PREDICATE)) {
+ return(FALSE);
+ }
+
+ ut_ad(lock2->type_mode & LOCK_PREDICATE);
+
+ if (!(type_mode & LOCK_INSERT_INTENTION)) {
+ /* PREDICATE locks without LOCK_INSERT_INTENTION flag
+ do not need to wait for anything. This is because
+ different users can have conflicting lock types
+ on predicates. */
+
+ return(FALSE);
+ }
+
+ if (lock2->type_mode & LOCK_INSERT_INTENTION) {
+
+ /* No lock request needs to wait for an insert
+ intention lock to be removed. This makes it similar
+ to GAP lock, that allows conflicting insert intention
+ locks */
+ return(FALSE);
+ }
+
+ if (!lock_prdt_consistent(cur_prdt, prdt, 0)) {
+ return(false);
+ }
+
+ return(TRUE);
+ }
+
+ return(FALSE);
+}
+
+/*********************************************************************//**
+Checks if a transaction has a GRANTED stronger or equal predicate lock
+on the page
+@return lock or NULL */
+UNIV_INLINE
+lock_t*
+lock_prdt_has_lock(
+/*===============*/
+ ulint precise_mode, /*!< in: LOCK_S or LOCK_X */
+ unsigned type_mode, /*!< in: LOCK_PREDICATE etc. */
+ const buf_block_t* block, /*!< in: buffer block
+ containing the record */
+ lock_prdt_t* prdt, /*!< in: The predicate to be
+ attached to the new lock */
+ const trx_t* trx) /*!< in: transaction */
+{
+ lock_t* lock;
+
+ ut_ad(lock_mutex_own());
+ 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 = lock_rec_get_first(
+ lock_hash_get(type_mode), block, PRDT_HEAPNO);
+ lock != NULL;
+ lock = lock_rec_get_next(PRDT_HEAPNO, lock)) {
+ ut_ad(lock->type_mode & (LOCK_PREDICATE | LOCK_PRDT_PAGE));
+
+ if (lock->trx == trx
+ && !(lock->type_mode & LOCK_INSERT_INTENTION)
+ && !lock_get_wait(lock)
+ && lock_mode_stronger_or_eq(
+ lock_get_mode(lock),
+ static_cast<lock_mode>(
+ precise_mode & LOCK_MODE_MASK))) {
+ if (lock->type_mode & LOCK_PRDT_PAGE) {
+ return(lock);
+ }
+
+ ut_ad(lock->type_mode & LOCK_PREDICATE);
+ lock_prdt_t* cur_prdt = lock_get_prdt_from_lock(
+ lock);
+
+ /* if the lock predicate operator is the same
+ as the one to look, and prdicate test is successful,
+ then we find a lock */
+ if (cur_prdt->op == prdt->op
+ && lock_prdt_consistent(cur_prdt, prdt, 0)) {
+
+ return(lock);
+ }
+ }
+ }
+
+ return(NULL);
+}
+
+/*********************************************************************//**
+Checks if some other transaction has a conflicting predicate
+lock request in the queue, so that we have to wait.
+@return lock or NULL */
+static
+lock_t*
+lock_prdt_other_has_conflicting(
+/*============================*/
+ unsigned mode, /*!< in: LOCK_S or LOCK_X,
+ possibly ORed to LOCK_PREDICATE or
+ LOCK_PRDT_PAGE, LOCK_INSERT_INTENTION */
+ const buf_block_t* block, /*!< in: buffer block containing
+ the record */
+ lock_prdt_t* prdt, /*!< in: Predicates (currently)
+ the Minimum Bounding Rectangle)
+ the new lock will be on */
+ const trx_t* trx) /*!< in: our transaction */
+{
+ ut_ad(lock_mutex_own());
+
+ for (lock_t* lock = lock_rec_get_first(
+ lock_hash_get(mode), block, PRDT_HEAPNO);
+ lock != NULL;
+ lock = lock_rec_get_next(PRDT_HEAPNO, lock)) {
+
+ if (lock->trx == trx) {
+ continue;
+ }
+
+ if (lock_prdt_has_to_wait(trx, mode, prdt, lock)) {
+ return(lock);
+ }
+ }
+
+ return(NULL);
+}
+
+/*********************************************************************//**
+Reset the Minimum Bounding Rectangle (to a large area) */
+static
+void
+lock_prdt_enlarge_mbr(
+/*==================*/
+ const lock_t* lock, /*!< in/out: lock to modify */
+ rtr_mbr_t* mbr) /*!< in: Minimum Bounding Rectangle */
+{
+ rtr_mbr_t* cur_mbr = lock_prdt_get_mbr_from_lock(lock);
+
+ if (cur_mbr->xmin > mbr->xmin) {
+ cur_mbr->xmin = mbr->xmin;
+ }
+
+ if (cur_mbr->ymin > mbr->ymin) {
+ cur_mbr->ymin = mbr->ymin;
+ }
+
+ if (cur_mbr->xmax < mbr->xmax) {
+ cur_mbr->xmax = mbr->xmax;
+ }
+
+ if (cur_mbr->ymax < mbr->ymax) {
+ cur_mbr->ymax = mbr->ymax;
+ }
+}
+
+/*********************************************************************//**
+Reset the predicates to a "covering" (larger) predicates */
+static
+void
+lock_prdt_enlarge_prdt(
+/*===================*/
+ lock_t* lock, /*!< in/out: lock to modify */
+ lock_prdt_t* prdt) /*!< in: predicate */
+{
+ rtr_mbr_t* mbr = prdt_get_mbr_from_prdt(prdt);
+
+ lock_prdt_enlarge_mbr(lock, mbr);
+}
+
+/*********************************************************************//**
+Check two predicates' MBRs are the same
+@return true if they are the same */
+static
+bool
+lock_prdt_is_same(
+/*==============*/
+ lock_prdt_t* prdt1, /*!< in: MBR with the lock */
+ lock_prdt_t* prdt2) /*!< in: MBR with the lock */
+{
+ rtr_mbr_t* mbr1 = prdt_get_mbr_from_prdt(prdt1);
+ rtr_mbr_t* mbr2 = prdt_get_mbr_from_prdt(prdt2);
+
+ if (prdt1->op == prdt2->op && MBR_EQUAL_CMP(mbr1, mbr2)) {
+ return(true);
+ }
+
+ return(false);
+}
+
+/*********************************************************************//**
+Looks for a similar predicate lock struct by the same trx on the same page.
+This can be used to save space when a new record lock should be set on a page:
+no new struct is needed, if a suitable old one is found.
+@return lock or NULL */
+static
+lock_t*
+lock_prdt_find_on_page(
+/*===================*/
+ unsigned type_mode, /*!< in: lock type_mode field */
+ const buf_block_t* block, /*!< in: buffer block */
+ lock_prdt_t* prdt, /*!< in: MBR with the lock */
+ const trx_t* trx) /*!< in: transaction */
+{
+ lock_t* lock;
+
+ ut_ad(lock_mutex_own());
+
+ for (lock = lock_sys.get_first(*lock_hash_get(type_mode),
+ block->page.id());
+ lock != NULL;
+ lock = lock_rec_get_next_on_page(lock)) {
+
+ if (lock->trx == trx
+ && lock->type_mode == type_mode) {
+ if (lock->type_mode & LOCK_PRDT_PAGE) {
+ return(lock);
+ }
+
+ ut_ad(lock->type_mode & LOCK_PREDICATE);
+
+ if (lock_prdt_is_same(lock_get_prdt_from_lock(lock),
+ prdt)) {
+ return(lock);
+ }
+ }
+ }
+
+ return(NULL);
+}
+
+/*********************************************************************//**
+Adds a predicate lock request in the predicate lock queue.
+@return lock where the bit was set */
+static
+lock_t*
+lock_prdt_add_to_queue(
+/*===================*/
+ unsigned type_mode,/*!< in: lock mode, wait, predicate
+ etc. flags; type is ignored
+ and replaced by LOCK_REC */
+ const buf_block_t* block, /*!< in: buffer block containing
+ the record */
+ dict_index_t* index, /*!< in: index of record */
+ trx_t* trx, /*!< in/out: transaction */
+ lock_prdt_t* prdt, /*!< in: Minimum Bounding Rectangle
+ the new lock will be on */
+ bool caller_owns_trx_mutex)
+ /*!< in: TRUE if caller owns the
+ transaction mutex */
+{
+ ut_ad(lock_mutex_own());
+ ut_ad(caller_owns_trx_mutex == trx_mutex_own(trx));
+ ut_ad(!dict_index_is_clust(index) && !dict_index_is_online_ddl(index));
+ ut_ad(type_mode & (LOCK_PREDICATE | LOCK_PRDT_PAGE));
+
+#ifdef UNIV_DEBUG
+ switch (type_mode & LOCK_MODE_MASK) {
+ case LOCK_X:
+ case LOCK_S:
+ break;
+ default:
+ ut_error;
+ }
+#endif /* UNIV_DEBUG */
+
+ type_mode |= LOCK_REC;
+
+ /* Look for a waiting lock request on the same record or on a gap */
+
+ lock_t* lock;
+
+ for (lock = lock_sys.get_first(*lock_hash_get(type_mode),
+ block->page.id());
+ lock != NULL;
+ lock = lock_rec_get_next_on_page(lock)) {
+
+ if (lock_get_wait(lock)
+ && lock_rec_get_nth_bit(lock, PRDT_HEAPNO)
+ && lock->type_mode & (LOCK_PREDICATE | LOCK_PRDT_PAGE)) {
+
+ break;
+ }
+ }
+
+ if (lock == NULL && !(type_mode & LOCK_WAIT)) {
+
+ /* Look for a similar record lock on the same page:
+ if one is found and there are no waiting lock requests,
+ we can just set the bit */
+
+ lock = lock_prdt_find_on_page(type_mode, block, prdt, trx);
+
+ if (lock != NULL) {
+
+ if (lock->type_mode & LOCK_PREDICATE) {
+ lock_prdt_enlarge_prdt(lock, prdt);
+ }
+
+ return(lock);
+ }
+ }
+
+ lock = lock_rec_create(
+#ifdef WITH_WSREP
+ NULL, NULL, /* FIXME: replicate SPATIAL INDEX locks */
+#endif
+ type_mode, block, PRDT_HEAPNO, index, trx,
+ caller_owns_trx_mutex);
+
+ if (lock->type_mode & LOCK_PREDICATE) {
+ lock_prdt_set_prdt(lock, prdt);
+ }
+
+ return lock;
+}
+
+/*********************************************************************//**
+Checks if locks of other transactions prevent an immediate insert of
+a predicate record.
+@return DB_SUCCESS, DB_LOCK_WAIT, or DB_DEADLOCK */
+dberr_t
+lock_prdt_insert_check_and_lock(
+/*============================*/
+ ulint flags, /*!< in: if BTR_NO_LOCKING_FLAG bit is
+ set, does nothing */
+ const rec_t* rec, /*!< in: record after which to insert */
+ buf_block_t* block, /*!< in/out: buffer block of rec */
+ dict_index_t* index, /*!< in: index */
+ que_thr_t* thr, /*!< in: query thread */
+ mtr_t* mtr, /*!< in/out: mini-transaction */
+ lock_prdt_t* prdt) /*!< in: Predicates with Minimum Bound
+ Rectangle */
+{
+ ut_ad(block->frame == page_align(rec));
+
+ if (flags & BTR_NO_LOCKING_FLAG) {
+
+ return(DB_SUCCESS);
+ }
+
+ ut_ad(!index->table->is_temporary());
+ ut_ad(!dict_index_is_clust(index));
+
+ trx_t* trx = thr_get_trx(thr);
+
+ lock_mutex_enter();
+
+ /* 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. */
+
+ ut_ad(lock_table_has(trx, index->table, LOCK_IX));
+
+ lock_t* lock;
+
+ /* Only need to check locks on prdt_hash */
+ lock = lock_rec_get_first(&lock_sys.prdt_hash, block, PRDT_HEAPNO);
+
+ if (lock == NULL) {
+ lock_mutex_exit();
+
+ /* Update the page max trx id field */
+ page_update_max_trx_id(block, buf_block_get_page_zip(block),
+ trx->id, mtr);
+
+ return(DB_SUCCESS);
+ }
+
+ ut_ad(lock->type_mode & LOCK_PREDICATE);
+
+ dberr_t err;
+
+ /* If another transaction has an explicit lock request which locks
+ the predicate, waiting or granted, on the successor, the insert
+ has to wait.
+
+ Similar to GAP lock, we do not consider lock from inserts conflicts
+ with each other */
+
+ const ulint mode = LOCK_X | LOCK_PREDICATE | LOCK_INSERT_INTENTION;
+
+ const lock_t* wait_for = lock_prdt_other_has_conflicting(
+ mode, block, prdt, trx);
+
+ if (wait_for != NULL) {
+ rtr_mbr_t* mbr = prdt_get_mbr_from_prdt(prdt);
+
+ /* Allocate MBR on the lock heap */
+ lock_init_prdt_from_mbr(prdt, mbr, 0, trx->lock.lock_heap);
+
+ /* Note that we may get DB_SUCCESS also here! */
+ trx_mutex_enter(trx);
+
+ err = lock_rec_enqueue_waiting(
+#ifdef WITH_WSREP
+ NULL, /* FIXME: replicate SPATIAL INDEX locks */
+#endif
+ LOCK_X | LOCK_PREDICATE | LOCK_INSERT_INTENTION,
+ block, PRDT_HEAPNO, index, thr, prdt);
+
+ trx_mutex_exit(trx);
+ } else {
+ err = DB_SUCCESS;
+ }
+
+ lock_mutex_exit();
+
+ switch (err) {
+ case DB_SUCCESS_LOCKED_REC:
+ err = DB_SUCCESS;
+ /* fall through */
+ case DB_SUCCESS:
+ /* Update the page max trx id field */
+ page_update_max_trx_id(block,
+ buf_block_get_page_zip(block),
+ trx->id, mtr);
+ default:
+ /* We only care about the two return values. */
+ break;
+ }
+
+ return(err);
+}
+
+/**************************************************************//**
+Check whether any predicate lock in parent needs to propagate to
+child page after split. */
+void
+lock_prdt_update_parent(
+/*====================*/
+ buf_block_t* left_block, /*!< in/out: page to be split */
+ buf_block_t* right_block, /*!< in/out: the new half page */
+ lock_prdt_t* left_prdt, /*!< in: MBR on the old page */
+ lock_prdt_t* right_prdt, /*!< in: MBR on the new page */
+ const page_id_t page_id) /*!< in: parent page */
+{
+ lock_mutex_enter();
+
+ /* Get all locks in parent */
+ for (lock_t *lock = lock_sys.get_first_prdt(page_id);
+ lock;
+ lock = lock_rec_get_next_on_page(lock)) {
+ lock_prdt_t* lock_prdt;
+ ulint op = PAGE_CUR_DISJOINT;
+
+ ut_ad(lock);
+
+ if (!(lock->type_mode & LOCK_PREDICATE)
+ || (lock->type_mode & LOCK_MODE_MASK) == LOCK_X) {
+ continue;
+ }
+
+ lock_prdt = lock_get_prdt_from_lock(lock);
+
+ /* Check each lock in parent to see if it intersects with
+ left or right child */
+ if (!lock_prdt_consistent(lock_prdt, left_prdt, op)
+ && !lock_prdt_find_on_page(lock->type_mode, left_block,
+ lock_prdt, lock->trx)) {
+ lock_prdt_add_to_queue(lock->type_mode,
+ left_block, lock->index,
+ lock->trx, lock_prdt,
+ FALSE);
+ }
+
+ if (!lock_prdt_consistent(lock_prdt, right_prdt, op)
+ && !lock_prdt_find_on_page(lock->type_mode, right_block,
+ lock_prdt, lock->trx)) {
+ lock_prdt_add_to_queue(lock->type_mode, right_block,
+ lock->index, lock->trx,
+ lock_prdt, FALSE);
+ }
+ }
+
+ lock_mutex_exit();
+}
+
+/**************************************************************//**
+Update predicate lock when page splits */
+static
+void
+lock_prdt_update_split_low(
+/*=======================*/
+ buf_block_t* new_block, /*!< in/out: the new half page */
+ lock_prdt_t* prdt, /*!< in: MBR on the old page */
+ lock_prdt_t* new_prdt, /*!< in: MBR on the new page */
+ const page_id_t page_id, /*!< in: page number */
+ unsigned type_mode) /*!< in: LOCK_PREDICATE or
+ LOCK_PRDT_PAGE */
+{
+ lock_t* lock;
+
+ for (lock = lock_sys.get_first(*lock_hash_get(type_mode), page_id);
+ lock;
+ lock = lock_rec_get_next_on_page(lock)) {
+ /* First dealing with Page Lock */
+ if (lock->type_mode & LOCK_PRDT_PAGE) {
+ /* Duplicate the lock to new page */
+ trx_mutex_enter(lock->trx);
+ lock_prdt_add_to_queue(lock->type_mode,
+ new_block,
+ lock->index,
+ lock->trx, NULL, TRUE);
+
+ trx_mutex_exit(lock->trx);
+ continue;
+ }
+
+ /* Now dealing with Predicate Lock */
+ lock_prdt_t* lock_prdt;
+ ulint op = PAGE_CUR_DISJOINT;
+
+ ut_ad(lock->type_mode & LOCK_PREDICATE);
+
+ /* No need to duplicate waiting X locks */
+ if ((lock->type_mode & LOCK_MODE_MASK) == LOCK_X) {
+ continue;
+ }
+
+ lock_prdt = lock_get_prdt_from_lock(lock);
+
+ if (lock_prdt_consistent(lock_prdt, prdt, op)) {
+
+ if (!lock_prdt_consistent(lock_prdt, new_prdt, op)) {
+ /* Move the lock to new page */
+ trx_mutex_enter(lock->trx);
+ lock_prdt_add_to_queue(lock->type_mode,
+ new_block,
+ lock->index,
+ lock->trx, lock_prdt,
+ TRUE);
+ trx_mutex_exit(lock->trx);
+ }
+ } else if (!lock_prdt_consistent(lock_prdt, new_prdt, op)) {
+ /* Duplicate the lock to new page */
+ trx_mutex_enter(lock->trx);
+ lock_prdt_add_to_queue(lock->type_mode,
+ new_block,
+ lock->index,
+ lock->trx, lock_prdt, TRUE);
+
+ trx_mutex_exit(lock->trx);
+ }
+ }
+}
+
+/**************************************************************//**
+Update predicate lock when page splits */
+void
+lock_prdt_update_split(
+/*===================*/
+ buf_block_t* new_block, /*!< in/out: the new half page */
+ lock_prdt_t* prdt, /*!< in: MBR on the old page */
+ lock_prdt_t* new_prdt, /*!< in: MBR on the new page */
+ const page_id_t page_id) /*!< in: page number */
+{
+ lock_mutex_enter();
+
+ lock_prdt_update_split_low(new_block, prdt, new_prdt,
+ page_id, LOCK_PREDICATE);
+
+ lock_prdt_update_split_low(new_block, NULL, NULL,
+ page_id, LOCK_PRDT_PAGE);
+
+ lock_mutex_exit();
+}
+
+/*********************************************************************//**
+Initiate a Predicate Lock from a MBR */
+void
+lock_init_prdt_from_mbr(
+/*====================*/
+ lock_prdt_t* prdt, /*!< in/out: predicate to initialized */
+ rtr_mbr_t* mbr, /*!< in: Minimum Bounding Rectangle */
+ ulint mode, /*!< in: Search mode */
+ mem_heap_t* heap) /*!< in: heap for allocating memory */
+{
+ memset(prdt, 0, sizeof(*prdt));
+
+ if (heap != NULL) {
+ prdt->data = mem_heap_alloc(heap, sizeof(*mbr));
+ memcpy(prdt->data, mbr, sizeof(*mbr));
+ } else {
+ prdt->data = static_cast<void*>(mbr);
+ }
+
+ prdt->op = static_cast<uint16>(mode);
+}
+
+/*********************************************************************//**
+Acquire a predicate lock on a block
+@return DB_SUCCESS, DB_LOCK_WAIT, or DB_DEADLOCK */
+dberr_t
+lock_prdt_lock(
+/*===========*/
+ buf_block_t* block, /*!< in/out: buffer block of rec */
+ lock_prdt_t* prdt, /*!< in: Predicate for the lock */
+ dict_index_t* index, /*!< in: secondary index */
+ lock_mode mode, /*!< in: mode of the lock which
+ the read cursor should set on
+ records: LOCK_S or LOCK_X; the
+ latter is possible in
+ SELECT FOR UPDATE */
+ unsigned type_mode,
+ /*!< in: LOCK_PREDICATE or LOCK_PRDT_PAGE */
+ que_thr_t* thr) /*!< in: query thread
+ (can be NULL if BTR_NO_LOCKING_FLAG) */
+{
+ trx_t* trx = thr_get_trx(thr);
+ dberr_t err = DB_SUCCESS;
+ lock_rec_req_status status = LOCK_REC_SUCCESS;
+
+ if (trx->read_only || index->table->is_temporary()) {
+ return(DB_SUCCESS);
+ }
+
+ ut_ad(!dict_index_is_clust(index));
+ 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;
+
+ /* Another transaction cannot have an implicit lock on the record,
+ because when we come here, we already have modified the clustered
+ index record, and this would not have been possible if another active
+ transaction had modified this secondary index record. */
+
+ lock_mutex_enter();
+
+ const unsigned prdt_mode = type_mode | mode;
+ lock_t* lock = lock_sys.get_first(hash, block->page.id());
+
+ if (lock == NULL) {
+ lock = lock_rec_create(
+#ifdef WITH_WSREP
+ NULL, NULL, /* FIXME: replicate SPATIAL INDEX locks */
+#endif
+ prdt_mode, block, PRDT_HEAPNO,
+ index, trx, FALSE);
+
+ status = LOCK_REC_SUCCESS_CREATED;
+ } else {
+ trx_mutex_enter(trx);
+
+ if (lock_rec_get_next_on_page(lock)
+ || lock->trx != trx
+ || lock->type_mode != (LOCK_REC | prdt_mode)
+ || lock_rec_get_n_bits(lock) == 0
+ || ((type_mode & LOCK_PREDICATE)
+ && (!lock_prdt_consistent(
+ lock_get_prdt_from_lock(lock), prdt, 0)))) {
+
+ lock = lock_prdt_has_lock(
+ mode, type_mode, block, prdt, trx);
+
+ if (lock == NULL) {
+
+ lock_t* wait_for;
+
+ wait_for = lock_prdt_other_has_conflicting(
+ prdt_mode, block, prdt, trx);
+
+ if (wait_for != NULL) {
+
+ err = lock_rec_enqueue_waiting(
+#ifdef WITH_WSREP
+ NULL, /* FIXME: replicate
+ SPATIAL INDEX locks */
+#endif
+ prdt_mode,
+ block, PRDT_HEAPNO,
+ index, thr, prdt);
+ } else {
+
+ lock_prdt_add_to_queue(
+ prdt_mode, block, index, trx,
+ prdt, true);
+
+ status = LOCK_REC_SUCCESS;
+ }
+ }
+
+ trx_mutex_exit(trx);
+
+ } else {
+ trx_mutex_exit(trx);
+
+ if (!lock_rec_get_nth_bit(lock, PRDT_HEAPNO)) {
+ lock_rec_set_nth_bit(lock, PRDT_HEAPNO);
+ status = LOCK_REC_SUCCESS_CREATED;
+ }
+ }
+ }
+
+ lock_mutex_exit();
+
+ if (status == LOCK_REC_SUCCESS_CREATED && type_mode == LOCK_PREDICATE) {
+ /* Append the predicate in the lock record */
+ lock_prdt_set_prdt(lock, prdt);
+ }
+
+ return(err);
+}
+
+/*********************************************************************//**
+Acquire a "Page" lock on a block
+@return DB_SUCCESS, DB_LOCK_WAIT, or DB_DEADLOCK */
+dberr_t
+lock_place_prdt_page_lock(
+ const page_id_t page_id, /*!< in: page identifier */
+ dict_index_t* index, /*!< in: secondary index */
+ que_thr_t* thr) /*!< in: query thread */
+{
+ ut_ad(thr != NULL);
+ ut_ad(!srv_read_only_mode);
+
+ ut_ad(!dict_index_is_clust(index));
+ ut_ad(!dict_index_is_online_ddl(index));
+
+ /* Another transaction cannot have an implicit lock on the record,
+ because when we come here, we already have modified the clustered
+ index record, and this would not have been possible if another active
+ transaction had modified this secondary index record. */
+
+ lock_mutex_enter();
+
+ const lock_t* lock = lock_sys.get_first_prdt_page(page_id);
+ const ulint mode = LOCK_S | LOCK_PRDT_PAGE;
+ trx_t* trx = thr_get_trx(thr);
+
+ if (lock != NULL) {
+
+ trx_mutex_enter(trx);
+
+ /* Find a matching record lock owned by this transaction. */
+
+ while (lock != NULL && lock->trx != trx) {
+
+ lock = lock_rec_get_next_on_page_const(lock);
+ }
+
+ ut_ad(lock == NULL || lock->type_mode == (mode | LOCK_REC));
+ ut_ad(lock == NULL || lock_rec_get_n_bits(lock) != 0);
+
+ trx_mutex_exit(trx);
+ }
+
+ if (lock == NULL) {
+ lock = lock_rec_create_low(
+#ifdef WITH_WSREP
+ NULL, NULL, /* FIXME: replicate SPATIAL INDEX locks */
+#endif
+ mode, page_id, NULL, PRDT_HEAPNO,
+ index, trx, FALSE);
+
+#ifdef PRDT_DIAG
+ printf("GIS_DIAGNOSTIC: page lock %d\n", (int) page_no);
+#endif /* PRDT_DIAG */
+ }
+
+ lock_mutex_exit();
+
+ return(DB_SUCCESS);
+}
+
+/** Check whether there are R-tree Page lock on a page
+@param[in] trx trx to test the lock
+@param[in] page_id page identifier
+@return true if there is none */
+bool lock_test_prdt_page_lock(const trx_t *trx, const page_id_t page_id)
+{
+ lock_t* lock;
+
+ lock_mutex_enter();
+
+ lock = lock_sys.get_first_prdt_page(page_id);
+
+ lock_mutex_exit();
+
+ return(!lock || trx == lock->trx);
+}
+
+/*************************************************************//**
+Moves the locks of a page to another page and resets the lock bits of
+the donating records. */
+void
+lock_prdt_rec_move(
+/*===============*/
+ const buf_block_t* receiver, /*!< in: buffer block containing
+ the receiving record */
+ const buf_block_t* donator) /*!< in: buffer block containing
+ the donating record */
+{
+ lock_mutex_enter();
+
+ for (lock_t *lock = lock_rec_get_first(&lock_sys.prdt_hash,
+ donator, PRDT_HEAPNO);
+ lock != NULL;
+ lock = lock_rec_get_next(PRDT_HEAPNO, lock)) {
+
+ const auto type_mode = lock->type_mode;
+ lock_prdt_t* lock_prdt = lock_get_prdt_from_lock(lock);
+
+ lock_rec_reset_nth_bit(lock, PRDT_HEAPNO);
+ lock_reset_lock_and_trx_wait(lock);
+
+ lock_prdt_add_to_queue(
+ type_mode, receiver, lock->index, lock->trx,
+ lock_prdt, FALSE);
+ }
+
+ lock_mutex_exit();
+}
+
+/** Removes predicate lock objects set on an index page which is discarded.
+@param[in] block page to be discarded
+@param[in] lock_hash lock hash */
+void
+lock_prdt_page_free_from_discard(
+ const buf_block_t* block,
+ hash_table_t* lock_hash)
+{
+ lock_t* lock;
+ lock_t* next_lock;
+
+ ut_ad(lock_mutex_own());
+
+ lock = lock_sys.get_first(*lock_hash, block->page.id());
+
+ while (lock != NULL) {
+ next_lock = lock_rec_get_next_on_page(lock);
+
+ lock_rec_discard(lock);
+
+ lock = next_lock;
+ }
+}
diff --git a/storage/innobase/lock/lock0wait.cc b/storage/innobase/lock/lock0wait.cc
new file mode 100644
index 00000000..e5f71e0b
--- /dev/null
+++ b/storage/innobase/lock/lock0wait.cc
@@ -0,0 +1,515 @@
+/*****************************************************************************
+
+Copyright (c) 1996, 2016, Oracle and/or its affiliates. All Rights Reserved.
+Copyright (c) 2014, 2020, MariaDB Corporation.
+
+This program is free software; you can redistribute it and/or modify it under
+the terms of the GNU General Public License as published by the Free Software
+Foundation; version 2 of the License.
+
+This program is distributed in the hope that it will be useful, but WITHOUT
+ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or FITNESS
+FOR A PARTICULAR PURPOSE. See the GNU General Public License for more details.
+
+You should have received a copy of the GNU General Public License along with
+this program; if not, write to the Free Software Foundation, Inc.,
+51 Franklin Street, Fifth Floor, Boston, MA 02110-1335 USA
+
+*****************************************************************************/
+
+/**************************************************//**
+@file lock/lock0wait.cc
+The transaction lock system
+
+Created 25/5/2010 Sunny Bains
+*******************************************************/
+
+#define LOCK_MODULE_IMPLEMENTATION
+
+#include "univ.i"
+#include <mysql/service_thd_wait.h>
+#include <mysql/service_wsrep.h>
+
+#include "srv0mon.h"
+#include "que0que.h"
+#include "lock0lock.h"
+#include "row0mysql.h"
+#include "srv0start.h"
+#include "lock0priv.h"
+#include "srv0srv.h"
+
+/*********************************************************************//**
+Print the contents of the lock_sys_t::waiting_threads array. */
+static
+void
+lock_wait_table_print(void)
+/*=======================*/
+{
+ ut_ad(lock_wait_mutex_own());
+
+ const srv_slot_t* slot = lock_sys.waiting_threads;
+
+ for (ulint i = 0; i < srv_max_n_threads; i++, ++slot) {
+
+ fprintf(stderr,
+ "Slot %lu:"
+ " in use %lu, timeout %lu, time %lu\n",
+ (ulong) i,
+ (ulong) slot->in_use,
+ slot->wait_timeout,
+ (ulong) difftime(time(NULL), slot->suspend_time));
+ }
+}
+
+/*********************************************************************//**
+Release a slot in the lock_sys_t::waiting_threads. Adjust the array last pointer
+if there are empty slots towards the end of the table. */
+static
+void
+lock_wait_table_release_slot(
+/*=========================*/
+ srv_slot_t* slot) /*!< in: slot to release */
+{
+#ifdef UNIV_DEBUG
+ srv_slot_t* upper = lock_sys.waiting_threads + srv_max_n_threads;
+#endif /* UNIV_DEBUG */
+
+ lock_wait_mutex_enter();
+
+ ut_ad(slot->in_use);
+ ut_ad(slot->thr != NULL);
+ ut_ad(slot->thr->slot != NULL);
+ ut_ad(slot->thr->slot == slot);
+
+ /* Must be within the array boundaries. */
+ ut_ad(slot >= lock_sys.waiting_threads);
+ ut_ad(slot < upper);
+
+ /* Note: When we reserve the slot we use the trx_t::mutex to update
+ the slot values to change the state to reserved. Here we are using the
+ lock mutex to change the state of the slot to free. This is by design,
+ because when we query the slot state we always hold both the lock and
+ trx_t::mutex. To reduce contention on the lock mutex when reserving the
+ slot we avoid acquiring the lock mutex. */
+
+ lock_mutex_enter();
+
+ slot->thr->slot = NULL;
+ slot->thr = NULL;
+ slot->in_use = FALSE;
+
+ lock_mutex_exit();
+
+ /* Scan backwards and adjust the last free slot pointer. */
+ for (slot = lock_sys.last_slot;
+ slot > lock_sys.waiting_threads && !slot->in_use;
+ --slot) {
+ /* No op */
+ }
+
+ /* Either the array is empty or the last scanned slot is in use. */
+ ut_ad(slot->in_use || slot == lock_sys.waiting_threads);
+
+ lock_sys.last_slot = slot + 1;
+
+ /* The last slot is either outside of the array boundary or it's
+ on an empty slot. */
+ ut_ad(lock_sys.last_slot == upper || !lock_sys.last_slot->in_use);
+
+ ut_ad(lock_sys.last_slot >= lock_sys.waiting_threads);
+ ut_ad(lock_sys.last_slot <= upper);
+
+ lock_wait_mutex_exit();
+}
+
+/*********************************************************************//**
+Reserves a slot in the thread table for the current user OS thread.
+@return reserved slot */
+static
+srv_slot_t*
+lock_wait_table_reserve_slot(
+/*=========================*/
+ que_thr_t* thr, /*!< in: query thread associated
+ with the user OS thread */
+ ulong wait_timeout) /*!< in: lock wait timeout value */
+{
+ ulint i;
+ srv_slot_t* slot;
+
+ ut_ad(lock_wait_mutex_own());
+ ut_ad(trx_mutex_own(thr_get_trx(thr)));
+
+ slot = lock_sys.waiting_threads;
+
+ for (i = srv_max_n_threads; i--; ++slot) {
+ if (!slot->in_use) {
+ slot->in_use = TRUE;
+ slot->thr = thr;
+ slot->thr->slot = slot;
+
+ if (slot->event == NULL) {
+ slot->event = os_event_create(0);
+ ut_a(slot->event);
+ }
+
+ os_event_reset(slot->event);
+ slot->suspend_time = time(NULL);
+ slot->wait_timeout = wait_timeout;
+
+ if (slot == lock_sys.last_slot) {
+ ++lock_sys.last_slot;
+ }
+
+ ut_ad(lock_sys.last_slot
+ <= lock_sys.waiting_threads + srv_max_n_threads);
+ if (!lock_sys.timeout_timer_active) {
+ lock_sys.timeout_timer_active = true;
+ lock_sys.timeout_timer->set_time(1000, 0);
+ }
+ return(slot);
+ }
+ }
+
+ ib::error() << "There appear to be " << srv_max_n_threads << " user"
+ " threads currently waiting inside InnoDB, which is the upper"
+ " limit. Cannot continue operation. Before aborting, we print"
+ " a list of waiting threads.";
+ lock_wait_table_print();
+
+ ut_error;
+ return(NULL);
+}
+
+#ifdef WITH_WSREP
+/*********************************************************************//**
+check if lock timeout was for priority thread,
+as a side effect trigger lock monitor
+@param[in] trx transaction owning the lock
+@param[in] locked true if trx and lock_sys.mutex is ownd
+@return false for regular lock timeout */
+static
+bool
+wsrep_is_BF_lock_timeout(
+ const trx_t* trx,
+ bool locked = true)
+{
+ bool long_wait= (trx->error_state != DB_DEADLOCK &&
+ srv_monitor_timer && trx->is_wsrep() &&
+ wsrep_thd_is_BF(trx->mysql_thd, false));
+ bool was_wait= true;
+
+ DBUG_EXECUTE_IF("wsrep_instrument_BF_lock_wait",
+ was_wait=false; long_wait=true;);
+
+ if (long_wait) {
+ ib::info() << "WSREP: BF lock wait long for trx:" << trx->id
+ << " query: " << wsrep_thd_query(trx->mysql_thd);
+
+ if (!locked)
+ lock_mutex_enter();
+
+ ut_ad(lock_mutex_own());
+
+ trx_print_latched(stderr, trx, 3000);
+ /* Note this will release lock_sys mutex */
+ lock_print_info_all_transactions(stderr);
+
+ if (locked)
+ lock_mutex_enter();
+
+ return was_wait;
+ } else
+ return false;
+}
+#endif /* WITH_WSREP */
+
+/***************************************************************//**
+Puts a user OS thread to wait for a lock to be released. If an error
+occurs during the wait trx->error_state associated with thr is
+!= DB_SUCCESS when we return. DB_LOCK_WAIT_TIMEOUT and DB_DEADLOCK
+are possible errors. DB_DEADLOCK is returned if selective deadlock
+resolution chose this transaction as a victim. */
+void
+lock_wait_suspend_thread(
+/*=====================*/
+ que_thr_t* thr) /*!< in: query thread associated with the
+ user OS thread */
+{
+ srv_slot_t* slot;
+ trx_t* trx;
+ ulong lock_wait_timeout;
+
+ ut_a(lock_sys.timeout_timer.get());
+ trx = thr_get_trx(thr);
+
+ if (trx->mysql_thd != 0) {
+ DEBUG_SYNC_C("lock_wait_suspend_thread_enter");
+ }
+
+ /* InnoDB system transactions (such as the purge, and
+ incomplete transactions that are being rolled back after crash
+ recovery) will use the global value of
+ innodb_lock_wait_timeout, because trx->mysql_thd == NULL. */
+ lock_wait_timeout = trx_lock_wait_timeout_get(trx);
+
+ lock_wait_mutex_enter();
+
+ trx_mutex_enter(trx);
+
+ trx->error_state = DB_SUCCESS;
+
+ if (thr->state == QUE_THR_RUNNING) {
+
+ ut_ad(thr->is_active);
+
+ /* The lock has already been released or this transaction
+ was chosen as a deadlock victim: no need to suspend */
+
+ if (trx->lock.was_chosen_as_deadlock_victim) {
+
+ trx->error_state = DB_DEADLOCK;
+ trx->lock.was_chosen_as_deadlock_victim = false;
+ }
+
+ lock_wait_mutex_exit();
+ trx_mutex_exit(trx);
+ return;
+ }
+
+ ut_ad(!thr->is_active);
+
+ slot = lock_wait_table_reserve_slot(thr, lock_wait_timeout);
+
+ lock_wait_mutex_exit();
+ trx_mutex_exit(trx);
+
+ ulonglong start_time = 0;
+
+ if (thr->lock_state == QUE_THR_LOCK_ROW) {
+ srv_stats.n_lock_wait_count.inc();
+ srv_stats.n_lock_wait_current_count++;
+ start_time = my_interval_timer();
+ }
+
+ ulint lock_type = ULINT_UNDEFINED;
+
+ /* The wait_lock can be cleared by another thread when the
+ lock is released. But the wait can only be initiated by the
+ current thread which owns the transaction. Only acquire the
+ mutex if the wait_lock is still active. */
+ if (const lock_t* wait_lock = trx->lock.wait_lock) {
+ lock_mutex_enter();
+ wait_lock = trx->lock.wait_lock;
+ if (wait_lock) {
+ lock_type = lock_get_type_low(wait_lock);
+ }
+ lock_mutex_exit();
+ }
+
+ ulint had_dict_lock = trx->dict_operation_lock_mode;
+
+ switch (had_dict_lock) {
+ case 0:
+ break;
+ case RW_S_LATCH:
+ /* Release foreign key check latch */
+ row_mysql_unfreeze_data_dictionary(trx);
+
+ DEBUG_SYNC_C("lock_wait_release_s_latch_before_sleep");
+ break;
+ default:
+ /* There should never be a lock wait when the
+ dictionary latch is reserved in X mode. Dictionary
+ transactions should only acquire locks on dictionary
+ tables, not other tables. All access to dictionary
+ tables should be covered by dictionary
+ transactions. */
+ ut_error;
+ }
+
+ ut_a(trx->dict_operation_lock_mode == 0);
+
+ /* Suspend this thread and wait for the event. */
+
+ /* Unknown is also treated like a record lock */
+ if (lock_type == ULINT_UNDEFINED || lock_type == LOCK_REC) {
+ thd_wait_begin(trx->mysql_thd, THD_WAIT_ROW_LOCK);
+ } else {
+ ut_ad(lock_type == LOCK_TABLE);
+ thd_wait_begin(trx->mysql_thd, THD_WAIT_TABLE_LOCK);
+ }
+
+ os_event_wait(slot->event);
+
+ thd_wait_end(trx->mysql_thd);
+
+ /* After resuming, reacquire the data dictionary latch if
+ necessary. */
+
+ if (had_dict_lock) {
+
+ row_mysql_freeze_data_dictionary(trx);
+ }
+
+ double wait_time = difftime(time(NULL), slot->suspend_time);
+
+ /* Release the slot for others to use */
+
+ lock_wait_table_release_slot(slot);
+
+ if (thr->lock_state == QUE_THR_LOCK_ROW) {
+ const ulonglong finish_time = my_interval_timer();
+
+ if (finish_time >= start_time) {
+ const ulint diff_time = static_cast<ulint>
+ ((finish_time - start_time) / 1000);
+ srv_stats.n_lock_wait_time.add(diff_time);
+ /* Only update the variable if we successfully
+ retrieved the start and finish times. See Bug#36819. */
+ if (diff_time > lock_sys.n_lock_max_wait_time) {
+ lock_sys.n_lock_max_wait_time = diff_time;
+ }
+ /* Record the lock wait time for this thread */
+ thd_storage_lock_wait(trx->mysql_thd, diff_time);
+ }
+
+ srv_stats.n_lock_wait_current_count--;
+
+ DBUG_EXECUTE_IF("lock_instrument_slow_query_log",
+ os_thread_sleep(1000););
+ }
+
+ /* The transaction is chosen as deadlock victim during sleep. */
+ if (trx->error_state == DB_DEADLOCK) {
+ return;
+ }
+
+ if (lock_wait_timeout < 100000000
+ && wait_time > (double) lock_wait_timeout
+#ifdef WITH_WSREP
+ && (!trx->is_wsrep()
+ || (!wsrep_is_BF_lock_timeout(trx, false)
+ && trx->error_state != DB_DEADLOCK))
+#endif /* WITH_WSREP */
+ ) {
+
+ trx->error_state = DB_LOCK_WAIT_TIMEOUT;
+
+ MONITOR_INC(MONITOR_TIMEOUT);
+ }
+
+ if (trx_is_interrupted(trx)) {
+
+ trx->error_state = DB_INTERRUPTED;
+ }
+}
+
+/********************************************************************//**
+Releases a user OS thread waiting for a lock to be released, if the
+thread is already suspended. */
+void
+lock_wait_release_thread_if_suspended(
+/*==================================*/
+ que_thr_t* thr) /*!< in: query thread associated with the
+ user OS thread */
+{
+ ut_ad(lock_mutex_own());
+ ut_ad(trx_mutex_own(thr_get_trx(thr)));
+
+ /* We own both the lock mutex and the trx_t::mutex but not the
+ lock wait mutex. This is OK because other threads will see the state
+ of this slot as being in use and no other thread can change the state
+ of the slot to free unless that thread also owns the lock mutex. */
+
+ if (thr->slot != NULL && thr->slot->in_use && thr->slot->thr == thr) {
+ trx_t* trx = thr_get_trx(thr);
+
+ if (trx->lock.was_chosen_as_deadlock_victim) {
+
+ trx->error_state = DB_DEADLOCK;
+ trx->lock.was_chosen_as_deadlock_victim = false;
+ }
+
+ os_event_set(thr->slot->event);
+ }
+}
+
+/*********************************************************************//**
+Check if the thread lock wait has timed out. Release its locks if the
+wait has actually timed out. */
+static
+void
+lock_wait_check_and_cancel(
+/*=======================*/
+ const srv_slot_t* slot) /*!< in: slot reserved by a user
+ thread when the wait started */
+{
+ ut_ad(lock_wait_mutex_own());
+ ut_ad(slot->in_use);
+
+ double wait_time = difftime(time(NULL), slot->suspend_time);
+ trx_t* trx = thr_get_trx(slot->thr);
+
+ if (trx_is_interrupted(trx)
+ || (slot->wait_timeout < 100000000
+ && (wait_time > (double) slot->wait_timeout
+ || wait_time < 0))) {
+
+ /* Timeout exceeded or a wrap-around in system
+ time counter: cancel the lock request queued
+ by the transaction and release possible
+ other transactions waiting behind; it is
+ possible that the lock has already been
+ granted: in that case do nothing */
+
+ lock_mutex_enter();
+
+ trx_mutex_enter(trx);
+
+ if (trx->lock.wait_lock != NULL) {
+
+ ut_a(trx->lock.que_state == TRX_QUE_LOCK_WAIT);
+
+#ifdef WITH_WSREP
+ if (!wsrep_is_BF_lock_timeout(trx)) {
+#endif /* WITH_WSREP */
+ lock_cancel_waiting_and_release(trx->lock.wait_lock);
+#ifdef WITH_WSREP
+ }
+#endif /* WITH_WSREP */
+ }
+
+ lock_mutex_exit();
+
+ trx_mutex_exit(trx);
+ }
+}
+
+/** A task which wakes up threads whose lock wait may have lasted too long */
+void lock_wait_timeout_task(void*)
+{
+ lock_wait_mutex_enter();
+
+ /* Check all slots for user threads that are waiting
+ on locks, and if they have exceeded the time limit. */
+ bool any_slot_in_use= false;
+ for (srv_slot_t *slot= lock_sys.waiting_threads;
+ slot < lock_sys.last_slot; ++slot)
+ {
+ /* We are doing a read without the lock mutex and/or the trx
+ mutex. This is OK because a slot can't be freed or reserved
+ without the lock wait mutex. */
+ if (slot->in_use)
+ {
+ any_slot_in_use= true;
+ lock_wait_check_and_cancel(slot);
+ }
+ }
+
+ if (any_slot_in_use)
+ lock_sys.timeout_timer->set_time(1000, 0);
+ else
+ lock_sys.timeout_timer_active= false;
+
+ lock_wait_mutex_exit();
+}