1
0
mirror of https://github.com/MariaDB/server.git synced 2025-07-13 02:22:51 +03:00

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.
This commit is contained in:
Marko Mäkelä
2021-01-26 16:39:56 +02:00
parent 7f1ab8f742
commit e71e613353
25 changed files with 601 additions and 1527 deletions

View File

@ -1921,7 +1921,7 @@ struct find_interesting_trx
return; return;
if (trx.mysql_thd == nullptr) if (trx.mysql_thd == nullptr)
return; return;
if (withdraw_started <= trx.start_time) if (withdraw_started <= trx.start_time_micro)
return; return;
if (!found) if (!found)
@ -1939,8 +1939,9 @@ struct find_interesting_trx
} }
bool &found; bool &found;
time_t withdraw_started; /** microsecond_interval_timer() */
time_t current_time; const ulonglong withdraw_started;
const my_hrtime_t current_time;
}; };
} // namespace } // namespace
@ -2004,8 +2005,8 @@ inline void buf_pool_t::resize()
buf_resize_status("Withdrawing blocks to be shrunken."); buf_resize_status("Withdrawing blocks to be shrunken.");
time_t withdraw_started = time(NULL); ulonglong withdraw_started = microsecond_interval_timer();
double message_interval = 60; ulonglong message_interval = 60ULL * 1000 * 1000;
ulint retry_interval = 1; ulint retry_interval = 1;
withdraw_retry: withdraw_retry:
@ -2021,24 +2022,25 @@ withdraw_retry:
/* abort buffer pool load */ /* abort buffer pool load */
buf_load_abort(); buf_load_abort();
const time_t current_time = time(NULL); const ulonglong current_time = microsecond_interval_timer();
if (should_retry_withdraw if (should_retry_withdraw
&& difftime(current_time, withdraw_started) >= message_interval) { && current_time - withdraw_started >= message_interval) {
if (message_interval > 900) { if (message_interval > 900000000) {
message_interval = 1800; message_interval = 1800000000;
} else { } else {
message_interval *= 2; message_interval *= 2;
} }
lock_sys.mutex_lock(); bool found= false;
bool found = false; find_interesting_trx f
trx_sys.trx_list.for_each(find_interesting_trx{ {found, withdraw_started, my_hrtime_coarse()};
found, withdraw_started, current_time});
lock_sys.mutex_unlock();
withdraw_started = current_time; withdraw_started = current_time;
lock_sys.mutex_lock();
trx_sys.trx_list.for_each(f);
lock_sys.mutex_unlock();
} }
if (should_retry_withdraw) { if (should_retry_withdraw) {

View File

@ -4428,8 +4428,6 @@ static int innobase_close_connection(handlerton *hton, THD *thd)
return 0; return 0;
} }
void lock_cancel_waiting_and_release(lock_t *lock);
/** Cancel any pending lock request associated with the current THD. /** Cancel any pending lock request associated with the current THD.
@sa THD::awake() @sa ha_kill_query() */ @sa THD::awake() @sa ha_kill_query() */
static void innobase_kill_query(handlerton*, THD *thd, enum thd_kill_levels) 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. */ Also, BF thread should own trx mutex for the victim. */
DBUG_VOID_RETURN; DBUG_VOID_RETURN;
#endif /* WITH_WSREP */ #endif /* WITH_WSREP */
if (trx->lock.wait_lock)
{
lock_sys.mutex_lock(); lock_sys.mutex_lock();
mysql_mutex_lock(&lock_sys.wait_mutex);
if (lock_t *lock= trx->lock.wait_lock) if (lock_t *lock= trx->lock.wait_lock)
{ {
trx->mutex.wr_lock(); trx->mutex.wr_lock();
trx->error_state= DB_INTERRUPTED;
lock_cancel_waiting_and_release(lock); lock_cancel_waiting_and_release(lock);
trx->mutex.wr_unlock(); trx->mutex.wr_unlock();
} }
lock_sys.mutex_unlock(); lock_sys.mutex_unlock();
mysql_mutex_unlock(&lock_sys.wait_mutex);
}
} }
DBUG_VOID_RETURN; 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); wsrep_thd_UNLOCK(thd);
DEBUG_SYNC(bf_thd, "before_wsrep_thd_abort"); DEBUG_SYNC(bf_thd, "before_wsrep_thd_abort");
if (wsrep_thd_bf_abort(bf_thd, thd, signal)) if (!wsrep_thd_bf_abort(bf_thd, thd, signal)) {
{ } else if (victim_trx->lock.wait_lock) {
lock_t* wait_lock = victim_trx->lock.wait_lock; mysql_mutex_lock(&lock_sys.wait_mutex);
if (wait_lock) { if (lock_t* wait_lock = victim_trx->lock.wait_lock) {
DBUG_ASSERT(victim_trx->is_wsrep()); DBUG_ASSERT(victim_trx->is_wsrep());
WSREP_DEBUG("victim has wait flag: %lu", WSREP_DEBUG("victim has wait flag: %lu",
thd_get_thread_id(thd)); 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; victim_trx->lock.was_chosen_as_deadlock_victim= TRUE;
lock_cancel_waiting_and_release(wait_lock); lock_cancel_waiting_and_release(wait_lock);
} }
mysql_mutex_unlock(&lock_sys.wait_mutex);
} }
DBUG_RETURN(0); DBUG_RETURN(0);

View File

@ -7083,8 +7083,6 @@ op_ok:
row_mysql_unlock_data_dictionary(ctx->trx); row_mysql_unlock_data_dictionary(ctx->trx);
dict_locked = false; dict_locked = false;
ut_ad(!ctx->trx->lock.n_active_thrs);
if (ctx->old_table->fts) { if (ctx->old_table->fts) {
fts_sync_during_ddl(ctx->old_table); fts_sync_during_ddl(ctx->old_table);
} }

View File

@ -500,9 +500,9 @@ lock_print_info_summary(
/** Prints transaction lock wait and MVCC state. /** Prints transaction lock wait and MVCC state.
@param[in,out] file file where to print @param[in,out] file file where to print
@param[in] trx transaction @param[in] trx transaction
@param[in] now current time */ @param[in] now current my_hrtime_coarse() */
void void lock_trx_print_wait_and_mvcc_state(FILE *file, const trx_t *trx,
lock_trx_print_wait_and_mvcc_state(FILE* file, const trx_t* trx, time_t now); my_hrtime_t now);
/*********************************************************************//** /*********************************************************************//**
Prints info of locks for each transaction. This function assumes that the Prints info of locks for each transaction. This function assumes that the
@ -554,15 +554,6 @@ lock_rec_get_index(
/*===============*/ /*===============*/
const lock_t* lock); /*!< in: lock */ 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. Check if there are any locks (table or rec) against table.
@return TRUE if locks exist */ @return TRUE if locks exist */
@ -573,29 +564,12 @@ lock_table_has_locks(
held on records in this table or on the held on records in this table or on the
table itself */ table itself */
/** A task which wakes up threads whose lock wait may have lasted too long */ /** Wait for a lock to be released.
void lock_wait_timeout_task(void*); @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
Releases a user OS thread waiting for a lock to be released, if the @retval DB_SUCCESS if the lock was granted */
thread is already suspended. */ dberr_t lock_wait(que_thr_t *thr);
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 */
/*********************************************************************//** /*********************************************************************//**
Unlocks AUTO_INC type locks that were possibly reserved by a trx. This 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 function should be called at the the end of an SQL statement, by the
@ -684,20 +658,18 @@ public:
/** page locks for SPATIAL INDEX */ /** page locks for SPATIAL INDEX */
hash_table_t prdt_page_hash; 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; MY_ALIGNED(CACHE_LINE_SIZE) mysql_mutex_t wait_mutex;
srv_slot_t* waiting_threads; /*!< Array of user threads private:
suspended while waiting for /** Pending number of lock waits; protected by wait_mutex */
locks within InnoDB */ ulint wait_pending;
srv_slot_t* last_slot; /*!< highest slot ever used /** Cumulative number of lock waits; protected by wait_mutex */
in the waiting_threads array */ ulint wait_count;
/** Cumulative wait time; protected by wait_mutex */
ulint n_lock_max_wait_time; /*!< Max wait time */ ulint wait_time;
/** Longest wait time; protected by wait_mutex */
std::unique_ptr<tpool::timer> timeout_timer; /*!< Thread pool timer task */ ulint wait_time_max;
bool timeout_timer_active; public:
/** /**
Constructor. Constructor.
@ -752,6 +724,22 @@ public:
/** Closes the lock system at database shutdown. */ /** Closes the lock system at database shutdown. */
void close(); 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 */ /** @return the hash value for a page address */
ulint hash(const page_id_t id) const ulint hash(const page_id_t id) const
{ mysql_mutex_assert_owner(&mutex); return rec_hash.calc_hash(id.fold()); } { 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 */ /** The lock system */
extern lock_sys_t lock_sys; extern lock_sys_t lock_sys;
#ifdef WITH_WSREP /** Cancel a waiting lock request and release possibly waiting transactions */
/*********************************************************************//** void lock_cancel_waiting_and_release(lock_t *lock);
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 */
#include "lock0lock.ic" #include "lock0lock.ic"

View File

@ -605,32 +605,6 @@ lock_table_has(
const dict_table_t* table, /*!< in: table */ const dict_table_t* table, /*!< in: table */
enum lock_mode mode); /*!< in: lock mode */ 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" #include "lock0priv.ic"
#endif /* lock0priv_h */ #endif /* lock0priv_h */

View File

@ -88,43 +88,14 @@ que_graph_free(
to this graph: if not, then use to this graph: if not, then use
que_graph_free_recursive and free the heap que_graph_free_recursive and free the heap
afterwards! */ 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. */ Run a query thread. Handles lock waits. */
void void
que_run_threads( que_run_threads(
/*============*/ /*============*/
que_thr_t* thr); /*!< in: query thread */ 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 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 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 can be chosen, a situation which may arise in parallelized fetches, NULL
@ -228,17 +199,6 @@ ulint
que_node_list_get_len( que_node_list_get_len(
/*==================*/ /*==================*/
que_node_t* node_list); /*!< in: node list, or NULL */ 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 Evaluate the given SQL
@return error code or DB_SUCCESS */ @return error code or DB_SUCCESS */
@ -265,13 +225,11 @@ que_fork_scheduler_round_robin(
/** Query thread states */ /** Query thread states */
enum que_thr_state_t { enum que_thr_state_t {
QUE_THR_RUNNING,
/** in selects this means that the thread is at the end of its /** 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 result set (or start, in case of a scroll cursor); in other
statements, this means the thread has done its task */ statements, this means the thread has done its task */
QUE_THR_COMPLETED, QUE_THR_COMPLETED,
QUE_THR_COMMAND_WAIT, QUE_THR_RUNNING
QUE_THR_LOCK_WAIT
}; };
/** Query thread lock states */ /** Query thread lock states */
@ -289,7 +247,6 @@ struct que_thr_t{
que_node_t* child; /*!< graph child node */ que_node_t* child; /*!< graph child node */
que_t* graph; /*!< graph where this node belongs */ que_t* graph; /*!< graph where this node belongs */
que_thr_state_t state; /*!< state of the query thread */ 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 /* The following fields are private to the OS thread executing the
query thread, and are not protected by any mutex: */ query thread, and are not protected by any mutex: */
@ -303,9 +260,6 @@ struct que_thr_t{
thus far */ thus far */
ulint lock_state; /*!< lock state of thread (table or ulint lock_state; /*!< lock state of thread (table or
row) */ 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 /* The following fields are links for the various lists that
this type can be on. */ this type can be on. */
@ -320,39 +274,12 @@ struct que_thr_t{
related delete/updates */ related delete/updates */
row_prebuilt_t* prebuilt; /*!< prebuilt structure processed by row_prebuilt_t* prebuilt; /*!< prebuilt structure processed by
the query thread */ 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 */ /* Query graph fork node: its fields are protected by the query thread mutex */
struct que_fork_t{ struct que_fork_t{
que_common_t common; /*!< type: QUE_NODE_FORK */ que_common_t common; /*!< type: QUE_NODE_FORK */
que_t* graph; /*!< query graph of this node */ 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 trx_t* trx; /*!< transaction: this is set only in
the root node */ the root node */
ulint state; /*!< state of the fork 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 */ /* Query fork (or graph) states */
#define QUE_FORK_ACTIVE 1 #define QUE_FORK_ACTIVE 1
#define QUE_FORK_COMMAND_WAIT 2 #define QUE_FORK_COMMAND_WAIT 2

View File

@ -243,32 +243,3 @@ que_node_get_parent(
{ {
return(((que_common_t*) node)->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);
}

View File

@ -146,16 +146,6 @@ struct srv_stats_t
/** Number of data read in total (in bytes) */ /** Number of data read in total (in bytes) */
ulint_ctr_1_t data_read; 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. */ /** Number of rows read. */
ulint_ctr_64_t n_rows_read; ulint_ctr_64_t n_rows_read;
@ -855,30 +845,6 @@ struct export_var_t{
int64_t innodb_key_rotation_list_length; 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 tpool::thread_pool *srv_thread_pool;
extern std::unique_ptr<tpool::timer> srv_master_timer; extern std::unique_ptr<tpool::timer> srv_master_timer;
extern std::unique_ptr<tpool::timer> srv_monitor_timer; extern std::unique_ptr<tpool::timer> srv_monitor_timer;

View File

@ -416,48 +416,19 @@ The transaction must have mysql_thd assigned. */
typedef std::vector<ib_lock_t*, ut_allocator<ib_lock_t*> > lock_list; 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 /** The locks and state of an active transaction. Protected by
lock_sys.mutex, trx->mutex or both. */ lock_sys.mutex, trx->mutex or both. */
struct trx_lock_t { struct trx_lock_t {
#ifdef UNIV_DEBUG /** Lock request being waited for.
/** number of active query threads; at most 1, except for the Set to nonnull when holding lock_sys.mutex, lock_sys.wait_mutex and
dummy transaction in trx_purge() */ trx->mutex, by the thread that is executing the transaction.
ulint n_active_thrs; Set to nullptr when holding lock_sys.wait_mutex. */
#endif Atomic_relaxed<lock_t*> wait_lock;
trx_que_t que_state; /*!< valid when trx->state /** condition variable for !wait_lock; used with lock_sys.wait_mutex */
== TRX_STATE_ACTIVE: TRX_QUE_RUNNING, mysql_cond_t cond;
TRX_QUE_LOCK_WAIT, ... */ /** lock wait start time, protected only by lock_sys.wait_mutex */
my_hrtime_t suspend_time;
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 */
ib_uint64_t deadlock_mark; /*!< A mark field that is initialized ib_uint64_t deadlock_mark; /*!< A mark field that is initialized
to and checked against lock_mark_counter to and checked against lock_mark_counter
by lock_deadlock_recursive(). */ by lock_deadlock_recursive(). */
@ -468,11 +439,8 @@ struct trx_lock_t {
transaction as a victim in deadlock transaction as a victim in deadlock
resolution, it sets this to true. resolution, it sets this to true.
Protected by trx->mutex. */ 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 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 state. For threads suspended in a
lock wait, this is protected by lock wait, this is protected by
lock_sys.mutex. Otherwise, this may lock_sys.mutex. Otherwise, this may
@ -1067,6 +1035,7 @@ public:
#endif #endif
ut_ad(!read_view.is_open()); ut_ad(!read_view.is_open());
ut_ad(!lock.wait_thr); ut_ad(!lock.wait_thr);
ut_ad(!lock.wait_lock);
ut_ad(UT_LIST_GET_LEN(lock.trx_locks) == 0); ut_ad(UT_LIST_GET_LEN(lock.trx_locks) == 0);
ut_ad(lock.table_locks.empty()); ut_ad(lock.table_locks.empty());
ut_ad(!autoinc_locks || ib_vector_is_empty(autoinc_locks)); ut_ad(!autoinc_locks || ib_vector_is_empty(autoinc_locks));

View File

@ -48,14 +48,6 @@ static const ulint TRX_MAGIC_N = 91118598;
constexpr uint innodb_purge_threads_MAX= 32; 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) */ /** Transaction states (trx_t::state) */
enum trx_state_t { enum trx_state_t {
TRX_STATE_NOT_STARTED, TRX_STATE_NOT_STARTED,

View File

@ -53,19 +53,6 @@ Created 5/7/1996 Heikki Tuuri
/** The value of innodb_deadlock_detect */ /** The value of innodb_deadlock_detect */
my_bool innobase_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" 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_wait_reports(const MYSQL_THD thd);
extern "C" int thd_need_ordering_with(const MYSQL_THD thd, const MYSQL_THD other_thd); extern "C" int thd_need_ordering_with(const MYSQL_THD thd, const MYSQL_THD other_thd);
@ -367,16 +354,10 @@ lock_check_trx_id_sanity(
void lock_sys_t::create(ulint n_cells) 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_mutex_key, &mutex, nullptr);
mysql_mutex_init(lock_wait_mutex_key, &wait_mutex, nullptr); mysql_mutex_init(lock_wait_mutex_key, &wait_mutex, nullptr);
@ -384,11 +365,11 @@ void lock_sys_t::create(ulint n_cells)
prdt_hash.create(n_cells); prdt_hash.create(n_cells);
prdt_page_hash.create(n_cells); prdt_page_hash.create(n_cells);
if (!srv_read_only_mode) { if (!srv_read_only_mode)
lock_latest_err_file = os_file_create_tmpfile(); {
lock_latest_err_file= os_file_create_tmpfile();
ut_a(lock_latest_err_file); ut_a(lock_latest_err_file);
} }
timeout_timer_active = false;
} }
@ -448,11 +429,13 @@ void lock_sys_t::close()
{ {
ut_ad(this == &lock_sys); ut_ad(this == &lock_sys);
if (!m_initialised) return; if (!m_initialised)
return;
if (lock_latest_err_file != NULL) { if (lock_latest_err_file)
{
my_fclose(lock_latest_err_file, MYF(MY_WME)); my_fclose(lock_latest_err_file, MYF(MY_WME));
lock_latest_err_file = NULL; lock_latest_err_file= nullptr;
} }
rec_hash.free(); rec_hash.free();
@ -462,11 +445,6 @@ void lock_sys_t::close()
mysql_mutex_destroy(&mutex); mysql_mutex_destroy(&mutex);
mysql_mutex_destroy(&wait_mutex); mysql_mutex_destroy(&wait_mutex);
for (ulint i = srv_max_n_threads; i--; ) {
mysql_cond_destroy(&waiting_threads[i].cond);
}
ut_free(waiting_threads);
m_initialised= false; m_initialised= false;
} }
@ -956,7 +934,7 @@ wsrep_kill_victim(
(wsrep_thd_order_before( (wsrep_thd_order_before(
trx->mysql_thd, lock->trx->mysql_thd))) { 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)) { if (UNIV_UNLIKELY(wsrep_debug)) {
ib::info() << "WSREP: BF victim waiting\n"; ib::info() << "WSREP: BF victim waiting\n";
} }
@ -1149,6 +1127,34 @@ wsrep_print_wait_locks(
} }
#endif /* WITH_WSREP */ #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, /** Create a new record lock and inserts it to the lock queue,
without checking for deadlocks or conflicts. without checking for deadlocks or conflicts.
@param[in] type_mode lock mode and wait flag; type will be replaced @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, * delayed conflict resolution '...kill_one_trx' was not called,
* if victim was waiting for some other lock * 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(); 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; c_lock->trx->lock.was_chosen_as_deadlock_victim = TRUE;
@ -1276,12 +1289,10 @@ lock_rec_create_low(
wsrep_print_wait_locks(c_lock); wsrep_print_wait_locks(c_lock);
} }
trx->lock.que_state = TRX_QUE_LOCK_WAIT;
lock_set_lock_and_trx_wait(lock, trx); lock_set_lock_and_trx_wait(lock, trx);
UT_LIST_ADD_LAST(trx->lock.trx_locks, lock); UT_LIST_ADD_LAST(trx->lock.trx_locks, lock);
trx->lock.wait_thr = thr; trx->lock.wait_thr = thr;
thr->state = QUE_THR_LOCK_WAIT;
/* have to release trx mutex for the duration of /* have to release trx mutex for the duration of
victim lock release. This will eventually call victim lock release. This will eventually call
@ -1300,8 +1311,10 @@ lock_rec_create_low(
c_lock->trx->mutex.wr_unlock(); c_lock->trx->mutex.wr_unlock();
/* have to bail out here to avoid lock_set_lock... */ /* have to bail out here to avoid lock_set_lock... */
mysql_mutex_unlock(&lock_sys.wait_mutex);
return(lock); return(lock);
} }
mysql_mutex_unlock(&lock_sys.wait_mutex);
c_lock->trx->mutex.wr_unlock(); c_lock->trx->mutex.wr_unlock();
} else } else
#endif /* WITH_WSREP */ #endif /* WITH_WSREP */
@ -1312,7 +1325,9 @@ lock_rec_create_low(
trx->mutex.wr_lock(); trx->mutex.wr_lock();
} }
if (type_mode & LOCK_WAIT) { 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); UT_LIST_ADD_LAST(trx->lock.trx_locks, lock);
if (!holds_trx_mutex) { if (!holds_trx_mutex) {
@ -1360,8 +1375,6 @@ lock_rec_enqueue_waiting(
trx_t* trx = thr_get_trx(thr); trx_t* trx = thr_get_trx(thr);
ut_a(!que_thr_stop(thr));
switch (trx_get_dict_operation(trx)) { switch (trx_get_dict_operation(trx)) {
case TRX_DICT_OP_NONE: case TRX_DICT_OP_NONE:
break; break;
@ -1396,6 +1409,10 @@ lock_rec_enqueue_waiting(
if (ut_d(const trx_t* victim =) if (ut_d(const trx_t* victim =)
DeadlockChecker::check_and_resolve(lock, trx)) { DeadlockChecker::check_and_resolve(lock, trx)) {
ut_ad(victim == 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_reset_lock_and_trx_wait(lock);
lock_rec_reset_nth_bit(lock, heap_no); lock_rec_reset_nth_bit(lock, heap_no);
return DB_DEADLOCK; return DB_DEADLOCK;
@ -1414,12 +1431,9 @@ lock_rec_enqueue_waiting(
return DB_SUCCESS_LOCKED_REC; 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.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) DBUG_LOG("ib_lock", "trx " << ib::hex(trx->id)
<< " waits for lock in index " << index->name << " waits for lock in index " << index->name
@ -1546,7 +1560,7 @@ lock_rec_add_to_queue(
lock != NULL; lock != NULL;
lock = lock_rec_get_next_on_page(lock)) { 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)) { && lock_rec_get_nth_bit(lock, heap_no)) {
goto create; goto create;
@ -1709,7 +1723,7 @@ lock_rec_has_to_wait_in_queue(
ut_ad(wait_lock); ut_ad(wait_lock);
lock_sys.mutex_assert_locked(); 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); ut_ad(lock_get_type_low(wait_lock) == LOCK_REC);
heap_no = lock_rec_find_set_bit(wait_lock); heap_no = lock_rec_find_set_bit(wait_lock);
@ -1733,52 +1747,48 @@ lock_rec_has_to_wait_in_queue(
return(NULL); 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. */ /** Resume a lock wait */
static void lock_grant_after_reset(lock_t* lock) static void lock_wait_end(trx_t *trx)
{ {
lock_sys.mutex_assert_locked(); mysql_mutex_assert_owner(&lock_sys.wait_mutex);
if (lock_get_mode(lock) == LOCK_AUTO_INC) { que_thr_t *thr= trx->lock.wait_thr;
dict_table_t* table = lock->un_member.tab_lock.table; ut_ad(thr);
if (trx->lock.was_chosen_as_deadlock_victim)
if (table->autoinc_trx == lock->trx) { {
ib::error() << "Transaction already had an" trx->error_state= DB_DEADLOCK;
<< " AUTO-INC lock!"; trx->lock.was_chosen_as_deadlock_victim= false;
} else {
table->autoinc_trx = lock->trx;
ib_vector_push(lock->trx->autoinc_locks, &lock);
}
}
DBUG_PRINT("ib_lock", ("wait for trx " TRX_ID_FMT " ends",
lock->trx->id));
/* If we are resolving a deadlock by choosing another transaction
as a victim, then our original transaction may not be in the
TRX_QUE_LOCK_WAIT state, and there is no need to end the lock wait
for it */
if (lock->trx->lock.que_state == TRX_QUE_LOCK_WAIT) {
que_thr_t* thr;
thr = que_thr_end_lock_wait(lock->trx);
if (thr != NULL) {
lock_wait_release_thread_if_suspended(thr);
}
} }
trx->lock.wait_thr= nullptr;
mysql_cond_signal(&trx->lock.cond);
} }
/** Grant a lock to a waiting lock request and release the waiting transaction. */ /** Grant a waiting lock request and release the waiting transaction. */
static void lock_grant(lock_t* lock) 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); lock_reset_lock_and_trx_wait(lock);
auto mutex= &lock->trx->mutex; trx_t *trx= lock->trx;
mutex->wr_lock(); trx->mutex.wr_lock();
lock_grant_after_reset(lock); if (lock->mode() == LOCK_AUTO_INC)
mutex->wr_unlock(); {
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);
}
DBUG_PRINT("ib_lock", ("wait for trx " TRX_ID_FMT " ends", trx->id));
/* If we are resolving a deadlock by choosing another transaction as
a victim, then our original transaction may not be waiting anymore */
if (trx->lock.wait_thr)
lock_wait_end(trx);
trx->mutex.wr_unlock();
} }
/*************************************************************//** /*************************************************************//**
@ -1799,16 +1809,15 @@ lock_rec_cancel(
/* Reset the wait flag and the back pointer to lock in trx */ /* 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); lock_reset_lock_and_trx_wait(lock);
/* The following releases the trx from lock wait */ /* The following releases the trx from lock wait */
trx_t *trx = lock->trx; trx_t *trx = lock->trx;
auto mutex = &trx->mutex; trx->mutex.wr_lock();
mutex->wr_lock(); lock_wait_end(trx);
if (que_thr_t* thr = que_thr_end_lock_wait(trx)) { mysql_mutex_unlock(&lock_sys.wait_mutex);
lock_wait_release_thread_if_suspended(thr); trx->mutex.wr_unlock();
}
mutex->wr_unlock();
} }
/** Remove a record lock request, waiting or granted, from the queue and /** 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; hash_table_t* lock_hash;
lock_sys.mutex_assert_locked(); lock_sys.mutex_assert_locked();
mysql_mutex_assert_owner(&lock_sys.wait_mutex);
ut_ad(lock_get_type_low(in_lock) == LOCK_REC); ut_ad(lock_get_type_low(in_lock) == LOCK_REC);
/* We may or may not be holding in_lock->trx->mutex here. */ /* We may or may not be holding in_lock->trx->mutex here. */
@ -2092,12 +2102,12 @@ lock_rec_move_low(
lock != NULL; lock != NULL;
lock = lock_rec_get_next(donator_heap_no, lock)) { 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); lock_rec_reset_nth_bit(lock, donator_heap_no);
const auto type_mode = lock->type_mode;
if (type_mode & LOCK_WAIT) { 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: /* Note that we FIRST reset the bit, and then set the lock:
@ -2213,9 +2223,9 @@ lock_move_reorganize_page(
/* Reset bitmap of lock */ /* Reset bitmap of lock */
lock_rec_bitmap_reset(lock); lock_rec_bitmap_reset(lock);
if (lock_get_wait(lock)) { if (lock->is_waiting()) {
ut_ad(lock->trx->lock.wait_lock == lock);
lock_reset_lock_and_trx_wait(lock); lock->type_mode &= ~LOCK_WAIT;
} }
lock = lock_rec_get_next_on_page(lock); lock = lock_rec_get_next_on_page(lock);
@ -2394,7 +2404,8 @@ lock_move_rec_list_end(
ut_ad(!page_rec_is_metadata(orec)); ut_ad(!page_rec_is_metadata(orec));
if (type_mode & LOCK_WAIT) { 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( lock_rec_add_to_queue(
@ -2496,7 +2507,8 @@ lock_move_rec_list_start(
ut_ad(!page_rec_is_metadata(prev)); ut_ad(!page_rec_is_metadata(prev));
if (type_mode & LOCK_WAIT) { 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( 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 if (rec1_heap_no < lock->un_member.rec_lock.n_bits
&& lock_rec_reset_nth_bit(lock, rec1_heap_no)) { && lock_rec_reset_nth_bit(lock, rec1_heap_no)) {
if (type_mode & LOCK_WAIT) { 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( lock_rec_add_to_queue(
@ -3098,9 +3111,11 @@ lock_table_create(
ut_list_append(table->locks, lock, TableLockGetNode()); ut_list_append(table->locks, lock, TableLockGetNode());
} }
trx->mutex.wr_unlock();
mysql_mutex_lock(&lock_sys.wait_mutex);
c_lock->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; c_lock->trx->lock.was_chosen_as_deadlock_victim = TRUE;
if (UNIV_UNLIKELY(wsrep_debug)) { if (UNIV_UNLIKELY(wsrep_debug)) {
@ -3111,20 +3126,21 @@ lock_table_create(
/* The lock release will call lock_grant(), /* The lock release will call lock_grant(),
which would acquire trx->mutex again. */ which would acquire trx->mutex again. */
trx->mutex.wr_unlock();
lock_cancel_waiting_and_release( lock_cancel_waiting_and_release(
c_lock->trx->lock.wait_lock); c_lock->trx->lock.wait_lock);
trx->mutex.wr_lock();
} }
mysql_mutex_unlock(&lock_sys.wait_mutex);
c_lock->trx->mutex.wr_unlock(); c_lock->trx->mutex.wr_unlock();
trx->mutex.wr_lock();
} else } else
#endif /* WITH_WSREP */ #endif /* WITH_WSREP */
ut_list_append(table->locks, lock, TableLockGetNode()); ut_list_append(table->locks, lock, TableLockGetNode());
if (type_mode & LOCK_WAIT) { if (type_mode & LOCK_WAIT) {
ut_ad(!trx->lock.wait_lock
lock_set_lock_and_trx_wait(lock, trx); || (*trx->lock.wait_lock).trx == trx);
trx->lock.wait_lock = lock;
} }
lock->trx->lock.table_locks.push_back(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 /* Remove the table from the transaction's AUTOINC vector, if
the lock that is being released is an AUTOINC lock. */ 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) { if (table->autoinc_trx == trx) {
table->autoinc_trx = NULL; table->autoinc_trx = NULL;
}
/* The locks must be freed in the reverse order from /* The locks must be freed in the reverse order from
the one in which they were acquired. This is to avoid the one in which they were acquired. This is to avoid
traversing the AUTOINC lock vector unnecessarily. traversing the AUTOINC lock vector unnecessarily.
We only store locks that were granted in the We only store locks that were granted in the
trx->autoinc_locks vector (see lock_table_create() trx->autoinc_locks vector (see lock_table_create()
and lock_grant()). Therefore it can be empty and we and lock_grant()). */
need to check for that. */
if (!lock_get_wait(lock)
&& !ib_vector_is_empty(trx->autoinc_locks)) {
lock_table_remove_autoinc_lock(lock, trx); lock_table_remove_autoinc_lock(lock, trx);
} }
@ -3292,7 +3300,6 @@ lock_table_enqueue_waiting(
ut_ad(!srv_read_only_mode); ut_ad(!srv_read_only_mode);
trx = thr_get_trx(thr); trx = thr_get_trx(thr);
ut_a(!que_thr_stop(thr));
switch (trx_get_dict_operation(trx)) { switch (trx_get_dict_operation(trx)) {
case TRX_DICT_OP_NONE: case TRX_DICT_OP_NONE:
@ -3321,12 +3328,15 @@ lock_table_enqueue_waiting(
const trx_t* victim_trx = const trx_t* victim_trx =
DeadlockChecker::check_and_resolve(lock, trx); DeadlockChecker::check_and_resolve(lock, trx);
if (victim_trx != 0) { if (victim_trx) {
ut_ad(victim_trx == trx); ut_ad(victim_trx == trx);
/* The order here is important, we don't want to /* The order here is important, we don't want to
lose the state of the lock before calling remove. */ lose the state of the lock before calling remove. */
lock_table_remove_low(lock); 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); lock_reset_lock_and_trx_wait(lock);
return(DB_DEADLOCK); return(DB_DEADLOCK);
@ -3338,13 +3348,9 @@ lock_table_enqueue_waiting(
return(DB_SUCCESS); return(DB_SUCCESS);
} }
trx->lock.que_state = TRX_QUE_LOCK_WAIT; trx->lock.wait_thr = thr;
trx->lock.wait_started = time(NULL);
trx->lock.was_chosen_as_deadlock_victim = false; trx->lock.was_chosen_as_deadlock_victim = false;
ut_a(que_thr_stop(thr));
MONITOR_INC(MONITOR_TABLELOCK_WAIT); MONITOR_INC(MONITOR_TABLELOCK_WAIT);
return(DB_LOCK_WAIT); return(DB_LOCK_WAIT);
@ -3583,6 +3589,7 @@ lock_table_dequeue(
they are now qualified to it */ they are now qualified to it */
{ {
lock_sys.mutex_assert_locked(); lock_sys.mutex_assert_locked();
mysql_mutex_assert_owner(&lock_sys.wait_mutex);
ut_a(lock_get_type_low(in_lock) == LOCK_TABLE); ut_a(lock_get_type_low(in_lock) == LOCK_TABLE);
lock_t* lock = UT_LIST_GET_NEXT(un_member.tab_lock.locks, in_lock); lock_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); ut_ad(!trx->is_recovered);
lock_sys.mutex_lock(); lock_sys.mutex_lock();
mysql_mutex_lock(&lock_sys.wait_mutex);
for (lock_t*& lock : trx->lock.table_locks) for (lock_t*& lock : trx->lock.table_locks)
{ {
if (lock && lock->trx == trx && lock->type_mode == (LOCK_TABLE | LOCK_X)) 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_table_dequeue(lock);
lock= nullptr; lock= nullptr;
goto func_exit; goto func_exit;
@ -3630,6 +3638,7 @@ void lock_table_x_unlock(dict_table_t *table, trx_t *trx)
func_exit: func_exit:
lock_sys.mutex_unlock(); lock_sys.mutex_unlock();
mysql_mutex_unlock(&lock_sys.wait_mutex);
} }
/** Sets a lock on a table based on the given mode. /** Sets a lock on a table based on the given mode.
@ -3660,8 +3669,6 @@ lock_table_for_trx(
que_fork_get_first_thr( que_fork_get_first_thr(
static_cast<que_fork_t*>(que_node_get_parent(thr)))); static_cast<que_fork_t*>(que_node_get_parent(thr))));
thr->start_running();
run_again: run_again:
thr->run_node = thr; thr->run_node = thr;
thr->prev_node = thr->common.parent; thr->prev_node = thr->common.parent;
@ -3670,11 +3677,7 @@ run_again:
trx->error_state = err; trx->error_state = err;
if (UNIV_LIKELY(err == DB_SUCCESS)) { if (UNIV_UNLIKELY(err != DB_SUCCESS)) {
thr->stop_no_error();
} else {
que_thr_stop_for_mysql(thr);
if (row_mysql_handle_errors(&err, trx, thr, NULL)) { if (row_mysql_handle_errors(&err, trx, thr, NULL)) {
goto run_again; goto run_again;
} }
@ -3758,7 +3761,9 @@ released:
if (!c) { if (!c) {
/* Grant the lock */ /* Grant the lock */
ut_ad(trx != lock->trx); ut_ad(trx != lock->trx);
mysql_mutex_lock(&lock_sys.wait_mutex);
lock_grant(lock); lock_grant(lock);
mysql_mutex_unlock(&lock_sys.wait_mutex);
#ifdef WITH_WSREP #ifdef WITH_WSREP
} else { } else {
wsrep_assert_no_bf_bf_wait(c, lock, c->trx); 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(); trx_id_t max_trx_id = trx_sys.get_max_trx_id();
lock_sys.mutex_lock(); lock_sys.mutex_lock();
mysql_mutex_lock(&lock_sys.wait_mutex);
for (lock_t* lock = UT_LIST_GET_LAST(trx->lock.trx_locks); for (lock_t* lock = UT_LIST_GET_LAST(trx->lock.trx_locks);
lock != NULL; lock != NULL;
@ -3851,16 +3857,17 @@ void lock_release(trx_t* trx)
do not monopolize it */ do not monopolize it */
lock_sys.mutex_unlock(); lock_sys.mutex_unlock();
mysql_mutex_unlock(&lock_sys.wait_mutex);
lock_sys.mutex_lock();
count = 0; count = 0;
lock_sys.mutex_lock();
mysql_mutex_lock(&lock_sys.wait_mutex);
} }
++count; ++count;
} }
lock_sys.mutex_unlock(); lock_sys.mutex_unlock();
mysql_mutex_unlock(&lock_sys.wait_mutex);
} }
/* True if a lock mode is S or X */ /* True if a lock mode is S or X */
@ -4133,27 +4140,26 @@ lock_print_info_summary(
/** Prints transaction lock wait and MVCC state. /** Prints transaction lock wait and MVCC state.
@param[in,out] file file where to print @param[in,out] file file where to print
@param[in] trx transaction @param[in] trx transaction
@param[in] now current time */ @param[in] now current my_hrtime_coarse() */
void void lock_trx_print_wait_and_mvcc_state(FILE *file, const trx_t *trx,
lock_trx_print_wait_and_mvcc_state(FILE* file, const trx_t* trx, time_t now) my_hrtime_t now)
{ {
fprintf(file, "---"); fprintf(file, "---");
trx_print_latched(file, trx, 600); trx_print_latched(file, trx, 600);
trx->read_view.print_limits(file); 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, fprintf(file,
"------- TRX HAS BEEN WAITING %lu SEC" "------- TRX HAS BEEN WAITING %llu ns"
" FOR THIS LOCK TO BE GRANTED:\n", " 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; mtr_t mtr;
lock_rec_print(file, trx->lock.wait_lock, mtr); lock_rec_print(file, wait_lock, mtr);
} else { } else {
lock_table_print(file, trx->lock.wait_lock); lock_table_print(file, wait_lock);
} }
fprintf(file, "------------------\n"); fprintf(file, "------------------\n");
@ -4198,9 +4204,9 @@ lock_trx_print_locks(
/** Functor to display all transactions */ /** Functor to display all transactions */
struct lock_print_info 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), 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 void operator()(const trx_t &trx) const
@ -4214,7 +4220,7 @@ struct lock_print_info
} }
FILE* const file; FILE* const file;
const time_t now; const my_hrtime_t now;
const trx_t* const purge_trx; const trx_t* const purge_trx;
}; };
@ -4231,7 +4237,7 @@ lock_print_info_all_transactions(
fprintf(file, "LIST OF TRANSACTIONS FOR EACH SESSION:\n"); 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(); lock_sys.mutex_unlock();
ut_ad(lock_validate()); ut_ad(lock_validate());
@ -5380,17 +5386,14 @@ lock_release_autoinc_last_lock(
lock_t* lock; lock_t* lock;
lock_sys.mutex_assert_locked(); 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. */ /* The lock to be release must be the last lock acquired. */
last = ib_vector_size(autoinc_locks) - 1; last = ib_vector_size(autoinc_locks) - 1;
lock = *static_cast<lock_t**>(ib_vector_get(autoinc_locks, last)); lock = *static_cast<lock_t**>(ib_vector_get(autoinc_locks, last));
/* Should have only AUTOINC locks in the vector. */ ut_ad(lock->type_mode == (LOCK_AUTO_INC | LOCK_TABLE));
ut_a(lock_get_mode(lock) == LOCK_AUTO_INC); ut_ad(lock->un_member.tab_lock.table);
ut_a(lock_get_type(lock) == LOCK_TABLE);
ut_a(lock->un_member.tab_lock.table != NULL);
/* This will remove the lock from the trx autoinc_locks too. */ /* This will remove the lock from the trx autoinc_locks too. */
lock_table_dequeue(lock); lock_table_dequeue(lock);
@ -5422,6 +5425,7 @@ lock_release_autoinc_locks(
trx_t* trx) /*!< in/out: transaction */ trx_t* trx) /*!< in/out: transaction */
{ {
lock_sys.mutex_assert_locked(); lock_sys.mutex_assert_locked();
mysql_mutex_assert_owner(&lock_sys.wait_mutex);
/* If this is invoked for a running transaction by the thread /* If this is invoked for a running transaction by the thread
that is serving the transaction, then it is not necessary to that is serving the transaction, then it is not necessary to
hold trx->mutex here. */ hold trx->mutex here. */
@ -5514,61 +5518,33 @@ lock_rec_get_index(
return(lock->index); return(lock->index);
} }
/*******************************************************************//** /** Cancel a waiting lock request and release possibly waiting transactions */
For a record lock, gets the name of the index on which the lock is. void lock_cancel_waiting_and_release(lock_t *lock)
The string should not be free()'d or modified.
@return name of the index */
const char*
lock_rec_get_index_name(
/*====================*/
const lock_t* lock) /*!< in: lock */
{
ut_a(lock_get_type_low(lock) == LOCK_REC);
ut_ad(dict_index_is_clust(lock->index)
|| !dict_index_is_online_ddl(lock->index));
return(lock->index->name);
}
/*********************************************************************//**
Cancels a waiting lock request and releases possible other transactions
waiting behind it. */
void
lock_cancel_waiting_and_release(
/*============================*/
lock_t* lock) /*!< in/out: waiting lock request */
{ {
lock_sys.mutex_assert_locked(); lock_sys.mutex_assert_locked();
trx_t* trx = lock->trx; mysql_mutex_assert_owner(&lock_sys.wait_mutex);
trx_t *trx= lock->trx;
ut_ad(trx->state == TRX_STATE_ACTIVE); ut_ad(trx->state == TRX_STATE_ACTIVE);
trx->lock.cancel = true; trx->lock.cancel= true;
if (lock_get_type_low(lock) == LOCK_REC) {
if (lock_get_type_low(lock) == LOCK_REC)
lock_rec_dequeue_from_page(lock); lock_rec_dequeue_from_page(lock);
} else { else
ut_ad(lock_get_type_low(lock) & LOCK_TABLE); {
if (trx->autoinc_locks)
if (trx->autoinc_locks) {
/* Release the transaction's AUTOINC locks. */
lock_release_autoinc_locks(trx); lock_release_autoinc_locks(trx);
}
lock_table_dequeue(lock); lock_table_dequeue(lock);
/* Remove the lock from table lock vector too. */ /* Remove the lock from table lock vector too. */
lock_trx_table_locks_remove(lock); lock_trx_table_locks_remove(lock);
} }
/* Reset the wait flag and the back pointer to lock in trx. */ /* Reset the wait flag and the back pointer to lock in trx. */
lock_reset_lock_and_trx_wait(lock); lock_reset_lock_and_trx_wait(lock);
if (que_thr_t *thr = que_thr_end_lock_wait(trx)) { lock_wait_end(trx);
lock_wait_release_thread_if_suspended(thr);
}
trx->lock.cancel = false; trx->lock.cancel= false;
} }
/*********************************************************************//** /*********************************************************************//**
@ -5595,24 +5571,25 @@ lock_unlock_table_autoinc(
if (lock_trx_holds_autoinc_locks(trx)) { if (lock_trx_holds_autoinc_locks(trx)) {
lock_sys.mutex_lock(); lock_sys.mutex_lock();
mysql_mutex_lock(&lock_sys.wait_mutex);
lock_release_autoinc_locks(trx); lock_release_autoinc_locks(trx);
lock_sys.mutex_unlock(); lock_sys.mutex_unlock();
mysql_mutex_unlock(&lock_sys.wait_mutex);
} }
} }
static inline dberr_t lock_trx_handle_wait_low(trx_t* trx) 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) { if (trx->lock.was_chosen_as_deadlock_victim)
return DB_DEADLOCK; return DB_DEADLOCK;
} if (!trx->lock.wait_lock)
if (!trx->lock.wait_lock) {
/* The lock was probably granted before we got here. */ /* The lock was probably granted before we got here. */
return DB_SUCCESS; return DB_SUCCESS;
}
lock_cancel_waiting_and_release(trx->lock.wait_lock); lock_cancel_waiting_and_release(trx->lock.wait_lock);
return DB_LOCK_WAIT; return DB_LOCK_WAIT;
@ -5629,15 +5606,17 @@ lock_trx_handle_wait(
trx_t* trx) /*!< in/out: trx lock state */ trx_t* trx) /*!< in/out: trx lock state */
{ {
#ifdef WITH_WSREP #ifdef WITH_WSREP
/* We already own mutexes */ if (UNIV_UNLIKELY(trx->lock.was_chosen_as_wsrep_victim))
if (trx->lock.was_chosen_as_wsrep_victim) { /* FIXME: we do not hold lock_sys.wait_mutex! */
return lock_trx_handle_wait_low(trx); return lock_trx_handle_wait_low(trx);
}
#endif /* WITH_WSREP */ #endif /* WITH_WSREP */
dberr_t err;
lock_sys.mutex_lock(); lock_sys.mutex_lock();
mysql_mutex_lock(&lock_sys.wait_mutex);
trx->mutex.wr_lock(); trx->mutex.wr_lock();
dberr_t err = lock_trx_handle_wait_low(trx); err= lock_trx_handle_wait_low(trx);
lock_sys.mutex_unlock(); lock_sys.mutex_unlock();
mysql_mutex_unlock(&lock_sys.wait_mutex);
trx->mutex.wr_unlock(); trx->mutex.wr_unlock();
return err; return err;
} }
@ -6092,6 +6071,7 @@ const trx_t*
DeadlockChecker::search() DeadlockChecker::search()
{ {
lock_sys.mutex_assert_locked(); lock_sys.mutex_assert_locked();
mysql_mutex_assert_owner(&lock_sys.wait_mutex);
ut_ad(m_start != NULL); ut_ad(m_start != NULL);
ut_ad(m_wait_lock != NULL); ut_ad(m_wait_lock != NULL);
@ -6143,7 +6123,9 @@ DeadlockChecker::search()
continue; continue;
} }
if (lock->trx == m_start) { trx_t *trx = lock->trx;
if (trx == m_start) {
/* Found a cycle. */ /* Found a cycle. */
notify(lock); notify(lock);
return select_victim(); return select_victim();
@ -6163,10 +6145,12 @@ DeadlockChecker::search()
&& (lock_get_type_low(lock) != LOCK_TABLE && (lock_get_type_low(lock) != LOCK_TABLE
|| lock_get_mode(lock) != LOCK_AUTO_INC)) { || lock_get_mode(lock) != LOCK_AUTO_INC)) {
thd_rpl_deadlock_check(m_start->mysql_thd, 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 /* Another trx ahead has requested a lock in an
incompatible mode, and is itself waiting for a lock. */ incompatible mode, and is itself waiting for a lock. */
@ -6177,7 +6161,7 @@ DeadlockChecker::search()
return m_start; return m_start;
} }
m_wait_lock = lock->trx->lock.wait_lock; m_wait_lock = wait_lock;
lock = get_first_lock(&heap_no); lock = get_first_lock(&heap_no);
@ -6225,6 +6209,7 @@ void
DeadlockChecker::trx_rollback() DeadlockChecker::trx_rollback()
{ {
lock_sys.mutex_assert_locked(); lock_sys.mutex_assert_locked();
mysql_mutex_assert_owner(&lock_sys.wait_mutex);
trx_t* trx = m_wait_lock->trx; 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); wsrep_handle_SR_rollback(m_start->mysql_thd, trx->mysql_thd);
} }
#endif #endif
trx->mutex.wr_lock(); trx->mutex.wr_lock();
trx->lock.was_chosen_as_deadlock_victim = true; trx->lock.was_chosen_as_deadlock_victim = true;
lock_cancel_waiting_and_release(trx->lock.wait_lock); lock_cancel_waiting_and_release(trx->lock.wait_lock);
trx->mutex.wr_unlock(); 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. */ /* Try and resolve as many deadlocks as possible. */
do { do {
mysql_mutex_lock(&lock_sys.wait_mutex);
DeadlockChecker checker(trx, lock, s_lock_mark_counter, DeadlockChecker checker(trx, lock, s_lock_mark_counter,
report_waiters); report_waiters);
@ -6300,7 +6282,7 @@ DeadlockChecker::check_and_resolve(const lock_t* lock, trx_t* trx)
MONITOR_INC(MONITOR_DEADLOCK); MONITOR_INC(MONITOR_DEADLOCK);
srv_stats.lock_deadlock_count.inc(); srv_stats.lock_deadlock_count.inc();
mysql_mutex_unlock(&lock_sys.wait_mutex);
break; break;
} else if (victim_trx != NULL && victim_trx != trx) { } 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(); srv_stats.lock_deadlock_count.inc();
} }
mysql_mutex_unlock(&lock_sys.wait_mutex);
} while (victim_trx != NULL && victim_trx != trx); } while (victim_trx != NULL && victim_trx != trx);
/* If the joining transaction was selected as the victim. */ /* If the joining transaction was selected as the victim. */

View File

@ -946,8 +946,10 @@ lock_prdt_rec_move(
lock_prdt_t* lock_prdt = lock_get_prdt_from_lock(lock); lock_prdt_t* lock_prdt = lock_get_prdt_from_lock(lock);
lock_rec_reset_nth_bit(lock, PRDT_HEAPNO); 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( lock_prdt_add_to_queue(
type_mode, receiver, lock->index, lock->trx, type_mode, receiver, lock->index, lock->trx,
lock_prdt, false); lock_prdt, false);

View File

@ -1,7 +1,7 @@
/***************************************************************************** /*****************************************************************************
Copyright (c) 1996, 2016, Oracle and/or its affiliates. All Rights Reserved. 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 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 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 "row0mysql.h"
#include "srv0start.h" #include "srv0start.h"
#include "lock0priv.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 #ifdef WITH_WSREP
/*********************************************************************//** /*********************************************************************//**
@ -210,290 +75,160 @@ wsrep_is_BF_lock_timeout(
} }
#endif /* WITH_WSREP */ #endif /* WITH_WSREP */
/***************************************************************//** /** Note that a record lock wait started */
Puts a user OS thread to wait for a lock to be released. If an error inline void lock_sys_t::wait_start()
occurs during the wait trx->error_state associated with thr is
!= DB_SUCCESS when we return. DB_LOCK_WAIT_TIMEOUT and DB_DEADLOCK
are possible errors. DB_DEADLOCK is returned if selective deadlock
resolution chose this transaction as a victim. */
void
lock_wait_suspend_thread(
/*=====================*/
que_thr_t* thr) /*!< in: query thread associated with the
user OS thread */
{ {
srv_slot_t* slot; mysql_mutex_assert_owner(&wait_mutex);
trx_t* trx; wait_pending++;
ulong lock_wait_timeout; wait_count++;
}
ut_a(lock_sys.timeout_timer.get()); /** Note that a record lock wait resumed */
trx = thr_get_trx(thr); 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->mysql_thd != 0) { if (diff_time > wait_time_max)
DEBUG_SYNC_C("lock_wait_suspend_thread_enter"); wait_time_max= diff_time;
thd_storage_lock_wait(thd, diff_time);
} }
}
/* InnoDB system transactions (such as the purge, and /** Wait for a lock to be released.
incomplete transactions that are being rolled back after crash @retval DB_DEADLOCK if this transaction was chosen as the deadlock victim
recovery) will use the global value of @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)
{
trx_t *trx= thr_get_trx(thr);
if (trx->mysql_thd)
DEBUG_SYNC_C("lock_wait_suspend_thread_enter");
/* InnoDB system transactions may use the global value of
innodb_lock_wait_timeout, because trx->mysql_thd == NULL. */ innodb_lock_wait_timeout, because trx->mysql_thd == NULL. */
lock_wait_timeout = trx_lock_wait_timeout_get(trx); 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;
mysql_mutex_lock(&lock_sys.wait_mutex); mysql_mutex_lock(&lock_sys.wait_mutex);
trx->mutex.wr_lock(); trx->mutex.wr_lock();
trx->error_state= DB_SUCCESS;
trx->error_state = DB_SUCCESS; if (!trx->lock.wait_lock)
{
if (thr->state == QUE_THR_RUNNING) {
ut_ad(thr->is_active);
/* The lock has already been released or this transaction /* The lock has already been released or this transaction
was chosen as a deadlock victim: no need to suspend */ was chosen as a deadlock victim: no need to suspend */
if (trx->lock.was_chosen_as_deadlock_victim) { if (trx->lock.was_chosen_as_deadlock_victim)
{
trx->error_state = DB_DEADLOCK; trx->error_state= DB_DEADLOCK;
trx->lock.was_chosen_as_deadlock_victim = false; trx->lock.was_chosen_as_deadlock_victim= false;
} }
mysql_mutex_unlock(&lock_sys.wait_mutex); mysql_mutex_unlock(&lock_sys.wait_mutex);
trx->mutex.wr_unlock(); trx->mutex.wr_unlock();
return; return trx->error_state;
} }
ut_ad(!thr->is_active); trx->lock.suspend_time= suspend_time;
slot = lock_wait_table_reserve_slot(thr, lock_wait_timeout);
mysql_mutex_unlock(&lock_sys.wait_mutex);
trx->mutex.wr_unlock(); trx->mutex.wr_unlock();
ulonglong start_time = 0; if (row_lock_wait)
lock_sys.wait_start();
if (thr->lock_state == QUE_THR_LOCK_ROW) { int err= 0;
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 in lock_grant(),
lock_rec_cancel(), or lock_cancel_waiting_and_release(). But, a wait
/* The wait_lock can be cleared by another thread when the can only be initiated by the current thread which owns the transaction. */
lock is released. But the wait can only be initiated by the if (const lock_t *wait_lock= trx->lock.wait_lock)
current thread which owns the transaction. Only acquire the {
mutex if the wait_lock is still active. */ if (had_dict_lock) /* Release foreign key check latch */
if (const lock_t* wait_lock = trx->lock.wait_lock) { {
lock_sys.mutex_lock(); mysql_mutex_unlock(&lock_sys.wait_mutex);
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); row_mysql_unfreeze_data_dictionary(trx);
mysql_mutex_lock(&lock_sys.wait_mutex);
DEBUG_SYNC_C("lock_wait_release_s_latch_before_sleep"); }
break; 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: default:
/* There should never be a lock wait when the if (trx_is_interrupted(trx))
dictionary latch is reserved in X mode. Dictionary /* innobase_kill_query() can only set trx->error_state=DB_INTERRUPTED
transactions should only acquire locks on dictionary for any transaction that is attached to a connection. */
tables, not other tables. All access to dictionary trx->error_state= DB_INTERRUPTED;
tables should be covered by dictionary else if (!err)
transactions. */ continue;
ut_error; else
break;
/* fall through */
case DB_DEADLOCK:
case DB_INTERRUPTED:
err= 0;
} }
break;
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); thd_wait_end(trx->mysql_thd);
}
else
had_dict_lock= false;
/* After resuming, reacquire the data dictionary latch if if (row_lock_wait)
necessary. */ lock_sys.wait_resume(trx->mysql_thd, suspend_time, my_hrtime_coarse());
if (had_dict_lock) { mysql_mutex_unlock(&lock_sys.wait_mutex);
if (had_dict_lock)
row_mysql_freeze_data_dictionary(trx); row_mysql_freeze_data_dictionary(trx);
}
double wait_time = difftime(time(NULL), slot->suspend_time); if (!err);
/* 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 #ifdef WITH_WSREP
&& (!trx->is_wsrep() else if (trx->is_wsrep() && wsrep_is_BF_lock_timeout(trx, false));
|| (!wsrep_is_BF_lock_timeout(trx, false) #endif
&& trx->error_state != DB_DEADLOCK)) else
#endif /* WITH_WSREP */ {
) { trx->error_state= DB_LOCK_WAIT_TIMEOUT;
trx->error_state = DB_LOCK_WAIT_TIMEOUT;
MONITOR_INC(MONITOR_TIMEOUT); MONITOR_INC(MONITOR_TIMEOUT);
} }
if (trx_is_interrupted(trx)) { if (trx->lock.wait_lock)
{
trx->error_state = DB_INTERRUPTED; {
}
}
/********************************************************************//**
Releases a user OS thread waiting for a lock to be released, if the
thread is already suspended. */
void
lock_wait_release_thread_if_suspended(
/*==================================*/
que_thr_t* thr) /*!< in: query thread associated with the
user OS thread */
{
lock_sys.mutex_assert_locked();
/* We own both the lock mutex and the trx_t::mutex but not the
lock wait mutex. This is OK because other threads will see the state
of this slot as being in use and no other thread can change the state
of the slot to free unless that thread also owns the lock mutex. */
if (thr->slot != NULL && thr->slot->in_use && thr->slot->thr == thr) {
trx_t* trx = thr_get_trx(thr);
if (trx->lock.was_chosen_as_deadlock_victim) {
trx->error_state = DB_DEADLOCK;
trx->lock.was_chosen_as_deadlock_victim = false;
}
mysql_cond_signal(&thr->slot->cond);
}
}
/*********************************************************************//**
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);
double wait_time = difftime(time(NULL), slot->suspend_time);
trx_t* trx = thr_get_trx(slot->thr);
if (trx_is_interrupted(trx)
|| (slot->wait_timeout < 100000000
&& (wait_time > (double) slot->wait_timeout
|| wait_time < 0))) {
/* Timeout exceeded or a wrap-around in system
time counter: cancel the lock request queued
by the transaction and release possible
other transactions waiting behind; it is
possible that the lock has already been
granted: in that case do nothing */
lock_sys.mutex_lock(); lock_sys.mutex_lock();
if (trx->lock.wait_lock != NULL) {
ut_a(trx->lock.que_state == TRX_QUE_LOCK_WAIT);
#ifdef WITH_WSREP
if (!wsrep_is_BF_lock_timeout(trx)) {
#endif /* WITH_WSREP */
trx->mutex.wr_lock();
lock_cancel_waiting_and_release(trx->lock.wait_lock);
trx->mutex.wr_unlock();
#ifdef WITH_WSREP
}
#endif /* WITH_WSREP */
}
lock_sys.mutex_unlock();
}
}
/** 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); mysql_mutex_lock(&lock_sys.wait_mutex);
if (lock_t *lock= trx->lock.wait_lock)
/* Check all slots for user threads that are waiting
on locks, and if they have exceeded the time limit. */
bool any_slot_in_use= false;
for (srv_slot_t *slot= lock_sys.waiting_threads;
slot < lock_sys.last_slot; ++slot)
{ {
/* We are doing a read without the lock mutex and/or the trx trx->mutex.wr_lock();
mutex. This is OK because a slot can't be freed or reserved lock_cancel_waiting_and_release(lock);
without the lock wait mutex. */ trx->mutex.wr_unlock();
if (slot->in_use)
{
any_slot_in_use= true;
lock_wait_check_and_cancel(slot);
} }
} lock_sys.mutex_unlock();
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); mysql_mutex_unlock(&lock_sys.wait_mutex);
}
}
return trx->error_state;
} }

View File

@ -2,7 +2,7 @@
Copyright (c) 1995, 2017, Oracle and/or its affiliates. All Rights Reserved. Copyright (c) 1995, 2017, Oracle and/or its affiliates. All Rights Reserved.
Copyright (c) 2009, Google Inc. 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 Portions of this file contain modifications contributed and copyrighted by
Google, Inc. Those modifications are gratefully acknowledged and are described 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(); buf_dump_start();
} }
srv_monitor_timer.reset(); srv_monitor_timer.reset();
lock_sys.timeout_timer.reset();
if (do_srv_shutdown) { if (do_srv_shutdown) {
srv_shutdown(srv_fast_shutdown == 0); srv_shutdown(srv_fast_shutdown == 0);
} }

View File

@ -146,10 +146,6 @@ que_thr_create(
thr->common.type = QUE_NODE_THR; thr->common.type = QUE_NODE_THR;
thr->state = QUE_THR_COMMAND_WAIT;
thr->lock_state = QUE_THR_LOCK_NOLOCK;
thr->prebuilt = prebuilt; thr->prebuilt = prebuilt;
UT_LIST_ADD_LAST(parent->thrs, thr); UT_LIST_ADD_LAST(parent->thrs, thr);
@ -157,44 +153,6 @@ que_thr_create(
return(thr); return(thr);
} }
/**********************************************************************//**
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. */ Inits a query thread for a command. */
UNIV_INLINE UNIV_INLINE
@ -205,7 +163,7 @@ que_thr_init_command(
{ {
thr->run_node = thr; thr->run_node = thr;
thr->prev_node = thr->common.parent; 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->state = QUE_FORK_ACTIVE;
fork->last_sel_node = NULL; fork->last_sel_node = NULL;
ut_ad(thr->state == QUE_THR_COMPLETED);
switch (thr->state) {
case QUE_THR_COMMAND_WAIT:
case QUE_THR_COMPLETED:
ut_a(!thr->is_active);
que_thr_init_command(thr); que_thr_init_command(thr);
break;
case QUE_THR_LOCK_WAIT:
default:
ut_error;
}
} }
fork->trx->mutex.wr_unlock(); fork->trx->mutex.wr_unlock();
@ -266,57 +213,15 @@ que_fork_start_command(
/*===================*/ /*===================*/
que_fork_t* fork) /*!< in: a query fork */ que_fork_t* fork) /*!< in: a query fork */
{ {
que_thr_t* thr;
que_thr_t* completed_thr = NULL;
fork->state = QUE_FORK_ACTIVE; fork->state = QUE_FORK_ACTIVE;
fork->last_sel_node = NULL; fork->last_sel_node = NULL;
completed_thr = NULL; que_thr_t* thr = UT_LIST_GET_FIRST(fork->thrs);
/* 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 */
if (thr) {
ut_ad(thr->state == QUE_THR_COMPLETED);
que_thr_init_command(thr); 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;
que_thr_init_command(thr);
} else {
ut_error;
} }
return(thr); return(thr);
@ -560,173 +465,18 @@ que_thr_node_step(
return(thr); return(thr);
} }
auto mutex = &thr->graph->trx->mutex; trx_t *trx= thr->graph->trx;
trx->mutex.wr_lock();
mutex->wr_lock(); if (!trx->lock.wait_thr && thr->graph->state == QUE_FORK_ACTIVE) {
if (!que_thr_peek_stop(thr)) {
/* Thread execution completed */
thr->state = QUE_THR_COMPLETED; thr->state = QUE_THR_COMPLETED;
thr = NULL; thr = NULL;
} }
mutex->wr_unlock(); trx->mutex.wr_unlock();
return(thr); 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();
}
#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 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. given node, or NULL if the node is not within a loop.
@ -967,17 +717,6 @@ que_run_threads_low(
if (next_thr) { if (next_thr) {
ut_a(trx->error_state == DB_SUCCESS); ut_a(trx->error_state == DB_SUCCESS);
ut_a(next_thr == thr); 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)); ut_ad(trx == thr_get_trx(thr));
@ -991,40 +730,18 @@ que_run_threads(
/*============*/ /*============*/
que_thr_t* thr) /*!< in: query thread */ que_thr_t* thr) /*!< in: query thread */
{ {
trx_t* trx = thr->graph->trx;
loop: loop:
ut_a(thr_get_trx(thr)->error_state == DB_SUCCESS); ut_a(trx->error_state == DB_SUCCESS);
que_run_threads_low(thr); que_run_threads_low(thr);
switch (thr->state) { if (thr->state != QUE_THR_COMPLETED) {
default: if (trx->lock.wait_thr) {
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); ut_ad(trx->id);
const dberr_t err = trx->error_state; if (lock_wait(thr) == DB_SUCCESS) {
if (err != DB_SUCCESS) { goto loop;
/* 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(); } else if (trx->error_state == DB_SUCCESS) {
if (err == DB_SUCCESS) {
goto loop; goto loop;
} }
} }

View File

@ -1825,25 +1825,22 @@ do_possible_lock_wait:
if (err == DB_LOCK_WAIT) { if (err == DB_LOCK_WAIT) {
trx->error_state = err; trx->error_state = err;
que_thr_stop_for_mysql(thr);
thr->lock_state = QUE_THR_LOCK_ROW; thr->lock_state = QUE_THR_LOCK_ROW;
check_table->inc_fk_checks(); check_table->inc_fk_checks();
lock_wait_suspend_thread(thr); err = lock_wait(thr);
thr->lock_state = QUE_THR_LOCK_NOLOCK; thr->lock_state = QUE_THR_LOCK_NOLOCK;
err = trx->error_state; check_table->dec_fk_checks();
if (err != DB_SUCCESS) { if (err != DB_SUCCESS) {
} else if (check_table->to_be_dropped) { } else if (check_table->to_be_dropped) {
err = DB_LOCK_WAIT_TIMEOUT; err = DB_LOCK_WAIT_TIMEOUT;
} else { } else {
err = DB_LOCK_WAIT; err = DB_LOCK_WAIT;
} }
check_table->dec_fk_checks();
} }
exit_func: exit_func:

View File

@ -688,14 +688,14 @@ row_mysql_handle_errors(
DBUG_ENTER("row_mysql_handle_errors"); DBUG_ENTER("row_mysql_handle_errors");
handle_new_error:
err = trx->error_state; err = trx->error_state;
handle_new_error:
ut_a(err != DB_SUCCESS); ut_a(err != DB_SUCCESS);
trx->error_state = 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); << ";id=" << ib::hex(trx->id) << ", " << trx);
switch (err) { switch (err) {
@ -733,11 +733,8 @@ handle_new_error:
/* MySQL will roll back the latest SQL statement */ /* MySQL will roll back the latest SQL statement */
break; break;
case DB_LOCK_WAIT: case DB_LOCK_WAIT:
lock_wait_suspend_thread(thr); err = lock_wait(thr);
if (err != DB_SUCCESS) {
if (trx->error_state != DB_SUCCESS) {
que_thr_stop_for_mysql(thr);
goto handle_new_error; goto handle_new_error;
} }
@ -786,14 +783,13 @@ handle_new_error:
ib::fatal() << "Unknown error " << err; ib::fatal() << "Unknown error " << err;
} }
if (trx->error_state != DB_SUCCESS) { if (dberr_t n_err = trx->error_state) {
*new_err = trx->error_state; trx->error_state = DB_SUCCESS;
*new_err = n_err;
} else { } else {
*new_err = err; *new_err = err;
} }
trx->error_state = DB_SUCCESS;
DBUG_RETURN(false); DBUG_RETURN(false);
} }
@ -1139,12 +1135,12 @@ row_get_prebuilt_insert_row(
dict_table_copy_types(row, table); dict_table_copy_types(row, table);
ins_node_set_new_row(node, row); 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*>( prebuilt->ins_graph = static_cast<que_fork_t*>(
que_node_get_parent( que_node_get_parent(fork));
pars_complete_graph_for_exec(
node,
prebuilt->trx, prebuilt->heap, prebuilt)));
prebuilt->ins_graph->state = QUE_FORK_ACTIVE; prebuilt->ins_graph->state = QUE_FORK_ACTIVE;
@ -1171,7 +1167,6 @@ row_lock_table_autoinc_for_mysql(
const dict_table_t* table = prebuilt->table; const dict_table_t* table = prebuilt->table;
que_thr_t* thr; que_thr_t* thr;
dberr_t err; dberr_t err;
ibool was_lock_wait;
/* If we already hold an AUTOINC lock on the table then do nothing. /* 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 Note: We peek at the value of the current owner without acquiring
@ -1191,9 +1186,7 @@ row_lock_table_autoinc_for_mysql(
thr = que_fork_get_first_thr(prebuilt->ins_graph); thr = que_fork_get_first_thr(prebuilt->ins_graph);
thr->start_running(); do {
run_again:
thr->run_node = node; thr->run_node = node;
thr->prev_node = node; thr->prev_node = node;
@ -1205,22 +1198,8 @@ run_again:
err = lock_table(0, prebuilt->table, LOCK_AUTO_INC, thr); err = lock_table(0, prebuilt->table, LOCK_AUTO_INC, thr);
trx->error_state = err; trx->error_state = err;
} while (err != DB_SUCCESS
if (err != DB_SUCCESS) { && row_mysql_handle_errors(&err, trx, thr, NULL));
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();
trx->op_info = ""; trx->op_info = "";
@ -1236,7 +1215,6 @@ row_lock_table(row_prebuilt_t* prebuilt)
trx_t* trx = prebuilt->trx; trx_t* trx = prebuilt->trx;
que_thr_t* thr; que_thr_t* thr;
dberr_t err; dberr_t err;
ibool was_lock_wait;
trx->op_info = "setting table lock"; trx->op_info = "setting table lock";
@ -1250,9 +1228,7 @@ row_lock_table(row_prebuilt_t* prebuilt)
thr = que_fork_get_first_thr(prebuilt->sel_graph); thr = que_fork_get_first_thr(prebuilt->sel_graph);
thr->start_running(); do {
run_again:
thr->run_node = thr; thr->run_node = thr;
thr->prev_node = thr->common.parent; thr->prev_node = thr->common.parent;
@ -1261,28 +1237,11 @@ run_again:
trx_start_if_not_started_xa(trx, false); trx_start_if_not_started_xa(trx, false);
err = lock_table(0, prebuilt->table, err = lock_table(0, prebuilt->table, static_cast<lock_mode>(
static_cast<enum lock_mode>( prebuilt->select_lock_type), thr);
prebuilt->select_lock_type),
thr);
trx->error_state = err; trx->error_state = err;
} while (err != DB_SUCCESS
if (err != DB_SUCCESS) { && row_mysql_handle_errors(&err, trx, thr, NULL));
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();
trx->op_info = ""; trx->op_info = "";
@ -1418,8 +1377,6 @@ row_insert_for_mysql(
node->trx_id = trx->id; node->trx_id = trx->id;
} }
thr->start_running();
run_again: run_again:
thr->run_node = node; thr->run_node = node;
thr->prev_node = node; thr->prev_node = node;
@ -1432,8 +1389,6 @@ run_again:
if (err != DB_SUCCESS) { if (err != DB_SUCCESS) {
error_exit: error_exit:
que_thr_stop_for_mysql(thr);
/* FIXME: What's this ? */ /* FIXME: What's this ? */
thr->lock_state = QUE_THR_LOCK_ROW; thr->lock_state = QUE_THR_LOCK_ROW;
@ -1521,8 +1476,6 @@ error_exit:
} }
} }
thr->stop_no_error();
if (table->is_system_db) { if (table->is_system_db) {
srv_stats.n_system_rows_inserted.inc(size_t(trx->id)); srv_stats.n_system_rows_inserted.inc(size_t(trx->id));
} else { } else {
@ -1566,12 +1519,12 @@ row_prebuild_sel_graph(
node = sel_node_create(prebuilt->heap); 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*>( prebuilt->sel_graph = static_cast<que_fork_t*>(
que_node_get_parent( que_node_get_parent(fork));
pars_complete_graph_for_exec(
static_cast<sel_node_t*>(node),
prebuilt->trx, prebuilt->heap,
prebuilt)));
prebuilt->sel_graph->state = QUE_FORK_ACTIVE; 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); ut_ad(!prebuilt->sql_stat_start);
thr->start_running();
ut_ad(!prebuilt->versioned_write || node->table->versioned()); ut_ad(!prebuilt->versioned_write || node->table->versioned());
if (prebuilt->versioned_write) { if (prebuilt->versioned_write) {
@ -1859,8 +1810,6 @@ row_update_for_mysql(row_prebuilt_t* prebuilt)
break; break;
} }
que_thr_stop_for_mysql(thr);
if (err == DB_RECORD_NOT_FOUND) { if (err == DB_RECORD_NOT_FOUND) {
trx->error_state = DB_SUCCESS; trx->error_state = DB_SUCCESS;
goto error; 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) if (dict_table_has_fts_index(table)
&& trx->fts_next_doc_id != UINT64_UNDEFINED) { && trx->fts_next_doc_id != UINT64_UNDEFINED) {
err = row_fts_update_or_delete(prebuilt); 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) { switch (trx->error_state) {
case DB_LOCK_WAIT: case DB_LOCK_WAIT:
que_thr_stop_for_mysql(thr); if (lock_wait(thr) == DB_SUCCESS) {
lock_wait_suspend_thread(thr);
if (trx->error_state == DB_SUCCESS) {
continue; continue;
} }
@ -2263,10 +2207,7 @@ row_update_cascade_for_mysql(
switch (trx->error_state) { switch (trx->error_state) {
case DB_LOCK_WAIT: case DB_LOCK_WAIT:
que_thr_stop_for_mysql(thr); if (lock_wait(thr) == DB_SUCCESS) {
lock_wait_suspend_thread(thr);
if (trx->error_state == DB_SUCCESS) {
continue; continue;
} }
@ -3202,25 +3143,15 @@ row_mysql_lock_table(
thr = que_fork_get_first_thr( thr = que_fork_get_first_thr(
static_cast<que_fork_t*>(que_node_get_parent(thr))); static_cast<que_fork_t*>(que_node_get_parent(thr)));
thr->start_running(); do {
run_again:
thr->run_node = thr; thr->run_node = thr;
thr->prev_node = thr->common.parent; thr->prev_node = thr->common.parent;
err = lock_table(0, table, mode, thr); err = lock_table(0, table, mode, thr);
trx->error_state = err; trx->error_state = err;
} while (err != DB_SUCCESS
if (err == DB_SUCCESS) { && row_mysql_handle_errors(&err, trx, thr, NULL));
thr->stop_no_error();
} else {
que_thr_stop_for_mysql(thr);
if (row_mysql_handle_errors(&err, trx, thr, NULL)) {
goto run_again;
}
}
que_graph_free(thr->graph); que_graph_free(thr->graph);
trx->op_info = ""; trx->op_info = "";

View File

@ -1110,7 +1110,6 @@ retry:
re_scan: re_scan:
mtr->commit(); mtr->commit();
trx->error_state = err; trx->error_state = err;
que_thr_stop_for_mysql(thr);
thr->lock_state = QUE_THR_LOCK_ROW; thr->lock_state = QUE_THR_LOCK_ROW;
if (row_mysql_handle_errors( if (row_mysql_handle_errors(
&err, trx, thr, NULL)) { &err, trx, thr, NULL)) {
@ -4589,8 +4588,6 @@ aborted:
thr = que_fork_get_first_thr(prebuilt->sel_graph); thr = que_fork_get_first_thr(prebuilt->sel_graph);
thr->start_running();
clust_index = dict_table_get_first_index(prebuilt->table); clust_index = dict_table_get_first_index(prebuilt->table);
dberr_t err = DB_SUCCESS; dberr_t err = DB_SUCCESS;
@ -5735,13 +5732,6 @@ lock_table_wait:
mtr_has_extra_clust_latch = FALSE; mtr_has_extra_clust_latch = FALSE;
trx->error_state = err; 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; thr->lock_state = QUE_THR_LOCK_ROW;
if (row_mysql_handle_errors(&err, trx, thr, NULL)) { if (row_mysql_handle_errors(&err, trx, thr, NULL)) {
@ -5796,16 +5786,6 @@ lock_table_wait:
goto func_exit; goto func_exit;
normal_return: 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(); mtr.commit();
DEBUG_SYNC_C("row_search_for_mysql_before_return"); DEBUG_SYNC_C("row_search_for_mysql_before_return");

View File

@ -1719,32 +1719,38 @@ srv_mon_process_existing_counter(
/* innodb_row_lock_current_waits */ /* innodb_row_lock_current_waits */
case MONITOR_OVLD_ROW_LOCK_CURRENT_WAIT: 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; break;
/* innodb_row_lock_time */ /* innodb_row_lock_time */
case MONITOR_OVLD_LOCK_WAIT_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; break;
/* innodb_row_lock_time_max */ /* innodb_row_lock_time_max */
case MONITOR_OVLD_LOCK_MAX_WAIT_TIME: 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; break;
/* innodb_row_lock_time_avg */ /* innodb_row_lock_time_avg */
case MONITOR_OVLD_LOCK_AVG_WAIT_TIME: case MONITOR_OVLD_LOCK_AVG_WAIT_TIME:
if (srv_stats.n_lock_wait_count > 0) { mysql_mutex_lock(&lock_sys.wait_mutex);
value = srv_stats.n_lock_wait_time / 1000 if (auto count = lock_sys.get_wait_cumulative()) {
/ srv_stats.n_lock_wait_count; value = lock_sys.get_wait_time_cumulative() / 1000
/ count;
} else { } else {
value = 0; value = 0;
} }
mysql_mutex_unlock(&lock_sys.wait_mutex);
break; break;
/* innodb_row_lock_waits */ /* innodb_row_lock_waits */
case MONITOR_OVLD_ROW_LOCK_WAIT: 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; break;
case MONITOR_RSEG_HISTORY_LEN: case MONITOR_RSEG_HISTORY_LEN:

View File

@ -1106,25 +1106,21 @@ srv_export_innodb_status(void)
export_vars.innodb_pages_written = buf_pool.stat.n_pages_written; 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 = export_vars.innodb_row_lock_current_waits= lock_sys.get_wait_pending();
srv_stats.n_lock_wait_current_count;
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= export_vars.innodb_row_lock_waits
? static_cast<ulint>(export_vars.innodb_row_lock_time
export_vars.innodb_row_lock_time_avg = (ulint) / export_vars.innodb_row_lock_waits)
(srv_stats.n_lock_wait_time : 0;
/ 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_rows_read = srv_stats.n_rows_read; export_vars.innodb_rows_read = srv_stats.n_rows_read;

View File

@ -3,7 +3,7 @@
Copyright (c) 1996, 2017, Oracle and/or its affiliates. All rights reserved. Copyright (c) 1996, 2017, Oracle and/or its affiliates. All rights reserved.
Copyright (c) 2008, Google Inc. Copyright (c) 2008, Google Inc.
Copyright (c) 2009, Percona 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 Portions of this file contain modifications contributed and copyrighted by
Google, Inc. Those modifications are gratefully acknowledged and are described Google, Inc. Those modifications are gratefully acknowledged and are described
@ -825,7 +825,6 @@ static void srv_shutdown_threads()
ut_ad(!srv_undo_sources); ut_ad(!srv_undo_sources);
srv_shutdown_state = SRV_SHUTDOWN_EXIT_THREADS; srv_shutdown_state = SRV_SHUTDOWN_EXIT_THREADS;
ut_d(srv_master_thread_enable()); ut_d(srv_master_thread_enable());
lock_sys.timeout_timer.reset();
srv_master_timer.reset(); srv_master_timer.reset();
if (purge_sys.enabled()) { if (purge_sys.enabled()) {
@ -1787,11 +1786,6 @@ file_checked:
srv_startup_is_before_trx_rollback_phase = false; srv_startup_is_before_trx_rollback_phase = false;
if (!srv_read_only_mode) { 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;); DBUG_EXECUTE_IF("innodb_skip_monitors", goto skip_monitors;);
/* Create the task which warns of long semaphore waits */ /* Create the task which warns of long semaphore waits */
srv_start_periodic_timer(srv_monitor_timer, srv_monitor_task, srv_start_periodic_timer(srv_monitor_timer, srv_monitor_task,

View File

@ -424,35 +424,29 @@ fill_trx_row(
lock_sys.mutex_assert_locked(); lock_sys.mutex_assert_locked();
const lock_t* wait_lock = trx->lock.wait_lock;
row->trx_id = trx->id; row->trx_id = trx->id;
row->trx_started = trx->start_time; row->trx_started = trx->start_time;
switch (trx->lock.que_state) { if (trx->in_rollback) {
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:
row->trx_state = "ROLLING BACK"; row->trx_state = "ROLLING BACK";
break; } else if (trx->state == TRX_STATE_COMMITTED_IN_MEMORY) {
default: row->trx_state = "COMMITTING";
row->trx_state = nullptr; } else if (wait_lock) {
row->trx_state = "LOCK WAIT";
} else {
row->trx_state = "RUNNING";
} }
row->requested_lock_row = requested_lock_row; row->requested_lock_row = requested_lock_row;
ut_ad(requested_lock_row == NULL ut_ad(requested_lock_row == NULL
|| i_s_locks_row_validate(requested_lock_row)); || 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 = wait_lock
row->trx_wait_started = trx->lock.wait_started; ? hrtime_to_time(trx->lock.suspend_time)
} else { : 0;
ut_a(requested_lock_row == NULL);
row->trx_wait_started = 0;
}
row->trx_weight = static_cast<uintmax_t>(TRX_WEIGHT(trx)); row->trx_weight = static_cast<uintmax_t>(TRX_WEIGHT(trx));
@ -749,7 +743,7 @@ static bool fill_locks_row(
if (lock_type == LOCK_REC) { if (lock_type == LOCK_REC) {
row->lock_index = ha_storage_put_str_memlim( 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)); MAX_ALLOWED_FOR_STORAGE(cache));
/* memory could not be allocated */ /* memory could not be allocated */
@ -1063,20 +1057,17 @@ add_trx_relevant_locks_to_cache(
/* If transaction is waiting we add the wait lock and all locks /* If transaction is waiting we add the wait lock and all locks
from another transactions that are blocking the wait lock. */ 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; const lock_t* curr_lock;
i_s_locks_row_t* blocking_lock_row; i_s_locks_row_t* blocking_lock_row;
lock_queue_iterator_t iter; lock_queue_iterator_t iter;
ut_a(trx->lock.wait_lock != NULL);
uint16_t wait_lock_heap_no 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 */ /* add the requested lock */
*requested_lock_row *requested_lock_row = add_lock_to_cache(cache, wait_lock,
= add_lock_to_cache(cache, trx->lock.wait_lock,
wait_lock_heap_no); wait_lock_heap_no);
/* memory could not be allocated */ /* memory could not be allocated */
@ -1088,18 +1079,16 @@ add_trx_relevant_locks_to_cache(
/* then iterate over the locks before the wait lock and /* then iterate over the locks before the wait lock and
add the ones that are blocking it */ add the ones that are blocking it */
lock_queue_iterator_reset(&iter, trx->lock.wait_lock, lock_queue_iterator_reset(&iter, wait_lock, ULINT_UNDEFINED);
ULINT_UNDEFINED);
for (curr_lock = lock_queue_iterator_get_prev(&iter); for (curr_lock = lock_queue_iterator_get_prev(&iter);
curr_lock != NULL; curr_lock != NULL;
curr_lock = lock_queue_iterator_get_prev(&iter)) { curr_lock = lock_queue_iterator_get_prev(&iter)) {
if (lock_has_to_wait(trx->lock.wait_lock, if (lock_has_to_wait(wait_lock, curr_lock)) {
curr_lock)) {
/* add the lock that is /* add the lock that is
blocking trx->lock.wait_lock */ blocking wait_lock */
blocking_lock_row blocking_lock_row
= add_lock_to_cache( = add_lock_to_cache(
cache, curr_lock, cache, curr_lock,

View File

@ -1152,8 +1152,6 @@ trx_purge_attach_undo_recs(ulint n_purge_threads)
purge_node_t* node; purge_node_t* node;
trx_purge_rec_t purge_rec; trx_purge_rec_t purge_rec;
ut_a(!thr->is_active);
/* Get the purge node. */ /* Get the purge node. */
node = (purge_node_t*) thr->child; node = (purge_node_t*) thr->child;
ut_a(que_node_get_type(node) == QUE_NODE_PURGE); ut_a(que_node_get_type(node) == QUE_NODE_PURGE);

View File

@ -90,7 +90,6 @@ inline bool trx_t::rollback_finish()
undo= nullptr; undo= nullptr;
} }
commit_low(); commit_low();
lock.que_state= TRX_QUE_RUNNING;
return false; return false;
} }
@ -143,7 +142,6 @@ inline void trx_t::rollback_low(trx_savept_t *savept)
mod_tables.erase(j); mod_tables.erase(j);
} }
} }
lock.que_state= TRX_QUE_RUNNING;
MONITOR_INC(MONITOR_TRX_ROLLBACK_SAVEPOINT); MONITOR_INC(MONITOR_TRX_ROLLBACK_SAVEPOINT);
} }
@ -640,8 +638,6 @@ trx_rollback_active(
goto func_exit; goto func_exit;
} }
ut_a(trx->lock.que_state == TRX_QUE_RUNNING);
if (!dictionary_locked || !trx->table_id) { if (!dictionary_locked || !trx->table_id) {
} else if (dict_table_t* table = dict_table_open_on_id( } else if (dict_table_t* table = dict_table_open_on_id(
trx->table_id, TRUE, DICT_TABLE_OP_NORMAL)) { trx->table_id, TRUE, DICT_TABLE_OP_NORMAL)) {
@ -912,8 +908,6 @@ trx_rollback_start(
trx->graph = roll_graph; trx->graph = roll_graph;
trx->lock.que_state = TRX_QUE_ROLLING_BACK;
return(que_fork_start_command(roll_graph)); return(que_fork_start_command(roll_graph));
} }

View File

@ -143,8 +143,6 @@ trx_init(
trx->magic_n = TRX_MAGIC_N; trx->magic_n = TRX_MAGIC_N;
trx->lock.que_state = TRX_QUE_RUNNING;
trx->last_sql_stat_start.least_undo_no = 0; trx->last_sql_stat_start.least_undo_no = 0;
ut_ad(!trx->read_view.is_open()); ut_ad(!trx->read_view.is_open());
@ -186,6 +184,7 @@ struct TrxFactory {
trx->lock.lock_heap = mem_heap_create_typed( trx->lock.lock_heap = mem_heap_create_typed(
1024, MEM_HEAP_FOR_LOCK_HEAP); 1024, MEM_HEAP_FOR_LOCK_HEAP);
mysql_cond_init(0, &trx->lock.cond, nullptr);
lock_trx_lock_list_init(&trx->lock.trx_locks); lock_trx_lock_list_init(&trx->lock.trx_locks);
@ -227,6 +226,8 @@ struct TrxFactory {
trx->lock.lock_heap = NULL; trx->lock.lock_heap = NULL;
} }
mysql_cond_destroy(&trx->lock.cond);
ut_a(UT_LIST_GET_LEN(trx->lock.trx_locks) == 0); ut_a(UT_LIST_GET_LEN(trx->lock.trx_locks) == 0);
ut_ad(UT_LIST_GET_LEN(trx->lock.evicted_tables) == 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_ACTIVE:
case TRX_STATE_PREPARED: case TRX_STATE_PREPARED:
case TRX_STATE_PREPARED_RECOVERED: 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.wait_thr = NULL;
trx->lock.que_state = TRX_QUE_RUNNING;
}
ut_ad(trx->lock.n_active_thrs == 1);
return; return;
case TRX_STATE_COMMITTED_IN_MEMORY: case TRX_STATE_COMMITTED_IN_MEMORY:
@ -1638,13 +1627,11 @@ trx_commit_step(
trx = thr_get_trx(thr); trx = thr_get_trx(thr);
ut_a(trx->lock.wait_thr == NULL); 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_or_rollback_prepare(trx);
trx->commit(); trx->commit();
ut_ad(trx->lock.wait_thr == NULL); ut_ad(trx->lock.wait_thr == NULL);
trx->lock.que_state = TRX_QUE_RUNNING;
thr = NULL; thr = NULL;
} else { } else {
@ -1814,19 +1801,12 @@ state_ok:
newline = TRUE; newline = TRUE;
/* trx->lock.que_state of an ACTIVE transaction may change if (trx->in_rollback) { /* dirty read for performance reasons */
while we are not holding trx->mutex. We perform a dirty read fputs("ROLLING BACK ", f);
for performance reasons. */ } else if (trx->lock.wait_lock) {
fputs("LOCK WAIT ", f);
switch (trx->lock.que_state) { } else {
case TRX_QUE_RUNNING: newline = FALSE;
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 (n_trx_locks > 0 || heap_size > 400) { if (n_trx_locks > 0 || heap_size > 400) {