Reuse THD for new user connections

- To ensure that mallocs are marked for the correct THD, even if it's
  allocated in another thread, I added the thread_id to the THD constructor
- Added st_my_thread_var to thr_lock_info_init() to avoid a call to my_thread_var
- Moved things from THD::THD() to THD::init()
- Moved some things to THD::cleanup()
- Added THD::free_connection() and THD::reset_for_reuse()
- Added THD to CONNECT::create_thd()
- Added THD::thread_dbug_id and st_my_thread_var->dbug_id. These are needed
  to ensure that we have a constant thread_id used for debugging with a THD,
  even if it changes thread_id (=connection_id)
- Set variables.pseudo_thread_id in constructor. Removed not needed sets.
This commit is contained in:
Monty 2016-04-07 19:51:40 +03:00 committed by Sergei Golubchik
parent 54f3e18f6e
commit 89685d55d7
34 changed files with 220 additions and 130 deletions

View File

@ -691,7 +691,7 @@ struct st_my_thread_var
mysql_mutex_t * volatile current_mutex;
mysql_cond_t * volatile current_cond;
pthread_t pthread_self;
my_thread_id id;
my_thread_id id, dbug_id;
int volatile abort;
my_bool init;
struct st_my_thread_var *next,**prev;

View File

@ -139,9 +139,10 @@ typedef struct st_thr_lock {
extern LIST *thr_lock_thread_list;
extern mysql_mutex_t THR_LOCK_lock;
struct st_my_thread_var;
my_bool init_thr_lock(void); /* Must be called once/thread */
void thr_lock_info_init(THR_LOCK_INFO *info);
void thr_lock_info_init(THR_LOCK_INFO *info, struct st_my_thread_var *tmp);
void thr_lock_init(THR_LOCK *lock);
void thr_lock_delete(THR_LOCK *lock);
void thr_lock_data_init(THR_LOCK *lock,THR_LOCK_DATA *data,

View File

@ -667,8 +667,7 @@ void init_embedded_mysql(MYSQL *mysql, int client_flag)
*/
void *create_embedded_thd(int client_flag)
{
THD * thd= new THD;
thd->thread_id= thd->variables.pseudo_thread_id= next_thread_id();
THD * thd= new THD(next_thread_id());
thd->thread_stack= (char*) &thd;
if (thd->store_globals())

View File

@ -294,7 +294,7 @@ my_bool my_thread_init(void)
STACK_DIRECTION * (long)my_thread_stack_size;
mysql_mutex_lock(&THR_LOCK_threads);
tmp->id= ++thread_id;
tmp->id= tmp->dbug_id= ++thread_id;
++THR_thread_count;
mysql_mutex_unlock(&THR_LOCK_threads);
tmp->init= 1;
@ -400,7 +400,7 @@ my_thread_id my_thread_dbug_id()
my_thread_init().
*/
struct st_my_thread_var *tmp= my_thread_var;
return tmp ? tmp->id : 0;
return tmp ? tmp->dbug_id : 0;
}
#ifdef DBUG_OFF

View File

@ -465,9 +465,10 @@ void thr_lock_delete(THR_LOCK *lock)
}
void thr_lock_info_init(THR_LOCK_INFO *info)
void thr_lock_info_init(THR_LOCK_INFO *info, struct st_my_thread_var *tmp)
{
struct st_my_thread_var *tmp= my_thread_var;
if (tmp)
tmp= my_thread_var;
info->thread= tmp->pthread_self;
info->thread_id= tmp->id;
}
@ -1816,7 +1817,7 @@ static void *test_thread(void *arg)
printf("Thread %s (%d) started\n",my_thread_name(),param); fflush(stdout);
thr_lock_info_init(&lock_info);
thr_lock_info_init(&lock_info, 0);
for (i=0; i < lock_counts[param] ; i++)
thr_lock_data_init(locks+tests[param][i].lock_nr,data+i,NULL);
for (j=1 ; j < 10 ; j++) /* try locking 10 times */

View File

@ -89,7 +89,7 @@ static int prepare_for_fill(TABLE_LIST *tables)
(every increment of global thread_id counts as a new connection
in SHOW STATUS and we want to avoid skewing the statistics)
*/
thd->thread_id= thd->variables.pseudo_thread_id= thd_thread_id;
thd->variables.pseudo_thread_id= thd->thread_id;
mysql_mutex_lock(&LOCK_thread_count);
thread_count++;
threads.append(thd);
@ -205,10 +205,10 @@ static void send_report(const char *when)
/*
otherwise, prepare the THD and TABLE_LIST,
create and fill the temporary table with data just like
SELECT * FROM IFROEMATION_SCHEMA.feedback is doing,
SELECT * FROM INFORMATION_SCHEMA.feedback is doing,
read and concatenate table data into a String.
*/
if (!(thd= new THD()))
if (!(thd= new THD(thd_thread_id)))
return;
if (prepare_for_fill(&tables))

View File

@ -278,7 +278,7 @@ dbcontext::init_thread(const void *stack_bottom, volatile int& shutdown_flag)
DBG_THR(fprintf(stderr, "HNDSOCK init thread\n"));
{
my_thread_init();
thd = new THD;
thd = new THD(0);
thd->thread_stack = (char *)stack_bottom;
DBG_THR(fprintf(stderr,
"thread_stack = %p sizeof(THD)=%zu sizeof(mtx)=%zu "

View File

@ -189,7 +189,6 @@ pre_init_event_thread(THD* thd)
thd->net.read_timeout= slave_net_timeout;
thd->variables.option_bits|= OPTION_AUTO_IS_NULL;
thd->client_capabilities|= CLIENT_MULTI_RESULTS;
thd->thread_id= thd->variables.pseudo_thread_id= next_thread_id();
/*
Guarantees that we will see the thread in SHOW PROCESSLIST though its
@ -396,7 +395,7 @@ Event_scheduler::start(int *err_no)
if (state > INITIALIZED)
goto end;
if (!(new_thd= new THD))
if (!(new_thd= new THD(next_thread_id())))
{
sql_print_error("Event Scheduler: Cannot initialize the scheduler thread");
ret= true;
@ -542,7 +541,7 @@ Event_scheduler::execute_top(Event_queue_element_for_exec *event_name)
int res= 0;
DBUG_ENTER("Event_scheduler::execute_top");
if (!(new_thd= new THD()))
if (!(new_thd= new THD(next_thread_id())))
goto error;
pre_init_event_thread(new_thd);

View File

@ -854,7 +854,7 @@ Events::init(THD *thd, bool opt_noacl_or_bootstrap)
if (!thd)
{
if (!(thd= new THD()))
if (!(thd= new THD(0)))
{
res= TRUE;
goto end;

View File

@ -9510,10 +9510,9 @@ binlog_background_thread(void *arg __attribute__((unused)))
my_thread_init();
DBUG_ENTER("binlog_background_thread");
thd= new THD;
thd= new THD(next_thread_id());
thd->system_thread= SYSTEM_THREAD_BINLOG_BACKGROUND;
thd->thread_stack= (char*) &thd; /* Set approximate stack start */
thd->thread_id= next_thread_id();
thd->store_globals();
thd->security_ctx->skip_grants();
thd->set_command(COM_DAEMON);

View File

@ -2949,7 +2949,7 @@ void signal_thd_deleted()
/*
Unlink thd from global list of available connections and free thd
Unlink thd from global list of available connections
SYNOPSIS
unlink_thd()
@ -2971,7 +2971,7 @@ void unlink_thd(THD *thd)
thd->add_status_to_global();
unlink_not_visible_thd(thd);
delete thd;
thd->free_connection();
dec_thread_count();
DBUG_VOID_RETURN;
@ -2983,6 +2983,7 @@ void unlink_thd(THD *thd)
SYNOPSIS
cache_thread()
thd Thread handler
NOTES
LOCK_thread_cache is used to protect the cache variables
@ -2994,10 +2995,11 @@ void unlink_thd(THD *thd)
*/
static bool cache_thread()
static bool cache_thread(THD *thd)
{
struct timespec abstime;
DBUG_ENTER("cache_thread");
DBUG_ASSERT(thd);
mysql_mutex_lock(&LOCK_thread_cache);
if (cached_thread_count < thread_cache_size &&
@ -3041,13 +3043,12 @@ static bool cache_thread()
if (wake_thread)
{
CONNECT *connect;
THD *thd;
wake_thread--;
connect= thread_cache.get();
mysql_mutex_unlock(&LOCK_thread_cache);
if (!(thd= connect->create_thd()))
if (!(connect->create_thd(thd)))
{
/* Out of resources. Free thread to get more resources */
connect->close_and_delete();
@ -3055,8 +3056,11 @@ static bool cache_thread()
}
delete connect;
thd->thread_stack= (char*) &thd; // For store_globals
(void) thd->store_globals();
/*
We have to call store_globals to update mysys_var->id and lock_info
with the new thread_id
*/
thd->store_globals();
#ifdef HAVE_PSI_THREAD_INTERFACE
/*
@ -3068,11 +3072,7 @@ static bool cache_thread()
PSI_THREAD_CALL(set_thread)(psi);
#endif
/*
THD::mysys_var::abort is associated with physical thread rather
than with THD object. So we need to reset this flag before using
this thread for handling of new THD object/connection.
*/
/* reset abort flag for the thread */
thd->mysys_var->abort= 0;
thd->thr_create_utime= microsecond_interval_timer();
thd->start_utime= thd->thr_create_utime;
@ -3108,13 +3108,16 @@ static bool cache_thread()
bool one_thread_per_connection_end(THD *thd, bool put_in_cache)
{
DBUG_ENTER("one_thread_per_connection_end");
const bool wsrep_applier= IF_WSREP(thd->wsrep_applier, false);
if (thd)
unlink_thd(thd);
{
const bool wsrep_applier= IF_WSREP(thd->wsrep_applier, false);
if (!wsrep_applier && put_in_cache && cache_thread())
DBUG_RETURN(0); // Thread is reused
unlink_thd(thd);
if (!wsrep_applier && put_in_cache && cache_thread(thd))
DBUG_RETURN(0); // Thread is reused
delete thd;
}
DBUG_PRINT("info", ("killing thread"));
DBUG_LEAVE; // Must match DBUG_ENTER()
@ -4076,9 +4079,9 @@ void init_com_statement_info()
extern "C" my_thread_id mariadb_dbug_id()
{
THD *thd;
if ((thd= current_thd))
if ((thd= current_thd) && thd->thread_dbug_id)
{
return thd->thread_id;
return thd->thread_dbug_id;
}
return my_thread_dbug_id();
}
@ -6318,7 +6321,7 @@ static void bootstrap(MYSQL_FILE *file)
{
DBUG_ENTER("bootstrap");
THD *thd= new THD;
THD *thd= new THD(next_thread_id());
#ifdef WITH_WSREP
thd->variables.wsrep_on= 0;
#endif
@ -6326,7 +6329,6 @@ static void bootstrap(MYSQL_FILE *file)
my_net_init(&thd->net,(st_vio*) 0, (void*) 0, MYF(0));
thd->max_client_packet_length= thd->net.max_packet;
thd->security_ctx->master_access= ~(ulong)0;
thd->thread_id= thd->variables.pseudo_thread_id= next_thread_id();
thread_count++; // Safe as only one thread running
in_bootstrap= TRUE;

View File

@ -967,9 +967,8 @@ handle_rpl_parallel_thread(void *arg)
struct rpl_parallel_thread *rpt= (struct rpl_parallel_thread *)arg;
my_thread_init();
thd = new THD;
thd = new THD(next_thread_id());
thd->thread_stack = (char*)&thd;
thd->thread_id= thd->variables.pseudo_thread_id= next_thread_id();
add_to_active_threads(thd);
set_current_thd(thd);
pthread_detach_this_thread();

View File

@ -36,7 +36,10 @@
static bool no_threads_end(THD *thd, bool put_in_cache)
{
if (thd)
{
unlink_thd(thd);
delete thd;
}
return 1; // Abort handle_one_connection
}

View File

@ -292,9 +292,8 @@ handle_slave_init(void *arg __attribute__((unused)))
THD *thd;
my_thread_init();
thd= new THD;
thd= new THD(next_thread_id());
thd->thread_stack= (char*) &thd; /* Set approximate stack start */
thd->thread_id= next_thread_id();
thd->system_thread = SYSTEM_THREAD_SLAVE_INIT;
thread_safe_increment32(&service_thread_count);
thd->store_globals();
@ -3071,7 +3070,6 @@ static int init_slave_thread(THD* thd, Master_info *mi,
thd->variables.log_slow_filter= global_system_variables.log_slow_filter;
set_slave_thread_options(thd);
thd->client_capabilities = CLIENT_LOCAL_FILES;
thd->thread_id= thd->variables.pseudo_thread_id= next_thread_id();
if (thd_type == SLAVE_THD_SQL)
THD_STAGE_INFO(thd, stage_waiting_for_the_next_event_in_relay_log);
@ -3940,7 +3938,7 @@ pthread_handler_t handle_slave_io(void *arg)
mysql= NULL ;
retry_count= 0;
thd= new THD; // note that contructor of THD uses DBUG_ !
thd= new THD(next_thread_id()); // note that contructor of THD uses DBUG_ !
mysql_mutex_lock(&mi->run_lock);
/* Inform waiting threads that slave has started */
@ -4512,7 +4510,7 @@ pthread_handler_t handle_slave_sql(void *arg)
#endif
serial_rgi= new rpl_group_info(rli);
thd = new THD; // note that contructor of THD uses DBUG_ !
thd = new THD(next_thread_id()); // note that contructor of THD uses DBUG_ !
thd->thread_stack = (char*)&thd; // remember where our stack is
thd->system_thread_info.rpl_sql_info= &sql_info;

View File

@ -1133,7 +1133,7 @@ bool acl_init(bool dont_read_acl_tables)
/*
To be able to run this from boot, we allocate a temporary THD
*/
if (!(thd=new THD))
if (!(thd=new THD(0)))
DBUG_RETURN(1); /* purecov: inspected */
thd->thread_stack= (char*) &thd;
thd->store_globals();
@ -6570,7 +6570,7 @@ bool grant_init()
bool return_val;
DBUG_ENTER("grant_init");
if (!(thd= new THD))
if (!(thd= new THD(0)))
DBUG_RETURN(1); /* purecov: deadcode */
thd->thread_stack= (char*) &thd;
thd->store_globals();

View File

@ -460,5 +460,12 @@ void mysql_audit_release(THD *thd)
{
}
void mysql_audit_init_thd(THD *thd)
{
}
void mysql_audit_free_thd(THD *thd)
{
}
#endif /* EMBEDDED_LIBRARY */

View File

@ -9038,7 +9038,7 @@ my_bool mysql_rm_tmp_tables(void)
THD *thd;
DBUG_ENTER("mysql_rm_tmp_tables");
if (!(thd= new THD))
if (!(thd= new THD(0)))
DBUG_RETURN(1);
thd->thread_stack= (char*) &thd;
thd->store_globals();

View File

@ -336,17 +336,6 @@ void thd_set_killed(THD *thd)
thd->killed= KILL_CONNECTION;
}
/**
Clear errors from the previous THD
@param thd THD object
*/
void thd_clear_errors(THD *thd)
{
my_errno= 0;
thd->mysys_var->abort= 0;
}
/**
Set thread stack in THD object
@ -456,7 +445,7 @@ my_socket thd_get_fd(THD *thd)
{
return mysql_socket_getfd(thd->net.vio->mysql_socket);
}
#endif
#endif /* ONLY_FOR_MYSQL_CLOSED_SOURCE_SCHEDULED */
/**
Get current THD object from thread local data
@ -468,6 +457,18 @@ THD *thd_get_current_thd()
return current_thd;
}
/**
Clear errors from the previous THD
@param thd THD object
*/
void thd_clear_errors(THD *thd)
{
my_errno= 0;
thd->mysys_var->abort= 0;
}
/**
Get thread attributes for connection threads
@ -845,7 +846,7 @@ extern "C" void thd_kill_timeout(THD* thd)
}
THD::THD(bool is_wsrep_applier)
THD::THD(my_thread_id id, bool is_wsrep_applier)
:Statement(&main_lex, &main_mem_root, STMT_CONVENTIONAL_EXECUTION,
/* statement id */ 0),
rli_fake(0), rgi_fake(0), rgi_slave(NULL),
@ -854,17 +855,13 @@ THD::THD(bool is_wsrep_applier)
binlog_unsafe_warning_flags(0),
binlog_table_maps(0),
table_map_for_update(0),
arg_of_last_insert_id_function(FALSE),
first_successful_insert_id_in_prev_stmt(0),
first_successful_insert_id_in_prev_stmt_for_binlog(0),
first_successful_insert_id_in_cur_stmt(0),
stmt_depends_on_first_successful_insert_id_in_prev_stmt(FALSE),
m_examined_row_count(0),
accessed_rows_and_keys(0),
m_digest(NULL),
m_statement_psi(NULL),
m_idle_psi(NULL),
thread_id(0),
thread_id(id),
thread_dbug_id(id),
os_thread_id(0),
global_disable_checkpoint(0),
failed_com_change_user(0),
@ -910,6 +907,7 @@ THD::THD(bool is_wsrep_applier)
set_current_thd(this);
status_var.local_memory_used= sizeof(THD);
status_var.global_memory_used= 0;
variables.pseudo_thread_id= thread_id;
main_da.init();
/*
@ -973,14 +971,12 @@ THD::THD(bool is_wsrep_applier)
#ifndef DBUG_OFF
dbug_sentry=THD_SENTRY_MAGIC;
#endif
#ifndef EMBEDDED_LIBRARY
mysql_audit_init_thd(this);
#endif
net.vio=0;
net.buff= 0;
client_capabilities= 0; // minimalistic client
system_thread= NON_SYSTEM_THREAD;
cleanup_done= abort_on_warning= 0;
cleanup_done= free_connection_done= abort_on_warning= 0;
peer_port= 0; // For SHOW PROCESSLIST
transaction.m_pending_rows_event= 0;
transaction.on= 1;
@ -1003,7 +999,6 @@ THD::THD(bool is_wsrep_applier)
/* Variables with default values */
proc_info="login";
where= THD::DEFAULT_WHERE;
variables.server_id = global_system_variables.server_id;
slave_net = 0;
m_command=COM_CONNECT;
*scramble= '\0';
@ -1064,7 +1059,6 @@ THD::THD(bool is_wsrep_applier)
tmp= (ulong) (my_rnd(&sql_rand) * 0xffffffff);
my_rnd_init(&rand, tmp + (ulong) &rand, tmp + (ulong) ::global_query_id);
substitute_null_with_insert_id = FALSE;
thr_lock_info_init(&lock_info); /* safety: will be reset after start */
lock_info.mysql_thd= (void *)this;
m_token_array= NULL;
@ -1425,10 +1419,16 @@ void THD::init(void)
reset_binlog_local_stmt_filter();
set_status_var_init();
bzero((char *) &org_status_var, sizeof(org_status_var));
status_in_global= 0;
start_bytes_received= 0;
last_commit_gtid.seq_no= 0;
last_stmt= NULL;
status_in_global= 0;
/* Reset status of last insert id */
arg_of_last_insert_id_function= FALSE;
stmt_depends_on_first_successful_insert_id_in_prev_stmt= FALSE;
first_successful_insert_id_in_prev_stmt= 0;
first_successful_insert_id_in_prev_stmt_for_binlog= 0;
first_successful_insert_id_in_cur_stmt= 0;
#ifdef WITH_WSREP
wsrep_exec_mode= wsrep_applier ? REPL_RECV : LOCAL_STATE;
wsrep_conflict_state= NO_CONFLICT;
@ -1547,12 +1547,14 @@ void THD::init_for_queries()
void THD::change_user(void)
{
add_status_to_global();
if (!status_in_global) // Reset in init()
add_status_to_global();
cleanup();
reset_killed();
if (!cleanup_done)
cleanup();
cleanup_done= 0;
status_in_global= 0;
reset_killed();
thd_clear_errors(this);
init();
stmt_map.reset();
my_hash_init(&user_vars, system_charset_info, USER_VARS_HASH_SIZE, 0, 0,
@ -1616,6 +1618,8 @@ void THD::cleanup(void)
my_hash_free(&user_vars);
sp_cache_clear(&sp_proc_cache);
sp_cache_clear(&sp_func_cache);
auto_inc_intervals_forced.empty();
auto_inc_intervals_in_cur_stmt_for_binlog.empty();
mysql_ull_cleanup(this);
/* All metadata locks must have been released by now. */
@ -1627,6 +1631,63 @@ void THD::cleanup(void)
}
/*
Free all connection related resources associated with a THD.
This is used when we put a thread into the thread cache.
After this call should either call ~THD or reset_for_reuse() depending on
circumstances.
*/
void THD::free_connection()
{
DBUG_ASSERT(free_connection_done == 0);
my_free(db);
db= NULL;
#ifndef EMBEDDED_LIBRARY
if (net.vio)
vio_delete(net.vio);
net.vio= 0;
net_end(&net);
#endif
ha_close_connection(this);
plugin_thdvar_cleanup(this);
mysql_audit_free_thd(this);
main_security_ctx.destroy();
/* close all prepared statements, to save memory */
stmt_map.reset();
free_connection_done= 1;
profiling.restart(); // Reset profiling
}
/*
Reset thd for reuse by another connection
This is only used for user connections, so the following variables doesn't
have to be reset:
- Replication (slave) variables.
- Variables not reset between each statements. See reset_for_next_command.
*/
void THD::reset_for_reuse()
{
mysql_audit_init_thd(this);
change_user(); // Calls cleanup() & init()
get_stmt_da()->reset_diagnostics_area();
main_security_ctx.init();
failed_com_change_user= 0;
is_fatal_error= 0;
client_capabilities= 0;
peer_port= 0;
query_name_consts= 0; // Safety
abort_on_warning= 0;
free_connection_done= 0;
m_command= COM_CONNECT;
profiling.reset();
#ifdef SIGNAL_WITH_VIO_CLOSE
active_vio = 0;
#endif
}
THD::~THD()
{
THD *orig_thd= current_thd;
@ -1653,26 +1714,15 @@ THD::~THD()
mysql_mutex_lock(&LOCK_wsrep_thd);
mysql_mutex_unlock(&LOCK_wsrep_thd);
mysql_mutex_destroy(&LOCK_wsrep_thd);
if (wsrep_rgi) delete wsrep_rgi;
delete wsrep_rgi;
#endif
/* Close connection */
#ifndef EMBEDDED_LIBRARY
if (net.vio)
vio_delete(net.vio);
net_end(&net);
#endif
stmt_map.reset(); /* close all prepared statements */
if (!free_connection_done)
free_connection();
if (!cleanup_done)
cleanup();
mdl_context.destroy();
ha_close_connection(this);
mysql_audit_release(this);
plugin_thdvar_cleanup(this);
main_security_ctx.destroy();
my_free(db);
db= NULL;
free_root(&transaction.mem_root,MYF(0));
mysql_cond_destroy(&COND_wakeup_ready);
mysql_mutex_destroy(&LOCK_wakeup_ready);
@ -1692,7 +1742,6 @@ THD::~THD()
rli_fake= NULL;
}
mysql_audit_free_thd(this);
if (rgi_slave)
rgi_slave->cleanup_after_session();
my_free(semisync_info);
@ -2089,7 +2138,16 @@ bool THD::store_globals()
Let mysqld define the thread id (not mysys)
This allows us to move THD to different threads if needed.
*/
mysys_var->id= thread_id;
mysys_var->id= thread_id;
/* thread_dbug_id should not change for a THD */
if (!thread_dbug_id)
thread_dbug_id= mysys_var->dbug_id;
else
{
/* This only changes if we are using pool-of-threads */
mysys_var->dbug_id= thread_dbug_id;
}
#ifdef __NR_gettid
os_thread_id= (uint32)syscall(__NR_gettid);
#else
@ -2106,7 +2164,7 @@ bool THD::store_globals()
We have to call thr_lock_info_init() again here as THD may have been
created in another thread
*/
thr_lock_info_init(&lock_info);
thr_lock_info_init(&lock_info, mysys_var);
return 0;
}

View File

@ -2652,7 +2652,7 @@ public:
ulong query_plan_flags;
ulong query_plan_fsort_passes;
pthread_t real_id; /* For debugging */
my_thread_id thread_id;
my_thread_id thread_id, thread_dbug_id;
uint32 os_thread_id;
uint tmp_table, global_disable_checkpoint;
uint server_status,open_options;
@ -2764,7 +2764,7 @@ public:
/* for IS NULL => = last_insert_id() fix in remove_eq_conds() */
bool substitute_null_with_insert_id;
bool in_lock_tables;
bool bootstrap, cleanup_done;
bool bootstrap, cleanup_done, free_connection_done;
/** is set if some thread specific value(s) used in a statement. */
bool thread_specific_used;
@ -2906,7 +2906,7 @@ public:
/* Debug Sync facility. See debug_sync.cc. */
struct st_debug_sync_control *debug_sync_control;
#endif /* defined(ENABLED_DEBUG_SYNC) */
THD(bool is_wsrep_applier= false);
THD(my_thread_id id, bool is_wsrep_applier= false);
~THD();
@ -2926,6 +2926,8 @@ public:
void change_user(void);
void cleanup(void);
void cleanup_after_query();
void free_connection();
void reset_for_reuse();
bool store_globals();
void reset_globals();
#ifdef SIGNAL_WITH_VIO_CLOSE

View File

@ -1299,7 +1299,7 @@ void do_handle_one_connection(CONNECT *connect)
ulonglong thr_create_utime= microsecond_interval_timer();
THD *thd;
if (connect->scheduler->init_new_connection_thread() ||
!(thd= connect->create_thd()))
!(thd= connect->create_thd(NULL)))
{
scheduler_functions *scheduler= connect->scheduler;
connect->close_with_error(0, 0, ER_OUT_OF_RESOURCES);
@ -1426,7 +1426,7 @@ void CONNECT::close_and_delete()
void CONNECT::close_with_error(uint sql_errno,
const char *message, uint close_error)
{
THD *thd= create_thd();
THD *thd= create_thd(NULL);
if (thd)
{
if (sql_errno)
@ -1460,17 +1460,28 @@ CONNECT::~CONNECT()
vio_delete(vio);
}
/* Create a THD based on a CONNECT object */
THD *CONNECT::create_thd()
/* Reuse or create a THD based on a CONNECT object */
THD *CONNECT::create_thd(THD *thd)
{
my_bool res;
THD *thd;
bool res, thd_reused= thd != 0;
DBUG_ENTER("create_thd");
DBUG_EXECUTE_IF("simulate_failed_connection_2", DBUG_RETURN(0); );
if (!(thd= new THD))
if (thd)
{
/* reuse old thd */
thd->reset_for_reuse();
/*
reset tread_id's, but not thread_dbug_id's as the later isn't allowed
to change as there is already structures in thd marked with the old
value.
*/
thd->thread_id= thd->variables.pseudo_thread_id= thread_id;
}
else if (!(thd= new THD(thread_id)))
DBUG_RETURN(0);
set_current_thd(thd);
@ -1479,7 +1490,8 @@ THD *CONNECT::create_thd()
if (res)
{
delete thd;
if (!thd_reused)
delete thd;
set_current_thd(0);
DBUG_RETURN(0);
}
@ -1489,7 +1501,6 @@ THD *CONNECT::create_thd()
thd->security_ctx->host= host;
thd->extra_port= extra_port;
thd->scheduler= scheduler;
thd->thread_id= thd->variables.pseudo_thread_id= thread_id;
thd->real_id= real_id;
DBUG_RETURN(thd);
}

View File

@ -52,7 +52,7 @@ public:
void close_and_delete();
void close_with_error(uint sql_errno,
const char *message, uint close_error);
THD *create_thd();
THD *create_thd(THD *thd);
};

View File

@ -1,6 +1,6 @@
/*
Copyright (c) 2000, 2015, Oracle and/or its affiliates.
Copyright (c) 2010, 2015, MariaDB
Copyright (c) 2010, 2016, MariaDB
This program is free software; you can redistribute it and/or modify
it under the terms of the GNU General Public License as published by
@ -2041,7 +2041,8 @@ public:
MDL_request grl_protection;
Delayed_insert(SELECT_LEX *current_select)
:locks_in_memory(0), table(0),tables_in_use(0),stacked_inserts(0),
:locks_in_memory(0), thd(next_thread_id()),
table(0),tables_in_use(0), stacked_inserts(0),
status(0), retry(0), handler_thread_initialized(FALSE), group_count(0)
{
DBUG_ENTER("Delayed_insert constructor");
@ -2819,7 +2820,6 @@ pthread_handler_t handle_delayed_insert(void *arg)
pthread_detach_this_thread();
/* Add thread to THD list so that's it's visible in 'show processlist' */
thd->thread_id= thd->variables.pseudo_thread_id= next_thread_id();
thd->set_current_time();
add_to_active_threads(thd);
if (abort_loop)

View File

@ -1769,7 +1769,7 @@ static void plugin_load(MEM_ROOT *tmp_root)
TABLE *table;
READ_RECORD read_record_info;
int error;
THD *new_thd= new THD;
THD *new_thd= new THD(0);
bool result;
DBUG_ENTER("plugin_load");

View File

@ -328,14 +328,28 @@ PROFILING::PROFILING()
}
PROFILING::~PROFILING()
{
restart();
}
/*
Restart profiling from scratch
*/
void PROFILING::restart()
{
while (! history.is_empty())
delete history.pop();
if (current != NULL)
delete current;
/* Ensure that profiling object can be reused */
profile_id_counter= 1;
current= NULL;
last= NULL;
}
/**
Throw away the current profile, because it's useless or unwanted
or corrupted.
@ -675,6 +689,6 @@ int PROFILING::fill_statistics_info(THD *thd_arg, TABLE_LIST *tables, Item *cond
void PROFILING::reset()
{
enabled= thd->variables.option_bits & OPTION_PROFILING;
enabled= (thd->variables.option_bits & OPTION_PROFILING) != 0;
}
#endif /* ENABLED_PROFILING */

View File

@ -324,6 +324,7 @@ public:
/* ... from INFORMATION_SCHEMA.PROFILING ... */
int fill_statistics_info(THD *thd, TABLE_LIST *tables, Item *cond);
void reset();
void restart();
};
# endif /* ENABLED_PROFILING */

View File

@ -72,7 +72,7 @@ bool reload_acl_and_cache(THD *thd, unsigned long long options,
If reload_acl_and_cache() is called from SIGHUP handler we have to
allocate temporary THD for execution of acl_reload()/grant_reload().
*/
if (!thd && (thd= (tmp_thd= new THD)))
if (!thd && (thd= (tmp_thd= new THD(0))))
{
thd->thread_stack= (char*) &tmp_thd;
thd->store_globals();

View File

@ -162,7 +162,7 @@ bool servers_init(bool dont_read_servers_table)
/*
To be able to run this from boot, we allocate a temporary THD
*/
if (!(thd=new THD))
if (!(thd=new THD(0)))
DBUG_RETURN(TRUE);
thd->thread_stack= (char*) &thd;
thd->store_globals();

View File

@ -1642,7 +1642,7 @@ void execute_ddl_log_recovery()
/*
To be able to run this from boot, we allocate a temporary THD
*/
if (!(thd=new THD))
if (!(thd=new THD(0)))
DBUG_VOID_RETURN;
thd->thread_stack= (char*) &thd;
thd->store_globals();

View File

@ -154,7 +154,7 @@ void udf_init()
mysql_rwlock_init(key_rwlock_THR_LOCK_udf, &THR_LOCK_udf);
init_sql_alloc(&mem, UDF_ALLOC_BLOCK_SIZE, 0, MYF(0));
THD *new_thd = new THD;
THD *new_thd = new THD(0);
if (!new_thd ||
my_hash_init(&udf_hash,system_charset_info,32,0,0,get_hash_key, NULL, 0))
{

View File

@ -122,7 +122,7 @@ THD* threadpool_add_connection(CONNECT *connect, void *scheduler_data)
pthread_setspecific(THR_KEY_mysys, 0);
my_thread_init();
st_my_thread_var* mysys_var= (st_my_thread_var *)pthread_getspecific(THR_KEY_mysys);
if (!mysys_var ||!(thd= connect->create_thd()))
if (!mysys_var ||!(thd= connect->create_thd(NULL)))
{
/* Out of memory? */
connect->close_and_delete();
@ -200,6 +200,7 @@ void threadpool_remove_connection(THD *thd)
end_connection(thd);
close_connection(thd, 0);
unlink_thd(thd);
delete thd;
mysql_cond_broadcast(&COND_thread_count);
/*

View File

@ -1620,7 +1620,7 @@ my_tz_init(THD *org_thd, const char *default_tzname, my_bool bootstrap)
/*
To be able to run this from boot, we allocate a temporary THD
*/
if (!(thd= new THD))
if (!(thd= new THD(0)))
DBUG_RETURN(1);
thd->thread_stack= (char*) &thd;
thd->store_globals();

View File

@ -414,7 +414,7 @@ process::wait ()
return err_;
}
thd::thd (my_bool won) : init(), ptr(new THD)
thd::thd (my_bool won) : init(), ptr(new THD(0))
{
if (ptr)
{

View File

@ -2259,7 +2259,7 @@ void *spider_bg_conn_action(
my_thread_init();
DBUG_ENTER("spider_bg_conn_action");
/* init start */
if (!(thd = new THD()))
if (!(thd = new THD(next_thread_id())))
{
pthread_mutex_lock(&conn->bg_conn_sync_mutex);
pthread_cond_signal(&conn->bg_conn_sync_cond);
@ -2267,7 +2267,6 @@ void *spider_bg_conn_action(
my_thread_end();
DBUG_RETURN(NULL);
}
thd->thread_id = next_thread_id();
#ifdef HAVE_PSI_INTERFACE
mysql_thread_set_psi_id(thd->thread_id);
#endif
@ -2766,7 +2765,7 @@ void *spider_bg_sts_action(
}
#endif
pthread_mutex_lock(&share->sts_mutex);
if (!(thd = new THD()))
if (!(thd = new THD(next_thread_id())))
{
share->bg_sts_thd_wait = FALSE;
share->bg_sts_kill = FALSE;
@ -2778,7 +2777,6 @@ void *spider_bg_sts_action(
#endif
DBUG_RETURN(NULL);
}
thd->thread_id = next_thread_id();
#ifdef HAVE_PSI_INTERFACE
mysql_thread_set_psi_id(thd->thread_id);
#endif
@ -3146,7 +3144,7 @@ void *spider_bg_crd_action(
}
#endif
pthread_mutex_lock(&share->crd_mutex);
if (!(thd = new THD()))
if (!(thd = new THD(next_thread_id())))
{
share->bg_crd_thd_wait = FALSE;
share->bg_crd_kill = FALSE;
@ -3158,7 +3156,6 @@ void *spider_bg_crd_action(
#endif
DBUG_RETURN(NULL);
}
thd->thread_id = next_thread_id();
#ifdef HAVE_PSI_INTERFACE
mysql_thread_set_psi_id(thd->thread_id);
#endif
@ -3636,7 +3633,7 @@ void *spider_bg_mon_action(
DBUG_ENTER("spider_bg_mon_action");
/* init start */
pthread_mutex_lock(&share->bg_mon_mutexes[link_idx]);
if (!(thd = new THD()))
if (!(thd = new THD(next_thread_id())))
{
share->bg_mon_kill = FALSE;
share->bg_mon_init = FALSE;
@ -3645,7 +3642,6 @@ void *spider_bg_mon_action(
my_thread_end();
DBUG_RETURN(NULL);
}
thd->thread_id = next_thread_id();
#ifdef HAVE_PSI_INTERFACE
mysql_thread_set_psi_id(thd->thread_id);
#endif

View File

@ -4055,7 +4055,7 @@ THD *spider_create_tmp_thd()
{
THD *thd;
DBUG_ENTER("spider_create_tmp_thd");
if (!(thd = new THD))
if (!(thd = new THD(0)))
DBUG_RETURN(NULL);
#if defined(MARIADB_BASE_VERSION) && MYSQL_VERSION_ID >= 100000
thd->killed = NOT_KILLED;
@ -4066,7 +4066,6 @@ THD *spider_create_tmp_thd()
thd->locked_tables = FALSE;
#endif
thd->proc_info = "";
thd->thread_id = thd->variables.pseudo_thread_id = 0;
thd->thread_stack = (char*) &thd;
if (thd->store_globals())
DBUG_RETURN(NULL);