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.
mysql-test/r/ps_ddl.result:
Update results (Bug#30977).
mysql-test/r/ps_ddl1.result:
Update results (Bug#30977).
mysql-test/r/sp-error.result:
Update results (Bug#30977).
mysql-test/r/sp-lock.result:
Update results (Bug#30977).
mysql-test/suite/rpl/r/rpl_sp.result:
Update results (Bug#30977).
mysql-test/suite/rpl/t/rpl_sp.test:
Add a test case for Bug#30977.
mysql-test/t/ps_ddl.test:
Update comments. We no longer re-prepare a prepared statement
when a stored procedure used in top-level CALL is changed.
mysql-test/t/ps_ddl1.test:
Modifying stored procedure p1 no longer invalidates prepared
statement "call p1" -- we can re-use the prepared statement
without invalidation.
mysql-test/t/sp-error.test:
Use a constant for an error value.
mysql-test/t/sp-lock.test:
Add test coverage for Bug#30977.
sql/lock.cc:
Implement lock_routine_name() - a way to acquire an
exclusive metadata lock (ex- name-lock) on
stored procedure/function.
sql/sp.cc:
Change semantics of sp_cache_routine() -- now it has an option
to make sure that the routine that is cached is up to date (has
the latest sp cache version).
Add sp_cache_invalidate() to sp_drop_routine(), where it was
missing (a bug!).
Acquire metadata locks for SP DDL (ALTER/CREATE/DROP). This is
the core of the fix for Bug#30977.
Since caching and cache invalidation scheme was changed, make
sure we don't invalidate the SP cache in the middle of a stored
routine execution. At the same time, make sure we don't access
stale data due to lack of invalidation.
For that, change ALTER FUNCTION/PROCEDURE to not use the cache,
and SHOW PROCEDURE CODE/SHOW CREATE PROCEDURE/FUNCTION to always
read an up to date version of the routine from the cache.
sql/sp.h:
Add a helper wrapper around sp_cache_routine().
sql/sp_cache.cc:
Implement new sp_cache_version() and sp_cache_flush_obsolete().
Now we flush stale routines individually, rather than all at once.
sql/sp_cache.h:
Update signatures of sp_cache_version() and sp_cache_flush_obsolete().
sql/sp_head.cc:
Add a default initialization of sp_head::m_sp_cache_version.
Remove a redundant sp_head::create().
sql/sp_head.h:
Add m_sp_cache_version to sp_head class - we now
keep track of every routine in the stored procedure cache, rather than
of the entire cache.
sql/sql_base.cc:
Implement prelocking for stored routines. Validate stored
routines after they were locked.
Flush obsolete routines upon next access, one by one, not all at once
(Bug#41804).
Style fixes.
sql/sql_class.h:
Rename a Open_table_context method.
sql/sql_parse.cc:
Make sure stored procedures DDL commits the active transaction
(issues an implicit commit before and after).
Remove sp_head::create(), a pure redundancy.
Move the semantical check during alter routine inside sp_update_routine() code in order to:
- avoid using SP cache during update, it may be obsolete.
- speed up and simplify the update procedure.
Remove sp_cache_flush_obsolete() calls, we no longer flush the entire
cache, ever, stale routines are flushed before next use, one at a time.
sql/sql_prepare.cc:
Move routine metadata validation to open_and_process_routine().
Fix Bug#49972 (don't swap flags at reprepare).
Reset Sroutine_hash_entries in reinit_stmt_before_use().
Remove SP cache invalidation, it's now done by open_tables().
sql/sql_show.cc:
Fix a warning: remove an unused label.
sql/sql_table.cc:
Reset mdl_request.ticket for tickets acquired for routines inlined
through a view, in CHECK TABLE statement, to satisfy an MDL assert.
sql/sql_update.cc:
Move the cleanup of "translation items" to close_tables_for_reopen(),
since it's needed in all cases when we back off, not just
the back-off in multi-update. This fixes a bug when the server
would crash on attempt to back off when opening tables
for a statement that uses information_schema tables.
"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.
mysql-test/include/handler.inc:
Add test coverage for Bug#46224 "HANDLER statements within a
transaction might lead to deadlocks".
Extended HANDLER coverage to cover a mix of HANDLER, transactions
and DDL statements.
mysql-test/r/handler_innodb.result:
Update results (Bug#46224).
mysql-test/r/handler_myisam.result:
Update results (Bug#46224).
sql/lock.cc:
Remove thd->some_tables_deleted, it's never used.
sql/log_event.cc:
No need to check for thd->locked_tables_mode,
it's done inside release_transactional_locks().
sql/mdl.cc:
Implement the concept of HANDLER and LOCK TABLES "sentinel".
Implement a method to clone an acquired ticket.
Do not return tickets beyond the sentinel when acquiring
locks, create a copy.
Remove methods to merge and backup MDL_context, they are now
not used (Hurra!). This opens a path to a proper constructor
and destructor of class MDL_context (to be done in a separate
patch).
Modify find_ticket() to provide information about where
the ticket position is with regard to the sentinel.
sql/mdl.h:
Add declarations necessary for the implementation of the concept
of "sentinel", a dedicated ticket separating transactional and
non-transactional locks.
sql/mysql_priv.h:
Add mark_tmp_table_for_reuse() declaration,
a function to "close" a single session (temporary) table.
sql/sql_base.cc:
Remove thd->some_tables_deleted.
Modify deadlock-prevention asserts and deadlock detection
heuristics to take into account that from now on HANDLER locks
reside in the same locking context.
Add broadcast_refresh() to mysql_notify_thread_having_shared_lock():
this is necessary for the case when a thread having a shared lock
is asleep in tdc_wait_for_old_versions(). This situation is only
possible with HANDLER t1 OPEN; FLUSH TABLE (since all over code paths
that lead to tdc_wait_for_old_versions() always have an
empty MDL_context). Previously the server would simply deadlock
in this situation.
sql/sql_class.cc:
Remove now unused member "THD::some_tables_deleted".
Move mysql_ha_cleanup() a few lines above in THD::cleanup()
to make sure that all handlers are closed when it's time to
destroy the MDL_context of this connection.
Remove handler_mdl_context and handler_tables.
sql/sql_class.h:
Remove THD::handler_tables, THD::handler_mdl_context,
THD::some_tables_deleted.
sql/sql_handler.cc:
Remove thd->handler_tables.
Remove thd->handler_mdl_context.
Rewrite mysql_ha_open() to have no special provision for MERGE
tables, now that we don't have to manipulate with thd->handler_tables
it's easy to do.
Remove dead code.
Fix a bug in mysql_ha_flush() when we would always flush
a temporary HANDLER when mysql_ha_flush() is called (actually
mysql_ha_flush() never needs to flush temporary tables).
sql/sql_insert.cc:
Update a comment, no more thd->some_tables_deleted.
sql/sql_parse.cc:
Implement an incompatible change: entering LOCK TABLES closes
active HANDLERs, if any.
Now that we have a sentinel, we don't need to check
for thd->locked_tables_mode when releasing metadata locks in
COMMIT/ROLLBACK.
sql/sql_plist.h:
Add new (now necessary) methods to the list class.
sql/sql_prepare.cc:
Make sure we don't release HANDLER locks when rollback to a
savepoint, set to not keep locks taken at PREPARE.
sql/sql_servers.cc:
Update to a new signature of MDL_context::release_all_locks().
sql/sql_table.cc:
Remove thd->some_tables_deleted.
sql/transaction.cc:
Add comments.
Make sure rollback to (MDL) savepoint works under LOCK TABLES and
with HANDLER 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.
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.
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.
sql/sql_parse.cc:
Added CF_AUTO_COMMIT_TRANS to SQLCOM_ALTER_DB.
Removed thd->active_transaction() checks from SQLCOM_DROP_DB,
SQLCOM_ALTER_DB_UPGRADE and SQLCOM_ALTER_DB as these statements
cause an implicit commit.
------------------------------------------------------------
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.
mysql-test/r/merge.result:
Bug#20667 - Truncate table fails for a write locked table
Updated test result.
mysql-test/r/truncate.result:
Bug#20667 - Truncate table fails for a write locked table
Updated test result.
mysql-test/r/truncate_coverage.result:
Bug#20667 - Truncate table fails for a write locked table
New test result.
mysql-test/t/merge.test:
Bug#20667 - Truncate table fails for a write locked table
Updated test case due to now working TRUNCATE under LOCK TABLES.
Added some SELECTs to show that child tables are truncated.
mysql-test/t/truncate.test:
Bug#20667 - Truncate table fails for a write locked table
Added test cases for TRUNCATE under LOCK TABLE.
mysql-test/t/truncate_coverage.test:
Bug#20667 - Truncate table fails for a write locked table
New test file. Coverage tests for TRUNCATE.
sql/sql_delete.cc:
Bug#20667 - Truncate table fails for a write locked table
Added branches for thd->locked_tables_mode.
sql/sql_parse.cc:
Bug#20667 - Truncate table fails for a write locked table
Deleted rejection of TRUNCATE in case of LOCK TABLES.
-----------------------------------------------------------
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).
-----------------------------------------------------------
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.
sql/sql_table.cc:
Call extra(HA_EXTRA_PREPARE_FOR_RENAME) before renaming a table.
----------------------------------------------------
2736.2.10 Michael Widenius 2008-10-22
Fix for bug#39395 Maria: ma_extra.c:286: maria_extra:
Assertion `share->reopen == 1' failed
sql/sql_base.cc:
Race condition in wait_while_table_is_used() where a table used
by another connection could be forced closed, but there was no protection against the other thread re-opening the table and trying to lock it
again before the table was name locked by original thread.
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)
(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)
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.
2630.16.14 Sergei Golubchik 2008-08-25
fixed a crash in partition tests
introduced by HA_EXTRA_PREPARE_FOR_DROP patch
sql/sql_base.cc:
Don't call ::extra() for closed tables.
"ha_maria.cc:2415: assertion in ha_maria::store_lock()".
sql/lock.cc:
Fixed wrong cleanup of mysql_lock_tables()
- We must call read_lock_data() BEFORE we set
lock_count to 0. Added DBUG statements.
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.
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.
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.
mysql-test/t/ps_ddl.test:
Extended the VIEW->TEMPORARY TABLE transition test to cover not only
merged views, but now also materialized views and views containing
a reference to an information schema table.
Test also updated to reflect the change to prepared statement
invalidatation.
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
Bug#42546 Backup: RESTORE fails, thinking it finds an existing table
The problem occured when a MDL locking conflict happened for a non-existent
table between a CREATE and a INSERT statement. The code for CREATE
interpreted this lock conflict to mean that the table existed,
which meant that the statement failed when it should not have.
The problem could occur for CREATE TABLE, CREATE TABLE LIKE and
ALTER TABLE RENAME.
This patch fixes the problem for CREATE TABLE and CREATE TABLE LIKE.
It is based on code backported from the mysql-6.1-fk tree written
by Dmitry Lenev. CREATE now uses normal open_and_lock_tables() code
to acquire exclusive locks. This means that for the test case in the bug
description, CREATE will wait until INSERT completes so that it can
get the exclusive lock. This resolves the reported bug.
The patch also prohibits CREATE TABLE and CREATE TABLE LIKE under
LOCK TABLES. Note that this is an incompatible change and must
be reflected in the documentation. Affected test cases have been
updated.
mdl_sync.test contains tests for CREATE TABLE and CREATE TABLE LIKE.
Fixing the issue for ALTER TABLE RENAME is beyond the scope of this
patch. ALTER TABLE cannot be prohibited from working under LOCK TABLES
as this could seriously impact customers and a proper fix would require
a significant rewrite.
An assert in reload_acl_and_cache didn't account for the
case when the function is called with a NULL thd. A
null thd is used whenever the function is called from the
SIGHUP signal handler.
Backported from 6.0-codebase (revid: 2617.69.35)
------------------------------------------------------------
revno: 2617.68.25
committer: Dmitry Lenev <dlenev@mysql.com>
branch nick: mysql-next-bg-pre2-2
timestamp: Wed 2009-09-16 18:26:50 +0400
message:
Follow-up for one of pre-requisite patches for fixing bug #30977
"Concurrent statement using stored function and DROP FUNCTION
breaks SBR".
Made enum_mdl_namespace enum part of MDL_key class and removed MDL_
prefix from the names of enum members. In order to do the latter
changed name of PROCEDURE symbol to PROCEDURE_SYM (otherwise macro
which was automatically generated for this symbol conflicted with
MDL_key::PROCEDURE enum member).
------------------------------------------------------------
revno: 2617.68.24
committer: Dmitry Lenev <dlenev@mysql.com>
branch nick: mysql-next-bg-pre2-2
timestamp: Wed 2009-09-16 17:25:29 +0400
message:
Pre-requisite patch for fixing bug #30977 "Concurrent statement
using stored function and DROP FUNCTION breaks SBR".
Added MDL_request for stored routine as member to Sroutine_hash_entry
in order to be able perform metadata locking for stored routines in
future (Sroutine_hash_entry is an equivalent of TABLE_LIST class for
stored routines).
(WL#4284, follow up fixes).
sql/mdl.cc:
Introduced version of MDL_request::init() method which initializes
lock request using pre-built MDL key.
MDL_key::table_name/table_name_length() getters were
renamed to reflect the fact that MDL_key objects are
now created not only for tables.
sql/mdl.h:
Extended enum_mdl_namespace enum with values which correspond
to namespaces for stored functions and triggers.
Renamed MDL_key::table_name/table_name_length() getters
to MDL_key::name() and name_length() correspondingly to
reflect the fact that MDL_key objects are now created
not only for tables.
Added MDL_key::mdl_namespace() getter.
Also added version of MDL_request::init() method which
initializes lock request using pre-built MDL key.
sql/sp.cc:
Added MDL_request for stored routine as member to Sroutine_hash_entry.
Changed code to use MDL_key from this request as a key for LEX::sroutines
set. Removed separate "key" member from Sroutine_hash_entry as it became
unnecessary.
sql/sp.h:
Added MDL_request for stored routine as member to Sroutine_hash_entry
in order to be able perform metadata locking for stored routines in
future (Sroutine_hash_entry is an equivalent of TABLE_LIST class for
stored routines).
Removed Sroutine_hash_entry::key member as now we can use MDL_key from
this request as a key for LEX::sroutines set.
sql/sp_head.cc:
Removed sp_name::m_sroutines_key member and set_routine_type() method.
Since key for routine in LEX::sroutines set has no longer sp_name::m_qname
as suffix we won't save anything by creating it at sp_name construction
time.
Adjusted sp_name constructor used for creating temporary objects for
lookups in SP-cache to accept MDL_key as parameter and to avoid any
memory allocation.
Finally, removed sp_head::m_soutines_key member for reasons similar
to why sp_name::m_sroutines_key was removed
sql/sp_head.h:
Removed sp_name::m_sroutines_key member and set_routine_type() method.
Since key for routine in LEX::sroutines set has no longer sp_name::m_qname
as suffix we won't save anything by creating it at sp_name construction
time.
Adjusted sp_name constructor used for creating temporary objects for
lookups in SP-cache to accept MDL_key as parameter and to avoid any
memory allocation.
Finally, removed sp_head::m_soutines_key member for reasons similar
to why sp_name::m_sroutines_key was removed.
sql/sql_base.cc:
Adjusted code to the fact that we now use MDL_key from
Sroutine_hash_entry::mdl_request as a key for LEX::sroutines set.
MDL_key::table_name/table_name_length() getters were
renamed to reflect the fact that MDL_key objects are
now created not only for tables.
sql/sql_trigger.cc:
sp_add_used_routine() now takes MDL_key as parameter as now we use
instance of this class as a key for LEX::sroutines set.
revno: 2617.68.23
committer: Dmitry Lenev <dlenev@mysql.com>
branch nick: mysql-next-bg-pre1
timestamp: Wed 2009-09-16 09:34:42 +0400
message:
Pre-requisite patch for fixing bug #30977 "Concurrent statement
using stored function and DROP FUNCTION breaks SBR".
CREATE TABLE SELECT statements take exclusive metadata lock on table
being created. Invariant of metadata locking subsystem states that
such lock should be taken before taking any kind of shared locks.
Once metadata locks on stored routines are introduced statements like
"CREATE TABLE ... SELECT f1()" will break this invariant by taking
shared locks on routines before exclusive lock on target table.
To avoid this, open_tables() is reworked to process tables which are
directly used by the statement before stored routines are processed.
sql/sql_base.cc:
Refactored open_tables() implementation to process stored routines
only after tables which are directly used by statement were processed.
To achieve this moved handling of routines in open_tables() out of
loop which iterates over tables to a new separate loop. And in its
turn this allowed to split handling of particular table or view to
an auxiliary function, which made code in open_tables() simpler and
more easy to understand.
------------------------------------------------------------
revno: 2617.68.10
committer: Dmitry Lenev <dlenev@mysql.com>
branch nick: mysql-next-bg46673
timestamp: Tue 2009-09-01 19:57:05 +0400
message:
Fix for bug #46673 "Deadlock between FLUSH TABLES WITH READ LOCK and DML".
Deadlocks occured when one concurrently executed transactions with
several statements modifying data and FLUSH TABLES WITH READ LOCK
statement or SET READ_ONLY=1 statement.
These deadlocks were introduced by the patch for WL 4284: "Transactional
DDL locking"/Bug 989: "If DROP TABLE while there's an active transaction,
wrong binlog order" which has changed FLUSH TABLES WITH READ LOCK/SET
READ_ONLY=1 to wait for pending transactions.
What happened was that FLUSH TABLES WITH READ LOCK blocked all further
statements changing tables by setting global_read_lock global variable
and has started waiting for all pending transactions to complete.
Then one of those transactions tried to executed DML, detected that
global_read_lock non-zero and tried to wait until global read lock will
be released (i.e. global_read_lock becomes 0), indeed, this led to a
deadlock.
Proper solution for this problem should probably involve full integration
of global read lock with metadata locking subsystem (which will allow to
implement waiting for pending transactions without blocking DML in them).
But since it requires significant changes another, short-term solution
for the problem is implemented in this patch.
Basically, this patch restores behavior of FLUSH TABLES WITH READ LOCK/
SET READ_ONLY=1 before the patch for WL 4284/bug 989. By ensuring that
extra references to TABLE_SHARE are not stored for active metadata locks
it changes these statements not to wait for pending transactions.
As result deadlock is eliminated.
Note that this does not change the fact that active FLUSH TABLES WITH
READ LOCK lock or SET READ_ONLY=1 prevent modifications to tables as
they also block transaction commits.
mysql-test/r/flush_block_commit.result:
Adjusted test case after change in FLUSH TABLES WITH READ LOCK behavior
- it is no longer blocked by a pending transaction.
mysql-test/r/mdl_sync.result:
Added test for bug #46673 "Deadlock between FLUSH TABLES WITH READ LOCK
and DML".
mysql-test/r/read_only_innodb.result:
Adjusted test case after change in SET READ_ONLY behavior - it is no
longer blocked by a pending transaction.
mysql-test/t/flush_block_commit.test:
Adjusted test case after change in FLUSH TABLES WITH READ LOCK behavior
- it is no longer blocked by a pending transaction.
mysql-test/t/mdl_sync.test:
Added test for bug #46673 "Deadlock between FLUSH TABLES WITH READ LOCK
and DML".
mysql-test/t/read_only_innodb.test:
Adjusted test case after change in SET READ_ONLY behavior - it is no
longer blocked by a pending transaction.
sql/sql_base.cc:
Disable caching of pointers to TABLE_SHARE objects in MDL subsystem.
This means that transactions holding metadata lock on the table will
no longer have extra reference to the TABLE_SHARE (due to this lock)
and will no longer block concurrent FLUSH TABLES/FLUSH TABLES WITH
READ LOCK. Note that this does not change the fact that FLUSH TABLES
WITH READ LOCK prevents concurrent transactions from modifying data
as it also blocks all commits.
------------------------------------------------------------
revno: 2617.68.7
committer: Dmitry Lenev <dlenev@mysql.com>
branch nick: mysql-next-bg46044
timestamp: Thu 2009-08-27 10:22:17 +0400
message:
Fix for bug #46044 "MDL deadlock on LOCK TABLE + CREATE TABLE HIGH_PRIORITY
FOR UPDATE".
Deadlock occured when during execution of query to I_S we tried to open
a table or its .FRM in order to get information about it and had to wait
because we have encountered exclusive metadata lock on this table held by
a DDL operation from another connection which in its turn waited for some
resource currently owned by connection executing this I_S query.
For example, this might have happened if one under LOCK TABLES executed I_S
query targeted to particular table (which was not among locked) and also
concurrently tried to create this table using CREATE TABLE SELECT which
had to wait for one of tables locked by the first connection.
Another situation in which deadlock might have occured is when I_S query,
which was executed as part of transaction, tried to get information about
table which just has been dropped by concurrent DROP TABLES executed under
LOCK TABLES and this DROP TABLES for its completion also had to wait
transaction from the first connection.
This problem stemmed from the fact that opening of tables/.FRMs for I_S
filling is happening outside of connection's main MDL_context so code
which tries to detect deadlocks due to conflicting metadata locks doesn't
work in this case. Indeed, this led to deadlocks when during I_S filling
we tried to wait for conflicting metadata lock to go away, while its owner
was waiting for some resource held by connection executing I_S query.
This patch solves this problem by avoiding waiting in such situation.
Instead we skip this table and produce warning that information about
it was omitted from I_S due to concurrent DDL operation. We still wait
for conflicting metadata lock to go away when it is known that deadlock
is not possible (i.e. when connection executing I_S query does not hold
any metadata or table-level locks).
Basically, we apply our standard deadlock avoidance technique for metadata
locks to the process of filling of I_S tables but replace ER_LOCK_DEADLOCK
error with a warning.
Note that this change is supposed to be safe for 'mysqldump' since the
only its mode which is affected by this change is --single-transaction mode
is not safe in the presence of concurrent DDL anyway (and this fact is
documented). Other modes are unaffected because they either use
SHOW TABLES/SELECT * FROM I_S.TABLE_NAMES which do not take any metadata
locks in the process of I_S table filling and thus cannot skip tables or
execute I_S queries for tables which were previously locked by LOCK TABLES
(or in the presence of global read lock) which excludes possibility of
encountering conflicting metadata lock.
mysql-test/r/mdl_sync.result:
Added test for bug #46044 "MDL deadlock on LOCK TABLE + CREATE TABLE
HIGH_PRIORITY FOR UPDATE".
mysql-test/t/mdl_sync.test:
Added test for bug #46044 "MDL deadlock on LOCK TABLE + CREATE TABLE
HIGH_PRIORITY FOR UPDATE".
sql/mysql_priv.h:
Added a new flag for open_table() call which allows it to fail
with an error in cases when conflicting metadata lock is discovered
instead of waiting until this lock goes away.
sql/share/errmsg-utf8.txt:
Added error/warning message to be generated in cases when information
about table is omitted from I_S since there is conflicting metadata lock
on the table.
sql/share/errmsg.txt:
Added error/warning message to be generated in cases when information
about table is omitted from I_S since there is conflicting metadata lock
on the table.
sql/sql_base.cc:
Added a new flag for open_table() call which allows it to fail
with an error in cases when conflicting metadata lock is discovered
instead of waiting until this lock goes away.
sql/sql_show.cc:
When we are opening a table (or just .FRM) in order to fill I_S with
information about this table and encounter conflicting metadata lock
waiting for this lock to go away can lead to a deadlock in some
situations (under LOCK TABLES, within transaction, etc.). To avoid
these deadlocks we detect such situations and don't do waiting.
Instead, we skip table for which we have conflicting metadata lock,
thus omitting information about it from I_S table, and produce an
appropriate warning.
Introduce a counter for protection against global read lock on thread level.
The functions for protection against global read lock sometimes need a local
variable to signal when the protection is set, and hence need to be released.
It would be better to control this behaviour via a counter on the THD struct,
telling how many times the protection has been claimed by the current thread.
A side-effect of the fix is that if protection is claimed twice for a thread,
only a simple increment is required for the second claim, instead of a
mutex-protected increment of the global variable protect_against_global_read_lock.
sql/lock.cc:
Count how many times that we have claimed protection against global read lock.
Assert that we really have the protection when releasing it.
Added comments to all functions operating on global_read_lock.
sql/sql_class.cc:
Added the counter variable global_read_lock_protection.
sql/sql_class.h:
Added the counter variable global_read_lock_protection.
sql/sql_parse.cc:
Replaced test on local variable need_start_waiting with test on
thd->global_read_lock_protection.
sql/sql_table.cc:
Replaced test on local variable need_start_waiting with test on
thd->global_read_lock_protection.
sql/sql_trigger.cc:
Inserted test on thd->global_read_lock_protection.
---------------------------------------------
This is a patch for bug#47098 assert in MDL_context::destroy on
HANDLER <damaged merge table> OPEN.
The assert occurs in MDL_context::destroy when the connection is terminated,
because all mdl_tickets have not been released.
MERGE tables do not support being opened using the HANDLER ... OPEN command,
and trying to do so will result in an error. In the event of an error, all
tables that are opened, should be closed again. The fix for bug#45781 made
sure that this also works for MERGE tables, which causes multiple tables to
be opened.
This fix extends the fix for bug#45781, by ensuring that also all locks are
released, when MERGE tables are involved.
mysql-test/r/merge.result:
The result of the test.
mysql-test/t/merge.test:
Added a test based on the bug report, as well as a test of the more general
scenario.
sql/sql_handler.cc:
Added code to release all the locks.
Bug #21793 Missing CF_CHANGES_DATA and CF_STATUS_COMMAND for
handful of commands
CF_CHANGES_DATA and CF_STATUS_COMMAND flags added to the
commands mentioned in the bug description. With the following
two exceptions:
1) 4 commands do not exist:
SQLCOM_RENAME_DB
SQLCOM_LOAD_MASTER_DATA
SQLCOM_LOAD_MASTER_TABLE
SQLCOM_SHOW_COLUMN_TYPES
2) All SQLCOM_SHOW_* commands already had CF_STATUS_COMMAND,
leaving only SQLCOM_BINLOG_BASE64_EVENT.
Further, check_prepared_statement() in sql_prepare.cc has been
simplified by taking advantage of the CF_STATUS_COMMAND flag.
Note that no test case has been added.
Bug #48248 assert in MDL_ticket::upgrade_shared_lock_to_exclusive
The assert would happen if REPAIR TABLE was used on a table already
locked by LOCK TABLES READ. REPAIR mistakenly tried to upgrade the
read-lock to exclusive, thereby triggering the assert.
The cause of the problem was that REPAIR TABLE ignored errors
from opening and locking tables. This is by design, as REPAIR
can be used to broken tables that cannot be opened. However,
repair also ignored logical errors such as the inability to
exclusivly lock a table due to conflicting LOCK TABLES.
This patch fixes the problem by not ignoring errors from
opening and locking tables if inside LOCK TABLES mode.
In LOCK TABLES we already know that the table can be opened,
so that the failure to open must be a logical error.
Test added to repair.test.
Bug #47107 assert in notify_shared_lock on incorrect CREATE TABLE , HANDLER
Attempts to create a table (using CREATE TABLE, CREATE TABLE LIKE or
CREATE TABLE SELECT statements) which already existed and was opened
by the same connection through HANDLER statement, led to a stalled
connection (for production builds of the server) or to the server being
aborted due to an assertion failure (for debug builds of the server).
This problem was introduced by the new implementation of a metadata
locking subsystem and didn't affect earlier versions of the server.
The cause of the problem was that the HANDLER was not closed by CREATE TABLE
before CREATE tried to open and lock the table. Acquiring an exclusive MDL
lock on the table to be created would therefore fail since HANDLER
already had a shared MDL lock. This triggered an assert as the
HANDLER and CREATE statements came from the same thread (self-deadlock).
This patch resolves the issue by closing any open HANDLERs on tables
to be created by CREATE TABLE, similar to what is already done for
DROP and ALTER TABLE.
Test case added to create.test.
Bug #48725 Assert !thd->is_error() in delayed_get_table()
This bug is a regression introduced by the patch for Bug #45949.
If the handler thread for INSERT DELAYED was killed by e.g.
FLUSH TABLES, the error message is copied from the handler thread
to the connection thread. But the error was not reacted on, so the
connection thread continued as normal, leading to an eventual assert.
No test case added as it would have required sync points to work
for handler threads. The plan is to add this in the scope of
Bug #48725 / Bug #48541. The patch has been tested with the
non-deterministic test case given in the bug description.
Bug #45949 Assertion `!tables->table' in open_tables() on
ALTER + INSERT DELAYED
The assertion was caused by improperly closing tables when
INSERT DELAYED needed to reopen tables. This patch replaces
the call to close_thread_tables with close_tables_for_reopen
which fixes the problem.
The only way I was able to trigger the reopen code path and
thus the assertion, was if ALTER TABLE killed the delayed
insert thread and the delayed insert thread was able to enter
the reopen code path before it noticed that thd->killed had
been set. Note that in these cases reopen will always fail
since open_table() will check thd->killed and return. This patch
therefore adds two more thd->killed checks to minimize the
chance of entering the reopen code path without hope for success.
The patch also changes it so that if the delayed insert is killed
using KILL_CONNECTION, the error message that is copied to the
connection thread is ER_QUERY_INTERRUPTED rather than
ER_SERVER_SHUTDOWN. This means that if INSERT DELAYED fails,
the user will now see "Query execution was interrupted" rather
than the misleading "Server shutdown in progress".
No test case is supplied. This is for two reasons:
1) Unable to reproduce the error without having the delayed insert
thread in a killed state which means that reopen is futile and
was not supposed to be attempted.
2) Difficulty of using sync points in other threads than
the connection thread.
The patch has been successfully tested with the RQG and the grammar
supplied in the bug description.
------------------------------------------------------------
revno: 2617.69.37
committer: Dmitry Lenev <dlenev@mysql.com>
branch nick: mysql-next-bg46748
timestamp: Fri 2009-08-21 18:17:02 +0400
message:
Fix for bug #46748 "Assertion in MDL_context::wait_for_locks()
on INSERT + CREATE TRIGGER".
Concurrent execution of statements involving stored functions or triggers
which were using several tables and DDL statements which affected those
tables on debug build of server might have led to assertion failures in
MDL_context::wait_for_locks(). Non-debug build was not affected.
The problem was that during back-off which happens when open_tables()
encounters conflicting metadata lock for one of the tables being open
we didn't reset MDL_request::ticket value for requests which correspond
to tables from extended prelocking set. Since these requests are part
of of list of requests to be waited for in Open_table_context this broke
assumption that ticket value for them is 0 in MDL_context::wait_for_locks()
and caused assertion failure.
This fix ensures that close_tables_for_reopen(), which performs this back-off
resets MDL_request::ticket value not only for tables directly used by the
statement but also for tables from extended prelocking set, thus satisfying
assumption described above.
mysql-test/r/mdl_sync.result:
Added test case for bug #46748 "Assertion in MDL_context::wait_for_locks()
on INSERT + CREATE TRIGGER".
mysql-test/t/mdl_sync.test:
Added test case for bug #46748 "Assertion in MDL_context::wait_for_locks()
on INSERT + CREATE TRIGGER".
sql/sql_base.cc:
Since metadata lock requests for tables from extended part of prelocking
set are also part of list of requests to be waited for in Open_table_context
in close_tables_for_reopen() we have to reset MDL_request::ticket
values for them to assumptions in MDL_context::wait_for_locks().
Bug #47249 assert in MDL_global_lock::is_lock_type_compatible
This assert could be triggered if LOCK TABLES were used to lock
both a table and a view that used the same table. The table would have
to be first WRITE locked and then READ locked. So "LOCK TABLES v1
WRITE, t1 READ" would eventually trigger the assert, "LOCK TABLES
v1 READ, t1 WRITE" would not. The reason is that the ordering of locks
in the interal representation made a difference when executing
FLUSH TABLE on the table.
During FLUSH TABLE, a lock was upgraded to exclusive. If this lock
was of type MDL_SHARED and not MDL_SHARED_UPGRADABLE, an internal
counter in the MDL subsystem would get out of sync. This would happen
if the *last* mention of the table in LOCK TABLES was a READ lock.
The counter in question is the number exclusive locks (active or intention).
This is used to make sure a global metadata lock is only taken when the
counter is zero (= no conflicts). The counter is increased when a
MDL_EXCLUSIVE or MDL_SHARED_UPGRADABLE lock is taken, but not when
upgrade_shared_lock_to_exclusive() is used to upgrade directly
from MDL_SHARED to MDL_EXCLUSIVE.
This patch fixes the problem by searching for a TABLE instance locked
with MDL_SHARED_UPGRADABLE or MDL_EXCLUSIVE before calling
upgrade_shared_lock_to_exclusive(). The patch also adds an assert checking
that only MDL_SHARED_UPGRADABLE locks are upgraded to exclusive.
Test case added to lock_multi.test.
------------------------------------------------------------
revno: 2617.69.32
committer: Dmitry Lenev <dlenev@mysql.com>
branch nick: mysql-next-bg46747
timestamp: Wed 2009-08-19 18:12:27 +0400
message:
Fix for bug #46747 "Crash in MDL_ticket::upgrade_shared_lock_to_exclusive
on TRIGGER + TEMP table".
Server crashed when one tried to drop trigger which had its subject
table shadowed by a temporary table with the same name.
This problem occured because in such situation DROP TRIGGER has opened
temporary table instead of base table on which trigger was defined.
Attempt to upgrade metadata lock on this temporary table led to crash
(we don't acquire metadata locks for temporary tables).
This fix ensures that DROP TRIGGER ignores temporary tables when
trying to open table on which trigger to be dropped is defined.
mysql-test/r/trigger.result:
Added test case for bug #46747 "Crash in
MDL_ticket::upgrade_shared_lock_to_exclusive
on TRIGGER + TEMP table".
mysql-test/t/trigger.test:
Added test case for bug #46747 "Crash in
MDL_ticket::upgrade_shared_lock_to_exclusive
on TRIGGER + TEMP table".
sql/sql_trigger.cc:
Prevent DROP TRIGGER from opening temporary table which might
shadow base table on which trigger to be dropped is defined.
----------------------------------------------------------
revno: 2617.69.28
committer: Konstantin Osipov <kostja@sun.com>
branch nick: 5.4-azalea-bugfixing
timestamp: Tue 2009-08-18 15:27:35 +0400
message:
An attempt to fix a link failure on Windows -- Sroutine_hash_entry
is forward-declared as class.
(Part of WL#4284).
----------------------------------------------------------
revno: 2617.69.25
committer: Konstantin Osipov <kostja@sun.com>
branch nick: 5.4-42546
timestamp: Fri 2009-08-14 23:52:00 +0400
message:
A cleanup in open_tables() and lock_tables():
change return type of these functions to bool from int,
to follow convention in the rest of the code.
(Part of WL#4284 review fixes).
sql/mysql_priv.h:
Change return type of open_talbes() and lock_tables() from
int to bool.
sql/sql_base.cc:
Change return type of open_tables() and lock_tables() from int to bool.
sql/sql_handler.cc:
Use bool now that open_tables() returns bool.
A pre-requisite patch for Bug#30977 "Concurrent statement using
stored function and DROP FUNCTION breaks SBR".
This patch changes the MDL API by introducing a namespace for
lock keys: MDL_TABLE for tables and views and MDL_PROCEDURE
for stored procedures and functions. The latter is needed for
the fix for Bug#30977.
Bug #42074 concurrent optimize table and
alter table = Assertion failed: thd->is_error()
This assertion could occur if OPTIMIZE TABLE was started on a InnoDB table
and the table was altered to different storage engine after OPTIMIZE
had started. This allowed OPTIMIZE to pass the initial checks for
storage engine support, but fail once it reached "recreate+analyze"
if this operation was not supported by the new storage engine.
The bug had no consequences for non-debug builds of the server.
In detail, the assertion was triggered when ha_analyze() returned
HA_ADMIN_NOT_IMPLEMENTED. This led to a code path which included an
assert checking for diagnostics area contents. Since this area had
not been filled, the assertion was triggered. The diagnostics area
is in this case only used to provide more detailed information about
why optimize failed. The triggered code path sends this information
to the client and clears the diagnostic area.
This patch fixed the problem by adding an error message to the diagnostic
area if ha_analyze() fails. This error message contains the error code
returned by ha_analyze().
Test case added to innodb_mysql_sync.test.