summaryrefslogtreecommitdiff
path: root/sql/sql_base.cc
diff options
context:
space:
mode:
Diffstat (limited to 'sql/sql_base.cc')
-rw-r--r--sql/sql_base.cc2465
1 files changed, 1371 insertions, 1094 deletions
diff --git a/sql/sql_base.cc b/sql/sql_base.cc
index 8f31ef6999a..de4aaac633e 100644
--- a/sql/sql_base.cc
+++ b/sql/sql_base.cc
@@ -96,17 +96,32 @@ bool Prelock_error_handler::safely_trapped_errors()
@defgroup Data_Dictionary Data Dictionary
@{
*/
-TABLE *unused_tables; /* Used by mysql_test */
-HASH open_cache; /* Used by mysql_test */
-static HASH table_def_cache;
+
+/**
+ Total number of TABLE instances for tables in the table definition cache
+ (both in use by threads and not in use). This value is accessible to user
+ as "Open_tables" status variable.
+*/
+uint table_cache_count= 0;
+/**
+ List that contains all TABLE instances for tables in the table definition
+ cache that are not in use by any thread. Recently used TABLE instances are
+ appended to the end of the list. Thus the beginning of the list contains
+ tables which have been least recently used.
+*/
+TABLE *unused_tables;
+HASH table_def_cache;
static TABLE_SHARE *oldest_unused_share, end_of_unused_share;
static pthread_mutex_t LOCK_table_share;
static bool table_def_inited= 0;
-static int open_unireg_entry(THD *thd, TABLE *entry, TABLE_LIST *table_list,
- const char *alias,
- char *cache_key, uint cache_key_length,
- MEM_ROOT *mem_root, uint flags);
+static bool check_and_update_table_version(THD *thd, TABLE_LIST *tables,
+ TABLE_SHARE *table_share);
+static bool reopen_table_entry(THD *thd, TABLE *entry, TABLE_LIST *table_list,
+ const char *alias, char *cache_key,
+ uint cache_key_length);
+static bool open_table_entry_fini(THD *thd, TABLE_SHARE *share, TABLE *entry);
+static bool auto_repair_table(THD *thd, TABLE_LIST *table_list);
static void free_cache_entry(TABLE *entry);
static bool open_new_frm(THD *thd, TABLE_SHARE *share, const char *alias,
uint db_stat, uint prgflag,
@@ -114,41 +129,14 @@ static bool open_new_frm(THD *thd, TABLE_SHARE *share, const char *alias,
TABLE_LIST *table_desc, MEM_ROOT *mem_root);
static void close_old_data_files(THD *thd, TABLE *table, bool morph_locks,
bool send_refresh);
+static bool tdc_wait_for_old_versions(THD *thd, MDL_CONTEXT *context);
static bool
has_write_table_with_auto_increment(TABLE_LIST *tables);
-extern "C" uchar *table_cache_key(const uchar *record, size_t *length,
- my_bool not_used __attribute__((unused)))
-{
- TABLE *entry=(TABLE*) record;
- *length= entry->s->table_cache_key.length;
- return (uchar*) entry->s->table_cache_key.str;
-}
-
-
-bool table_cache_init(void)
-{
- return my_hash_init(&open_cache, &my_charset_bin, table_cache_size+16,
- 0, 0, table_cache_key,
- (my_hash_free_key) free_cache_entry, 0) != 0;
-}
-
-void table_cache_free(void)
-{
- DBUG_ENTER("table_cache_free");
- if (table_def_inited)
- {
- close_cached_tables(NULL, NULL, FALSE, FALSE, FALSE);
- if (!open_cache.records) // Safety first
- my_hash_free(&open_cache);
- }
- DBUG_VOID_RETURN;
-}
-
uint cached_open_tables(void)
{
- return open_cache.records;
+ return table_cache_count;
}
@@ -156,7 +144,8 @@ uint cached_open_tables(void)
static void check_unused(void)
{
uint count= 0, open_files= 0, idx= 0;
- TABLE *cur_link,*start_link;
+ TABLE *cur_link, *start_link, *entry;
+ TABLE_SHARE *share;
if ((start_link=cur_link=unused_tables))
{
@@ -167,45 +156,42 @@ static void check_unused(void)
DBUG_PRINT("error",("Unused_links aren't linked properly")); /* purecov: inspected */
return; /* purecov: inspected */
}
- } while (count++ < open_cache.records &&
+ } while (count++ < table_cache_count &&
(cur_link=cur_link->next) != start_link);
if (cur_link != start_link)
{
DBUG_PRINT("error",("Unused_links aren't connected")); /* purecov: inspected */
}
}
- for (idx=0 ; idx < open_cache.records ; idx++)
+ for (idx=0 ; idx < table_def_cache.records ; idx++)
{
- TABLE *entry=(TABLE*) my_hash_element(&open_cache,idx);
- if (!entry->in_use)
+ share= (TABLE_SHARE*) my_hash_element(&table_def_cache, idx);
+
+ I_P_List_iterator<TABLE, TABLE_share> it(share->free_tables);
+ while ((entry= it++))
+ {
+ if (entry->in_use)
+ {
+ DBUG_PRINT("error",("Used table is in share's list of unused tables")); /* purecov: inspected */
+ }
count--;
- if (entry->file)
open_files++;
+ }
+ it.init(share->used_tables);
+ while ((entry= it++))
+ {
+ if (!entry->in_use)
+ {
+ DBUG_PRINT("error",("Unused table is in share's list of used tables")); /* purecov: inspected */
+ }
+ open_files++;
+ }
}
if (count != 0)
{
DBUG_PRINT("error",("Unused_links doesn't match open_cache: diff: %d", /* purecov: inspected */
count)); /* purecov: inspected */
}
-
-#ifdef NOT_SAFE_FOR_REPAIR
- /*
- check that open cache and table definition cache has same number of
- aktive tables
- */
- count= 0;
- for (idx=0 ; idx < table_def_cache.records ; idx++)
- {
- TABLE_SHARE *entry= (TABLE_SHARE*) hash_element(&table_def_cache,idx);
- count+= entry->ref_count;
- }
- if (count != open_files)
- {
- DBUG_PRINT("error", ("table_def ref_count: %u open_cache: %u",
- count, open_files));
- DBUG_ASSERT(count == open_files);
- }
-#endif
}
#else
#define check_unused()
@@ -300,8 +286,11 @@ void table_def_free(void)
DBUG_ENTER("table_def_free");
if (table_def_inited)
{
+ /* Free all open TABLEs first. */
+ close_cached_tables(NULL, NULL, FALSE, FALSE);
table_def_inited= 0;
pthread_mutex_destroy(&LOCK_table_share);
+ /* Free table definitions. */
my_hash_free(&table_def_cache);
}
DBUG_VOID_RETURN;
@@ -315,6 +304,128 @@ uint cached_table_definitions(void)
/*
+ Auxiliary routines for manipulating with per-share used/unused and
+ global unused lists of TABLE objects and table_cache_count counter.
+ Responsible for preserving invariants between those lists, counter
+ and TABLE::in_use member.
+ In fact those routines implement sort of implicit table cache as
+ part of table definition cache.
+*/
+
+
+/**
+ Add newly created TABLE object for table share which is going
+ to be used right away.
+*/
+
+static void table_def_add_used_table(THD *thd, TABLE *table)
+{
+ DBUG_ASSERT(table->in_use == thd);
+ table->s->used_tables.push_front(table);
+ table_cache_count++;
+}
+
+
+/**
+ Prepare used or unused TABLE instance for destruction by removing
+ it from share's and global list.
+*/
+
+static void table_def_remove_table(TABLE *table)
+{
+ if (table->in_use)
+ {
+ /* Remove from per-share chain of used TABLE objects. */
+ table->s->used_tables.remove(table);
+ }
+ else
+ {
+ /* Remove from per-share chain of unused TABLE objects. */
+ table->s->free_tables.remove(table);
+
+ /* And global unused chain. */
+ table->next->prev=table->prev;
+ table->prev->next=table->next;
+ if (table == unused_tables)
+ {
+ unused_tables=unused_tables->next;
+ if (table == unused_tables)
+ unused_tables=0;
+ }
+ check_unused();
+ }
+ table_cache_count--;
+}
+
+
+/**
+ Mark already existing TABLE instance as used.
+*/
+
+static void table_def_use_table(THD *thd, TABLE *table)
+{
+ DBUG_ASSERT(!table->in_use);
+
+ /* Unlink table from list of unused tables for this share. */
+ table->s->free_tables.remove(table);
+ /* Unlink able from global unused tables list. */
+ if (table == unused_tables)
+ { // First unused
+ unused_tables=unused_tables->next; // Remove from link
+ if (table == unused_tables)
+ unused_tables=0;
+ }
+ table->prev->next=table->next; /* Remove from unused list */
+ table->next->prev=table->prev;
+ check_unused();
+ /* Add table to list of used tables for this share. */
+ table->s->used_tables.push_front(table);
+ table->in_use= thd;
+}
+
+
+/**
+ Mark already existing used TABLE instance as unused.
+*/
+
+static void table_def_unuse_table(TABLE *table)
+{
+ DBUG_ASSERT(table->in_use);
+
+ table->in_use= 0;
+ /* Remove table from the list of tables used in this share. */
+ table->s->used_tables.remove(table);
+ /* Add table to the list of unused TABLE objects for this share. */
+ table->s->free_tables.push_front(table);
+ /* Also link it last in the global list of unused TABLE objects. */
+ if (unused_tables)
+ {
+ table->next=unused_tables;
+ table->prev=unused_tables->prev;
+ unused_tables->prev=table;
+ table->prev->next=table;
+ }
+ else
+ unused_tables=table->next=table->prev=table;
+ check_unused();
+}
+
+
+/**
+ Bind used TABLE instance to another table share.
+
+ @note Will go away once we refactor code responsible
+ for reopening tables under lock tables.
+*/
+
+static void table_def_change_share(TABLE *table, TABLE_SHARE *new_share)
+{
+ table->s->used_tables.remove(table);
+ new_share->used_tables.push_front(table);
+}
+
+
+/*
Get TABLE_SHARE for a table.
get_table_share()
@@ -347,6 +458,13 @@ TABLE_SHARE *get_table_share(THD *thd, TABLE_LIST *table_list, char *key,
*error= 0;
+ /*
+ To be able perform any operation on table we should own
+ some kind of metadata lock on it.
+ */
+ DBUG_ASSERT(mdl_is_lock_owner(&thd->mdl_context, 0, table_list->db,
+ table_list->table_name));
+
/* Read table definition from cache */
if ((share= (TABLE_SHARE*) my_hash_search(&table_def_cache,(uchar*) key,
key_length)))
@@ -569,6 +687,7 @@ void release_table_share(TABLE_SHARE *share, enum release_type type)
safe_mutex_assert_owner(&LOCK_open);
pthread_mutex_lock(&share->mutex);
+ DBUG_ASSERT(share->ref_count);
if (!--share->ref_count)
{
if (share->version != refresh_version)
@@ -629,6 +748,26 @@ TABLE_SHARE *get_cached_table_share(const char *db, const char *table_name)
}
+/**
+ @brief Mark table share as having one more user (increase its reference
+ count).
+
+ @param share Table share for which reference count should be increased.
+*/
+
+static void reference_table_share(TABLE_SHARE *share)
+{
+ DBUG_ENTER("reference_table_share");
+ DBUG_ASSERT(share->ref_count);
+ pthread_mutex_lock(&share->mutex);
+ share->ref_count++;
+ pthread_mutex_unlock(&share->mutex);
+ DBUG_PRINT("exit", ("share: 0x%lx ref_count: %u",
+ (ulong) share, share->ref_count));
+ DBUG_VOID_RETURN;
+}
+
+
/*
Close file handle, but leave the table in the table cache
@@ -680,6 +819,7 @@ void close_handle_and_leave_table_as_lock(TABLE *table)
detach_merge_children(table, FALSE);
table->file->close();
table->db_stat= 0; // Mark file closed
+ table_def_change_share(table, share);
release_table_share(table->s, RELEASE_NORMAL);
table->s= share;
table->file->change_table_ptr(table, table->s);
@@ -719,11 +859,9 @@ OPEN_TABLE_LIST *list_open_tables(THD *thd, const char *db, const char *wild)
start_list= &open_list;
open_list=0;
- for (uint idx=0 ; result == 0 && idx < open_cache.records; idx++)
+ for (uint idx=0 ; result == 0 && idx < table_def_cache.records; idx++)
{
- OPEN_TABLE_LIST *table;
- TABLE *entry=(TABLE*) my_hash_element(&open_cache,idx);
- TABLE_SHARE *share= entry->s;
+ TABLE_SHARE *share= (TABLE_SHARE *)my_hash_element(&table_def_cache, idx);
if (db && my_strcasecmp(system_charset_info, db, share->db.str))
continue;
@@ -737,21 +875,7 @@ OPEN_TABLE_LIST *list_open_tables(THD *thd, const char *db, const char *wild)
if (check_table_access(thd,SELECT_ACL,&table_list, TRUE, 1, TRUE))
continue;
- /* need to check if we haven't already listed it */
- for (table= open_list ; table ; table=table->next)
- {
- if (!strcmp(table->table, share->table_name.str) &&
- !strcmp(table->db, share->db.str))
- {
- if (entry->in_use)
- table->in_use++;
- if (entry->locked_by_name)
- table->locked++;
- break;
- }
- }
- if (table)
- continue;
+
if (!(*start_list = (OPEN_TABLE_LIST *)
sql_alloc(sizeof(**start_list)+share->table_cache_key.length)))
{
@@ -762,8 +886,11 @@ OPEN_TABLE_LIST *list_open_tables(THD *thd, const char *db, const char *wild)
strmov(((*start_list)->db= (char*) ((*start_list)+1)),
share->db.str)+1,
share->table_name.str);
- (*start_list)->in_use= entry->in_use ? 1 : 0;
- (*start_list)->locked= entry->locked_by_name ? 1 : 0;
+ (*start_list)->in_use= 0;
+ I_P_List_iterator<TABLE, TABLE_share> it(share->used_tables);
+ while (it++)
+ ++(*start_list)->in_use;
+ (*start_list)->locked= (share->version == 0) ? 1 : 0;
start_list= &(*start_list)->next;
*start_list=0;
}
@@ -809,19 +936,11 @@ static void free_cache_entry(TABLE *table)
/* Assert that MERGE children are not attached before final close. */
DBUG_ASSERT(!table->is_children_attached());
+ /* This should be done before releasing table share. */
+ table_def_remove_table(table);
+
intern_close_table(table);
- if (!table->in_use)
- {
- table->next->prev=table->prev; /* remove from used chain */
- table->prev->next=table->next;
- if (table == unused_tables)
- {
- unused_tables=unused_tables->next;
- if (table == unused_tables)
- unused_tables=0;
- }
- check_unused(); // consisty check
- }
+
my_free((uchar*) table,MYF(0));
DBUG_VOID_RETURN;
}
@@ -841,6 +960,38 @@ void free_io_cache(TABLE *table)
}
+/**
+ Auxiliary function which allows to kill delayed threads for
+ particular table identified by its share.
+
+ @param share Table share.
+*/
+
+static void kill_delayed_threads_for_table(TABLE_SHARE *share)
+{
+ I_P_List_iterator<TABLE, TABLE_share> it(share->used_tables);
+ TABLE *tab;
+ while ((tab= it++))
+ {
+ THD *in_use= tab->in_use;
+
+ if ((in_use->system_thread & SYSTEM_THREAD_DELAYED_INSERT) &&
+ ! in_use->killed)
+ {
+ in_use->killed= THD::KILL_CONNECTION;
+ pthread_mutex_lock(&in_use->mysys_var->mutex);
+ if (in_use->mysys_var->current_cond)
+ {
+ pthread_mutex_lock(in_use->mysys_var->current_mutex);
+ pthread_cond_broadcast(in_use->mysys_var->current_cond);
+ pthread_mutex_unlock(in_use->mysys_var->current_mutex);
+ }
+ pthread_mutex_unlock(&in_use->mysys_var->mutex);
+ }
+ }
+}
+
+
/*
Close all tables which aren't in use by any thread
@@ -848,18 +999,23 @@ void free_io_cache(TABLE *table)
@param tables List of tables to remove from the cache
@param have_lock If LOCK_open is locked
@param wait_for_refresh Wait for a impending flush
- @param wait_for_placeholders Wait for tables being reopened so that the GRL
- won't proceed while write-locked tables are being reopened by other
- threads.
- @remark THD can be NULL, but then wait_for_refresh must be FALSE
- and tables must be NULL.
+ @note THD can be NULL, but then wait_for_refresh must be FALSE
+ and tables must be NULL.
+
+ @note When called as part of FLUSH TABLES WITH READ LOCK this function
+ ignores metadata locks held by other threads. In order to avoid
+ situation when FLUSH TABLES WITH READ LOCK sneaks in at the moment
+ when some write-locked table is being reopened (by FLUSH TABLES or
+ ALTER TABLE) we have to rely on additional global shared metadata
+ lock taken by thread trying to obtain global read lock.
*/
bool close_cached_tables(THD *thd, TABLE_LIST *tables, bool have_lock,
- bool wait_for_refresh, bool wait_for_placeholders)
+ bool wait_for_refresh)
{
- bool result=0;
+ bool result= FALSE;
+ bool found= TRUE;
DBUG_ENTER("close_cached_tables");
DBUG_ASSERT(thd || (!wait_for_refresh && !tables));
@@ -868,165 +1024,181 @@ bool close_cached_tables(THD *thd, TABLE_LIST *tables, bool have_lock,
if (!tables)
{
refresh_version++; // Force close of open tables
- while (unused_tables)
- {
-#ifdef EXTRA_DEBUG
- if (my_hash_delete(&open_cache,(uchar*) unused_tables))
- printf("Warning: Couldn't delete open table from hash\n");
-#else
- (void) my_hash_delete(&open_cache,(uchar*) unused_tables);
-#endif
- }
- /* Free table shares */
- while (oldest_unused_share->next)
- {
- pthread_mutex_lock(&oldest_unused_share->mutex);
- (void) my_hash_delete(&table_def_cache, (uchar*) oldest_unused_share);
- }
DBUG_PRINT("tcache", ("incremented global refresh_version to: %lu",
refresh_version));
- if (wait_for_refresh)
- {
- /*
- Other threads could wait in a loop in open_and_lock_tables(),
- trying to lock one or more of our tables.
-
- If they wait for the locks in thr_multi_lock(), their lock
- request is aborted. They loop in open_and_lock_tables() and
- enter open_table(). Here they notice the table is refreshed and
- wait for COND_refresh. Then they loop again in
- open_and_lock_tables() and this time open_table() succeeds. At
- this moment, if we (the FLUSH TABLES thread) are scheduled and
- on another FLUSH TABLES enter close_cached_tables(), they could
- awake while we sleep below, waiting for others threads (us) to
- close their open tables. If this happens, the other threads
- would find the tables unlocked. They would get the locks, one
- after the other, and could do their destructive work. This is an
- issue if we have LOCK TABLES in effect.
-
- The problem is that the other threads passed all checks in
- open_table() before we refresh the table.
-
- The fix for this problem is to set some_tables_deleted for all
- threads with open tables. These threads can still get their
- locks, but will immediately release them again after checking
- this variable. They will then loop in open_and_lock_tables()
- again. There they will wait until we update all tables version
- below.
-
- Setting some_tables_deleted is done by remove_table_from_cache()
- in the other branch.
-
- In other words (reviewer suggestion): You need this setting of
- some_tables_deleted for the case when table was opened and all
- related checks were passed before incrementing refresh_version
- (which you already have) but attempt to lock the table happened
- after the call to close_old_data_files() i.e. after removal of
- current thread locks.
- */
- for (uint idx=0 ; idx < open_cache.records ; idx++)
- {
- TABLE *table=(TABLE*) my_hash_element(&open_cache,idx);
- if (table->in_use)
- table->in_use->some_tables_deleted= 1;
- }
- }
+ kill_delayed_threads();
}
else
{
bool found=0;
for (TABLE_LIST *table= tables; table; table= table->next_local)
{
- if (remove_table_from_cache(thd, table->db, table->table_name,
- RTFC_OWNED_BY_THD_FLAG))
+ TABLE_SHARE *share= get_cached_table_share(table->db, table->table_name);
+
+ if (share)
+ {
+ share->version= 0;
+ kill_delayed_threads_for_table(share);
found=1;
+ }
}
if (!found)
wait_for_refresh=0; // Nothing to wait for
}
-#ifndef EMBEDDED_LIBRARY
- if (!tables)
- kill_delayed_threads();
-#endif
- if (wait_for_refresh)
+
+ /*
+ Get rid of all unused TABLE and TABLE_SHARE instances. By doing
+ this we automatically close all tables which were marked as "old".
+
+ FIXME: Do not close all unused TABLE instances when flushing
+ particular table.
+ */
+ while (unused_tables)
+ free_cache_entry(unused_tables);
+ /* Free table shares */
+ while (oldest_unused_share->next)
+ {
+ pthread_mutex_lock(&oldest_unused_share->mutex);
+ (void) my_hash_delete(&table_def_cache, (uchar*) oldest_unused_share);
+ }
+
+ if (!wait_for_refresh)
+ {
+ if (!have_lock)
+ pthread_mutex_unlock(&LOCK_open);
+ DBUG_RETURN(result);
+ }
+
+ DBUG_ASSERT(!have_lock);
+ pthread_mutex_unlock(&LOCK_open);
+
+ if (thd->locked_tables)
{
/*
- If there is any table that has a lower refresh_version, wait until
- this is closed (or this thread is killed) before returning
+ If we are under LOCK TABLES we need to reopen tables without
+ opening a door for any concurrent threads to sneak in and get
+ lock on our tables. To achieve this we use exclusive metadata
+ locks.
*/
- thd->mysys_var->current_mutex= &LOCK_open;
- thd->mysys_var->current_cond= &COND_refresh;
- thd_proc_info(thd, "Flushing tables");
+ if (!tables)
+ {
+ for (TABLE *tab= thd->open_tables; tab; tab= tab->next)
+ {
+ /*
+ Checking TABLE::db_stat is essential in case when we have
+ several instances of the table open and locked.
+ */
+ if (tab->db_stat)
+ {
+ char dbname[NAME_LEN+1];
+ char tname[NAME_LEN+1];
+ /*
+ Since close_data_files_and_morph_locks() frees share's memroot
+ we need to make copies of database and table names.
+ */
+ strmov(dbname, tab->s->db.str);
+ strmov(tname, tab->s->table_name.str);
+ if (wait_while_table_is_used(thd, tab, HA_EXTRA_FORCE_REOPEN))
+ {
+ result= TRUE;
+ goto err_with_reopen;
+ }
+ pthread_mutex_lock(&LOCK_open);
+ close_data_files_and_morph_locks(thd, dbname, tname);
+ pthread_mutex_unlock(&LOCK_open);
+ }
+ }
+ }
+ else
+ {
+ for (TABLE_LIST *table= tables; table; table= table->next_local)
+ {
+ TABLE *tab= find_locked_table(thd->open_tables, table->db,
+ table->table_name);
+ /*
+ Checking TABLE::db_stat is essential in case when we have
+ several instances of the table open and locked.
+ */
+ if (tab->db_stat)
+ {
+ if (wait_while_table_is_used(thd, tab, HA_EXTRA_FORCE_REOPEN))
+ {
+ result= TRUE;
+ goto err_with_reopen;
+ }
+ pthread_mutex_lock(&LOCK_open);
+ close_data_files_and_morph_locks(thd, table->db, table->table_name);
+ pthread_mutex_unlock(&LOCK_open);
+ }
+ }
+ }
+ }
- close_old_data_files(thd,thd->open_tables,1,1);
+ /* Wait until all threads have closed all the tables we are flushing. */
+ DBUG_PRINT("info", ("Waiting for other threads to close their open tables"));
+
+ while (found && ! thd->killed)
+ {
+ found= FALSE;
+ /*
+ To avoid self and other kinds of deadlock we have to flush open HANDLERs.
+ */
mysql_ha_flush(thd);
DEBUG_SYNC(thd, "after_flush_unlock");
- bool found=1;
- /* Wait until all threads has closed all the tables we had locked */
- DBUG_PRINT("info",
- ("Waiting for other threads to close their open tables"));
- while (found && ! thd->killed)
+ pthread_mutex_lock(&LOCK_open);
+
+ thd->enter_cond(&COND_refresh, &LOCK_open, "Flushing tables");
+
+ if (!tables)
{
- found=0;
- for (uint idx=0 ; idx < open_cache.records ; idx++)
+ for (uint idx=0 ; idx < table_def_cache.records ; idx++)
{
- TABLE *table=(TABLE*) my_hash_element(&open_cache,idx);
- /* Avoid a self-deadlock. */
- if (table->in_use == thd)
- continue;
- /*
- Note that we wait here only for tables which are actually open, and
- not for placeholders with TABLE::open_placeholder set. Waiting for
- latter will cause deadlock in the following scenario, for example:
-
- conn1: lock table t1 write;
- conn2: lock table t2 write;
- conn1: flush tables;
- conn2: flush tables;
-
- It also does not make sense to wait for those of placeholders that
- are employed by CREATE TABLE as in this case table simply does not
- exist yet.
- */
- if (table->needs_reopen_or_name_lock() && (table->db_stat ||
- (table->open_placeholder && wait_for_placeholders)))
- {
- found=1;
- DBUG_PRINT("signal", ("Waiting for COND_refresh"));
- pthread_cond_wait(&COND_refresh,&LOCK_open);
- break;
- }
+ TABLE_SHARE *share=(TABLE_SHARE*) my_hash_element(&table_def_cache,
+ idx);
+ if (share->version != refresh_version)
+ {
+ found= TRUE;
+ break;
+ }
+ }
+ }
+ else
+ {
+ for (TABLE_LIST *table= tables; table; table= table->next_local)
+ {
+ TABLE_SHARE *share= get_cached_table_share(table->db, table->table_name);
+ if (share && share->version != refresh_version)
+ {
+ found= TRUE;
+ break;
+ }
}
}
+
+ if (found)
+ {
+ DBUG_PRINT("signal", ("Waiting for COND_refresh"));
+ pthread_cond_wait(&COND_refresh,&LOCK_open);
+ }
+
+ thd->exit_cond(NULL);
+ }
+
+err_with_reopen:
+ if (thd->locked_tables)
+ {
+ pthread_mutex_lock(&LOCK_open);
/*
No other thread has the locked tables open; reopen them and get the
old locks. This should always succeed (unless some external process
has removed the tables)
*/
thd->in_lock_tables=1;
- result=reopen_tables(thd,1,1);
+ result|= reopen_tables(thd, 1);
thd->in_lock_tables=0;
- /* Set version for table */
- for (TABLE *table=thd->open_tables; table ; table= table->next)
- {
- /*
- Preserve the version (0) of write locked tables so that a impending
- global read lock won't sneak in.
- */
- if (table->reginfo.lock_type < TL_WRITE_ALLOW_WRITE)
- table->s->version= refresh_version;
- }
- }
- if (!have_lock)
pthread_mutex_unlock(&LOCK_open);
- if (wait_for_refresh)
- {
- pthread_mutex_lock(&thd->mysys_var->mutex);
- thd->mysys_var->current_mutex= 0;
- thd->mysys_var->current_cond= 0;
- thd_proc_info(thd, 0);
- pthread_mutex_unlock(&thd->mysys_var->mutex);
+ mdl_downgrade_exclusive_locks(&thd->mdl_context);
}
DBUG_RETURN(result);
}
@@ -1079,7 +1251,7 @@ bool close_cached_connection_tables(THD *thd, bool if_wait_for_refresh,
}
if (tables)
- result= close_cached_tables(thd, tables, TRUE, FALSE, FALSE);
+ result= close_cached_tables(thd, tables, TRUE, FALSE);
if (!have_lock)
pthread_mutex_unlock(&LOCK_open);
@@ -1208,9 +1380,8 @@ static void close_open_tables(THD *thd)
thd->some_tables_deleted= 0;
/* Free tables to hold down open files */
- while (open_cache.records > table_cache_size && unused_tables)
- my_hash_delete(&open_cache,(uchar*) unused_tables); /* purecov: tested */
- check_unused();
+ while (table_cache_count > table_cache_size && unused_tables)
+ free_cache_entry(unused_tables);
if (found_old_table)
{
/* Tell threads waiting for refresh that something has happened */
@@ -1239,7 +1410,8 @@ static void close_open_tables(THD *thd)
leave prelocked mode if needed.
*/
-void close_thread_tables(THD *thd)
+void close_thread_tables(THD *thd,
+ bool skip_mdl)
{
TABLE *table;
prelocked_mode_type prelocked_mode= thd->prelocked_mode;
@@ -1328,6 +1500,10 @@ void close_thread_tables(THD *thd)
if (prelocked_mode == PRELOCKED_UNDER_LOCK_TABLES)
DBUG_VOID_RETURN;
+ /*
+ Note that we are leaving prelocked mode so we don't need
+ to care about THD::locked_tables_root.
+ */
thd->lock= thd->locked_tables;
thd->locked_tables= 0;
/* Fallthrough */
@@ -1358,6 +1534,12 @@ void close_thread_tables(THD *thd)
if (thd->open_tables)
close_open_tables(thd);
+ mdl_release_locks(&thd->mdl_context);
+ if (!skip_mdl)
+ {
+ mdl_remove_all_locks(&thd->mdl_context);
+ }
+
if (prelocked_mode == PRELOCKED)
{
/*
@@ -1381,8 +1563,7 @@ bool close_thread_table(THD *thd, TABLE **table_ptr)
DBUG_ENTER("close_thread_table");
DBUG_ASSERT(table->key_read == 0);
DBUG_ASSERT(!table->file || table->file->inited == handler::NONE);
- DBUG_PRINT("tcache", ("table: '%s'.'%s' 0x%lx", table->s->db.str,
- table->s->table_name.str, (long) table));
+ safe_mutex_assert_owner(&LOCK_open);
*table_ptr=table->next;
/*
@@ -1392,10 +1573,11 @@ bool close_thread_table(THD *thd, TABLE **table_ptr)
if (table->child_l || table->parent)
detach_merge_children(table, TRUE);
+ table->mdl_lock= 0;
if (table->needs_reopen_or_name_lock() ||
thd->version != refresh_version || !table->db_stat)
{
- my_hash_delete(&open_cache,(uchar*) table);
+ free_cache_entry(table);
found_old_table=1;
}
else
@@ -1413,16 +1595,7 @@ bool close_thread_table(THD *thd, TABLE **table_ptr)
free_field_buffers_larger_than(table,MAX_TDC_BLOB_SIZE);
table->file->ha_reset();
- table->in_use=0;
- if (unused_tables)
- {
- table->next=unused_tables; /* Link in last */
- table->prev=unused_tables->prev;
- unused_tables->prev=table;
- table->prev->next=table;
- }
- else
- unused_tables=table->next=table->prev=table;
+ table_def_unuse_table(table);
}
DBUG_RETURN(found_old_table);
}
@@ -2121,7 +2294,7 @@ void unlink_open_table(THD *thd, TABLE *find, bool unlock)
/* Remove table from open_tables list. */
*prev= list->next;
/* Close table. */
- my_hash_delete(&open_cache,(uchar*) list); // Close table
+ free_cache_entry(list);
}
else
{
@@ -2231,43 +2404,34 @@ void wait_for_condition(THD *thd, pthread_mutex_t *mutex, pthread_cond_t *cond)
bool name_lock_locked_table(THD *thd, TABLE_LIST *tables)
{
+ bool result= TRUE;
+
DBUG_ENTER("name_lock_locked_table");
/* Under LOCK TABLES we must only accept write locked tables. */
- tables->table= find_locked_table(thd, tables->db, tables->table_name);
+ tables->table= find_write_locked_table(thd->open_tables, tables->db,
+ tables->table_name);
- if (!tables->table)
- my_error(ER_TABLE_NOT_LOCKED, MYF(0), tables->alias);
- else if (tables->table->reginfo.lock_type < TL_WRITE_LOW_PRIORITY)
- my_error(ER_TABLE_NOT_LOCKED_FOR_WRITE, MYF(0), tables->alias);
- else
+ if (tables->table)
{
/*
Ensures that table is opened only by this thread and that no
other statement will open this table.
*/
- wait_while_table_is_used(thd, tables->table, HA_EXTRA_FORCE_REOPEN);
- DBUG_RETURN(FALSE);
+ result= wait_while_table_is_used(thd, tables->table, HA_EXTRA_FORCE_REOPEN);
}
- DBUG_RETURN(TRUE);
+ DBUG_RETURN(result);
}
/*
- Open table which is already name-locked by this thread.
+ Open table for which this thread has exclusive meta-data lock.
SYNOPSIS
reopen_name_locked_table()
thd Thread handle
- table_list TABLE_LIST object for table to be open, TABLE_LIST::table
- member should point to TABLE object which was used for
- name-locking.
- link_in TRUE - if TABLE object for table to be opened should be
- linked into THD::open_tables list.
- FALSE - placeholder used for name-locking is already in
- this list so we only need to preserve TABLE::next
- pointer.
+ table_list TABLE_LIST object for table to be open.
NOTE
This function assumes that its caller already acquired LOCK_open mutex.
@@ -2277,33 +2441,32 @@ bool name_lock_locked_table(THD *thd, TABLE_LIST *tables)
TRUE - Error
*/
-bool reopen_name_locked_table(THD* thd, TABLE_LIST* table_list, bool link_in)
+bool reopen_name_locked_table(THD* thd, TABLE_LIST* table_list)
{
- TABLE *table= table_list->table;
+ TABLE *table;
TABLE_SHARE *share;
+ char key[MAX_DBKEY_LENGTH];
+ uint key_length;
char *table_name= table_list->table_name;
- TABLE orig_table;
DBUG_ENTER("reopen_name_locked_table");
- safe_mutex_assert_owner(&LOCK_open);
-
- if (thd->killed || !table)
+ if (thd->killed)
DBUG_RETURN(TRUE);
- orig_table= *table;
+ key_length= create_table_def_key(thd, key, table_list, 0);
+
+ if (!(table=(TABLE*) my_malloc(sizeof(*table),MYF(MY_WME))))
+ DBUG_RETURN(TRUE);
- if (open_unireg_entry(thd, table, table_list, table_name,
- table->s->table_cache_key.str,
- table->s->table_cache_key.length, thd->mem_root, 0))
+ if (reopen_table_entry(thd, table, table_list, table_name, key, key_length))
{
- intern_close_table(table);
/*
If there was an error during opening of table (for example if it
does not exist) '*table' object can be wiped out. To be able
properly release name-lock in this case we should restore this
object to its original state.
*/
- *table= orig_table;
+ my_free((uchar*)table, MYF(0));
DBUG_RETURN(TRUE);
}
@@ -2318,21 +2481,11 @@ bool reopen_name_locked_table(THD* thd, TABLE_LIST* table_list, bool link_in)
*/
share->version=0;
table->in_use = thd;
- check_unused();
- if (link_in)
- {
- table->next= thd->open_tables;
- thd->open_tables= table;
- }
- else
- {
- /*
- TABLE object should be already in THD::open_tables list so we just
- need to set TABLE::next correctly.
- */
- table->next= orig_table.next;
- }
+ table_def_add_used_table(thd, table);
+
+ table->next= thd->open_tables;
+ thd->open_tables= table;
table->tablenr=thd->current_tablenr++;
table->used_fields=0;
@@ -2340,109 +2493,7 @@ bool reopen_name_locked_table(THD* thd, TABLE_LIST* table_list, bool link_in)
table->null_row= table->maybe_null= 0;
table->force_index= table->force_index_order= table->force_index_group= 0;
table->status=STATUS_NO_RECORD;
- DBUG_RETURN(FALSE);
-}
-
-
-/**
- Create and insert into table cache placeholder for table
- which will prevent its opening (or creation) (a.k.a lock
- table name).
-
- @param thd Thread context
- @param key Table cache key for name to be locked
- @param key_length Table cache key length
-
- @return Pointer to TABLE object used for name locking or 0 in
- case of failure.
-*/
-
-TABLE *table_cache_insert_placeholder(THD *thd, const char *key,
- uint key_length)
-{
- TABLE *table;
- TABLE_SHARE *share;
- char *key_buff;
- DBUG_ENTER("table_cache_insert_placeholder");
-
- safe_mutex_assert_owner(&LOCK_open);
-
- /*
- Create a table entry with the right key and with an old refresh version
- Note that we must use my_multi_malloc() here as this is freed by the
- table cache
- */
- if (!my_multi_malloc(MYF(MY_WME | MY_ZEROFILL),
- &table, sizeof(*table),
- &share, sizeof(*share),
- &key_buff, key_length,
- NULL))
- DBUG_RETURN(NULL);
-
- table->s= share;
- share->set_table_cache_key(key_buff, key, key_length);
- share->tmp_table= INTERNAL_TMP_TABLE; // for intern_close_table
- table->in_use= thd;
- table->locked_by_name=1;
-
- if (my_hash_insert(&open_cache, (uchar*)table))
- {
- my_free((uchar*) table, MYF(0));
- DBUG_RETURN(NULL);
- }
-
- DBUG_RETURN(table);
-}
-
-
-/**
- Obtain an exclusive name lock on the table if it is not cached
- in the table cache.
-
- @param thd Thread context
- @param db Name of database
- @param table_name Name of table
- @param[out] table Out parameter which is either:
- - set to NULL if table cache contains record for
- the table or
- - set to point to the TABLE instance used for
- name-locking.
-
- @note This function takes into account all records for table in table
- cache, even placeholders used for name-locking. This means that
- 'table' parameter can be set to NULL for some situations when
- table does not really exist.
-
- @retval TRUE Error occured (OOM)
- @retval FALSE Success. 'table' parameter set according to above rules.
-*/
-
-bool lock_table_name_if_not_cached(THD *thd, const char *db,
- const char *table_name, TABLE **table)
-{
- char key[MAX_DBKEY_LENGTH];
- uint key_length;
- DBUG_ENTER("lock_table_name_if_not_cached");
-
- key_length= (uint)(strmov(strmov(key, db) + 1, table_name) - key) + 1;
- pthread_mutex_lock(&LOCK_open);
-
- if (my_hash_search(&open_cache, (uchar *)key, key_length))
- {
- pthread_mutex_unlock(&LOCK_open);
- DBUG_PRINT("info", ("Table is cached, name-lock is not obtained"));
- *table= 0;
- DBUG_RETURN(FALSE);
- }
- if (!(*table= table_cache_insert_placeholder(thd, key, key_length)))
- {
- pthread_mutex_unlock(&LOCK_open);
- DBUG_RETURN(TRUE);
- }
- (*table)->open_placeholder= 1;
- (*table)->next= thd->open_tables;
- thd->open_tables= *table;
- pthread_mutex_unlock(&LOCK_open);
+ table_list->table= table;
DBUG_RETURN(FALSE);
}
@@ -2511,6 +2562,20 @@ bool check_if_table_exists(THD *thd, TABLE_LIST *table, bool *exists)
}
+/**
+ @brief Helper function used by MDL subsystem for releasing TABLE_SHARE
+ objects in cases when it no longer wants to cache reference to it.
+*/
+
+void table_share_release_hook(void *share)
+{
+ pthread_mutex_lock(&LOCK_open);
+ release_table_share((TABLE_SHARE*)share, RELEASE_NORMAL);
+ broadcast_refresh();
+ pthread_mutex_unlock(&LOCK_open);
+}
+
+
/*
Open a table.
@@ -2518,13 +2583,17 @@ bool check_if_table_exists(THD *thd, TABLE_LIST *table, bool *exists)
open_table()
thd Thread context.
table_list Open first table in list.
- refresh INOUT Pointer to memory that will be set to 1 if
- we need to close all tables and reopen them.
+ action INOUT Pointer to variable of enum_open_table_action type
+ which will be set according to action which is
+ required to remedy problem appeared during attempt
+ to open table.
If this is a NULL pointer, then the table is not
put in the thread-open-list.
flags Bitmap of flags to modify how open works:
MYSQL_LOCK_IGNORE_FLUSH - Open table even if
- someone has done a flush or namelock on it.
+ someone has done a flush or there is a pending
+ exclusive metadata lock requests against it
+ (i.e. request high priority metadata lock).
No version number checking is done.
MYSQL_OPEN_TEMPORARY_ONLY - Open only temporary
table not the base table or view.
@@ -2545,21 +2614,23 @@ bool check_if_table_exists(THD *thd, TABLE_LIST *table, bool *exists)
TABLE *open_table(THD *thd, TABLE_LIST *table_list, MEM_ROOT *mem_root,
- bool *refresh, uint flags)
+ enum_open_table_action *action, uint flags)
{
reg1 TABLE *table;
char key[MAX_DBKEY_LENGTH];
uint key_length;
char *alias= table_list->alias;
- HASH_SEARCH_STATE state;
+ MDL_LOCK *mdl_lock;
+ int error;
+ TABLE_SHARE *share;
DBUG_ENTER("open_table");
/* Parsing of partitioning information from .frm needs thd->lex set up. */
DBUG_ASSERT(thd->lex->is_lex_started);
/* find a unused table in the open table cache */
- if (refresh)
- *refresh=0;
+ if (action)
+ *action= OT_NO_ACTION;
/* an open table operation needs a lot of the stack space */
if (check_stack_overrun(thd, STACK_MIN_SIZE_FOR_OPEN, (uchar *)&alias))
@@ -2685,7 +2756,13 @@ TABLE *open_table(THD *thd, TABLE_LIST *table_list, MEM_ROOT *mem_root,
Is this table a view and not a base table?
(it is work around to allow to open view with locked tables,
real fix will be made after definition cache will be made)
+
+ Since opening of view which was not explicitly locked by LOCK
+ TABLES breaks metadata locking protocol (potentially can lead
+ to deadlocks) it should be disallowed.
*/
+ if (mdl_is_lock_owner(&thd->mdl_context, 0, table_list->db,
+ table_list->table_name))
{
char path[FN_REFLEN + 1];
enum legacy_db_type not_used;
@@ -2693,21 +2770,12 @@ TABLE *open_table(THD *thd, TABLE_LIST *table_list, MEM_ROOT *mem_root,
table_list->db, table_list->table_name, reg_ext, 0);
if (mysql_frm_type(thd, path, &not_used) == FRMTYPE_VIEW)
{
- /*
- Will not be used (because it's VIEW) but has to be passed.
- Also we will not free it (because it is a stack variable).
- */
- TABLE tab;
- table= &tab;
- pthread_mutex_lock(&LOCK_open);
- if (!open_unireg_entry(thd, table, table_list, alias,
- key, key_length, mem_root, 0))
+ if (!tdc_open_view(thd, table_list, alias, key, key_length,
+ mem_root, 0))
{
DBUG_ASSERT(table_list->view != 0);
- pthread_mutex_unlock(&LOCK_open);
DBUG_RETURN(0); // VIEW
}
- pthread_mutex_unlock(&LOCK_open);
}
}
/*
@@ -2740,6 +2808,32 @@ TABLE *open_table(THD *thd, TABLE_LIST *table_list, MEM_ROOT *mem_root,
on disk.
*/
+ mdl_lock= table_list->mdl_lock;
+ mdl_add_lock(&thd->mdl_context, mdl_lock);
+
+ if (table_list->open_table_type)
+ {
+ mdl_set_lock_type(mdl_lock, MDL_EXCLUSIVE);
+ /* TODO: This case can be significantly optimized. */
+ if (mdl_acquire_exclusive_locks(&thd->mdl_context))
+ DBUG_RETURN(0);
+ }
+ else
+ {
+ bool retry;
+
+ if (table_list->mdl_upgradable)
+ mdl_set_upgradable(mdl_lock);
+ mdl_set_lock_priority(mdl_lock, (flags & MYSQL_LOCK_IGNORE_FLUSH) ?
+ MDL_HIGH_PRIO : MDL_NORMAL_PRIO);
+ if (mdl_acquire_shared_lock(mdl_lock, &retry))
+ {
+ if (action && retry)
+ *action= OT_BACK_OFF_AND_RETRY;
+ DBUG_RETURN(0);
+ }
+ }
+
pthread_mutex_lock(&LOCK_open);
/*
@@ -2756,222 +2850,210 @@ TABLE *open_table(THD *thd, TABLE_LIST *table_list, MEM_ROOT *mem_root,
! (flags & MYSQL_LOCK_IGNORE_FLUSH))
{
/* Someone did a refresh while thread was opening tables */
- if (refresh)
- *refresh=1;
+ if (action)
+ *action= OT_BACK_OFF_AND_RETRY;
pthread_mutex_unlock(&LOCK_open);
DBUG_RETURN(0);
}
- /*
- In order for the back off and re-start process to work properly,
- handler tables having old versions (due to FLUSH TABLES or pending
- name-lock) MUST be closed. This is specially important if a name-lock
- is pending for any table of the handler_tables list, otherwise a
- deadlock may occur.
- */
- if (thd->handler_tables)
- mysql_ha_flush(thd);
-
- /*
- Actually try to find the table in the open_cache.
- The cache may contain several "TABLE" instances for the same
- physical table. The instances that are currently "in use" by
- some thread have their "in_use" member != NULL.
- There is no good reason for having more than one entry in the
- hash for the same physical table, except that we use this as
- an implicit "pending locks queue" - see
- wait_for_locked_table_names for details.
- */
- for (table= (TABLE*) my_hash_first(&open_cache, (uchar*) key, key_length,
- &state);
- table && table->in_use ;
- table= (TABLE*) my_hash_next(&open_cache, (uchar*) key, key_length,
- &state))
+ if (table_list->open_table_type == TABLE_LIST::OPEN_OR_CREATE)
{
- DBUG_PRINT("tcache", ("in_use table: '%s'.'%s' 0x%lx", table->s->db.str,
- table->s->table_name.str, (long) table));
- /*
- Here we flush tables marked for flush.
- Normally, table->s->version contains the value of
- refresh_version from the moment when this table was
- (re-)opened and added to the cache.
- If since then we did (or just started) FLUSH TABLES
- statement, refresh_version has been increased.
- For "name-locked" TABLE instances, table->s->version is set
- to 0 (see lock_table_name for details).
- In case there is a pending FLUSH TABLES or a name lock, we
- need to back off and re-start opening tables.
- If we do not back off now, we may dead lock in case of lock
- order mismatch with some other thread:
- c1: name lock t1; -- sort of exclusive lock
- c2: open t2; -- sort of shared lock
- c1: name lock t2; -- blocks
- c2: open t1; -- blocks
- */
- if (table->needs_reopen_or_name_lock())
- {
- DBUG_PRINT("note",
- ("Found table '%s.%s' with different refresh version",
- table_list->db, table_list->table_name));
+ bool exists;
- if (flags & MYSQL_LOCK_IGNORE_FLUSH)
- {
- /* Force close at once after usage */
- thd->version= table->s->version;
- continue;
- }
+ if (check_if_table_exists(thd, table_list, &exists))
+ goto err_unlock2;
- /* Avoid self-deadlocks by detecting self-dependencies. */
- if (table->open_placeholder && table->in_use == thd)
- {
- pthread_mutex_unlock(&LOCK_open);
- my_error(ER_UPDATE_TABLE_USED, MYF(0), table->s->table_name.str);
- DBUG_RETURN(0);
- }
+ if (!exists)
+ {
+ pthread_mutex_unlock(&LOCK_open);
+ DBUG_RETURN(0);
+ }
+ /* Table exists. Let us try to open it. */
+ }
+ else if (table_list->open_table_type == TABLE_LIST::TAKE_EXCLUSIVE_MDL)
+ {
+ pthread_mutex_unlock(&LOCK_open);
+ DBUG_RETURN(0);
+ }
+ if (!(share= (TABLE_SHARE *)mdl_get_cached_object(mdl_lock)))
+ {
+ if (!(share= get_table_share_with_create(thd, table_list, key,
+ key_length, OPEN_VIEW,
+ &error)))
+ goto err_unlock2;
+
+ if (share->is_view)
+ {
/*
- Back off, part 1: mark the table as "unused" for the
- purpose of name-locking by setting table->db_stat to 0. Do
- that only for the tables in this thread that have an old
- table->s->version (this is an optimization (?)).
- table->db_stat == 0 signals wait_for_locked_table_names
- that the tables in question are not used any more. See
- table_is_used call for details.
-
- Notice that HANDLER tables were already taken care of by
- the earlier call to mysql_ha_flush() in this same critical
- section.
- */
- close_old_data_files(thd,thd->open_tables,0,0);
- /*
- Back-off part 2: try to avoid "busy waiting" on the table:
- if the table is in use by some other thread, we suspend
- and wait till the operation is complete: when any
- operation that juggles with table->s->version completes,
- it broadcasts COND_refresh condition variable.
- If 'old' table we met is in use by current thread we return
- without waiting since in this situation it's this thread
- which is responsible for broadcasting on COND_refresh
- (and this was done already in close_old_data_files()).
- Good example of such situation is when we have statement
- that needs two instances of table and FLUSH TABLES comes
- after we open first instance but before we open second
- instance.
+ This table is a view. Validate its metadata version: in particular,
+ that it was a view when the statement was prepared.
*/
- if (table->in_use != thd)
+ if (check_and_update_table_version(thd, table_list, share))
+ goto err_unlock;
+ if (table_list->i_s_requested_object & OPEN_TABLE_ONLY)
+ goto err_unlock;
+
+ /* Open view */
+ if (open_new_frm(thd, share, alias,
+ (uint) (HA_OPEN_KEYFILE | HA_OPEN_RNDFILE |
+ HA_GET_INDEX | HA_TRY_READ_ONLY),
+ READ_KEYINFO | COMPUTE_TYPES | EXTRA_RECORD |
+ (flags & OPEN_VIEW_NO_PARSE), thd->open_options,
+ 0, table_list, mem_root))
+ goto err_unlock;
+
+ /* TODO: Don't free this */
+ release_table_share(share, RELEASE_NORMAL);
+
+ if (flags & OPEN_VIEW_NO_PARSE)
{
- /* wait_for_conditionwill unlock LOCK_open for us */
- wait_for_condition(thd, &LOCK_open, &COND_refresh);
+ /*
+ VIEW not really opened, only frm were read.
+ Set 1 as a flag here
+ */
+ table_list->view= (LEX*)1;
}
else
{
- pthread_mutex_unlock(&LOCK_open);
+ DBUG_ASSERT(table_list->view);
}
+
+ pthread_mutex_unlock(&LOCK_open);
+ DBUG_RETURN(0);
+ }
+ else if (table_list->view)
+ {
/*
- There is a refresh in progress for this table.
- Signal the caller that it has to try again.
+ We're trying to open a table for what was a view.
+ This can only happen during (re-)execution.
+ At prepared statement prepare the view has been opened and
+ merged into the statement parse tree. After that, someone
+ performed a DDL and replaced the view with a base table.
+ Don't try to open the table inside a prepared statement,
+ invalidate it instead.
+
+ Note, the assert below is known to fail inside stored
+ procedures (Bug#27011).
*/
- if (refresh)
- *refresh=1;
- DBUG_RETURN(0);
+ DBUG_ASSERT(thd->m_reprepare_observer);
+ check_and_update_table_version(thd, table_list, share);
+ /* Always an error. */
+ DBUG_ASSERT(thd->is_error());
+ goto err_unlock;
}
+
+ if (table_list->i_s_requested_object & OPEN_VIEW_ONLY)
+ goto err_unlock;
+
+ /*
+ We are going to to store extra reference to the share in MDL-subsystem
+ so we need to increase reference counter;
+ */
+ reference_table_share(share);
+ mdl_set_cached_object(mdl_lock, share, table_share_release_hook);
}
- if (table)
+ else
{
- DBUG_PRINT("tcache", ("unused table: '%s'.'%s' 0x%lx", table->s->db.str,
- table->s->table_name.str, (long) table));
- /* Unlink the table from "unused_tables" list. */
- if (table == unused_tables)
- { // First unused
- unused_tables=unused_tables->next; // Remove from link
- if (table == unused_tables)
- unused_tables=0;
+ if (table_list->view)
+ {
+ DBUG_ASSERT(thd->m_reprepare_observer);
+ check_and_update_table_version(thd, table_list, share);
+ /* Always an error. */
+ DBUG_ASSERT(thd->is_error());
+ goto err_unlock;
}
- table->prev->next=table->next; /* Remove from unused list */
- table->next->prev=table->prev;
- table->in_use= thd;
+ /* When we have cached TABLE_SHARE we know that is not a view. */
+ if (table_list->i_s_requested_object & OPEN_VIEW_ONLY)
+ goto err_unlock;
+
+ /*
+ We are going to use this share for construction of new TABLE object
+ so reference counter should be increased.
+ */
+ reference_table_share(share);
+ }
+
+ if (share->version != refresh_version)
+ {
+ if (!(flags & MYSQL_LOCK_IGNORE_FLUSH))
+ {
+ if (action)
+ *action= OT_BACK_OFF_AND_RETRY;
+ release_table_share(share, RELEASE_NORMAL);
+ pthread_mutex_unlock(&LOCK_open);
+ DBUG_RETURN(0);
+ }
+ /* Force close at once after usage */
+ thd->version= share->version;
+ }
+
+ if (!share->free_tables.is_empty())
+ {
+ table= share->free_tables.head();
+ table_def_use_table(thd, table);
+ /* We need to release share as we have EXTRA reference to it in our hands. */
+ release_table_share(share, RELEASE_NORMAL);
}
else
{
- /* Insert a new TABLE instance into the open cache */
- int error;
- DBUG_PRINT("tcache", ("opening new table"));
- /* Free cache if too big */
- while (open_cache.records > table_cache_size && unused_tables)
- my_hash_delete(&open_cache,(uchar*) unused_tables); /* purecov: tested */
+ /* We have too many TABLE instances around let us try to get rid of them. */
+ while (table_cache_count > table_cache_size && unused_tables)
+ free_cache_entry(unused_tables);
- if (table_list->create)
- {
- bool exists;
+ /* make a new table */
+ if (!(table=(TABLE*) my_malloc(sizeof(*table),MYF(MY_WME))))
+ goto err_unlock;
- if (check_if_table_exists(thd, table_list, &exists))
- {
- pthread_mutex_unlock(&LOCK_open);
- DBUG_RETURN(NULL);
- }
+ error= open_table_from_share(thd, share, alias,
+ (uint) (HA_OPEN_KEYFILE |
+ HA_OPEN_RNDFILE |
+ HA_GET_INDEX |
+ HA_TRY_READ_ONLY),
+ (READ_KEYINFO | COMPUTE_TYPES |
+ EXTRA_RECORD),
+ thd->open_options, table, FALSE);
+
+ if (error)
+ {
+ my_free(table, MYF(0));
- if (!exists)
+ if (action)
{
- /*
- Table to be created, so we need to create placeholder in table-cache.
- */
- if (!(table= table_cache_insert_placeholder(thd, key, key_length)))
+ if (error == 7)
{
- pthread_mutex_unlock(&LOCK_open);
- DBUG_RETURN(NULL);
+ share->version= 0;
+ *action= OT_DISCOVER;
+ }
+ else if (share->crashed)
+ {
+ share->version= 0;
+ *action= OT_REPAIR;
}
- /*
- Link placeholder to the open tables list so it will be automatically
- removed once tables are closed. Also mark it so it won't be ignored
- by other trying to take name-lock.
- */
- table->open_placeholder= 1;
- table->next= thd->open_tables;
- thd->open_tables= table;
- pthread_mutex_unlock(&LOCK_open);
- DBUG_RETURN(table);
}
- /* Table exists. Let us try to open it. */
- }
- /* make a new table */
- if (!(table=(TABLE*) my_malloc(sizeof(*table),MYF(MY_WME))))
- {
- pthread_mutex_unlock(&LOCK_open);
- DBUG_RETURN(NULL);
+ goto err_unlock;
}
- error= open_unireg_entry(thd, table, table_list, alias, key, key_length,
- mem_root, (flags & OPEN_VIEW_NO_PARSE));
- if (error > 0)
+ if (open_table_entry_fini(thd, share, table))
{
+ closefrm(table, 0);
my_free((uchar*)table, MYF(0));
- pthread_mutex_unlock(&LOCK_open);
- DBUG_RETURN(NULL);
+ goto err_unlock;
}
- if (table_list->view || error < 0)
- {
- /*
- VIEW not really opened, only frm were read.
- Set 1 as a flag here
- */
- if (error < 0)
- table_list->view= (LEX*)1;
- my_free((uchar*)table, MYF(0));
- pthread_mutex_unlock(&LOCK_open);
- DBUG_RETURN(0); // VIEW
- }
- DBUG_PRINT("info", ("inserting table '%s'.'%s' 0x%lx into the cache",
- table->s->db.str, table->s->table_name.str,
- (long) table));
- (void) my_hash_insert(&open_cache,(uchar*) table);
+ /* Add table to the share's used tables list. */
+ table_def_add_used_table(thd, table);
}
- check_unused(); // Debugging call
-
pthread_mutex_unlock(&LOCK_open);
- if (refresh)
+
+ // Table existed
+ if (table_list->open_table_type == TABLE_LIST::OPEN_OR_CREATE)
+ mdl_downgrade_exclusive_locks(&thd->mdl_context);
+
+ table->mdl_lock= mdl_lock;
+ if (action)
{
table->next=thd->open_tables; /* Link into simple list */
thd->open_tables=table;
@@ -3013,15 +3095,32 @@ TABLE *open_table(THD *thd, TABLE_LIST *table_list, MEM_ROOT *mem_root,
table->clear_column_bitmaps();
DBUG_ASSERT(table->key_read == 0);
DBUG_RETURN(table);
+
+err_unlock:
+ release_table_share(share, RELEASE_NORMAL);
+err_unlock2:
+ pthread_mutex_unlock(&LOCK_open);
+ mdl_release_lock(&thd->mdl_context, mdl_lock);
+ DBUG_RETURN(0);
}
-TABLE *find_locked_table(THD *thd, const char *db,const char *table_name)
+/**
+ Find table in the list of open tables.
+
+ @param list List of TABLE objects to be inspected.
+ @param db Database name
+ @param table_name Table name
+
+ @return Pointer to the TABLE object found, 0 if no table found.
+*/
+
+TABLE *find_locked_table(TABLE *list, const char *db, const char *table_name)
{
char key[MAX_DBKEY_LENGTH];
uint key_length=(uint) (strmov(strmov(key,db)+1,table_name)-key)+1;
- for (TABLE *table=thd->open_tables; table ; table=table->next)
+ for (TABLE *table= list; table ; table=table->next)
{
if (table->s->table_cache_key.length == key_length &&
!memcmp(table->s->table_cache_key.str, key, key_length))
@@ -3031,6 +3130,41 @@ TABLE *find_locked_table(THD *thd, 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.
+
+ @param thd List of TABLE objects to be searched
+ @param db Database name.
+ @param table_name Name of table.
+
+ @return Pointer to write-locked TABLE instance, 0 - otherwise.
+*/
+
+TABLE *find_write_locked_table(TABLE *list, const char *db, const char *table_name)
+{
+ TABLE *tab= find_locked_table(list, db, table_name);
+
+ if (!tab)
+ {
+ my_error(ER_TABLE_NOT_LOCKED, MYF(0), table_name);
+ return 0;
+ }
+ else
+ {
+ while (tab->reginfo.lock_type < TL_WRITE_LOW_PRIORITY &&
+ (tab= find_locked_table(tab->next, db, table_name)))
+ continue;
+ if (!tab)
+ {
+ my_error(ER_TABLE_NOT_LOCKED_FOR_WRITE, MYF(0), table_name);
+ return 0;
+ }
+ }
+ return tab;
+}
+
+
/*
Reopen an table because the definition has changed.
@@ -3073,14 +3207,10 @@ bool reopen_table(TABLE *table)
table_list.table_name= table->s->table_name.str;
table_list.table= table;
- if (wait_for_locked_table_names(thd, &table_list))
- DBUG_RETURN(1); // Thread was killed
-
- if (open_unireg_entry(thd, &tmp, &table_list,
- table->alias,
- table->s->table_cache_key.str,
- table->s->table_cache_key.length,
- thd->mem_root, 0))
+ if (reopen_table_entry(thd, &tmp, &table_list,
+ table->alias,
+ table->s->table_cache_key.str,
+ table->s->table_cache_key.length))
goto end;
/* This list copies variables set by open_table */
@@ -3112,6 +3242,12 @@ bool reopen_table(TABLE *table)
(void) closefrm(&tmp, 1); // close file, free everything
goto end;
}
+ tmp.mdl_lock= table->mdl_lock;
+
+ table_def_change_share(table, tmp.s);
+ /* Avoid wiping out TABLE's position in new share's used tables list. */
+ tmp.share_next= table->share_next;
+ tmp.share_prev= table->share_prev;
delete table->triggers;
if (table->file)
@@ -3214,6 +3350,7 @@ void close_data_files_and_morph_locks(THD *thd, const char *db,
mysql_lock_remove(thd, thd->locked_tables, table, TRUE);
}
table->open_placeholder= 1;
+ table->s->version= 0;
close_handle_and_leave_table_as_lock(table);
}
}
@@ -3282,8 +3419,6 @@ static bool reattach_merge(THD *thd, TABLE **err_tables_p)
@param thd Thread context
@param get_locks Should we get locks after reopening tables ?
- @param mark_share_as_old Mark share as old to protect from a impending
- global read lock.
@note Since this function can't properly handle prelocking and
create placeholders it should be used in very special
@@ -3297,11 +3432,11 @@ static bool reattach_merge(THD *thd, TABLE **err_tables_p)
@return FALSE in case of success, TRUE - otherwise.
*/
-bool reopen_tables(THD *thd, bool get_locks, bool mark_share_as_old)
+bool reopen_tables(THD *thd, bool get_locks)
{
TABLE *table,*next,**prev;
TABLE **tables,**tables_ptr; // For locks
- TABLE *err_tables= NULL;
+ TABLE *err_tables= NULL, *err_tab_tmp;
bool error=0, not_used;
bool merge_table_found= FALSE;
const uint flags= MYSQL_LOCK_NOTIFY_IF_NEED_REOPEN |
@@ -3356,7 +3491,7 @@ bool reopen_tables(THD *thd, bool get_locks, bool mark_share_as_old)
*/
if (table->child_l || table->parent)
detach_merge_children(table, TRUE);
- my_hash_delete(&open_cache,(uchar*) table);
+ free_cache_entry(table);
error=1;
}
else
@@ -3367,11 +3502,15 @@ bool reopen_tables(THD *thd, bool get_locks, bool mark_share_as_old)
prev= &table->next;
/* Do not handle locks of MERGE children. */
if (get_locks && !db_stat && !table->parent)
- *tables_ptr++= table; // need new lock on this
- if (mark_share_as_old)
{
- table->s->version=0;
- table->open_placeholder= 0;
+ *tables_ptr++= table; // need new lock on this
+ /*
+ We rely on having exclusive metadata lock on the table to be
+ able safely re-acquire table locks on it.
+ */
+ DBUG_ASSERT(mdl_is_exclusive_lock_owner(&thd->mdl_context, 0,
+ table->s->db.str,
+ table->s->table_name.str));
}
}
}
@@ -3385,8 +3524,9 @@ bool reopen_tables(THD *thd, bool get_locks, bool mark_share_as_old)
{
while (err_tables)
{
- my_hash_delete(&open_cache, (uchar*) err_tables);
- err_tables= err_tables->next;
+ err_tab_tmp= err_tables->next;
+ free_cache_entry(err_tables);
+ err_tables= err_tab_tmp;
}
}
DBUG_PRINT("tcache", ("open tables to lock: %u",
@@ -3534,41 +3674,24 @@ bool table_is_used(TABLE *table, bool wait_for_name_lock)
{
char *key= table->s->table_cache_key.str;
uint key_length= table->s->table_cache_key.length;
-
- DBUG_PRINT("loop", ("table_name: %s", table->alias));
- HASH_SEARCH_STATE state;
- for (TABLE *search= (TABLE*) my_hash_first(&open_cache, (uchar*) key,
- key_length, &state);
- search ;
- search= (TABLE*) my_hash_next(&open_cache, (uchar*) key,
- key_length, &state))
- {
- DBUG_PRINT("info", ("share: 0x%lx "
- "open_placeholder: %d locked_by_name: %d "
- "db_stat: %u version: %lu",
- (ulong) search->s,
- search->open_placeholder, search->locked_by_name,
- search->db_stat,
- search->s->version));
- if (search->in_use == table->in_use)
- continue; // Name locked by this thread
- /*
- We can't use the table under any of the following conditions:
- - There is an name lock on it (Table is to be deleted or altered)
- - If we are in flush table and we didn't execute the flush
- - If the table engine is open and it's an old version
- (We must wait until all engines are shut down to use the table)
- */
- if ( (search->locked_by_name && wait_for_name_lock) ||
- (search->is_name_opened() && search->needs_reopen_or_name_lock()))
- DBUG_RETURN(1);
- }
+ /* Note that 'table' can use artificial TABLE_SHARE object. */
+ TABLE_SHARE *share= (TABLE_SHARE*)my_hash_search(&table_def_cache,
+ (uchar*) key, key_length);
+ if (share && !share->used_tables.is_empty() &&
+ share->version != refresh_version)
+ DBUG_RETURN(1);
} while ((table=table->next));
DBUG_RETURN(0);
}
-/* Wait until all used tables are refreshed */
+/*
+ Wait until all used tables are refreshed.
+
+ FIXME We should remove this function since for several functions which
+ are invoked by it new scenarios of usage are introduced, while
+ this function implements optimization useful only in rare cases.
+*/
bool wait_for_tables(THD *thd)
{
@@ -3593,7 +3716,7 @@ bool wait_for_tables(THD *thd)
/* Now we can open all tables without any interference */
thd_proc_info(thd, "Reopen tables");
thd->version= refresh_version;
- result=reopen_tables(thd,0,0);
+ result=reopen_tables(thd, 0);
}
pthread_mutex_unlock(&LOCK_open);
thd_proc_info(thd, 0);
@@ -3601,111 +3724,27 @@ bool wait_for_tables(THD *thd)
}
-/*
- drop tables from locked list
-
- SYNOPSIS
- drop_locked_tables()
- thd Thread thandler
- db Database
- table_name Table name
-
- INFORMATION
- This is only called on drop tables
-
- The TABLE object for the dropped table is unlocked but still kept around
- as a name lock, which means that the table will be available for other
- thread as soon as we call unlock_table_names().
- If there is multiple copies of the table locked, all copies except
- the first, which acts as a name lock, is removed.
+/**
+ Unlock and close tables open and locked by LOCK TABLES statement.
- RETURN
- # If table existed, return table
- 0 Table was not locked
+ @param thd Current thread's context.
*/
-
-TABLE *drop_locked_tables(THD *thd,const char *db, const char *table_name)
+void unlock_locked_tables(THD *thd)
{
- TABLE *table,*next,**prev, *found= 0;
- prev= &thd->open_tables;
- DBUG_ENTER("drop_locked_tables");
-
- /*
- Note that we need to hold LOCK_open while changing the
- open_tables list. Another thread may work on it.
- (See: remove_table_from_cache(), mysql_wait_completed_table())
- Closing a MERGE child before the parent would be fatal if the
- other thread tries to abort the MERGE lock in between.
- */
- for (table= thd->open_tables; table ; table=next)
- {
- next=table->next;
- if (!strcmp(table->s->table_name.str, table_name) &&
- !strcmp(table->s->db.str, db))
- {
- /* If MERGE child, forward lock handling to parent. */
- mysql_lock_remove(thd, thd->locked_tables,
- table->parent ? table->parent : table, TRUE);
- /*
- When closing a MERGE parent or child table, detach the children first.
- Clear child table references in case this object is opened again.
- */
- if (table->child_l || table->parent)
- detach_merge_children(table, TRUE);
+ DBUG_ASSERT(!thd->in_sub_stmt &&
+ !(thd->state_flags & Open_tables_state::BACKUPS_AVAIL));
- if (!found)
- {
- found= table;
- /* Close engine table, but keep object around as a name lock */
- if (table->db_stat)
- {
- table->db_stat= 0;
- table->file->close();
- }
- }
- else
- {
- /* We already have a name lock, remove copy */
- my_hash_delete(&open_cache,(uchar*) table);
- }
- }
- else
- {
- *prev=table;
- prev= &table->next;
- }
- }
- *prev=0;
- if (found)
- broadcast_refresh();
- if (thd->locked_tables && thd->locked_tables->table_count == 0)
+ if (thd->locked_tables)
{
- my_free((uchar*) thd->locked_tables,MYF(0));
+ thd->lock= thd->locked_tables;
thd->locked_tables=0;
- }
- DBUG_RETURN(found);
-}
-
-
-/*
- If we have the table open, which only happens when a LOCK TABLE has been
- done on the table, change the lock type to a lock that will abort all
- other threads trying to get the lock.
-*/
-
-void abort_locked_tables(THD *thd,const char *db, const char *table_name)
-{
- TABLE *table;
- for (table= thd->open_tables; table ; table= table->next)
- {
- if (!strcmp(table->s->table_name.str, table_name) &&
- !strcmp(table->s->db.str, db))
- {
- /* If MERGE child, forward lock handling to parent. */
- mysql_lock_abort(thd, table->parent ? table->parent : table, TRUE);
- break;
- }
+ close_thread_tables(thd);
+ /*
+ After closing tables we can free memory used for storing lock
+ request objects for metadata locks
+ */
+ free_root(&thd->locked_tables_root, MYF(MY_MARK_BLOCKS_FREE));
}
}
@@ -3812,7 +3851,7 @@ static bool inject_reprepare(THD *thd)
@retval FALSE success, version in TABLE_LIST has been updated
*/
-bool
+static bool
check_and_update_table_version(THD *thd,
TABLE_LIST *tables, TABLE_SHARE *table_share)
{
@@ -3838,41 +3877,98 @@ check_and_update_table_version(THD *thd,
return FALSE;
}
-/*
- Load a table definition from file and open unireg table
- SYNOPSIS
- open_unireg_entry()
- thd Thread handle
- entry Store open table definition here
- table_list TABLE_LIST with db, table_name & belong_to_view
- alias Alias name
- cache_key Key for share_cache
- cache_key_length length of cache_key
- mem_root temporary mem_root for parsing
- flags the OPEN_VIEW_NO_PARSE flag to be passed to
- openfrm()/open_new_frm()
+/**
+ Open view by getting its definition from disk (and table cache in future).
- NOTES
- Extra argument for open is taken from thd->open_options
- One must have a lock on LOCK_open when calling this function
+ @param thd Thread handle
+ @param table_list TABLE_LIST with db, table_name & belong_to_view
+ @param alias Alias name
+ @param cache_key Key for table definition cache
+ @param cache_key_length Length of cache_key
+ @param mem_root Memory to be used for .frm parsing.
+ @param flags Flags which modify how we open the view
- RETURN
- 0 ok
- # Error
+ @todo This function is needed for special handling of views under
+ LOCK TABLES. We probably should get rid of it in long term.
+
+ @return FALSE if success, TRUE - otherwise.
*/
-static int open_unireg_entry(THD *thd, TABLE *entry, TABLE_LIST *table_list,
- const char *alias,
- char *cache_key, uint cache_key_length,
- MEM_ROOT *mem_root, uint flags)
+bool tdc_open_view(THD *thd, TABLE_LIST *table_list, const char *alias,
+ char *cache_key, uint cache_key_length,
+ MEM_ROOT *mem_root, uint flags)
+{
+ TABLE not_used;
+ int error;
+ TABLE_SHARE *share;
+
+ pthread_mutex_lock(&LOCK_open);
+
+ if (!(share= get_table_share_with_create(thd, table_list, cache_key,
+ cache_key_length,
+ OPEN_VIEW, &error)))
+ goto err;
+
+ if (share->is_view &&
+ !open_new_frm(thd, share, alias,
+ (uint) (HA_OPEN_KEYFILE | HA_OPEN_RNDFILE |
+ HA_GET_INDEX | HA_TRY_READ_ONLY),
+ READ_KEYINFO | COMPUTE_TYPES | EXTRA_RECORD |
+ flags, thd->open_options, &not_used, table_list,
+ mem_root))
+ {
+ release_table_share(share, RELEASE_NORMAL);
+ pthread_mutex_unlock(&LOCK_open);
+ return FALSE;
+ }
+
+ my_error(ER_WRONG_OBJECT, MYF(0), share->db.str, share->table_name.str, "VIEW");
+ release_table_share(share, RELEASE_NORMAL);
+err:
+ pthread_mutex_unlock(&LOCK_open);
+ return TRUE;
+}
+
+
+/**
+ Load table definition from file and open table while holding exclusive
+ meta-data lock on it.
+
+ @param thd Thread handle
+ @param entry Memory for TABLE object to be created
+ @param table_list TABLE_LIST with db, table_name & belong_to_view
+ @param alias Alias name
+ @param cache_key Key for table definition cache
+ @param cache_key_length Length of cache_key
+
+ @note This auxiliary function is mostly inteded for re-opening table
+ in situations when we hold exclusive meta-data lock. It is not
+ intended for normal case in which we have only shared meta-data
+ lock on the table to be open.
+
+ @note Extra argument for open is taken from thd->open_options.
+
+ @note One must have a lock on LOCK_open as well as exclusive meta-data
+ lock on the table when calling this function.
+
+ @return FALSE in case of success, TRUE otherwise.
+*/
+
+static bool reopen_table_entry(THD *thd, TABLE *entry, TABLE_LIST *table_list,
+ const char *alias, char *cache_key,
+ uint cache_key_length)
{
int error;
TABLE_SHARE *share;
uint discover_retry_count= 0;
- DBUG_ENTER("open_unireg_entry");
+ DBUG_ENTER("reopen_table_entry");
safe_mutex_assert_owner(&LOCK_open);
+ DBUG_ASSERT(mdl_is_exclusive_lock_owner(&thd->mdl_context, 0,
+ table_list->db,
+ table_list->table_name));
+
retry:
if (!(share= get_table_share_with_create(thd, table_list, cache_key,
cache_key_length,
@@ -3901,40 +3997,11 @@ retry:
goto err;
if (table_list->i_s_requested_object & OPEN_TABLE_ONLY)
goto err;
-
- /* Open view */
- error= (int) open_new_frm(thd, share, alias,
- (uint) (HA_OPEN_KEYFILE | HA_OPEN_RNDFILE |
- HA_GET_INDEX | HA_TRY_READ_ONLY),
- READ_KEYINFO | COMPUTE_TYPES | EXTRA_RECORD |
- (flags & OPEN_VIEW_NO_PARSE),
- thd->open_options, entry, table_list,
- mem_root);
- if (error)
- goto err;
- /* TODO: Don't free this */
+ /* Attempt to reopen view will bring havoc to upper layers anyway. */
release_table_share(share, RELEASE_NORMAL);
- DBUG_RETURN((flags & OPEN_VIEW_NO_PARSE)? -1 : 0);
- }
- else if (table_list->view)
- {
- /*
- We're trying to open a table for what was a view.
- This can only happen during (re-)execution.
- At prepared statement prepare the view has been opened and
- merged into the statement parse tree. After that, someone
- performed a DDL and replaced the view with a base table.
- Don't try to open the table inside a prepared statement,
- invalidate it instead.
-
- Note, the assert below is known to fail inside stored
- procedures (Bug#27011).
- */
- DBUG_ASSERT(thd->m_reprepare_observer);
- check_and_update_table_version(thd, table_list, share);
- /* Always an error. */
- DBUG_ASSERT(thd->is_error());
- goto err;
+ my_error(ER_WRONG_OBJECT, MYF(0), share->db.str, share->table_name.str,
+ "BASE TABLE");
+ DBUG_RETURN(1);
}
if (table_list->i_s_requested_object & OPEN_VIEW_ONLY)
@@ -3956,89 +4023,67 @@ retry:
goto err;
/*
- TODO:
- Here we should wait until all threads has released the table.
- For now we do one retry. This may cause a deadlock if there
- is other threads waiting for other tables used by this thread.
-
- Proper fix would be to if the second retry failed:
- - Mark that table def changed
- - Return from open table
- - Close all tables used by this thread
- - Start waiting that the share is released
- - Retry by opening all tables again
+ Since we have exclusive metadata lock on the table here the only
+ practical case when share->ref_count != 1 is when we have several
+ instances of the table opened by this thread (i.e we are under LOCK
+ TABLES).
*/
+ if (share->ref_count != 1)
+ goto err;
+
+ release_table_share(share, RELEASE_NORMAL);
+
if (ha_create_table_from_engine(thd, table_list->db,
table_list->table_name))
goto err;
- /*
- TO BE FIXED
- To avoid deadlock, only wait for release if no one else is
- using the share.
- */
- if (share->ref_count != 1)
- goto err;
- /* Free share and wait until it's released by all threads */
- release_table_share(share, RELEASE_WAIT_FOR_DROP);
- if (!thd->killed)
- {
- thd->warning_info->clear_warning_info(thd->query_id);
- thd->clear_error(); // Clear error message
- goto retry;
- }
- DBUG_RETURN(1);
+
+ thd->warning_info->clear_warning_info(thd->query_id);
+ thd->clear_error(); // Clear error message
+ goto retry;
}
if (!entry->s || !entry->s->crashed)
goto err;
- // Code below is for repairing a crashed file
- if ((error= lock_table_name(thd, table_list, TRUE)))
- {
- if (error < 0)
- goto err;
- if (wait_for_locked_table_names(thd, table_list))
- {
- unlock_table_name(thd, table_list);
- goto err;
- }
- }
- pthread_mutex_unlock(&LOCK_open);
- thd->clear_error(); // Clear error message
- error= 0;
- if (open_table_from_share(thd, share, alias,
- (uint) (HA_OPEN_KEYFILE | HA_OPEN_RNDFILE |
- HA_GET_INDEX |
- HA_TRY_READ_ONLY),
- READ_KEYINFO | COMPUTE_TYPES | EXTRA_RECORD,
- ha_open_options | HA_OPEN_FOR_REPAIR,
- entry, FALSE) || ! entry->file ||
- (entry->file->is_crashed() && entry->file->ha_check_and_repair(thd)))
- {
- /* Give right error message */
- thd->clear_error();
- my_error(ER_NOT_KEYFILE, MYF(0), share->table_name.str, my_errno);
- sql_print_error("Couldn't repair table: %s.%s", share->db.str,
- share->table_name.str);
- if (entry->file)
- closefrm(entry, 0);
- error=1;
- }
- else
- thd->clear_error(); // Clear error message
- pthread_mutex_lock(&LOCK_open);
- unlock_table_name(thd, table_list);
-
- if (error)
- goto err;
- break;
- }
- if (Table_triggers_list::check_n_load(thd, share->db.str,
- share->table_name.str, entry, 0))
+ entry->s->version= 0;
+
+ /* TODO: We don't need to release share here. */
+ release_table_share(share, RELEASE_NORMAL);
+ pthread_mutex_unlock(&LOCK_open);
+ error= (int)auto_repair_table(thd, table_list);
+ pthread_mutex_lock(&LOCK_open);
+
+ if (error)
+ goto err;
+
+ goto retry;
+ }
+
+ if (open_table_entry_fini(thd, share, entry))
{
closefrm(entry, 0);
goto err;
}
+ DBUG_RETURN(0);
+
+err:
+ release_table_share(share, RELEASE_NORMAL);
+ DBUG_RETURN(1);
+}
+
+
+/**
+ Auxiliary routine which finalizes process of TABLE object creation
+ by loading triggers and handling implicitly emptied tables.
+*/
+
+static bool open_table_entry_fini(THD *thd, TABLE_SHARE *share, TABLE *entry)
+{
+
+ if (Table_triggers_list::check_n_load(thd, share->db.str,
+ share->table_name.str, entry, 0))
+ return TRUE;
+
/*
If we are here, there was no fatal error (but error may be still
unitialized).
@@ -4070,18 +4115,141 @@ retry:
*/
sql_print_error("When opening HEAP table, could not allocate memory "
"to write 'DELETE FROM `%s`.`%s`' to the binary log",
- table_list->db, table_list->table_name);
+ share->db.str, share->table_name.str);
delete entry->triggers;
- closefrm(entry, 0);
- goto err;
+ return TRUE;
}
}
}
- DBUG_RETURN(0);
+ return FALSE;
+}
-err:
+
+/**
+ Auxiliary routine which is used for performing automatical table repair.
+*/
+
+static bool auto_repair_table(THD *thd, TABLE_LIST *table_list)
+{
+ char cache_key[MAX_DBKEY_LENGTH];
+ uint cache_key_length;
+ TABLE_SHARE *share;
+ TABLE *entry;
+ int not_used;
+ bool result= FALSE;
+
+ cache_key_length= create_table_def_key(thd, cache_key, table_list, 0);
+
+ thd->clear_error();
+
+ pthread_mutex_lock(&LOCK_open);
+
+ if (!(share= get_table_share_with_create(thd, table_list, cache_key,
+ cache_key_length,
+ OPEN_VIEW, &not_used)))
+ {
+ pthread_mutex_unlock(&LOCK_open);
+ return TRUE;
+ }
+
+ if (share->is_view)
+ goto end_with_lock_open;
+
+ if (!(entry= (TABLE*)my_malloc(sizeof(TABLE), MYF(MY_WME))))
+ {
+ result= TRUE;
+ goto end_with_lock_open;
+ }
+ share->version= 0;
+ pthread_mutex_unlock(&LOCK_open);
+
+ if (open_table_from_share(thd, share, table_list->alias,
+ (uint) (HA_OPEN_KEYFILE | HA_OPEN_RNDFILE |
+ HA_GET_INDEX |
+ HA_TRY_READ_ONLY),
+ READ_KEYINFO | COMPUTE_TYPES | EXTRA_RECORD,
+ ha_open_options | HA_OPEN_FOR_REPAIR,
+ entry, FALSE) || ! entry->file ||
+ (entry->file->is_crashed() && entry->file->ha_check_and_repair(thd)))
+ {
+ /* Give right error message */
+ thd->clear_error();
+ my_error(ER_NOT_KEYFILE, MYF(0), share->table_name.str, my_errno);
+ sql_print_error("Couldn't repair table: %s.%s", share->db.str,
+ share->table_name.str);
+ if (entry->file)
+ closefrm(entry, 0);
+ result= TRUE;
+ }
+ else
+ {
+ thd->clear_error(); // Clear error message
+ closefrm(entry, 0);
+ }
+ my_free(entry, MYF(0));
+
+ pthread_mutex_lock(&LOCK_open);
+
+end_with_lock_open:
release_table_share(share, RELEASE_NORMAL);
- DBUG_RETURN(1);
+ pthread_mutex_unlock(&LOCK_open);
+ return result;
+}
+
+
+/**
+ Handle failed attempt ot open table by performing requested action.
+
+ @param thd Thread context
+ @param table Table list element for table that caused problem
+ @param action Type of action requested by failed open_table() call
+
+ @retval FALSE - Success. One should try to open tables once again.
+ @retval TRUE - Error
+*/
+
+static bool handle_failed_open_table_attempt(THD *thd, TABLE_LIST *table,
+ enum_open_table_action action)
+{
+ bool result= FALSE;
+
+ switch (action)
+ {
+ case OT_BACK_OFF_AND_RETRY:
+ result= (mdl_wait_for_locks(&thd->mdl_context) ||
+ tdc_wait_for_old_versions(thd, &thd->mdl_context));
+ mdl_remove_all_locks(&thd->mdl_context);
+ break;
+ case OT_DISCOVER:
+ mdl_set_lock_type(table->mdl_lock, MDL_EXCLUSIVE);
+ mdl_add_lock(&thd->mdl_context, table->mdl_lock);
+ if (mdl_acquire_exclusive_locks(&thd->mdl_context))
+ return TRUE;
+ pthread_mutex_lock(&LOCK_open);
+ expel_table_from_cache(0, table->db, table->table_name);
+ ha_create_table_from_engine(thd, table->db, table->table_name);
+ pthread_mutex_unlock(&LOCK_open);
+
+ thd->warning_info->clear_warning_info(thd->query_id);
+ thd->clear_error(); // Clear error message
+ mdl_release_exclusive_locks(&thd->mdl_context);
+ break;
+ case OT_REPAIR:
+ mdl_set_lock_type(table->mdl_lock, MDL_EXCLUSIVE);
+ mdl_add_lock(&thd->mdl_context, table->mdl_lock);
+ if (mdl_acquire_exclusive_locks(&thd->mdl_context))
+ return TRUE;
+ pthread_mutex_lock(&LOCK_open);
+ expel_table_from_cache(0, table->db, table->table_name);
+ pthread_mutex_unlock(&LOCK_open);
+
+ result= auto_repair_table(thd, table);
+ mdl_release_exclusive_locks(&thd->mdl_context);
+ break;
+ default:
+ DBUG_ASSERT(0);
+ }
+ return result;
}
@@ -4132,6 +4300,7 @@ static int add_merge_table_list(TABLE_LIST *tlist)
/* Set lock type. */
child_l->lock_type= tlist->lock_type;
+ child_l->mdl_upgradable= tlist->mdl_upgradable;
/* Set parent reference. */
child_l->parent_l= tlist;
@@ -4487,7 +4656,7 @@ thr_lock_type read_lock_type_for_table(THD *thd, TABLE *table)
int open_tables(THD *thd, TABLE_LIST **start, uint *counter, uint flags)
{
TABLE_LIST *tables= NULL;
- bool refresh;
+ enum_open_table_action action;
int result=0;
MEM_ROOT new_frm_mem;
/* Also used for indicating that prelocking is need */
@@ -4508,6 +4677,18 @@ int open_tables(THD *thd, TABLE_LIST **start, uint *counter, uint flags)
thd_proc_info(thd, "Opening tables");
/*
+ Close HANDLER tables which are marked for flush or against which there
+ are pending exclusive metadata locks. Note that we do this not to avoid
+ deadlocks (calls to mysql_ha_flush() in mdl_wait_for_locks() and
+ tdc_wait_for_old_version() are enough for this) but in order to have
+ a point during statement execution at which such HANDLERs are closed
+ even if they don't create problems for current thread (i.e. to avoid
+ having DDL blocked by HANDLERs opened for long time).
+ */
+ if (thd->handler_tables)
+ mysql_ha_flush(thd);
+
+ /*
If we are not already executing prelocked statement and don't have
statement for which table list for prelocking is already built, let
us cache routines and try to build such table list.
@@ -4556,9 +4737,18 @@ int open_tables(THD *thd, TABLE_LIST **start, uint *counter, uint flags)
*/
if (tables->derived)
{
- if (tables->view)
- goto process_view_routines;
- continue;
+ if (!tables->view)
+ continue;
+ /*
+ We restore view's name and database wiped out by derived tables
+ processing and fall back to standard open process in order to
+ obtain proper metadata locks and do other necessary steps like
+ stored routine processing.
+ */
+ tables->db= tables->view_db.str;
+ tables->db_length= tables->view_db.length;
+ tables->table_name= tables->view_name.str;
+ tables->table_name_length= tables->view_name.length;
}
/*
If this TABLE_LIST object is a placeholder for an information_schema
@@ -4602,12 +4792,12 @@ int open_tables(THD *thd, TABLE_LIST **start, uint *counter, uint flags)
*/
Prelock_error_handler prelock_handler;
thd->push_internal_handler(& prelock_handler);
- tables->table= open_table(thd, tables, &new_frm_mem, &refresh, flags);
+ tables->table= open_table(thd, tables, &new_frm_mem, &action, flags);
thd->pop_internal_handler();
safe_to_ignore_table= prelock_handler.safely_trapped_errors();
}
else
- tables->table= open_table(thd, tables, &new_frm_mem, &refresh, flags);
+ tables->table= open_table(thd, tables, &new_frm_mem, &action, flags);
}
else
DBUG_PRINT("tcache", ("referenced table: '%s'.'%s' 0x%lx",
@@ -4657,7 +4847,16 @@ int open_tables(THD *thd, TABLE_LIST **start, uint *counter, uint flags)
parent_l->next_global= *parent_l->table->child_last_l;
}
- if (refresh) // Refresh in progress
+ /*
+ FIXME This is a temporary hack. Actually we need check that will
+ allow us to differentiate between error while opening/creating
+ table and successful table creation.
+ ...
+ */
+ if (tables->open_table_type)
+ continue;
+
+ if (action)
{
/*
We have met name-locked or old version of table. Now we have
@@ -4675,7 +4874,17 @@ int open_tables(THD *thd, TABLE_LIST **start, uint *counter, uint flags)
*/
if (query_tables_last_own)
thd->lex->mark_as_requiring_prelocking(query_tables_last_own);
- close_tables_for_reopen(thd, start);
+ close_tables_for_reopen(thd, start, (action == OT_BACK_OFF_AND_RETRY));
+ /*
+ Here we rely on the fact that 'tables' still points to the valid
+ TABLE_LIST element. Altough currently this assumption is valid
+ it may change in future.
+ */
+ if (handle_failed_open_table_attempt(thd, tables, action))
+ {
+ result= -1;
+ goto err;
+ }
goto restart;
}
@@ -4929,6 +5138,7 @@ TABLE *open_ltable(THD *thd, TABLE_LIST *table_list, thr_lock_type lock_type,
uint lock_flags)
{
TABLE *table;
+ enum_open_table_action action;
bool refresh;
DBUG_ENTER("open_ltable");
@@ -4939,9 +5149,20 @@ TABLE *open_ltable(THD *thd, TABLE_LIST *table_list, thr_lock_type lock_type,
thd->current_tablenr= 0;
/* open_ltable can be used only for BASIC TABLEs */
table_list->required_type= FRMTYPE_TABLE;
- while (!(table= open_table(thd, table_list, thd->mem_root, &refresh, 0)) &&
- refresh)
- ;
+
+retry:
+ while (!(table= open_table(thd, table_list, thd->mem_root, &action, 0)) &&
+ action)
+ {
+ /*
+ Even altough we have failed to open table we still need to
+ call close_thread_tables() to release metadata locks which
+ might have been acquired successfully.
+ */
+ close_thread_tables(thd, (action == OT_BACK_OFF_AND_RETRY));
+ if (handle_failed_open_table_attempt(thd, table_list, action))
+ break;
+ }
if (table)
{
@@ -4969,8 +5190,22 @@ TABLE *open_ltable(THD *thd, TABLE_LIST *table_list, thr_lock_type lock_type,
DBUG_ASSERT(thd->lock == 0); // You must lock everything at once
if ((table->reginfo.lock_type= lock_type) != TL_UNLOCK)
if (! (thd->lock= mysql_lock_tables(thd, &table_list->table, 1,
- lock_flags, &refresh)))
- table= 0;
+ (lock_flags |
+ MYSQL_LOCK_NOTIFY_IF_NEED_REOPEN),
+ &refresh)))
+ {
+ /*
+ FIXME: Actually we should get rid of MYSQL_LOCK_NOTIFY_IF_NEED_REOPEN option
+ as all reopening should happen outside of mysql_lock_tables() code.
+ */
+ if (refresh)
+ {
+ close_thread_tables(thd);
+ goto retry;
+ }
+ else
+ table= 0;
+ }
}
}
@@ -5026,7 +5261,7 @@ int open_and_lock_tables_derived(THD *thd, TABLE_LIST *tables, bool derived)
break;
if (!need_reopen)
DBUG_RETURN(-1);
- close_tables_for_reopen(thd, &tables);
+ close_tables_for_reopen(thd, &tables, FALSE);
}
if (derived &&
(mysql_handle_derived(thd->lex, &mysql_derived_prepare) ||
@@ -5383,6 +5618,10 @@ int lock_tables(THD *thd, TABLE_LIST *tables, uint count, bool *need_reopen)
table->table->query_id= thd->query_id;
if (check_lock_and_start_stmt(thd, table->table, table->lock_type))
{
+ /*
+ This was an attempt to enter prelocked mode so there is no
+ need to care about THD::locked_tables_root here.
+ */
mysql_unlock_tables(thd, thd->locked_tables);
thd->locked_tables= 0;
thd->options&= ~(OPTION_TABLE_LOCK);
@@ -5469,7 +5708,7 @@ int lock_tables(THD *thd, TABLE_LIST *tables, uint count, bool *need_reopen)
*/
-void close_tables_for_reopen(THD *thd, TABLE_LIST **tables)
+void close_tables_for_reopen(THD *thd, TABLE_LIST **tables, bool skip_mdl)
{
/*
If table list consists only from tables from prelocking set, table list
@@ -5481,7 +5720,7 @@ void close_tables_for_reopen(THD *thd, TABLE_LIST **tables)
sp_remove_not_own_routines(thd->lex);
for (TABLE_LIST *tmp= *tables; tmp; tmp= tmp->next_global)
tmp->table= 0;
- close_thread_tables(thd);
+ close_thread_tables(thd, skip_mdl);
}
@@ -8393,36 +8632,6 @@ my_bool mysql_rm_tmp_tables(void)
*****************************************************************************/
/*
- Invalidate any cache entries that are for some DB
-
- SYNOPSIS
- remove_db_from_cache()
- db Database name. This will be in lower case if
- lower_case_table_name is set
-
- NOTE:
- We can't use hash_delete when looping hash_elements. We mark them first
- and afterwards delete those marked unused.
-*/
-
-void remove_db_from_cache(const char *db)
-{
- for (uint idx=0 ; idx < open_cache.records ; idx++)
- {
- TABLE *table=(TABLE*) my_hash_element(&open_cache,idx);
- if (!strcmp(table->s->db.str, db))
- {
- table->s->version= 0L; /* Free when thread is ready */
- if (!table->in_use)
- relink_unused(table);
- }
- }
- while (unused_tables && !unused_tables->s->version)
- my_hash_delete(&open_cache,(uchar*) unused_tables);
-}
-
-
-/*
free all unused tables
NOTE
@@ -8434,7 +8643,7 @@ void flush_tables()
{
(void) pthread_mutex_lock(&LOCK_open);
while (unused_tables)
- my_hash_delete(&open_cache,(uchar*) unused_tables);
+ free_cache_entry(unused_tables);
(void) pthread_mutex_unlock(&LOCK_open);
}
@@ -8468,90 +8677,82 @@ bool remove_table_from_cache(THD *thd, const char *db, const char *table_name,
key_length=(uint) (strmov(strmov(key,db)+1,table_name)-key)+1;
for (;;)
{
- HASH_SEARCH_STATE state;
result= signalled= 0;
- for (table= (TABLE*) my_hash_first(&open_cache, (uchar*) key, key_length,
- &state);
- table;
- table= (TABLE*) my_hash_next(&open_cache, (uchar*) key, key_length,
- &state))
+ if ((share= (TABLE_SHARE*) my_hash_search(&table_def_cache, (uchar*) key,
+ key_length)))
{
- THD *in_use;
- DBUG_PRINT("tcache", ("found table: '%s'.'%s' 0x%lx", table->s->db.str,
- table->s->table_name.str, (long) table));
-
- table->s->version=0L; /* Free when thread is ready */
- if (!(in_use=table->in_use))
- {
- DBUG_PRINT("info",("Table was not in use"));
+ I_P_List_iterator<TABLE, TABLE_share> it(share->free_tables);
+ share->version= 0;
+ while ((table= it++))
relink_unused(table);
- }
- else if (in_use != thd)
+
+ it.init(share->used_tables);
+ while ((table= it++))
{
- DBUG_PRINT("info", ("Table was in use by other thread"));
- /*
- Mark that table is going to be deleted from cache. This will
- force threads that are in mysql_lock_tables() (but not yet
- in thr_multi_lock()) to abort it's locks, close all tables and retry
- */
- in_use->some_tables_deleted= 1;
- if (table->is_name_opened())
- {
- DBUG_PRINT("info", ("Found another active instance of the table"));
- result=1;
- }
- /* Kill delayed insert threads */
- if ((in_use->system_thread & SYSTEM_THREAD_DELAYED_INSERT) &&
- ! in_use->killed)
+ THD *in_use= table->in_use;
+ DBUG_ASSERT(in_use);
+ if (in_use != thd)
{
- in_use->killed= THD::KILL_CONNECTION;
- pthread_mutex_lock(&in_use->mysys_var->mutex);
- if (in_use->mysys_var->current_cond)
- {
- pthread_mutex_lock(in_use->mysys_var->current_mutex);
- signalled= 1;
- pthread_cond_broadcast(in_use->mysys_var->current_cond);
- pthread_mutex_unlock(in_use->mysys_var->current_mutex);
- }
- pthread_mutex_unlock(&in_use->mysys_var->mutex);
+ DBUG_PRINT("info", ("Table was in use by other thread"));
+ /*
+ Mark that table is going to be deleted from cache. This will
+ force threads that are in mysql_lock_tables() (but not yet
+ in thr_multi_lock()) to abort it's locks, close all tables and retry
+ */
+ in_use->some_tables_deleted= 1;
+
+ if (table->is_name_opened())
+ {
+ DBUG_PRINT("info", ("Found another active instance of the table"));
+ result=1;
+ }
+ /* Kill delayed insert threads */
+ if ((in_use->system_thread & SYSTEM_THREAD_DELAYED_INSERT) &&
+ ! in_use->killed)
+ {
+ in_use->killed= THD::KILL_CONNECTION;
+ pthread_mutex_lock(&in_use->mysys_var->mutex);
+ if (in_use->mysys_var->current_cond)
+ {
+ pthread_mutex_lock(in_use->mysys_var->current_mutex);
+ signalled= 1;
+ pthread_cond_broadcast(in_use->mysys_var->current_cond);
+ pthread_mutex_unlock(in_use->mysys_var->current_mutex);
+ }
+ pthread_mutex_unlock(&in_use->mysys_var->mutex);
+ }
+ /*
+ Now we must abort all tables locks used by this thread
+ as the thread may be waiting to get a lock for another table.
+ Note that we need to hold LOCK_open while going through the
+ list. So that the other thread cannot change it. The other
+ thread must also hold LOCK_open whenever changing the
+ open_tables list. Aborting the MERGE lock after a child was
+ closed and before the parent is closed would be fatal.
+ */
+ for (TABLE *thd_table= in_use->open_tables;
+ thd_table ;
+ thd_table= thd_table->next)
+ {
+ /* Do not handle locks of MERGE children. */
+ if (thd_table->db_stat && !thd_table->parent) // If table is open
+ signalled|= mysql_lock_abort_for_thread(thd, thd_table);
+ }
}
- /*
- Now we must abort all tables locks used by this thread
- as the thread may be waiting to get a lock for another table.
- Note that we need to hold LOCK_open while going through the
- list. So that the other thread cannot change it. The other
- thread must also hold LOCK_open whenever changing the
- open_tables list. Aborting the MERGE lock after a child was
- closed and before the parent is closed would be fatal.
- */
- for (TABLE *thd_table= in_use->open_tables;
- thd_table ;
- thd_table= thd_table->next)
+ else
{
- /* Do not handle locks of MERGE children. */
- if (thd_table->db_stat && !thd_table->parent) // If table is open
- signalled|= mysql_lock_abort_for_thread(thd, thd_table);
+ DBUG_PRINT("info", ("Table was in use by current thread. db_stat: %u",
+ table->db_stat));
+ result= result || (flags & RTFC_OWNED_BY_THD_FLAG);
}
}
- else
- {
- DBUG_PRINT("info", ("Table was in use by current thread. db_stat: %u",
- table->db_stat));
- result= result || (flags & RTFC_OWNED_BY_THD_FLAG);
- }
- }
- while (unused_tables && !unused_tables->s->version)
- my_hash_delete(&open_cache,(uchar*) unused_tables);
- DBUG_PRINT("info", ("Removing table from table_def_cache"));
- /* Remove table from table definition cache if it's not in use */
- if ((share= (TABLE_SHARE*) my_hash_search(&table_def_cache,(uchar*) key,
- key_length)))
- {
+ while (unused_tables && !unused_tables->s->version)
+ free_cache_entry(unused_tables);
+
DBUG_PRINT("info", ("share version: %lu ref_count: %u",
share->version, share->ref_count));
- share->version= 0; // Mark for delete
if (share->ref_count == 0)
{
pthread_mutex_lock(&share->mutex);
@@ -8598,6 +8799,160 @@ bool remove_table_from_cache(THD *thd, const char *db, const char *table_name,
}
+/**
+ A callback to the server internals that is used to address
+ special cases of the locking protocol.
+ Invoked when acquiring an exclusive lock, for each thread that
+ has a conflicting shared metadata lock.
+
+ This function:
+ - aborts waiting of the thread on a data lock, to make it notice
+ the pending exclusive lock and back off.
+ - if the thread is an INSERT DELAYED thread, sends it a KILL
+ signal to terminate it.
+
+ @note This function does not wait for the thread to give away its
+ locks. Waiting is done outside for all threads at once.
+
+ @param thd Current thread context
+ @param in_use The thread to wake up
+
+ @retval TRUE if the thread was woken up
+ @retval FALSE otherwise (e.g. it was not waiting for a table-level lock).
+
+ @note It is one of two places where border between MDL and the
+ rest of the server is broken.
+*/
+
+bool notify_thread_having_shared_lock(THD *thd, THD *in_use)
+{
+ bool signalled= FALSE;
+ if ((in_use->system_thread & SYSTEM_THREAD_DELAYED_INSERT) &&
+ !in_use->killed)
+ {
+ in_use->killed= THD::KILL_CONNECTION;
+ pthread_mutex_lock(&in_use->mysys_var->mutex);
+ if (in_use->mysys_var->current_cond)
+ pthread_cond_broadcast(in_use->mysys_var->current_cond);
+ pthread_mutex_unlock(&in_use->mysys_var->mutex);
+ signalled= TRUE;
+ }
+ pthread_mutex_lock(&LOCK_open);
+ for (TABLE *thd_table= in_use->open_tables;
+ thd_table ;
+ thd_table= thd_table->next)
+ {
+ /* TODO With new MDL check for db_stat is probably a legacy */
+ if (thd_table->db_stat)
+ signalled|= mysql_lock_abort_for_thread(thd, thd_table);
+ }
+ pthread_mutex_unlock(&LOCK_open);
+ return signalled;
+}
+
+
+/**
+ Remove all instances of the table from cache assuming that current thread
+ has exclusive meta-data lock on it (optionally leave instances belonging
+ to the current thread in cache).
+
+ @param leave_thd 0 If we should remove all instances
+ non-0 Pointer to current thread context if we should
+ leave instances belonging to this thread.
+ @param db Name of database
+ @param table_name Name of table
+
+ @note Unlike remove_table_from_cache() it assumes that table instances
+ are already not used by any (other) thread (this should be achieved
+ by using meta-data locks).
+*/
+
+void expel_table_from_cache(THD *leave_thd, const char *db, const char *table_name)
+{
+ char key[MAX_DBKEY_LENGTH];
+ uint key_length;
+ TABLE *table;
+ TABLE_SHARE *share;
+
+ key_length=(uint) (strmov(strmov(key,db)+1,table_name)-key)+1;
+
+ if ((share= (TABLE_SHARE*) my_hash_search(&table_def_cache,(uchar*) key,
+ key_length)))
+ {
+ I_P_List_iterator<TABLE, TABLE_share> it(share->free_tables);
+ share->version= 0;
+
+ while ((table= it++))
+ relink_unused(table);
+ }
+
+ /* This may destroy share so we have to do new look-up later. */
+ while (unused_tables && !unused_tables->s->version)
+ free_cache_entry(unused_tables);
+
+ if ((share= (TABLE_SHARE*) my_hash_search(&table_def_cache,(uchar*) key,
+ key_length)))
+ {
+ DBUG_ASSERT(leave_thd || share->ref_count == 0);
+ if (share->ref_count == 0)
+ {
+ pthread_mutex_lock(&share->mutex);
+ my_hash_delete(&table_def_cache, (uchar*) share);
+ }
+ }
+}
+
+
+/**
+ Wait until there are no old versions of tables in the table
+ definition cache for the metadata locks that we try to acquire.
+
+ @param thd Thread context
+ @param context Metadata locking context with locks.
+*/
+
+static bool tdc_wait_for_old_versions(THD *thd, MDL_CONTEXT *context)
+{
+ MDL_LOCK *l;
+ TABLE_SHARE *share;
+ const char *old_msg;
+ LEX_STRING key;
+
+ while (!thd->killed)
+ {
+ /*
+ Here we have situation as in mdl_wait_for_locks() we need to
+ get rid of offending HANDLERs to avoid deadlock.
+ TODO: We should also investigate in which situations we have
+ to broadcast on COND_refresh because of this.
+ */
+ mysql_ha_flush(thd);
+ pthread_mutex_lock(&LOCK_open);
+
+ I_P_List_iterator<MDL_LOCK, MDL_LOCK_context> it= mdl_get_locks(context);
+ while ((l= it++))
+ {
+ mdl_get_tdc_key(l, &key);
+ if ((share= (TABLE_SHARE*) my_hash_search(&table_def_cache, (uchar*) key.str,
+ key.length)) &&
+ share->version != refresh_version &&
+ !share->used_tables.is_empty())
+ break;
+ }
+ if (!l)
+ {
+ pthread_mutex_unlock(&LOCK_open);
+ break;
+ }
+ old_msg= thd->enter_cond(&COND_refresh, &LOCK_open, "Waiting for table");
+ pthread_cond_wait(&COND_refresh, &LOCK_open);
+ /* LOCK_open mutex is unlocked by THD::exit_cond() as side-effect. */
+ thd->exit_cond(old_msg);
+ }
+ return thd->killed;
+}
+
+
int setup_ftfuncs(SELECT_LEX *select_lex)
{
List_iterator<Item_func_match> li(*(select_lex->ftfunc_list)),
@@ -8695,7 +9050,6 @@ open_new_frm(THD *thd, TABLE_SHARE *share, const char *alias,
}
err:
- bzero(outparam, sizeof(TABLE)); // do not run repair
DBUG_RETURN(1);
}
@@ -8728,15 +9082,23 @@ bool is_equal(const LEX_STRING *a, const LEX_STRING *b)
int abort_and_upgrade_lock(ALTER_PARTITION_PARAM_TYPE *lpt)
{
- uint flags= RTFC_WAIT_OTHER_THREAD_FLAG | RTFC_CHECK_KILLED_FLAG;
DBUG_ENTER("abort_and_upgrade_locks");
lpt->old_lock_type= lpt->table->reginfo.lock_type;
- pthread_mutex_lock(&LOCK_open);
/* If MERGE child, forward lock handling to parent. */
mysql_lock_abort(lpt->thd, lpt->table->parent ? lpt->table->parent :
lpt->table, TRUE);
- (void) remove_table_from_cache(lpt->thd, lpt->db, lpt->table_name, flags);
+ if (mdl_upgrade_shared_lock_to_exclusive(&lpt->thd->mdl_context, 0,
+ lpt->db, lpt->table_name))
+ {
+ mysql_lock_downgrade_write(lpt->thd,
+ lpt->table->parent ? lpt->table->parent :
+ lpt->table,
+ lpt->old_lock_type);
+ DBUG_RETURN(1);
+ }
+ pthread_mutex_lock(&LOCK_open);
+ expel_table_from_cache(lpt->thd, lpt->db, lpt->table_name);
pthread_mutex_unlock(&LOCK_open);
DBUG_RETURN(0);
}
@@ -8771,106 +9133,12 @@ void close_open_tables_and_downgrade(ALTER_PARTITION_PARAM_TYPE *lpt)
/*
- SYNOPSIS
- mysql_wait_completed_table()
- lpt Parameter passing struct
- my_table My table object
- All parameters passed through the ALTER_PARTITION_PARAM object
- RETURN VALUES
- TRUE Failure
- FALSE Success
- DESCRIPTION
- We have changed the frm file and now we want to wait for all users of
- the old frm to complete before proceeding to ensure that no one
- remains that uses the old frm definition.
- Start by ensuring that all users of the table will be removed from cache
- once they are done. Then abort all that have stumbled on locks and
- haven't been started yet.
-
- thd Thread object
- table Table object
- db Database name
- table_name Table name
-*/
+ Tells if two (or more) tables have auto_increment columns and we want to
+ lock those tables with a write lock.
-void mysql_wait_completed_table(ALTER_PARTITION_PARAM_TYPE *lpt, TABLE *my_table)
-{
- char key[MAX_DBKEY_LENGTH];
- uint key_length;
- TABLE *table;
- DBUG_ENTER("mysql_wait_completed_table");
-
- key_length=(uint) (strmov(strmov(key,lpt->db)+1,lpt->table_name)-key)+1;
- pthread_mutex_lock(&LOCK_open);
- HASH_SEARCH_STATE state;
- for (table= (TABLE*) my_hash_first(&open_cache,(uchar*) key,key_length,
- &state) ;
- table;
- table= (TABLE*) my_hash_next(&open_cache,(uchar*) key,key_length,
- &state))
- {
- THD *in_use= table->in_use;
- table->s->version= 0L;
- if (!in_use)
- {
- relink_unused(table);
- }
- else
- {
- /* Kill delayed insert threads */
- if ((in_use->system_thread & SYSTEM_THREAD_DELAYED_INSERT) &&
- ! in_use->killed)
- {
- in_use->killed= THD::KILL_CONNECTION;
- pthread_mutex_lock(&in_use->mysys_var->mutex);
- if (in_use->mysys_var->current_cond)
- {
- pthread_mutex_lock(in_use->mysys_var->current_mutex);
- pthread_cond_broadcast(in_use->mysys_var->current_cond);
- pthread_mutex_unlock(in_use->mysys_var->current_mutex);
- }
- pthread_mutex_unlock(&in_use->mysys_var->mutex);
- }
- /*
- Now we must abort all tables locks used by this thread
- as the thread may be waiting to get a lock for another table.
- Note that we need to hold LOCK_open while going through the
- list. So that the other thread cannot change it. The other
- thread must also hold LOCK_open whenever changing the
- open_tables list. Aborting the MERGE lock after a child was
- closed and before the parent is closed would be fatal.
- */
- for (TABLE *thd_table= in_use->open_tables;
- thd_table ;
- thd_table= thd_table->next)
- {
- /* Do not handle locks of MERGE children. */
- if (thd_table->db_stat && !thd_table->parent) // If table is open
- mysql_lock_abort_for_thread(lpt->thd, thd_table);
- }
- }
- }
- /*
- We start by removing all unused objects from the cache and marking
- those in use for removal after completion. Now we also need to abort
- all that are locked and are not progressing due to being locked
- by our lock. We don't upgrade our lock here.
- If MERGE child, forward lock handling to parent.
- */
- mysql_lock_abort(lpt->thd, my_table->parent ? my_table->parent : my_table,
- FALSE);
- pthread_mutex_unlock(&LOCK_open);
- DBUG_VOID_RETURN;
-}
-
-
-/*
- Check if one (or more) write tables have auto_increment columns.
-
- @param[in] tables Table list
-
- @retval 0 if at least one write tables has an auto_increment column
- @retval 1 otherwise
+ SYNOPSIS
+ has_two_write_locked_tables_with_auto_increment
+ tables Table list
NOTES:
Call this function only when you have established the list of all tables
@@ -8924,10 +9192,13 @@ open_system_tables_for_read(THD *thd, TABLE_LIST *table_list,
{
DBUG_ENTER("open_system_tables_for_read");
+ alloc_mdl_locks(table_list, thd->mem_root);
+
thd->reset_n_backup_open_tables_state(backup);
uint count= 0;
- bool not_used;
+ enum_open_table_action not_used;
+ bool not_used_2;
for (TABLE_LIST *tables= table_list; tables; tables= tables->next_global)
{
TABLE *table= open_table(thd, tables, thd->mem_root, &not_used,
@@ -8950,7 +9221,7 @@ open_system_tables_for_read(THD *thd, TABLE_LIST *table_list,
*(ptr++)= tables->table;
thd->lock= mysql_lock_tables(thd, list, count,
- MYSQL_LOCK_IGNORE_FLUSH, &not_used);
+ MYSQL_LOCK_IGNORE_FLUSH, &not_used_2);
}
if (thd->lock)
DBUG_RETURN(FALSE);
@@ -9002,6 +9273,8 @@ open_system_table_for_update(THD *thd, TABLE_LIST *one_table)
{
DBUG_ENTER("open_system_table_for_update");
+ alloc_mdl_locks(one_table, thd->mem_root);
+
TABLE *table= open_ltable(thd, one_table, one_table->lock_type, 0);
if (table)
{
@@ -9038,6 +9311,7 @@ open_performance_schema_table(THD *thd, TABLE_LIST *one_table,
thd->reset_n_backup_open_tables_state(backup);
+ alloc_mdl_locks(one_table, thd->mem_root);
if ((table= open_ltable(thd, one_table, one_table->lock_type, flags)))
{
DBUG_ASSERT(table->s->table_category == TABLE_CATEGORY_PERFORMANCE);
@@ -9116,6 +9390,9 @@ void close_performance_schema_table(THD *thd, Open_tables_state *backup)
pthread_mutex_unlock(&LOCK_open);
+ mdl_release_locks(&thd->mdl_context);
+ mdl_remove_all_locks(&thd->mdl_context);
+
thd->restore_backup_open_tables_state(backup);
}