Commit 91734431 authored by Sergey Vojtovich's avatar Sergey Vojtovich

Move all thread cache specific code to a new class

Part of
MDEV-18353 - Shutdown may miss to wait for connection thread
parent 8ad3c615
......@@ -33,6 +33,7 @@
#include "sql_table.h" // release_ddl_log, execute_ddl_log_recovery
#include "sql_connect.h" // free_max_user_conn, init_max_user_conn,
// handle_one_connection
#include "thread_cache.h"
#include "sql_time.h" // known_date_time_formats,
// get_date_time_format_str,
// date_time_format_make
......@@ -328,9 +329,7 @@ static my_bool opt_debugging= 0, opt_external_locking= 0, opt_console= 0;
static my_bool opt_short_log_format= 0, opt_silent_startup= 0;
bool my_disable_leak_check= false;
uint kill_cached_threads;
ulong max_used_connections;
volatile ulong cached_thread_count= 0;
static char *mysqld_user, *mysqld_chroot;
static char *default_character_set_name;
static char *character_set_filesystem_name;
......@@ -344,11 +343,9 @@ char *enforced_storage_engine=NULL;
char *gtid_pos_auto_engines;
plugin_ref *opt_gtid_pos_auto_plugins;
static char compiled_default_collation_name[]= MYSQL_DEFAULT_COLLATION_NAME;
static I_List<CONNECT> thread_cache;
Thread_cache thread_cache;
static bool binlog_format_used= false;
LEX_STRING opt_init_connect, opt_init_slave;
mysql_cond_t COND_thread_cache;
static mysql_cond_t COND_flush_thread_cache;
mysql_cond_t COND_slave_background;
static DYNAMIC_ARRAY all_options;
static longlong start_memory_used;
......@@ -682,7 +679,6 @@ void set_current_thd(THD *thd) { THR_THD= thd; }
*/
mysql_mutex_t LOCK_start_thread;
mysql_mutex_t LOCK_thread_cache;
mysql_mutex_t
LOCK_status, LOCK_error_log, LOCK_short_uuid_generator,
LOCK_delayed_insert, LOCK_delayed_status, LOCK_delayed_create,
......@@ -892,7 +888,6 @@ PSI_mutex_key key_BINLOG_LOCK_index, key_BINLOG_LOCK_xid_list,
key_structure_guard_mutex, key_TABLE_SHARE_LOCK_ha_data,
key_LOCK_error_messages,
key_LOCK_start_thread,
key_LOCK_thread_cache,
key_PARTITION_LOCK_auto_inc;
PSI_mutex_key key_RELAYLOG_LOCK_index;
PSI_mutex_key key_LOCK_relaylog_end_pos;
......@@ -984,7 +979,6 @@ static PSI_mutex_info all_server_mutexes[]=
{ &key_LOCK_after_binlog_sync, "LOCK_after_binlog_sync", PSI_FLAG_GLOBAL},
{ &key_LOCK_commit_ordered, "LOCK_commit_ordered", PSI_FLAG_GLOBAL},
{ &key_LOCK_slave_background, "LOCK_slave_background", PSI_FLAG_GLOBAL},
{ &key_LOCK_thread_cache, "LOCK_thread_cache", PSI_FLAG_GLOBAL},
{ &key_PARTITION_LOCK_auto_inc, "HA_DATA_PARTITION::LOCK_auto_inc", 0},
{ &key_LOCK_slave_state, "LOCK_slave_state", 0},
{ &key_LOCK_start_thread, "LOCK_start_thread", PSI_FLAG_GLOBAL},
......@@ -1043,7 +1037,6 @@ PSI_cond_key key_BINLOG_COND_xid_list,
key_relay_log_info_start_cond, key_relay_log_info_stop_cond,
key_rpl_group_info_sleep_cond,
key_TABLE_SHARE_cond, key_user_level_lock_cond,
key_COND_thread_cache, key_COND_flush_thread_cache,
key_COND_start_thread, key_COND_binlog_send,
key_BINLOG_COND_queue_busy;
PSI_cond_key key_RELAYLOG_COND_relay_log_updated,
......@@ -1093,8 +1086,6 @@ static PSI_cond_info all_server_conds[]=
{ &key_rpl_group_info_sleep_cond, "Rpl_group_info::sleep_cond", 0},
{ &key_TABLE_SHARE_cond, "TABLE_SHARE::cond", 0},
{ &key_user_level_lock_cond, "User_level_lock::cond", 0},
{ &key_COND_thread_cache, "COND_thread_cache", PSI_FLAG_GLOBAL},
{ &key_COND_flush_thread_cache, "COND_flush_thread_cache", PSI_FLAG_GLOBAL},
{ &key_COND_rpl_thread, "COND_rpl_thread", 0},
{ &key_COND_rpl_thread_queue, "COND_rpl_thread_queue", 0},
{ &key_COND_rpl_thread_stop, "COND_rpl_thread_stop", 0},
......@@ -1728,8 +1719,7 @@ static void close_connections(void)
DBUG_ENTER("close_connections");
/* Clear thread cache */
kill_cached_threads++;
flush_thread_cache();
thread_cache.final_flush();
/* Abort listening to new connections */
DBUG_PRINT("quit",("Closing sockets"));
......@@ -2092,8 +2082,8 @@ static void clean_up_mutexes()
{
DBUG_ENTER("clean_up_mutexes");
server_threads.destroy();
thread_cache.destroy();
mysql_rwlock_destroy(&LOCK_grant);
mysql_mutex_destroy(&LOCK_thread_cache);
mysql_mutex_destroy(&LOCK_start_thread);
mysql_mutex_destroy(&LOCK_status);
mysql_rwlock_destroy(&LOCK_all_status_vars);
......@@ -2127,9 +2117,7 @@ static void clean_up_mutexes()
mysql_mutex_destroy(&LOCK_short_uuid_generator);
mysql_mutex_destroy(&LOCK_prepared_stmt_count);
mysql_mutex_destroy(&LOCK_error_messages);
mysql_cond_destroy(&COND_thread_cache);
mysql_cond_destroy(&COND_start_thread);
mysql_cond_destroy(&COND_flush_thread_cache);
mysql_mutex_destroy(&LOCK_server_started);
mysql_cond_destroy(&COND_server_started);
mysql_mutex_destroy(&LOCK_prepare_ordered);
......@@ -2628,93 +2616,6 @@ void unlink_thd(THD *thd)
}
/*
Store thread in cache for reuse by new connections
SYNOPSIS
cache_thread()
thd Thread handler
NOTES
LOCK_thread_cache is used to protect the cache variables
RETURN
0 Thread was not put in cache
1 Thread is to be reused by new connection.
(ie, caller should return, not abort with pthread_exit())
*/
CONNECT *cache_thread(THD *thd)
{
struct timespec abstime;
CONNECT *connect;
bool flushed= false;
DBUG_ENTER("cache_thread");
DBUG_ASSERT(thd);
set_timespec(abstime, THREAD_CACHE_TIMEOUT);
/*
Delete the instrumentation for the job that just completed,
before parking this pthread in the cache (blocked on COND_thread_cache).
*/
PSI_CALL_delete_current_thread();
#ifndef DBUG_OFF
while (_db_is_pushed_())
_db_pop_();
#endif
mysql_mutex_lock(&LOCK_thread_cache);
if ((connect= thread_cache.get()))
cached_thread_count++;
else if (cached_thread_count < thread_cache_size && !kill_cached_threads)
{
/* Don't kill the thread, just put it in cache for reuse */
DBUG_PRINT("info", ("Adding thread to cache"));
cached_thread_count++;
for (;;)
{
int error= mysql_cond_timedwait(&COND_thread_cache, &LOCK_thread_cache,
&abstime);
flushed= kill_cached_threads;
if ((connect= thread_cache.get()))
break;
else if (flushed || error == ETIMEDOUT || error == ETIME)
{
/*
If timeout, end thread.
If a new thread is requested, we will handle
the call, even if we got a timeout (as we are already awake and free)
*/
cached_thread_count--;
break;
}
}
}
mysql_mutex_unlock(&LOCK_thread_cache);
if (flushed)
mysql_cond_signal(&COND_flush_thread_cache);
DBUG_RETURN(connect);
}
void flush_thread_cache()
{
DBUG_ENTER("flush_thread_cache");
mysql_mutex_lock(&LOCK_thread_cache);
kill_cached_threads++;
while (cached_thread_count)
{
mysql_cond_broadcast(&COND_thread_cache);
mysql_cond_wait(&COND_flush_thread_cache, &LOCK_thread_cache);
}
kill_cached_threads--;
mysql_mutex_unlock(&LOCK_thread_cache);
DBUG_VOID_RETURN;
}
/******************************************************************************
Setup a signal thread with handles all signals.
Because Linux doesn't support schemas use a mutex to check that
......@@ -4277,7 +4178,6 @@ static int init_thread_environment()
{
DBUG_ENTER("init_thread_environment");
server_threads.init();
mysql_mutex_init(key_LOCK_thread_cache, &LOCK_thread_cache, MY_MUTEX_INIT_FAST);
mysql_mutex_init(key_LOCK_start_thread, &LOCK_start_thread, MY_MUTEX_INIT_FAST);
mysql_mutex_init(key_LOCK_status, &LOCK_status, MY_MUTEX_INIT_FAST);
mysql_mutex_init(key_LOCK_delayed_insert,
......@@ -4339,9 +4239,7 @@ static int init_thread_environment()
mysql_rwlock_init(key_rwlock_LOCK_ssl_refresh, &LOCK_ssl_refresh);
mysql_rwlock_init(key_rwlock_LOCK_grant, &LOCK_grant);
mysql_rwlock_init(key_rwlock_LOCK_all_status_vars, &LOCK_all_status_vars);
mysql_cond_init(key_COND_thread_cache, &COND_thread_cache, NULL);
mysql_cond_init(key_COND_start_thread, &COND_start_thread, NULL);
mysql_cond_init(key_COND_flush_thread_cache, &COND_flush_thread_cache, NULL);
#ifdef HAVE_REPLICATION
mysql_mutex_init(key_LOCK_rpl_status, &LOCK_rpl_status, MY_MUTEX_INIT_FAST);
#endif
......@@ -6038,18 +5936,8 @@ void create_thread_to_handle_connection(CONNECT *connect)
{
DBUG_ENTER("create_thread_to_handle_connection");
mysql_mutex_lock(&LOCK_thread_cache);
if (cached_thread_count)
{
/* Get thread from cache */
thread_cache.push_back(connect);
cached_thread_count--;
mysql_mutex_unlock(&LOCK_thread_cache);
mysql_cond_signal(&COND_thread_cache);
DBUG_PRINT("info",("Thread created"));
if (thread_cache.enqueue(connect))
DBUG_VOID_RETURN;
}
mysql_mutex_unlock(&LOCK_thread_cache);
/* Create new thread to handle connection */
inc_thread_created();
......@@ -7372,6 +7260,17 @@ static int show_threadpool_threads(THD *thd, SHOW_VAR *var, char *buff,
}
#endif
static int show_cached_thread_count(THD *thd, SHOW_VAR *var, char *buff,
enum enum_var_type scope)
{
var->type= SHOW_LONG;
var->value= buff;
*(reinterpret_cast<ulong*>(buff))= thread_cache.size();
return 0;
}
/*
Variables shown by SHOW STATUS in alphabetical order
*/
......@@ -7591,7 +7490,7 @@ SHOW_VAR status_vars[]= {
{"Threadpool_idle_threads", (char *) &show_threadpool_idle_threads, SHOW_SIMPLE_FUNC},
{"Threadpool_threads", (char *) &show_threadpool_threads, SHOW_SIMPLE_FUNC},
#endif
{"Threads_cached", (char*) &cached_thread_count, SHOW_LONG_NOFLUSH},
{"Threads_cached", (char*) &show_cached_thread_count, SHOW_SIMPLE_FUNC},
{"Threads_connected", (char*) &connection_count, SHOW_INT},
{"Threads_created", (char*) &thread_created, SHOW_LONG_NOFLUSH},
{"Threads_running", (char*) offsetof(STATUS_VAR, threads_running), SHOW_UINT32_STATUS},
......@@ -7786,9 +7685,8 @@ static int mysql_init_variables(void)
mqh_used= 0;
cleanup_done= 0;
test_flags= select_errors= dropping_tables= ha_open_options=0;
thread_count= kill_cached_threads= 0;
thread_count= 0;
slave_open_temp_tables= 0;
cached_thread_count= 0;
opt_endinfo= using_udf_functions= 0;
opt_using_transactions= 0;
abort_loop= select_thread_in_use= signal_thread_in_use= 0;
......@@ -7834,7 +7732,7 @@ static int mysql_init_variables(void)
global_query_id= 1;
global_thread_id= 0;
strnmov(server_version, MYSQL_SERVER_VERSION, sizeof(server_version)-1);
thread_cache.empty();
thread_cache.init();
key_caches.empty();
if (!(dflt_key_cache= get_or_create_key_cache(default_key_cache_base.str,
default_key_cache_base.length)))
......
......@@ -79,8 +79,6 @@ void close_connection(THD *thd, uint sql_errno= 0);
void handle_connection_in_main_thread(CONNECT *thd);
void create_thread_to_handle_connection(CONNECT *connect);
void unlink_thd(THD *thd);
CONNECT *cache_thread(THD *thd);
void flush_thread_cache();
void refresh_status(THD *thd);
bool is_secure_file_path(char *path);
extern void init_net_server_extension(THD *thd);
......@@ -237,7 +235,6 @@ extern ulong slave_trans_retries;
extern ulong slave_trans_retry_interval;
extern uint slave_net_timeout;
extern int max_user_connections;
extern volatile ulong cached_thread_count;
extern ulong what_to_log,flush_time;
extern uint max_prepared_stmt_count, prepared_stmt_count;
extern MYSQL_PLUGIN_IMPORT ulong open_files_limit;
......@@ -373,8 +370,7 @@ extern PSI_cond_key key_BINLOG_COND_xid_list, key_BINLOG_update_cond,
key_relay_log_info_start_cond, key_relay_log_info_stop_cond,
key_rpl_group_info_sleep_cond,
key_TABLE_SHARE_cond, key_user_level_lock_cond,
key_COND_start_thread,
key_COND_thread_cache, key_COND_flush_thread_cache;
key_COND_start_thread;
extern PSI_cond_key key_RELAYLOG_COND_relay_log_updated,
key_RELAYLOG_COND_bin_log_updated, key_COND_wakeup_ready,
key_COND_wait_commit;
......
......@@ -28,6 +28,7 @@
#endif
#include "sql_audit.h"
#include "sql_connect.h"
#include "thread_cache.h"
#include "probes_mysql.h"
#include "sql_parse.h" // sql_command_flags,
// execute_init_command,
......@@ -1420,7 +1421,7 @@ void do_handle_one_connection(CONNECT *connect, bool put_in_cache)
unlink_thd(thd);
if (IF_WSREP(thd->wsrep_applier, false) || !put_in_cache ||
!(connect= cache_thread(thd)))
!(connect= thread_cache.park()))
break;
/* Create new instrumentation for the new THD job */
......
......@@ -22,6 +22,7 @@
#include "sql_acl.h" // acl_reload
#include "sql_servers.h" // servers_reload
#include "sql_connect.h" // reset_mqh
#include "thread_cache.h"
#include "sql_base.h" // close_cached_tables
#include "sql_db.h" // my_dbopt_cleanup
#include "hostname.h" // hostname_cache_refresh
......@@ -351,7 +352,7 @@ bool reload_acl_and_cache(THD *thd, unsigned long long options,
if (thd && (options & REFRESH_STATUS))
refresh_status(thd);
if (options & REFRESH_THREADS)
flush_thread_cache();
thread_cache.flush();
#ifdef HAVE_REPLICATION
if (options & REFRESH_MASTER)
{
......
......@@ -27,6 +27,8 @@
#include "my_json_writer.h"
#include <hash.h>
#include <thr_alarm.h>
#include "sql_connect.h"
#include "thread_cache.h"
#if defined(HAVE_MALLINFO) && defined(HAVE_MALLOC_H)
#include <malloc.h>
#elif defined(HAVE_MALLINFO) && defined(HAVE_SYS_MALLOC_H)
......@@ -568,7 +570,7 @@ void mysql_print_status()
(void) my_getwd(current_dir, sizeof(current_dir),MYF(0));
printf("Current dir: %s\n", current_dir);
printf("Running threads: %d Cached threads: %lu Stack size: %ld\n",
count, cached_thread_count,
count, thread_cache.size(),
(long) my_thread_stack_size);
#ifdef EXTRA_DEBUG
thr_print_locks(); // Write some debug info
......@@ -641,7 +643,8 @@ Memory allocated by threads: %s\n",
llstr(info.uordblks, llbuff[4]),
llstr(info.fordblks, llbuff[5]),
llstr(info.keepcost, llbuff[6]),
llstr((count + cached_thread_count)* my_thread_stack_size + info.hblkhd + info.arena, llbuff[7]),
llstr((count + thread_cache.size()) * my_thread_stack_size +
info.hblkhd + info.arena, llbuff[7]),
llstr(tmp.global_memory_used, llbuff[8]),
llstr(tmp.local_memory_used, llbuff[9]));
......
/*
Copyright (C) 2020 MariaDB Foundation
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
the Free Software Foundation; version 2 of the License.
This program is distributed in the hope that it will be useful,
but WITHOUT ANY WARRANTY; without even the implied warranty of
MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
GNU General Public License for more details.
You should have received a copy of the GNU General Public License
along with this program; if not, write to the Free Software
Foundation, Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1335 USA
*/
/**
MariaDB thread cache for "one thread per connection" scheduler.
Thread cache allows to re-use threads (as well as THD objects) for
subsequent connections.
*/
class Thread_cache
{
mutable mysql_cond_t COND_thread_cache;
mutable mysql_cond_t COND_flush_thread_cache;
mutable mysql_mutex_t LOCK_thread_cache;
/** Queue of new connection requests. */
I_List<CONNECT> list;
/** Number of threads parked in the cache. */
ulong cached_thread_count;
/** Number of active flush requests. */
uint32_t kill_cached_threads;
/**
PFS stuff, only used during initialization.
Unfortunately needs to survive till destruction.
*/
PSI_cond_key key_COND_thread_cache, key_COND_flush_thread_cache;
PSI_mutex_key key_LOCK_thread_cache;
public:
void init()
{
#ifdef HAVE_PSI_INTERFACE
PSI_cond_info conds[]=
{
{ &key_COND_thread_cache, "COND_thread_cache", PSI_FLAG_GLOBAL },
{ &key_COND_flush_thread_cache, "COND_flush_thread_cache",
PSI_FLAG_GLOBAL }
};
PSI_mutex_info mutexes[]=
{
{ &key_LOCK_thread_cache, "LOCK_thread_cache", PSI_FLAG_GLOBAL }
};
mysql_mutex_register("sql", mutexes, array_elements(mutexes));
mysql_cond_register("sql", conds, array_elements(conds));
#endif
mysql_mutex_init(key_LOCK_thread_cache, &LOCK_thread_cache,
MY_MUTEX_INIT_FAST);
mysql_cond_init(key_COND_thread_cache, &COND_thread_cache, 0);
mysql_cond_init(key_COND_flush_thread_cache, &COND_flush_thread_cache, 0);
list.empty();
kill_cached_threads= 0;
cached_thread_count= 0;
}
void destroy()
{
DBUG_ASSERT(cached_thread_count == 0);
DBUG_ASSERT(list.is_empty());
mysql_cond_destroy(&COND_flush_thread_cache);
mysql_cond_destroy(&COND_thread_cache);
mysql_mutex_destroy(&LOCK_thread_cache);
}
/**
Flushes thread cache.
Awakes parked threads and requests them to shutdown.
Waits until last parked thread leaves the cache.
*/
void flush()
{
mysql_mutex_lock(&LOCK_thread_cache);
kill_cached_threads++;
while (cached_thread_count)
{
mysql_cond_broadcast(&COND_thread_cache);
mysql_cond_wait(&COND_flush_thread_cache, &LOCK_thread_cache);
}
kill_cached_threads--;
mysql_mutex_unlock(&LOCK_thread_cache);
}
/**
Flushes thread cache and forbids threads parking in the cache.
This is a pre-shutdown hook.
*/
void final_flush()
{
kill_cached_threads++;
flush();
}
/**
Requests parked thread to serve new connection.
@return
@retval true connection is enqueued and parked thread is about to serve it
@retval false thread cache is empty
*/
bool enqueue(CONNECT *connect)
{
mysql_mutex_lock(&LOCK_thread_cache);
if (cached_thread_count)
{
list.push_back(connect);
cached_thread_count--;
mysql_mutex_unlock(&LOCK_thread_cache);
mysql_cond_signal(&COND_thread_cache);
return true;
}
mysql_mutex_unlock(&LOCK_thread_cache);
return false;
}
/**
Parks thread in the cache.
Thread execution is suspended until either of the following occurs:
- thread is requested to serve new connection;
- thread cache is flushed;
- THREAD_CACHE_TIMEOUT elapsed.
@return
@retval pointer to CONNECT if requested to serve new connection
@retval 0 if thread cache is flushed or on timeout
*/
CONNECT *park()
{
struct timespec abstime;
CONNECT *connect;
bool flushed= false;
DBUG_ENTER("Thread_cache::park");
set_timespec(abstime, THREAD_CACHE_TIMEOUT);
/*
Delete the instrumentation for the job that just completed,
before parking this pthread in the cache (blocked on COND_thread_cache).
*/
PSI_CALL_delete_current_thread();
#ifndef DBUG_OFF
while (_db_is_pushed_())
_db_pop_();
#endif
mysql_mutex_lock(&LOCK_thread_cache);
if ((connect= list.get()))
cached_thread_count++;
else if (cached_thread_count < thread_cache_size && !kill_cached_threads)
{
/* Don't kill the thread, just put it in cache for reuse */
DBUG_PRINT("info", ("Adding thread to cache"));
cached_thread_count++;
for (;;)
{
int error= mysql_cond_timedwait(&COND_thread_cache, &LOCK_thread_cache,
&abstime);
flushed= kill_cached_threads;
if ((connect= list.get()))
break;
else if (flushed || error == ETIMEDOUT || error == ETIME)
{
/*
If timeout, end thread.
If a new thread is requested, we will handle the call, even if we
got a timeout (as we are already awake and free)
*/
cached_thread_count--;
break;
}
}
}
mysql_mutex_unlock(&LOCK_thread_cache);
if (flushed)
mysql_cond_signal(&COND_flush_thread_cache);
DBUG_RETURN(connect);
}
/** Returns the number of parked threads. */
ulong size() const
{
mysql_mutex_lock(&LOCK_thread_cache);
ulong r= cached_thread_count;
mysql_mutex_unlock(&LOCK_thread_cache);
return r;
}
};
extern Thread_cache thread_cache;
......@@ -46,6 +46,8 @@
#include <cstdlib>
#include <string>
#include "log_event.h"
#include "sql_connect.h"
#include "thread_cache.h"
#include <sstream>
......@@ -64,8 +66,6 @@ const char *wsrep_SR_store_types[]= { "none", "table", NullS };
*/
extern my_bool plugins_are_initialized;
extern uint kill_cached_threads;
extern mysql_cond_t COND_thread_cache;
/* System variables. */
const char *wsrep_provider;
......@@ -2593,8 +2593,7 @@ static my_bool kill_remaining_threads(THD *thd, THD *caller_thd)
void wsrep_close_client_connections(my_bool wait_to_end, THD* except_caller_thd)
{
/* Clear thread cache */
kill_cached_threads++;
flush_thread_cache();
thread_cache.final_flush();
/*
First signal all threads that it's time to die
......
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment