Commit 9dbd9ce1 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.

mysql-test/r/mdl_sync.result:
  Added test coverage for some aspects of deadlock handling in
  metadata locking subsystem.
  Adjusted test case after removing back-off in general case
  when conflicting metadata lock is encountered during
  open_tables() (now this happens only if waiting for
  conflicting lock to go away leads to a deadlock).
mysql-test/r/sp_sync.result:
  Adjusted test case after removing back-off in general case
  when conflicting metadata lock is encountered during
  open_tables() (now this happens only if waiting for
  conflicting lock to go away leads to a deadlock).
mysql-test/suite/perfschema/r/dml_setup_instruments.result:
  Adjusted test results after renaming MDL_context::
  m_waiting_for_lock rwlock to m_LOCK_waiting_for.
mysql-test/suite/rpl/r/rpl_sp.result:
  Adjusted test case after implementing new approach to
  acquiring metadata locks in open_tables(). We no longer
  release all MDL locks acquired by statement before waiting
  for conflicting lock to go away. As result DROP FUNCTION
  statement has to wait for DML statement which managed to
  acquire metadata lock on function being dropped and now
  waits for other conflicting metadata lock to go away.
mysql-test/suite/rpl/t/rpl_sp.test:
  Adjusted test case after implementing new approach to
  acquiring metadata locks in open_tables(). We no longer
  release all MDL locks acquired by statement before waiting
  for conflicting lock to go away. As result DROP FUNCTION
  statement has to wait for DML statement which managed to
  acquire metadata lock on function being dropped and now
  waits for other conflicting metadata lock to go away.
mysql-test/t/mdl_sync.test:
  Added test coverage for some aspects of deadlock handling in
  metadata locking subsystem.
  Adjusted test case after removing back-off in general case
  when conflicting metadata lock is encountered during
  open_tables() (now this happens only if waiting for
  conflicting lock to go away leads to a deadlock).
mysql-test/t/sp_sync.test:
  Adjusted test case after removing back-off in general case
  when conflicting metadata lock is encountered during
  open_tables() (now this happens only if waiting for
  conflicting lock to go away leads to a deadlock).
sql/mdl.cc:
  Changed MDL subsystem to support new approach to acquring
  metadata locks in open tables and more fair and efficient
  scheduling of metadata locks. To implement this:
  - Made releasing of the lock and granting it to the most
    appropriate pending request atomic operation. As result it
    became possible to wake up only those waiters requests from
    which 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 solved 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".
    To emphasize above changes wake_up_waiters() was renamed
    to MDL_context::reschedule_waiters().
  - Changed code to add tickets for new requests to the back of
    waiters queue and to select tickets to be satisfied from
    the head of the queue if possible (this makes scheduling of
    requests with the same priority fair). To be able to do
    this efficiently we now use for waiting and granted queues
    version of I_P_List class which provides fast push_back()
    method.
  - Members and methods of MDL_context related to sending
    and waiting for signal were moved to separate MDL_wait
    class.
  - Since in order to avoid race conditions we must grant the
    lock only to the context which was not chosen as a victim
    of deadlock, killed or aborted due to timeout
    MDL_wait::set_status() (former awake()) was changed not to
    send signal if signal slot is already occupied and to
    indicate this fact through its return value. As another
    consequence MDL_wait::timed_wait() method was changed to
    handle timeout (optionally) and abort due to kill as
    signals which make signal slot occupied.
  - Renamed MDL_context::acquire_lock_impl() to acquire_lock().
    Changed it to be able correctly process requests for shared
    locks when there are open HANDLERs, made this method more
    optimized for acquisition of shared locks. As part of this
    change moved code common between try_acquire_lock() and
    acquire_lock() to new try_acquire_lock_impl() method.
    Also adjusted acquire_lock()'s code to take into account
    the fact that in cases when lock is granted as result of
    MDL_context::reschedule_waiters() call (i.e. when it is
    granted after waiting for lock conflict to go away)
    updating MDL_lock state is responsibility of the thread
    calling reschedule_waiters().
  - Changed MDL_context::find_deadlock() to send VICTIM
    signal even if victim is the context which has initiated
    deadlock detection. This is required in order to avoid
    races in cases when the same context simultaneously is
    chosen as a victim and its request for lock is satisfied.
    As result return value of this method became unnecessary
    and it was changed to return void.
    Adjusted MDL_lock::find_deadlock() method to take into
    account that now there can be a discrepancy between
    MDL_context::m_waiting_for value being set and real state
    of the ticket this member points to.
  - Renamed MDL_context::m_waiting_for_lock to m_LOCK_waiting_for
    and MDL_context::stop_waiting() to done_waiting_for().
  - Finally, removed MDL_context::wait_for_lock() method.
sql/mdl.h:
  Changed MDL subsystem to support new approach to acquring
  metadata locks in open tables and more fair and efficient
  scheduling of metadata locks. To implement this:
  - Members and methods of MDL_context related to sending
    and waiting for signal were moved to separate MDL_wait
    class.
  - Since now in order to avoid race conditions we must grant
    the lock only to the context which was not chosen as a
    victim of deadlock, killed or aborted due to timeout
    MDL_wait::set_status (former awake()) was changed not to
    send signal if signal slot is already occupied and to
    indicate this fact through its return value.
    Also NORMAL_WAKE_UP signal became GRANTED, and timeouts
    and aborts due to kill became full blown signals rather
    than simple return values.
  - MDL_wait::timed_wait() now takes extra parameter that
    indicates whether signal should be set if timeout is
    reached.
  - Enabled fast push_back() operation in MDL_context::m_tickets
    list to make move_ticket_after_trans_sentinel() method more
    efficient.
  - Removed MDL_context::wait_for_lock() method.
  - Renamed MDL_context::m_waiting_for_lock to m_LOCK_waiting_for
    and MDL_context::stop_waiting() to done_waiting_for().
  - MDL_context::acquire_lock_impl() became acquire_lock().
  - Introduced MDL_context::try_acquire_lock_impl() as a
    place for code shared by try_acquire_lock and
    acquire_lock().
  - Due to fact that now VICTIM signal is sent even if victim
    is the context which has initiated deadlock detection
    find_deadlock() no longer needs a return value.
sql/sql_base.cc:
  Implemented new approach to acquiring metadata locks in
  open_tables(). We no longer perform back-off when conflicting
  metadata lock is encountered. Instead we wait for this lock
  to go away while holding all locks which were acquired so
  far. Back-off is only used in situation when further waiting
  will cause a deadlock which could be avoided by performing
  back-off and restarting open_tables() process. Absence of
  waiting between back-off and restart of acquiring metadata
  locks can't lead to livelocks as MDL subsystem was changed
  to make release of lock and granting it to waiting lock
  an atomic action, so back-off will automatically give way
  to other participants of deadlock loop.
  Accordingly:
  - open_table_get_mdl_lock() and open_and_process_routine()
    were changed to wait for conflicting metadata lock to
    go away without back-off. Only if such wait leads to a
    deadlock back-off is requested. As part of this change
    new error handler class was introduced which converts,
    if possible, ER_LOCK_DEADLOCK error to a request for
    back-off and re-start of open_tables() process.
  - Open_table_context::recover_from_failed_open() was changed
    not to wait in case of metadata lock conflict. Instead we
    immediately proceed to re-acquiring locks.
  - Open_table_context::request_backoff_action() now always
    emits error if back-off is requested in the middle of
    transaction as we can't be sure that releasing lock
    which were acquired only by current statement will
    resolve a deadlock. Before this patch such situations were
    successfully detected thanks to the fact that we called
    MDL_context::wait_for_lock() method in
    recover_from_failed_open().
  - In order to avoid deadlocks open_tables() code was adjusted
    to flush open HANDLERs for which there are pending requests
    for X locks before restarting the process of acquiring
    metadata locks.
  - Changed close_tables_for_reopen() not to reset MDL_request
    for tables belonging to the tail of prelocking list. It is
    no longer necessary as these MDL_request objects won't be
    used for any waiting.
  - Adjusted comment in tdc_wait_for_old_version() to avoid
    mentioning removed MDL_context::wait_for_lock() method.
sql/sql_base.h:
  As we no longer wait for conflicting metadata lock away in
  Open_table_context::recover_from_failed_open() method,
  Open_table_context::OT_WAIT_MDL_LOCK action was renamed to
  OT_MDL_CONFLICT.
  Also Open_table_context::m_failed_mdl_request became
  unnecessary and was removed.
sql/sql_plist.h:
  Extended I_P_List template to support efficient push_back()
  operation if it is parameterized with an appropriate policy
  class.
sql/sql_show.cc:
  Adjusted code after removal of MDL_context::wait_for_lock()
  method. Now if one needs to acquire metadata lock with waiting
  one has to use a variant of MDL_context::acquire_lock() method.
parent 142a162c
......@@ -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';
......
......@@ -21,28 +21,28 @@
#ifdef HAVE_PSI_INTERFACE
static PSI_mutex_key key_MDL_map_mutex;
static PSI_mutex_key key_MDL_context_signal_mutex;
static PSI_mutex_key key_MDL_wait_LOCK_wait_status;
static PSI_mutex_info all_mdl_mutexes[]=
{
{ &key_MDL_map_mutex, "MDL_map::mutex", PSI_FLAG_GLOBAL},
{ &key_MDL_context_signal_mutex, "MDL_context::signal", 0}
{ &key_MDL_wait_LOCK_wait_status, "MDL_wait::LOCK_wait_status", 0}
};
static PSI_rwlock_key key_MDL_lock_rwlock;
static PSI_rwlock_key key_MDL_context_waiting_for_rwlock;
static PSI_rwlock_key key_MDL_context_LOCK_waiting_for;
static PSI_rwlock_info all_mdl_rwlocks[]=
{
{ &key_MDL_lock_rwlock, "MDL_lock::rwlock", 0},
{ &key_MDL_context_waiting_for_rwlock, "MDL_context::waiting_for_lock", 0}
{ &key_MDL_context_LOCK_waiting_for, "MDL_context::LOCK_waiting_for", 0}
};
static PSI_cond_key key_MDL_context_signal_cond;
static PSI_cond_key key_MDL_wait_COND_wait_status;
static PSI_cond_info all_mdl_conds[]=
{
{ &key_MDL_context_signal_cond, "MDL_context::signal", 0}
{ &key_MDL_wait_COND_wait_status, "MDL_context::COND_wait_status", 0}
};
/**
......@@ -259,7 +259,9 @@ class MDL_lock
typedef I_P_List<MDL_ticket,
I_P_List_adapter<MDL_ticket,
&MDL_ticket::next_in_lock,
&MDL_ticket::prev_in_lock> >
&MDL_ticket::prev_in_lock>,
I_P_List_null_counter,
I_P_List_fast_push_back<MDL_ticket> >
List;
operator const List &() const { return m_list; }
Ticket_list() :m_bitmap(0) {}
......@@ -345,21 +347,8 @@ class MDL_lock
}
}
/**
Wake up contexts which are waiting to acquire lock on the object and
which may succeed now, when we released some lock on it or removed
some pending request from its waiters list (the latter can happen,
for example, when context trying to acquire exclusive on the object
lock is killed).
*/
void wake_up_waiters()
{
MDL_lock::Ticket_iterator it(m_waiting);
MDL_ticket *awake_ticket;
void reschedule_waiters();
while ((awake_ticket= it++))
awake_ticket->get_ctx()->awake(MDL_context::NORMAL_WAKE_UP);
}
void remove_ticket(Ticket_list MDL_lock::*queue, MDL_ticket *ticket);
bool find_deadlock(MDL_ticket *waiting_ticket,
......@@ -499,7 +488,7 @@ mdl_locks_key(const uchar *record, size_t *length,
statement, the design capitalizes on that to later save on
look ups in the table definition cache. This leads to reduced
contention overall and on LOCK_open in particular.
Please see the description of MDL_context::acquire_lock_impl()
Please see the description of MDL_context::acquire_lock()
for details.
*/
......@@ -726,13 +715,9 @@ MDL_context::MDL_context()
:m_trans_sentinel(NULL),
m_thd(NULL),
m_needs_thr_lock_abort(FALSE),
m_waiting_for(NULL),
m_deadlock_weight(0),
m_signal(NO_WAKE_UP)
m_waiting_for(NULL)
{
mysql_prlock_init(key_MDL_context_waiting_for_rwlock, &m_waiting_for_lock);
mysql_mutex_init(key_MDL_context_signal_mutex, &m_signal_lock, NULL);
mysql_cond_init(key_MDL_context_signal_mutex, &m_signal_cond, NULL);
mysql_prlock_init(key_MDL_context_LOCK_waiting_for, &m_LOCK_waiting_for);
}
......@@ -752,9 +737,7 @@ void MDL_context::destroy()
{
DBUG_ASSERT(m_tickets.is_empty());
mysql_prlock_destroy(&m_waiting_for_lock);
mysql_mutex_destroy(&m_signal_lock);
mysql_cond_destroy(&m_signal_cond);
mysql_prlock_destroy(&m_LOCK_waiting_for);
}
......@@ -964,27 +947,120 @@ static inline void mdl_exit_cond(THD *thd,
}
MDL_context::mdl_signal_type MDL_context::timed_wait(struct timespec
*abs_timeout)
/** Construct an empty wait slot. */
MDL_wait::MDL_wait()
:m_wait_status(EMPTY)
{
mysql_mutex_init(key_MDL_wait_LOCK_wait_status, &m_LOCK_wait_status, NULL);
mysql_cond_init(key_MDL_wait_COND_wait_status, &m_COND_wait_status, NULL);
}
/** Destroy system resources. */
MDL_wait::~MDL_wait()
{
mysql_mutex_destroy(&m_LOCK_wait_status);
mysql_cond_destroy(&m_COND_wait_status);
}
/**
Set the status unless it's already set. Return FALSE if set,
TRUE otherwise.
*/
bool MDL_wait::set_status(enum_wait_status status_arg)
{
bool was_occupied= TRUE;
mysql_mutex_lock(&m_LOCK_wait_status);
if (m_wait_status == EMPTY)
{
was_occupied= FALSE;
m_wait_status= status_arg;
mysql_cond_signal(&m_COND_wait_status);
}
mysql_mutex_unlock(&m_LOCK_wait_status);
return was_occupied;
}
/** Query the current value of the wait slot. */
MDL_wait::enum_wait_status MDL_wait::get_status()
{
enum_wait_status result;
mysql_mutex_lock(&m_LOCK_wait_status);
result= m_wait_status;
mysql_mutex_unlock(&m_LOCK_wait_status);
return result;
}
/** Clear the current value of the wait slot. */
void MDL_wait::reset_status()
{
mysql_mutex_lock(&m_LOCK_wait_status);
m_wait_status= EMPTY;
mysql_mutex_unlock(&m_LOCK_wait_status);
}
/**
Wait for the status to be assigned to this wait slot.
@param abs_timeout Absolute time after which waiting should stop.
@param set_status_on_tiemout TRUE - If in case of timeout waiting
context should close the wait slot by
sending TIMEOUT to itself.
FALSE - Otherwise.
@returns Signal posted.
*/
MDL_wait::enum_wait_status
MDL_wait::timed_wait(THD *thd, struct timespec *abs_timeout,
bool set_status_on_timeout)
{
const char *old_msg;
mdl_signal_type result;
enum_wait_status result;
st_my_thread_var *mysys_var= my_thread_var;
int wait_result= 0;
mysql_mutex_lock(&m_signal_lock);
mysql_mutex_lock(&m_LOCK_wait_status);
old_msg= MDL_ENTER_COND(m_thd, mysys_var, &m_signal_cond, &m_signal_lock);
old_msg= MDL_ENTER_COND(thd, mysys_var, &m_COND_wait_status,
&m_LOCK_wait_status);
while (!m_signal && !mysys_var->abort &&
while (!m_wait_status && !mysys_var->abort &&
wait_result != ETIMEDOUT && wait_result != ETIME)
wait_result= mysql_cond_timedwait(&m_signal_cond, &m_signal_lock,
wait_result= mysql_cond_timedwait(&m_COND_wait_status, &m_LOCK_wait_status,
abs_timeout);
result= (m_signal != NO_WAKE_UP || mysys_var->abort) ?
m_signal : TIMEOUT_WAKE_UP;
if (m_wait_status == EMPTY)
{
/*
Wait has ended not due to a status being set from another
thread but due to this connection/statement being killed or a
time out.
To avoid races, which may occur if another thread sets
GRANTED status before the code which calls this method
processes the abort/timeout, we assign the status under
protection of the m_LOCK_wait_status, within the critical
section. An exception is when set_status_on_timeout is
false, which means that the caller intends to restart the
wait.
*/
if (mysys_var->abort)
m_wait_status= KILLED;
else if (set_status_on_timeout)
m_wait_status= TIMEOUT;
}
result= m_wait_status;
MDL_EXIT_COND(m_thd, mysys_var, &m_signal_lock, old_msg);
MDL_EXIT_COND(thd, mysys_var, &m_LOCK_wait_status, old_msg);
return result;
}
......@@ -1024,7 +1100,11 @@ void MDL_lock::Ticket_list::add_ticket(MDL_ticket *ticket)
called by other threads.
*/
DBUG_ASSERT(ticket->get_lock());
m_list.push_front(ticket);
/*
Add ticket to the *back* of the queue to ensure fairness
among requests with the same priority.
*/
m_list.push_back(ticket);
m_bitmap|= MDL_BIT(ticket->get_type());
}
......@@ -1051,6 +1131,75 @@ void MDL_lock::Ticket_list::remove_ticket(MDL_ticket *ticket)
}
/**
Determine waiting contexts which requests for the lock can be
satisfied, grant lock to them and wake them up.
@note Together with MDL_lock::add_ticket() this method implements
fair scheduling among requests with the same priority.
It tries to grant lock from the head of waiters list, while
add_ticket() adds new requests to the back of this list.
*/
void MDL_lock::reschedule_waiters()
{
MDL_lock::Ticket_iterator it(m_waiting);
MDL_ticket *ticket;
/*
Find the first (and hence the oldest) waiting request which
can be satisfied (taking into account priority). Grant lock to it.
Repeat the process for the remainder of waiters.
Note we don't need to re-start iteration from the head of the
list after satisfying the first suitable request as in our case
all compatible types of requests have the same priority.
TODO/FIXME: We should:
- Either switch to scheduling without priorities
which will allow to stop iteration through the
list of waiters once we found the first ticket
which can't be satisfied
- Or implement some check using bitmaps which will
allow to stop iteration in cases when, e.g., we
grant SNRW lock and there are no pending S or
SH locks.
*/
while ((ticket= it++))
{
if (can_grant_lock(ticket->get_type(), ticket->get_ctx()))
{
if (! ticket->get_ctx()->m_wait.set_status(MDL_wait::GRANTED))
{
/*
Satisfy the found request by updating lock structures.
It is OK to do so even after waking up the waiter since any
session which tries to get any information about the state of
this lock has to acquire MDL_lock::m_rwlock first and thus,
when manages to do so, already sees an updated state of the
MDL_lock object.
*/
m_waiting.remove_ticket(ticket);
m_granted.add_ticket(ticket);
/* If we are granting an X lock, release the cached object. */
if (ticket->get_type() == MDL_EXCLUSIVE && cached_object)
(*cached_object_release_hook)(cached_object);
cached_object= NULL;
}
/*
If we could not update the wait slot of the waiter,
it can be due to fact that its connection/statement was
killed or it has timed out (i.e. the slot is not empty).
Since in all such cases the waiter assumes that the lock was
not been granted, we should keep the request in the waiting
queue and look for another request to reschedule.
*/
}
}
}
/**
Compatibility (or rather "incompatibility") matrices for global metadata
lock. Arrays of bitmaps which elements specify which granted/waiting locks
......@@ -1258,9 +1407,19 @@ void MDL_lock::remove_ticket(Ticket_list MDL_lock::*list, MDL_ticket *ticket)
{
/*
There can be some contexts waiting to acquire a lock
which now might be able to do it. Wake them up!
which now might be able to do it. Grant the lock to
them and wake them up!
We always try to reschedule locks, since there is no easy way
(i.e. by looking at the bitmaps) to find out whether it is
required or not.
In a general case, even when the queue's bitmap is not changed
after removal of the ticket, there is a chance that some request
can be satisfied (due to the fact that a granted request
reflected in the bitmap might belong to the same context as a
pending request).
*/
wake_up_waiters();
reschedule_waiters();
mysql_prlock_unlock(&m_rwlock);
}
}
......@@ -1360,29 +1519,6 @@ MDL_context::find_ticket(MDL_request *mdl_request,
}
/**
Acquire one lock with waiting for conflicting locks to go away if needed.
@note This is an internal method which should not be used outside of MDL
subsystem as in most cases simply waiting for conflicting locks to
go away will lead to deadlock.
@param mdl_request [in/out] Lock request object for lock to be acquired
@param lock_wait_timeout [in] Seconds to wait before timeout.
@retval FALSE Success. MDL_request::ticket points to the ticket
for the lock.
@retval TRUE Failure (Out of resources or waiting is aborted),
*/
bool
MDL_context::acquire_lock(MDL_request *mdl_request, ulong lock_wait_timeout)
{
return acquire_lock_impl(mdl_request, lock_wait_timeout);
}
/**
Try to acquire one lock.
......@@ -1405,14 +1541,54 @@ MDL_context::acquire_lock(MDL_request *mdl_request, ulong lock_wait_timeout)
@retval FALSE Success. The lock may have not been acquired.
Check the ticket, if it's NULL, a conflicting lock
exists and another attempt should be made after releasing
all current locks and waiting for conflicting lock go
away (using MDL_context::wait_for_lock()).
exists.
@retval TRUE Out of resources, an error has been reported.
*/
bool
MDL_context::try_acquire_lock(MDL_request *mdl_request)
{
MDL_ticket *ticket;
if (try_acquire_lock_impl(mdl_request, &ticket))
return TRUE;
if (! mdl_request->ticket)
{
/*
Our attempt to acquire lock without waiting has failed.
Let us release resources which were acquired in the process.
We can't get here if we allocated a new lock object so there
is no need to release it.
*/
DBUG_ASSERT(! ticket->m_lock->is_empty());
mysql_prlock_unlock(&ticket->m_lock->m_rwlock);
MDL_ticket::destroy(ticket);
}
return FALSE;
}
/**
Auxiliary method for acquiring lock without waiting.
@param mdl_request [in/out] Lock request object for lock to be acquired
@param out_ticket [out] Ticket for the request in case when lock
has not been acquired.
@retval FALSE Success. The lock may have not been acquired.
Check MDL_request::ticket, if it's NULL, a conflicting
lock exists. In this case "out_ticket" out parameter
points to ticket which was constructed for the request.
MDL_ticket::m_lock points to the corresponding MDL_lock
object and MDL_lock::m_rwlock write-locked.
@retval TRUE Out of resources, an error has been reported.
*/
bool
MDL_context::try_acquire_lock_impl(MDL_request *mdl_request,
MDL_ticket **out_ticket)
{
MDL_lock *lock;
MDL_key *key= &mdl_request->key;
......@@ -1473,10 +1649,16 @@ MDL_context::try_acquire_lock(MDL_request *mdl_request)
return TRUE;
}
ticket->m_lock= lock;
if (lock->can_grant_lock(mdl_request->type, this))
{
ticket->m_lock= lock;
lock->m_granted.add_ticket(ticket);
if (mdl_request->type == MDL_EXCLUSIVE && lock->cached_object)
(*lock->cached_object_release_hook)(lock->cached_object);
lock->cached_object= NULL;
mysql_prlock_unlock(&lock->m_rwlock);
m_tickets.push_front(ticket);
......@@ -1484,12 +1666,7 @@ MDL_context::try_acquire_lock(MDL_request *mdl_request)
mdl_request->ticket= ticket;
}
else
{
/* We can't get here if we allocated a new lock. */
DBUG_ASSERT(! lock->is_empty());
mysql_prlock_unlock(&lock->m_rwlock);
MDL_ticket::destroy(ticket);
}
*out_ticket= ticket;
return FALSE;
}
......@@ -1567,120 +1744,125 @@ void notify_shared_lock(THD *thd, MDL_ticket *conflicting_ticket)
/**
Auxiliary method for acquiring an exclusive lock.
@param mdl_request Request for the lock to be acqured.
Acquire one lock with waiting for conflicting locks to go away if needed.
@param lock_wait_timeout Seconds to wait before timeout.
@param mdl_request [in/out] Lock request object for lock to be acquired
@note Should not be used outside of MDL subsystem. Instead one
should call acquire_lock() or acquire_locks()
methods which ensure that conditions for deadlock-free
lock acquisition are fulfilled.
@param lock_wait_timeout [in] Seconds to wait before timeout.
@retval FALSE Success
@retval TRUE Failure
@retval FALSE Success. MDL_request::ticket points to the ticket
for the lock.
@retval TRUE Failure (Out of resources or waiting is aborted),
*/
bool MDL_context::acquire_lock_impl(MDL_request *mdl_request,
ulong lock_wait_timeout)
bool
MDL_context::acquire_lock(MDL_request *mdl_request, ulong lock_wait_timeout)
{
MDL_lock *lock;
MDL_ticket *ticket;
bool not_used;
st_my_thread_var *mysys_var= my_thread_var;
MDL_key *key= &mdl_request->key;
struct timespec abs_timeout;
struct timespec abs_shortwait;
MDL_wait::enum_wait_status wait_status;
/* Do some work outside the critical section. */
set_timespec(abs_timeout, lock_wait_timeout);
mysql_mutex_assert_not_owner(&LOCK_open);
DBUG_ASSERT(mdl_request->ticket == NULL);
/* Don't take chances in production. */
mdl_request->ticket= NULL;
if (try_acquire_lock_impl(mdl_request, &ticket))
return TRUE;
/*
Check whether the context already holds an exclusive lock on the object,
and if so, grant the request.
*/
if ((ticket= find_ticket(mdl_request, &not_used)))
if (mdl_request->ticket)
{
DBUG_ASSERT(ticket->m_lock);
mdl_request->ticket= ticket;
/*
We have managed to acquire lock without waiting.
MDL_lock, MDL_context and MDL_request were updated
accordingly, so we can simply return success.
*/
return FALSE;
}
DBUG_ASSERT(mdl_request->type < MDL_SHARED_NO_WRITE ||
is_lock_owner(MDL_key::GLOBAL, "", "", MDL_INTENTION_EXCLUSIVE));
/* Early allocation: ticket will be needed in any case. */
if (!(ticket= MDL_ticket::create(this, mdl_request->type)))
return TRUE;
/*
Our attempt to acquire lock without waiting has failed.
As a result of this attempt we got MDL_ticket with m_lock
member pointing to the corresponding MDL_lock object which
has MDL_lock::m_rwlock write-locked.
*/
lock= ticket->m_lock;
/* The below call implicitly locks MDL_lock::m_rwlock on success. */
if (!(lock= mdl_locks.find_or_insert(key)))
{
MDL_ticket::destroy(ticket);
return TRUE;
}
lock->m_waiting.add_ticket(ticket);
ticket->m_lock= lock;
/*
Once we added a pending ticket to the waiting queue,
we must ensure that our wait slot is empty, so
that our lock request can be scheduled. Do that in the
critical section formed by the acquired write lock on MDL_lock.
*/
m_wait.reset_status();
lock->m_waiting.add_ticket(ticket);
if (ticket->is_upgradable_or_exclusive())
lock->notify_shared_locks(this);
while (!lock->can_grant_lock(mdl_request->type, this))
{
wait_reset();
mysql_prlock_unlock(&lock->m_rwlock);
if (ticket->is_upgradable_or_exclusive())
lock->notify_shared_locks(this);
will_wait_for(ticket);
mysql_prlock_unlock(&lock->m_rwlock);
/* There is a shared or exclusive lock on the object. */
DEBUG_SYNC(m_thd, "mdl_acquire_lock_wait");
will_wait_for(ticket);
find_deadlock();
/* There is a shared or exclusive lock on the object. */
DEBUG_SYNC(m_thd, "mdl_acquire_lock_wait");
if (ticket->is_upgradable_or_exclusive())
{
struct timespec abs_shortwait;
set_timespec(abs_shortwait, 1);
wait_status= MDL_wait::EMPTY;
bool is_deadlock= find_deadlock();
bool is_timeout= FALSE;
if (!is_deadlock)
while (cmp_timespec(abs_shortwait, abs_timeout) <= 0)
{
/* abs_timeout is far away. Wait a short while and notify locks. */
wait_status= m_wait.timed_wait(m_thd, &abs_shortwait, FALSE);
if (wait_status != MDL_wait::EMPTY)
break;
mysql_prlock_wrlock(&lock->m_rwlock);
lock->notify_shared_locks(this);
mysql_prlock_unlock(&lock->m_rwlock);
set_timespec(abs_shortwait, 1);
bool timeout_is_near= cmp_timespec(abs_shortwait, abs_timeout) > 0;
mdl_signal_type wait_result=
timed_wait(timeout_is_near ? &abs_timeout : &abs_shortwait);
if (timeout_is_near && wait_result == TIMEOUT_WAKE_UP)
is_timeout= TRUE;
else if (wait_result == VICTIM_WAKE_UP)
is_deadlock= TRUE;
}
if (wait_status == MDL_wait::EMPTY)
wait_status= m_wait.timed_wait(m_thd, &abs_timeout, TRUE);
}
else
wait_status= m_wait.timed_wait(m_thd, &abs_timeout, TRUE);
stop_waiting();
done_waiting_for();
if (mysys_var->abort || is_deadlock || is_timeout)
if (wait_status != MDL_wait::GRANTED)
{
lock->remove_ticket(&MDL_lock::m_waiting, ticket);
MDL_ticket::destroy(ticket);
switch (wait_status)
{
lock->remove_ticket(&MDL_lock::m_waiting, ticket);
MDL_ticket::destroy(ticket);
if (is_deadlock)
my_error(ER_LOCK_DEADLOCK, MYF(0));
else if (is_timeout)
my_error(ER_LOCK_WAIT_TIMEOUT, MYF(0));
return TRUE;
case MDL_wait::VICTIM:
my_error(ER_LOCK_DEADLOCK, MYF(0));
break;
case MDL_wait::TIMEOUT:
my_error(ER_LOCK_WAIT_TIMEOUT, MYF(0));
break;
case MDL_wait::KILLED:
break;
default:
DBUG_ASSERT(0);
break;
}
mysql_prlock_wrlock(&lock->m_rwlock);
return TRUE;
}
lock->m_waiting.remove_ticket(ticket);
lock->m_granted.add_ticket(ticket);
if (ticket->get_type() == MDL_EXCLUSIVE && lock->cached_object)
(*lock->cached_object_release_hook)(lock->cached_object);
lock->cached_object= NULL;
mysql_prlock_unlock(&lock->m_rwlock);
/*
We have been granted our request.
State of MDL_lock object is already being appropriately updated by a
concurrent thread (@sa MDL_lock:reschedule_waiters()).
So all we need to do is to update MDL_context and MDL_request objects.
*/
DBUG_ASSERT(wait_status == MDL_wait::GRANTED);
m_tickets.push_front(ticket);
......@@ -1750,7 +1932,7 @@ bool MDL_context::acquire_locks(MDL_request_list *mdl_requests,
for (p_req= sort_buf; p_req < sort_buf + req_count; p_req++)
{
if (acquire_lock_impl(*p_req, lock_wait_timeout))
if (acquire_lock(*p_req, lock_wait_timeout))
goto err;
}
my_free(sort_buf, MYF(0));
......@@ -1818,7 +2000,7 @@ MDL_context::upgrade_shared_lock_to_exclusive(MDL_ticket *mdl_ticket,
mdl_xlock_request.init(&mdl_ticket->m_lock->key, MDL_EXCLUSIVE);
if (acquire_lock_impl(&mdl_xlock_request, lock_wait_timeout))
if (acquire_lock(&mdl_xlock_request, lock_wait_timeout))
DBUG_RETURN(TRUE);
is_new_ticket= ! has_lock(mdl_svp, mdl_xlock_request.ticket);
......@@ -1855,15 +2037,72 @@ bool MDL_lock::find_deadlock(MDL_ticket *waiting_ticket,
MDL_context *src_ctx= waiting_ticket->get_ctx();
bool result= TRUE;
if (dvisitor->enter_node(src_ctx))
return TRUE;
mysql_prlock_rdlock(&m_rwlock);
/* Must be initialized after taking a read lock. */
Ticket_iterator granted_it(m_granted);
Ticket_iterator waiting_it(m_waiting);
/*
MDL_lock's waiting and granted queues and MDL_context::m_waiting_for
member are updated by different threads when the lock is granted
(see MDL_context::acquire_lock() and MDL_lock::reschedule_waiters()).
As a result, here we may encounter a situation when MDL_lock data
already reflects the fact that the lock was granted but
m_waiting_for member has not been updated yet.
For example, imagine that:
thread1: Owns SNW lock on table t1.
thread2: Attempts to acquire SW lock on t1,
but sees an active SNW lock.
Thus adds the ticket to the waiting queue and
sets m_waiting_for to point to the ticket.
thread1: Releases SNW lock, updates MDL_lock object to
grant SW lock to thread2 (moves the ticket for
SW from waiting to the active queue).
Attempts to acquire a new SNW lock on t1,
sees an active SW lock (since it is present in the
active queue), adds ticket for SNW lock to the waiting
queue, sets m_waiting_for to point to this ticket.
At this point deadlock detection algorithm run by thread1 will see that:
- Thread1 waits for SNW lock on t1 (since m_waiting_for is set).
- SNW lock is not granted, because it conflicts with active SW lock
owned by thread 2 (since ticket for SW is present in granted queue).
- Thread2 waits for SW lock (since its m_waiting_for has not been
updated yet!).
- SW lock is not granted because there is pending SNW lock from thread1.
Therefore deadlock should exist [sic!].
To avoid detection of such false deadlocks we need to check the "actual"
status of the ticket being waited for, before analyzing its blockers.
We do this by checking the wait status of the context which is waiting
for it. To avoid races this has to be done under protection of
MDL_lock::m_rwlock lock.
*/
if (src_ctx->m_wait.get_status() != MDL_wait::EMPTY)
{
result= FALSE;
goto end;
}
/*
To avoid visiting nodes which were already marked as victims of
deadlock detection (or whose requests were already satisfied) we
enter the node only after peeking at its wait status.
This is necessary to avoid active waiting in a situation
when previous searches for a deadlock already selected the
node we're about to enter as a victim (see the comment
in MDL_context::find_deadlock() for explanation why several searches
can be performed for the same wait).
There is no guarantee that the node isn't chosen a victim while we
are visiting it but this is OK: in the worst case we might do some
extra work and one more context might be chosen as a victim.
*/
if (dvisitor->enter_node(src_ctx))
goto end;
/*
We do a breadth-first search first -- that is, inspect all
edges of the current node, and only then follow up to the next
......@@ -1877,7 +2116,7 @@ bool MDL_lock::find_deadlock(MDL_ticket *waiting_ticket,
ticket->is_incompatible_when_granted(waiting_ticket->get_type()) &&
dvisitor->inspect_edge(ticket->get_ctx()))
{
goto end;
goto end_leave_node;
}
}
......@@ -1888,7 +2127,7 @@ bool MDL_lock::find_deadlock(MDL_ticket *waiting_ticket,
ticket->is_incompatible_when_waiting(waiting_ticket->get_type()) &&
dvisitor->inspect_edge(ticket->get_ctx()))
{
goto end;
goto end_leave_node;
}
}
......@@ -1900,7 +2139,7 @@ bool MDL_lock::find_deadlock(MDL_ticket *waiting_ticket,
ticket->is_incompatible_when_granted(waiting_ticket->get_type()) &&
ticket->get_ctx()->find_deadlock(dvisitor))
{
goto end;
goto end_leave_node;
}
}
......@@ -1911,14 +2150,17 @@ bool MDL_lock::find_deadlock(MDL_ticket *waiting_ticket,
ticket->is_incompatible_when_waiting(waiting_ticket->get_type()) &&
ticket->get_ctx()->find_deadlock(dvisitor))
{
goto end;
goto end_leave_node;
}
}
result= FALSE;
end_leave_node:
dvisitor->leave_node(src_ctx);
end:
mysql_prlock_unlock(&m_rwlock);
dvisitor->leave_node(src_ctx);
return result;
}
......@@ -1937,22 +2179,13 @@ bool MDL_context::find_deadlock(Deadlock_detection_visitor *dvisitor)
MDL_context *m_unlock_ctx= this;
bool result= FALSE;
mysql_prlock_rdlock(&m_waiting_for_lock);
mysql_prlock_rdlock(&m_LOCK_waiting_for);
if (m_waiting_for)
{
/*
QQ: should we rather be checking for NO_WAKE_UP ?
We want to do check signal only when m_waiting_for is set
to avoid reading left-overs from previous kills.
*/
if (peek_signal() != VICTIM_WAKE_UP)
{
result= m_waiting_for->m_lock->find_deadlock(m_waiting_for, dvisitor);
if (result)
m_unlock_ctx= dvisitor->opt_change_victim_to(this);
}
result= m_waiting_for->m_lock->find_deadlock(m_waiting_for, dvisitor);
if (result)
m_unlock_ctx= dvisitor->opt_change_victim_to(this);
}
/*
We may recurse into the same MDL_context more than once
......@@ -1961,7 +2194,7 @@ bool MDL_context::find_deadlock(Deadlock_detection_visitor *dvisitor)
the deadlock victim.
*/
if (m_unlock_ctx)
mysql_prlock_unlock(&m_unlock_ctx->m_waiting_for_lock);
mysql_prlock_unlock(&m_unlock_ctx->m_LOCK_waiting_for);
return result;
}
......@@ -1970,21 +2203,25 @@ bool MDL_context::find_deadlock(Deadlock_detection_visitor *dvisitor)
/**
Try to find a deadlock. This function produces no errors.
@note If during deadlock resolution context which performs deadlock
detection is chosen as a victim it will be informed about the
fact by setting VICTIM status to its wait slot.
@retval TRUE A deadlock is found.
@retval FALSE No deadlock found.
*/
bool MDL_context::find_deadlock()
void MDL_context::find_deadlock()
{
while (1)
{
MDL_context *victim;
/*
The fact that we use fresh instance of dvisitor for each
search performed by find_deadlock() below is important, code
responsible for victim selection relies on this.
search performed by find_deadlock() below is important,
the code responsible for victim selection relies on this.
*/
Deadlock_detection_visitor dvisitor(this);
MDL_context *victim;
if (! find_deadlock(&dvisitor))
{
......@@ -1994,125 +2231,31 @@ bool MDL_context::find_deadlock()
victim= dvisitor.get_victim();
if (victim != this)
{
victim->awake(VICTIM_WAKE_UP);
mysql_prlock_unlock(&victim->m_waiting_for_lock);
/*
After adding new arc to waiting graph we found that it participates
in some loop (i.e. there is a deadlock). We decided to destroy this
loop by removing some arc other than newly added. Since this doesn't
guarantee that all loops created by addition of this arc are
destroyed we have to repeat search.
*/
continue;
}
else
{
DBUG_ASSERT(&victim->m_waiting_for_lock == &m_waiting_for_lock);
mysql_prlock_unlock(&victim->m_waiting_for_lock);
return TRUE;
}
}
return FALSE;
}
/**
Wait until there will be no locks that conflict with lock requests
in the given list.
This is a part of the locking protocol and must be used by the
acquirer of shared locks after a back-off.
Does not acquire the locks!
@param lock_wait_timeout Seconds to wait before timeout.
@retval FALSE Success. One can try to obtain metadata locks.
@retval TRUE Failure (thread was killed or deadlock is possible).
*/
bool
MDL_context::wait_for_lock(MDL_request *mdl_request, ulong lock_wait_timeout)
{
MDL_lock *lock;
st_my_thread_var *mysys_var= my_thread_var;
struct timespec abs_timeout;
set_timespec(abs_timeout, lock_wait_timeout);
mysql_mutex_assert_not_owner(&LOCK_open);
DBUG_ASSERT(mdl_request->ticket == NULL);
while (TRUE)
{
/*
We have to check if there are some HANDLERs open by this thread
which conflict with some pending exclusive locks. Otherwise we
might have a deadlock in situations when we are waiting for
pending writer to go away, which in its turn waits for HANDLER
open by our thread.
TODO: investigate situations in which we need to broadcast on
COND_mdl because of above scenario.
Failure to change status of the victim is OK as it means
that the victim has received some other message and is
about to stop its waiting/to break deadlock loop.
Even when the initiator of the deadlock search is
chosen the victim, we need to set the respective wait
result in order to "close" it for any attempt to
schedule the request.
This is needed to avoid a possible race during
cleanup in case when the lock request on which the
context was waiting is concurrently satisfied.
*/
mysql_ha_flush(m_thd);
MDL_key *key= &mdl_request->key;
/* The below call implicitly locks MDL_lock::m_rwlock on success. */
if (! (lock= mdl_locks.find(key)))
return FALSE;
if (lock->can_grant_lock(mdl_request->type, this))
{
mysql_prlock_unlock(&lock->m_rwlock);
return FALSE;
}
MDL_ticket *pending_ticket;
if (! (pending_ticket= MDL_ticket::create(this, mdl_request->type)))
{
mysql_prlock_unlock(&lock->m_rwlock);
return TRUE;
}
(void) victim->m_wait.set_status(MDL_wait::VICTIM);
mysql_prlock_unlock(&victim->m_LOCK_waiting_for);
pending_ticket->m_lock= lock;
lock->m_waiting.add_ticket(pending_ticket);
wait_reset();
mysql_prlock_unlock(&lock->m_rwlock);
will_wait_for(pending_ticket);
bool is_deadlock= find_deadlock();
bool is_timeout= FALSE;
if (!is_deadlock)
{
mdl_signal_type wait_result= timed_wait(&abs_timeout);
if (wait_result == TIMEOUT_WAKE_UP)
is_timeout= TRUE;
else if (wait_result == VICTIM_WAKE_UP)
is_deadlock= TRUE;
}
stop_waiting();
lock->remove_ticket(&MDL_lock::m_waiting, pending_ticket);
MDL_ticket::destroy(pending_ticket);
if (mysys_var->abort || is_deadlock || is_timeout)
{
if (is_deadlock)
my_error(ER_LOCK_DEADLOCK, MYF(0));
else if (is_timeout)
my_error(ER_LOCK_WAIT_TIMEOUT, MYF(0));
return TRUE;
}
if (victim == this)
break;
/*
After adding a new edge to the waiting graph we found that it
creates a loop (i.e. there is a deadlock). We decided to destroy
this loop by removing an edge, but not the one that we added.
Since this doesn't guarantee that all loops created by addition
of the new edge are destroyed, we have to repeat the search.
*/
}
return TRUE;
}
......@@ -2241,7 +2384,7 @@ void MDL_ticket::downgrade_exclusive_lock(enum_mdl_type type)
m_lock->m_granted.remove_ticket(this);
m_type= type;
m_lock->m_granted.add_ticket(this);
m_lock->wake_up_waiters();
m_lock->reschedule_waiters();
mysql_prlock_unlock(&m_lock->m_rwlock);
}
......@@ -2446,7 +2589,6 @@ void MDL_context::move_ticket_after_trans_sentinel(MDL_ticket *mdl_ticket)
if (m_trans_sentinel == NULL)
{
m_trans_sentinel= mdl_ticket;
/* sic: linear from the number of transactional tickets acquired so-far! */
m_tickets.push_back(mdl_ticket);
}
else
......
......@@ -447,6 +447,37 @@ class MDL_ticket
};
/**
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 @@ class MDL_context
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 @@ class MDL_context
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 @@ class MDL_context
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 @@ class MDL_context
}
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 @@ class MDL_context
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).
*/
mysql_mutex_t m_signal_lock;
mysql_cond_t m_signal_cond;
mdl_signal_type m_signal;
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.
*/
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,
......
......@@ -2364,6 +2364,66 @@ void table_share_release_hook(void *share)
}
/**
An error handler which converts, if possible, ER_LOCK_DEADLOCK error
that can occur when we are trying to acquire a metadata lock to
a request for back-off and re-start of open_tables() process.
*/
class MDL_deadlock_handler : public Internal_error_handler
{
public:
MDL_deadlock_handler(Open_table_context *ot_ctx_arg)
: m_ot_ctx(ot_ctx_arg), m_is_active(FALSE)
{}
virtual ~MDL_deadlock_handler() {}
virtual bool handle_condition(THD *thd,
uint sql_errno,
const char* sqlstate,
MYSQL_ERROR::enum_warning_level level,
const char* msg,
MYSQL_ERROR ** cond_hdl);
private:
/** Open table context to be used for back-off request. */
Open_table_context *m_ot_ctx;
/**
Indicates that we are already in the process of handling
ER_LOCK_DEADLOCK error. Allows to re-emit the error from
the error handler without falling into infinite recursion.
*/
bool m_is_active;
};
bool MDL_deadlock_handler::handle_condition(THD *,
uint sql_errno,
const char*,
MYSQL_ERROR::enum_warning_level,
const char*,
MYSQL_ERROR ** cond_hdl)
{
*cond_hdl= NULL;
if (! m_is_active && sql_errno == ER_LOCK_DEADLOCK)
{
/* Disable the handler to avoid infinite recursion. */
m_is_active= TRUE;
(void) m_ot_ctx->request_backoff_action(Open_table_context::OT_MDL_CONFLICT,
NULL);
m_is_active= FALSE;
/*
If the above back-off request failed, a new instance of
ER_LOCK_DEADLOCK error was emitted. Thus the current
instance of error condition can be treated as handled.
*/
return TRUE;
}
return FALSE;
}
/**
Try to acquire an MDL lock for a table being opened.
......@@ -2432,19 +2492,61 @@ open_table_get_mdl_lock(THD *thd, Open_table_context *ot_ctx,
ot_ctx->add_request(mdl_request);
if (thd->mdl_context.try_acquire_lock(mdl_request))
return TRUE;
if (mdl_request->ticket == NULL)
if (flags & MYSQL_OPEN_FAIL_ON_MDL_CONFLICT)
{
if (flags & MYSQL_OPEN_FAIL_ON_MDL_CONFLICT)
/*
When table is being open in order to get data for I_S table,
we might have some tables not only open but also locked (e.g. when
this happens under LOCK TABLES or in a stored function).
As a result by waiting on a conflicting metadata lock to go away
we may 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 we skip the trouble-making table if
there is a conflicting lock.
*/
if (thd->mdl_context.try_acquire_lock(mdl_request))
return TRUE;
if (mdl_request->ticket == NULL)
{
my_error(ER_WARN_I_S_SKIPPED_TABLE, MYF(0),
mdl_request->key.db_name(), mdl_request->key.name());
return TRUE;
}
if (ot_ctx->request_backoff_action(Open_table_context::OT_WAIT_MDL_LOCK,
mdl_request, NULL))
}
else
{
/*
We are doing a normal table open. Let us try to acquire a metadata
lock on the table. If there is a conflicting lock, acquire_lock()
will wait for it to go away. Sometimes this waiting may lead to a
deadlock, with the following results:
1) If a deadlock is entirely within MDL subsystem, it is
detected by the deadlock detector of this subsystem.
ER_LOCK_DEADLOCK error is produced. Then, the error handler
that is installed prior to the call to acquire_lock() attempts
to request a back-off and retry. Upon success, ER_LOCK_DEADLOCK
error is suppressed, otherwise propagated up the calling stack.
2) Otherwise, a deadlock may occur when the wait-for graph
includes edges not visible to the MDL deadlock detector.
One such example is a wait on an InnoDB row lock, e.g. when:
conn C1 gets SR MDL lock on t1 with SELECT * FROM t1
conn C2 gets a row lock on t2 with SELECT * FROM t2 FOR UPDATE
conn C3 gets in and waits on C1 with DROP TABLE t0, t1
conn C2 continues and blocks on C3 with SELECT * FROM t0
conn C1 deadlocks by waiting on C2 by issuing SELECT * FROM
t2 LOCK IN SHARE MODE.
Such circular waits are currently only resolved by timeouts,
e.g. @@innodb_lock_wait_timeout or @@lock_wait_timeout.
*/
MDL_deadlock_handler mdl_deadlock_handler(ot_ctx);
thd->push_internal_handler(&mdl_deadlock_handler);
bool result= thd->mdl_context.acquire_lock(mdl_request,
ot_ctx->get_timeout());
thd->pop_internal_handler();
if (result && !ot_ctx->can_recover_from_failed_open())
return TRUE;
}
*mdl_ticket= mdl_request->ticket;
......@@ -2542,7 +2644,7 @@ bool open_table(THD *thd, TABLE_LIST *table_list, MEM_ROOT *mem_root,
if (thd->version != refresh_version)
{
(void) ot_ctx->request_backoff_action(Open_table_context::OT_WAIT_TDC,
NULL, NULL);
NULL);
DBUG_RETURN(TRUE);
}
}
......@@ -2754,7 +2856,7 @@ bool open_table(THD *thd, TABLE_LIST *table_list, MEM_ROOT *mem_root,
/* Someone did a refresh while thread was opening tables */
mysql_mutex_unlock(&LOCK_open);
(void) ot_ctx->request_backoff_action(Open_table_context::OT_WAIT_TDC,
NULL, NULL);
NULL);
DBUG_RETURN(TRUE);
}
......@@ -2885,7 +2987,7 @@ bool open_table(THD *thd, TABLE_LIST *table_list, MEM_ROOT *mem_root,
release_table_share(share);
mysql_mutex_unlock(&LOCK_open);
(void) ot_ctx->request_backoff_action(Open_table_context::OT_WAIT_TDC,
NULL, NULL);
NULL);
DBUG_RETURN(TRUE);
}
/* Force close at once after usage */
......@@ -2926,13 +3028,13 @@ bool open_table(THD *thd, TABLE_LIST *table_list, MEM_ROOT *mem_root,
{
share->version= 0;
(void) ot_ctx->request_backoff_action(Open_table_context::OT_DISCOVER,
NULL, table_list);
table_list);
}
else if (share->crashed)
{
share->version= 0;
(void) ot_ctx->request_backoff_action(Open_table_context::OT_REPAIR,
NULL, table_list);
table_list);
}
goto err_unlock;
......@@ -3789,8 +3891,7 @@ static bool auto_repair_table(THD *thd, TABLE_LIST *table_list)
/** Open_table_context */
Open_table_context::Open_table_context(THD *thd, uint flags)
:m_failed_mdl_request(NULL),
m_failed_table(NULL),
:m_failed_table(NULL),
m_start_of_statement_svp(thd->mdl_context.mdl_savepoint()),
m_global_mdl_request(NULL),
m_timeout(flags & MYSQL_LOCK_IGNORE_TIMEOUT ?
......@@ -3834,37 +3935,44 @@ MDL_request *Open_table_context::get_global_mdl_request(THD *thd)
bool
Open_table_context::
request_backoff_action(enum_open_table_action action_arg,
MDL_request *mdl_request, TABLE_LIST *table)
TABLE_LIST *table)
{
/*
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.
A back off action may be one of the three kinds:
* We met a broken table that needs repair, or a table that
is not present on this MySQL server and needs re-discovery.
To perform the action, we need an exclusive metadata lock on
the table. Acquiring an X lock while holding other shared
locks is very deadlock-prone. If this is a multi- statement
transaction that holds metadata locks for completed
statements, we don't do it, and report an error instead.
* Our attempt to acquire an MDL lock lead to a deadlock,
detected by the MDL deadlock detector. The current
session was chosen a victim. If this is a multi-statement
transaction that holds metadata locks for completed statements,
restarting locking for the current statement may lead
to a livelock. Thus, again, if m_has_locks is set,
we report an error. Otherwise, when there are no metadata
locks other than which belong to this statement, we can
try to recover from error by releasing all locks and
restarting the pre-locking.
* Finally, we could have met a TABLE_SHARE with old version.
Again, if this is a first statement in a transaction we can
close all tables, release all metadata locks and wait for
the old version to go away. Otherwise, waiting with MDL locks
may lead to criss-cross wait between this connection and a
connection that has an open table and waits on a metadata lock,
i.e. to a deadlock.
Since there is no way to detect such a deadlock, we prevent
it by reporting an error.
*/
if (m_has_locks && action_arg != OT_WAIT_MDL_LOCK)
if (m_has_locks)
{
my_error(ER_LOCK_DEADLOCK, MYF(0));
return TRUE;
}
m_action= action_arg;
/*
If waiting for metadata lock is requested, a pointer to
MDL_request object for which we were unable to acquire the
lock is required.
*/
DBUG_ASSERT(m_action != OT_WAIT_MDL_LOCK || mdl_request);
m_failed_mdl_request= mdl_request;
/*
If auto-repair or discovery are requested, a pointer to table
list element must be provided.
......@@ -3900,9 +4008,7 @@ recover_from_failed_open(THD *thd)
/* Execute the action. */
switch (m_action)
{
case OT_WAIT_MDL_LOCK:
result= thd->mdl_context.wait_for_lock(m_failed_mdl_request,
get_timeout());
case OT_MDL_CONFLICT:
break;
case OT_WAIT_TDC:
result= tdc_wait_for_old_versions(thd, &m_mdl_requests, get_timeout());
......@@ -3973,7 +4079,6 @@ recover_from_failed_open(THD *thd)
TABLE_LIST element, set when we need auto-discovery or repair,
for safety.
*/
m_failed_mdl_request= NULL;
m_failed_table= NULL;
/* Prepare for possible another back-off. */
m_action= OT_NO_ACTION;
......@@ -4097,16 +4202,25 @@ open_and_process_routine(THD *thd, Query_tables_list *prelocking_ctx,
*/
DBUG_ASSERT(rt->mdl_request.type == MDL_SHARED);
if (thd->mdl_context.try_acquire_lock(&rt->mdl_request))
DBUG_RETURN(TRUE);
/*
Waiting for a conflicting metadata lock to go away may
lead to a deadlock, detected by MDL subsystem.
If possible, we try to resolve such deadlocks by releasing all
metadata locks and restarting the pre-locking process.
To prevent the error from polluting the diagnostics area
in case of successful resolution, install a special error
handler for ER_LOCK_DEADLOCK error.
*/
MDL_deadlock_handler mdl_deadlock_handler(ot_ctx);
if (rt->mdl_request.ticket == NULL)
{
/* A lock conflict. Someone's trying to modify SP metadata. */
ot_ctx->request_backoff_action(Open_table_context::OT_WAIT_MDL_LOCK,
&rt->mdl_request, NULL);
thd->push_internal_handler(&mdl_deadlock_handler);
bool result= thd->mdl_context.acquire_lock(&rt->mdl_request,
ot_ctx->get_timeout());
thd->pop_internal_handler();
if (result)
DBUG_RETURN(TRUE);
}
DEBUG_SYNC(thd, "after_shared_lock_pname");
/* Ensures the routine is up-to-date and cached, if exists. */
......@@ -4613,18 +4727,6 @@ bool open_tables(THD *thd, TABLE_LIST **start, uint *counter, uint flags,
bool has_prelocking_list;
DBUG_ENTER("open_tables");
/*
Close HANDLER tables which are marked for flush or against which there
are pending exclusive metadata locks. Note that we do this not to avoid
deadlocks (calls to mysql_ha_flush() in mdl_wait_for_locks() and
tdc_wait_for_old_version() are enough for this) but in order to have
a point during statement execution at which such HANDLERs are closed
even if they don't create problems for current thread (i.e. to avoid
having DDL blocked by HANDLERs opened for long time).
*/
if (thd->handler_tables_hash.records)
mysql_ha_flush(thd);
/*
temporary mem_root for new .frm parsing.
TODO: variables for size
......@@ -4633,6 +4735,17 @@ bool open_tables(THD *thd, TABLE_LIST **start, uint *counter, uint flags,
thd->current_tablenr= 0;
restart:
/*
Close HANDLER tables which are marked for flush or against which there
are pending exclusive metadata locks. This is needed both in order to
avoid deadlocks and to have a point during statement execution at
which such HANDLERs are closed even if they don't create problems for
the current session (i.e. to avoid having a DDL blocked by HANDLERs
opened for a long time).
*/
if (thd->handler_tables_hash.records)
mysql_ha_flush(thd);
has_prelocking_list= thd->lex->requires_prelocking();
table_to_open= start;
sroutine_to_open= (Sroutine_hash_entry**) &thd->lex->sroutines_list.first;
......@@ -5578,15 +5691,6 @@ void close_tables_for_reopen(THD *thd, TABLE_LIST **tables,
/* We have to cleanup translation tables of views. */
tmp->cleanup_items();
}
/*
Metadata lock requests for tables from extended part of prelocking set
are part of list of requests to be waited for in Open_table_context.
So to satisfy assumptions in MDL_context::wait_for_locks(), which will
performs the waiting, we have to reset MDL_request::ticket values for
them as well.
*/
for (tmp= first_not_own_table; tmp; tmp= tmp->next_global)
tmp->mdl_request.ticket= NULL;
close_thread_tables(thd);
thd->mdl_context.rollback_to_savepoint(start_of_statement_svp);
}
......@@ -8692,8 +8796,12 @@ tdc_wait_for_old_versions(THD *thd, MDL_request_list *mdl_requests,
while (!thd->killed)
{
/*
Here we have situation as in mdl_wait_for_locks() we need to
get rid of offending HANDLERs to avoid deadlock.
We have to get rid of HANDLERs which are open by this thread
and have old TABLE versions. Otherwise we might get a deadlock
in situation when we are waiting for an old TABLE object which
corresponds to a HANDLER open by another session. And this
other session waits for our HANDLER object to get closed.
TODO: We should also investigate in which situations we have
to broadcast on COND_refresh because of this.
*/
......
......@@ -451,7 +451,7 @@ class Open_table_context
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 @@ class Open_table_context
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 @@ class Open_table_context
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 @@ class I_P_List : public C
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 @@ class I_P_List : public C
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 @@ class I_P_List : public C
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 @@ class I_P_List : public C
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 @@ class I_P_List_counter
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