Commit graph

59694 commits

Author SHA1 Message Date
Dmitry Lenev
afd15c43a9 Implement new type-of-operation-aware metadata locks.
Add a wait-for graph based deadlock detector to the
MDL subsystem.

Fixes bug #46272 "MySQL 5.4.4, new MDL: unnecessary deadlock" and
bug #37346 "innodb does not detect deadlock between update and
alter table".

The first bug manifested itself as an unwarranted abort of a
transaction with ER_LOCK_DEADLOCK error by a concurrent ALTER
statement, when this transaction tried to repeat use of a
table, which it has already used in a similar fashion before
ALTER started.

The second bug showed up as a deadlock between table-level
locks and InnoDB row locks, which was "detected" only after
innodb_lock_wait_timeout timeout.

A transaction would start using the table and modify a few
rows.
Then ALTER TABLE would come in, and start copying rows
into a temporary table. Eventually it would stumble on
the modified records and get blocked on a row lock.
The first transaction would try to do more updates, and get
blocked on thr_lock.c lock.
This situation of circular wait would only get resolved
by a timeout.

Both these bugs stemmed from inadequate solutions to the
problem of deadlocks occurring between different
locking subsystems.

In the first case we tried to avoid deadlocks between metadata
locking and table-level locking subsystems, when upgrading shared
metadata lock to exclusive one.
Transactions holding the shared lock on the table and waiting for
some table-level lock used to be aborted too aggressively.

We also allowed ALTER TABLE to start in presence of transactions
that modify the subject table. ALTER TABLE acquires
TL_WRITE_ALLOW_READ lock at start, and that block all writes
against the table (naturally, we don't want any writes to be lost
when switching the old and the new table). TL_WRITE_ALLOW_READ
lock, in turn, would block the started transaction on thr_lock.c
lock, should they do more updates. This, again, lead to the need
to abort such transactions.

The second bug occurred simply because we didn't have any
mechanism to detect deadlocks between the table-level locks
in thr_lock.c and row-level locks in InnoDB, other than
innodb_lock_wait_timeout.

This patch solves both these problems by moving lock conflicts
which are causing these deadlocks into the metadata locking
subsystem, thus making it possible to avoid or detect such
deadlocks inside MDL.

To do this we introduce new type-of-operation-aware metadata
locks, which allow MDL subsystem to know not only the fact that
transaction has used or is going to use some object but also what
kind of operation it has carried out or going to carry out on the
object.

This, along with the addition of a special kind of upgradable
metadata lock, allows ALTER TABLE to wait until all
transactions which has updated the table to go away.
This solves the second issue.
Another special type of upgradable metadata lock is acquired
by LOCK TABLE WRITE. This second lock type allows to solve the
first issue, since abortion of table-level locks in event of
DDL under LOCK TABLES becomes also unnecessary.

Below follows the list of incompatible changes introduced by
this patch:

- From now on, ALTER TABLE and CREATE/DROP TRIGGER SQL (i.e. those
  statements that acquire TL_WRITE_ALLOW_READ lock)
  wait for all transactions which has *updated* the table to
  complete.

- From now on, LOCK TABLES ... WRITE, REPAIR/OPTIMIZE TABLE
  (i.e. all statements which acquire TL_WRITE table-level lock) wait
  for all transaction which *updated or read* from the table
  to complete.
  As a consequence, innodb_table_locks=0 option no longer applies
  to LOCK TABLES ... WRITE.

- DROP DATABASE, DROP TABLE, RENAME TABLE no longer abort
  statements or transactions which use tables being dropped or
  renamed, and instead wait for these transactions to complete.

- Since LOCK TABLES WRITE now takes a special metadata lock,
  not compatible with with reads or writes against the subject table
  and transaction-wide, thr_lock.c deadlock avoidance algorithm
  that used to ensure absence of deadlocks between LOCK TABLES
  WRITE and other statements is no longer sufficient, even for
  MyISAM. The wait-for graph based deadlock detector of MDL
  subsystem may sometimes be necessary and is involved. This may
  lead to ER_LOCK_DEADLOCK error produced for multi-statement
  transactions even if these only use MyISAM:

  session 1:         session 2:
  begin;

  update t1 ...      lock table t2 write, t1 write;
                     -- gets a lock on t2, blocks on t1

  update t2 ...
  (ER_LOCK_DEADLOCK)

- Finally,  support of LOW_PRIORITY option for LOCK TABLES ... WRITE
  was abandoned.
  LOCK TABLE ... LOW_PRIORITY WRITE from now on has the same
  priority as the usual LOCK TABLE ... WRITE.
  SELECT HIGH PRIORITY no longer trumps LOCK TABLE ... WRITE  in
  the wait queue.

- We do not take upgradable metadata locks on implicitly
  locked tables. So if one has, say, a view v1 that uses
  table t1, and issues:
  LOCK TABLE v1 WRITE;
  FLUSH TABLE t1; -- (or just 'FLUSH TABLES'),
  an error is produced.
  In order to be able to perform DDL on a table under LOCK TABLES,
  the table must be locked explicitly in the LOCK TABLES list.
2010-02-01 14:43:06 +03:00
Dmitry Lenev
a63f8480db Patch that changes metadata locking subsystem to use mutex per lock and
condition variable per context instead of one mutex and one conditional
variable for the whole subsystem.

This should increase concurrency in this subsystem.

It also opens the way for further changes which are necessary to solve
such bugs as bug #46272 "MySQL 5.4.4, new MDL: unnecessary deadlock"
and bug #37346 "innodb does not detect deadlock between update and alter
table".

Two other notable changes done by this patch:

- MDL subsystem no longer implicitly acquires global intention exclusive
  metadata lock when per-object metadata lock is acquired. Now this has
  to be done by explicit calls outside of MDL subsystem.
- Instead of using separate MDL_context for opening system tables/tables
  for purposes of I_S we now create MDL savepoint in the main context
  before opening tables and rollback to this savepoint after closing
  them. This means that it is now possible to get ER_LOCK_DEADLOCK error
  even not inside a transaction. This might happen in unlikely case when
  one runs DDL on one of system tables while also running DDL on some
  other tables. Cases when this ER_LOCK_DEADLOCK error is not justified
  will be addressed by advanced deadlock detector for MDL subsystem which
  we plan to implement.
2010-01-21 23:43:03 +03:00
Jon Olav Hauglid
c005126107 Bug #50412 Assertion `! is_set()' failed in
Diagnostics_area::set_ok_status at PREPARE

The problem occured during processing of stored routines. 
Routines are loaded from mysql.proc, parsed and put into the sp cache by
sp_cache_routine().  The assert occured because the return value from
sp_cache_routine() was not checked for top level CALLs. This meant that any
errors during sp_cache_routine() went unoticed and triggered the assert when
my_ok() was later called.

This is a regression introduced by the patch for Bug#30977, only visible in
source trees with MDL and using debug builds of the server.

This patch fixes the problem by checking the return value from sp_cache_routine() 
for top level CALLs and propagating any errors similar to what is done for other 
calls to sp_cache_routine().

No test case added.
2010-01-20 09:17:31 +01:00
Jon Olav Hauglid
c2beb68385 Bug #43685 Lock table affects other non-related tables
The problem was that FLUSH TABLE <table_list> would block, 
waiting for all tables with old versions to be removed from 
the table definition cache, rather than waiting for only 
the tables in <table_list>. This could happen if FLUSH TABLE
was used in combination with LOCK TABLES.

With the new MDL code, this problem is no longer repeatable.
Regression test case added to lock.test. This commit contains
no code changes.
2010-01-15 12:47:22 +01:00
Jon Olav Hauglid
5045ad38de Partial backport of:
revno: 2762 [merge]
committer: Matthias Leich <mleich@mysql.com>
branch nick: mysql-6.0-bugteam-push
timestamp: Wed 2008-08-13 22:05:34 +0200
message:
  Upmerge 5.1 -> 6.0
    ------------------------------------------------------------
    revno: 2497.374.2
    committer: Matthias Leich <mleich@mysql.com>
    branch nick: mysql-5.1-bugteam-push
    timestamp: Wed 2008-08-13 21:44:54 +0200
    message:
      Fix for Bug#37853
          Test "funcs_1.processlist_val_ps" fails in various ways
      + corrections of logic in poll routines
      + minor improvements
2010-01-14 14:03:24 +01:00
Jon Olav Hauglid
db1888b53c Bug #49988 MDL deadlocks with mysql_create_db, reload_acl_and_cache
This was a deadlock between LOCK TABLES/CREATE DATABASE in one connection
and DROP DATABASE in another. It only happened if the table locked by 
LOCK TABLES was in the database to be dropped. The deadlock is similar
to the one in Bug#48940, but with LOCK TABLES instead of an active
transaction.

The order of events needed to trigger the deadlock was:
1) Connection 1 locks table db1.t1 using LOCK TABLES. It will now
have a metadata lock on the table name.
2) Connection 2 issues DROP DATABASE db1. This will wait inside
the MDL subsystem for the lock on db1.t1 to go away. While waiting, it
will hold the LOCK_mysql_create_db mutex.
3) Connection 1 issues CREATE DATABASE (database name irrelevant).
This will hang trying to lock the same mutex. Since this is the connection
holding the metadata lock blocking Connection 2, we have a deadlock.

This deadlock would also happen for earlier trees without MDL, but 
there DROP DATABASE would wait for a table to be removed from the
table definition cache.

This patch fixes the problem by prohibiting CREATE DATABASE in LOCK TABLES
mode. In the example above, this prevents Connection 1 from hanging trying
to get the LOCK_mysql_create_db mutex. Note that other commands that use
LOCK_mysql_create_db (ALTER/DROP DATABASE) are already prohibited in 
LOCK TABLES mode.

