Backport of:

----------------------------------------------------------
revno: 2617.23.20
committer: Konstantin Osipov <kostja@sun.com>
branch nick: mysql-6.0-runtime
timestamp: Wed 2009-03-04 16:31:31 +0300
message:
  WL#4284 "Transactional DDL locking"
  Review comments: "Objectify" the MDL API.
  MDL_request and MDL_context still need manual construction and
  destruction, since they are used in environment that is averse
  to constructors/destructors.
This commit is contained in:
Konstantin Osipov 2009-12-04 02:52:05 +03:00
parent 195adcd201
commit a3a23ec4d3
20 changed files with 829 additions and 797 deletions

View File

@ -140,7 +140,7 @@ static Uint64 *p_latest_trans_gci= 0;
*/
static TABLE *ndb_binlog_index= 0;
static TABLE_LIST binlog_tables;
static MDL_LOCK_REQUEST binlog_mdl_lock_request;
static MDL_request binlog_mdl_request;
/*
Helper functions
@ -2342,8 +2342,8 @@ static int open_ndb_binlog_index(THD *thd, TABLE **ndb_binlog_index)
tables->alias= tables->table_name= reptable;
tables->lock_type= TL_WRITE;
thd->proc_info= "Opening " NDB_REP_DB "." NDB_REP_TABLE;
mdl_request_init(&binlog_mdl_lock_request, 0, tables->db, tables->table_name);
tables->mdl_lock_request= &binlog_mdl_lock_request;
binlog_mdl_request.init(0, tables->db, tables->table_name);
tables->mdl_request= &binlog_mdl_request;
tables->required_type= FRMTYPE_TABLE;
uint counter;
thd->clear_error();

View File

@ -944,7 +944,7 @@ static MYSQL_LOCK *get_lock_data(THD *thd, TABLE **table_ptr, uint count,
@note This function assumes that no metadata locks were acquired
before calling it. Also it cannot be called while holding
LOCK_open mutex. Both these invariants are enforced by asserts
in mdl_acquire_exclusive_locks() functions.
in MDL_context::acquire_exclusive_locks().
@retval FALSE Success.
@retval TRUE Failure (OOM or thread was killed).
@ -953,24 +953,24 @@ static MYSQL_LOCK *get_lock_data(THD *thd, TABLE **table_ptr, uint count,
bool lock_table_names(THD *thd, TABLE_LIST *table_list)
{
TABLE_LIST *lock_table;
MDL_LOCK_REQUEST *mdl_lock_req;
MDL_request *mdl_request;
for (lock_table= table_list; lock_table; lock_table= lock_table->next_local)
{
mdl_lock_req= mdl_request_alloc(0, lock_table->db, lock_table->table_name,
thd->mem_root);
if (!mdl_lock_req)
mdl_request= MDL_request::create(0, lock_table->db, lock_table->table_name,
thd->mem_root);
if (!mdl_request)
goto end;
mdl_request_set_type(mdl_lock_req, MDL_EXCLUSIVE);
mdl_request_add(&thd->mdl_context, mdl_lock_req);
lock_table->mdl_lock_request= mdl_lock_req;
mdl_request->set_type(MDL_EXCLUSIVE);
thd->mdl_context.add_request(mdl_request);
lock_table->mdl_request= mdl_request;
}
if (mdl_acquire_exclusive_locks(&thd->mdl_context))
if (thd->mdl_context.acquire_exclusive_locks())
goto end;
return 0;
end:
mdl_request_remove_all(&thd->mdl_context);
thd->mdl_context.remove_all_requests();
return 1;
}
@ -986,8 +986,8 @@ end:
void unlock_table_names(THD *thd)
{
DBUG_ENTER("unlock_table_names");
mdl_ticket_release_all(&thd->mdl_context);
mdl_request_remove_all(&thd->mdl_context);
thd->mdl_context.release_all_locks();
thd->mdl_context.remove_all_requests();
DBUG_VOID_RETURN;
}
@ -1147,7 +1147,7 @@ bool lock_global_read_lock(THD *thd)
redundancy between metadata locks, global read lock and DDL
blocker (see WL#4399 and WL#4400).
*/
if (mdl_acquire_global_shared_lock(&thd->mdl_context))
if (thd->mdl_context.acquire_global_shared_lock())
{
/* Our thread was killed -- return back to initial state. */
pthread_mutex_lock(&LOCK_global_read_lock);
@ -1181,7 +1181,7 @@ void unlock_global_read_lock(THD *thd)
("global_read_lock: %u global_read_lock_blocks_commit: %u",
global_read_lock, global_read_lock_blocks_commit));
mdl_release_global_shared_lock(&thd->mdl_context);
thd->mdl_context.release_global_shared_lock();
pthread_mutex_lock(&LOCK_global_read_lock);
tmp= --global_read_lock;

View File

@ -8060,7 +8060,7 @@ int Table_map_log_event::do_apply_event(Relay_log_info const *rli)
{
RPL_TABLE_LIST *table_list;
char *db_mem, *tname_mem;
MDL_LOCK_REQUEST *mdl_lock_request;
MDL_request *mdl_request;
size_t dummy_len;
void *memory;
DBUG_ENTER("Table_map_log_event::do_apply_event(Relay_log_info*)");
@ -8075,7 +8075,7 @@ int Table_map_log_event::do_apply_event(Relay_log_info const *rli)
&table_list, (uint) sizeof(RPL_TABLE_LIST),
&db_mem, (uint) NAME_LEN + 1,
&tname_mem, (uint) NAME_LEN + 1,
&mdl_lock_request, sizeof(MDL_LOCK_REQUEST),
&mdl_request, sizeof(MDL_request),
NullS)))
DBUG_RETURN(HA_ERR_OUT_OF_MEM);
@ -8088,9 +8088,8 @@ int Table_map_log_event::do_apply_event(Relay_log_info const *rli)
table_list->updating= 1;
strmov(table_list->db, rpl_filter->get_rewrite_db(m_dbnam, &dummy_len));
strmov(table_list->table_name, m_tblnam);
mdl_request_init(mdl_lock_request, 0, table_list->db,
table_list->table_name);
table_list->mdl_lock_request= mdl_lock_request;
mdl_request->init(0, table_list->db, table_list->table_name);
table_list->mdl_request= mdl_request;
int error= 0;

File diff suppressed because it is too large Load Diff

274
sql/mdl.h
View File

