1. 03 Feb, 2010 1 commit
    • Dmitry Lenev's avatar
      Fix for bug #50784 "MDL: Assertion `m_tickets.is_empty() || · 0ede7125
      Dmitry Lenev authored
      m_tickets.front() == m_trans_sentinel'".
      
      Debug build of server crashed due to assert failure in MDL
      subsystem when one tried to execute multi-table REPAIR or
      OPTIMIZE in autocommit=0 mode.
      
      The assert failure occured when multi-table REPAIR or OPTIMIZE
      started processing of second table from its table list and
      tried to acquire upgradable metadata lock on this table.
      The cause of the assert failure were MDL locks left over from
      processing of previous table. It turned out that in autocommit=0
      mode close_thread_tables() which happens at the end of table
      processing doesn't release metadata locks.
      
      This fix solves problem by releasing locks explicitly using
      MDL_context::release_trans_locks() call.
      0ede7125
  2. 02 Feb, 2010 2 commits
  3. 01 Feb, 2010 6 commits
    • Konstantin Osipov's avatar
      Merge next-mr -> next-4284. · 2c6015e8
      Konstantin Osipov authored
      2c6015e8
    • Konstantin Osipov's avatar
    • Dmitry Lenev's avatar
      Fix for sporadical hangs of mdl_sync.test caused by patch · 5ff4406b
      Dmitry Lenev authored
      which implemented new type-of-operation-aware metadata
      locks and added a wait-for graph based deadlock detector
      to the MDL subsystem (this patch fixed bug #46272 "MySQL
      5.4.4, new MDL: unnecessary deadlock" and bug #37346
      "innodb does not detect deadlock between update and alter
      table").
      
      These hangs were caused by missing include of
      wait_condition.inc. This fix simply adds them.
      5ff4406b
    • Dmitry Lenev's avatar
      Fix for sporadical crashes of lock_multi_bug38499.test · 19940fa7
      Dmitry Lenev authored
      caused by patch which implemented new type-of-operation-aware
      metadata locks and added a wait-for graph based deadlock
      detector to the MDL subsystem (this patch fixed bug #46272
      "MySQL 5.4.4, new MDL: unnecessary deadlock" and bug #37346
      "innodb does not detect deadlock between update and alter
      table").
      
      Crashes were caused by a race in MDL_context::try_acquire_lock().
      This method added MDL_ticket to the list of granted tickets and
      released lock protecting list before setting MDL_ticket::m_lock.
      Thus some other thread was able to see ticket without properly
      set m_lock member for some short period of time. If this thread
      called method involving this member during this period crash
      happened.
      
      This fix ensures that MDL_ticket::m_lock is set in all cases
      when ticket is added to granted/pending lists in MDL_lock.
      19940fa7
    • Konstantin Osipov's avatar
      Fix a Windows compilation warning (req_count is later used · c5b48ab3
      Konstantin Osipov authored
      in a pointer arithmetics expression).
      c5b48ab3
    • Dmitry Lenev's avatar
      Implement new type-of-operation-aware metadata locks. · afd15c43
      Dmitry Lenev authored
      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.
      afd15c43
  4. 21 Jan, 2010 1 commit
    • Dmitry Lenev's avatar
      Patch that changes metadata locking subsystem to use mutex per lock and · a63f8480
      Dmitry Lenev authored
      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.
      a63f8480
  5. 20 Jan, 2010 1 commit
    • Jon Olav Hauglid's avatar
      Bug #50412 Assertion `! is_set()' failed in · c0051261
      Jon Olav Hauglid authored
                 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.
      c0051261
  6. 15 Jan, 2010 1 commit
    • Jon Olav Hauglid's avatar
      Bug #43685 Lock table affects other non-related tables · c2beb683
      Jon Olav Hauglid authored
      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.
      c2beb683
  7. 14 Jan, 2010 1 commit
    • Jon Olav Hauglid's avatar
      Partial backport of: · 5045ad38
      Jon Olav Hauglid authored
      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
      5045ad38
  8. 12 Jan, 2010 2 commits
    • Jon Olav Hauglid's avatar
      Bug #49988 MDL deadlocks with mysql_create_db, reload_acl_and_cache · db1888b5
      Jon Olav Hauglid authored
      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.
      db1888b5
    • Tor Didriksen's avatar
      Backport of · 6766c0d6
      Tor Didriksen authored
      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.
      6766c0d6
  9. 08 Jan, 2010 1 commit
    • Jon Olav Hauglid's avatar
      Fix for bug #48538 "Assertion in thr_lock() on LOAD DATA CONCURRENT · 0bce0c90
      Jon Olav Hauglid authored
                         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.
      0bce0c90
  10. 30 Dec, 2009 1 commit
    • Dmitry Lenev's avatar
      Implementation of simple deadlock detection for metadata locks. · 236539b4
      Dmitry Lenev authored
      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.
      236539b4
  11. 29 Dec, 2009 3 commits
    • Alexander Nozdrin's avatar
      Disable test case for Bug#49972. · cd6fbffc
      Alexander Nozdrin authored
      cd6fbffc
    • Alexander Nozdrin's avatar
    • Konstantin Osipov's avatar
      Apply and review: · 3b311f39
      Konstantin Osipov authored
      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.
      3b311f39
  12. 25 Dec, 2009 1 commit
  13. 24 Dec, 2009 4 commits
  14. 23 Dec, 2009 9 commits
  15. 22 Dec, 2009 6 commits
    • Vladislav Vaintroub's avatar
      merge · fd14e3dd
      Vladislav Vaintroub authored
      fd14e3dd
    • Alexey Kopytov's avatar
    • Vladislav Vaintroub's avatar
      Fix build error with CMake 2.8 (mysql_stmt_next_result not exported · 2c4015fb
      Vladislav Vaintroub authored
      by shared embedded library)
      2c4015fb
    • Alexey Kopytov's avatar
      Fixed a Windows build failure · ff4eeae1
      Alexey Kopytov authored
      ff4eeae1
    • Alexey Kopytov's avatar
      Backport of WL #2934: Make/find library for doing float/double · f02525be
      Alexey Kopytov authored
                            to string conversions and vice versa" 
      Initial import of the dtoa.c code and custom wrappers around it 
      to allow its usage from the server code. 
       
      Conversion of FLOAT/DOUBLE values to DECIMAL ones or strings 
      and vice versa has been significantly reworked. As the new 
      algoritms are more precise than the older ones, results of such 
      conversions may not always match those obtained from older 
      server versions. This in turn may break compatibility for some 
      applications. 
       
      This patch also fixes the following bugs: 
      - bug #12860 "Difference in zero padding of exponent between 
      Unix and Windows" 
      - bug #21497 "DOUBLE truncated to unusable value" 
      - bug #26788 "mysqld (debug) aborts when inserting specific 
      numbers into char fields" 
      - bug #24541 "Data truncated..." on decimal type columns 
      without any good reason" 
      f02525be
    • Konstantin Osipov's avatar
      A prerequisite patch for the fix for Bug#46224 · 39a1a50d
      Konstantin Osipov authored
      "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.
      39a1a50d