Incompatible change: CREATE DATABASE is now disallowed in LOCK TABLES mode.

Test case added to schema.test.
2010-01-12 16:15:21 +01:00
Tor Didriksen
6766c0d676 Backport of
Bug#45523 "Objects of class base_ilist should not be copyable".
               
Suppress the compiler-generated public copy constructor
and assignment operator of class base_ilist; instead, implement
move_elements_to() function which transfers ownership of elements
from one list to another.
2010-01-12 12:32:55 +01:00
Jon Olav Hauglid
0bce0c9041 Fix for bug #48538 "Assertion in thr_lock() on LOAD DATA CONCURRENT
INFILE".

Attempts to execute an INSERT statement for a MEMORY table which invoked
a trigger or called a stored function which tried to perform LOW_PRIORITY
update on the table being inserted into, resulted in debug servers aborting
due to an assertion failure. On non-debug servers such INSERTs failed with
"Can't update table t1 in stored function/trigger because it is already used
by statement which invoked this stored function/trigger" as expected.

The problem was that in the above scenario TL_WRITE_CONCURRENT_INSERT
is converted to TL_WRITE inside the thr_lock() function since the MEMORY
engine does not support concurrent inserts. This triggered an assertion
which assumed that for the same table, one thread always requests locks with
higher thr_lock_type value first. When TL_WRITE_CONCURRENT_INSERT is
upgraded to TL_WRITE after the locks have been sorted, this is no longer true.
In this case, TL_WRITE was requested after acquiring a TL_WRITE_LOW_PRIORITY
lock on the table, triggering the assert.

This fix solves the problem by adjusting this assert to take this
scenario into account.

An alternative approach to change handler::store_locks() methods for all engines
which do not support concurrent inserts in such way that
TL_WRITE_CONCURRENT_INSERT is upgraded to TL_WRITE there instead, 
was considered too intrusive.

Commit on behalf of Dmitry Lenev.
2010-01-08 11:26:32 +01:00
Dmitry Lenev
236539b471 Implementation of simple deadlock detection for metadata locks.
This change is supposed to reduce number of ER_LOCK_DEADLOCK
errors which occur when multi-statement transaction encounters
conflicting metadata lock in cases when waiting is possible.

The idea is not to fail ER_LOCK_DEADLOCK error immediately when
we encounter conflicting metadata lock. Instead we release all
metadata locks acquired by current statement and start to wait
until conflicting lock go away. To avoid deadlocks we use simple
empiric which aborts waiting with ER_LOCK_DEADLOCK error if it
turns out that somebody is waiting for metadata locks owned by
this transaction.

This patch also fixes bug #46273 "MySQL 5.4.4 new MDL: Bug#989
is not fully fixed in case of ALTER".

The bug was that concurrent execution of UPDATE or MULTI-UPDATE
statement as a part of multi-statement transaction that already
has used table being updated and ALTER TABLE statement might have
resulted of loss of isolation between this transaction and ALTER
TABLE statement, which manifested itself as changes performed by
ALTER TABLE becoming visible in transaction and wrong binary log
order as a consequence.

This problem occurred when UPDATE or MULTI-UPDATE's wait in
mysql_lock_tables() call was aborted due to metadata lock
upgrade performed by concurrent ALTER TABLE. After such abort all
metadata locks held by transaction were released but transaction
silently continued to be executed as if nothing has happened.

We solve this problem by changing our code not to release all
locks in such case. Instead we release only locks which were
acquired by current statement and then try to reacquire them
by restarting open/lock tables process. We piggyback on simple
deadlock detector implementation since this change has to be
done anyway for it.
2009-12-30 20:53:30 +03:00
Alexander Nozdrin
cd6fbffc38 Disable test case for Bug#49972. 2009-12-29 21:12:06 +03:00
Alexander Nozdrin
93613d4e07 A test case for Bug#49972 (Crash in prepared statements). 2009-12-29 18:50:01 +03:00
Konstantin Osipov
3b311f399d Apply and review:
3655 Jon Olav Hauglid   2009-10-19
Bug #30977 Concurrent statement using stored function and DROP FUNCTION 
           breaks SBR
Bug #48246 assert in close_thread_table

Implement a fix for:
Bug #41804 purge stored procedure cache causes mysterious hang for many
           minutes
Bug #49972 Crash in prepared statements

The problem was that concurrent execution of DML statements that
use stored functions and DDL statements that drop/modify the same
function might result in incorrect binary log in statement (and
mixed) mode and therefore break replication.

This patch fixes the problem by introducing metadata locking for
stored procedures and functions. This is similar to what is done
in Bug#25144 for views. Procedures and functions now are
locked using metadata locks until the transaction is either
committed or rolled back. This prevents other statements from
modifying the procedure/function while it is being executed. This
provides commit ordering - guaranteeing serializability across
multiple transactions and thus fixes the reported binlog problem.