@ -23,10 +23,9 @@
class THD;
struct MDL_LOCK_REQUEST;
struct MDL_LOCK_TICKET;
struct MDL_LOCK;
struct MDL_CONTEXT;
class MDL_context;
class MDL_lock;
class MDL_ticket;
/**
Type of metadata lock request.
@ -62,7 +61,7 @@ enum enum_mdl_state { MDL_PENDING, MDL_ACQUIRED };
or "name".
*/
class MDL_KEY
class MDL_key
{
public:
const uchar *ptr() const { return (uchar*) m_ptr; }
@ -90,21 +89,34 @@ public:
m_db_name_length= (uint) (strmov(m_ptr + 1, db) - m_ptr - 1);
m_length= (uint) (strmov(m_ptr + m_db_name_length + 2, name) - m_ptr + 1);
}
void mdl_key_init(const MDL_KEY *rhs)
void mdl_key_init(const MDL_key *rhs)
{
memcpy(m_ptr, rhs->m_ptr, rhs->m_length);
m_length= rhs->m_length;
m_db_name_length= rhs->m_db_name_length;
}
bool is_equal(const MDL_KEY *rhs) const
bool is_equal(const MDL_key *rhs) const
{
return (m_length == rhs->m_length &&
memcmp(m_ptr, rhs->m_ptr, m_length) == 0);
}
MDL_key(const MDL_key *rhs)
{
mdl_key_init(rhs);
}
MDL_key(char type_arg, const char *db_arg, const char *name_arg)
{
mdl_key_init(type_arg, db_arg, name_arg);
}
MDL_key() {} /* To use when part of MDL_request. */
private:
char m_ptr[MAX_MDLKEY_LENGTH];
uint m_length;
uint m_db_name_length;
private:
MDL_key(const MDL_key &); /* not implemented */
MDL_key &operator=(const MDL_key &); /* not implemented */
};
@ -125,14 +137,18 @@ struct I_P_List_adapter
/**
A pending metadata lock request.
A pending lock request or a granted metadata lock share the same abstract
base but are presented individually because they have different allocation
A lock request and a granted metadata lock are represented by
different classes because they have different allocation
sites and hence different lifetimes. The allocation of lock requests is
controlled from outside of the MDL subsystem, while allocation of granted
locks (tickets) is controlled within the MDL subsystem.
MDL_request is a C structure, you don't need to call a constructor
or destructor for it.
*/
struct MDL_LOCK_REQUEST
struct MDL_request
{
/** Type of metadata lock. */
enum enum_mdl_type type;
@ -140,51 +156,94 @@ struct MDL_LOCK_REQUEST
/**
Pointers for participating in the list of lock requests for this context.
*/
MDL_LOCK_REQUEST *next_in_context;
MDL_LOCK_REQUEST **prev_in_context;
MDL_request *next_in_context;
MDL_request **prev_in_context;
/** A lock is requested based on a fully qualified name and type. */
MDL_KEY key;
MDL_key key;
void init(unsigned char type_arg, const char *db_arg, const char *name_arg);
/** Set type of lock request. Can be only applied to pending locks. */
inline void set_type(enum_mdl_type type_arg)
{
DBUG_ASSERT(ticket == NULL);
type= type_arg;
}
bool is_shared() const { return type < MDL_EXCLUSIVE; }
/**
Pointer to the lock ticket object for this lock request.
Valid only if this lock request is satisfied.
*/
MDL_LOCK_TICKET *ticket;
MDL_ticket *ticket;
static MDL_request *create(unsigned char type, const char *db,
const char *name, MEM_ROOT *root);
};
typedef void (*mdl_cached_object_release_hook)(void *);
/**
A granted metadata lock.
@warning MDL_LOCK_TICKET members are private to the MDL subsystem.
@warning MDL_ticket members are private to the MDL subsystem.
@note Multiple shared locks on a same object are represented by a
single ticket. The same does not apply for other lock types.
*/
struct MDL_LOCK_TICKET
class MDL_ticket
{
/** Type of metadata lock. */
enum enum_mdl_type type;
/** State of the metadata lock ticket. */
enum enum_mdl_state state;
public:
/**
Pointers for participating in the list of lock requests for this context.
*/
MDL_LOCK_TICKET *next_in_context;
MDL_LOCK_TICKET **prev_in_context;
MDL_ticket *next_in_context;
MDL_ticket **prev_in_context;
/**
Pointers for participating in the list of satisfied/pending requests
for the lock.
*/
MDL_LOCK_TICKET *next_in_lock;
MDL_LOCK_TICKET **prev_in_lock;
MDL_ticket *next_in_lock;
MDL_ticket **prev_in_lock;
public:
bool has_pending_conflicting_lock() const;
void *get_cached_object();
void set_cached_object(void *cached_object,
mdl_cached_object_release_hook release_hook);
const MDL_context *get_ctx() const { return m_ctx; }
bool is_shared() const { return m_type < MDL_EXCLUSIVE; }
bool upgrade_shared_lock_to_exclusive();
void downgrade_exclusive_lock();
private:
friend class MDL_context;
MDL_ticket(MDL_context *ctx_arg, enum_mdl_type type_arg)
: m_type(type_arg),
m_state(MDL_PENDING),
m_ctx(ctx_arg),
m_lock(NULL)
{}
static MDL_ticket *create(MDL_context *ctx_arg, enum_mdl_type type_arg);
static void destroy(MDL_ticket *ticket);
private:
/** Type of metadata lock. */
enum enum_mdl_type m_type;
/** State of the metadata lock ticket. */
enum enum_mdl_state m_state;
/** Context of the owner of the metadata lock ticket. */
MDL_CONTEXT *ctx;
MDL_context *m_ctx;
/** Pointer to the lock object for this lock ticket. */
MDL_LOCK *lock;
MDL_lock *m_lock;
private:
MDL_ticket(const MDL_ticket &); /* not implemented */
MDL_ticket &operator=(const MDL_ticket &); /* not implemented */
};
@ -193,116 +252,87 @@ struct MDL_LOCK_TICKET
connection has such a context.
*/
struct MDL_CONTEXT
class MDL_context
{
typedef I_P_List<MDL_LOCK_REQUEST,
I_P_List_adapter<MDL_LOCK_REQUEST,
&MDL_LOCK_REQUEST::next_in_context,
&MDL_LOCK_REQUEST::prev_in_context> >
public:
typedef I_P_List<MDL_request,
I_P_List_adapter<MDL_request,
&MDL_request::next_in_context,
&MDL_request::prev_in_context> >
Request_list;
typedef Request_list::Iterator Request_iterator;
typedef I_P_List<MDL_LOCK_TICKET,
I_P_List_adapter<MDL_LOCK_TICKET,
&MDL_LOCK_TICKET::next_in_context,
&MDL_LOCK_TICKET::prev_in_context> >
typedef I_P_List<MDL_ticket,
I_P_List_adapter<MDL_ticket,
&MDL_ticket::next_in_context,
&MDL_ticket::prev_in_context> >
Ticket_list;
typedef Ticket_list::Iterator Ticket_iterator;
Request_list requests;
Ticket_list tickets;
bool has_global_shared_lock;
THD *thd;
void init(THD *thd);
void destroy();
void backup_and_reset(MDL_context *backup);
void restore_from_backup(MDL_context *backup);
void merge(MDL_context *source);
void add_request(MDL_request *mdl_request);
void remove_request(MDL_request *mdl_request);
void remove_all_requests();
bool acquire_shared_lock(MDL_request *mdl_request, bool *retry);
bool acquire_exclusive_locks();
bool try_acquire_exclusive_lock(MDL_request *mdl_request, bool *conflict);
bool acquire_global_shared_lock();
bool wait_for_locks();
void release_all_locks();
void release_all_locks_for_name(MDL_ticket *ticket);
void release_lock(MDL_ticket *ticket);
void release_global_shared_lock();
bool is_exclusive_lock_owner(unsigned char type,
const char *db,
const char *name);
bool is_lock_owner(unsigned char type, const char *db, const char *name);
inline bool has_locks() const
{
return !m_tickets.is_empty();
}
inline MDL_ticket *mdl_savepoint()
{
return m_tickets.head();
}
void rollback_to_savepoint(MDL_ticket *mdl_savepoint);
/**
Get iterator for walking through all lock requests in the context.
*/
inline Request_iterator get_requests()
{
return Request_iterator(m_requests);
}
inline THD *get_thd() const { return m_thd; }
private:
Request_list m_requests;
Ticket_list m_tickets;
bool m_has_global_shared_lock;
THD *m_thd;
private:
void release_ticket(MDL_ticket *ticket);
MDL_ticket *find_ticket(MDL_request *mdl_req);
};
void mdl_init();
void mdl_destroy();
void mdl_context_init(MDL_CONTEXT *context, THD *thd);
void mdl_context_destroy(MDL_CONTEXT *context);
void mdl_context_backup_and_reset(MDL_CONTEXT *ctx, MDL_CONTEXT *backup);
void mdl_context_restore(MDL_CONTEXT *ctx, MDL_CONTEXT *backup);
void mdl_context_merge(MDL_CONTEXT *target, MDL_CONTEXT *source);
void mdl_request_init(MDL_LOCK_REQUEST *lock_req, unsigned char type,
const char *db, const char *name);
MDL_LOCK_REQUEST *mdl_request_alloc(unsigned char type, const char *db,
const char *name, MEM_ROOT *root);
void mdl_request_add(MDL_CONTEXT *context, MDL_LOCK_REQUEST *lock_req);
void mdl_request_remove(MDL_CONTEXT *context, MDL_LOCK_REQUEST *lock_req);
void mdl_request_remove_all(MDL_CONTEXT *context);
/**
Set type of lock request. Can be only applied to pending locks.
*/
inline void mdl_request_set_type(MDL_LOCK_REQUEST *lock_req, enum_mdl_type lock_type)
{
DBUG_ASSERT(lock_req->ticket == NULL);
lock_req->type= lock_type;
}
bool mdl_acquire_shared_lock(MDL_CONTEXT *context, MDL_LOCK_REQUEST *lock_req,
bool *retry);
bool mdl_acquire_exclusive_locks(MDL_CONTEXT *context);
bool mdl_upgrade_shared_lock_to_exclusive(MDL_CONTEXT *context,
MDL_LOCK_TICKET *ticket);
bool mdl_try_acquire_exclusive_lock(MDL_CONTEXT *context,
MDL_LOCK_REQUEST *lock_req,
bool *conflict);
bool mdl_acquire_global_shared_lock(MDL_CONTEXT *context);
bool mdl_wait_for_locks(MDL_CONTEXT *context);
void mdl_ticket_release_all(MDL_CONTEXT *context);
void mdl_ticket_release_all_for_name(MDL_CONTEXT *context,
MDL_LOCK_TICKET *ticket);
void mdl_ticket_release(MDL_CONTEXT *context, MDL_LOCK_TICKET *ticket);
void mdl_downgrade_exclusive_lock(MDL_CONTEXT *context,
MDL_LOCK_TICKET *ticket);
void mdl_release_global_shared_lock(MDL_CONTEXT *context);
bool mdl_is_exclusive_lock_owner(MDL_CONTEXT *context, unsigned char type,
const char *db, const char *name);
bool mdl_is_lock_owner(MDL_CONTEXT *context, unsigned char type,
const char *db, const char *name);
bool mdl_has_pending_conflicting_lock(MDL_LOCK_TICKET *ticket);
inline bool mdl_has_locks(MDL_CONTEXT *context)
{
return !context->tickets.is_empty();
}
inline MDL_LOCK_TICKET *mdl_savepoint(MDL_CONTEXT *ctx)
{
return ctx->tickets.head();
}
void mdl_rollback_to_savepoint(MDL_CONTEXT *ctx,
MDL_LOCK_TICKET *mdl_savepoint);
/**
Get iterator for walking through all lock requests in the context.
*/
inline MDL_CONTEXT::Request_iterator
mdl_get_requests(MDL_CONTEXT *ctx)
{
MDL_CONTEXT::Request_iterator result(ctx->requests);
return result;
}
void mdl_get_tdc_key(MDL_LOCK_TICKET *ticket, LEX_STRING *key);
typedef void (* mdl_cached_object_release_hook)(void *);
void *mdl_get_cached_object(MDL_LOCK_TICKET *ticket);
void mdl_set_cached_object(MDL_LOCK_TICKET *ticket, void *cached_object,
mdl_cached_object_release_hook release_hook);
/*
Functions in the server's kernel used by metadata locking subsystem.

View File

@ -1205,7 +1205,7 @@ void Relay_log_info::clear_tables_to_lock()
meta-data locks are stored. So we want to be sure that we don't have
any references to this memory left.
*/
DBUG_ASSERT(!mdl_has_locks(&(current_thd->mdl_context)));
DBUG_ASSERT(!current_thd->mdl_context.has_locks());
while (tables_to_lock)
{

View File

@ -3981,10 +3981,10 @@ sp_head::add_used_tables_to_table_list(THD *thd,
table->prelocking_placeholder= 1;
table->belong_to_view= belong_to_view;
table->trg_event_map= stab->trg_event_map;
table->mdl_lock_request= mdl_request_alloc(0, table->db, table->table_name,
thd->locked_tables_root ?
thd->locked_tables_root :
thd->mem_root);
table->mdl_request= MDL_request::create(0, table->db, table->table_name,
thd->locked_tables_root ?
thd->locked_tables_root :
thd->mem_root);
/* Everyting else should be zeroed */
@ -4026,10 +4026,10 @@ sp_add_to_query_tables(THD *thd, LEX *lex,
table->lock_type= locktype;
table->select_lex= lex->current_select;
table->cacheable_table= 1;
table->mdl_lock_request= mdl_request_alloc(0, table->db, table->table_name,
thd->locked_tables_root ?
thd->locked_tables_root :
thd->mem_root);
table->mdl_request= MDL_request::create(0, table->db, table->table_name,
thd->locked_tables_root ?
thd->locked_tables_root :
thd->mem_root);
lex->add_to_query_tables(table);
return table;

View File

@ -124,7 +124,7 @@ static bool open_new_frm(THD *thd, TABLE_SHARE *share, const char *alias,
uint db_stat, uint prgflag,
uint ha_open_flags, TABLE *outparam,
TABLE_LIST *table_desc, MEM_ROOT *mem_root);
static bool tdc_wait_for_old_versions(THD *thd, MDL_CONTEXT *context);
static bool tdc_wait_for_old_versions(THD *thd, MDL_context *context);
static bool
has_write_table_with_auto_increment(TABLE_LIST *tables);
@ -449,8 +449,8 @@ TABLE_SHARE *get_table_share(THD *thd, TABLE_LIST *table_list, char *key,
To be able perform any operation on table we should own
some kind of metadata lock on it.
*/
DBUG_ASSERT(mdl_is_lock_owner(&thd->mdl_context, 0, table_list->db,
table_list->table_name));
DBUG_ASSERT(thd->mdl_context.is_lock_owner(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,
@ -1050,12 +1050,12 @@ err_with_reopen:
*/
thd->locked_tables_list.reopen_tables(thd);
/*
Since mdl_downgrade_exclusive_lock() won't do anything with shared
metadata lock it is much simplier to go through all open tables rather
Since downgrade_exclusive_lock() won't do anything with shared
metadata lock it is much simpler to go through all open tables rather
than picking only those tables that were flushed.
*/
for (TABLE *tab= thd->open_tables; tab; tab= tab->next)
mdl_downgrade_exclusive_lock(&thd->mdl_context, tab->mdl_lock_ticket);
tab->mdl_ticket->downgrade_exclusive_lock();
}
DBUG_RETURN(result);
}
@ -1336,7 +1336,7 @@ close_all_tables_for_name(THD *thd, TABLE_SHARE *share,
*/
void close_thread_tables(THD *thd,
bool skip_mdl)
bool is_back_off)
{
TABLE *table;
DBUG_ENTER("close_thread_tables");
@ -1478,10 +1478,10 @@ void close_thread_tables(THD *thd,
if (thd->open_tables)
close_open_tables(thd);
mdl_ticket_release_all(&thd->mdl_context);
if (!skip_mdl)
thd->mdl_context.release_all_locks();
if (!is_back_off)
{
mdl_request_remove_all(&thd->mdl_context);
thd->mdl_context.remove_all_requests();
}
DBUG_VOID_RETURN;
}
@ -1500,7 +1500,7 @@ bool close_thread_table(THD *thd, TABLE **table_ptr)
*table_ptr=table->next;
table->mdl_lock_ticket= NULL;
table->mdl_ticket= NULL;
if (table->needs_reopen() ||
thd->version != refresh_version || !table->db_stat)
{
@ -2095,8 +2095,7 @@ bool wait_while_table_is_used(THD *thd, TABLE *table,
old_lock_type= table->reginfo.lock_type;
mysql_lock_abort(thd, table, TRUE); /* end threads waiting on lock */
if (mdl_upgrade_shared_lock_to_exclusive(&thd->mdl_context,
table->mdl_lock_ticket))
if (table->mdl_ticket->upgrade_shared_lock_to_exclusive())
{
mysql_lock_downgrade_write(thd, table, old_lock_type);
DBUG_RETURN(TRUE);
@ -2279,11 +2278,11 @@ void table_share_release_hook(void *share)
static bool
open_table_get_mdl_lock(THD *thd, TABLE_LIST *table_list,
MDL_LOCK_REQUEST *mdl_lock_request,
MDL_request *mdl_request,
uint flags,
enum_open_table_action *action)
{
mdl_request_add(&thd->mdl_context, mdl_lock_request);
thd->mdl_context.add_request(mdl_request);
if (table_list->open_type)
{
@ -2296,10 +2295,10 @@ open_table_get_mdl_lock(THD *thd, TABLE_LIST *table_list,
shared locks. This invariant is preserved here and is also
enforced by asserts in metadata locking subsystem.
*/
mdl_request_set_type(mdl_lock_request, MDL_EXCLUSIVE);
if (mdl_acquire_exclusive_locks(&thd->mdl_context))
mdl_request->set_type(MDL_EXCLUSIVE);
if (thd->mdl_context.acquire_exclusive_locks())
{
mdl_request_remove(&thd->mdl_context, mdl_lock_request);
thd->mdl_context.remove_request(mdl_request);
return 1;
}
}
@ -2316,16 +2315,16 @@ open_table_get_mdl_lock(THD *thd, TABLE_LIST *table_list,
if (flags & MYSQL_OPEN_TAKE_UPGRADABLE_MDL &&
table_list->lock_type >= TL_WRITE_ALLOW_WRITE)
mdl_request_set_type(mdl_lock_request, MDL_SHARED_UPGRADABLE);
mdl_request->set_type(MDL_SHARED_UPGRADABLE);
if (flags & MYSQL_LOCK_IGNORE_FLUSH)
mdl_request_set_type(mdl_lock_request, MDL_SHARED_HIGH_PRIO);
mdl_request->set_type(MDL_SHARED_HIGH_PRIO);
if (mdl_acquire_shared_lock(&thd->mdl_context, mdl_lock_request, &retry))
if (thd->mdl_context.acquire_shared_lock(mdl_request, &retry))
{
if (retry)
*action= OT_BACK_OFF_AND_RETRY;
else
mdl_request_remove(&thd->mdl_context, mdl_lock_request);
thd->mdl_context.remove_request(mdl_request);
return 1;
}
}
@ -2380,8 +2379,8 @@ bool open_table(THD *thd, TABLE_LIST *table_list, MEM_ROOT *mem_root,
char key[MAX_DBKEY_LENGTH];
uint key_length;
char *alias= table_list->alias;
MDL_LOCK_REQUEST *mdl_lock_request;
MDL_LOCK_TICKET *mdl_lock_ticket;
MDL_request *mdl_request;
MDL_ticket *mdl_ticket;
int error;
TABLE_SHARE *share;
DBUG_ENTER("open_table");
@ -2517,8 +2516,8 @@ bool open_table(THD *thd, TABLE_LIST *table_list, MEM_ROOT *mem_root,
TABLES breaks metadata locking protocol (potentially can lead
to deadlocks) it should be disallowed.
*/
if (mdl_is_lock_owner(&thd->mdl_context, 0, table_list->db,
table_list->table_name))
if (thd->mdl_context.is_lock_owner(0, table_list->db,
table_list->table_name))
{
char path[FN_REFLEN + 1];
enum legacy_db_type not_used;
@ -2560,10 +2559,10 @@ bool open_table(THD *thd, TABLE_LIST *table_list, MEM_ROOT *mem_root,
This is the normal use case.
*/
mdl_lock_request= table_list->mdl_lock_request;
mdl_request= table_list->mdl_request;
if (! (flags & MYSQL_OPEN_HAS_MDL_LOCK))
{
if (open_table_get_mdl_lock(thd, table_list, mdl_lock_request, flags,
if (open_table_get_mdl_lock(thd, table_list, mdl_request, flags,
action))
DBUG_RETURN(TRUE);
}
@ -2573,7 +2572,7 @@ bool open_table(THD *thd, TABLE_LIST *table_list, MEM_ROOT *mem_root,
open_table_get_mdl_lock as the lock on the table might have been
acquired previously (MYSQL_OPEN_HAS_MDL_LOCK).
*/
mdl_lock_ticket= mdl_lock_request->ticket;
mdl_ticket= mdl_request->ticket;
pthread_mutex_lock(&LOCK_open);
@ -2616,7 +2615,7 @@ bool open_table(THD *thd, TABLE_LIST *table_list, MEM_ROOT *mem_root,
DBUG_RETURN(FALSE);
}
if (!(share= (TABLE_SHARE *)mdl_get_cached_object(mdl_lock_ticket)))
if (!(share= (TABLE_SHARE *) mdl_ticket->get_cached_object()))
{
if (!(share= get_table_share_with_create(thd, table_list, key,
key_length, OPEN_VIEW,
@ -2687,7 +2686,7 @@ bool open_table(THD *thd, TABLE_LIST *table_list, MEM_ROOT *mem_root,
so we need to increase reference counter;
*/
reference_table_share(share);
mdl_set_cached_object(mdl_lock_ticket, share, table_share_release_hook);
mdl_ticket->set_cached_object(share, table_share_release_hook);
}
else
{
@ -2796,9 +2795,9 @@ bool open_table(THD *thd, TABLE_LIST *table_list, MEM_ROOT *mem_root,
lock on this table to shared metadata lock.
*/
if (table_list->open_type == TABLE_LIST::OPEN_OR_CREATE)
mdl_downgrade_exclusive_lock(&thd->mdl_context, mdl_lock_ticket);
mdl_ticket->downgrade_exclusive_lock();
table->mdl_lock_ticket= mdl_lock_ticket;
table->mdl_ticket= mdl_ticket;
table->next=thd->open_tables; /* Link into simple list */
thd->open_tables=table;
@ -2850,8 +2849,8 @@ err_unlock2:
pthread_mutex_unlock(&LOCK_open);
if (! (flags & MYSQL_OPEN_HAS_MDL_LOCK))
{
mdl_ticket_release(&thd->mdl_context, mdl_lock_ticket);
mdl_request_remove(&thd->mdl_context, mdl_lock_request);
thd->mdl_context.release_lock(mdl_ticket);
thd->mdl_context.remove_request(mdl_request);
}
DBUG_RETURN(TRUE);
}
@ -2969,7 +2968,7 @@ Locked_tables_list::init_locked_tables(THD *thd)
dst_table_list->init_one_table(db, db_len, table_name, table_name_len,
alias,
src_table_list->table->reginfo.lock_type);
dst_table_list->mdl_lock_request= src_table_list->mdl_lock_request;
dst_table_list->mdl_request= src_table_list->mdl_request;
dst_table_list->table= table;
memcpy(db, src_table_list->db, db_len + 1);
memcpy(table_name, src_table_list->table_name, table_name_len + 1);
@ -3020,8 +3019,6 @@ Locked_tables_list::unlock_locked_tables(THD *thd)
thd->locked_tables_mode= LTM_NONE;
close_thread_tables(thd);
mdl_ticket_release_all(&thd->mdl_context);
}
/*
After closing tables we can free memory used for storing lock
@ -3506,21 +3503,21 @@ recover_from_failed_open_table_attempt(THD *thd, TABLE_LIST *table,
enum_open_table_action action)
{
bool result= FALSE;
MDL_LOCK_REQUEST *mdl_lock_request= table->mdl_lock_request;
MDL_request *mdl_request= table->mdl_request;
switch (action)
{
case OT_BACK_OFF_AND_RETRY:
result= (mdl_wait_for_locks(&thd->mdl_context) ||
result= (thd->mdl_context.wait_for_locks() ||
tdc_wait_for_old_versions(thd, &thd->mdl_context));
mdl_request_remove_all(&thd->mdl_context);
thd->mdl_context.remove_all_requests();
break;
case OT_DISCOVER:
mdl_request_set_type(mdl_lock_request, MDL_EXCLUSIVE);
mdl_request_add(&thd->mdl_context, mdl_lock_request);
if (mdl_acquire_exclusive_locks(&thd->mdl_context))
mdl_request->set_type(MDL_EXCLUSIVE);
thd->mdl_context.add_request(mdl_request);
if (thd->mdl_context.acquire_exclusive_locks())
{
mdl_request_remove(&thd->mdl_context, mdl_lock_request);
thd->mdl_context.remove_request(mdl_request);
return TRUE;
}
pthread_mutex_lock(&LOCK_open);
@ -3530,15 +3527,15 @@ recover_from_failed_open_table_attempt(THD *thd, TABLE_LIST *table,
thd->warning_info->clear_warning_info(thd->query_id);
thd->clear_error(); // Clear error message
mdl_ticket_release(&thd->mdl_context, mdl_lock_request->ticket);
mdl_request_remove(&thd->mdl_context, mdl_lock_request);
thd->mdl_context.release_lock(mdl_request->ticket);
thd->mdl_context.remove_request(mdl_request);
break;
case OT_REPAIR:
mdl_request_set_type(mdl_lock_request, MDL_EXCLUSIVE);
mdl_request_add(&thd->mdl_context, mdl_lock_request);
if (mdl_acquire_exclusive_locks(&thd->mdl_context))
mdl_request->set_type(MDL_EXCLUSIVE);
thd->mdl_context.add_request(mdl_request);
if (thd->mdl_context.acquire_exclusive_locks())
{
mdl_request_remove(&thd->mdl_context, mdl_lock_request);
thd->mdl_context.remove_request(mdl_request);
return TRUE;
}
pthread_mutex_lock(&LOCK_open);
@ -3546,8 +3543,8 @@ recover_from_failed_open_table_attempt(THD *thd, TABLE_LIST *table,
pthread_mutex_unlock(&LOCK_open);
result= auto_repair_table(thd, table);
mdl_ticket_release(&thd->mdl_context, mdl_lock_request->ticket);
mdl_request_remove(&thd->mdl_context, mdl_lock_request);
thd->mdl_context.release_lock(mdl_request->ticket);
thd->mdl_context.remove_request(mdl_request);
break;
default:
DBUG_ASSERT(0);
@ -4652,7 +4649,7 @@ int lock_tables(THD *thd, TABLE_LIST *tables, uint count,
*/
void close_tables_for_reopen(THD *thd, TABLE_LIST **tables, bool skip_mdl)
void close_tables_for_reopen(THD *thd, TABLE_LIST **tables, bool is_back_off)
{
/*
If table list consists only from tables from prelocking set, table list
@ -4664,7 +4661,7 @@ void close_tables_for_reopen(THD *thd, TABLE_LIST **tables, bool skip_mdl)
sp_remove_not_own_routines(thd->lex);
for (TABLE_LIST *tmp= *tables; tmp; tmp= tmp->next_global)
tmp->table= 0;
close_thread_tables(thd, skip_mdl);
close_thread_tables(thd, is_back_off);
}
@ -7691,8 +7688,7 @@ void tdc_remove_table(THD *thd, enum_tdc_remove_table_type remove_type,
safe_mutex_assert_owner(&LOCK_open);
DBUG_ASSERT(remove_type == TDC_RT_REMOVE_UNUSED ||
mdl_is_exclusive_lock_owner(&thd->mdl_context, 0,
db, table_name));
thd->mdl_context.is_exclusive_lock_owner(0, db, table_name));
key_length=(uint) (strmov(strmov(key,db)+1,table_name)-key)+1;
@ -7739,11 +7735,11 @@ void tdc_remove_table(THD *thd, enum_tdc_remove_table_type remove_type,
@param context Metadata locking context with locks.
*/
static bool tdc_wait_for_old_versions(THD *thd, MDL_CONTEXT *context)
static bool tdc_wait_for_old_versions(THD *thd, MDL_context *mdl_context)
{
TABLE_SHARE *share;
const char *old_msg;
MDL_LOCK_REQUEST *lock_req;
MDL_request *mdl_request;
while (!thd->killed)
{
@ -7756,16 +7752,16 @@ static bool tdc_wait_for_old_versions(THD *thd, MDL_CONTEXT *context)
mysql_ha_flush(thd);
pthread_mutex_lock(&LOCK_open);
MDL_CONTEXT::Request_iterator it= mdl_get_requests(context);
while ((lock_req= it++))
MDL_context::Request_iterator it= mdl_context->get_requests();
while ((mdl_request= it++))
{
if ((share= get_cached_table_share(lock_req->key.db_name(),
lock_req->key.table_name())) &&
if ((share= get_cached_table_share(mdl_request->key.db_name(),
mdl_request->key.table_name())) &&
share->version != refresh_version &&
!share->used_tables.is_empty())
break;
}
if (!lock_req)
if (!mdl_request)
{
pthread_mutex_unlock(&LOCK_open);
break;
@ -8169,8 +8165,8 @@ void close_performance_schema_table(THD *thd, Open_tables_state *backup)
pthread_mutex_unlock(&LOCK_open);
mdl_ticket_release_all(&thd->mdl_context);
mdl_request_remove_all(&thd->mdl_context);
thd->mdl_context.release_all_locks();
thd->mdl_context.remove_all_requests();
thd->restore_backup_open_tables_state(backup);
}

View File

@ -1050,8 +1050,8 @@ THD::~THD()
if (!cleanup_done)
cleanup();
mdl_context_destroy(&mdl_context);
mdl_context_destroy(&handler_mdl_context);
mdl_context.destroy();
handler_mdl_context.destroy();
ha_close_connection(this);
plugin_thdvar_cleanup(this);
@ -3033,8 +3033,8 @@ void THD::restore_backup_open_tables_state(Open_tables_state *backup)
lock == 0 &&
locked_tables_mode == LTM_NONE &&
m_reprepare_observer == NULL);
mdl_context_destroy(&mdl_context);
mdl_context_destroy(&handler_mdl_context);
mdl_context.destroy();
handler_mdl_context.destroy();
set_open_tables_state(backup);
DBUG_VOID_RETURN;

View File

@ -780,7 +780,7 @@ struct st_savepoint {
uint length;
Ha_trx_info *ha_list;
/** Last acquired lock before this savepoint was set. */
MDL_LOCK_TICKET *mdl_savepoint;
MDL_ticket *mdl_savepoint;
};
enum xa_states {XA_NOTR=0, XA_ACTIVE, XA_IDLE, XA_PREPARED, XA_ROLLBACK_ONLY};
@ -983,8 +983,8 @@ public:
*/
uint state_flags;
MDL_CONTEXT mdl_context;
MDL_CONTEXT handler_mdl_context;
MDL_context mdl_context;
MDL_context handler_mdl_context;
/**
This constructor initializes Open_tables_state instance which can only
@ -1015,8 +1015,8 @@ public:
locked_tables_mode= LTM_NONE;
state_flags= 0U;
m_reprepare_observer= NULL;
mdl_context_init(&mdl_context, thd);
mdl_context_init(&handler_mdl_context, thd);
mdl_context.init(thd);
handler_mdl_context.init(thd);
}
};

View File

@ -1100,7 +1100,7 @@ bool mysql_truncate(THD *thd, TABLE_LIST *table_list, bool dont_send_ok)
TABLE *table;
bool error;
uint path_length;
MDL_LOCK_REQUEST *mdl_lock_request= NULL;
MDL_request *mdl_request= NULL;
DBUG_ENTER("mysql_truncate");
bzero((char*) &create_info,sizeof(create_info));
@ -1175,13 +1175,13 @@ bool mysql_truncate(THD *thd, TABLE_LIST *table_list, bool dont_send_ok)
tries to get table enging and therefore accesses table in some way
without holding any kind of meta-data lock.
*/
mdl_lock_request= mdl_request_alloc(0, table_list->db,
table_list->table_name, thd->mem_root);
mdl_request_set_type(mdl_lock_request, MDL_EXCLUSIVE);
mdl_request_add(&thd->mdl_context, mdl_lock_request);
if (mdl_acquire_exclusive_locks(&thd->mdl_context))
mdl_request= MDL_request::create(0, table_list->db,
table_list->table_name, thd->mem_root);
mdl_request->set_type(MDL_EXCLUSIVE);
thd->mdl_context.add_request(mdl_request);
if (thd->mdl_context.acquire_exclusive_locks())
{
mdl_request_remove(&thd->mdl_context, mdl_lock_request);
thd->mdl_context.remove_request(mdl_request);
DBUG_RETURN(TRUE);
}
pthread_mutex_lock(&LOCK_open);
@ -1212,18 +1212,18 @@ end:
write_bin_log(thd, TRUE, thd->query(), thd->query_length());
my_ok(thd); // This should return record count
}
if (mdl_lock_request)
if (mdl_request)
{
mdl_ticket_release(&thd->mdl_context, mdl_lock_request->ticket);
mdl_request_remove(&thd->mdl_context, mdl_lock_request);
thd->mdl_context.release_lock(mdl_request->ticket);
thd->mdl_context.remove_request(mdl_request);
}
}
else if (error)
{
if (mdl_lock_request)
if (mdl_request)
{
mdl_ticket_release(&thd->mdl_context, mdl_lock_request->ticket);
mdl_request_remove(&thd->mdl_context, mdl_lock_request);
thd->mdl_context.release_lock(mdl_request->ticket);
thd->mdl_context.remove_request(mdl_request);
}
}
DBUG_RETURN(error);

View File

@ -125,7 +125,7 @@ static void mysql_ha_hash_free(TABLE_LIST *tables)
static void mysql_ha_close_table(THD *thd, TABLE_LIST *tables)
{
TABLE **table_ptr;
MDL_LOCK_TICKET *mdl_lock_ticket;
MDL_ticket *mdl_ticket;
/*
Though we could take the table pointer from hash_tables->table,
@ -141,7 +141,7 @@ static void mysql_ha_close_table(THD *thd, TABLE_LIST *tables)
if (*table_ptr)
{
(*table_ptr)->file->ha_index_or_rnd_end();
mdl_lock_ticket= (*table_ptr)->mdl_lock_ticket;
mdl_ticket= (*table_ptr)->mdl_ticket;
pthread_mutex_lock(&LOCK_open);
if (close_thread_table(thd, table_ptr))
{
@ -149,8 +149,8 @@ static void mysql_ha_close_table(THD *thd, TABLE_LIST *tables)
broadcast_refresh();
}
pthread_mutex_unlock(&LOCK_open);
mdl_ticket_release(&thd->handler_mdl_context, mdl_lock_ticket);
mdl_request_remove(&thd->handler_mdl_context, tables->mdl_lock_request);
thd->handler_mdl_context.release_lock(mdl_ticket);
thd->handler_mdl_context.remove_request(tables->mdl_request);
}
else if (tables->table)
{
@ -194,8 +194,8 @@ bool mysql_ha_open(THD *thd, TABLE_LIST *tables, bool reopen)
uint dblen, namelen, aliaslen, counter;
int error;
TABLE *backup_open_tables;
MDL_CONTEXT backup_mdl_context;
MDL_LOCK_REQUEST *mdl_lock_request;
MDL_context backup_mdl_context;
MDL_request *mdl_request;
DBUG_ENTER("mysql_ha_open");
DBUG_PRINT("enter",("'%s'.'%s' as '%s' reopen: %d",
tables->db, tables->table_name, tables->alias,
@ -246,7 +246,7 @@ bool mysql_ha_open(THD *thd, TABLE_LIST *tables, bool reopen)
&db, (uint) dblen,
&name, (uint) namelen,
&alias, (uint) aliaslen,
&mdl_lock_request, sizeof(MDL_LOCK_REQUEST),
&mdl_request, sizeof(MDL_request),
NullS)))
{
DBUG_PRINT("exit",("ERROR"));
@ -260,8 +260,8 @@ bool mysql_ha_open(THD *thd, TABLE_LIST *tables, bool reopen)
memcpy(hash_tables->db, tables->db, dblen);
memcpy(hash_tables->table_name, tables->table_name, namelen);
memcpy(hash_tables->alias, tables->alias, aliaslen);
mdl_request_init(mdl_lock_request, 0, db, name);
hash_tables->mdl_lock_request= mdl_lock_request;
mdl_request->init(0, db, name);
hash_tables->mdl_request= mdl_request;
/* add to hash */
if (my_hash_insert(&thd->handler_tables_hash, (uchar*) hash_tables))
@ -289,7 +289,7 @@ bool mysql_ha_open(THD *thd, TABLE_LIST *tables, bool reopen)
*/
backup_open_tables= thd->open_tables;
thd->open_tables= NULL;
mdl_context_backup_and_reset(&thd->mdl_context, &backup_mdl_context);
thd->mdl_context.backup_and_reset(&backup_mdl_context);
/*
open_tables() will set 'hash_tables->table' if successful.
@ -328,10 +328,10 @@ bool mysql_ha_open(THD *thd, TABLE_LIST *tables, bool reopen)
thd->handler_tables= thd->open_tables;
}
}
mdl_context_merge(&thd->handler_mdl_context, &thd->mdl_context);
thd->handler_mdl_context.merge(&thd->mdl_context);
thd->open_tables= backup_open_tables;
mdl_context_restore(&thd->mdl_context, &backup_mdl_context);
thd->mdl_context.restore_from_backup(&backup_mdl_context);
if (error)
goto err;
@ -800,11 +800,11 @@ void mysql_ha_flush(THD *thd)
{
hash_tables= (TABLE_LIST*) my_hash_element(&thd->handler_tables_hash, i);
/*
TABLE::mdl_lock_ticket is 0 for temporary tables so we need extra check.
TABLE::mdl_ticket is 0 for temporary tables so we need extra check.
*/
if (hash_tables->table &&
(hash_tables->table->mdl_lock_ticket &&
mdl_has_pending_conflicting_lock(hash_tables->table->mdl_lock_ticket) ||
(hash_tables->table->mdl_ticket &&
hash_tables->table->mdl_ticket->has_pending_conflicting_lock() ||
hash_tables->table->needs_reopen()))
mysql_ha_close_table(thd, hash_tables);
}

View File

@ -5977,9 +5977,9 @@ TABLE_LIST *st_select_lex::add_table_to_list(THD *thd,
ptr->next_name_resolution_table= NULL;
/* Link table in global list (all used tables) */
lex->add_to_query_tables(ptr);
ptr->mdl_lock_request=
mdl_request_alloc(0, ptr->db, ptr->table_name, thd->locked_tables_root ?
thd->locked_tables_root : thd->mem_root);
ptr->mdl_request=
MDL_request::create(0, ptr->db, ptr->table_name, thd->locked_tables_root ?
thd->locked_tables_root : thd->mem_root);
DBUG_RETURN(ptr);
}

View File

@ -79,6 +79,7 @@ public:
**B::prev_ptr(a)= next;
}
inline T* head() { return first; }
inline const T *head() const { return first; }
void swap(I_P_List<T,B> &rhs)
{
swap_variables(T *, first, rhs.first);

View File

@ -3050,7 +3050,7 @@ uint get_table_open_method(TABLE_LIST *tables,
Acquire high priority share metadata lock on a table.
@param thd Thread context.
@param mdl_lock_req Pointer to memory to be used for MDL_LOCK_REQUEST
@param mdl_request Pointer to memory to be used for MDL_request
object for a lock request.
@param table Table list element for the table
@ -3065,23 +3065,23 @@ uint get_table_open_method(TABLE_LIST *tables,
*/
static bool
acquire_high_prio_shared_mdl_lock(THD *thd, MDL_LOCK_REQUEST *mdl_lock_req,
acquire_high_prio_shared_mdl_lock(THD *thd, MDL_request *mdl_request,
TABLE_LIST *table)
{
bool retry;
mdl_request_init(mdl_lock_req, 0, table->db, table->table_name);
table->mdl_lock_request= mdl_lock_req;
mdl_request_add(&thd->mdl_context, mdl_lock_req);
mdl_request_set_type(mdl_lock_req, MDL_SHARED_HIGH_PRIO);
mdl_request->init(0, table->db, table->table_name);
table->mdl_request= mdl_request;
thd->mdl_context.add_request(mdl_request);
mdl_request->set_type(MDL_SHARED_HIGH_PRIO);
while (1)
{
if (mdl_acquire_shared_lock(&thd->mdl_context, mdl_lock_req, &retry))
if (thd->mdl_context.acquire_shared_lock(mdl_request, &retry))
{
if (!retry || mdl_wait_for_locks(&thd->mdl_context))
if (!retry || thd->mdl_context.wait_for_locks())
{
mdl_request_remove_all(&thd->mdl_context);
thd->mdl_context.remove_all_requests();
return TRUE;
}
continue;
@ -3123,7 +3123,7 @@ static int fill_schema_table_from_frm(THD *thd,TABLE *table,
char key[MAX_DBKEY_LENGTH];
uint key_length;
char db_name_buff[NAME_LEN + 1], table_name_buff[NAME_LEN + 1];
MDL_LOCK_REQUEST mdl_lock_request;
MDL_request mdl_request;
bzero((char*) &table_list, sizeof(TABLE_LIST));
bzero((char*) &tbl, sizeof(TABLE));
@ -3153,7 +3153,7 @@ static int fill_schema_table_from_frm(THD *thd,TABLE *table,
simply obtaining internal lock of data-dictionary (ATM it
is LOCK_open) instead of obtaning full-blown metadata lock.
*/
if (acquire_high_prio_shared_mdl_lock(thd, &mdl_lock_request, &table_list))
if (acquire_high_prio_shared_mdl_lock(thd, &mdl_request, &table_list))
{
/*
Some error occured (most probably we have been killed while
@ -3213,9 +3213,8 @@ err_share:
err_unlock:
pthread_mutex_unlock(&LOCK_open);
err:
mdl_ticket_release(&thd->mdl_context, mdl_lock_request.ticket);
mdl_request_remove(&thd->mdl_context, &mdl_lock_request);
thd->mdl_context.release_lock(mdl_request.ticket);
thd->mdl_context.remove_request(&mdl_request);
thd->clear_error();
return res;
}

View File

@ -1910,7 +1910,7 @@ int mysql_rm_table_part2(THD *thd, TABLE_LIST *tables, bool if_exists,
Since we don't acquire metadata lock if we have found temporary
table, we should do something to avoid releasing it at the end.
*/
table->mdl_lock_request= NULL;
table->mdl_request= NULL;
}
else
{
@ -1923,7 +1923,7 @@ int mysql_rm_table_part2(THD *thd, TABLE_LIST *tables, bool if_exists,
table->table_name);
if (!table->table)
DBUG_RETURN(1);
table->mdl_lock_request->ticket= table->table->mdl_lock_ticket;
table->mdl_request->ticket= table->table->mdl_ticket;
}
}
}
@ -2202,15 +2202,14 @@ err:
}
for (table= tables; table; table= table->next_local)
{
if (table->mdl_lock_request)
if (table->mdl_request)
{
/*
Under LOCK TABLES we may have several instances of table open
and locked and therefore have to remove several metadata lock
requests associated with them.
*/
mdl_ticket_release_all_for_name(&thd->mdl_context,
table->mdl_lock_request->ticket);
thd->mdl_context.release_all_locks_for_name(table->mdl_request->ticket);
}
}
}
@ -4108,28 +4107,28 @@ warn:
static bool lock_table_name_if_not_cached(THD *thd, const char *db,
const char *table_name,
MDL_LOCK_REQUEST **lock_req)
MDL_request **mdl_request)
{
bool conflict;
if (!(*lock_req= mdl_request_alloc(0, db, table_name, thd->mem_root)))
if (!(*mdl_request= MDL_request::create(0, db, table_name, thd->mem_root)))
return TRUE;
mdl_request_set_type(*lock_req, MDL_EXCLUSIVE);
mdl_request_add(&thd->mdl_context, *lock_req);
if (mdl_try_acquire_exclusive_lock(&thd->mdl_context, *lock_req, &conflict))
(*mdl_request)->set_type(MDL_EXCLUSIVE);
thd->mdl_context.add_request(*mdl_request);
if (thd->mdl_context.try_acquire_exclusive_lock(*mdl_request, &conflict))
{
/*
To simplify our life under LOCK TABLES we remove unsatisfied
lock request from the context.
*/
mdl_request_remove(&thd->mdl_context, *lock_req);
thd->mdl_context.remove_request(*mdl_request);
if (!conflict)
{
/* Probably OOM. */
return TRUE;
}
else
*lock_req= NULL;
*mdl_request= NULL;
}
return FALSE;
}
@ -4145,7 +4144,7 @@ bool mysql_create_table(THD *thd, const char *db, const char *table_name,
bool internal_tmp_table,
uint select_field_count)
{
MDL_LOCK_REQUEST *target_lock_req= NULL;
MDL_request *target_mdl_request= NULL;
bool result;
DBUG_ENTER("mysql_create_table");
@ -4168,12 +4167,12 @@ bool mysql_create_table(THD *thd, const char *db, const char *table_name,
if (!(create_info->options & HA_LEX_CREATE_TMP_TABLE))
{
if (lock_table_name_if_not_cached(thd, db, table_name, &target_lock_req))
if (lock_table_name_if_not_cached(thd, db, table_name, &target_mdl_request))
{
result= TRUE;
goto unlock;
}
if (!target_lock_req)
if (!target_mdl_request)
{
if (create_info->options & HA_LEX_CREATE_IF_NOT_EXISTS)
{
@ -4199,10 +4198,10 @@ bool mysql_create_table(THD *thd, const char *db, const char *table_name,
select_field_count);
unlock:
if (target_lock_req)
if (target_mdl_request)
{
mdl_ticket_release(&thd->mdl_context, target_lock_req->ticket);
mdl_request_remove(&thd->mdl_context, target_lock_req);
thd->mdl_context.release_lock(target_mdl_request->ticket);
thd->mdl_context.remove_request(target_mdl_request);
}
pthread_mutex_lock(&LOCK_lock_db);
if (!--creating_table && creating_database)
@ -4367,7 +4366,7 @@ static int prepare_for_repair(THD *thd, TABLE_LIST *table_list,
char from[FN_REFLEN],tmp[FN_REFLEN+32];
const char **ext;
MY_STAT stat_info;
MDL_LOCK_REQUEST *mdl_lock_request= NULL;
MDL_request *mdl_request= NULL;
enum enum_open_table_action ot_action_unused;
DBUG_ENTER("prepare_for_repair");
uint reopen_for_repair_flags= (MYSQL_LOCK_IGNORE_FLUSH |
@ -4386,13 +4385,13 @@ static int prepare_for_repair(THD *thd, TABLE_LIST *table_list,
uint key_length;
key_length= create_table_def_key(thd, key, table_list, 0);
mdl_lock_request= mdl_request_alloc(0, table_list->db,
table_list->table_name, thd->mem_root);
mdl_request_set_type(mdl_lock_request, MDL_EXCLUSIVE);
mdl_request_add(&thd->mdl_context, mdl_lock_request);
if (mdl_acquire_exclusive_locks(&thd->mdl_context))
mdl_request= MDL_request::create(0, table_list->db,
table_list->table_name, thd->mem_root);
mdl_request->set_type(MDL_EXCLUSIVE);
thd->mdl_context.add_request(mdl_request);
if (thd->mdl_context.acquire_exclusive_locks())
{
mdl_request_remove(&thd->mdl_context, mdl_lock_request);
thd->mdl_context.remove_request(mdl_request);
DBUG_RETURN(0);
}
@ -4412,7 +4411,7 @@ static int prepare_for_repair(THD *thd, TABLE_LIST *table_list,
}
pthread_mutex_unlock(&LOCK_open);
table= &tmp_table;
table_list->mdl_lock_request= mdl_lock_request;
table_list->mdl_request= mdl_request;
}
/* A MERGE table must not come here. */
@ -4523,10 +4522,10 @@ end:
pthread_mutex_unlock(&LOCK_open);
}
/* In case of a temporary table there will be no metadata lock. */
if (error && mdl_lock_request)
if (error && mdl_request)
{
mdl_ticket_release(&thd->mdl_context, mdl_lock_request->ticket);
mdl_request_remove(&thd->mdl_context, mdl_lock_request);
thd->mdl_context.release_lock(mdl_request->ticket);
thd->mdl_context.remove_request(mdl_request);
}
DBUG_RETURN(error);
}
@ -5229,7 +5228,7 @@ bool mysql_create_like_schema_frm(THD* thd, TABLE_LIST* schema_table,
bool mysql_create_like_table(THD* thd, TABLE_LIST* table, TABLE_LIST* src_table,
HA_CREATE_INFO *create_info)
{
MDL_LOCK_REQUEST *target_lock_req= NULL;
MDL_request *target_mdl_request= NULL;
char src_path[FN_REFLEN], dst_path[FN_REFLEN + 1];
uint dst_path_length;
char *db= table->db;
@ -5286,9 +5285,9 @@ bool mysql_create_like_table(THD* thd, TABLE_LIST* table, TABLE_LIST* src_table,
}
else
{
if (lock_table_name_if_not_cached(thd, db, table_name, &target_lock_req))
if (lock_table_name_if_not_cached(thd, db, table_name, &target_mdl_request))
goto err;
if (!target_lock_req)
if (!target_mdl_request)
goto table_exists;
dst_path_length= build_table_filename(dst_path, sizeof(dst_path) - 1,
db, table_name, reg_ext, 0);
@ -5298,7 +5297,7 @@ bool mysql_create_like_table(THD* thd, TABLE_LIST* table, TABLE_LIST* src_table,
Make the metadata lock available to open_table() called to
reopen the table down the road.
*/
table->mdl_lock_request= target_lock_req;
table->mdl_request= target_mdl_request;
}
DBUG_EXECUTE_IF("sleep_create_like_before_copy", my_sleep(6000000););
@ -5469,10 +5468,10 @@ binlog:
res= FALSE;
err:
if (target_lock_req)
if (target_mdl_request)
{
mdl_ticket_release(&thd->mdl_context, target_lock_req->ticket);
mdl_request_remove(&thd->mdl_context, target_lock_req);
thd->mdl_context.release_lock(target_mdl_request->ticket);
thd->mdl_context.remove_request(target_mdl_request);
}
DBUG_RETURN(res);
}
@ -6411,8 +6410,8 @@ bool mysql_alter_table(THD *thd,char *new_db, char *new_name,
uint order_num, ORDER *order, bool ignore)
{
TABLE *table, *new_table= 0;
MDL_LOCK_TICKET *mdl_lock_ticket;
MDL_LOCK_REQUEST *target_lock_req= NULL;
MDL_ticket *mdl_ticket;
MDL_request *target_mdl_request= NULL;
int error= 0;
char tmp_name[80],old_name[32],new_name_buff[FN_REFLEN + 1];
char new_alias_buff[FN_REFLEN], *table_name, *db, *new_alias, *alias;
@ -6583,7 +6582,7 @@ view_err:
MYSQL_OPEN_TAKE_UPGRADABLE_MDL)))
DBUG_RETURN(TRUE);
table->use_all_columns();
mdl_lock_ticket= table->mdl_lock_ticket;
mdl_ticket= table->mdl_ticket;
/*
Prohibit changing of the UNION list of a non-temporary MERGE table
@ -6636,9 +6635,9 @@ view_err:
else
{
if (lock_table_name_if_not_cached(thd, new_db, new_name,
&target_lock_req))
&target_mdl_request))
DBUG_RETURN(TRUE);
if (!target_lock_req)
if (!target_mdl_request)
{
my_error(ER_TABLE_EXISTS_ERROR, MYF(0), new_alias);
DBUG_RETURN(TRUE);
@ -6831,12 +6830,12 @@ view_err:
*/
if (new_name != table_name || new_db != db)
{
mdl_ticket_release(&thd->mdl_context, target_lock_req->ticket);
mdl_request_remove(&thd->mdl_context, target_lock_req);
mdl_ticket_release_all_for_name(&thd->mdl_context, mdl_lock_ticket);
thd->mdl_context.release_lock(target_mdl_request->ticket);
thd->mdl_context.remove_request(target_mdl_request);
thd->mdl_context.release_all_locks_for_name(mdl_ticket);
}
else
mdl_downgrade_exclusive_lock(&thd->mdl_context, mdl_lock_ticket);
mdl_ticket->downgrade_exclusive_lock();
}
DBUG_RETURN(error);
}
@ -7069,7 +7068,7 @@ view_err:
#ifdef WITH_PARTITION_STORAGE_ENGINE
if (fast_alter_partition)
{
DBUG_ASSERT(!target_lock_req);
DBUG_ASSERT(!target_mdl_request);
DBUG_RETURN(fast_alter_partition_table(thd, table, alter_info,
create_info, table_list,
db, table_name,
@ -7436,7 +7435,7 @@ view_err:
table_list->table_name_length= strlen(new_name);
table_list->db= new_db;
table_list->db_length= strlen(new_db);
table_list->mdl_lock_request= target_lock_req;
table_list->mdl_request= target_mdl_request;
}
else
{
@ -7445,7 +7444,7 @@ view_err:
points to a different instance than the one set initially
to request the lock.
*/
table_list->mdl_lock_request->ticket= mdl_lock_ticket;
table_list->mdl_request->ticket= mdl_ticket;
}
if (open_table(thd, table_list, thd->mem_root,
&ot_action_unused, MYSQL_OPEN_REOPEN))
@ -7511,12 +7510,12 @@ view_err:
{
if ((new_name != table_name || new_db != db))
{
mdl_ticket_release(&thd->mdl_context, target_lock_req->ticket);
mdl_request_remove(&thd->mdl_context, target_lock_req);
mdl_ticket_release_all_for_name(&thd->mdl_context, mdl_lock_ticket);
thd->mdl_context.release_lock(target_mdl_request->ticket);
thd->mdl_context.remove_request(target_mdl_request);
thd->mdl_context.release_all_locks_for_name(mdl_ticket);
}
else
mdl_downgrade_exclusive_lock(&thd->mdl_context, mdl_lock_ticket);
mdl_ticket->downgrade_exclusive_lock();
}
end_temporary:
@ -7571,10 +7570,10 @@ err:
alter_info->datetime_field->field_name);
thd->abort_on_warning= save_abort_on_warning;
}
if (target_lock_req)
if (target_mdl_request)
{
mdl_ticket_release(&thd->mdl_context, target_lock_req->ticket);
mdl_request_remove(&thd->mdl_context, target_lock_req);
thd->mdl_context.release_lock(target_mdl_request->ticket);
thd->mdl_context.remove_request(target_mdl_request);
}
DBUG_RETURN(TRUE);
@ -7586,12 +7585,12 @@ err_with_mdl:
tables and release the exclusive metadata lock.
*/
thd->locked_tables_list.unlink_all_closed_tables();
if (target_lock_req)
if (target_mdl_request)
{
mdl_ticket_release(&thd->mdl_context, target_lock_req->ticket);
mdl_request_remove(&thd->mdl_context, target_lock_req);
thd->mdl_context.release_lock(target_mdl_request->ticket);
thd->mdl_context.remove_request(target_mdl_request);
}
mdl_ticket_release_all_for_name(&thd->mdl_context, mdl_lock_ticket);
thd->mdl_context.release_all_locks_for_name(mdl_ticket);
DBUG_RETURN(TRUE);
}
/* mysql_alter_table */

View File

@ -329,7 +329,7 @@ bool mysql_create_or_drop_trigger(THD *thd, TABLE_LIST *tables, bool create)
String stmt_query;
bool need_start_waiting= FALSE;
bool lock_upgrade_done= FALSE;
MDL_LOCK_TICKET *mdl_lock_ticket= NULL;
MDL_ticket *mdl_ticket= NULL;
DBUG_ENTER("mysql_create_or_drop_trigger");
@ -465,7 +465,7 @@ bool mysql_create_or_drop_trigger(THD *thd, TABLE_LIST *tables, bool create)
table= tables->table;
/* Later on we will need it to downgrade the lock */
mdl_lock_ticket= table->mdl_lock_ticket;
mdl_ticket= table->mdl_ticket;
if (wait_while_table_is_used(thd, table, HA_EXTRA_FORCE_REOPEN))
goto end;
@ -513,7 +513,7 @@ end:
TABLE instance created by open_n_lock_single_table() and metadata lock.
*/
if (thd->locked_tables_mode && tables && lock_upgrade_done)
mdl_downgrade_exclusive_lock(&thd->mdl_context, mdl_lock_ticket);
mdl_ticket->downgrade_exclusive_lock();
if (need_start_waiting)
start_waiting_global_read_lock(thd);
@ -1882,7 +1882,7 @@ bool Table_triggers_list::change_table_name(THD *thd, const char *db,
In the future, only an exclusive metadata lock will be enough.
*/
#ifndef DBUG_OFF
if (mdl_is_exclusive_lock_owner(&thd->mdl_context, 0, db, old_table))
if (thd->mdl_context.is_exclusive_lock_owner(0, db, old_table))
safe_mutex_assert_owner(&LOCK_open);
#endif

View File

@ -4814,8 +4814,8 @@ size_t max_row_length(TABLE *table, const uchar *data)
void alloc_mdl_requests(TABLE_LIST *table_list, MEM_ROOT *root)
{
for ( ; table_list ; table_list= table_list->next_global)
table_list->mdl_lock_request=
mdl_request_alloc(0, table_list->db, table_list->table_name, root);
table_list->mdl_request=
MDL_request::create(0, table_list->db, table_list->table_name, root);
}

View File

@ -30,8 +30,8 @@ class st_select_lex;
class partition_info;
class COND_EQUAL;
class Security_context;
struct MDL_LOCK_REQUEST;
struct MDL_LOCK_TICKET;
struct MDL_request;
struct MDL_ticket;
/*************************************************************************/
@ -814,7 +814,7 @@ public:
partition_info *part_info; /* Partition related information */
bool no_partitions_used; /* If true, all partitions have been pruned away */
#endif
MDL_LOCK_TICKET *mdl_lock_ticket;
MDL_ticket *mdl_ticket;
bool fill_item_list(List<Item> *item_list) const;
void reset_item_list(List<Item> *item_list) const;
@ -1417,7 +1417,7 @@ struct TABLE_LIST
uint table_open_method;
enum enum_schema_table_state schema_table_state;
MDL_LOCK_REQUEST *mdl_lock_request;
MDL_request *mdl_request;
void calc_md5(char *buffer);
void set_underlying_merge();

View File

@ -434,7 +434,7 @@ int ha_myisammrg::add_children_list(void)
/* Copy select_lex. Used in unique_table() at least. */
child_l->select_lex= parent_l->select_lex;
child_l->mdl_lock_request= NULL; /* Safety, if alloc_mdl_requests fails. */
child_l->mdl_request= NULL; /* Safety, if alloc_mdl_requests fails. */
/* Break when this was the last child. */
if (&child_l->next_global == this->children_last_l)