summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorMarko Mäkelä <marko.makela@mariadb.com>2021-01-26 16:39:56 +0200
committerMarko Mäkelä <marko.makela@mariadb.com>2021-01-27 15:45:39 +0200
commite71e6133535da8d5eab86e504f0b116a03680780 (patch)
tree160adf5e0ded57571fa18e376534953f2602d1e8
parent7f1ab8f742ac42e82dd39a6e11390254cb72319c (diff)
downloadmariadb-git-e71e6133535da8d5eab86e504f0b116a03680780.tar.gz
MDEV-24671: Replace lock_wait_timeout_task with mysql_cond_timedwait()
lock_wait(): Replaces lock_wait_suspend_thread(). Wait for the lock to be granted or the transaction to be killed using mysql_cond_timedwait() or mysql_cond_wait(). lock_wait_end(): Replaces que_thr_end_lock_wait() and lock_wait_release_thread_if_suspended(). lock_wait_timeout_task: Remove. The operating system kernel will resume the mysql_cond_timedwait() in lock_wait(). An added benefit is that innodb_lock_wait_timeout no longer has a 'jitter' of 1 second, which was caused by this wake-up task waking up only once per second, and then waking up any threads for which the timeout (which was only measured in seconds) was exceeded. innobase_kill_query(): Set trx->error_state=DB_INTERRUPTED, so that a call trx_is_interrupted(trx) in lock_wait() can be avoided. We will protect things more consistently with lock_sys.wait_mutex, which will be moved below lock_sys.mutex in the latching order. trx_lock_t::cond: Condition variable for !wait_lock, used with lock_sys.wait_mutex. srv_slot_t: Remove. Replaced by trx_lock_t::cond, lock_grant_after_reset(): Merged to to lock_grant(). lock_rec_get_index_name(): Remove. lock_sys_t: Introduce wait_pending, wait_count, wait_time, wait_time_max that are protected by wait_mutex. trx_lock_t::que_state: Remove. que_thr_state_t: Remove QUE_THR_COMMAND_WAIT, QUE_THR_LOCK_WAIT. que_thr_t: Remove is_active, start_running(), stop_no_error(). que_fork_t::n_active_thrs, trx_lock_t::n_active_thrs: Remove.
-rw-r--r--storage/innobase/buf/buf0buf.cc30
-rw-r--r--storage/innobase/handler/ha_innodb.cc29
-rw-r--r--storage/innobase/handler/handler0alter.cc2
-rw-r--r--storage/innobase/include/lock0lock.h104
-rw-r--r--storage/innobase/include/lock0priv.h26
-rw-r--r--storage/innobase/include/que0que.h79
-rw-r--r--storage/innobase/include/que0que.ic29
-rw-r--r--storage/innobase/include/srv0srv.h34
-rw-r--r--storage/innobase/include/trx0trx.h55
-rw-r--r--storage/innobase/include/trx0types.h8
-rw-r--r--storage/innobase/lock/lock0lock.cc491
-rw-r--r--storage/innobase/lock/lock0prdt.cc6
-rw-r--r--storage/innobase/lock/lock0wait.cc525
-rw-r--r--storage/innobase/log/log0log.cc4
-rw-r--r--storage/innobase/que/que0que.cc321
-rw-r--r--storage/innobase/row/row0ins.cc9
-rw-r--r--storage/innobase/row/row0mysql.cc161
-rw-r--r--storage/innobase/row/row0sel.cc20
-rw-r--r--storage/innobase/srv/srv0mon.cc20
-rw-r--r--storage/innobase/srv/srv0srv.cc28
-rw-r--r--storage/innobase/srv/srv0start.cc8
-rw-r--r--storage/innobase/trx/trx0i_s.cc53
-rw-r--r--storage/innobase/trx/trx0purge.cc2
-rw-r--r--storage/innobase/trx/trx0roll.cc6
-rw-r--r--storage/innobase/trx/trx0trx.cc40
25 files changed, 582 insertions, 1508 deletions
diff --git a/storage/innobase/buf/buf0buf.cc b/storage/innobase/buf/buf0buf.cc
index 8f7f2ff0e9b..fc9816d02a1 100644
--- a/storage/innobase/buf/buf0buf.cc
+++ b/storage/innobase/buf/buf0buf.cc
@@ -1921,7 +1921,7 @@ struct find_interesting_trx
return;
if (trx.mysql_thd == nullptr)
return;
- if (withdraw_started <= trx.start_time)
+ if (withdraw_started <= trx.start_time_micro)
return;
if (!found)
@@ -1939,8 +1939,9 @@ struct find_interesting_trx
}
bool &found;
- time_t withdraw_started;
- time_t current_time;
+ /** microsecond_interval_timer() */
+ const ulonglong withdraw_started;
+ const my_hrtime_t current_time;
};
} // namespace
@@ -2004,8 +2005,8 @@ inline void buf_pool_t::resize()
buf_resize_status("Withdrawing blocks to be shrunken.");
- time_t withdraw_started = time(NULL);
- double message_interval = 60;
+ ulonglong withdraw_started = microsecond_interval_timer();
+ ulonglong message_interval = 60ULL * 1000 * 1000;
ulint retry_interval = 1;
withdraw_retry:
@@ -2021,24 +2022,25 @@ withdraw_retry:
/* abort buffer pool load */
buf_load_abort();
- const time_t current_time = time(NULL);
+ const ulonglong current_time = microsecond_interval_timer();
if (should_retry_withdraw
- && difftime(current_time, withdraw_started) >= message_interval) {
+ && current_time - withdraw_started >= message_interval) {
- if (message_interval > 900) {
- message_interval = 1800;
+ if (message_interval > 900000000) {
+ message_interval = 1800000000;
} else {
message_interval *= 2;
}
+ bool found= false;
+ find_interesting_trx f
+ {found, withdraw_started, my_hrtime_coarse()};
+ withdraw_started = current_time;
+
lock_sys.mutex_lock();
- bool found = false;
- trx_sys.trx_list.for_each(find_interesting_trx{
- found, withdraw_started, current_time});
+ trx_sys.trx_list.for_each(f);
lock_sys.mutex_unlock();
-
- withdraw_started = current_time;
}
if (should_retry_withdraw) {
diff --git a/storage/innobase/handler/ha_innodb.cc b/storage/innobase/handler/ha_innodb.cc
index a24b7ed3110..2ef1f77318d 100644
--- a/storage/innobase/handler/ha_innodb.cc
+++ b/storage/innobase/handler/ha_innodb.cc
@@ -4428,8 +4428,6 @@ static int innobase_close_connection(handlerton *hton, THD *thd)
return 0;
}
-void lock_cancel_waiting_and_release(lock_t *lock);
-
/** Cancel any pending lock request associated with the current THD.
@sa THD::awake() @sa ha_kill_query() */
static void innobase_kill_query(handlerton*, THD *thd, enum thd_kill_levels)
@@ -4446,14 +4444,20 @@ static void innobase_kill_query(handlerton*, THD *thd, enum thd_kill_levels)
Also, BF thread should own trx mutex for the victim. */
DBUG_VOID_RETURN;
#endif /* WITH_WSREP */
- lock_sys.mutex_lock();
- if (lock_t *lock= trx->lock.wait_lock)
+ if (trx->lock.wait_lock)
{
- trx->mutex.wr_lock();
- lock_cancel_waiting_and_release(lock);
- trx->mutex.wr_unlock();
+ lock_sys.mutex_lock();
+ mysql_mutex_lock(&lock_sys.wait_mutex);
+ if (lock_t *lock= trx->lock.wait_lock)
+ {
+ trx->mutex.wr_lock();
+ trx->error_state= DB_INTERRUPTED;
+ lock_cancel_waiting_and_release(lock);
+ trx->mutex.wr_unlock();
+ }
+ lock_sys.mutex_unlock();
+ mysql_mutex_unlock(&lock_sys.wait_mutex);
}
- lock_sys.mutex_unlock();
}
DBUG_VOID_RETURN;
@@ -18031,10 +18035,10 @@ int wsrep_innobase_kill_one_trx(THD *bf_thd, trx_t *victim_trx, bool signal)
wsrep_thd_UNLOCK(thd);
DEBUG_SYNC(bf_thd, "before_wsrep_thd_abort");
- if (wsrep_thd_bf_abort(bf_thd, thd, signal))
- {
- lock_t* wait_lock = victim_trx->lock.wait_lock;
- if (wait_lock) {
+ if (!wsrep_thd_bf_abort(bf_thd, thd, signal)) {
+ } else if (victim_trx->lock.wait_lock) {
+ mysql_mutex_lock(&lock_sys.wait_mutex);
+ if (lock_t* wait_lock = victim_trx->lock.wait_lock) {
DBUG_ASSERT(victim_trx->is_wsrep());
WSREP_DEBUG("victim has wait flag: %lu",
thd_get_thread_id(thd));
@@ -18043,6 +18047,7 @@ int wsrep_innobase_kill_one_trx(THD *bf_thd, trx_t *victim_trx, bool signal)
victim_trx->lock.was_chosen_as_deadlock_victim= TRUE;
lock_cancel_waiting_and_release(wait_lock);
}
+ mysql_mutex_unlock(&lock_sys.wait_mutex);
}
DBUG_RETURN(0);
diff --git a/storage/innobase/handler/handler0alter.cc b/storage/innobase/handler/handler0alter.cc
index fcdab5b0836..864d6bfddda 100644
--- a/storage/innobase/handler/handler0alter.cc
+++ b/storage/innobase/handler/handler0alter.cc
@@ -7083,8 +7083,6 @@ op_ok:
row_mysql_unlock_data_dictionary(ctx->trx);
dict_locked = false;
- ut_ad(!ctx->trx->lock.n_active_thrs);
-
if (ctx->old_table->fts) {
fts_sync_during_ddl(ctx->old_table);
}
diff --git a/storage/innobase/include/lock0lock.h b/storage/innobase/include/lock0lock.h
index b6c548d5bad..fb4ec9e7f51 100644
--- a/storage/innobase/include/lock0lock.h
+++ b/storage/innobase/include/lock0lock.h
@@ -500,9 +500,9 @@ lock_print_info_summary(
/** 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);
+@param[in] now current my_hrtime_coarse() */
+void lock_trx_print_wait_and_mvcc_state(FILE *file, const trx_t *trx,
+ my_hrtime_t now);
/*********************************************************************//**
Prints info of locks for each transaction. This function assumes that the
@@ -555,15 +555,6 @@ lock_rec_get_index(
const lock_t* lock); /*!< in: lock */
/*******************************************************************//**
-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 */
-
-/*******************************************************************//**
Check if there are any locks (table or rec) against table.
@return TRUE if locks exist */
bool
@@ -573,29 +564,12 @@ lock_table_has_locks(
held on records in this table or on the
table itself */
-/** A task which wakes up threads whose lock wait may have lasted too long */
-void lock_wait_timeout_task(void*);
-
-/********************************************************************//**
-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 */
-
-/***************************************************************//**
-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 */
+/** Wait for a lock to be released.
+@retval DB_DEADLOCK if this transaction was chosen as the deadlock victim
+@retval DB_INTERRUPTED if the execution was interrupted by the user
+@retval DB_LOCK_WAIT_TIMEOUT if the lock wait timed out
+@retval DB_SUCCESS if the lock was granted */
+dberr_t lock_wait(que_thr_t *thr);
/*********************************************************************//**
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
@@ -684,20 +658,18 @@ public:
/** page locks for SPATIAL INDEX */
hash_table_t prdt_page_hash;
- /** mutex protecting waiting_threads, last_slot */
+ /** mutex covering lock waits; @see trx_lock_t::wait_lock */
MY_ALIGNED(CACHE_LINE_SIZE) mysql_mutex_t wait_mutex;
- srv_slot_t* waiting_threads; /*!< Array of user threads
- suspended while waiting for
- locks within InnoDB */
- srv_slot_t* last_slot; /*!< highest slot ever used
- in the waiting_threads array */
-
- ulint n_lock_max_wait_time; /*!< Max wait time */
-
- std::unique_ptr<tpool::timer> timeout_timer; /*!< Thread pool timer task */
- bool timeout_timer_active;
-
-
+private:
+ /** Pending number of lock waits; protected by wait_mutex */
+ ulint wait_pending;
+ /** Cumulative number of lock waits; protected by wait_mutex */
+ ulint wait_count;
+ /** Cumulative wait time; protected by wait_mutex */
+ ulint wait_time;
+ /** Longest wait time; protected by wait_mutex */
+ ulint wait_time_max;
+public:
/**
Constructor.
@@ -752,6 +724,22 @@ public:
/** Closes the lock system at database shutdown. */
void close();
+
+ /** Note that a record lock wait started */
+ inline void wait_start();
+
+ /** Note that a record lock wait resumed */
+ inline void wait_resume(THD *thd, my_hrtime_t start, my_hrtime_t now);
+
+ /** @return pending number of lock waits */
+ ulint get_wait_pending() const { return wait_pending; }
+ /** @return cumulative number of lock waits */
+ ulint get_wait_cumulative() const { return wait_count; }
+ /** Cumulative wait time; protected by wait_mutex */
+ ulint get_wait_time_cumulative() const { return wait_time; }
+ /** Longest wait time; protected by wait_mutex */
+ ulint get_wait_time_max() const { return wait_time_max; }
+
/** @return the hash value for a page address */
ulint hash(const page_id_t id) const
{ mysql_mutex_assert_owner(&mutex); return rec_hash.calc_hash(id.fold()); }
@@ -904,24 +892,8 @@ lock_rec_free_all_from_discard_page(
/** The lock system */
extern lock_sys_t lock_sys;
-#ifdef WITH_WSREP
-/*********************************************************************//**
-Cancels a waiting lock request and releases possible other transactions
-waiting behind it. */
-UNIV_INTERN
-void
-lock_cancel_waiting_and_release(
-/*============================*/
- lock_t* lock); /*!< in/out: waiting lock request */
-
-/*******************************************************************//**
-Get lock mode and table/index name
-@return string containing lock info */
-std::string
-lock_get_info(
- const lock_t*);
-
-#endif /* WITH_WSREP */
+/** Cancel a waiting lock request and release possibly waiting transactions */
+void lock_cancel_waiting_and_release(lock_t *lock);
#include "lock0lock.ic"
diff --git a/storage/innobase/include/lock0priv.h b/storage/innobase/include/lock0priv.h
index 906a2a2b514..2252189ca47 100644
--- a/storage/innobase/include/lock0priv.h
+++ b/storage/innobase/include/lock0priv.h
@@ -605,32 +605,6 @@ lock_table_has(
const dict_table_t* table, /*!< in: table */
enum lock_mode mode); /*!< in: lock mode */
-/** Set the wait status of a lock.
-@param[in,out] lock lock that will be waited for
-@param[in,out] trx transaction that will wait for the lock */
-inline void lock_set_lock_and_trx_wait(lock_t* lock, trx_t* trx)
-{
- ut_ad(lock);
- ut_ad(lock->trx == trx);
- ut_ad(trx->lock.wait_lock == NULL);
- lock_sys.mutex_assert_locked();
-
- trx->lock.wait_lock = lock;
- lock->type_mode |= LOCK_WAIT;
-}
-
-/** Reset the wait status of a lock.
-@param[in,out] lock lock that was possibly being waited for */
-inline void lock_reset_lock_and_trx_wait(lock_t* lock)
-{
- ut_ad(lock_get_wait(lock));
- lock_sys.mutex_assert_locked();
- ut_ad(lock->trx->lock.wait_lock == NULL
- || lock->trx->lock.wait_lock == lock);
- lock->trx->lock.wait_lock = NULL;
- lock->type_mode &= ~LOCK_WAIT;
-}
-
#include "lock0priv.ic"
#endif /* lock0priv_h */
diff --git a/storage/innobase/include/que0que.h b/storage/innobase/include/que0que.h
index 74a48d13f1f..1a30ea45194 100644
--- a/storage/innobase/include/que0que.h
+++ b/storage/innobase/include/que0que.h
@@ -88,43 +88,14 @@ que_graph_free(
to this graph: if not, then use
que_graph_free_recursive and free the heap
afterwards! */
-/**********************************************************************//**
-Stops a query thread if graph or trx is in a state requiring it. The
-conditions are tested in the order (1) graph, (2) trx. The lock_sys_t::mutex
-has to be reserved.
-@return TRUE if stopped */
-ibool
-que_thr_stop(
-/*=========*/
- que_thr_t* thr); /*!< in: query thread */
/**********************************************************************//**
-A patch for MySQL used to 'stop' a dummy query thread used in MySQL. The
-query thread is stopped and made inactive, except in the case where
-it was put to the lock wait state in lock0lock.cc, but the lock has already
-been granted or the transaction chosen as a victim in deadlock resolution. */
-void
-que_thr_stop_for_mysql(
-/*===================*/
- que_thr_t* thr); /*!< in: query thread */
-/**********************************************************************//**
Run a query thread. Handles lock waits. */
void
que_run_threads(
/*============*/
que_thr_t* thr); /*!< in: query thread */
/**********************************************************************//**
-Moves a suspended query thread to the QUE_THR_RUNNING state and release
-a worker thread to execute it. This function should be used to end
-the wait state of a query thread waiting for a lock or a stored procedure
-completion.
-@return query thread instance of thread to wakeup or NULL */
-que_thr_t*
-que_thr_end_lock_wait(
-/*==================*/
- trx_t* trx); /*!< in: transaction in the
- QUE_THR_LOCK_WAIT state */
-/**********************************************************************//**
Starts execution of a command in a query fork. Picks a query thread which
is not in the QUE_THR_RUNNING state and moves it to that state. If none
can be chosen, a situation which may arise in parallelized fetches, NULL
@@ -228,17 +199,6 @@ ulint
que_node_list_get_len(
/*==================*/
que_node_t* node_list); /*!< in: node list, or NULL */
-/**********************************************************************//**
-Checks if graph, trx, or session is in a state where the query thread should
-be stopped.
-@return TRUE if should be stopped; NOTE that if the peek is made
-without reserving the trx_t::mutex, then another peek with the mutex
-reserved is necessary before deciding the actual stopping */
-UNIV_INLINE
-ibool
-que_thr_peek_stop(
-/*==============*/
- que_thr_t* thr); /*!< in: query thread */
/*********************************************************************//**
Evaluate the given SQL
@return error code or DB_SUCCESS */
@@ -265,13 +225,11 @@ que_fork_scheduler_round_robin(
/** Query thread states */
enum que_thr_state_t {
- QUE_THR_RUNNING,
/** in selects this means that the thread is at the end of its
result set (or start, in case of a scroll cursor); in other
statements, this means the thread has done its task */
QUE_THR_COMPLETED,
- QUE_THR_COMMAND_WAIT,
- QUE_THR_LOCK_WAIT
+ QUE_THR_RUNNING
};
/** Query thread lock states */
@@ -289,7 +247,6 @@ struct que_thr_t{
que_node_t* child; /*!< graph child node */
que_t* graph; /*!< graph where this node belongs */
que_thr_state_t state; /*!< state of the query thread */
- bool is_active; /*!< whether the thread is active */
/*------------------------------*/
/* The following fields are private to the OS thread executing the
query thread, and are not protected by any mutex: */
@@ -303,9 +260,6 @@ struct que_thr_t{
thus far */
ulint lock_state; /*!< lock state of thread (table or
row) */
- struct srv_slot_t*
- slot; /* The thread slot in the wait
- array in srv_sys_t */
/*------------------------------*/
/* The following fields are links for the various lists that
this type can be on. */
@@ -320,39 +274,12 @@ struct que_thr_t{
related delete/updates */
row_prebuilt_t* prebuilt; /*!< prebuilt structure processed by
the query thread */
-
-#ifdef UNIV_DEBUG
- /** Change the 'active' status */
- inline void set_active(bool active);
-#endif
- /** Transition to the QUE_THR_RUNNING state. */
- inline void start_running()
- {
- ut_d(if (!is_active) set_active(true));
- is_active= true;
- state= QUE_THR_RUNNING;
- }
-
- /** Stop query execution when there is no error or lock wait. */
- void stop_no_error()
- {
- ut_ad(is_active);
- ut_d(set_active(false));
- state= QUE_THR_COMPLETED;
- is_active= false;
- }
};
/* Query graph fork node: its fields are protected by the query thread mutex */
struct que_fork_t{
que_common_t common; /*!< type: QUE_NODE_FORK */
que_t* graph; /*!< query graph of this node */
-#ifdef UNIV_DEBUG
- /** For the query graph root, updated in set_active() */
- ulint n_active_thrs;
- /** Change the 'active' status */
- void set_active(bool active);
-#endif
trx_t* trx; /*!< transaction: this is set only in
the root node */
ulint state; /*!< state of the fork node */
@@ -378,10 +305,6 @@ struct que_fork_t{
};
-#ifdef UNIV_DEBUG
-inline void que_thr_t::set_active(bool active) { graph->set_active(active); };
-#endif
-
/* Query fork (or graph) states */
#define QUE_FORK_ACTIVE 1
#define QUE_FORK_COMMAND_WAIT 2
diff --git a/storage/innobase/include/que0que.ic b/storage/innobase/include/que0que.ic
index 084b9d5bf0e..e21cbad3815 100644
--- a/storage/innobase/include/que0que.ic
+++ b/storage/innobase/include/que0que.ic
@@ -243,32 +243,3 @@ que_node_get_parent(
{
return(((que_common_t*) node)->parent);
}
-
-/**********************************************************************//**
-Checks if graph, trx, or session is in a state where the query thread should
-be stopped.
-@return TRUE if should be stopped; NOTE that if the peek is made
-without reserving the trx mutex, then another peek with the mutex
-reserved is necessary before deciding the actual stopping */
-UNIV_INLINE
-ibool
-que_thr_peek_stop(
-/*==============*/
- que_thr_t* thr) /*!< in: query thread */
-{
- trx_t* trx;
- que_t* graph;
-
- graph = thr->graph;
- trx = graph->trx;
-
- if (graph->state != QUE_FORK_ACTIVE
- || trx->lock.que_state == TRX_QUE_LOCK_WAIT
- || (trx->lock.que_state != TRX_QUE_ROLLING_BACK
- && trx->lock.que_state != TRX_QUE_RUNNING)) {
-
- return(TRUE);
- }
-
- return(FALSE);
-}
diff --git a/storage/innobase/include/srv0srv.h b/storage/innobase/include/srv0srv.h
index 3fba1e7468c..504f26f7218 100644
--- a/storage/innobase/include/srv0srv.h
+++ b/storage/innobase/include/srv0srv.h
@@ -146,16 +146,6 @@ struct srv_stats_t
/** Number of data read in total (in bytes) */
ulint_ctr_1_t data_read;
- /** Wait time of database locks */
- int64_ctr_1_t n_lock_wait_time;
-
- /** Number of database lock waits */
- ulint_ctr_1_t n_lock_wait_count;
-
- /** Number of threads currently waiting on database locks */
- MY_ALIGNED(CACHE_LINE_SIZE) Atomic_counter<ulint>
- n_lock_wait_current_count;
-
/** Number of rows read. */
ulint_ctr_64_t n_rows_read;
@@ -855,30 +845,6 @@ struct export_var_t{
int64_t innodb_key_rotation_list_length;
};
-/** Thread slot in the thread table. */
-struct srv_slot_t{
- bool in_use; /*!< true if this slot
- is in use */
- /** time(NULL) when the thread was suspended.
- FIXME: Use my_interval_timer() or similar, to avoid bogus
- timeouts in lock_wait_check_and_cancel() or lock_wait_suspend_thread()
- when the system time is adjusted to the past!
-
- FIXME: This is duplicating trx_lock_t::wait_started,
- which is being used for diagnostic purposes only. */
- time_t suspend_time;
- ulong wait_timeout; /*!< wait time that if exceeded
- the thread will be timed out.
- Initialized by
- lock_wait_table_reserve_slot()
- for lock wait */
- mysql_cond_t cond; /*!< condition variable for
- waking up suspended thread,
- under lock_sys.mutex */
- que_thr_t* thr; /*!< suspended query thread
- (only used for user threads) */
-};
-
extern tpool::thread_pool *srv_thread_pool;
extern std::unique_ptr<tpool::timer> srv_master_timer;
extern std::unique_ptr<tpool::timer> srv_monitor_timer;
diff --git a/storage/innobase/include/trx0trx.h b/storage/innobase/include/trx0trx.h
index 878b611864d..ca327c9a594 100644
--- a/storage/innobase/include/trx0trx.h
+++ b/storage/innobase/include/trx0trx.h
@@ -416,48 +416,19 @@ The transaction must have mysql_thd assigned. */
typedef std::vector<ib_lock_t*, ut_allocator<ib_lock_t*> > lock_list;
-/*******************************************************************//**
-Latching protocol for trx_lock_t::que_state. trx_lock_t::que_state
-captures the state of the query thread during the execution of a query.
-This is different from a transaction state. The query state of a transaction
-can be updated asynchronously by other threads. The other threads can be
-system threads, like the timeout monitor thread or user threads executing
-other queries. Another thing to be mindful of is that there is a delay between
-when a query thread is put into LOCK_WAIT state and before it actually starts
-waiting. Between these two events it is possible that the query thread is
-granted the lock it was waiting for, which implies that the state can be changed
-asynchronously.
-
-All these operations take place within the context of locking. Therefore state
-changes within the locking code must acquire both the lock mutex and the
-trx->mutex when changing trx->lock.que_state to TRX_QUE_LOCK_WAIT or
-trx->lock.wait_lock to non-NULL but when the lock wait ends it is sufficient
-to only acquire the trx->mutex.
-To query the state either of the mutexes is sufficient within the locking
-code and no mutex is required when the query thread is no longer waiting. */
-
/** The locks and state of an active transaction. Protected by
lock_sys.mutex, trx->mutex or both. */
struct trx_lock_t {
-#ifdef UNIV_DEBUG
- /** number of active query threads; at most 1, except for the
- dummy transaction in trx_purge() */
- ulint n_active_thrs;
-#endif
- trx_que_t que_state; /*!< valid when trx->state
- == TRX_STATE_ACTIVE: TRX_QUE_RUNNING,
- TRX_QUE_LOCK_WAIT, ... */
-
- lock_t* wait_lock; /*!< if trx execution state is
- TRX_QUE_LOCK_WAIT, this points to
- the lock request, otherwise this is
- NULL; set to non-NULL when holding
- both trx->mutex and lock_sys.mutex;
- set to NULL when holding
- lock_sys.mutex; readers should
- hold lock_sys.mutex, except when
- they are holding trx->mutex and
- wait_lock==NULL */
+ /** Lock request being waited for.
+ Set to nonnull when holding lock_sys.mutex, lock_sys.wait_mutex and
+ trx->mutex, by the thread that is executing the transaction.
+ Set to nullptr when holding lock_sys.wait_mutex. */
+ Atomic_relaxed<lock_t*> wait_lock;
+ /** condition variable for !wait_lock; used with lock_sys.wait_mutex */
+ mysql_cond_t cond;
+ /** lock wait start time, protected only by lock_sys.wait_mutex */
+ my_hrtime_t suspend_time;
+
ib_uint64_t deadlock_mark; /*!< A mark field that is initialized
to and checked against lock_mark_counter
by lock_deadlock_recursive(). */
@@ -468,11 +439,8 @@ struct trx_lock_t {
transaction as a victim in deadlock
resolution, it sets this to true.
Protected by trx->mutex. */
- time_t wait_started; /*!< lock wait started at this time,
- protected only by lock_sys.mutex */
-
que_thr_t* wait_thr; /*!< query thread belonging to this
- trx that is in QUE_THR_LOCK_WAIT
+ trx that is in waiting
state. For threads suspended in a
lock wait, this is protected by
lock_sys.mutex. Otherwise, this may
@@ -1067,6 +1035,7 @@ public:
#endif
ut_ad(!read_view.is_open());
ut_ad(!lock.wait_thr);
+ ut_ad(!lock.wait_lock);
ut_ad(UT_LIST_GET_LEN(lock.trx_locks) == 0);
ut_ad(lock.table_locks.empty());
ut_ad(!autoinc_locks || ib_vector_is_empty(autoinc_locks));
diff --git a/storage/innobase/include/trx0types.h b/storage/innobase/include/trx0types.h
index 27f7829d919..dbabf7754fb 100644
--- a/storage/innobase/include/trx0types.h
+++ b/storage/innobase/include/trx0types.h
@@ -48,14 +48,6 @@ static const ulint TRX_MAGIC_N = 91118598;
constexpr uint innodb_purge_threads_MAX= 32;
-/** Transaction execution states when trx->state == TRX_STATE_ACTIVE */
-enum trx_que_t {
- TRX_QUE_RUNNING, /*!< transaction is running */
- TRX_QUE_LOCK_WAIT, /*!< transaction is waiting for
- a lock */
- TRX_QUE_ROLLING_BACK /*!< transaction is rolling back */
-};
-
/** Transaction states (trx_t::state) */
enum trx_state_t {
TRX_STATE_NOT_STARTED,
diff --git a/storage/innobase/lock/lock0lock.cc b/storage/innobase/lock/lock0lock.cc
index e0090f52307..f31ca723dff 100644
--- a/storage/innobase/lock/lock0lock.cc
+++ b/storage/innobase/lock/lock0lock.cc
@@ -53,19 +53,6 @@ Created 5/7/1996 Heikki Tuuri
/** 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);
@@ -366,29 +353,23 @@ lock_check_trx_id_sanity(
*/
void lock_sys_t::create(ulint n_cells)
{
- ut_ad(this == &lock_sys);
+ ut_ad(this == &lock_sys);
+ ut_ad(!is_initialised());
- m_initialised= true;
+ m_initialised= true;
- waiting_threads = static_cast<srv_slot_t*>
- (ut_zalloc_nokey(srv_max_n_threads * sizeof *waiting_threads));
- last_slot = waiting_threads;
- for (ulint i = srv_max_n_threads; i--; ) {
- mysql_cond_init(0, &waiting_threads[i].cond, nullptr);
- }
-
- mysql_mutex_init(lock_mutex_key, &mutex, nullptr);
- mysql_mutex_init(lock_wait_mutex_key, &wait_mutex, nullptr);
+ mysql_mutex_init(lock_mutex_key, &mutex, nullptr);
+ mysql_mutex_init(lock_wait_mutex_key, &wait_mutex, nullptr);
- rec_hash.create(n_cells);
- prdt_hash.create(n_cells);
- prdt_page_hash.create(n_cells);
+ 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;
+ if (!srv_read_only_mode)
+ {
+ lock_latest_err_file= os_file_create_tmpfile();
+ ut_a(lock_latest_err_file);
+ }
}
@@ -446,28 +427,25 @@ void lock_sys_t::resize(ulint n_cells)
/** Closes the lock system at database shutdown. */
void lock_sys_t::close()
{
- ut_ad(this == &lock_sys);
-
- if (!m_initialised) return;
+ ut_ad(this == &lock_sys);
- if (lock_latest_err_file != NULL) {
- my_fclose(lock_latest_err_file, MYF(MY_WME));
- lock_latest_err_file = NULL;
- }
+ if (!m_initialised)
+ return;
- rec_hash.free();
- prdt_hash.free();
- prdt_page_hash.free();
+ if (lock_latest_err_file)
+ {
+ my_fclose(lock_latest_err_file, MYF(MY_WME));
+ lock_latest_err_file= nullptr;
+ }
- mysql_mutex_destroy(&mutex);
- mysql_mutex_destroy(&wait_mutex);
+ rec_hash.free();
+ prdt_hash.free();
+ prdt_page_hash.free();
- for (ulint i = srv_max_n_threads; i--; ) {
- mysql_cond_destroy(&waiting_threads[i].cond);
- }
+ mysql_mutex_destroy(&mutex);
+ mysql_mutex_destroy(&wait_mutex);
- ut_free(waiting_threads);
- m_initialised= false;
+ m_initialised= false;
}
/*********************************************************************//**
@@ -956,7 +934,7 @@ wsrep_kill_victim(
(wsrep_thd_order_before(
trx->mysql_thd, lock->trx->mysql_thd))) {
- if (lock->trx->lock.que_state == TRX_QUE_LOCK_WAIT) {
+ if (lock->trx->lock.wait_thr) {
if (UNIV_UNLIKELY(wsrep_debug)) {
ib::info() << "WSREP: BF victim waiting\n";
}
@@ -1149,6 +1127,34 @@ wsrep_print_wait_locks(
}
#endif /* WITH_WSREP */
+/** Reset the wait status of a lock.
+@param[in,out] lock lock that was possibly being waited for */
+static void lock_reset_lock_and_trx_wait(lock_t *lock)
+{
+ lock_sys.mutex_assert_locked();
+ ut_ad(lock->is_waiting());
+ ut_ad(!lock->trx->lock.wait_lock || lock->trx->lock.wait_lock == lock);
+ lock->trx->lock.wait_lock= nullptr;
+ lock->type_mode&= ~LOCK_WAIT;
+}
+
+#ifdef WITH_WSREP
+/** Set the wait status of a lock.
+@param[in,out] lock lock that will be waited for
+@param[in,out] trx transaction that will wait for the lock */
+static void lock_set_lock_and_trx_wait(lock_t *lock, trx_t *trx)
+{
+ ut_ad(lock);
+ ut_ad(lock->trx == trx);
+ ut_ad(!trx->lock.wait_lock || trx->lock.wait_lock != lock);
+ ut_ad(!trx->lock.wait_lock || (*trx->lock.wait_lock).trx == trx);
+ lock_sys.mutex_assert_locked();
+
+ trx->lock.wait_lock= lock;
+ lock->type_mode|= LOCK_WAIT;
+}
+#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
@@ -1267,8 +1273,15 @@ lock_rec_create_low(
* delayed conflict resolution '...kill_one_trx' was not called,
* if victim was waiting for some other lock
*/
+ if (holds_trx_mutex) {
+ trx->mutex.wr_unlock();
+ }
+ mysql_mutex_lock(&lock_sys.wait_mutex);
+ if (holds_trx_mutex) {
+ trx->mutex.wr_lock();
+ }
c_lock->trx->mutex.wr_lock();
- if (c_lock->trx->lock.que_state == TRX_QUE_LOCK_WAIT) {
+ if (c_lock->trx->lock.wait_thr) {
c_lock->trx->lock.was_chosen_as_deadlock_victim = TRUE;
@@ -1276,12 +1289,10 @@ lock_rec_create_low(
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
@@ -1300,8 +1311,10 @@ lock_rec_create_low(
c_lock->trx->mutex.wr_unlock();
/* have to bail out here to avoid lock_set_lock... */
+ mysql_mutex_unlock(&lock_sys.wait_mutex);
return(lock);
}
+ mysql_mutex_unlock(&lock_sys.wait_mutex);
c_lock->trx->mutex.wr_unlock();
} else
#endif /* WITH_WSREP */
@@ -1312,7 +1325,9 @@ lock_rec_create_low(
trx->mutex.wr_lock();
}
if (type_mode & LOCK_WAIT) {
- lock_set_lock_and_trx_wait(lock, trx);
+ ut_ad(!trx->lock.wait_lock
+ || (*trx->lock.wait_lock).trx == trx);
+ trx->lock.wait_lock = lock;
}
UT_LIST_ADD_LAST(trx->lock.trx_locks, lock);
if (!holds_trx_mutex) {
@@ -1360,8 +1375,6 @@ lock_rec_enqueue_waiting(
trx_t* trx = thr_get_trx(thr);
- ut_a(!que_thr_stop(thr));
-
switch (trx_get_dict_operation(trx)) {
case TRX_DICT_OP_NONE:
break;
@@ -1396,6 +1409,10 @@ lock_rec_enqueue_waiting(
if (ut_d(const trx_t* victim =)
DeadlockChecker::check_and_resolve(lock, trx)) {
ut_ad(victim == trx);
+ /* There is no need to hold lock_sys.wait_mutex here,
+ because we are clearing the wait flag on a lock request
+ that is associated with the current transaction. So,
+ this is not conflicting with lock_wait(). */
lock_reset_lock_and_trx_wait(lock);
lock_rec_reset_nth_bit(lock, heap_no);
return DB_DEADLOCK;
@@ -1414,12 +1431,9 @@ lock_rec_enqueue_waiting(
return DB_SUCCESS_LOCKED_REC;
}
- trx->lock.que_state = TRX_QUE_LOCK_WAIT;
+ trx->lock.wait_thr = thr;
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
@@ -1546,7 +1560,7 @@ lock_rec_add_to_queue(
lock != NULL;
lock = lock_rec_get_next_on_page(lock)) {
- if (lock_get_wait(lock)
+ if (lock->is_waiting()
&& lock_rec_get_nth_bit(lock, heap_no)) {
goto create;
@@ -1709,7 +1723,7 @@ lock_rec_has_to_wait_in_queue(
ut_ad(wait_lock);
lock_sys.mutex_assert_locked();
- ut_ad(lock_get_wait(wait_lock));
+ ut_ad(wait_lock->is_waiting());
ut_ad(lock_get_type_low(wait_lock) == LOCK_REC);
heap_no = lock_rec_find_set_bit(wait_lock);
@@ -1733,52 +1747,48 @@ lock_rec_has_to_wait_in_queue(
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)
-{
- lock_sys.mutex_assert_locked();
-
- 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);
- }
- }
+/** Resume a lock wait */
+static void lock_wait_end(trx_t *trx)
+{
+ mysql_mutex_assert_owner(&lock_sys.wait_mutex);
- DBUG_PRINT("ib_lock", ("wait for trx " TRX_ID_FMT " ends",
- lock->trx->id));
+ que_thr_t *thr= trx->lock.wait_thr;
+ ut_ad(thr);
+ if (trx->lock.was_chosen_as_deadlock_victim)
+ {
+ trx->error_state= DB_DEADLOCK;
+ trx->lock.was_chosen_as_deadlock_victim= false;
+ }
+ trx->lock.wait_thr= nullptr;
+ mysql_cond_signal(&trx->lock.cond);
+}
- /* 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 */
+/** Grant a waiting lock request and release the waiting transaction. */
+static void lock_grant(lock_t *lock)
+{
+ lock_sys.mutex_assert_locked();
+ mysql_mutex_assert_owner(&lock_sys.wait_mutex);
+ lock_reset_lock_and_trx_wait(lock);
+ trx_t *trx= lock->trx;
+ trx->mutex.wr_lock();
+ if (lock->mode() == LOCK_AUTO_INC)
+ {
+ dict_table_t *table= lock->un_member.tab_lock.table;
+ ut_ad(!table->autoinc_trx);
+ table->autoinc_trx= trx;
+ ib_vector_push(trx->autoinc_locks, &lock);
+ }
- if (lock->trx->lock.que_state == TRX_QUE_LOCK_WAIT) {
- que_thr_t* thr;
+ DBUG_PRINT("ib_lock", ("wait for trx " TRX_ID_FMT " ends", trx->id));
- thr = que_thr_end_lock_wait(lock->trx);
+ /* If we are resolving a deadlock by choosing another transaction as
+ a victim, then our original transaction may not be waiting anymore */
- if (thr != NULL) {
- lock_wait_release_thread_if_suspended(thr);
- }
- }
-}
+ if (trx->lock.wait_thr)
+ lock_wait_end(trx);
-/** 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);
- auto mutex= &lock->trx->mutex;
- mutex->wr_lock();
- lock_grant_after_reset(lock);
- mutex->wr_unlock();
+ trx->mutex.wr_unlock();
}
/*************************************************************//**
@@ -1799,16 +1809,15 @@ lock_rec_cancel(
/* Reset the wait flag and the back pointer to lock in trx */
+ mysql_mutex_lock(&lock_sys.wait_mutex);
lock_reset_lock_and_trx_wait(lock);
/* The following releases the trx from lock wait */
trx_t *trx = lock->trx;
- auto mutex = &trx->mutex;
- mutex->wr_lock();
- if (que_thr_t* thr = que_thr_end_lock_wait(trx)) {
- lock_wait_release_thread_if_suspended(thr);
- }
- mutex->wr_unlock();
+ trx->mutex.wr_lock();
+ lock_wait_end(trx);
+ mysql_mutex_unlock(&lock_sys.wait_mutex);
+ trx->mutex.wr_unlock();
}
/** Remove a record lock request, waiting or granted, from the queue and
@@ -1820,6 +1829,7 @@ static void lock_rec_dequeue_from_page(lock_t* in_lock)
hash_table_t* lock_hash;
lock_sys.mutex_assert_locked();
+ mysql_mutex_assert_owner(&lock_sys.wait_mutex);
ut_ad(lock_get_type_low(in_lock) == LOCK_REC);
/* We may or may not be holding in_lock->trx->mutex here. */
@@ -2092,12 +2102,12 @@ lock_rec_move_low(
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);
+ const auto type_mode = lock->type_mode;
if (type_mode & LOCK_WAIT) {
- lock_reset_lock_and_trx_wait(lock);
+ ut_ad(lock->trx->lock.wait_lock == lock);
+ lock->type_mode &= ~LOCK_WAIT;
}
/* Note that we FIRST reset the bit, and then set the lock:
@@ -2213,9 +2223,9 @@ lock_move_reorganize_page(
/* Reset bitmap of lock */
lock_rec_bitmap_reset(lock);
- if (lock_get_wait(lock)) {
-
- lock_reset_lock_and_trx_wait(lock);
+ if (lock->is_waiting()) {
+ ut_ad(lock->trx->lock.wait_lock == lock);
+ lock->type_mode &= ~LOCK_WAIT;
}
lock = lock_rec_get_next_on_page(lock);
@@ -2394,7 +2404,8 @@ lock_move_rec_list_end(
ut_ad(!page_rec_is_metadata(orec));
if (type_mode & LOCK_WAIT) {
- lock_reset_lock_and_trx_wait(lock);
+ ut_ad(lock->trx->lock.wait_lock==lock);
+ lock->type_mode &= ~LOCK_WAIT;
}
lock_rec_add_to_queue(
@@ -2496,7 +2507,8 @@ lock_move_rec_list_start(
ut_ad(!page_rec_is_metadata(prev));
if (type_mode & LOCK_WAIT) {
- lock_reset_lock_and_trx_wait(lock);
+ ut_ad(lock->trx->lock.wait_lock==lock);
+ lock->type_mode &= ~LOCK_WAIT;
}
lock_rec_add_to_queue(
@@ -2592,7 +2604,8 @@ lock_rtr_move_rec_list(
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);
+ ut_ad(lock->trx->lock.wait_lock==lock);
+ lock->type_mode &= ~LOCK_WAIT;
}
lock_rec_add_to_queue(
@@ -3098,9 +3111,11 @@ lock_table_create(
ut_list_append(table->locks, lock, TableLockGetNode());
}
+ trx->mutex.wr_unlock();
+ mysql_mutex_lock(&lock_sys.wait_mutex);
c_lock->trx->mutex.wr_lock();
- if (c_lock->trx->lock.que_state == TRX_QUE_LOCK_WAIT) {
+ if (c_lock->trx->lock.wait_thr) {
c_lock->trx->lock.was_chosen_as_deadlock_victim = TRUE;
if (UNIV_UNLIKELY(wsrep_debug)) {
@@ -3111,20 +3126,21 @@ lock_table_create(
/* The lock release will call lock_grant(),
which would acquire trx->mutex again. */
- trx->mutex.wr_unlock();
lock_cancel_waiting_and_release(
c_lock->trx->lock.wait_lock);
- trx->mutex.wr_lock();
}
+ mysql_mutex_unlock(&lock_sys.wait_mutex);
c_lock->trx->mutex.wr_unlock();
+ trx->mutex.wr_lock();
} else
#endif /* WITH_WSREP */
ut_list_append(table->locks, lock, TableLockGetNode());
if (type_mode & LOCK_WAIT) {
-
- lock_set_lock_and_trx_wait(lock, trx);
+ ut_ad(!trx->lock.wait_lock
+ || (*trx->lock.wait_lock).trx == trx);
+ trx->lock.wait_lock = lock;
}
lock->trx->lock.table_locks.push_back(lock);
@@ -3232,26 +3248,18 @@ lock_table_remove_low(
/* 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) {
+ if (lock->mode() == LOCK_AUTO_INC) {
+ ut_ad((table->autoinc_trx == trx) == !lock->is_waiting());
- /* 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)) {
+ /* 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()). */
lock_table_remove_autoinc_lock(lock, trx);
}
@@ -3292,7 +3300,6 @@ lock_table_enqueue_waiting(
ut_ad(!srv_read_only_mode);
trx = thr_get_trx(thr);
- ut_a(!que_thr_stop(thr));
switch (trx_get_dict_operation(trx)) {
case TRX_DICT_OP_NONE:
@@ -3321,12 +3328,15 @@ lock_table_enqueue_waiting(
const trx_t* victim_trx =
DeadlockChecker::check_and_resolve(lock, trx);
- if (victim_trx != 0) {
+ if (victim_trx) {
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);
+ /* There is no need to hold lock_sys.wait_mutex here,
+ because we are clearing the wait flag on a lock request
+ that is associated with the current transaction. So,
+ this is not conflicting with lock_wait(). */
lock_reset_lock_and_trx_wait(lock);
return(DB_DEADLOCK);
@@ -3338,13 +3348,9 @@ lock_table_enqueue_waiting(
return(DB_SUCCESS);
}
- trx->lock.que_state = TRX_QUE_LOCK_WAIT;
-
- trx->lock.wait_started = time(NULL);
+ trx->lock.wait_thr = thr;
trx->lock.was_chosen_as_deadlock_victim = false;
- ut_a(que_thr_stop(thr));
-
MONITOR_INC(MONITOR_TABLELOCK_WAIT);
return(DB_LOCK_WAIT);
@@ -3583,6 +3589,7 @@ lock_table_dequeue(
they are now qualified to it */
{
lock_sys.mutex_assert_locked();
+ mysql_mutex_assert_owner(&lock_sys.wait_mutex);
ut_a(lock_get_type_low(in_lock) == LOCK_TABLE);
lock_t* lock = UT_LIST_GET_NEXT(un_member.tab_lock.locks, in_lock);
@@ -3615,12 +3622,13 @@ void lock_table_x_unlock(dict_table_t *table, trx_t *trx)
ut_ad(!trx->is_recovered);
lock_sys.mutex_lock();
+ mysql_mutex_lock(&lock_sys.wait_mutex);
for (lock_t*& lock : trx->lock.table_locks)
{
if (lock && lock->trx == trx && lock->type_mode == (LOCK_TABLE | LOCK_X))
{
- ut_ad(!lock_get_wait(lock));
+ ut_ad(!lock->is_waiting());
lock_table_dequeue(lock);
lock= nullptr;
goto func_exit;
@@ -3630,6 +3638,7 @@ void lock_table_x_unlock(dict_table_t *table, trx_t *trx)
func_exit:
lock_sys.mutex_unlock();
+ mysql_mutex_unlock(&lock_sys.wait_mutex);
}
/** Sets a lock on a table based on the given mode.
@@ -3660,8 +3669,6 @@ lock_table_for_trx(
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;
@@ -3670,11 +3677,7 @@ run_again:
trx->error_state = err;
- if (UNIV_LIKELY(err == DB_SUCCESS)) {
- thr->stop_no_error();
- } else {
- que_thr_stop_for_mysql(thr);
-
+ if (UNIV_UNLIKELY(err != DB_SUCCESS)) {
if (row_mysql_handle_errors(&err, trx, thr, NULL)) {
goto run_again;
}
@@ -3758,7 +3761,9 @@ released:
if (!c) {
/* Grant the lock */
ut_ad(trx != lock->trx);
+ mysql_mutex_lock(&lock_sys.wait_mutex);
lock_grant(lock);
+ mysql_mutex_unlock(&lock_sys.wait_mutex);
#ifdef WITH_WSREP
} else {
wsrep_assert_no_bf_bf_wait(c, lock, c->trx);
@@ -3818,6 +3823,7 @@ void lock_release(trx_t* trx)
trx_id_t max_trx_id = trx_sys.get_max_trx_id();
lock_sys.mutex_lock();
+ mysql_mutex_lock(&lock_sys.wait_mutex);
for (lock_t* lock = UT_LIST_GET_LAST(trx->lock.trx_locks);
lock != NULL;
@@ -3851,16 +3857,17 @@ void lock_release(trx_t* trx)
do not monopolize it */
lock_sys.mutex_unlock();
-
- lock_sys.mutex_lock();
-
+ mysql_mutex_unlock(&lock_sys.wait_mutex);
count = 0;
+ lock_sys.mutex_lock();
+ mysql_mutex_lock(&lock_sys.wait_mutex);
}
++count;
}
lock_sys.mutex_unlock();
+ mysql_mutex_unlock(&lock_sys.wait_mutex);
}
/* True if a lock mode is S or X */
@@ -4133,27 +4140,26 @@ lock_print_info_summary(
/** 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)
+@param[in] now current my_hrtime_coarse() */
+void lock_trx_print_wait_and_mvcc_state(FILE *file, const trx_t *trx,
+ my_hrtime_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) {
-
+ if (const lock_t* wait_lock = trx->lock.wait_lock) {
fprintf(file,
- "------- TRX HAS BEEN WAITING %lu SEC"
+ "------- TRX HAS BEEN WAITING %llu ns"
" FOR THIS LOCK TO BE GRANTED:\n",
- (ulong) difftime(now, trx->lock.wait_started));
+ now.val - trx->lock.suspend_time.val);
- if (lock_get_type_low(trx->lock.wait_lock) == LOCK_REC) {
+ if (lock_get_type_low(wait_lock) == LOCK_REC) {
mtr_t mtr;
- lock_rec_print(file, trx->lock.wait_lock, mtr);
+ lock_rec_print(file, wait_lock, mtr);
} else {
- lock_table_print(file, trx->lock.wait_lock);
+ lock_table_print(file, wait_lock);
}
fprintf(file, "------------------\n");
@@ -4198,9 +4204,9 @@ lock_trx_print_locks(
/** Functor to display all transactions */
struct lock_print_info
{
- lock_print_info(FILE* file, time_t now) :
+ lock_print_info(FILE* file, my_hrtime_t now) :
file(file), now(now),
- purge_trx(purge_sys.query ? purge_sys.query->trx : NULL)
+ purge_trx(purge_sys.query ? purge_sys.query->trx : nullptr)
{}
void operator()(const trx_t &trx) const
@@ -4214,7 +4220,7 @@ struct lock_print_info
}
FILE* const file;
- const time_t now;
+ const my_hrtime_t now;
const trx_t* const purge_trx;
};
@@ -4231,7 +4237,7 @@ lock_print_info_all_transactions(
fprintf(file, "LIST OF TRANSACTIONS FOR EACH SESSION:\n");
- trx_sys.trx_list.for_each(lock_print_info(file, time(nullptr)));
+ trx_sys.trx_list.for_each(lock_print_info(file, my_hrtime_coarse()));
lock_sys.mutex_unlock();
ut_ad(lock_validate());
@@ -5380,17 +5386,14 @@ lock_release_autoinc_last_lock(
lock_t* lock;
lock_sys.mutex_assert_locked();
- ut_a(!ib_vector_is_empty(autoinc_locks));
+ ut_ad(!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);
+ ut_ad(lock->type_mode == (LOCK_AUTO_INC | LOCK_TABLE));
+ ut_ad(lock->un_member.tab_lock.table);
/* This will remove the lock from the trx autoinc_locks too. */
lock_table_dequeue(lock);
@@ -5422,6 +5425,7 @@ lock_release_autoinc_locks(
trx_t* trx) /*!< in/out: transaction */
{
lock_sys.mutex_assert_locked();
+ mysql_mutex_assert_owner(&lock_sys.wait_mutex);
/* 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. */
@@ -5514,61 +5518,33 @@ lock_rec_get_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 */
+/** Cancel a waiting lock request and release possibly waiting transactions */
+void lock_cancel_waiting_and_release(lock_t *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 */
-{
- lock_sys.mutex_assert_locked();
- trx_t* trx = lock->trx;
- ut_ad(trx->state == TRX_STATE_ACTIVE);
-
- 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 (trx->autoinc_locks) {
- /* Release the transaction's AUTOINC locks. */
- lock_release_autoinc_locks(trx);
- }
+ lock_sys.mutex_assert_locked();
+ mysql_mutex_assert_owner(&lock_sys.wait_mutex);
+ trx_t *trx= lock->trx;
+ ut_ad(trx->state == TRX_STATE_ACTIVE);
- lock_table_dequeue(lock);
- /* Remove the lock from table lock vector too. */
- lock_trx_table_locks_remove(lock);
- }
+ trx->lock.cancel= true;
- /* Reset the wait flag and the back pointer to lock in trx. */
+ if (lock_get_type_low(lock) == LOCK_REC)
+ lock_rec_dequeue_from_page(lock);
+ else
+ {
+ if (trx->autoinc_locks)
+ lock_release_autoinc_locks(trx);
+ lock_table_dequeue(lock);
+ /* Remove the lock from table lock vector too. */
+ lock_trx_table_locks_remove(lock);
+ }
- lock_reset_lock_and_trx_wait(lock);
+ /* Reset the wait flag and the back pointer to lock in trx. */
+ lock_reset_lock_and_trx_wait(lock);
- if (que_thr_t *thr = que_thr_end_lock_wait(trx)) {
- lock_wait_release_thread_if_suspended(thr);
- }
+ lock_wait_end(trx);
- trx->lock.cancel = false;
+ trx->lock.cancel= false;
}
/*********************************************************************//**
@@ -5595,27 +5571,28 @@ lock_unlock_table_autoinc(
if (lock_trx_holds_autoinc_locks(trx)) {
lock_sys.mutex_lock();
+ mysql_mutex_lock(&lock_sys.wait_mutex);
lock_release_autoinc_locks(trx);
lock_sys.mutex_unlock();
+ mysql_mutex_unlock(&lock_sys.wait_mutex);
}
}
static inline dberr_t lock_trx_handle_wait_low(trx_t* trx)
{
- lock_sys.mutex_assert_locked();
+ lock_sys.mutex_assert_locked();
+ mysql_mutex_assert_owner(&lock_sys.wait_mutex);
- 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;
- }
+ 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;
+ lock_cancel_waiting_and_release(trx->lock.wait_lock);
+ return DB_LOCK_WAIT;
}
/*********************************************************************//**
@@ -5629,17 +5606,19 @@ 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);
- }
+ if (UNIV_UNLIKELY(trx->lock.was_chosen_as_wsrep_victim))
+ /* FIXME: we do not hold lock_sys.wait_mutex! */
+ return lock_trx_handle_wait_low(trx);
#endif /* WITH_WSREP */
- lock_sys.mutex_lock();
- trx->mutex.wr_lock();
- dberr_t err = lock_trx_handle_wait_low(trx);
- lock_sys.mutex_unlock();
- trx->mutex.wr_unlock();
- return err;
+ dberr_t err;
+ lock_sys.mutex_lock();
+ mysql_mutex_lock(&lock_sys.wait_mutex);
+ trx->mutex.wr_lock();
+ err= lock_trx_handle_wait_low(trx);
+ lock_sys.mutex_unlock();
+ mysql_mutex_unlock(&lock_sys.wait_mutex);
+ trx->mutex.wr_unlock();
+ return err;
}
/*********************************************************************//**
@@ -6092,6 +6071,7 @@ const trx_t*
DeadlockChecker::search()
{
lock_sys.mutex_assert_locked();
+ mysql_mutex_assert_owner(&lock_sys.wait_mutex);
ut_ad(m_start != NULL);
ut_ad(m_wait_lock != NULL);
@@ -6143,7 +6123,9 @@ DeadlockChecker::search()
continue;
}
- if (lock->trx == m_start) {
+ trx_t *trx = lock->trx;
+
+ if (trx == m_start) {
/* Found a cycle. */
notify(lock);
return select_victim();
@@ -6163,10 +6145,12 @@ DeadlockChecker::search()
&& (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);
+ trx->mysql_thd);
}
- if (lock->trx->lock.que_state == TRX_QUE_LOCK_WAIT) {
+ lock_t* wait_lock = trx->lock.wait_lock;
+
+ if (wait_lock && trx->lock.wait_thr) {
/* Another trx ahead has requested a lock in an
incompatible mode, and is itself waiting for a lock. */
@@ -6177,7 +6161,7 @@ DeadlockChecker::search()
return m_start;
}
- m_wait_lock = lock->trx->lock.wait_lock;
+ m_wait_lock = wait_lock;
lock = get_first_lock(&heap_no);
@@ -6225,6 +6209,7 @@ void
DeadlockChecker::trx_rollback()
{
lock_sys.mutex_assert_locked();
+ mysql_mutex_assert_owner(&lock_sys.wait_mutex);
trx_t* trx = m_wait_lock->trx;
@@ -6234,13 +6219,9 @@ DeadlockChecker::trx_rollback()
wsrep_handle_SR_rollback(m_start->mysql_thd, trx->mysql_thd);
}
#endif
-
trx->mutex.wr_lock();
-
trx->lock.was_chosen_as_deadlock_victim = true;
-
lock_cancel_waiting_and_release(trx->lock.wait_lock);
-
trx->mutex.wr_unlock();
}
@@ -6282,6 +6263,7 @@ DeadlockChecker::check_and_resolve(const lock_t* lock, trx_t* trx)
/* Try and resolve as many deadlocks as possible. */
do {
+ mysql_mutex_lock(&lock_sys.wait_mutex);
DeadlockChecker checker(trx, lock, s_lock_mark_counter,
report_waiters);
@@ -6300,7 +6282,7 @@ DeadlockChecker::check_and_resolve(const lock_t* lock, trx_t* trx)
MONITOR_INC(MONITOR_DEADLOCK);
srv_stats.lock_deadlock_count.inc();
-
+ mysql_mutex_unlock(&lock_sys.wait_mutex);
break;
} else if (victim_trx != NULL && victim_trx != trx) {
@@ -6315,6 +6297,7 @@ DeadlockChecker::check_and_resolve(const lock_t* lock, trx_t* trx)
srv_stats.lock_deadlock_count.inc();
}
+ mysql_mutex_unlock(&lock_sys.wait_mutex);
} while (victim_trx != NULL && victim_trx != trx);
/* If the joining transaction was selected as the victim. */
diff --git a/storage/innobase/lock/lock0prdt.cc b/storage/innobase/lock/lock0prdt.cc
index 8a8c912f666..4cacd777e5e 100644
--- a/storage/innobase/lock/lock0prdt.cc
+++ b/storage/innobase/lock/lock0prdt.cc
@@ -946,8 +946,10 @@ lock_prdt_rec_move(
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);
-
+ if (type_mode & LOCK_WAIT) {
+ ut_ad(lock->trx->lock.wait_lock == lock);
+ lock->type_mode &= ~LOCK_WAIT;
+ }
lock_prdt_add_to_queue(
type_mode, receiver, lock->index, lock->trx,
lock_prdt, false);
diff --git a/storage/innobase/lock/lock0wait.cc b/storage/innobase/lock/lock0wait.cc
index 5c5caef9df8..1efa14938cf 100644
--- a/storage/innobase/lock/lock0wait.cc
+++ b/storage/innobase/lock/lock0wait.cc
@@ -1,7 +1,7 @@
/*****************************************************************************
Copyright (c) 1996, 2016, Oracle and/or its affiliates. All Rights Reserved.
-Copyright (c) 2014, 2020, MariaDB Corporation.
+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
@@ -36,141 +36,6 @@ Created 25/5/2010 Sunny Bains
#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)
-/*=======================*/
-{
- mysql_mutex_assert_owner(&lock_sys.wait_mutex);
-
- 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 */
-
- mysql_mutex_lock(&lock_sys.wait_mutex);
-
- 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_sys.mutex_lock();
-
- slot->thr->slot = NULL;
- slot->thr = NULL;
- slot->in_use = FALSE;
-
- lock_sys.mutex_unlock();
-
- /* 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);
-
- mysql_mutex_unlock(&lock_sys.wait_mutex);
-}
-
-/*********************************************************************//**
-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;
-
- mysql_mutex_assert_owner(&lock_sys.wait_mutex);
-
- 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;
- 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
/*********************************************************************//**
@@ -210,290 +75,160 @@ wsrep_is_BF_lock_timeout(
}
#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 */
+/** Note that a record lock wait started */
+inline void lock_sys_t::wait_start()
{
- 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);
-
- mysql_mutex_lock(&lock_sys.wait_mutex);
- trx->mutex.wr_lock();
-
- 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;
- }
-
- mysql_mutex_unlock(&lock_sys.wait_mutex);
- trx->mutex.wr_unlock();
- return;
- }
-
- ut_ad(!thr->is_active);
-
- slot = lock_wait_table_reserve_slot(thr, lock_wait_timeout);
-
- mysql_mutex_unlock(&lock_sys.wait_mutex);
- trx->mutex.wr_unlock();
-
- 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_sys.mutex_lock();
- wait_lock = trx->lock.wait_lock;
- if (wait_lock) {
- lock_type = lock_get_type_low(wait_lock);
- }
- lock_sys.mutex_unlock();
- }
-
- 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);
- }
-
- lock_sys.mutex_lock();
- lock_sys.wait_lock(&trx->lock.wait_lock, &slot->cond);
- lock_sys.mutex_unlock();
-
- 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;
+ mysql_mutex_assert_owner(&wait_mutex);
+ wait_pending++;
+ wait_count++;
+}
- MONITOR_INC(MONITOR_TIMEOUT);
- }
+/** Note that a record lock wait resumed */
+inline
+void lock_sys_t::wait_resume(THD *thd, my_hrtime_t start, my_hrtime_t now)
+{
+ mysql_mutex_assert_owner(&wait_mutex);
+ wait_pending--;
+ if (now.val >= start.val)
+ {
+ const ulint diff_time= static_cast<ulint>((now.val - start.val) / 1000);
+ wait_time+= diff_time;
- if (trx_is_interrupted(trx)) {
+ if (diff_time > wait_time_max)
+ wait_time_max= diff_time;
- trx->error_state = DB_INTERRUPTED;
- }
+ thd_storage_lock_wait(thd, diff_time);
+ }
}
-/********************************************************************//**
-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 */
+/** Wait for a lock to be released.
+@retval DB_DEADLOCK if this transaction was chosen as the deadlock victim
+@retval DB_INTERRUPTED if the execution was interrupted by the user
+@retval DB_LOCK_WAIT_TIMEOUT if the lock wait timed out
+@retval DB_SUCCESS if the lock was granted */
+dberr_t lock_wait(que_thr_t *thr)
{
- lock_sys.mutex_assert_locked();
+ trx_t *trx= 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 (trx->mysql_thd)
+ DEBUG_SYNC_C("lock_wait_suspend_thread_enter");
- if (thr->slot != NULL && thr->slot->in_use && thr->slot->thr == thr) {
- trx_t* trx = thr_get_trx(thr);
+ /* InnoDB system transactions may use the global value of
+ innodb_lock_wait_timeout, because trx->mysql_thd == NULL. */
+ const ulong innodb_lock_wait_timeout= trx_lock_wait_timeout_get(trx);
+ const bool no_timeout= innodb_lock_wait_timeout > 100000000;
+ const my_hrtime_t suspend_time= my_hrtime_coarse();
+ ut_ad(!trx->dict_operation_lock_mode ||
+ trx->dict_operation_lock_mode == RW_S_LATCH);
+ const bool row_lock_wait= thr->lock_state == QUE_THR_LOCK_ROW;
+ bool had_dict_lock= trx->dict_operation_lock_mode != 0;
- if (trx->lock.was_chosen_as_deadlock_victim) {
+ mysql_mutex_lock(&lock_sys.wait_mutex);
+ trx->mutex.wr_lock();
+ trx->error_state= DB_SUCCESS;
- trx->error_state = DB_DEADLOCK;
- trx->lock.was_chosen_as_deadlock_victim = false;
- }
+ if (!trx->lock.wait_lock)
+ {
+ /* The lock has already been released or this transaction
+ was chosen as a deadlock victim: no need to suspend */
- mysql_cond_signal(&thr->slot->cond);
- }
-}
+ if (trx->lock.was_chosen_as_deadlock_victim)
+ {
+ trx->error_state= DB_DEADLOCK;
+ trx->lock.was_chosen_as_deadlock_victim= false;
+ }
-/*********************************************************************//**
-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 */
-{
- mysql_mutex_assert_owner(&lock_sys.wait_mutex);
- ut_ad(slot->in_use);
+ mysql_mutex_unlock(&lock_sys.wait_mutex);
+ trx->mutex.wr_unlock();
+ return trx->error_state;
+ }
- double wait_time = difftime(time(NULL), slot->suspend_time);
- trx_t* trx = thr_get_trx(slot->thr);
+ trx->lock.suspend_time= suspend_time;
+ trx->mutex.wr_unlock();
- if (trx_is_interrupted(trx)
- || (slot->wait_timeout < 100000000
- && (wait_time > (double) slot->wait_timeout
- || wait_time < 0))) {
+ if (row_lock_wait)
+ lock_sys.wait_start();
- /* 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 */
+ int err= 0;
- lock_sys.mutex_lock();
+ /* The wait_lock can be cleared by another thread in lock_grant(),
+ lock_rec_cancel(), or lock_cancel_waiting_and_release(). But, a wait
+ can only be initiated by the current thread which owns the transaction. */
+ if (const lock_t *wait_lock= trx->lock.wait_lock)
+ {
+ if (had_dict_lock) /* Release foreign key check latch */
+ {
+ mysql_mutex_unlock(&lock_sys.wait_mutex);
+ row_mysql_unfreeze_data_dictionary(trx);
+ mysql_mutex_lock(&lock_sys.wait_mutex);
+ }
+ timespec abstime;
+ set_timespec_time_nsec(abstime, suspend_time.val * 1000);
+ abstime.MY_tv_sec+= innodb_lock_wait_timeout;
+ thd_wait_begin(trx->mysql_thd, lock_get_type_low(wait_lock) == LOCK_TABLE
+ ? THD_WAIT_TABLE_LOCK : THD_WAIT_ROW_LOCK);
+ while (trx->lock.wait_lock)
+ {
+ if (no_timeout)
+ mysql_cond_wait(&trx->lock.cond, &lock_sys.wait_mutex);
+ else
+ err= mysql_cond_timedwait(&trx->lock.cond, &lock_sys.wait_mutex,
+ &abstime);
+ switch (trx->error_state) {
+ default:
+ if (trx_is_interrupted(trx))
+ /* innobase_kill_query() can only set trx->error_state=DB_INTERRUPTED
+ for any transaction that is attached to a connection. */
+ trx->error_state= DB_INTERRUPTED;
+ else if (!err)
+ continue;
+ else
+ break;
+ /* fall through */
+ case DB_DEADLOCK:
+ case DB_INTERRUPTED:
+ err= 0;
+ }
+ break;
+ }
+ thd_wait_end(trx->mysql_thd);
+ }
+ else
+ had_dict_lock= false;
- if (trx->lock.wait_lock != NULL) {
- ut_a(trx->lock.que_state == TRX_QUE_LOCK_WAIT);
+ if (row_lock_wait)
+ lock_sys.wait_resume(trx->mysql_thd, suspend_time, my_hrtime_coarse());
-#ifdef WITH_WSREP
- if (!wsrep_is_BF_lock_timeout(trx)) {
-#endif /* WITH_WSREP */
- trx->mutex.wr_lock();
- lock_cancel_waiting_and_release(trx->lock.wait_lock);
- trx->mutex.wr_unlock();
-#ifdef WITH_WSREP
- }
-#endif /* WITH_WSREP */
- }
+ mysql_mutex_unlock(&lock_sys.wait_mutex);
- lock_sys.mutex_unlock();
- }
-}
+ if (had_dict_lock)
+ row_mysql_freeze_data_dictionary(trx);
-/** A task which wakes up threads whose lock wait may have lasted too long */
-void lock_wait_timeout_task(void*)
-{
- mysql_mutex_lock(&lock_sys.wait_mutex);
+ if (!err);
+#ifdef WITH_WSREP
+ else if (trx->is_wsrep() && wsrep_is_BF_lock_timeout(trx, false));
+#endif
+ else
+ {
+ trx->error_state= DB_LOCK_WAIT_TIMEOUT;
+ MONITOR_INC(MONITOR_TIMEOUT);
+ }
- /* 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)
+ if (trx->lock.wait_lock)
{
- /* 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);
+ lock_sys.mutex_lock();
+ mysql_mutex_lock(&lock_sys.wait_mutex);
+ if (lock_t *lock= trx->lock.wait_lock)
+ {
+ trx->mutex.wr_lock();
+ lock_cancel_waiting_and_release(lock);
+ trx->mutex.wr_unlock();
+ }
+ lock_sys.mutex_unlock();
+ mysql_mutex_unlock(&lock_sys.wait_mutex);
}
}
- if (any_slot_in_use)
- lock_sys.timeout_timer->set_time(1000, 0);
- else
- lock_sys.timeout_timer_active= false;
-
- mysql_mutex_unlock(&lock_sys.wait_mutex);
+ return trx->error_state;
}
diff --git a/storage/innobase/log/log0log.cc b/storage/innobase/log/log0log.cc
index 44eaecb7a29..44ae69e5042 100644
--- a/storage/innobase/log/log0log.cc
+++ b/storage/innobase/log/log0log.cc
@@ -2,7 +2,7 @@
Copyright (c) 1995, 2017, Oracle and/or its affiliates. All Rights Reserved.
Copyright (c) 2009, Google Inc.
-Copyright (c) 2014, 2020, MariaDB Corporation.
+Copyright (c) 2014, 2021, MariaDB Corporation.
Portions of this file contain modifications contributed and copyrighted by
Google, Inc. Those modifications are gratefully acknowledged and are described
@@ -1010,7 +1010,7 @@ ATTRIBUTE_COLD void logs_empty_and_mark_files_at_shutdown()
buf_dump_start();
}
srv_monitor_timer.reset();
- lock_sys.timeout_timer.reset();
+
if (do_srv_shutdown) {
srv_shutdown(srv_fast_shutdown == 0);
}
diff --git a/storage/innobase/que/que0que.cc b/storage/innobase/que/que0que.cc
index fe36ce3dc00..009667f860d 100644
--- a/storage/innobase/que/que0que.cc
+++ b/storage/innobase/que/que0que.cc
@@ -146,10 +146,6 @@ que_thr_create(
thr->common.type = QUE_NODE_THR;
- thr->state = QUE_THR_COMMAND_WAIT;
-
- thr->lock_state = QUE_THR_LOCK_NOLOCK;
-
thr->prebuilt = prebuilt;
UT_LIST_ADD_LAST(parent->thrs, thr);
@@ -158,44 +154,6 @@ que_thr_create(
}
/**********************************************************************//**
-Moves a suspended query thread to the QUE_THR_RUNNING state and may release
-a worker thread to execute it. This function should be used to end
-the wait state of a query thread waiting for a lock or a stored procedure
-completion.
-@return the query thread that needs to be released. */
-que_thr_t*
-que_thr_end_lock_wait(
-/*==================*/
- trx_t* trx) /*!< in: transaction with que_state in
- QUE_THR_LOCK_WAIT */
-{
- que_thr_t* thr;
-
- lock_sys.mutex_assert_locked();
-
- thr = trx->lock.wait_thr;
-
- ut_ad(thr != NULL);
-
- ut_ad(trx->lock.que_state == TRX_QUE_LOCK_WAIT);
- /* In MySQL this is the only possible state here */
- ut_a(thr->state == QUE_THR_LOCK_WAIT);
-
- bool was_active = thr->is_active;
-
- thr->start_running();
-
- trx->lock.que_state = TRX_QUE_RUNNING;
-
- trx->lock.wait_thr = NULL;
-
- /* In MySQL we let the OS thread (not just the query thread) to wait
- for the lock to be released: */
-
- return((!was_active && thr != NULL) ? thr : NULL);
-}
-
-/**********************************************************************//**
Inits a query thread for a command. */
UNIV_INLINE
void
@@ -205,7 +163,7 @@ que_thr_init_command(
{
thr->run_node = thr;
thr->prev_node = thr->common.parent;
- thr->start_running();
+ thr->state = QUE_THR_RUNNING;
}
/**********************************************************************//**
@@ -233,19 +191,8 @@ que_fork_scheduler_round_robin(
fork->state = QUE_FORK_ACTIVE;
fork->last_sel_node = NULL;
-
- switch (thr->state) {
- case QUE_THR_COMMAND_WAIT:
- case QUE_THR_COMPLETED:
- ut_a(!thr->is_active);
- que_thr_init_command(thr);
- break;
-
- case QUE_THR_LOCK_WAIT:
- default:
- ut_error;
-
- }
+ ut_ad(thr->state == QUE_THR_COMPLETED);
+ que_thr_init_command(thr);
}
fork->trx->mutex.wr_unlock();
@@ -266,57 +213,15 @@ que_fork_start_command(
/*===================*/
que_fork_t* fork) /*!< in: a query fork */
{
- que_thr_t* thr;
- que_thr_t* completed_thr = NULL;
-
fork->state = QUE_FORK_ACTIVE;
fork->last_sel_node = NULL;
- completed_thr = NULL;
-
- /* Choose the query thread to run: usually there is just one thread,
- but in a parallelized select, which necessarily is non-scrollable,
- there may be several to choose from */
-
- /* First we try to find a query thread in the QUE_THR_COMMAND_WAIT
- state. Finally we try to find a query thread in the QUE_THR_COMPLETED
- state */
-
- /* We make a single pass over the thr list within which we note which
- threads are ready to run. */
- for (thr = UT_LIST_GET_FIRST(fork->thrs);
- thr != NULL;
- thr = UT_LIST_GET_NEXT(thrs, thr)) {
-
- switch (thr->state) {
- case QUE_THR_COMMAND_WAIT:
-
- /* We have to send the initial message to query thread
- to start it */
+ que_thr_t* thr = UT_LIST_GET_FIRST(fork->thrs);
- que_thr_init_command(thr);
-
- return(thr);
-
- case QUE_THR_COMPLETED:
- if (!completed_thr) {
- completed_thr = thr;
- }
-
- break;
-
- case QUE_THR_RUNNING:
- case QUE_THR_LOCK_WAIT:
- ut_error;
- }
- }
-
- if (completed_thr) {
- thr = completed_thr;
+ if (thr) {
+ ut_ad(thr->state == QUE_THR_COMPLETED);
que_thr_init_command(thr);
- } else {
- ut_error;
}
return(thr);
@@ -560,173 +465,18 @@ que_thr_node_step(
return(thr);
}
- auto mutex = &thr->graph->trx->mutex;
-
- mutex->wr_lock();
+ trx_t *trx= thr->graph->trx;
+ trx->mutex.wr_lock();
- if (!que_thr_peek_stop(thr)) {
- /* Thread execution completed */
+ if (!trx->lock.wait_thr && thr->graph->state == QUE_FORK_ACTIVE) {
thr->state = QUE_THR_COMPLETED;
thr = NULL;
}
- mutex->wr_unlock();
- return(thr);
-}
-
-/**********************************************************************//**
-Stops a query thread if graph or trx is in a state requiring it. The
-conditions are tested in the order (1) graph, (2) trx.
-@return TRUE if stopped */
-ibool
-que_thr_stop(
-/*=========*/
- que_thr_t* thr) /*!< in: query thread */
-{
- que_t* graph;
- trx_t* trx = thr_get_trx(thr);
-
- graph = thr->graph;
-
- if (graph->state == QUE_FORK_COMMAND_WAIT) {
-
- thr->state = QUE_THR_COMMAND_WAIT;
-
- } else if (trx->lock.que_state == TRX_QUE_LOCK_WAIT) {
-
- trx->lock.wait_thr = thr;
- thr->state = QUE_THR_LOCK_WAIT;
-
- } else if (trx->error_state != DB_SUCCESS
- && trx->error_state != DB_LOCK_WAIT) {
-
- /* Error handling built for the MySQL interface */
- thr->state = QUE_THR_COMPLETED;
- } else {
- ut_ad(graph->state == QUE_FORK_ACTIVE);
-
- return(FALSE);
- }
-
- return(TRUE);
-}
-
-/**********************************************************************//**
-Decrements the query thread reference counts in the query graph and the
-transaction.
-*** NOTE ***:
-This and que_thr_stop_for_mysql are the only functions where the reference
-count can be decremented and this function may only be called from inside
-que_run_threads! These restrictions exist to make the rollback code easier
-to maintain. */
-static
-void
-que_thr_dec_refer_count(
-/*====================*/
- que_thr_t* thr, /*!< in: query thread */
- que_thr_t** next_thr) /*!< in/out: next query thread to run;
- if the value which is passed in is
- a pointer to a NULL pointer, then the
- calling function can start running
- a new query thread */
-{
- trx_t* trx;
-
- trx = thr_get_trx(thr);
-
- ut_a(thr->is_active);
-
- if (thr->state == QUE_THR_RUNNING) {
-
- if (!que_thr_stop(thr)) {
-
- ut_a(next_thr != NULL && *next_thr == NULL);
-
- /* The reason for the thr suspension or wait was
- already canceled before we came here: continue
- running the thread.
-
- This is also possible because in trx_commit_step() we
- assume a single query thread. We set the query thread
- state to QUE_THR_RUNNING. */
-
- /* fprintf(stderr,
- "Wait already ended: trx: %p\n", trx); */
-
- /* Normally srv_suspend_mysql_thread resets
- the state to DB_SUCCESS before waiting, but
- in this case we have to do it here,
- otherwise nobody does it. */
-
- trx->error_state = DB_SUCCESS;
-
- *next_thr = thr;
-
- return;
- }
- }
-
- ut_d(static_cast<que_fork_t*>(thr->common.parent)->set_active(false));
- thr->is_active = false;
-}
-
-/**********************************************************************//**
-A patch for MySQL used to 'stop' a dummy query thread used in MySQL. The
-query thread is stopped and made inactive, except in the case where
-it was put to the lock wait state in lock0lock.cc, but the lock has already
-been granted or the transaction chosen as a victim in deadlock resolution. */
-void
-que_thr_stop_for_mysql(
-/*===================*/
- que_thr_t* thr) /*!< in: query thread */
-{
- trx_t* trx;
-
- trx = thr_get_trx(thr);
-
- trx->mutex.wr_lock();
-
- if (thr->state == QUE_THR_RUNNING) {
- switch (trx->error_state) {
- default:
- /* Error handling built for the MariaDB interface */
- thr->state = QUE_THR_COMPLETED;
- break;
- case DB_SUCCESS:
- case DB_LOCK_WAIT:
- /* It must have been a lock wait but the lock was
- already released, or this transaction was chosen
- as a victim in selective deadlock resolution */
- goto func_exit;
- }
- }
-
- ut_ad(thr->is_active);
- ut_d(thr->set_active(false));
- thr->is_active= false;
-func_exit:
trx->mutex.wr_unlock();
+ return(thr);
}
-#ifdef UNIV_DEBUG
-/** Change the 'active' status */
-void que_fork_t::set_active(bool active)
-{
- if (active)
- {
- n_active_thrs++;
- trx->lock.n_active_thrs++;
- }
- else
- {
- ut_ad(n_active_thrs);
- ut_ad(trx->lock.n_active_thrs);
- n_active_thrs--;
- trx->lock.n_active_thrs--;
- }
-}
-#endif
-
/****************************************************************//**
Get the first containing loop node (e.g. while_node_t or for_node_t) for the
given node, or NULL if the node is not within a loop.
@@ -967,17 +717,6 @@ que_run_threads_low(
if (next_thr) {
ut_a(trx->error_state == DB_SUCCESS);
ut_a(next_thr == thr);
- } else {
- /* This can change next_thr to a non-NULL value
- if there was a lock wait that already completed. */
-
- trx->mutex.wr_lock();
- que_thr_dec_refer_count(thr, &next_thr);
- trx->mutex.wr_unlock();
-
- if (next_thr != NULL) {
- thr = next_thr;
- }
}
ut_ad(trx == thr_get_trx(thr));
@@ -991,40 +730,18 @@ que_run_threads(
/*============*/
que_thr_t* thr) /*!< in: query thread */
{
+ trx_t* trx = thr->graph->trx;
loop:
- ut_a(thr_get_trx(thr)->error_state == DB_SUCCESS);
-
+ ut_a(trx->error_state == DB_SUCCESS);
que_run_threads_low(thr);
- switch (thr->state) {
- default:
- ut_error;
- case QUE_THR_COMPLETED:
- case QUE_THR_COMMAND_WAIT:
- /* Do nothing */
- break;
-
- case QUE_THR_RUNNING:
- /* There probably was a lock wait, but it already ended
- before we came here: continue running thr */
-
- goto loop;
-
- case QUE_THR_LOCK_WAIT:
- lock_wait_suspend_thread(thr);
- trx_t* trx = thr->graph->trx;
-
- trx->mutex.wr_lock();
- ut_ad(trx->id);
- const dberr_t err = trx->error_state;
- if (err != DB_SUCCESS) {
- /* thr was chosen as a deadlock victim or there was
- a lock wait timeout */
- que_thr_dec_refer_count(thr, NULL);
- }
- trx->mutex.wr_unlock();
-
- if (err == DB_SUCCESS) {
+ if (thr->state != QUE_THR_COMPLETED) {
+ if (trx->lock.wait_thr) {
+ ut_ad(trx->id);
+ if (lock_wait(thr) == DB_SUCCESS) {
+ goto loop;
+ }
+ } else if (trx->error_state == DB_SUCCESS) {
goto loop;
}
}
diff --git a/storage/innobase/row/row0ins.cc b/storage/innobase/row/row0ins.cc
index 7fb685efb04..1ec3504e4d7 100644
--- a/storage/innobase/row/row0ins.cc
+++ b/storage/innobase/row/row0ins.cc
@@ -1825,25 +1825,22 @@ do_possible_lock_wait:
if (err == DB_LOCK_WAIT) {
trx->error_state = err;
- que_thr_stop_for_mysql(thr);
-
thr->lock_state = QUE_THR_LOCK_ROW;
check_table->inc_fk_checks();
- lock_wait_suspend_thread(thr);
+ err = lock_wait(thr);
thr->lock_state = QUE_THR_LOCK_NOLOCK;
- err = trx->error_state;
+ check_table->dec_fk_checks();
+
if (err != DB_SUCCESS) {
} else if (check_table->to_be_dropped) {
err = DB_LOCK_WAIT_TIMEOUT;
} else {
err = DB_LOCK_WAIT;
}
-
- check_table->dec_fk_checks();
}
exit_func:
diff --git a/storage/innobase/row/row0mysql.cc b/storage/innobase/row/row0mysql.cc
index c2c452309d3..9b03b099987 100644
--- a/storage/innobase/row/row0mysql.cc
+++ b/storage/innobase/row/row0mysql.cc
@@ -688,14 +688,14 @@ row_mysql_handle_errors(
DBUG_ENTER("row_mysql_handle_errors");
-handle_new_error:
err = trx->error_state;
+handle_new_error:
ut_a(err != DB_SUCCESS);
trx->error_state = DB_SUCCESS;
- DBUG_LOG("trx", "handle error: " << ut_strerr(err)
+ DBUG_LOG("trx", "handle error: " << err
<< ";id=" << ib::hex(trx->id) << ", " << trx);
switch (err) {
@@ -733,11 +733,8 @@ handle_new_error:
/* MySQL will roll back the latest SQL statement */
break;
case DB_LOCK_WAIT:
- lock_wait_suspend_thread(thr);
-
- if (trx->error_state != DB_SUCCESS) {
- que_thr_stop_for_mysql(thr);
-
+ err = lock_wait(thr);
+ if (err != DB_SUCCESS) {
goto handle_new_error;
}
@@ -786,14 +783,13 @@ handle_new_error:
ib::fatal() << "Unknown error " << err;
}
- if (trx->error_state != DB_SUCCESS) {
- *new_err = trx->error_state;
+ if (dberr_t n_err = trx->error_state) {
+ trx->error_state = DB_SUCCESS;
+ *new_err = n_err;
} else {
*new_err = err;
}
- trx->error_state = DB_SUCCESS;
-
DBUG_RETURN(false);
}
@@ -1139,12 +1135,12 @@ row_get_prebuilt_insert_row(
dict_table_copy_types(row, table);
ins_node_set_new_row(node, row);
+ que_thr_t* fork = pars_complete_graph_for_exec(
+ node, prebuilt->trx, prebuilt->heap, prebuilt);
+ fork->state = QUE_THR_RUNNING;
prebuilt->ins_graph = static_cast<que_fork_t*>(
- que_node_get_parent(
- pars_complete_graph_for_exec(
- node,
- prebuilt->trx, prebuilt->heap, prebuilt)));
+ que_node_get_parent(fork));
prebuilt->ins_graph->state = QUE_FORK_ACTIVE;
@@ -1171,7 +1167,6 @@ row_lock_table_autoinc_for_mysql(
const dict_table_t* table = prebuilt->table;
que_thr_t* thr;
dberr_t err;
- ibool was_lock_wait;
/* If we already hold an AUTOINC lock on the table then do nothing.
Note: We peek at the value of the current owner without acquiring
@@ -1191,36 +1186,20 @@ row_lock_table_autoinc_for_mysql(
thr = que_fork_get_first_thr(prebuilt->ins_graph);
- thr->start_running();
-
-run_again:
- thr->run_node = node;
- thr->prev_node = node;
-
- /* It may be that the current session has not yet started
- its transaction, or it has been committed: */
-
- trx_start_if_not_started_xa(trx, true);
-
- err = lock_table(0, prebuilt->table, LOCK_AUTO_INC, thr);
-
- trx->error_state = err;
-
- if (err != DB_SUCCESS) {
- que_thr_stop_for_mysql(thr);
+ do {
+ thr->run_node = node;
+ thr->prev_node = node;
- was_lock_wait = row_mysql_handle_errors(&err, trx, thr, NULL);
+ /* It may be that the current session has not yet started
+ its transaction, or it has been committed: */
- if (was_lock_wait) {
- goto run_again;
- }
+ trx_start_if_not_started_xa(trx, true);
- trx->op_info = "";
+ err = lock_table(0, prebuilt->table, LOCK_AUTO_INC, thr);
- return(err);
- }
-
- thr->stop_no_error();
+ trx->error_state = err;
+ } while (err != DB_SUCCESS
+ && row_mysql_handle_errors(&err, trx, thr, NULL));
trx->op_info = "";
@@ -1236,7 +1215,6 @@ row_lock_table(row_prebuilt_t* prebuilt)
trx_t* trx = prebuilt->trx;
que_thr_t* thr;
dberr_t err;
- ibool was_lock_wait;
trx->op_info = "setting table lock";
@@ -1250,39 +1228,20 @@ row_lock_table(row_prebuilt_t* prebuilt)
thr = que_fork_get_first_thr(prebuilt->sel_graph);
- thr->start_running();
-
-run_again:
- thr->run_node = thr;
- thr->prev_node = thr->common.parent;
-
- /* It may be that the current session has not yet started
- its transaction, or it has been committed: */
-
- trx_start_if_not_started_xa(trx, false);
+ do {
+ thr->run_node = thr;
+ thr->prev_node = thr->common.parent;
- err = lock_table(0, prebuilt->table,
- static_cast<enum lock_mode>(
- prebuilt->select_lock_type),
- thr);
+ /* It may be that the current session has not yet started
+ its transaction, or it has been committed: */
- trx->error_state = err;
+ trx_start_if_not_started_xa(trx, false);
- if (err != DB_SUCCESS) {
- que_thr_stop_for_mysql(thr);
-
- was_lock_wait = row_mysql_handle_errors(&err, trx, thr, NULL);
-
- if (was_lock_wait) {
- goto run_again;
- }
-
- trx->op_info = "";
-
- return(err);
- }
-
- thr->stop_no_error();
+ err = lock_table(0, prebuilt->table, static_cast<lock_mode>(
+ prebuilt->select_lock_type), thr);
+ trx->error_state = err;
+ } while (err != DB_SUCCESS
+ && row_mysql_handle_errors(&err, trx, thr, NULL));
trx->op_info = "";
@@ -1418,8 +1377,6 @@ row_insert_for_mysql(
node->trx_id = trx->id;
}
- thr->start_running();
-
run_again:
thr->run_node = node;
thr->prev_node = node;
@@ -1432,8 +1389,6 @@ run_again:
if (err != DB_SUCCESS) {
error_exit:
- que_thr_stop_for_mysql(thr);
-
/* FIXME: What's this ? */
thr->lock_state = QUE_THR_LOCK_ROW;
@@ -1521,8 +1476,6 @@ error_exit:
}
}
- thr->stop_no_error();
-
if (table->is_system_db) {
srv_stats.n_system_rows_inserted.inc(size_t(trx->id));
} else {
@@ -1566,12 +1519,12 @@ row_prebuild_sel_graph(
node = sel_node_create(prebuilt->heap);
+ que_thr_t* fork = pars_complete_graph_for_exec(
+ node, prebuilt->trx, prebuilt->heap, prebuilt);
+ fork->state = QUE_THR_RUNNING;
+
prebuilt->sel_graph = static_cast<que_fork_t*>(
- que_node_get_parent(
- pars_complete_graph_for_exec(
- static_cast<sel_node_t*>(node),
- prebuilt->trx, prebuilt->heap,
- prebuilt)));
+ que_node_get_parent(fork));
prebuilt->sel_graph->state = QUE_FORK_ACTIVE;
}
@@ -1834,8 +1787,6 @@ row_update_for_mysql(row_prebuilt_t* prebuilt)
ut_ad(!prebuilt->sql_stat_start);
- thr->start_running();
-
ut_ad(!prebuilt->versioned_write || node->table->versioned());
if (prebuilt->versioned_write) {
@@ -1859,8 +1810,6 @@ row_update_for_mysql(row_prebuilt_t* prebuilt)
break;
}
- que_thr_stop_for_mysql(thr);
-
if (err == DB_RECORD_NOT_FOUND) {
trx->error_state = DB_SUCCESS;
goto error;
@@ -1879,8 +1828,6 @@ row_update_for_mysql(row_prebuilt_t* prebuilt)
}
}
- thr->stop_no_error();
-
if (dict_table_has_fts_index(table)
&& trx->fts_next_doc_id != UINT64_UNDEFINED) {
err = row_fts_update_or_delete(prebuilt);
@@ -2189,10 +2136,7 @@ static dberr_t row_update_vers_insert(que_thr_t* thr, upd_node_t* node)
switch (trx->error_state) {
case DB_LOCK_WAIT:
- que_thr_stop_for_mysql(thr);
- lock_wait_suspend_thread(thr);
-
- if (trx->error_state == DB_SUCCESS) {
+ if (lock_wait(thr) == DB_SUCCESS) {
continue;
}
@@ -2263,10 +2207,7 @@ row_update_cascade_for_mysql(
switch (trx->error_state) {
case DB_LOCK_WAIT:
- que_thr_stop_for_mysql(thr);
- lock_wait_suspend_thread(thr);
-
- if (trx->error_state == DB_SUCCESS) {
+ if (lock_wait(thr) == DB_SUCCESS) {
continue;
}
@@ -3202,25 +3143,15 @@ row_mysql_lock_table(
thr = que_fork_get_first_thr(
static_cast<que_fork_t*>(que_node_get_parent(thr)));
- thr->start_running();
+ do {
+ thr->run_node = thr;
+ thr->prev_node = thr->common.parent;
-run_again:
- thr->run_node = thr;
- thr->prev_node = thr->common.parent;
+ err = lock_table(0, table, mode, thr);
- err = lock_table(0, table, mode, thr);
-
- trx->error_state = err;
-
- if (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;
- }
- }
+ trx->error_state = err;
+ } while (err != DB_SUCCESS
+ && row_mysql_handle_errors(&err, trx, thr, NULL));
que_graph_free(thr->graph);
trx->op_info = "";
diff --git a/storage/innobase/row/row0sel.cc b/storage/innobase/row/row0sel.cc
index 6e3a5e7613f..672550421e7 100644
--- a/storage/innobase/row/row0sel.cc
+++ b/storage/innobase/row/row0sel.cc
@@ -1110,7 +1110,6 @@ retry:
re_scan:
mtr->commit();
trx->error_state = err;
- que_thr_stop_for_mysql(thr);
thr->lock_state = QUE_THR_LOCK_ROW;
if (row_mysql_handle_errors(
&err, trx, thr, NULL)) {
@@ -4589,8 +4588,6 @@ aborted:
thr = que_fork_get_first_thr(prebuilt->sel_graph);
- thr->start_running();
-
clust_index = dict_table_get_first_index(prebuilt->table);
dberr_t err = DB_SUCCESS;
@@ -5735,13 +5732,6 @@ lock_table_wait:
mtr_has_extra_clust_latch = FALSE;
trx->error_state = err;
-
- /* The following is a patch for MySQL */
-
- if (thr->is_active) {
- que_thr_stop_for_mysql(thr);
- }
-
thr->lock_state = QUE_THR_LOCK_ROW;
if (row_mysql_handle_errors(&err, trx, thr, NULL)) {
@@ -5796,16 +5786,6 @@ lock_table_wait:
goto func_exit;
normal_return:
- /*-------------------------------------------------------------*/
- {
- /* handler_index_cond_check() may pull TR_table search
- which initates another row_search_mvcc(). */
- ut_d(ulint n_active_thrs= trx->lock.n_active_thrs);
- ut_d(trx->lock.n_active_thrs= 1);
- thr->stop_no_error();
- ut_d(trx->lock.n_active_thrs= n_active_thrs - 1);
- }
-
mtr.commit();
DEBUG_SYNC_C("row_search_for_mysql_before_return");
diff --git a/storage/innobase/srv/srv0mon.cc b/storage/innobase/srv/srv0mon.cc
index febbfa089f6..4ccd0adb360 100644
--- a/storage/innobase/srv/srv0mon.cc
+++ b/storage/innobase/srv/srv0mon.cc
@@ -1719,32 +1719,38 @@ srv_mon_process_existing_counter(
/* innodb_row_lock_current_waits */
case MONITOR_OVLD_ROW_LOCK_CURRENT_WAIT:
- value = srv_stats.n_lock_wait_current_count;
+ // dirty read without lock_sys.wait_mutex
+ value = lock_sys.get_wait_pending();
break;
/* innodb_row_lock_time */
case MONITOR_OVLD_LOCK_WAIT_TIME:
- value = srv_stats.n_lock_wait_time / 1000;
+ // dirty read without lock_sys.wait_mutex
+ value = lock_sys.get_wait_time_cumulative() / 1000;
break;
/* innodb_row_lock_time_max */
case MONITOR_OVLD_LOCK_MAX_WAIT_TIME:
- value = lock_sys.n_lock_max_wait_time / 1000;
+ // dirty read without lock_sys.wait_mutex
+ value = lock_sys.get_wait_time_max() / 1000;
break;
/* innodb_row_lock_time_avg */
case MONITOR_OVLD_LOCK_AVG_WAIT_TIME:
- if (srv_stats.n_lock_wait_count > 0) {
- value = srv_stats.n_lock_wait_time / 1000
- / srv_stats.n_lock_wait_count;
+ mysql_mutex_lock(&lock_sys.wait_mutex);
+ if (auto count = lock_sys.get_wait_cumulative()) {
+ value = lock_sys.get_wait_time_cumulative() / 1000
+ / count;
} else {
value = 0;
}
+ mysql_mutex_unlock(&lock_sys.wait_mutex);
break;
/* innodb_row_lock_waits */
case MONITOR_OVLD_ROW_LOCK_WAIT:
- value = srv_stats.n_lock_wait_count;
+ // dirty read without lock_sys.wait_mutex
+ value = lock_sys.get_wait_cumulative();
break;
case MONITOR_RSEG_HISTORY_LEN:
diff --git a/storage/innobase/srv/srv0srv.cc b/storage/innobase/srv/srv0srv.cc
index e71f725c660..6d8fcd8acde 100644
--- a/storage/innobase/srv/srv0srv.cc
+++ b/storage/innobase/srv/srv0srv.cc
@@ -1106,25 +1106,21 @@ srv_export_innodb_status(void)
export_vars.innodb_pages_written = buf_pool.stat.n_pages_written;
- export_vars.innodb_row_lock_waits = srv_stats.n_lock_wait_count;
+ mysql_mutex_lock(&lock_sys.wait_mutex);
+ export_vars.innodb_row_lock_waits = lock_sys.get_wait_cumulative();
- export_vars.innodb_row_lock_current_waits =
- srv_stats.n_lock_wait_current_count;
+ export_vars.innodb_row_lock_current_waits= lock_sys.get_wait_pending();
- export_vars.innodb_row_lock_time = srv_stats.n_lock_wait_time / 1000;
+ export_vars.innodb_row_lock_time = lock_sys.get_wait_time_cumulative()
+ / 1000;
+ export_vars.innodb_row_lock_time_max = lock_sys.get_wait_time_max()
+ / 1000;
+ mysql_mutex_unlock(&lock_sys.wait_mutex);
- if (srv_stats.n_lock_wait_count > 0) {
-
- export_vars.innodb_row_lock_time_avg = (ulint)
- (srv_stats.n_lock_wait_time
- / 1000 / srv_stats.n_lock_wait_count);
-
- } else {
- export_vars.innodb_row_lock_time_avg = 0;
- }
-
- export_vars.innodb_row_lock_time_max =
- lock_sys.n_lock_max_wait_time / 1000;
+ export_vars.innodb_row_lock_time_avg= export_vars.innodb_row_lock_waits
+ ? static_cast<ulint>(export_vars.innodb_row_lock_time
+ / export_vars.innodb_row_lock_waits)
+ : 0;
export_vars.innodb_rows_read = srv_stats.n_rows_read;
diff --git a/storage/innobase/srv/srv0start.cc b/storage/innobase/srv/srv0start.cc
index cdb96981996..ab4728de9ef 100644
--- a/storage/innobase/srv/srv0start.cc
+++ b/storage/innobase/srv/srv0start.cc
@@ -3,7 +3,7 @@
Copyright (c) 1996, 2017, Oracle and/or its affiliates. All rights reserved.
Copyright (c) 2008, Google Inc.
Copyright (c) 2009, Percona Inc.
-Copyright (c) 2013, 2020, MariaDB Corporation.
+Copyright (c) 2013, 2021, MariaDB Corporation.
Portions of this file contain modifications contributed and copyrighted by
Google, Inc. Those modifications are gratefully acknowledged and are described
@@ -825,7 +825,6 @@ static void srv_shutdown_threads()
ut_ad(!srv_undo_sources);
srv_shutdown_state = SRV_SHUTDOWN_EXIT_THREADS;
ut_d(srv_master_thread_enable());
- lock_sys.timeout_timer.reset();
srv_master_timer.reset();
if (purge_sys.enabled()) {
@@ -1787,11 +1786,6 @@ file_checked:
srv_startup_is_before_trx_rollback_phase = false;
if (!srv_read_only_mode) {
- /* timer task which watches the timeouts
- for lock waits */
- lock_sys.timeout_timer.reset(srv_thread_pool->create_timer(
- lock_wait_timeout_task));
-
DBUG_EXECUTE_IF("innodb_skip_monitors", goto skip_monitors;);
/* Create the task which warns of long semaphore waits */
srv_start_periodic_timer(srv_monitor_timer, srv_monitor_task,
diff --git a/storage/innobase/trx/trx0i_s.cc b/storage/innobase/trx/trx0i_s.cc
index 21bfbe5d92d..14b8a10cdc1 100644
--- a/storage/innobase/trx/trx0i_s.cc
+++ b/storage/innobase/trx/trx0i_s.cc
@@ -424,35 +424,29 @@ fill_trx_row(
lock_sys.mutex_assert_locked();
+ const lock_t* wait_lock = trx->lock.wait_lock;
+
row->trx_id = trx->id;
row->trx_started = trx->start_time;
- switch (trx->lock.que_state) {
- case TRX_QUE_RUNNING:
- row->trx_state = trx->state == TRX_STATE_COMMITTED_IN_MEMORY
- ? "COMMITTING" : "RUNNING";
- break;
- case TRX_QUE_LOCK_WAIT:
- row->trx_state = "LOCK WAIT";
- break;
- case TRX_QUE_ROLLING_BACK:
+ if (trx->in_rollback) {
row->trx_state = "ROLLING BACK";
- break;
- default:
- row->trx_state = nullptr;
+ } else if (trx->state == TRX_STATE_COMMITTED_IN_MEMORY) {
+ row->trx_state = "COMMITTING";
+ } else if (wait_lock) {
+ row->trx_state = "LOCK WAIT";
+ } else {
+ row->trx_state = "RUNNING";
}
row->requested_lock_row = requested_lock_row;
ut_ad(requested_lock_row == NULL
|| i_s_locks_row_validate(requested_lock_row));
- if (trx->lock.wait_lock != NULL) {
+ ut_ad(!wait_lock == !requested_lock_row);
- ut_a(requested_lock_row != NULL);
- row->trx_wait_started = trx->lock.wait_started;
- } else {
- ut_a(requested_lock_row == NULL);
- row->trx_wait_started = 0;
- }
+ row->trx_wait_started = wait_lock
+ ? hrtime_to_time(trx->lock.suspend_time)
+ : 0;
row->trx_weight = static_cast<uintmax_t>(TRX_WEIGHT(trx));
@@ -749,7 +743,7 @@ static bool fill_locks_row(
if (lock_type == LOCK_REC) {
row->lock_index = ha_storage_put_str_memlim(
- cache->storage, lock_rec_get_index_name(lock),
+ cache->storage, lock_rec_get_index(lock)->name,
MAX_ALLOWED_FOR_STORAGE(cache));
/* memory could not be allocated */
@@ -1063,21 +1057,18 @@ add_trx_relevant_locks_to_cache(
/* If transaction is waiting we add the wait lock and all locks
from another transactions that are blocking the wait lock. */
- if (trx->lock.que_state == TRX_QUE_LOCK_WAIT) {
+ if (const lock_t *wait_lock = trx->lock.wait_lock) {
const lock_t* curr_lock;
i_s_locks_row_t* blocking_lock_row;
lock_queue_iterator_t iter;
- ut_a(trx->lock.wait_lock != NULL);
-
uint16_t wait_lock_heap_no
- = wait_lock_get_heap_no(trx->lock.wait_lock);
+ = wait_lock_get_heap_no(wait_lock);
/* add the requested lock */
- *requested_lock_row
- = add_lock_to_cache(cache, trx->lock.wait_lock,
- wait_lock_heap_no);
+ *requested_lock_row = add_lock_to_cache(cache, wait_lock,
+ wait_lock_heap_no);
/* memory could not be allocated */
if (*requested_lock_row == NULL) {
@@ -1088,18 +1079,16 @@ add_trx_relevant_locks_to_cache(
/* then iterate over the locks before the wait lock and
add the ones that are blocking it */
- lock_queue_iterator_reset(&iter, trx->lock.wait_lock,
- ULINT_UNDEFINED);
+ lock_queue_iterator_reset(&iter, wait_lock, ULINT_UNDEFINED);
for (curr_lock = lock_queue_iterator_get_prev(&iter);
curr_lock != NULL;
curr_lock = lock_queue_iterator_get_prev(&iter)) {
- if (lock_has_to_wait(trx->lock.wait_lock,
- curr_lock)) {
+ if (lock_has_to_wait(wait_lock, curr_lock)) {
/* add the lock that is
- blocking trx->lock.wait_lock */
+ blocking wait_lock */
blocking_lock_row
= add_lock_to_cache(
cache, curr_lock,
diff --git a/storage/innobase/trx/trx0purge.cc b/storage/innobase/trx/trx0purge.cc
index 9fbb90c3bd6..4b2185ed7ab 100644
--- a/storage/innobase/trx/trx0purge.cc
+++ b/storage/innobase/trx/trx0purge.cc
@@ -1152,8 +1152,6 @@ trx_purge_attach_undo_recs(ulint n_purge_threads)
purge_node_t* node;
trx_purge_rec_t purge_rec;
- ut_a(!thr->is_active);
-
/* Get the purge node. */
node = (purge_node_t*) thr->child;
ut_a(que_node_get_type(node) == QUE_NODE_PURGE);
diff --git a/storage/innobase/trx/trx0roll.cc b/storage/innobase/trx/trx0roll.cc
index 73f2bbd7372..91bc4b8b69d 100644
--- a/storage/innobase/trx/trx0roll.cc
+++ b/storage/innobase/trx/trx0roll.cc
@@ -90,7 +90,6 @@ inline bool trx_t::rollback_finish()
undo= nullptr;
}
commit_low();
- lock.que_state= TRX_QUE_RUNNING;
return false;
}
@@ -143,7 +142,6 @@ inline void trx_t::rollback_low(trx_savept_t *savept)
mod_tables.erase(j);
}
}
- lock.que_state= TRX_QUE_RUNNING;
MONITOR_INC(MONITOR_TRX_ROLLBACK_SAVEPOINT);
}
@@ -640,8 +638,6 @@ trx_rollback_active(
goto func_exit;
}
- ut_a(trx->lock.que_state == TRX_QUE_RUNNING);
-
if (!dictionary_locked || !trx->table_id) {
} else if (dict_table_t* table = dict_table_open_on_id(
trx->table_id, TRUE, DICT_TABLE_OP_NORMAL)) {
@@ -912,8 +908,6 @@ trx_rollback_start(
trx->graph = roll_graph;
- trx->lock.que_state = TRX_QUE_ROLLING_BACK;
-
return(que_fork_start_command(roll_graph));
}
diff --git a/storage/innobase/trx/trx0trx.cc b/storage/innobase/trx/trx0trx.cc
index 7f704634a42..31cf701254d 100644
--- a/storage/innobase/trx/trx0trx.cc
+++ b/storage/innobase/trx/trx0trx.cc
@@ -143,8 +143,6 @@ trx_init(
trx->magic_n = TRX_MAGIC_N;
- trx->lock.que_state = TRX_QUE_RUNNING;
-
trx->last_sql_stat_start.least_undo_no = 0;
ut_ad(!trx->read_view.is_open());
@@ -186,6 +184,7 @@ struct TrxFactory {
trx->lock.lock_heap = mem_heap_create_typed(
1024, MEM_HEAP_FOR_LOCK_HEAP);
+ mysql_cond_init(0, &trx->lock.cond, nullptr);
lock_trx_lock_list_init(&trx->lock.trx_locks);
@@ -227,6 +226,8 @@ struct TrxFactory {
trx->lock.lock_heap = NULL;
}
+ mysql_cond_destroy(&trx->lock.cond);
+
ut_a(UT_LIST_GET_LEN(trx->lock.trx_locks) == 0);
ut_ad(UT_LIST_GET_LEN(trx->lock.evicted_tables) == 0);
@@ -1573,19 +1574,7 @@ trx_commit_or_rollback_prepare(
case TRX_STATE_ACTIVE:
case TRX_STATE_PREPARED:
case TRX_STATE_PREPARED_RECOVERED:
- /* If the trx is in a lock wait state, moves the waiting
- query thread to the suspended state */
-
- if (trx->lock.que_state == TRX_QUE_LOCK_WAIT) {
-
- ut_a(trx->lock.wait_thr != NULL);
- trx->lock.wait_thr->state = QUE_THR_COMMAND_WAIT;
- trx->lock.wait_thr = NULL;
-
- trx->lock.que_state = TRX_QUE_RUNNING;
- }
-
- ut_ad(trx->lock.n_active_thrs == 1);
+ trx->lock.wait_thr = NULL;
return;
case TRX_STATE_COMMITTED_IN_MEMORY:
@@ -1638,13 +1627,11 @@ trx_commit_step(
trx = thr_get_trx(thr);
ut_a(trx->lock.wait_thr == NULL);
- ut_a(trx->lock.que_state != TRX_QUE_LOCK_WAIT);
trx_commit_or_rollback_prepare(trx);
trx->commit();
ut_ad(trx->lock.wait_thr == NULL);
- trx->lock.que_state = TRX_QUE_RUNNING;
thr = NULL;
} else {
@@ -1814,19 +1801,12 @@ state_ok:
newline = TRUE;
- /* trx->lock.que_state of an ACTIVE transaction may change
- while we are not holding trx->mutex. We perform a dirty read
- for performance reasons. */
-
- switch (trx->lock.que_state) {
- case TRX_QUE_RUNNING:
- newline = FALSE; break;
- case TRX_QUE_LOCK_WAIT:
- fputs("LOCK WAIT ", f); break;
- case TRX_QUE_ROLLING_BACK:
- fputs("ROLLING BACK ", f); break;
- default:
- fprintf(f, "que state %lu ", (ulong) trx->lock.que_state);
+ if (trx->in_rollback) { /* dirty read for performance reasons */
+ fputs("ROLLING BACK ", f);
+ } else if (trx->lock.wait_lock) {
+ fputs("LOCK WAIT ", f);
+ } else {
+ newline = FALSE;
}
if (n_trx_locks > 0 || heap_size > 400) {