Note that we do not take locks for top-level CALLs. This means
that procedures called directly are not protected from changes by
simultaneous DDL operations so they are executed at the state they
had at the time of the CALL. By not taking locks for top-level
CALLs, we still allow transactions to be started inside
procedures.

This patch also changes stored procedure cache invalidation.
Upon a change of cache version, we no longer invalidate the entire
cache, but only those routines which we use, only when a statement
is executed that uses them.

This patch also changes the logic of prepared statement validation.
A stored procedure used by a prepared statement is now validated
only once a metadata lock has been acquired. A version mismatch
causes a flush of the obsolete routine from the cache and
statement reprepare.
Incompatible changes:
1) ER_LOCK_DEADLOCK is reported for a transaction trying to access
   a procedure/function that is locked by a DDL operation in
   another connection.

2) Procedure/function DDL operations are now prohibited in LOCK
   TABLES mode as exclusive locks must be taken all at once and
   LOCK TABLES provides no way to specifiy procedures/functions to
   be locked.

Test cases have been added to sp-lock.test and rpl_sp.test.

Work on this bug has very much been a team effort and this patch
includes and is based on contributions from Davi Arnaut, Dmitry
Lenev, Magne Mæhre and Konstantin Osipov.
2009-12-29 15:19:05 +03:00
Konstantin Osipov
39a1a50dfb A prerequisite patch for the fix for Bug#46224
"HANDLER statements within a transaction might lead to deadlocks".
Introduce a notion of a sentinel to MDL_context. A sentinel
is a ticket that separates all tickets in the context into two
groups: before and after it. Currently we can have (and need) only
one designated sentinel -- it separates all locks taken by LOCK
TABLE or HANDLER statement, which must survive COMMIT and ROLLBACK
and all other locks, which must be released at COMMIT or ROLLBACK.
The tricky part is maintaining the sentinel up to date when
someone release its corresponding ticket. This can happen, e.g.
if someone issues DROP TABLE under LOCK TABLES (generally,
see all calls to release_all_locks_for_name()).
MDL_context::release_ticket() is modified to take care of it.

******
A fix and a test case for Bug#46224 "HANDLER statements within a
transaction might lead to deadlocks".

An attempt to mix HANDLER SQL statements, which are transaction-
agnostic, an open multi-statement transaction,
and DDL against the involved tables (in a concurrent connection) 
could lead to a deadlock. The deadlock would occur when
HANDLER OPEN or HANDLER READ would have to wait on a conflicting
metadata lock. If the connection that issued HANDLER statement
also had other metadata locks (say, acquired in scope of a 
transaction), a classical deadlock situation of mutual wait
could occur.

Incompatible change: entering LOCK TABLES mode automatically
closes all open HANDLERs in the current connection.

Incompatible change: previously an attempt to wait on a lock
in a connection that has an open HANDLER statement could wait
indefinitely/deadlock. After this patch, an error ER_LOCK_DEADLOCK
is produced.

The idea of the fix is to merge thd->handler_mdl_context
with the main mdl_context of the connection, used for transactional
locks. This makes deadlock detection possible, since all waits
with locks are "visible" and available to analysis in a single
MDL context of the connection.

Since HANDLER locks and transactional locks have a different life
cycle -- HANDLERs are explicitly open and closed, and so
are HANDLER locks, explicitly acquired and released, whereas
transactional locks "accumulate" till the end of a transaction
and are released only with COMMIT, ROLLBACK and ROLLBACK TO SAVEPOINT,
a concept of "sentinel" was introduced to MDL_context.
All locks, HANDLER and others, reside in the same linked list.
However, a selected element of the list separates locks with
different life cycle. HANDLER locks always reside at the
end of the list, after the sentinel. Transactional locks are
prepended to the beginning of the list, before the sentinel.
Thus, ROLLBACK, COMMIT or ROLLBACK TO SAVEPOINT, only
release those locks that reside before the sentinel. HANDLER locks
must be released explicitly as part of HANDLER CLOSE statement,
or an implicit close. 
The same approach with sentinel
is also employed for LOCK TABLES locks. Since HANDLER and LOCK TABLES
statement has never worked together, the implementation is
made simple and only maintains one sentinel, which is used either
for HANDLER locks, or for LOCK TABLES locks.
2009-12-22 19:09:15 +03:00
Jon Olav Hauglid
f032795ccc Partial backport of:
------------------------------------------------------------
revno: 2617.14.26
committer: Vladislav Vaintroub <vvaintroub@mysql.com>
branch nick: mysql-6.0-wtf
timestamp: Wed 2008-11-05 11:19:19 +0100
message:
   CMakeLists.txt files cleanup.
  
  - remove SAFEMALLOC and SAFE_MUTEX definitions that were
  present in *each* CMakeLists.txt. Instead, put them into top level
  MakeLists.txt, but disable on Windows, because
  
  a) SAFEMALLOC does not add any functionality that is not already
  present in Debug C runtime ( and 2 safe malloc one on top of the other
  only unnecessarily slows down the server)
  
  b)SAFE_MUTEX does not work on Windows  and have been
  explicitely  disabled on Windows with #undef previously.  Fortunately,
  ntdll does  pretty good  job identifying l problems with  CRITICAL_SECTIONs.
  (DebugBreak()s on using uninited critical section, unlocking unowned
  critical section)
  
  -Remove occationally used -D_DEBUG (added by compiler
  anyway)
  
  -Remove MAP file generation, it became  obsolete .
  There are many ways to get callstack  of a crash now, with stacktrace in 
  error log , minidump etc
