summaryrefslogtreecommitdiff
path: root/sql
diff options
context:
space:
mode:
Diffstat (limited to 'sql')
-rw-r--r--sql/log_event_old.cc2
-rw-r--r--sql/mdl.cc172
-rw-r--r--sql/mdl.h13
-rw-r--r--sql/mysql_priv.h3
-rw-r--r--sql/sql_base.cc165
-rw-r--r--sql/sql_class.h15
-rw-r--r--sql/sql_insert.cc2
-rw-r--r--sql/sql_plist.h6
-rw-r--r--sql/sql_show.cc4
-rw-r--r--sql/sql_update.cc6
10 files changed, 303 insertions, 85 deletions
diff --git a/sql/log_event_old.cc b/sql/log_event_old.cc
index 942396fc3da..0f6f4d1d0e5 100644
--- a/sql/log_event_old.cc
+++ b/sql/log_event_old.cc
@@ -1506,7 +1506,7 @@ int Old_rows_log_event::do_apply_event(Relay_log_info const *rli)
*/
thd->binlog_flush_pending_rows_event(false);
TABLE_LIST *tables= rli->tables_to_lock;
- close_tables_for_reopen(thd, &tables);
+ close_tables_for_reopen(thd, &tables, NULL);
uint tables_count= rli->tables_to_lock_count;
if ((error= open_tables(thd, &tables, &tables_count, 0)))
diff --git a/sql/mdl.cc b/sql/mdl.cc
index 40074879e21..af7f310e598 100644
--- a/sql/mdl.cc
+++ b/sql/mdl.cc
@@ -196,6 +196,7 @@ void MDL_context::init(THD *thd_arg)
to empty the list.
*/
m_tickets.empty();
+ m_is_waiting_in_mdl= FALSE;
}
@@ -803,14 +804,28 @@ MDL_context::clone_ticket(MDL_request *mdl_request)
@retval FALSE Lock is not a shared one or no thread was woken up
*/
-static bool notify_shared_lock(THD *thd, MDL_ticket *conflicting_ticket)
+bool notify_shared_lock(THD *thd, MDL_ticket *conflicting_ticket)
{
bool woke= FALSE;
if (conflicting_ticket->is_shared())
{
THD *conflicting_thd= conflicting_ticket->get_ctx()->get_thd();
DBUG_ASSERT(thd != conflicting_thd); /* Self-deadlock */
- woke= mysql_notify_thread_having_shared_lock(thd, conflicting_thd);
+
+ /*
+ If the thread that holds the conflicting lock is waiting
+ on an MDL lock, wake it up by broadcasting on COND_mdl.
+ Otherwise it must be waiting on a table-level lock
+ or some other non-MDL resource, so delegate its waking up
+ to an external call.
+ */
+ if (conflicting_ticket->get_ctx()->is_waiting_in_mdl())
+ {
+ pthread_cond_broadcast(&COND_mdl);
+ woke= TRUE;
+ }
+ else
+ woke= mysql_notify_thread_having_shared_lock(thd, conflicting_thd);
}
return woke;
}
@@ -957,7 +972,7 @@ bool MDL_context::acquire_exclusive_locks(MDL_request_list *mdl_requests)
to abort this thread once again.
*/
struct timespec abstime;
- set_timespec(abstime, 10);
+ set_timespec(abstime, 1);
pthread_cond_timedwait(&COND_mdl, &LOCK_mdl, &abstime);
}
if (mysys_var->abort)
@@ -1032,6 +1047,7 @@ MDL_ticket::upgrade_shared_lock_to_exclusive()
const char *old_msg;
st_my_thread_var *mysys_var= my_thread_var;
THD *thd= m_ctx->get_thd();
+ MDL_ticket *pending_ticket;
DBUG_ENTER("MDL_ticket::upgrade_shared_lock_to_exclusive");
DEBUG_SYNC(thd, "mdl_upgrade_shared_lock_to_exclusive");
@@ -1045,8 +1061,22 @@ MDL_ticket::upgrade_shared_lock_to_exclusive()
/* Only allow upgrades from MDL_SHARED_UPGRADABLE */
DBUG_ASSERT(m_type == MDL_SHARED_UPGRADABLE);
+ /*
+ Create an auxiliary ticket to represent a pending exclusive
+ lock and add it to the 'waiting' queue for the duration
+ of upgrade. During upgrade we abort waits of connections
+ that own conflicting locks. A pending request is used
+ to signal such connections that upon waking up they
+ must back off, rather than fall into sleep again.
+ */
+ if (! (pending_ticket= MDL_ticket::create(m_ctx, MDL_EXCLUSIVE)))
+ DBUG_RETURN(TRUE);
+
pthread_mutex_lock(&LOCK_mdl);
+ pending_ticket->m_lock= m_lock;
+ m_lock->waiting.push_front(pending_ticket);
+
old_msg= MDL_ENTER_COND(thd, mysys_var);
/*
@@ -1088,6 +1118,30 @@ MDL_ticket::upgrade_shared_lock_to_exclusive()
MDL_ticket *conflicting_ticket;
MDL_lock::Ticket_iterator it(m_lock->granted);
+ /*
+ A temporary work-around to avoid deadlocks/livelocks in
+ a situation when in one connection ALTER TABLE tries to
+ upgrade its metadata lock and in another connection
+ the active transaction already got this lock in some
+ of its earlier statements.
+ In such case this transaction always succeeds with getting
+ a metadata lock on the table -- it already has one.
+ But later on it may block on the table level lock, since ALTER
+ got TL_WRITE_ALLOW_READ, and subsequently get aborted
+ by notify_shared_lock().
+ An abort will lead to a back off, and a second attempt to
+ get an MDL lock (successful), and a table lock (-> livelock).
+
+ The call below breaks this loop by forcing transactions to call
+ tdc_wait_for_old_versions() (even if the transaction doesn't need
+ any new metadata locks), which in turn will check if someone
+ is waiting on the owned MDL lock, and produce ER_LOCK_DEADLOCK.
+
+ TODO: Long-term such deadlocks/livelock will be resolved within
+ MDL subsystem and thus this call will become unnecessary.
+ */
+ mysql_abort_transactions_with_shared_lock(&m_lock->key);
+
while ((conflicting_ticket= it++))
{
if (conflicting_ticket->m_ctx != m_ctx)
@@ -1108,12 +1162,15 @@ MDL_ticket::upgrade_shared_lock_to_exclusive()
to abort this thread once again.
*/
struct timespec abstime;
- set_timespec(abstime, 10);
+ set_timespec(abstime, 1);
DBUG_PRINT("info", ("Failed to wake-up from table-level lock ... sleeping"));
pthread_cond_timedwait(&COND_mdl, &LOCK_mdl, &abstime);
}
if (mysys_var->abort)
{
+ /* Remove and destroy the auxiliary pending ticket. */
+ m_lock->waiting.remove(pending_ticket);
+ MDL_ticket::destroy(pending_ticket);
/* Pending requests for shared locks can be satisfied now. */
pthread_cond_broadcast(&COND_mdl);
MDL_EXIT_COND(thd, mysys_var, old_msg);
@@ -1124,6 +1181,11 @@ MDL_ticket::upgrade_shared_lock_to_exclusive()
m_lock->type= MDL_lock::MDL_LOCK_EXCLUSIVE;
/* Set the new type of lock in the ticket. */
m_type= MDL_EXCLUSIVE;
+
+ /* Remove and destroy the auxiliary pending ticket. */
+ m_lock->waiting.remove(pending_ticket);
+ MDL_ticket::destroy(pending_ticket);
+
if (m_lock->cached_object)
(*m_lock->cached_object_release_hook)(m_lock->cached_object);
m_lock->cached_object= 0;
@@ -1240,6 +1302,59 @@ bool MDL_context::acquire_global_shared_lock()
/**
+ Check if there are any pending exclusive locks which conflict
+ with shared locks held by this thread.
+
+ @pre The caller already has acquired LOCK_mdl.
+
+ @return TRUE If there are any pending conflicting locks.
+ FALSE Otherwise.
+*/
+
+bool MDL_context::can_wait_lead_to_deadlock_impl() const
+{
+ Ticket_iterator ticket_it(m_tickets);
+ MDL_ticket *ticket;
+
+ while ((ticket= ticket_it++))
+ {
+ /*
+ In MySQL we never call this method while holding exclusive or
+ upgradeable shared metadata locks.
+ Otherwise we would also have to check for the presence of pending
+ requests for conflicting types of global lock.
+ In addition MDL_ticket::has_pending_conflicting_lock_impl()
+ won't work properly for exclusive type of lock.
+ */
+ DBUG_ASSERT(! ticket->is_upgradable_or_exclusive());
+
+ if (ticket->has_pending_conflicting_lock_impl())
+ return TRUE;
+ }
+ return FALSE;
+}
+
+
+/**
+ Implement a simple deadlock detection heuristic: check if there
+ are any pending exclusive locks which conflict with shared locks
+ held by this thread. In that case waiting can be circular,
+ i.e. lead to a deadlock.
+
+ @return TRUE if there are any conflicting locks, FALSE otherwise.
+*/
+
+bool MDL_context::can_wait_lead_to_deadlock() const
+{
+ bool result;
+ pthread_mutex_lock(&LOCK_mdl);
+ result= can_wait_lead_to_deadlock_impl();
+ pthread_mutex_unlock(&LOCK_mdl);
+ return result;
+}
+
+
+/**
Wait until there will be no locks that conflict with lock requests
in the given list.
@@ -1249,7 +1364,7 @@ bool MDL_context::acquire_global_shared_lock()
Does not acquire the locks!
@retval FALSE Success. One can try to obtain metadata locks.
- @retval TRUE Failure (thread was killed)
+ @retval TRUE Failure (thread was killed or deadlock is possible).
*/
bool
@@ -1278,6 +1393,26 @@ MDL_context::wait_for_locks(MDL_request_list *mdl_requests)
mysql_ha_flush(m_thd);
pthread_mutex_lock(&LOCK_mdl);
old_msg= MDL_ENTER_COND(m_thd, mysys_var);
+
+ /*
+ In cases when we wait while still holding some metadata
+ locks deadlocks are possible.
+ To avoid them we use the following simple empiric - don't
+ wait for new lock request to be satisfied if for one of the
+ locks which are already held by this connection there is
+ a conflicting request (i.e. this connection should not wait
+ if someone waits for it).
+ This empiric should work well (e.g. give low number of false
+ negatives) in situations when conflicts are rare (in our
+ case this is true since DDL statements should be rare).
+ */
+ if (can_wait_lead_to_deadlock_impl())
+ {
+ MDL_EXIT_COND(m_thd, mysys_var, old_msg);
+ my_error(ER_LOCK_DEADLOCK, MYF(0));
+ return TRUE;
+ }
+
it.rewind();
while ((mdl_request= it++))
{
@@ -1301,7 +1436,9 @@ MDL_context::wait_for_locks(MDL_request_list *mdl_requests)
MDL_EXIT_COND(m_thd, mysys_var, old_msg);
break;
}
+ m_is_waiting_in_mdl= TRUE;
pthread_cond_wait(&COND_mdl, &LOCK_mdl);
+ m_is_waiting_in_mdl= FALSE;
/* As a side-effect MDL_EXIT_COND() unlocks LOCK_mdl. */
MDL_EXIT_COND(m_thd, mysys_var, old_msg);
}
@@ -1550,21 +1687,38 @@ MDL_context::is_lock_owner(MDL_key::enum_mdl_namespace mdl_namespace,
existing shared lock.
@pre The ticket must match an acquired lock.
+ @pre The caller already has acquired LOCK_mdl.
- @param ticket Shared lock against which check should be performed.
+ @return TRUE if there is a conflicting lock request, FALSE otherwise.
+*/
- @return TRUE if there are any conflicting locks, FALSE otherwise.
+bool MDL_ticket::has_pending_conflicting_lock_impl() const
+{
+ DBUG_ASSERT(is_shared());
+ safe_mutex_assert_owner(&LOCK_mdl);
+
+ return !m_lock->waiting.is_empty();
+}
+
+
+/**
+ Check if we have any pending exclusive locks which conflict with
+ existing shared lock.
+
+ @pre The ticket must match an acquired lock.
+
+ @return TRUE if there is a pending conflicting lock request,
+ FALSE otherwise.
*/
bool MDL_ticket::has_pending_conflicting_lock() const
{
bool result;
- DBUG_ASSERT(is_shared());
safe_mutex_assert_not_owner(&LOCK_open);
pthread_mutex_lock(&LOCK_mdl);
- result= !m_lock->waiting.is_empty();
+ result= has_pending_conflicting_lock_impl();
pthread_mutex_unlock(&LOCK_mdl);
return result;
}
diff --git a/sql/mdl.h b/sql/mdl.h
index e85f1232ff9..8edbfbc0777 100644
--- a/sql/mdl.h
+++ b/sql/mdl.h
@@ -300,6 +300,8 @@ private:
private:
MDL_ticket(const MDL_ticket &); /* not implemented */
MDL_ticket &operator=(const MDL_ticket &); /* not implemented */
+
+ bool has_pending_conflicting_lock_impl() const;
};
@@ -380,11 +382,20 @@ public:
void release_transactional_locks();
void rollback_to_savepoint(MDL_ticket *mdl_savepoint);
+ bool can_wait_lead_to_deadlock() const;
+
inline THD *get_thd() const { return m_thd; }
+
+ bool is_waiting_in_mdl() const { return m_is_waiting_in_mdl; }
private:
Ticket_list m_tickets;
bool m_has_global_shared_lock;
/**
+ Indicates that the owner of this context is waiting in
+ wait_for_locks() method.
+ */
+ bool m_is_waiting_in_mdl;
+ /**
This member has two uses:
1) When entering LOCK TABLES mode, remember the last taken
metadata lock. COMMIT/ROLLBACK must preserve these metadata
@@ -397,6 +408,7 @@ private:
THD *m_thd;
private:
void release_ticket(MDL_ticket *ticket);
+ bool can_wait_lead_to_deadlock_impl() const;
MDL_ticket *find_ticket(MDL_request *mdl_req,
bool *is_lt_or_ha);
void release_locks_stored_before(MDL_ticket *sentinel);
@@ -413,6 +425,7 @@ void mdl_destroy();
extern bool mysql_notify_thread_having_shared_lock(THD *thd, THD *in_use);
extern void mysql_ha_flush(THD *thd);
+extern void mysql_abort_transactions_with_shared_lock(const MDL_key *mdl_key);
extern "C" const char *set_thd_proc_info(THD *thd, const char *info,
const char *calling_function,
const char *calling_file,
diff --git a/sql/mysql_priv.h b/sql/mysql_priv.h
index 124392f4c63..bb5bf428ef0 100644
--- a/sql/mysql_priv.h
+++ b/sql/mysql_priv.h
@@ -1522,7 +1522,8 @@ void free_io_cache(TABLE *entry);
void intern_close_table(TABLE *entry);
bool close_thread_table(THD *thd, TABLE **table_ptr);
void close_temporary_tables(THD *thd);
-void close_tables_for_reopen(THD *thd, TABLE_LIST **tables);
+void close_tables_for_reopen(THD *thd, TABLE_LIST **tables,
+ MDL_ticket *mdl_savepoint);
TABLE_LIST *find_table_in_list(TABLE_LIST *table,
TABLE_LIST *TABLE_LIST::*link,
const char *db_name,
diff --git a/sql/sql_base.cc b/sql/sql_base.cc
index 78bb9f9bad7..d02e4f38807 100644
--- a/sql/sql_base.cc
+++ b/sql/sql_base.cc
@@ -2142,25 +2142,13 @@ bool rename_temporary_table(THD* thd, TABLE *table, const char *db,
bool wait_while_table_is_used(THD *thd, TABLE *table,
enum ha_extra_function function)
{
- enum thr_lock_type old_lock_type;
DBUG_ENTER("wait_while_table_is_used");
DBUG_PRINT("enter", ("table: '%s' share: 0x%lx db_stat: %u version: %lu",
table->s->table_name.str, (ulong) table->s,
table->db_stat, table->s->version));
- /* Ensure no one can reopen table before it's removed */
- pthread_mutex_lock(&LOCK_open);
- table->s->version= 0;
- pthread_mutex_unlock(&LOCK_open);
-
- old_lock_type= table->reginfo.lock_type;
- mysql_lock_abort(thd, table, TRUE); /* end threads waiting on lock */
-
if (table->mdl_ticket->upgrade_shared_lock_to_exclusive())
- {
- mysql_lock_downgrade_write(thd, table, old_lock_type);
DBUG_RETURN(TRUE);
- }
pthread_mutex_lock(&LOCK_open);
tdc_remove_table(thd, TDC_RT_REMOVE_NOT_OWN,
@@ -3722,9 +3710,10 @@ end_with_lock_open:
Open_table_context::Open_table_context(THD *thd)
:m_action(OT_NO_ACTION),
- m_can_deadlock((thd->in_multi_stmt_transaction() ||
- thd->mdl_context.lt_or_ha_sentinel())&&
- thd->mdl_context.has_locks())
+ m_start_of_statement_svp(thd->mdl_context.mdl_savepoint()),
+ m_has_locks((thd->in_multi_stmt_transaction() ||
+ thd->mdl_context.lt_or_ha_sentinel()) &&
+ thd->mdl_context.has_locks())
{}
@@ -3741,12 +3730,22 @@ Open_table_context::
request_backoff_action(enum_open_table_action action_arg)
{
/*
- We have met a exclusive metadata lock or a old version of
- table and we are inside a transaction that already hold locks.
- We can't follow the locking protocol in this scenario as it
- might lead to deadlocks.
+ We are inside a transaction that already holds locks and have
+ met a broken table or a table which needs re-discovery.
+ Performing any recovery action requires acquiring an exclusive
+ metadata lock on this table. Doing that with locks breaks the
+ metadata locking protocol and might lead to deadlocks,
+ so we report an error.
+
+ However, if we have only met a conflicting lock or an old
+ TABLE version, and just need to wait for the conflict to
+ disappear/old version to go away, allow waiting.
+ While waiting, we use a simple empiric to detect
+ deadlocks: we never wait on someone who's waiting too.
+ Waiting will be done after releasing metadata locks acquired
+ by this statement.
*/
- if (m_can_deadlock)
+ if (m_has_locks && action_arg != OT_WAIT)
{
my_error(ER_LOCK_DEADLOCK, MYF(0));
return TRUE;
@@ -4364,7 +4363,7 @@ restart:
elements from the table list (if MERGE tables are involved),
*/
TABLE_LIST *failed_table= *table_to_open;
- close_tables_for_reopen(thd, start);
+ close_tables_for_reopen(thd, start, ot_ctx.start_of_statement_svp());
/*
Here we rely on the fact that 'tables' still points to the valid
@@ -4414,7 +4413,8 @@ restart:
{
if (ot_ctx.can_recover_from_failed_open())
{
- close_tables_for_reopen(thd, start);
+ close_tables_for_reopen(thd, start,
+ ot_ctx.start_of_statement_svp());
if (ot_ctx.recover_from_failed_open(thd, &rt->mdl_request, NULL))
goto err;
@@ -4827,14 +4827,14 @@ retry:
while ((error= open_table(thd, table_list, thd->mem_root, &ot_ctx, 0)) &&
ot_ctx.can_recover_from_failed_open())
{
- /* We can't back off with an open HANDLER, we don't wait with locks. */
+ /* We never have an open HANDLER or LOCK TABLES here. */
DBUG_ASSERT(thd->mdl_context.lt_or_ha_sentinel() == NULL);
/*
Even though we have failed to open table we still need to
call release_transactional_locks() to release metadata locks which
might have been acquired successfully.
*/
- thd->mdl_context.release_transactional_locks();
+ thd->mdl_context.rollback_to_savepoint(ot_ctx.start_of_statement_svp());
table_list->mdl_request.ticket= 0;
if (ot_ctx.recover_from_failed_open(thd, &table_list->mdl_request,
table_list))
@@ -4876,24 +4876,13 @@ retry:
{
if (refresh)
{
- if (ot_ctx.can_deadlock())
- {
- my_error(ER_LOCK_DEADLOCK, MYF(0));
- table= 0;
- }
- else
- {
- close_thread_tables(thd);
- table_list->table= NULL;
- table_list->mdl_request.ticket= NULL;
- /*
- We can't back off with an open HANDLER,
- we don't wait with locks.
- */
- DBUG_ASSERT(thd->mdl_context.lt_or_ha_sentinel() == NULL);
- thd->mdl_context.release_transactional_locks();
- goto retry;
- }
+ close_thread_tables(thd);
+ table_list->table= NULL;
+ table_list->mdl_request.ticket= NULL;
+ /* We never have an open HANDLER or LOCK TABLES here. */
+ DBUG_ASSERT(thd->mdl_context.lt_or_ha_sentinel() == NULL);
+ thd->mdl_context.rollback_to_savepoint(ot_ctx.start_of_statement_svp());
+ goto retry;
}
else
table= 0;
@@ -4941,7 +4930,16 @@ bool open_and_lock_tables_derived(THD *thd, TABLE_LIST *tables,
{
uint counter;
bool need_reopen;
- bool has_locks= thd->mdl_context.has_locks();
+ /*
+ Remember the set of metadata locks which this connection
+ managed to acquire before the start of the current statement.
+ It can be either transaction-scope locks, or HANDLER locks,
+ or LOCK TABLES locks. If mysql_lock_tables() fails with
+ need_reopen request, we'll use it to instruct
+ close_tables_for_reopen() to release all locks of this
+ statement.
+ */
+ MDL_ticket *start_of_statement_svp= thd->mdl_context.mdl_savepoint();
DBUG_ENTER("open_and_lock_tables_derived");
DBUG_PRINT("enter", ("derived handling: %d", derived));
@@ -4960,13 +4958,7 @@ bool open_and_lock_tables_derived(THD *thd, TABLE_LIST *tables,
break;
if (!need_reopen)
DBUG_RETURN(TRUE);
- if ((thd->in_multi_stmt_transaction() ||
- thd->mdl_context.lt_or_ha_sentinel()) && has_locks)
- {
- my_error(ER_LOCK_DEADLOCK, MYF(0));
- DBUG_RETURN(TRUE);
- }
- close_tables_for_reopen(thd, &tables);
+ close_tables_for_reopen(thd, &tables, start_of_statement_svp);
}
if (derived &&
(mysql_handle_derived(thd->lex, &mysql_derived_prepare) ||
@@ -5280,6 +5272,8 @@ bool lock_tables(THD *thd, TABLE_LIST *tables, uint count,
flags, need_reopen)))
DBUG_RETURN(TRUE);
+ DEBUG_SYNC(thd, "after_lock_tables_takes_lock");
+
if (thd->lex->requires_prelocking() &&
thd->lex->sql_command != SQLCOM_LOCK_TABLES)
{
@@ -5379,18 +5373,24 @@ bool lock_tables(THD *thd, TABLE_LIST *tables, uint count,
}
-/*
+/**
Prepare statement for reopening of tables and recalculation of set of
prelocked tables.
- SYNOPSIS
- close_tables_for_reopen()
- thd in Thread context
- tables in/out List of tables which we were trying to open and lock
-
+ @param[in] thd Thread context.
+ @param[in,out] tables List of tables which we were trying to open
+ and lock.
+ @param[in] start_of_statement_svp MDL savepoint which represents the set
+ of metadata locks which the current transaction
+ managed to acquire before execution of the current
+ statement and to which we should revert before
+ trying to reopen tables. NULL if no metadata locks
+ were held and thus all metadata locks should be
+ released.
*/
-void close_tables_for_reopen(THD *thd, TABLE_LIST **tables)
+void close_tables_for_reopen(THD *thd, TABLE_LIST **tables,
+ MDL_ticket *start_of_statement_svp)
{
TABLE_LIST *first_not_own_table= thd->lex->first_not_own_table();
TABLE_LIST *tmp;
@@ -5425,13 +5425,7 @@ void close_tables_for_reopen(THD *thd, TABLE_LIST **tables)
for (tmp= first_not_own_table; tmp; tmp= tmp->next_global)
tmp->mdl_request.ticket= NULL;
close_thread_tables(thd);
- /* We can't back off with an open HANDLERs, we must not wait with locks. */
- DBUG_ASSERT(thd->mdl_context.lt_or_ha_sentinel() == NULL);
- /*
- Due to the above assert, this effectively releases *all* locks
- of this session, so that we can safely wait on tables.
- */
- thd->mdl_context.release_transactional_locks();
+ thd->mdl_context.rollback_to_savepoint(start_of_statement_svp);
}
@@ -8413,6 +8407,8 @@ bool mysql_notify_thread_having_shared_lock(THD *thd, THD *in_use)
But in case a thread has an open HANDLER statement,
(and thus already grabbed a metadata lock), it gets
blocked only too late -- at the table cache level.
+ Starting from 5.5, this could also easily happen in
+ a multi-statement transaction.
*/
broadcast_refresh();
pthread_mutex_unlock(&LOCK_open);
@@ -8421,6 +8417,28 @@ bool mysql_notify_thread_having_shared_lock(THD *thd, THD *in_use)
/**
+ Force transactions holding shared metadata lock on the table to call
+ MDL_context::can_wait_lead_to_deadlock() even if they don't need any
+ new metadata locks so they can detect potential deadlocks between
+ metadata locking subsystem and table-level locks.
+
+ @param mdl_key MDL key for the table on which we are upgrading
+ metadata lock.
+*/
+
+void mysql_abort_transactions_with_shared_lock(const MDL_key *mdl_key)
+{
+ if (mdl_key->mdl_namespace() == MDL_key::TABLE)
+ {
+ pthread_mutex_lock(&LOCK_open);
+ tdc_remove_table(NULL, TDC_RT_REMOVE_UNUSED, mdl_key->db_name(),
+ mdl_key->name());
+ pthread_mutex_unlock(&LOCK_open);
+ }
+}
+
+
+/**
Remove all or some (depending on parameter) instances of TABLE and
TABLE_SHARE from the table definition cache.
@@ -8525,6 +8543,25 @@ tdc_wait_for_old_versions(THD *thd, MDL_request_list *mdl_requests)
to broadcast on COND_refresh because of this.
*/
mysql_ha_flush(thd);
+
+ /*
+ Check if there is someone waiting for one of metadata locks
+ held by this connection and return an error if that's the
+ case, since this situation may lead to a deadlock.
+ This can happen, when, for example, this connection is
+ waiting for an old version of some table to go away and
+ another connection is trying to upgrade its shared
+ metadata lock to exclusive, and thus is waiting
+ for this to release its lock. We must check for
+ the condition on each iteration of the loop to remove
+ any window for a race.
+ */
+ if (thd->mdl_context.can_wait_lead_to_deadlock())
+ {
+ my_error(ER_LOCK_DEADLOCK, MYF(0));
+ return TRUE;
+ }
+
pthread_mutex_lock(&LOCK_open);
MDL_request_list::Iterator it(*mdl_requests);
diff --git a/sql/sql_class.h b/sql/sql_class.h
index ff1b51e7e87..5654dcb07a6 100644
--- a/sql/sql_class.h
+++ b/sql/sql_class.h
@@ -1297,18 +1297,29 @@ public:
bool can_recover_from_failed_open() const
{ return m_action != OT_NO_ACTION; }
- bool can_deadlock() const { return m_can_deadlock; }
+
+ /**
+ When doing a back-off, we close all tables acquired by this
+ statement. Return an MDL savepoint taken at the beginning of
+ the statement, so that we can rollback to it before waiting on
+ locks.
+ */
+ MDL_ticket *start_of_statement_svp() const
+ {
+ return m_start_of_statement_svp;
+ }
private:
/** List of requests for all locks taken so far. Used for waiting on locks. */
MDL_request_list m_mdl_requests;
/** Back off action. */
enum enum_open_table_action m_action;
+ MDL_ticket *m_start_of_statement_svp;
/**
Whether we had any locks when this context was created.
If we did, they are from the previous statement of a transaction,
and we can't safely do back-off (and release them).
*/
- bool m_can_deadlock;
+ bool m_has_locks;
};
diff --git a/sql/sql_insert.cc b/sql/sql_insert.cc
index 40ef55423a9..f2478213bbe 100644
--- a/sql/sql_insert.cc
+++ b/sql/sql_insert.cc
@@ -2530,7 +2530,7 @@ pthread_handler_t handle_delayed_insert(void *arg)
aborted. Try to reopen table and if it fails die.
*/
TABLE_LIST *tl_ptr = &di->table_list;
- close_tables_for_reopen(thd, &tl_ptr);
+ close_tables_for_reopen(thd, &tl_ptr, NULL);
di->table= 0;
if (di->open_and_lock_table())
{
diff --git a/sql/sql_plist.h b/sql/sql_plist.h
index 94e437362a9..8f2aee6bd5f 100644
--- a/sql/sql_plist.h
+++ b/sql/sql_plist.h
@@ -132,11 +132,11 @@ public:
template <typename T, typename B>
class I_P_List_iterator
{
- I_P_List<T, B> *list;
+ const I_P_List<T, B> *list;
T *current;
public:
- I_P_List_iterator(I_P_List<T, B> &a) : list(&a), current(a.first) {}
- I_P_List_iterator(I_P_List<T, B> &a, T* current_arg) : list(&a), current(current_arg) {}
+ I_P_List_iterator(const I_P_List<T, B> &a) : list(&a), current(a.first) {}
+ I_P_List_iterator(const I_P_List<T, B> &a, T* current_arg) : list(&a), current(current_arg) {}
inline void init(I_P_List<T, B> &a)
{
list= &a;
diff --git a/sql/sql_show.cc b/sql/sql_show.cc
index 72fb49cf38c..e9d1426b3e3 100644
--- a/sql/sql_show.cc
+++ b/sql/sql_show.cc
@@ -2941,7 +2941,7 @@ fill_schema_show_cols_or_idxs(THD *thd, TABLE_LIST *tables,
table, res, db_name,
table_name));
thd->temporary_tables= 0;
- close_tables_for_reopen(thd, &show_table_list);
+ close_tables_for_reopen(thd, &show_table_list, NULL);
DBUG_RETURN(error);
}
@@ -3500,7 +3500,7 @@ int get_all_tables(THD *thd, TABLE_LIST *tables, COND *cond)
res= schema_table->process_table(thd, show_table_list, table,
res, &orig_db_name,
&tmp_lex_string);
- close_tables_for_reopen(thd, &show_table_list);
+ close_tables_for_reopen(thd, &show_table_list, NULL);
}
DBUG_ASSERT(!lex->query_tables_own_last);
if (res)
diff --git a/sql/sql_update.cc b/sql/sql_update.cc
index 603ab1b9682..980f87f21bb 100644
--- a/sql/sql_update.cc
+++ b/sql/sql_update.cc
@@ -206,6 +206,7 @@ int mysql_update(THD *thd,
ulonglong id;
List<Item> all_fields;
THD::killed_state killed_status= THD::NOT_KILLED;
+ MDL_ticket *start_of_statement_svp= thd->mdl_context.mdl_savepoint();
DBUG_ENTER("mysql_update");
for ( ; ; )
@@ -226,7 +227,7 @@ int mysql_update(THD *thd,
break;
if (!need_reopen)
DBUG_RETURN(1);
- close_tables_for_reopen(thd, &table_list);
+ close_tables_for_reopen(thd, &table_list, start_of_statement_svp);
}
if (mysql_handle_derived(thd->lex, &mysql_derived_prepare) ||
@@ -981,6 +982,7 @@ int mysql_multi_update_prepare(THD *thd)
const bool using_lock_tables= thd->locked_tables_mode != LTM_NONE;
bool original_multiupdate= (thd->lex->sql_command == SQLCOM_UPDATE_MULTI);
bool need_reopen= FALSE;
+ MDL_ticket *start_of_statement_svp= thd->mdl_context.mdl_savepoint();
DBUG_ENTER("mysql_multi_update_prepare");
/* following need for prepared statements, to run next time multi-update */
@@ -1145,7 +1147,7 @@ reopen_tables:
*/
cleanup_items(thd->free_list);
- close_tables_for_reopen(thd, &table_list);
+ close_tables_for_reopen(thd, &table_list, start_of_statement_svp);
goto reopen_tables;
}