MDEV-16264 fixup: Remove unused code and data

LATCH_ID_OS_AIO_READ_MUTEX,
LATCH_ID_OS_AIO_WRITE_MUTEX,
LATCH_ID_OS_AIO_LOG_MUTEX,
LATCH_ID_OS_AIO_IBUF_MUTEX,
LATCH_ID_OS_AIO_SYNC_MUTEX: Remove. The tpool is not instrumented.

lock_set_timeout_event(): Remove.

srv_sys_mutex_key, srv_sys_t::mutex, SYNC_THREADS: Remove.

srv_slot_t::suspended: Remove. We only ever assigned this data member
true, so it is redundant.

ib_wqueue_wait(), ib_wqueue_timedwait(): Remove.

os_thread_join(): Remove.

os_thread_create(), os_thread_exit(): Remove redundant parameters.

These were missed in commit 5e62b6a5e0.
This commit is contained in:
Marko Mäkelä 2020-09-30 14:28:11 +03:00
parent c7399db645
commit a9550c47e4
20 changed files with 24 additions and 246 deletions

View file

@ -956,7 +956,7 @@ run_data_threads(datadir_iter_t *it, os_thread_func_t func, uint n)
data_threads[i].n_thread = i + 1;
data_threads[i].count = &count;
data_threads[i].count_mutex = &count_mutex;
os_thread_create(func, data_threads + i, &data_threads[i].id);
data_threads[i].id = os_thread_create(func, data_threads + i);
}
/* Wait for threads to exit */

View file

@ -75,7 +75,6 @@ bool have_multi_threaded_slave = false;
bool have_gtid_slave = false;
/* Kill long selects */
os_thread_id_t kill_query_thread_id;
os_event_t kill_query_thread_started;
os_event_t kill_query_thread_stopped;
os_event_t kill_query_thread_stop;
@ -856,7 +855,7 @@ start_query_killer()
kill_query_thread_started = os_event_create(0);
kill_query_thread_stopped = os_event_create(0);
os_thread_create(kill_query_thread, NULL, &kill_query_thread_id);
os_thread_create(kill_query_thread);
os_event_wait(kill_query_thread_started);
}

View file

@ -503,7 +503,7 @@ static os_event_t dbug_start_query_thread(
par->expect_errno = expected_errno;
par->done_event = os_event_create(0);
par->con = xb_mysql_connect();
os_thread_create(dbug_execute_in_new_connection, par, 0);
os_thread_create(dbug_execute_in_new_connection, par);
if (!wait_state)
return par->done_event;
@ -4060,9 +4060,6 @@ fail:
/* definition from recv_recovery_from_checkpoint_start() */
ulint max_cp_field;
/* start back ground thread to copy newer log */
os_thread_id_t log_copying_thread_id;
/* get current checkpoint_lsn */
/* Look for the latest checkpoint from any of the log groups */
@ -4152,14 +4149,11 @@ reread_log_header:
log_copying_running = true;
/* start io throttle */
if(xtrabackup_throttle) {
os_thread_id_t io_watching_thread_id;
io_ticket = xtrabackup_throttle;
wait_throttle = os_event_create(0);
io_watching_thread_running = true;
os_thread_create(io_watching_thread, NULL,
&io_watching_thread_id);
os_thread_create(io_watching_thread);
}
/* Populate fil_system with tablespaces to copy */
@ -4182,7 +4176,7 @@ fail_before_log_copying_thread_start:
DBUG_MARIABACKUP_EVENT("before_innodb_log_copy_thread_started",0);
log_copying_stop = os_event_create(0);
os_thread_create(log_copying_thread, NULL, &log_copying_thread_id);
os_thread_create(log_copying_thread);
/* FLUSH CHANGED_PAGE_BITMAPS call */
if (!flush_changed_page_bitmaps()) {
@ -4224,8 +4218,8 @@ fail_before_log_copying_thread_start:
data_threads[i].num = i+1;
data_threads[i].count = &count;
data_threads[i].count_mutex = &count_mutex;
os_thread_create(data_copy_thread_func, data_threads + i,
&data_threads[i].id);
data_threads[i].id = os_thread_create(data_copy_thread_func,
data_threads + i);
}
/* Wait for threads to exit */

View file

@ -2872,7 +2872,7 @@ void buf_flush_page_cleaner_init()
page_cleaner.is_running = true;
buf_page_cleaner_is_active = true;
os_thread_create(buf_flush_page_cleaner, NULL, NULL);
os_thread_create(buf_flush_page_cleaner);
}
/** Synchronously flush dirty blocks.

View file

@ -2219,11 +2219,10 @@ fil_crypt_set_thread_cnt(
uint add = new_cnt - srv_n_fil_crypt_threads;
srv_n_fil_crypt_threads = new_cnt;
for (uint i = 0; i < add; i++) {
os_thread_id_t rotation_thread_id;
os_thread_create(fil_crypt_thread, NULL, &rotation_thread_id);
ib::info() << "Creating #"
<< i+1 << " encryption thread id "
<< os_thread_pf(rotation_thread_id)
<< os_thread_pf(
os_thread_create(fil_crypt_thread))
<< " total threads " << new_cnt << ".";
}
} else if (new_cnt < srv_n_fil_crypt_threads) {

View file

@ -542,7 +542,6 @@ static PSI_mutex_info all_innodb_mutexes[] = {
PSI_KEY(buf_dblwr_mutex),
PSI_KEY(trx_pool_mutex),
PSI_KEY(trx_pool_manager_mutex),
PSI_KEY(srv_sys_mutex),
PSI_KEY(lock_mutex),
PSI_KEY(lock_wait_mutex),
PSI_KEY(trx_mutex),

View file

@ -670,11 +670,6 @@ lock_trx_lock_list_init(
/*====================*/
trx_lock_list_t* lock_list); /*!< List to initialise */
/*******************************************************************//**
Set the lock system timeout event. */
void
lock_set_timeout_event();
/*====================*/
/*********************************************************************//**
Checks that a transaction id is sensible, i.e., not in the future.
@return true if ok */

View file

@ -1,7 +1,7 @@
/*****************************************************************************
Copyright (c) 1995, 2016, Oracle and/or its affiliates. All Rights Reserved.
Copyright (c) 2017, MariaDB Corporation.
Copyright (c) 2017, 2020, MariaDB Corporation.
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
@ -46,10 +46,6 @@ typedef LPTHREAD_START_ROUTINE os_thread_func_t;
/** Macro for specifying a Windows thread start function. */
#define DECLARE_THREAD(func) WINAPI func
#define os_thread_create(f,a,i) \
os_thread_create_func(f, a, i)
#else
typedef pthread_t os_thread_t;
@ -60,8 +56,6 @@ extern "C" { typedef void* (*os_thread_func_t)(void*); }
/** Macro for specifying a POSIX thread start function. */
#define DECLARE_THREAD(func) func
#define os_thread_create(f,a,i) os_thread_create_func(f, a, i)
#endif /* _WIN32 */
/* Define a function pointer type to use in a typecast */
@ -98,28 +92,10 @@ NOTE: We count the number of threads in os_thread_exit(). A created
thread should always use that to exit so thatthe thread count will be
decremented.
We do not return an error code because if there is one, we crash here. */
os_thread_t
os_thread_create_func(
/*==================*/
os_thread_func_t func, /*!< in: pointer to function
from which to start */
void* arg, /*!< in: argument to start
function */
os_thread_id_t* thread_id); /*!< out: id of the created
thread, or NULL */
os_thread_t os_thread_create(os_thread_func_t func, void *arg= nullptr);
/** Waits until the specified thread completes and joins it.
Its return value is ignored.
@param[in,out] thread thread to join */
void
os_thread_join(
os_thread_id_t thread);
/** Exits the current thread.
@param[in] detach if true, the thread will be detached right before
exiting. If false, another thread is responsible for joining this thread */
ATTRIBUTE_NORETURN ATTRIBUTE_COLD
void os_thread_exit(bool detach = true);
/** Detach and terminate the current thread. */
ATTRIBUTE_NORETURN void os_thread_exit();
/*****************************************************************//**
Returns the thread identifier of current thread.

View file

@ -893,9 +893,6 @@ struct export_var_t{
struct srv_slot_t{
ibool in_use; /*!< TRUE if this slot
is in use */
ibool suspended; /*!< TRUE if the thread is
waiting for the event of this
slot */
/** time(NULL) when the thread was suspended.
FIXME: Use my_interval_timer() or similar, to avoid bogus
timeouts in lock_wait_check_and_cancel() or lock_wait_suspend_thread()

View file

@ -94,7 +94,6 @@ extern mysql_pfs_key_t trx_pool_manager_mutex_key;
extern mysql_pfs_key_t lock_mutex_key;
extern mysql_pfs_key_t lock_wait_mutex_key;
extern mysql_pfs_key_t trx_sys_mutex_key;
extern mysql_pfs_key_t srv_sys_mutex_key;
extern mysql_pfs_key_t srv_threads_mutex_key;
extern mysql_pfs_key_t event_mutex_key;
extern mysql_pfs_key_t event_manager_mutex_key;

View file

@ -216,7 +216,6 @@ enum latch_level_t {
SYNC_PURGE_QUEUE,
SYNC_TRX_SYS_HEADER,
SYNC_REC_LOCK,
SYNC_THREADS,
SYNC_TRX,
SYNC_RW_TRX_HASH_ELEMENT,
SYNC_READ_VIEW,
@ -328,11 +327,6 @@ enum latch_id_t {
LATCH_ID_EVENT_MANAGER,
LATCH_ID_EVENT_MUTEX,
LATCH_ID_SYNC_ARRAY_MUTEX,
LATCH_ID_OS_AIO_READ_MUTEX,
LATCH_ID_OS_AIO_WRITE_MUTEX,
LATCH_ID_OS_AIO_LOG_MUTEX,
LATCH_ID_OS_AIO_IBUF_MUTEX,
LATCH_ID_OS_AIO_SYNC_MUTEX,
LATCH_ID_ROW_DROP_LIST,
LATCH_ID_INDEX_ONLINE_LOG,
LATCH_ID_WORK_QUEUE,

View file

@ -1,7 +1,7 @@
/*****************************************************************************
Copyright (c) 2006, 2014, Oracle and/or its affiliates. All Rights Reserved.
Copyright (c) 2017, 2019, MariaDB Corporation.
Copyright (c) 2017, 2020, MariaDB Corporation.
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
@ -79,23 +79,6 @@ ib_wqueue_add(ib_wqueue_t* wq, void* item, mem_heap_t* heap,
@return whether the queue is empty */
bool ib_wqueue_is_empty(ib_wqueue_t* wq);
/****************************************************************//**
Wait for a work item to appear in the queue.
@return work item */
void*
ib_wqueue_wait(
/*===========*/
ib_wqueue_t* wq); /*!< in: work queue */
/********************************************************************
Wait for a work item to appear in the queue for specified time. */
void*
ib_wqueue_timedwait(
/*================*/
/* out: work item or NULL on timeout*/
ib_wqueue_t* wq, /* in: work queue */
ulint wait_in_usecs); /* in: wait time in micro seconds */
/********************************************************************
Return first item on work queue or NULL if queue is empty
@return work item or NULL */