2009-12-17 16:40:02 +01:00
Jon Olav Hauglid
5ec9e5daab Bug #48724 Deadlock between INSERT DELAYED and FLUSH TABLES
If the handler (or delayed insert) thread failed to lock a table due
to being killed, the "dead" flag was used to notify the connection thread
of this failure. However, with the changes introduced by Bug#45949, 
the handler thread will no longer try to lock the table if it was killed.
This meant that the "dead" flag would not be set, and the connection
thread would not notice that the handler thread had failed.

This could happen with concurrent INSERT DELAYED and FLUSH TABLES.
FLUSH TABLES would kill any active INSERT DELAYED that had opened any
table(s) to be flushed. This could cause the INSERT DELAYED connection
thread to be stuck waiting for the handler thread to lock its table,
while the handler thread would be looping, trying to get the connection
thread to notice the error.

The root of the problem was that the handler thread had both the "dead"
flag and "thd->killed" to indicate that it had been killed. Most places
both were set, but some only set "thd->killed". And 
Delayed_insert::get_local_table() only checked "dead" while waiting for
the table to be locked.

This patch removes the "dead" variable and replaces its usage with
"thd->killed", thereby resolving the issue.
2009-12-17 13:43:07 +01:00
Jon Olav Hauglid
ddea504a7d Bug #48541 Deadlock between LOCK_open and LOCK_mdl
The reason for the deadlock was an improper exit from
MDL_context::wait_for_locks() which caused mysys_var->current_mutex to remain
LOCK_mdl even though LOCK_mdl was no longer held by that connection. 

This could for example lead to a deadlock in the following way:
1) INSERT DELAYED tries to open a table but fails, and trying to recover it
calls wait_for_locks().
2) Due to a pending exclusive request, wait_for_locks() fails and exits without
resetting mysys_var->current_mutex for the delayed insert handler thread. So it
continues to point to LOCK_mdl.
3) The handler thread manages to open a table.
4) A different connection takes LOCK_open and tries to take LOCK_mdl.
5) FLUSH TABLES from a third connection notices that the handler thread has a
table open, and tries to kill it. This involves locking mysys_var->current_mutex
while having LOCK_open locked. Since current_mutex mistakenly points to LOCK_mdl,
we have a deadlock.

This patch makes sure MDL_EXIT_COND() is called before exiting wait_for_locks().
This clears mysys->current_mutex which resolves the issue. 

An assert is added to recover_from_failed_open_table_attempt() after
wait_for_locks() is called, to check that current_mutex is indeed reset.
With this assert in place, existing tests in (e.g.) mdl_sync.test will fail
without this patch.
2009-12-16 12:32:11 +01:00
Konstantin Osipov
13348d3769 Merge next-mr -> next-4284. 2009-12-16 11:33:54 +03:00
Konstantin Osipov
58b24720b6 Merge next-mr -> next-4284. Null-merge
the fix for Bug#37148, since it is null-merged into 6.0.
2009-12-16 11:15:40 +03:00
Konstantin Osipov
a64e56ac93 Merge next-mr -> next-4284 2009-12-16 10:48:07 +03:00
Konstantin Osipov
e931ef415a Merge next-mr -> next-4284. 2009-12-15 22:59:07 +03:00
Konstantin Osipov
5e57d0916c Merge next-mr -> next-4284. 2009-12-15 22:03:56 +03:00
Konstantin Osipov
7a6e5c3f4c Merge next-mr -> next-4284. 2009-12-15 21:45:22 +03:00
Jon Olav Hauglid
3394cbf72c Bug #48940 MDL deadlocks against mysql_rm_db
This deadlock would occur between two connections A and B if statements
where executed in the following way:
1) Connection A executes a DML statement against table s1.t1 with
autocommit off. This causes a shared metadata lock on s1.t1 to be 
acquired. (With autocommit on, the metadata lock will be dropped once
the statment completes and the deadlock will not occour.)
2) Connection B tries to DROP DATABASE s1. This will block against the
metadata lock connection A holds on s1.t1. While blocking, connection B
will hold the LOCK_mysql_create_db mutex.
3) Connection A tries to ALTER DATABASE s1. This will block when trying
to get LOCK_mysql_create_db mutex held by connection B.
4) Deadlock between DROP DATABASE and ALTER DATABASE (which has autocommit
off).

If Connection A used an explicitly started transaction rather than having
autocommit off, this deadlock did not happen as ALTER DATABASE is 
disallowed inside transactions.

This patch fixes the problem by changing ALTER DATABASE to cause an
implicit commit before executing. This will cause the metadata 
lock on s1.t1 to be dropped, allowing DROP DATABASE to proceed. 
This will in turn cause the LOCK_mysql_create_db mutex to be unlocked, 
allowing ALTER DATABASE to proceed.

Note that SQL commands other than ALTER DATABASE that also use 
LOCK_mysql_create_db, already cause an implicit commit. 

Incompatible change: ALTER DATABASE (and its synonym ALTER SCHEMA)
now cause an implicit commit. This must be reflected in the 
documentation.

Test case added to schema.test.
2009-12-15 14:18:10 +01:00
Konstantin Osipov
700a361a6a Backport of:
------------------------------------------------------------
 2599.161.3 Ingo Struewing      2009-07-21
 Bug#20667 - Truncate table fails for a write locked table

 TRUNCATE TABLE was not allowed under LOCK TABLES.

 The patch removes this restriction. mysql_truncate()
 does now handle that case.
2009-12-11 15:24:23 +03:00
Konstantin Osipov
a3814e3635 Backport of:
-----------------------------------------------------------
2630.28.28 Magne Mahre  2008-12-05
Bug #38661 'all threads hang in "opening tables" or "waiting for table"
            and cpu is at 100%'
                      
Concurrent execution of FLUSH TABLES statement and at least two statements
using the same table might have led to live-lock which caused all three
connections to stall and hog 100% of CPU.
        
tdc_wait_for_old_versions() wrongly assumed that there cannot be a share
with an old version and no used TABLE instances and thus was failing to
perform wait in situation when such old share was cached in MDL subsystem
thanks to a still active metadata lock on the table. So it might have
happened that two or more connections simultaneously executing statements
which involve table being flushed managed to prevent each other from
waiting in this function by keeping shared metadata lock on the table 
constantly active (i.e. one of the statements managed to take/hold this
lock while other statements were calling tdc_wait_for_old_versions()).
Thus they were forcing each other to loop infinitely in open_tables() - 
close_thread_tables_for_reopen() - tdc_wait_for_old_versions() cycle
causing CPU hogging.
        
This patch fixes this problem by removing this false assumption from
tdc_wait_for_old_versions().
 
Note that the problem is specific only for server versions >= 6.0.
        
No test case is submitted for this test, as the test infrastructure
hasn't got the necessary primitives to test the behaviour.  The
manifestation is that throughput will decrease to a low level
(possibly 0) after some time, and stay at that level. Several
transactions will not complete. 
        
