Commit 571acc87 authored by Dmitry Lenev's avatar Dmitry Lenev

Patch that changes approach to how we acquire metadata

locks for DML statements and changes the way MDL locks
are acquired/granted in contended case.

Instead of backing-off when a lock conflict is encountered
and waiting for it to go away before restarting open_tables()
process we now wait for lock to be released without releasing
any previously acquired locks. If conflicting lock goes away
we resume opening tables. If waiting leads to a deadlock we
try to resolve it by backing-off and restarting open_tables()
immediately.

As result both waiting for possibility to acquire and
acquiring of a metadata lock now always happen within the
same MDL API call. This has allowed to make release of a lock
and granting it to the most appropriate pending request an
atomic operation.
Thanks to this it became possible to wake up during release
of lock only those waiters which requests can be satisfied
at the moment as well as wake up only one waiter in case
when granting its request would prevent all other requests
from being satisfied. This solves thundering herd problem
which occured in cases when we were releasing some lock and
woke up many waiters for SNRW or X locks (this was the issue
in bug#52289 "performance regression for MyISAM in sysbench
OLTP_RW test".
This also allowed to implement more fair (FIFO) scheduling
among waiters with the same priority.
It also opens the door for introducing new types of requests
for metadata locks such as low-prio SNRW lock which is
necessary in order to support LOCK TABLES LOW_PRIORITY WRITE.

Notice that after this sometimes can report ER_LOCK_DEADLOCK
error in cases in which it has not happened before.
Particularly we will always report this error if waiting for
conflicting lock has happened in the middle of transaction
and resulted in a deadlock. Before this patch the error was
not reported if deadlock could have been resolved by backing
off all metadata locks acquired by the current statement.
parent 1af11051
......@@ -1765,6 +1765,7 @@ drop tables t1, t2;
# locking subsystem.
#
drop tables if exists t0, t1, t2, t3, t4, t5;
set debug_sync= 'RESET';
create table t1 (i int);
create table t2 (j int);
create table t3 (k int);
......@@ -1943,6 +1944,98 @@ commit;
# Reap ALTER TABLE ... RENAME.
drop table t2;
#
# Test that in situation when MDL subsystem detects a deadlock
# but it turns out that it can be resolved by backing-off locks
# acquired by one of participating transactions (which is
# possible when one of transactions consists only of currently
# executed statement, e.g. in autocommit mode) no error is
# reported.
#
create table t1 (i int);
create table t2 (j int);
# Ensure that the below SELECT stops once it has acquired metadata
# lock on table 't2'.
set debug_sync= 'after_open_table_mdl_shared SIGNAL locked WAIT_FOR finish';
# Sending:
select * from t2, t1;
#
# Switching to connection 'deadlock_con1'.
# Wait till SELECT acquires MDL on 't2' and starts waiting for signal.
set debug_sync= 'now WAIT_FOR locked';
# Sending:
lock tables t1 write, t2 write;
#
# Switching to connection 'deadlock_con2'.
# Wait until LOCK TABLES acquires SNRW lock on 't1' and is blocked
# while trying to acquire SNRW lock on 't1'.
# Resume SELECT execution, this should eventually unblock LOCK TABLES.
set debug_sync= 'now SIGNAL finish';
#
# Switching to connection 'deadlock_con1'.
# Reaping LOCK TABLES.
unlock tables;
#
# Switching to connection 'default'.
# Reaping SELECT. It succeed and not report ER_LOCK_DEADLOCK error.
j i
drop tables t1, t2;
#
# Test coverage for situation in which a race has happened
# during deadlock detection process which led to unwarranted
# ER_LOCK_DEADLOCK error.
#
create table t1 (i int);
# Ensure that ALTER waits once it has acquired SNW lock.
set debug_sync='after_open_table_mdl_shared SIGNAL parked1 WAIT_FOR go1';
# Sending:
alter table t1 add column j int;
#
# Switching to connection 'deadlock_con1'.
# Wait till ALTER acquires SNW lock and stops.
set debug_sync='now WAIT_FOR parked1';
# Ensure that INSERT is paused once it detects that there is
# a conflicting metadata lock so it has to wait, but before
# deadlock detection is run.
set debug_sync='mdl_acquire_lock_wait SIGNAL parked2 WAIT_FOR go2';
# Sending:
insert into t1 values ();
#
# Switching to connection 'deadlock_con2'.
# Wait till INSERT is paused.
set debug_sync='now WAIT_FOR parked2';
# Resume ALTER execution. Eventually it will release its
# metadata lock and INSERT's request for SW lock will be
# satisified.
set debug_sync='now SIGNAL go1';
#
# Switching to connection 'default'.
# Reaping ALTER TABLE.
# Add a new request for SNW lock to waiting graph.
# Sending:
alter table t1 drop column j;
#
# Switching to connection 'deadlock_con2'.
# Wait until ALTER is blocked.
# Resume INSERT so it can start deadlock detection.
#
# At this point there is a discrepancy between the fact that INSERT's
# SW lock is already satisfied, but INSERT's connection is still
# marked as waiting for it. Looking for a loop in waiters graph
# without additional checks has detected a deadlock (INSERT waits
# for SW lock; which is not granted because of pending SNW lock from
# ALTER; which waits for active SW lock from INSERT). Since requests
# for SW and SNW locks have same weight ALTER was selected as a victim
# and ended with ER_LOCK_DEADLOCK error.
set debug_sync='now SIGNAL go2';
#
# Switching to connection 'deadlock_con1'.
# Reaping INSERT.
#
# Switching to connection 'default'.
# Reaping ALTER. It should succeed and not produce ER_LOCK_DEADLOCK.
drop table t1;
set debug_sync= 'RESET';
#
# Test for bug #46748 "Assertion in MDL_context::wait_for_locks()
# on INSERT + CREATE TRIGGER".
#
......@@ -2175,7 +2268,7 @@ alter table t1 add column e int, rename to t2;;
#
# Switching to connection 'default'.
set debug_sync='now WAIT_FOR alter_table_locked';
set debug_sync='before_open_table_wait_refresh SIGNAL alter_go';
set debug_sync='mdl_acquire_lock_wait SIGNAL alter_go';
# The below statement should get ER_LOCK_DEADLOCK error
# (i.e. it should not allow ALTER to proceed, and then
# fail due to 't1' changing its name to 't2').
......
......@@ -59,30 +59,31 @@ SET DEBUG_SYNC= 'RESET';
#
# Bug #48246 assert in close_thread_table
#
CREATE TABLE t0 (b INTEGER);
CREATE TABLE t1 (a INTEGER);
CREATE FUNCTION f1(b INTEGER) RETURNS INTEGER RETURN 1;
CREATE PROCEDURE p1() SELECT COUNT(f1(a)) FROM t1;
CREATE PROCEDURE p1() SELECT COUNT(f1(a)) FROM t1, t0;
INSERT INTO t0 VALUES(1);
INSERT INTO t1 VALUES(1), (2);
# Connection 2
CALL p1();
COUNT(f1(a))
2
# Connection default
SET DEBUG_SYNC= 'after_open_table_mdl_shared SIGNAL locked WAIT_FOR called';
# Sending:
CREATE TABLE t1 (a INTEGER);
# Connection 2
SET DEBUG_SYNC= 'now WAIT_FOR locked';
SET DEBUG_SYNC= 'before_open_table_wait_refresh SIGNAL called WAIT_FOR created';
# This call used to cause an assertion. MDL locking conflict will
# cause back-off and retry. A variable indicating if a prelocking list
# exists, used to be not reset properly causing an eventual assert.
SET DEBUG_SYNC= 'after_open_table_mdl_shared SIGNAL locked_t1 WAIT_FOR go_for_t0';
# This call used to cause an assertion. MDL deadlock with upcoming
# LOCK TABLES statement will cause back-off and retry.
# A variable indicating if a prelocking list exists, used to be not
# reset properly causing an eventual assert.
# Sending:
CALL p1();
# Connection default
# Reaping: CREATE TABLE t1 (a INTEGER)
ERROR 42S01: Table 't1' already exists
SET DEBUG_SYNC= 'now SIGNAL created';
SET DEBUG_SYNC= 'now WAIT_FOR locked_t1';
# Issue LOCK TABLES statement which will enter in MDL deadlock
# with CALL statement and as result will cause it to perform
# back-off and retry.
SET DEBUG_SYNC= 'mdl_acquire_lock_wait SIGNAL go_for_t0';
LOCK TABLES t0 WRITE, t1 WRITE;
UNLOCK TABLES;
# Connection 2
# Reaping: CALL p1()
COUNT(f1(a))
......@@ -90,5 +91,5 @@ COUNT(f1(a))
# Connection default
DROP PROCEDURE p1;
DROP FUNCTION f1;
DROP TABLE t1;
DROP TABLES t0, t1;
SET DEBUG_SYNC= 'RESET';
......@@ -25,7 +25,7 @@ wait/synch/rwlock/sql/LOCK_system_variables_hash YES YES
wait/synch/rwlock/sql/LOCK_sys_init_connect YES YES
wait/synch/rwlock/sql/LOCK_sys_init_slave YES YES
wait/synch/rwlock/sql/LOGGER::LOCK_logger YES YES
wait/synch/rwlock/sql/MDL_context::waiting_for_lock YES YES
wait/synch/rwlock/sql/MDL_context::LOCK_waiting_for YES YES
wait/synch/rwlock/sql/MDL_lock::rwlock YES YES
wait/synch/rwlock/sql/Query_cache_query::lock YES YES
wait/synch/rwlock/sql/THR_LOCK_servers YES YES
......
......@@ -1215,18 +1215,23 @@ lock table t2 write;
# Sending 'insert into t1 (a) values (f1())'...
insert into t1 (a) values (f1());
# Waitng for 'insert into t1 ...' to get blocked on table lock...
# Sending 'drop function f1'. It will abort the table lock wait.
drop function f1;
# Sending 'drop function f1'. It will wait till insert finishes.
drop function f1;;
# --> connection default
# Check that 'drop function f1' gets blocked.
# Now let's let 'insert' go through...
unlock tables;
# --> connection con1
# --> connection master
# Reaping 'insert into t1 (a) values (f1())'...
ERROR 42000: FUNCTION test.f1 does not exist
# --> connection master1
# Reaping 'drop function f1'
# --> connection master
select * from t1;
a
1
select * from t1;
a
1
drop table t1, t2;
drop function f1;
ERROR 42000: FUNCTION test.f1 does not exist
......
......@@ -657,17 +657,25 @@ connection master1;
let $wait_condition=select count(*)=1 from information_schema.processlist
where state='Waiting for table' and info='insert into t1 (a) values (f1())';
--source include/wait_condition.inc
--echo # Sending 'drop function f1'. It will abort the table lock wait.
drop function f1;
--echo # Sending 'drop function f1'. It will wait till insert finishes.
--send drop function f1;
--echo # --> connection default
connection default;
--echo # Check that 'drop function f1' gets blocked.
let $wait_condition=select count(*)=1 from information_schema.processlist
where state='Waiting for table' and info='drop function f1';
--source include/wait_condition.inc
--echo # Now let's let 'insert' go through...
unlock tables;
--echo # --> connection con1
--echo # --> connection master
connection master;
--echo # Reaping 'insert into t1 (a) values (f1())'...
--error ER_SP_DOES_NOT_EXIST
--reap
--echo # --> connection master1
connection master1;
--echo # Reaping 'drop function f1'
--reap
--echo # --> connection master
connection master;
select * from t1;
sync_slave_with_master;
......
......@@ -2411,6 +2411,7 @@ drop tables t1, t2;
--disable_warnings
drop tables if exists t0, t1, t2, t3, t4, t5;
--enable_warnings
set debug_sync= 'RESET';
connect(deadlock_con1,localhost,root,,);
connect(deadlock_con2,localhost,root,,);
......@@ -2700,6 +2701,136 @@ connection default;
drop table t2;
--echo #
--echo # Test that in situation when MDL subsystem detects a deadlock
--echo # but it turns out that it can be resolved by backing-off locks
--echo # acquired by one of participating transactions (which is
--echo # possible when one of transactions consists only of currently
--echo # executed statement, e.g. in autocommit mode) no error is
--echo # reported.
--echo #
create table t1 (i int);
create table t2 (j int);
--echo # Ensure that the below SELECT stops once it has acquired metadata
--echo # lock on table 't2'.
set debug_sync= 'after_open_table_mdl_shared SIGNAL locked WAIT_FOR finish';
--echo # Sending:
--send select * from t2, t1
--echo #
--echo # Switching to connection 'deadlock_con1'.
connection deadlock_con1;
--echo # Wait till SELECT acquires MDL on 't2' and starts waiting for signal.
set debug_sync= 'now WAIT_FOR locked';
--echo # Sending:
--send lock tables t1 write, t2 write
--echo #
--echo # Switching to connection 'deadlock_con2'.
connection deadlock_con2;
--echo # Wait until LOCK TABLES acquires SNRW lock on 't1' and is blocked
--echo # while trying to acquire SNRW lock on 't1'.
let $wait_condition=
select count(*) = 1 from information_schema.processlist
where state = "Waiting for table" and info = "lock tables t1 write, t2 write";
--source include/wait_condition.inc
--echo # Resume SELECT execution, this should eventually unblock LOCK TABLES.
set debug_sync= 'now SIGNAL finish';
--echo #
--echo # Switching to connection 'deadlock_con1'.
connection deadlock_con1;
--echo # Reaping LOCK TABLES.
--reap
unlock tables;
--echo #
--echo # Switching to connection 'default'.
connection default;
--echo # Reaping SELECT. It succeed and not report ER_LOCK_DEADLOCK error.
--reap
drop tables t1, t2;
--echo #
--echo # Test coverage for situation in which a race has happened
--echo # during deadlock detection process which led to unwarranted
--echo # ER_LOCK_DEADLOCK error.
--echo #
create table t1 (i int);
--echo # Ensure that ALTER waits once it has acquired SNW lock.
set debug_sync='after_open_table_mdl_shared SIGNAL parked1 WAIT_FOR go1';
--echo # Sending:
--send alter table t1 add column j int
--echo #
--echo # Switching to connection 'deadlock_con1'.
connection deadlock_con1;
--echo # Wait till ALTER acquires SNW lock and stops.
set debug_sync='now WAIT_FOR parked1';
--echo # Ensure that INSERT is paused once it detects that there is
--echo # a conflicting metadata lock so it has to wait, but before
--echo # deadlock detection is run.
set debug_sync='mdl_acquire_lock_wait SIGNAL parked2 WAIT_FOR go2';
--echo # Sending:
--send insert into t1 values ()
--echo #
--echo # Switching to connection 'deadlock_con2'.
connection deadlock_con2;
--echo # Wait till INSERT is paused.
set debug_sync='now WAIT_FOR parked2';
--echo # Resume ALTER execution. Eventually it will release its
--echo # metadata lock and INSERT's request for SW lock will be
--echo # satisified.
set debug_sync='now SIGNAL go1';
--echo #
--echo # Switching to connection 'default'.
connection default;
--echo # Reaping ALTER TABLE.
--reap
--echo # Add a new request for SNW lock to waiting graph.
--echo # Sending:
--send alter table t1 drop column j
--echo #
--echo # Switching to connection 'deadlock_con2'.
connection deadlock_con2;
--echo # Wait until ALTER is blocked.
let $wait_condition=
select count(*) = 1 from information_schema.processlist
where state = "Waiting for table" and info = "alter table t1 drop column j";
--source include/wait_condition.inc
--echo # Resume INSERT so it can start deadlock detection.
--echo #
--echo # At this point there is a discrepancy between the fact that INSERT's
--echo # SW lock is already satisfied, but INSERT's connection is still
--echo # marked as waiting for it. Looking for a loop in waiters graph
--echo # without additional checks has detected a deadlock (INSERT waits
--echo # for SW lock; which is not granted because of pending SNW lock from
--echo # ALTER; which waits for active SW lock from INSERT). Since requests
--echo # for SW and SNW locks have same weight ALTER was selected as a victim
--echo # and ended with ER_LOCK_DEADLOCK error.
set debug_sync='now SIGNAL go2';
--echo #
--echo # Switching to connection 'deadlock_con1'.
connection deadlock_con1;
--echo # Reaping INSERT.
--reap
--echo #
--echo # Switching to connection 'default'.
connection default;
--echo # Reaping ALTER. It should succeed and not produce ER_LOCK_DEADLOCK.
--reap
drop table t1;
set debug_sync= 'RESET';
disconnect deadlock_con1;
disconnect deadlock_con2;
disconnect deadlock_con3;
......@@ -3097,7 +3228,7 @@ set debug_sync='after_lock_tables_takes_lock SIGNAL alter_table_locked WAIT_FOR
--echo # Switching to connection 'default'.
connection default;
set debug_sync='now WAIT_FOR alter_table_locked';
set debug_sync='before_open_table_wait_refresh SIGNAL alter_go';
set debug_sync='mdl_acquire_lock_wait SIGNAL alter_go';
--echo # The below statement should get ER_LOCK_DEADLOCK error
--echo # (i.e. it should not allow ALTER to proceed, and then
--echo # fail due to 't1' changing its name to 't2').
......
......@@ -108,38 +108,35 @@ disconnect con3;
--echo # Bug #48246 assert in close_thread_table
--echo #
CREATE TABLE t0 (b INTEGER);
CREATE TABLE t1 (a INTEGER);
CREATE FUNCTION f1(b INTEGER) RETURNS INTEGER RETURN 1;
CREATE PROCEDURE p1() SELECT COUNT(f1(a)) FROM t1;
CREATE PROCEDURE p1() SELECT COUNT(f1(a)) FROM t1, t0;
INSERT INTO t0 VALUES(1);
INSERT INTO t1 VALUES(1), (2);
--echo # Connection 2
connect (con2, localhost, root);
CALL p1();
--echo # Connection default
connection default;
SET DEBUG_SYNC= 'after_open_table_mdl_shared SIGNAL locked WAIT_FOR called';
--echo # Sending:
--send CREATE TABLE t1 (a INTEGER)
--echo # Connection 2
connection con2;
SET DEBUG_SYNC= 'now WAIT_FOR locked';
SET DEBUG_SYNC= 'before_open_table_wait_refresh SIGNAL called WAIT_FOR created';
--echo # This call used to cause an assertion. MDL locking conflict will
--echo # cause back-off and retry. A variable indicating if a prelocking list
--echo # exists, used to be not reset properly causing an eventual assert.
SET DEBUG_SYNC= 'after_open_table_mdl_shared SIGNAL locked_t1 WAIT_FOR go_for_t0';
--echo # This call used to cause an assertion. MDL deadlock with upcoming
--echo # LOCK TABLES statement will cause back-off and retry.
--echo # A variable indicating if a prelocking list exists, used to be not
--echo # reset properly causing an eventual assert.
--echo # Sending:
--send CALL p1()
--echo # Connection default
connection default;
--echo # Reaping: CREATE TABLE t1 (a INTEGER)
--error ER_TABLE_EXISTS_ERROR
--reap
SET DEBUG_SYNC= 'now SIGNAL created';
SET DEBUG_SYNC= 'now WAIT_FOR locked_t1';
--echo # Issue LOCK TABLES statement which will enter in MDL deadlock
--echo # with CALL statement and as result will cause it to perform
--echo # back-off and retry.
SET DEBUG_SYNC= 'mdl_acquire_lock_wait SIGNAL go_for_t0';
LOCK TABLES t0 WRITE, t1 WRITE;
UNLOCK TABLES;
--echo # Connection 2
connection con2;
......@@ -151,7 +148,7 @@ connection default;
disconnect con2;
DROP PROCEDURE p1;
DROP FUNCTION f1;
DROP TABLE t1;
DROP TABLES t0, t1;
SET DEBUG_SYNC= 'RESET';
......
This diff is collapsed.
......@@ -447,6 +447,37 @@ private:
};
/**
A reliable way to wait on an MDL lock.
*/
class MDL_wait
{
public:
MDL_wait();
~MDL_wait();
enum enum_wait_status { EMPTY = 0, GRANTED, VICTIM, TIMEOUT, KILLED };
bool set_status(enum_wait_status result_arg);
enum_wait_status get_status();
void reset_status();
enum_wait_status timed_wait(THD *thd, struct timespec *abs_timeout,
bool signal_timeout);
private:
/**
Condvar which is used for waiting until this context's pending
request can be satisfied or this thread has to perform actions
to resolve a potential deadlock (we subscribe to such
notification by adding a ticket corresponding to the request
to an appropriate queue of waiters).
*/
mysql_mutex_t m_LOCK_wait_status;
mysql_cond_t m_COND_wait_status;
enum_wait_status m_wait_status;
};
typedef I_P_List<MDL_request, I_P_List_adapter<MDL_request,
&MDL_request::next_in_list,
&MDL_request::prev_in_list>,
......@@ -464,16 +495,13 @@ public:
typedef I_P_List<MDL_ticket,
I_P_List_adapter<MDL_ticket,
&MDL_ticket::next_in_context,
&MDL_ticket::prev_in_context> >
&MDL_ticket::prev_in_context>,
I_P_List_null_counter,
I_P_List_fast_push_back<MDL_ticket> >
Ticket_list;
typedef Ticket_list::Iterator Ticket_iterator;
enum mdl_signal_type { NO_WAKE_UP = 0,
NORMAL_WAKE_UP,
VICTIM_WAKE_UP,
TIMEOUT_WAKE_UP };
MDL_context();
void destroy();
......@@ -485,8 +513,6 @@ public:
bool clone_ticket(MDL_request *mdl_request);
bool wait_for_lock(MDL_request *mdl_request, ulong lock_wait_timeout);
void release_all_locks_for_name(MDL_ticket *ticket);
void release_lock(MDL_ticket *ticket);
......@@ -532,16 +558,13 @@ public:
inline uint get_deadlock_weight() const
{ return m_waiting_for->get_deadlock_weight(); }
/**
Wake up context which is waiting for a change of MDL_lock state.
*/
void awake(mdl_signal_type signal)
{
mysql_mutex_lock(&m_signal_lock);
m_signal= signal;
mysql_cond_signal(&m_signal_cond);
mysql_mutex_unlock(&m_signal_lock);
}
Post signal to the context (and wake it up if necessary).
@retval FALSE - Success, signal was posted.
@retval TRUE - Failure, signal was not posted since context
already has received some signal or closed
signal slot.
*/
void init(THD *thd_arg) { m_thd= thd_arg; }
void set_needs_thr_lock_abort(bool needs_thr_lock_abort)
......@@ -562,6 +585,12 @@ public:
}
bool find_deadlock(Deadlock_detection_visitor *dvisitor);
public:
/**
If our request for a lock is scheduled, or aborted by the deadlock
detector, the result is recorded in this class.
*/
MDL_wait m_wait;
private:
/**
All MDL tickets acquired by this connection.
......@@ -643,60 +672,38 @@ private:
important as deadlock detector won't work correctly
otherwise. @sa Comment for MDL_lock::m_rwlock.
*/
mysql_prlock_t m_waiting_for_lock;
MDL_ticket *m_waiting_for;
uint m_deadlock_weight;
mysql_prlock_t m_LOCK_waiting_for;
/**
Condvar which is used for waiting until this context's pending
request can be satisfied or this thread has to perform actions
to resolve a potential deadlock (we subscribe to such
notification by adding a ticket corresponding to the request
to an appropriate queue of waiters).
Tell the deadlock detector what lock this session is waiting for.
In principle, this is redundant, as information can be found
by inspecting waiting queues, but we'd very much like it to be
readily available to the wait-for graph iterator.
*/
mysql_mutex_t m_signal_lock;
mysql_cond_t m_signal_cond;
mdl_signal_type m_signal;
MDL_ticket *m_waiting_for;
private:
MDL_ticket *find_ticket(MDL_request *mdl_req,
bool *is_transactional);
void release_locks_stored_before(MDL_ticket *sentinel);
bool acquire_lock_impl(MDL_request *mdl_request, ulong lock_wait_timeout);
bool try_acquire_lock_impl(MDL_request *mdl_request,
MDL_ticket **out_ticket);
bool find_deadlock();
void find_deadlock();
/** Inform the deadlock detector there is an edge in the wait-for graph. */
void will_wait_for(MDL_ticket *pending_ticket)
{
mysql_prlock_wrlock(&m_waiting_for_lock);
mysql_prlock_wrlock(&m_LOCK_waiting_for);
m_waiting_for= pending_ticket;
mysql_prlock_unlock(&m_waiting_for_lock);
mysql_prlock_unlock(&m_LOCK_waiting_for);
}
void stop_waiting()
/** Remove the wait-for edge from the graph after we're done waiting. */
void done_waiting_for()
{
mysql_prlock_wrlock(&m_waiting_for_lock);
mysql_prlock_wrlock(&m_LOCK_waiting_for);
m_waiting_for= NULL;
mysql_prlock_unlock(&m_waiting_for_lock);
}
void wait_reset()
{
mysql_mutex_lock(&m_signal_lock);
m_signal= NO_WAKE_UP;
mysql_mutex_unlock(&m_signal_lock);
mysql_prlock_unlock(&m_LOCK_waiting_for);
}
mdl_signal_type timed_wait(struct timespec *abs_timeout);
mdl_signal_type peek_signal()
{
mdl_signal_type result;
mysql_mutex_lock(&m_signal_lock);
result= m_signal;
mysql_mutex_unlock(&m_signal_lock);
return result;
}
private:
MDL_context(const MDL_context &rhs); /* not implemented */
MDL_context &operator=(MDL_context &rhs); /* not implemented */
......@@ -713,7 +720,6 @@ void mdl_destroy();
extern bool mysql_notify_thread_having_shared_lock(THD *thd, THD *in_use,
bool needs_thr_lock_abort);
extern void mysql_ha_flush(THD *thd);
extern "C" const char *set_thd_proc_info(void *thd_arg, const char *info,
const char *calling_function,
const char *calling_file,
......
This diff is collapsed.
......@@ -451,7 +451,7 @@ public:
enum enum_open_table_action
{
OT_NO_ACTION= 0,
OT_WAIT_MDL_LOCK,
OT_MDL_CONFLICT,
OT_WAIT_TDC,
OT_DISCOVER,
OT_REPAIR
......@@ -460,7 +460,7 @@ public:
bool recover_from_failed_open(THD *thd);
bool request_backoff_action(enum_open_table_action action_arg,
MDL_request *mdl_request, TABLE_LIST *table);
TABLE_LIST *table);
void add_request(MDL_request *request)
{ m_mdl_requests.push_front(request); }
......@@ -490,8 +490,6 @@ public:
private:
/** List of requests for all locks taken so far. Used for waiting on locks. */
MDL_request_list m_mdl_requests;
/** For OT_WAIT_MDL_LOCK action, the request for which we should wait. */
MDL_request *m_failed_mdl_request;
/**
For OT_DISCOVER and OT_REPAIR actions, the table list element for
the table which definition should be re-discovered or which
......
......@@ -18,8 +18,10 @@
#include <my_global.h>
template <typename T, typename B, typename C> class I_P_List_iterator;
template <typename T, typename B, typename C, typename I>
class I_P_List_iterator;
class I_P_List_null_counter;
template <typename T> class I_P_List_no_push_back;
/**
......@@ -52,10 +54,17 @@ class I_P_List_null_counter;
should be done. Instance of this class is also used as a place
where information about number of list elements is stored.
@sa I_P_List_null_counter, I_P_List_counter
@param I Policy class specifying whether I_P_List should support
efficient push_back() operation. Instance of this class
is used as place where we store information to support
this operation.
@sa I_P_List_no_push_back, I_P_List_fast_push_back.
*/
template <typename T, typename B, typename C = I_P_List_null_counter>
class I_P_List : public C
template <typename T, typename B,
typename C = I_P_List_null_counter,
typename I = I_P_List_no_push_back<T> >
class I_P_List : public C, public I
{
T *first;
......@@ -65,31 +74,27 @@ class I_P_List : public C
is a bad idea.
*/
public:
I_P_List() : first(NULL) { };
inline void empty() { first= NULL; C::reset(); }
I_P_List() : I(&first), first(NULL) {};
inline void empty() { first= NULL; C::reset(); I::set_last(&first); }
inline bool is_empty() const { return (first == NULL); }
inline void push_front(T* a)
{
*B::next_ptr(a)= first;
if (first)
*B::prev_ptr(first)= B::next_ptr(a);
else
I::set_last(B::next_ptr(a));
first= a;
*B::prev_ptr(a)= &first;
C::inc();
}
inline void push_back(T *a)
{
insert_after(back(), a);
}
inline T *back()
{
T *t= front();
if (t)
{
while (*B::next_ptr(t))
t= *B::next_ptr(t);
}
return t;
T **last= I::get_last();
*B::next_ptr(a)= *last;
*last= a;
*B::prev_ptr(a)= last;
I::set_last(B::next_ptr(a));
}
inline void insert_after(T *pos, T *a)
{
......@@ -105,6 +110,8 @@ public:
T *old_next= *B::next_ptr(a);
*B::prev_ptr(old_next)= B::next_ptr(a);
}
else
I::set_last(B::next_ptr(a));
}
}
inline void remove(T *a)
......@@ -112,6 +119,8 @@ public:
T *next= *B::next_ptr(a);
if (next)
*B::prev_ptr(next)= *B::prev_ptr(a);
else
I::set_last(*B::prev_ptr(a));
**B::prev_ptr(a)= next;
C::dec();
}
......@@ -120,16 +129,21 @@ public:
void swap(I_P_List<T, B, C> &rhs)
{
swap_variables(T *, first, rhs.first);
I::swap(rhs);
if (first)
*B::prev_ptr(first)= &first;
else
I::set_last(&first);
if (rhs.first)
*B::prev_ptr(rhs.first)= &rhs.first;
else
I::set_last(&rhs.first);
C::swap(rhs);
}
#ifndef _lint
friend class I_P_List_iterator<T, B, C>;
friend class I_P_List_iterator<T, B, C, I>;
#endif
typedef I_P_List_iterator<T, B, C> Iterator;
typedef I_P_List_iterator<T, B, C, I> Iterator;
};
......@@ -137,15 +151,19 @@ public:
Iterator for I_P_List.
*/
template <typename T, typename B, typename C = I_P_List_null_counter>
template <typename T, typename B,
typename C = I_P_List_null_counter,
typename I = I_P_List_no_push_back<T> >
class I_P_List_iterator
{
const I_P_List<T, B, C> *list;
const I_P_List<T, B, C, I> *list;
T *current;
public:
I_P_List_iterator(const I_P_List<T, B, C> &a) : list(&a), current(a.first) {}
I_P_List_iterator(const I_P_List<T, B, C> &a, T* current_arg) : list(&a), current(current_arg) {}
inline void init(const I_P_List<T, B, C> &a)
I_P_List_iterator(const I_P_List<T, B, C, I> &a)
: list(&a), current(a.first) {}
I_P_List_iterator(const I_P_List<T, B, C, I> &a, T* current_arg)
: list(&a), current(current_arg) {}
inline void init(const I_P_List<T, B, C, I> &a)
{
list= &a;
current= a.first;
......@@ -203,4 +221,40 @@ public:
uint elements() const { return m_counter; }
};
/**
A null insertion policy class for I_P_List to be used
in cases when push_back() operation is not necessary.
*/
template <typename T> class I_P_List_no_push_back
{
protected:
I_P_List_no_push_back(T **a) {};
void set_last(T **a) {}
/*
T** get_last() const method is intentionally left unimplemented
in order to prohibit usage of push_back() method in lists which
use this policy.
*/
void swap(I_P_List_no_push_back<T> &rhs) {}
};
/**
An insertion policy class for I_P_List which can
be used when fast push_back() operation is required.
*/
template <typename T> class I_P_List_fast_push_back
{
T **last;
protected:
I_P_List_fast_push_back(T **a) : last(a) { };
void set_last(T **a) { last= a; }
T** get_last() const { return last; }
void swap(I_P_List_fast_push_back<T> &rhs)
{ swap_variables(T**, last, rhs.last); }
};
#endif
......@@ -3136,15 +3136,27 @@ try_acquire_high_prio_shared_mdl_lock(THD *thd, TABLE_LIST *table,
bool error;
table->mdl_request.init(MDL_key::TABLE, table->db, table->table_name,
MDL_SHARED_HIGH_PRIO);
while (!(error=
thd->mdl_context.try_acquire_lock(&table->mdl_request)) &&
!table->mdl_request.ticket && !can_deadlock)
if (can_deadlock)
{
if ((error=
thd->mdl_context.wait_for_lock(&table->mdl_request,
thd->variables.lock_wait_timeout)))
break;
/*
When .FRM is being open in order to get data for an I_S table,
we might have some tables not only open but also locked.
E.g. this happens when a SHOW or I_S statement is run
under LOCK TABLES or inside a stored function.
By waiting for the conflicting metadata lock to go away we
might create a deadlock which won't entirely belong to the
MDL subsystem and thus won't be detectable by this subsystem's
deadlock detector. To avoid such situation, when there are
other locked tables, we prefer not to wait on a conflicting
lock.
*/
error= thd->mdl_context.try_acquire_lock(&table->mdl_request);
}
else
error= thd->mdl_context.acquire_lock(&table->mdl_request,
thd->variables.lock_wait_timeout);
return error;
}
......
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment