diff options
Diffstat (limited to 'sql/sql_base.cc')
-rw-r--r-- | sql/sql_base.cc | 385 |
1 files changed, 206 insertions, 179 deletions
diff --git a/sql/sql_base.cc b/sql/sql_base.cc index 2f891375163..8dced67dd44 100644 --- a/sql/sql_base.cc +++ b/sql/sql_base.cc @@ -131,8 +131,6 @@ static bool tdc_wait_for_old_versions(THD *thd, static bool has_write_table_with_auto_increment(TABLE_LIST *tables); -TABLE *find_table_for_mdl_upgrade(TABLE *list, const char *db, - const char *table_name); uint cached_open_tables(void) { @@ -477,8 +475,10 @@ TABLE_SHARE *get_table_share(THD *thd, TABLE_LIST *table_list, char *key, To be able perform any operation on table we should own some kind of metadata lock on it. */ - DBUG_ASSERT(thd->mdl_context.is_lock_owner(MDL_key::TABLE, table_list->db, - table_list->table_name)); + DBUG_ASSERT(thd->mdl_context.is_lock_owner(MDL_key::TABLE, + table_list->db, + table_list->table_name, + MDL_SHARED)); /* Read table definition from cache */ if ((share= (TABLE_SHARE*) my_hash_search(&table_def_cache,(uchar*) key, @@ -1003,7 +1003,7 @@ bool close_cached_tables(THD *thd, TABLE_LIST *tables, bool have_lock, { /* A check that the table was locked for write is done by the caller. */ TABLE *table= find_table_for_mdl_upgrade(thd->open_tables, table_list->db, - table_list->table_name); + table_list->table_name, TRUE); /* May return NULL if this table has already been closed via an alias. */ if (! table) @@ -1084,7 +1084,7 @@ err_with_reopen: than picking only those tables that were flushed. */ for (TABLE *tab= thd->open_tables; tab; tab= tab->next) - tab->mdl_ticket->downgrade_exclusive_lock(); + tab->mdl_ticket->downgrade_exclusive_lock(MDL_SHARED_NO_READ_WRITE); } DBUG_RETURN(result); } @@ -2145,7 +2145,7 @@ bool wait_while_table_is_used(THD *thd, TABLE *table, table->s->table_name.str, (ulong) table->s, table->db_stat, table->s->version)); - if (table->mdl_ticket->upgrade_shared_lock_to_exclusive()) + if (thd->mdl_context.upgrade_shared_lock_to_exclusive(table->mdl_ticket)) DBUG_RETURN(TRUE); pthread_mutex_lock(&LOCK_open); @@ -2336,6 +2336,7 @@ open_table_get_mdl_lock(THD *thd, TABLE_LIST *table_list, { if (table_list->lock_strategy) { + MDL_request_list mdl_requests; MDL_request *global_request; /* In case of CREATE TABLE .. If NOT EXISTS .. SELECT, the table @@ -2350,67 +2351,51 @@ open_table_get_mdl_lock(THD *thd, TABLE_LIST *table_list, mdl_request->set_type(MDL_EXCLUSIVE); DBUG_ASSERT(! thd->mdl_context.has_locks() || thd->handler_tables_hash.records || - thd->global_read_lock); + thd->global_read_lock.is_acquired()); if (!(global_request= ot_ctx->get_global_mdl_request(thd))) return 1; - if (! global_request->ticket) - { - ot_ctx->add_request(global_request); - if (thd->mdl_context.acquire_global_intention_exclusive_lock( - global_request)) - return 1; - } + mdl_requests.push_front(mdl_request); + mdl_requests.push_front(global_request); - ot_ctx->add_request(mdl_request); - if (thd->mdl_context.acquire_exclusive_lock(mdl_request)) + if (thd->mdl_context.acquire_locks(&mdl_requests)) return 1; } else { - /* - There is no MDL_SHARED_UPGRADABLE_HIGH_PRIO type of metadata lock so we - want to be sure that caller doesn't pass us both flags simultaneously. - */ - DBUG_ASSERT(!(flags & MYSQL_OPEN_TAKE_UPGRADABLE_MDL) || - !(flags & MYSQL_LOCK_IGNORE_FLUSH)); - - if (flags & MYSQL_OPEN_TAKE_UPGRADABLE_MDL && - table_list->lock_type >= TL_WRITE_ALLOW_WRITE) - mdl_request->set_type(MDL_SHARED_UPGRADABLE); - if (flags & MYSQL_LOCK_IGNORE_FLUSH) - mdl_request->set_type(MDL_SHARED_HIGH_PRIO); - - if (mdl_request->type == MDL_SHARED_UPGRADABLE) + if (flags & MYSQL_OPEN_FORCE_SHARED_MDL) { - MDL_request *global_request; + /* + While executing PREPARE for prepared statement we override + type-of-operation aware type of shared metadata lock which + was set in the parser with simple shared metadata lock. + This is necessary to allow concurrent execution of PREPARE + and LOCK TABLES WRITE statement which locks one of the tables + used in the statement being prepared. + */ + DBUG_ASSERT(!(flags & (MYSQL_OPEN_TAKE_UPGRADABLE_MDL | + MYSQL_LOCK_IGNORE_FLUSH))); - if (!(global_request= ot_ctx->get_global_mdl_request(thd))) - return 1; - if (! global_request->ticket) - { - ot_ctx->add_request(global_request); - if (thd->mdl_context.try_acquire_global_intention_exclusive_lock( - global_request)) - return 1; - if (! global_request->ticket) - goto failure; - } + mdl_request->set_type(MDL_SHARED); + } + else if (flags & MYSQL_LOCK_IGNORE_FLUSH) + { + DBUG_ASSERT(!(flags & MYSQL_OPEN_TAKE_UPGRADABLE_MDL)); + mdl_request->set_type(MDL_SHARED_HIGH_PRIO); } ot_ctx->add_request(mdl_request); - if (thd->mdl_context.try_acquire_shared_lock(mdl_request)) + if (thd->mdl_context.try_acquire_lock(mdl_request)) return 1; -failure: if (mdl_request->ticket == NULL) { if (flags & MYSQL_OPEN_FAIL_ON_MDL_CONFLICT) my_error(ER_WARN_I_S_SKIPPED_TABLE, MYF(0), table_list->db, table_list->table_name); else - (void) ot_ctx->request_backoff_action(Open_table_context::OT_WAIT); + ot_ctx->request_backoff_action(Open_table_context::OT_WAIT_MDL_LOCK); return 1; } } @@ -2563,6 +2548,17 @@ bool open_table(THD *thd, TABLE_LIST *table_list, MEM_ROOT *mem_root, { int distance= ((int) table->reginfo.lock_type - (int) table_list->lock_type); + + /* + If we are performing DDL operation we also should ensure + that we will find TABLE instance with upgradable metadata + lock, + */ + if ((flags & MYSQL_OPEN_TAKE_UPGRADABLE_MDL) && + table_list->lock_type >= TL_WRITE_ALLOW_WRITE && + ! table->mdl_ticket->is_upgradable_or_exclusive()) + distance= -1; + /* Find a table that either has the exact lock type requested, or has the best suitable lock. In case there is no locked @@ -2596,6 +2592,13 @@ bool open_table(THD *thd, TABLE_LIST *table_list, MEM_ROOT *mem_root, } if (best_table) { + if ((flags & MYSQL_OPEN_TAKE_UPGRADABLE_MDL) && + table_list->lock_type >= TL_WRITE_ALLOW_WRITE && + ! best_table->mdl_ticket->is_upgradable_or_exclusive()) + { + my_error(ER_TABLE_NOT_LOCKED_FOR_WRITE, MYF(0), alias); + DBUG_RETURN(TRUE); + } table= best_table; table->query_id= thd->query_id; DBUG_PRINT("info",("Using locked table")); @@ -2610,8 +2613,10 @@ bool open_table(THD *thd, TABLE_LIST *table_list, MEM_ROOT *mem_root, TABLES breaks metadata locking protocol (potentially can lead to deadlocks) it should be disallowed. */ - if (thd->mdl_context.is_lock_owner(MDL_key::TABLE, table_list->db, - table_list->table_name)) + if (thd->mdl_context.is_lock_owner(MDL_key::TABLE, + table_list->db, + table_list->table_name, + MDL_SHARED)) { char path[FN_REFLEN + 1]; enum legacy_db_type not_used; @@ -2688,7 +2693,7 @@ bool open_table(THD *thd, TABLE_LIST *table_list, MEM_ROOT *mem_root, { /* Someone did a refresh while thread was opening tables */ pthread_mutex_unlock(&LOCK_open); - (void) ot_ctx->request_backoff_action(Open_table_context::OT_WAIT); + (void) ot_ctx->request_backoff_action(Open_table_context::OT_WAIT_TDC); DBUG_RETURN(TRUE); } @@ -2828,7 +2833,7 @@ bool open_table(THD *thd, TABLE_LIST *table_list, MEM_ROOT *mem_root, */ release_table_share(share); pthread_mutex_unlock(&LOCK_open); - (void) ot_ctx->request_backoff_action(Open_table_context::OT_WAIT); + (void) ot_ctx->request_backoff_action(Open_table_context::OT_WAIT_TDC); DBUG_RETURN(TRUE); } /* Force close at once after usage */ @@ -2895,11 +2900,11 @@ bool open_table(THD *thd, TABLE_LIST *table_list, MEM_ROOT *mem_root, /* In CREATE TABLE .. If NOT EXISTS .. SELECT we have found that table exists now we should downgrade our exclusive metadata - lock on this table to shared metadata lock. + lock on this table to SW metadata lock. */ if (table_list->lock_strategy == TABLE_LIST::EXCLUSIVE_DOWNGRADABLE_MDL && !(flags & MYSQL_OPEN_HAS_MDL_LOCK)) - mdl_ticket->downgrade_exclusive_lock(); + mdl_ticket->downgrade_exclusive_lock(MDL_SHARED_WRITE); table->mdl_ticket= mdl_ticket; @@ -2982,33 +2987,43 @@ TABLE *find_locked_table(TABLE *list, const char *db, const char *table_name) /** - Find write locked instance of table in the list of open tables, - emit error if no such instance found. + Find instance of TABLE with upgradable or exclusive metadata + lock from the list of open tables, emit error if no such table + found. - @param thd List of TABLE objects to be searched + @param list List of TABLE objects to be searched @param db Database name. @param table_name Name of table. + @param no_error Don't emit error if no suitable TABLE + instance were found. - @return Pointer to write-locked TABLE instance, 0 - otherwise. + @return Pointer to TABLE instance with MDL_SHARED_NO_WRITE, + MDL_SHARED_NO_READ_WRITE, or MDL_EXCLUSIVE metadata + lock, NULL otherwise. */ -TABLE *find_write_locked_table(TABLE *list, const char *db, const char *table_name) +TABLE *find_table_for_mdl_upgrade(TABLE *list, const char *db, + const char *table_name, + bool no_error) { TABLE *tab= find_locked_table(list, db, table_name); if (!tab) { - my_error(ER_TABLE_NOT_LOCKED, MYF(0), table_name); - return 0; + if (!no_error) + my_error(ER_TABLE_NOT_LOCKED, MYF(0), table_name); + return NULL; } else { - while (tab->reginfo.lock_type < TL_WRITE_LOW_PRIORITY && + while (tab->mdl_ticket != NULL && + !tab->mdl_ticket->is_upgradable_or_exclusive() && (tab= find_locked_table(tab->next, db, table_name))) continue; if (!tab) { - my_error(ER_TABLE_NOT_LOCKED_FOR_WRITE, MYF(0), table_name); + if (!no_error) + my_error(ER_TABLE_NOT_LOCKED_FOR_WRITE, MYF(0), table_name); return 0; } } @@ -3016,34 +3031,6 @@ TABLE *find_write_locked_table(TABLE *list, const char *db, const char *table_na } -/** - Find instance of TABLE with MDL_SHARED_UPGRADABLE or - MDL_EXCLUSIVE lock from the list of open tables. - - @param list List of TABLE objects to be searched - @param db Database name. - @param table_name Name of table. - - @return Pointer to MDL_SHARED_UPGRADABLE or MDL_EXCLUSIVE - TABLE instance, NULL otherwise. -*/ - -TABLE *find_table_for_mdl_upgrade(TABLE *list, const char *db, - const char *table_name) -{ - TABLE *tab= find_locked_table(list, db, table_name); - - while (tab != NULL) - { - if (tab->mdl_ticket != NULL && - tab->mdl_ticket->is_upgradable_or_exclusive()) - return tab; - tab= find_locked_table(tab->next, db, table_name); - } - return NULL; -} - - /*********************************************************************** class Locked_tables_list implementation. Declared in sql_class.h ************************************************************************/ @@ -3741,9 +3728,9 @@ end_with_lock_open: Open_table_context::Open_table_context(THD *thd) :m_action(OT_NO_ACTION), m_start_of_statement_svp(thd->mdl_context.mdl_savepoint()), - m_has_locks((thd->in_multi_stmt_transaction() || - thd->mdl_context.lt_or_ha_sentinel()) && - thd->mdl_context.has_locks()), + m_has_locks((thd->in_multi_stmt_transaction() && + thd->mdl_context.has_locks()) || + thd->mdl_context.trans_sentinel()), m_global_mdl_request(NULL) {} @@ -3798,7 +3785,7 @@ request_backoff_action(enum_open_table_action action_arg) Waiting will be done after releasing metadata locks acquired by this statement. */ - if (m_has_locks && action_arg != OT_WAIT) + if (m_has_locks && action_arg != OT_WAIT_MDL_LOCK) { my_error(ER_LOCK_DEADLOCK, MYF(0)); return TRUE; @@ -3838,33 +3825,28 @@ recover_from_failed_open(THD *thd, MDL_request *mdl_request, /* Execute the action. */ switch (m_action) { - case OT_WAIT: - result= (thd->mdl_context.wait_for_locks(&m_mdl_requests) || - tdc_wait_for_old_versions(thd, &m_mdl_requests)); + case OT_WAIT_MDL_LOCK: + result= thd->mdl_context.wait_for_lock(mdl_request); + break; + case OT_WAIT_TDC: + result= tdc_wait_for_old_versions(thd, &m_mdl_requests); DBUG_ASSERT(thd->mysys_var->current_mutex == NULL); break; case OT_DISCOVER: { MDL_request mdl_global_request; MDL_request mdl_xlock_request(mdl_request); + MDL_request_list mdl_requests; mdl_global_request.init(MDL_key::GLOBAL, "", "", MDL_INTENTION_EXCLUSIVE); mdl_xlock_request.set_type(MDL_EXCLUSIVE); + mdl_requests.push_front(&mdl_xlock_request); + mdl_requests.push_front(&mdl_global_request); - if ((result= thd->mdl_context.acquire_global_intention_exclusive_lock( - &mdl_global_request))) - break; - - if ((result= - thd->mdl_context.acquire_exclusive_lock(&mdl_xlock_request))) - { - /* - We rely on close_thread_tables() to release global lock eventually. - */ + if ((result= thd->mdl_context.acquire_locks(&mdl_requests))) break; - } DBUG_ASSERT(mdl_request->key.mdl_namespace() == MDL_key::TABLE); pthread_mutex_lock(&LOCK_open); @@ -3885,23 +3867,17 @@ recover_from_failed_open(THD *thd, MDL_request *mdl_request, { MDL_request mdl_global_request; MDL_request mdl_xlock_request(mdl_request); + MDL_request_list mdl_requests; mdl_global_request.init(MDL_key::GLOBAL, "", "", MDL_INTENTION_EXCLUSIVE); mdl_xlock_request.set_type(MDL_EXCLUSIVE); - if ((result= thd->mdl_context.acquire_global_intention_exclusive_lock( - &mdl_global_request))) - break; + mdl_requests.push_front(&mdl_xlock_request); + mdl_requests.push_front(&mdl_global_request); - if ((result= - thd->mdl_context.acquire_exclusive_lock(&mdl_xlock_request))) - { - /* - We rely on close_thread_tables() to release global lock eventually. - */ + if ((result= thd->mdl_context.acquire_locks(&mdl_requests))) break; - } DBUG_ASSERT(mdl_request->key.mdl_namespace() == MDL_key::TABLE); pthread_mutex_lock(&LOCK_open); @@ -4015,13 +3991,13 @@ open_and_process_routine(THD *thd, Query_tables_list *prelocking_ctx, */ DBUG_ASSERT(rt->mdl_request.type == MDL_SHARED); - if (thd->mdl_context.try_acquire_shared_lock(&rt->mdl_request)) + if (thd->mdl_context.try_acquire_lock(&rt->mdl_request)) DBUG_RETURN(TRUE); if (rt->mdl_request.ticket == NULL) { /* A lock conflict. Someone's trying to modify SP metadata. */ - ot_ctx->request_backoff_action(Open_table_context::OT_WAIT); + ot_ctx->request_backoff_action(Open_table_context::OT_WAIT_MDL_LOCK); DBUG_RETURN(TRUE); } DEBUG_SYNC(thd, "after_shared_lock_pname"); @@ -4345,6 +4321,66 @@ end: /** + Acquire upgradable (SNW, SNRW) metadata locks on tables to be opened + for LOCK TABLES or a DDL statement. + + @param thd Thread context. + @param tables_start Start of list of tables on which upgradable locks + should be acquired. + @param tables_end End of list of tables. + @param ot_ctx Context of open_tables() operation. + + @retval FALSE Success. + @retval TRUE Failure (e.g. connection was killed) +*/ + +static bool +open_tables_acquire_upgradable_mdl(THD *thd, TABLE_LIST *tables_start, + TABLE_LIST *tables_end, + Open_table_context *ot_ctx) +{ + MDL_request_list mdl_requests; + TABLE_LIST *table; + + for (table= tables_start; table && table != tables_end; + table= table->next_global) + { + if (table->lock_type >= TL_WRITE_ALLOW_WRITE) + { + table->mdl_request.set_type(table->lock_type > TL_WRITE_ALLOW_READ ? + MDL_SHARED_NO_READ_WRITE : + MDL_SHARED_NO_WRITE); + mdl_requests.push_front(&table->mdl_request); + } + } + + if (! mdl_requests.is_empty()) + { + MDL_request *global_request= ot_ctx->get_global_mdl_request(thd); + + if (global_request == NULL) + return TRUE; + mdl_requests.push_front(global_request); + } + + if (thd->mdl_context.acquire_locks(&mdl_requests)) + return TRUE; + + for (table= tables_start; table && table != tables_end; + table= table->next_global) + { + if (table->lock_type >= TL_WRITE_ALLOW_WRITE) + { + table->mdl_request.ticket= NULL; + table->mdl_request.set_type(MDL_SHARED_WRITE); + } + } + + return FALSE; +} + + +/** Open all tables in list @param[in] thd Thread context. @@ -4419,6 +4455,29 @@ restart: thd_proc_info(thd, "Opening tables"); /* + If we are executing LOCK TABLES statement or a DDL statement + (in non-LOCK TABLES mode) we might have to acquire upgradable + semi-exclusive metadata locks (SNW or SNRW) on some of the + tables to be opened. + So we acquire all such locks at once here as doing this in one + by one fashion may lead to deadlocks or starvation. Later when + we will be opening corresponding table pre-acquired metadata + lock will be reused (thanks to the fact that in recursive case + metadata locks are acquired without waiting). + */ + if ((flags & MYSQL_OPEN_TAKE_UPGRADABLE_MDL) && + ! thd->locked_tables_mode) + { + if (open_tables_acquire_upgradable_mdl(thd, *start, + thd->lex->first_not_own_table(), + &ot_ctx)) + { + error= TRUE; + goto err; + } + } + + /* Perform steps of prelocking algorithm until there are unprocessed elements in prelocking list/set. */ @@ -4922,8 +4981,8 @@ retry: while ((error= open_table(thd, table_list, thd->mem_root, &ot_ctx, 0)) && ot_ctx.can_recover_from_failed_open()) { - /* We never have an open HANDLER or LOCK TABLES here. */ - DBUG_ASSERT(thd->mdl_context.lt_or_ha_sentinel() == NULL); + /* We never have an open HANDLER, LOCK TABLES or GRL here. */ + DBUG_ASSERT(thd->mdl_context.trans_sentinel() == NULL); /* Even though we have failed to open table we still need to call release_transactional_locks() to release metadata locks which @@ -4974,8 +5033,8 @@ retry: close_thread_tables(thd); table_list->table= NULL; table_list->mdl_request.ticket= NULL; - /* We never have an open HANDLER or LOCK TABLES here. */ - DBUG_ASSERT(thd->mdl_context.lt_or_ha_sentinel() == NULL); + /* We never have an open HANDLER, LOCK TABLES or GRL here. */ + DBUG_ASSERT(thd->mdl_context.trans_sentinel() == NULL); thd->mdl_context.rollback_to_savepoint(ot_ctx.start_of_statement_svp()); goto retry; } @@ -8459,15 +8518,19 @@ void flush_tables() @param thd Current thread context @param in_use The thread to wake up + @param needs_thr_lock_abort Indicates that to wake up thread + this call needs to abort its waiting + on table-level lock. @retval TRUE if the thread was woken up - @retval FALSE otherwise (e.g. it was not waiting for a table-level lock). + @retval FALSE otherwise. @note It is one of two places where border between MDL and the rest of the server is broken. */ -bool mysql_notify_thread_having_shared_lock(THD *thd, THD *in_use) +bool mysql_notify_thread_having_shared_lock(THD *thd, THD *in_use, + bool needs_thr_lock_abort) { bool signalled= FALSE; if ((in_use->system_thread & SYSTEM_THREAD_DELAYED_INSERT) && @@ -8481,19 +8544,23 @@ bool mysql_notify_thread_having_shared_lock(THD *thd, THD *in_use) signalled= TRUE; } pthread_mutex_lock(&LOCK_open); - for (TABLE *thd_table= in_use->open_tables; - thd_table ; - thd_table= thd_table->next) + + if (needs_thr_lock_abort) { - /* - Check for TABLE::needs_reopen() is needed since in some places we call - handler::close() for table instance (and set TABLE::db_stat to 0) - and do not remove such instances from the THD::open_tables - for some time, during which other thread can see those instances - (e.g. see partitioning code). - */ - if (!thd_table->needs_reopen()) - signalled|= mysql_lock_abort_for_thread(thd, thd_table); + for (TABLE *thd_table= in_use->open_tables; + thd_table ; + thd_table= thd_table->next) + { + /* + Check for TABLE::needs_reopen() is needed since in some places we call + handler::close() for table instance (and set TABLE::db_stat to 0) + and do not remove such instances from the THD::open_tables + for some time, during which other thread can see those instances + (e.g. see partitioning code). + */ + if (!thd_table->needs_reopen()) + signalled|= mysql_lock_abort_for_thread(thd, thd_table); + } } /* Wake up threads waiting in tdc_wait_for_old_versions(). @@ -8512,28 +8579,6 @@ bool mysql_notify_thread_having_shared_lock(THD *thd, THD *in_use) /** - Force transactions holding shared metadata lock on the table to call - MDL_context::can_wait_lead_to_deadlock() even if they don't need any - new metadata locks so they can detect potential deadlocks between - metadata locking subsystem and table-level locks. - - @param mdl_key MDL key for the table on which we are upgrading - metadata lock. -*/ - -void mysql_abort_transactions_with_shared_lock(const MDL_key *mdl_key) -{ - if (mdl_key->mdl_namespace() == MDL_key::TABLE) - { - pthread_mutex_lock(&LOCK_open); - tdc_remove_table(NULL, TDC_RT_REMOVE_UNUSED, mdl_key->db_name(), - mdl_key->name()); - pthread_mutex_unlock(&LOCK_open); - } -} - - -/** Remove all or some (depending on parameter) instances of TABLE and TABLE_SHARE from the table definition cache. @@ -8574,8 +8619,8 @@ void tdc_remove_table(THD *thd, enum_tdc_remove_table_type remove_type, safe_mutex_assert_owner(&LOCK_open); DBUG_ASSERT(remove_type == TDC_RT_REMOVE_UNUSED || - thd->mdl_context.is_exclusive_lock_owner(MDL_key::TABLE, - db, table_name)); + thd->mdl_context.is_lock_owner(MDL_key::TABLE, db, table_name, + MDL_EXCLUSIVE)); key_length=(uint) (strmov(strmov(key,db)+1,table_name)-key)+1; @@ -8639,24 +8684,6 @@ tdc_wait_for_old_versions(THD *thd, MDL_request_list *mdl_requests) */ mysql_ha_flush(thd); - /* - Check if there is someone waiting for one of metadata locks - held by this connection and return an error if that's the - case, since this situation may lead to a deadlock. - This can happen, when, for example, this connection is - waiting for an old version of some table to go away and - another connection is trying to upgrade its shared - metadata lock to exclusive, and thus is waiting - for this to release its lock. We must check for - the condition on each iteration of the loop to remove - any window for a race. - */ - if (thd->mdl_context.can_wait_lead_to_deadlock()) - { - my_error(ER_LOCK_DEADLOCK, MYF(0)); - return TRUE; - } - pthread_mutex_lock(&LOCK_open); MDL_request_list::Iterator it(*mdl_requests); |