Manual testing can be done by running the code submitted by Shane 
Bester attached to the bug report.  If the bug persists, the 
transaction thruput will almost immediately drop to near zero 
(shown as the transaction count output from the test program staying 
on a close to constant value, instead of increasing rapidly).
2009-12-11 14:18:59 +03:00
Konstantin Osipov
30962f55c2 Partial backport of:
-----------------------------------------------------------
2497.392.1 Michael Widenius	2008-08-19
Fixes for Bug #38016 Maria: trying to access freed memory when
committing a transaction.
Don't write out states if they haven't changed.
2009-12-11 14:12:47 +03:00
Konstantin Osipov
efee99957c Partial backport of:
----------------------------------------------------
2736.2.10 Michael Widenius	2008-10-22
Fix for bug#39395 Maria: ma_extra.c:286: maria_extra: 
Assertion `share->reopen == 1' failed
2009-12-11 14:07:38 +03:00
Konstantin Osipov
5ecac3b4d0 Merge with next-4284. 2009-12-11 13:39:45 +03:00
Alexander Nozdrin
bd1f8f5bfa Auto-merge from mysql-trunk. 2009-12-11 09:20:49 +03:00
Alexander Nozdrin
2757eab544 Remove BitKeeper dir and .cvsignore. 2009-12-11 09:19:51 +03:00
Alexander Nozdrin
cfce3d0eb1 Auto-merge (empty) from mysql-next-mr. 2009-12-11 09:03:38 +03:00
Alexander Nozdrin
de06a06afa Auto-merge (empty) from mysql-trunk. 2009-12-11 08:56:28 +03:00
Alexander Nozdrin
bbea46f1f0 Auto-merge from mysql-next-mr-bugfixing. 2009-12-11 08:54:53 +03:00
Alexander Nozdrin
b8aace9485 Auto-merge from mysql-trunk-bugfixing. 2009-12-11 08:54:36 +03:00
Alexander Nozdrin
3a33af89d8 Auto-merge from mysql-next-mr-bugfixing. 2009-12-10 22:27:28 +03:00
Magne Mahre
351f28d0ce Bug#46374 crash, INSERT INTO t1 uses function, function modifies t1
An error occuring in the execution of a stored procedure, called
from do_select is masked, since the error condition is not
propagated back to the caller (join->conds->val_int() returns
a result value, and not an error code)
                  
An explicit check was added to see if the thd error code has been
set, and if so, the loop status is set to the error state.

Backport from 6.0-codebase (revid: 2617.68.31)
2009-12-10 16:22:41 +01:00
Magne Mahre
323f20eaef Bug#41425 Assertion in Protocol::end_statement() (pushbuild2)
(diagnostics_area)
      
Execution of CREATE TABLE ... SELECT statement was not atomic in
the sense that concurrent statements trying to affect its target
table might have sneaked in between the moment when the table was
created and moment when it was filled according to SELECT clause.
This resulted in inconsistent binary log, unexpected target table
contents. In cases when concurrent statement was a DDL statement
CREATE TABLE ... SELECT might have failed with ER_CANT_LOCK error.
      
In more detail:
Due to premature metadata lock downgrade which occured after CREATE
TABLE SELECT statement created table but before it managed to obtain
table-level lock on it other statements were allowed to open, lock
and change target table in the middle of CREATE TABLE SELECT
execution. This also meant that it was possible that CREATE TABLE
SELECT would wait in mysql_lock_tables() when it was called for newly
created table and that this wait could have been aborted by concurrent
DDL. The latter led to execution of unexpected branch of code and
CREATE TABLE SELECT ending with ER_CANT_LOCK error.
      
The premature downgrade occured because open_table(), which was called
for newly created table, decided that it is OK to downgrade metadata
lock from exclusive to shared since table exists, even although it
was not acquired within this call.
      
This fix ensures that open_table() does not downgrade metadata lock
if it is not acquired during its current invocation.
      
Testing:
The bug is exposed in a race condition, and is thus difficult to
expose in a standard mysql-test-run test case.  Instead, a stress
test using the Random Query Generator (https://launchpad.net/randgen)
will trip the problem occasionally.
      
   % perl  runall.pl \
            --basedir=<build dir> \
             --mysqld=--table-lock-wait-timeout=5 \
             --mysqld=--skip-safemalloc \
             --grammar=conf/maria_bulk_insert.yy \
             --reporters=ErrorLog,Backtrace,WinPackage \
             --mysqld=--log-output=file  \
             --queries=100000 \
             --threads=10 \
             --engine=myisam
      
Note: You will need a debug build to expose the bug
      
When the bug is tripped, the server will abort and dump core.


Backport from 6.0-codebase   (revid: 2617.53.4)
2009-12-10 15:49:15 +01:00
Alexander Nozdrin
28bc3aa791 Auto-merge (empty) from mysql-trunk-bugfixing. 2009-12-10 17:48:40 +03:00
Alexander Nozdrin
fcbbac8b37 Auto-merge (empty) from mysql-trunk. 2009-12-10 17:45:22 +03:00
Alexander Nozdrin
324e1a9dbc Auto-merge from mysql-next-mr. 2009-12-10 17:44:36 +03:00
Jon Olav Hauglid
d7f9583a9b Backport of revno: 3690
Postfix for Bug#48210 FLUSH TABLES WITH READ LOCK deadlocks
                      against concurrent CREATE PROCEDURE

Rewrote the second test to use DROP PROCEDURE instead of 
CREATE USER as CREATE USER does not work with embedded server.
2009-12-10 15:09:56 +01:00
Jon Olav Hauglid
8724320989 Backport of revno: 3685
Bug #48210 FLUSH TABLES WITH READ LOCK deadlocks
           against concurrent CREATE PROCEDURE

This deadlock occured between
a) CREATE PROCEDURE (or other commands listed below)
b) FLUSH TABLES WITH READ LOCK

If the execution of them happened in the following order:
- a) opens a table (e.g. mysql.proc)
- b) locks the global read lock (or GRL)
- a) sleeps inside wait_if_global_read_lock()
- b) increases refresh_version and sleeps waiting 
     for old tables to go away

Note that a) must start waiting on the GRL before FLUSH increases
refresh_version. Otherwise a) won't wait on the GRL and instead
close its tables for reopen, allowing FLUSH to complete and thus
avoid the deadlock.

With this patch the deadlock is avoided by making CREATE PROCEDURE
acquire a protection against global read locks before it starts
executing. This means that FLUSH TABLES WITH READ LOCK will have
to wait until CREATE PROCEDURE completes before acquiring the global
read lock, thereby avoiding the deadlock.

This is implemented by introducing a new SQL command flag called
CF_PROTECT_AGAINST_GRL. Commands marked with this flag will
acquire a GRL protection in the beginning of mysql_execute_command().
This patch adds the flag to CREATE, ALTER and DROP for PROCEDURE
and FUNCTION, as well as CREATE USER, DROP USER, RENAME USER and 
REVOKE ALL. All these commands either call open_grant_tables() or
open_system_table_for_updated() which make them susceptible for
this deadlock.

The patch also adds the CF_PROTECT_AGAINST_GRL flag to a number
of commands that previously acquired GRL protection in their
respective SQLCOM case in mysql_execute_command().

Test case that checks for GRL protection for CREATE PROCEDURE
and CREATE USER added to mdl_sync.test.
2009-12-10 15:09:00 +01:00
Konstantin Osipov
2264d619d8 Backport of:
2630.16.14 Sergei Golubchik	2008-08-25
 fixed a crash in partition tests
 introduced by HA_EXTRA_PREPARE_FOR_DROP patch
2009-12-10 16:43:04 +03:00
Jon Olav Hauglid
ff0001ed57 Backport of revno: 2617.80.1
Also re-enables the test for Bug #43867

Followup to Bug#46654 False deadlock on concurrent DML/DDL with partitions, 
                      inconsistent behavior

Partition_sync.test uses features only available in debug builds.
Disabling the test for non-debug builds.
2009-12-10 14:41:41 +01:00
Konstantin Osipov
e14d58f978 Backport a part of Monty's fix for Bug#39396, rev. 2736.2.11
"ha_maria.cc:2415: assertion in ha_maria::store_lock()".
2009-12-10 16:38:03 +03:00
Jon Olav Hauglid
2945f773f2 Backport of revno: 2617.68.37
Bug #46654 False deadlock on concurrent DML/DDL with partitions, 
           inconsistent behavior

The problem was that if one connection is running a multi-statement 
transaction which involves a single partitioned table, and another 
connection attempts to alter the table, the first connection gets 
ER_LOCK_DEADLOCK and cannot proceed anymore, even when the ALTER TABLE 
statement in another connection has timed out or failed.

The reason for this was that the prepare phase for ALTER TABLE for 
partitioned tables removed all instances of the table from the table 
definition cache before it started waiting on the lock. The transaction 
running in the first connection would notice this and report ER_LOCK_DEADLOCK. 

This patch changes the prep_alter_part_table() ALTER TABLE code so that 
tdc_remove_table() is no longer called. Instead, only the TABLE instance
changed by prep_alter_part_table() is marked as needing reopen.

The patch also removes an unnecessary call to tdc_remove_table() from 
mysql_unpack_partition() as the changed TABLE object is destroyed by the 
caller at a later point.

Test case added in partition_sync.test.
2009-12-10 14:26:00 +01:00
Jon Olav Hauglid
28b0eeff28 Backport of revno: 3514
Bug#40181 Made use of tdc_remove_table instead of just 
setting share->version to 0 to make sure all unused table
instances go away as part of CREATE/ALTER TABLE.
2009-12-10 14:15:50 +01:00
Jon Olav Hauglid
1cfcd2d210 Backport of revno: 3673
Bug #47313 assert in check_key_in_view during CALL procedure

View definitions are inlined in a stored procedure when the procedure
is fist called. This means that if a temporary table is later added
with the same name as the view, the stored procedure will still
use the view. This happens even if temporary tables normally shadow
base tables/views.

The reason for the assert was that even if the stored procedure
referenced the view, open_table() still tried to open the
temporary table. This "half view/half temporary table" state
caused the assert.

The bug was not present in 5.1 as open_table() is not called
for the view there. This code was changed with the introduction 
of MDL in order to properly lock the view and any objects it 
refers to.

This patch fixes the problem by instructing open_table()
to open base tables/views (using OT_BASE_ONLY) when reopening
tables/views used by stored procedures. This also means that
a prepared statement is no longer invalidated if a temporary
table is created with the same name as a view used in the
prepared statement.

Test case added to sp.test. The test case also demonstrates
the effect of sp cache invalidation between CALLs.
2009-12-10 13:37:18 +01:00
Jon Olav Hauglid
b6fb4dbab2 Backport of revno: 2617.68.45
Bug #47635 assert in start_waiting_global_read_lock during CREATE VIEW

The problem was that CREATE VIEW would trigger an assert if
a temporary table with the same name already existed.

This bug was fixed by the patch for Bug#47335. CREATE/ALTER VIEW
will now ignore temporary tables. See Bug#47335 for more information.

Test case added to view.test.
2009-12-10 13:15:20 +01:00
Jon Olav Hauglid
3173cf335b Backport of revno: 2617.68.43
Bug #47335 assert in get_table_share

The assert would happen if ALTER VIEW was used to alter a view (existing 
or non-existing) and a temporary table with the same name already existed.

The assert is triggered if the current statement does not have a MDL lock on 
the view to be altered. This would happen because open_table() would open 
the temporary table instead and MDL locks are not taken for temporary 
tables (since they are local to one connection).

The patch changes open_type for CREATE/ALTER VIEW to OT_BASE_ONLY. This prevents 
open_table() from trying to open a temporary table with the same name should
one exist. Now the view will be altered if it exists or ER_NO_SUCH_TABLE will
be reported if it does not.

Test case added to view.test
2009-12-10 13:02:37 +01:00