View file

@ -53,10 +53,9 @@ lock_wait_table_print(void)
fprintf(stderr,
"Slot %lu:"
" in use %lu, susp %lu, timeout %lu, time %lu\n",
" in use %lu, timeout %lu, time %lu\n",
(ulong) i,
(ulong) slot->in_use,
(ulong) slot->suspended,
slot->wait_timeout,
(ulong) difftime(time(NULL), slot->suspend_time));
}
@ -154,7 +153,6 @@ lock_wait_table_reserve_slot(
}
os_event_reset(slot->event);
slot->suspended = TRUE;
slot->suspend_time = time(NULL);
slot->wait_timeout = wait_timeout;
@ -443,7 +441,6 @@ lock_wait_check_and_cancel(
{
ut_ad(lock_wait_mutex_own());
ut_ad(slot->in_use);
ut_ad(slot->suspended);
double wait_time = difftime(time(NULL), slot->suspend_time);
trx_t* trx = thr_get_trx(slot->thr);

View file

@ -3577,7 +3577,7 @@ completed:
/* Spawn the background thread to flush dirty pages
from the buffer pools. */
recv_writer_thread_active = true;
os_thread_create(recv_writer_thread, 0, 0);
os_thread_create(recv_writer_thread);
if (rescan) {
contiguous_lsn = checkpoint_lsn;

View file

@ -1,7 +1,7 @@
/*****************************************************************************
Copyright (c) 1995, 2016, Oracle and/or its affiliates. All Rights Reserved.
Copyright (c) 2017, 2018, MariaDB Corporation.
Copyright (c) 2017, 2020, MariaDB Corporation.
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
@ -89,15 +89,7 @@ NOTE: We count the number of threads in os_thread_exit(). A created
thread should always use that to exit so thatthe thread count will be
decremented.
We do not return an error code because if there is one, we crash here. */
os_thread_t
os_thread_create_func(
/*==================*/
os_thread_func_t func, /*!< in: pointer to function
from which to start */
void* arg, /*!< in: argument to start
function */
os_thread_id_t* thread_id) /*!< out: id of the created
thread, or NULL */
os_thread_t os_thread_create(os_thread_func_t func, void *arg)
{
os_thread_id_t new_thread_id;
@ -145,39 +137,11 @@ os_thread_create_func(
ut_a(os_thread_count <= srv_max_n_threads);
/* Return the thread_id if the caller requests it. */
if (thread_id != NULL) {
*thread_id = new_thread_id;
}
return((os_thread_t)new_thread_id);
}
/** Waits until the specified thread completes and joins it.
Its return value is ignored.
@param[in,out] thread thread to join */
void
os_thread_join(
os_thread_id_t thread)
{
#ifdef _WIN32
/* Do nothing. */
#else
#ifdef UNIV_DEBUG
const int ret =
#endif /* UNIV_DEBUG */
pthread_join(thread, NULL);
/* Waiting on already-quit threads is allowed. */
ut_ad(ret == 0 || ret == ESRCH);
#endif /* _WIN32 */
}
/** Exits the current thread.
@param[in] detach if true, the thread will be detached right before
exiting. If false, another thread is responsible for joining this thread */
ATTRIBUTE_NORETURN
void
os_thread_exit(bool detach)
/** Detach and terminate the current thread. */
ATTRIBUTE_NORETURN void os_thread_exit()
{
#ifdef UNIV_DEBUG_THREAD_CREATION
ib::info() << "Thread exits, id "
@ -193,9 +157,7 @@ os_thread_exit(bool detach)
#ifdef _WIN32
ExitThread(0);
#else
if (detach) {
pthread_detach(pthread_self());
}
pthread_detach(pthread_self());
pthread_exit(NULL);
#endif
}

View file

@ -459,16 +459,6 @@ UNIV_INTERN uint srv_simulate_comp_failures;
/** Buffer pool dump status frequence in percentages */
UNIV_INTERN ulong srv_buf_dump_status_frequency;
/** Acquire the system_mutex. */
#define srv_sys_mutex_enter() do { \
mutex_enter(&srv_sys.mutex); \
} while (0)
/** Release the system mutex. */
#define srv_sys_mutex_exit() do { \
mutex_exit(&srv_sys.mutex); \
} while (0)
/*
IMPLEMENTATION OF THE SERVER MAIN PROGRAM
=========================================
@ -548,8 +538,6 @@ struct srv_sys_t{
UT_LIST_BASE_NODE_T(que_thr_t)
tasks; /*!< task queue */
ib_mutex_t mutex; /*!< variable protecting the
fields below. */
srv_stats_t::ulint_ctr_1_t
activity_count; /*!< For tracking server
activity */
@ -727,8 +715,6 @@ static void srv_init()
srv_thread_pool_init();
if (!srv_read_only_mode) {
mutex_create(LATCH_ID_SRV_SYS, &srv_sys.mutex);
mutex_create(LATCH_ID_SRV_SYS_TASKS, &srv_sys.tasks_mutex);
@ -778,7 +764,6 @@ srv_free(void)
mutex_free(&page_zip_stat_per_index_mutex);
if (!srv_read_only_mode) {
mutex_free(&srv_sys.mutex);
mutex_free(&srv_sys.tasks_mutex);
os_event_destroy(buf_flush_event);
}
@ -1513,7 +1498,6 @@ void
srv_wake_purge_thread_if_not_active()
{
ut_ad(!srv_read_only_mode);
ut_ad(!mutex_own(&srv_sys.mutex));
if (purge_sys.enabled() && !purge_sys.paused()
&& trx_sys.rseg_history_len) {

View file

@ -1893,7 +1893,7 @@ file_checked:
/* Rollback incomplete non-DDL transactions */
trx_rollback_is_active = true;
os_thread_create(trx_rollback_all_recovered, 0, 0);
os_thread_create(trx_rollback_all_recovered);
}
}

View file

@ -473,7 +473,6 @@ LatchDebug::LatchDebug()
LEVEL_MAP_INSERT(SYNC_PURGE_QUEUE);
LEVEL_MAP_INSERT(SYNC_TRX_SYS_HEADER);
LEVEL_MAP_INSERT(SYNC_REC_LOCK);
LEVEL_MAP_INSERT(SYNC_THREADS);
LEVEL_MAP_INSERT(SYNC_TRX);
LEVEL_MAP_INSERT(SYNC_RW_TRX_HASH_ELEMENT);
LEVEL_MAP_INSERT(SYNC_READ_VIEW);
@ -752,7 +751,6 @@ LatchDebug::check_order(
case SYNC_LOG_FLUSH_ORDER:
case SYNC_DOUBLEWRITE:
case SYNC_SEARCH_SYS:
case SYNC_THREADS:
case SYNC_LOCK_SYS:
case SYNC_LOCK_WAIT_SYS:
case SYNC_RW_TRX_HASH_ELEMENT:
@ -1358,8 +1356,6 @@ sync_latch_meta_init()
LATCH_ADD_MUTEX(TRX_SYS, SYNC_TRX_SYS, trx_sys_mutex_key);
LATCH_ADD_MUTEX(SRV_SYS, SYNC_THREADS, srv_sys_mutex_key);
LATCH_ADD_MUTEX(SRV_SYS_TASKS, SYNC_ANY_LATCH, srv_threads_mutex_key);
LATCH_ADD_MUTEX(PAGE_ZIP_STAT_PER_INDEX, SYNC_ANY_LATCH,
@ -1378,21 +1374,6 @@ sync_latch_meta_init()
LATCH_ADD_MUTEX(SYNC_ARRAY_MUTEX, SYNC_NO_ORDER_CHECK,
sync_array_mutex_key);
LATCH_ADD_MUTEX(OS_AIO_READ_MUTEX, SYNC_NO_ORDER_CHECK,
PFS_NOT_INSTRUMENTED);
LATCH_ADD_MUTEX(OS_AIO_WRITE_MUTEX, SYNC_NO_ORDER_CHECK,
PFS_NOT_INSTRUMENTED);
LATCH_ADD_MUTEX(OS_AIO_LOG_MUTEX, SYNC_NO_ORDER_CHECK,
PFS_NOT_INSTRUMENTED);
LATCH_ADD_MUTEX(OS_AIO_IBUF_MUTEX, SYNC_NO_ORDER_CHECK,
PFS_NOT_INSTRUMENTED);
LATCH_ADD_MUTEX(OS_AIO_SYNC_MUTEX, SYNC_NO_ORDER_CHECK,
PFS_NOT_INSTRUMENTED);
LATCH_ADD_MUTEX(ROW_DROP_LIST, SYNC_NO_ORDER_CHECK,
row_drop_list_mutex_key);

View file

@ -80,7 +80,6 @@ mysql_pfs_key_t trx_pool_manager_mutex_key;
mysql_pfs_key_t lock_mutex_key;
mysql_pfs_key_t lock_wait_mutex_key;
mysql_pfs_key_t trx_sys_mutex_key;
mysql_pfs_key_t srv_sys_mutex_key;
mysql_pfs_key_t srv_threads_mutex_key;
mysql_pfs_key_t event_mutex_key;
mysql_pfs_key_t event_manager_mutex_key;

View file

@ -1,7 +1,7 @@
/*****************************************************************************
Copyright (c) 2006, 2015, Oracle and/or its affiliates. All Rights Reserved.
Copyright (c) 2019, MariaDB Corporation.
Copyright (c) 2019, 2020, MariaDB Corporation.
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
@ -83,86 +83,6 @@ ib_wqueue_add(ib_wqueue_t* wq, void* item, mem_heap_t* heap, bool wq_locked)
}
}
/****************************************************************//**
Wait for a work item to appear in the queue.
@return work item */
void*
ib_wqueue_wait(
/*===========*/
ib_wqueue_t* wq) /*!< in: work queue */
{
ib_list_node_t* node;
for (;;) {
os_event_wait(wq->event);
mutex_enter(&wq->mutex);
node = ib_list_get_first(wq->items);
if (node) {
ib_list_remove(wq->items, node);
if (!ib_list_get_first(wq->items)) {
/* We must reset the event when the list
gets emptied. */
os_event_reset(wq->event);
}
break;
}
mutex_exit(&wq->mutex);
}
mutex_exit(&wq->mutex);
return(node->data);
}
/********************************************************************
Wait for a work item to appear in the queue for specified time. */
void*
ib_wqueue_timedwait(
/*================*/
/* out: work item or NULL on timeout*/
ib_wqueue_t* wq, /* in: work queue */
ulint wait_in_usecs) /* in: wait time in micro seconds */
{
ib_list_node_t* node = NULL;
for (;;) {
ulint error;
int64_t sig_count;
mutex_enter(&wq->mutex);
node = ib_list_get_first(wq->items);
if (node) {
ib_list_remove(wq->items, node);
mutex_exit(&wq->mutex);
break;
}
sig_count = os_event_reset(wq->event);
mutex_exit(&wq->mutex);
error = os_event_wait_time_low(wq->event,
(ulint) wait_in_usecs,
sig_count);
if (error == OS_SYNC_TIME_EXCEEDED) {
break;
}
}
return(node ? node->data : NULL);
}
/********************************************************************
Return first item on work queue or NULL if queue is empty
@return work item or NULL */