MDEV-32841 Provide Innodb async IO statistics

Provide some statistics about asynchronous IO reads and writes:
 - number of pending operations
 - number of completion callbacks that are currently being executed
 - number of completion callbacks that are currently queued
   (due to restriction on number of IO threads)
 - total number of IOs finished
 - total time to wait for free IO slot
 - total number of completions that were queued.

Also revert tpool InnoDB perfschema instrumentation (MDEV-31048)
That instrumentation of cache mutex did not bring any revelation (
the mutex is taken for a couple of instructions), and made it impossible
to use tpool outside of the server (e.g in mariadbimport/dump)
This commit is contained in:
Vladislav Vaintroub 2024-01-09 10:47:33 +01:00
parent 374783c3d9
commit 01466adc13
9 changed files with 190 additions and 55 deletions

View file

@ -5,6 +5,18 @@ AND variable_name NOT IN
'INNODB_MEM_ADAPTIVE_HASH',
'INNODB_BUFFERED_AIO_SUBMITTED','INNODB_BUFFER_POOL_PAGES_LATCHED');
variable_name
INNODB_ASYNC_READS_PENDING
INNODB_ASYNC_READS_TASKS_RUNNING
INNODB_ASYNC_READS_TOTAL_COUNT
INNODB_ASYNC_READS_TOTAL_ENQUEUES
INNODB_ASYNC_READS_QUEUE_SIZE
INNODB_ASYNC_READS_WAIT_SLOT_SEC
INNODB_ASYNC_WRITES_PENDING
INNODB_ASYNC_WRITES_TASKS_RUNNING
INNODB_ASYNC_WRITES_TOTAL_COUNT
INNODB_ASYNC_WRITES_TOTAL_ENQUEUES
INNODB_ASYNC_WRITES_QUEUE_SIZE
INNODB_ASYNC_WRITES_WAIT_SLOT_SEC
INNODB_BACKGROUND_LOG_SYNC
INNODB_BUFFER_POOL_DUMP_STATUS
INNODB_BUFFER_POOL_LOAD_STATUS

View file

@ -526,7 +526,6 @@ mysql_pfs_key_t trx_pool_manager_mutex_key;
mysql_pfs_key_t lock_wait_mutex_key;
mysql_pfs_key_t trx_sys_mutex_key;
mysql_pfs_key_t srv_threads_mutex_key;
mysql_pfs_key_t tpool_cache_mutex_key;
/* all_innodb_mutexes array contains mutexes that are
performance schema instrumented if "UNIV_PFS_MUTEX"
@ -558,7 +557,6 @@ static PSI_mutex_info all_innodb_mutexes[] = {
PSI_KEY(rtr_match_mutex),
PSI_KEY(rtr_path_mutex),
PSI_KEY(trx_sys_mutex),
PSI_KEY(tpool_cache_mutex),
};
# endif /* UNIV_PFS_MUTEX */
@ -886,6 +884,32 @@ static SHOW_VAR innodb_status_variables[]= {
{"adaptive_hash_non_hash_searches",
&export_vars.innodb_ahi_miss, SHOW_SIZE_T},
#endif
{"async_reads_pending",
&export_vars.async_read_stats.pending_ops, SHOW_SIZE_T},
{"async_reads_tasks_running",
&export_vars.async_read_stats.completion_stats.tasks_running, SHOW_SIZE_T},
{"async_reads_total_count",
&export_vars.async_read_stats.completion_stats.total_tasks_executed,SHOW_ULONGLONG},
{"async_reads_total_enqueues",
&export_vars.async_read_stats.completion_stats.total_tasks_enqueued,SHOW_ULONGLONG},
{"async_reads_queue_size",
&export_vars.async_read_stats.completion_stats.queue_size, SHOW_SIZE_T},
{"async_reads_wait_slot_sec",
&export_vars.async_read_stats.slot_wait_time_sec, SHOW_DOUBLE},
{"async_writes_pending",
&export_vars.async_write_stats.pending_ops,SHOW_SIZE_T},
{"async_writes_tasks_running",
&export_vars.async_write_stats.completion_stats.tasks_running, SHOW_SIZE_T},
{"async_writes_total_count",
&export_vars.async_write_stats.completion_stats.total_tasks_executed, SHOW_ULONGLONG},
{"async_writes_total_enqueues",
&export_vars.async_write_stats.completion_stats.total_tasks_enqueued, SHOW_ULONGLONG},
{"async_writes_queue_size",
&export_vars.async_write_stats.completion_stats.queue_size, SHOW_SIZE_T},
{"async_writes_wait_slot_sec",
&export_vars.async_write_stats.slot_wait_time_sec, SHOW_DOUBLE},
{"background_log_sync", &srv_log_writes_and_flush, SHOW_SIZE_T},
{"buffer_pool_dump_status",
(char*) &export_vars.innodb_buffer_pool_dump_status, SHOW_CHAR},

View file

@ -1187,4 +1187,44 @@ inline bool is_absolute_path(const char *path)
#include "os0file.inl"
/**
Structure used for async io statistics
There is one instance of this structure for each operation type
(read or write)
*/
struct innodb_async_io_stats_t
{
/**
Current of submitted and not yet finished IOs.
IO is considered finished when it finished in the OS
*and* the completion callback has been called
*/
size_t pending_ops;
/**
Time, in seconds, spent waiting for a slot to become
available. There is a limited number of slots for async IO
operations. If all slots are in use, the IO submission has
to wait.
*/
double slot_wait_time_sec;
/**
Information related to IO completion callbacks.
- number of tasks currently running (<= innodb_read/write_io_threads)
- total number of tasks that have been completed
- current task queue size . Queueing happens if running tasks is
maxed out (equal to innodb_read/write_io_threads)
- total number of tasks that have been queued
*/
tpool::group_stats completion_stats;
};
/**
Statistics for asynchronous I/O
@param[in] op operation - aio_opcode::AIO_PREAD or aio_opcode::AIO_PWRITE
@param[in] stats - structure to fill
*/
extern void innodb_io_slots_stats(tpool::aio_opcode op,
innodb_async_io_stats_t *stats);
#endif /* os0file_h */

View file

@ -573,6 +573,8 @@ struct export_var_t{
ulint innodb_ahi_hit;
ulint innodb_ahi_miss;
#endif /* BTR_CUR_HASH_ADAPT */
innodb_async_io_stats_t async_read_stats;
innodb_async_io_stats_t async_write_stats;
char innodb_buffer_pool_dump_status[OS_FILE_MAX_PATH + 128];/*!< Buf pool dump status */
char innodb_buffer_pool_load_status[OS_FILE_MAX_PATH + 128];/*!< Buf pool load status */
char innodb_buffer_pool_resize_status[512];/*!< Buf pool resize status */

View file

@ -80,7 +80,7 @@ Created 10/21/1995 Heikki Tuuri
class io_slots
{
private:
tpool::cache<tpool::aiocb> m_cache;
tpool::cache<tpool::aiocb, true> m_cache;
tpool::task_group m_group;
int m_max_aio;
public:
@ -106,9 +106,9 @@ public:
}
/* Wait for completions of all AIO operations */
void wait(mysql_mutex_t &m)
void wait(std::unique_lock<std::mutex> &lk)
{
m_cache.wait(m);
m_cache.wait(lk);
}
void wait()
@ -121,6 +121,11 @@ public:
return m_cache.pos();
}
std::chrono::duration<float> wait_time()
{
return m_cache.wait_time();
}
tpool::task_group* get_task_group()
{
return &m_group;
@ -131,7 +136,7 @@ public:
wait();
}
mysql_mutex_t& mutex()
std::mutex &mutex()
{
return m_cache.mutex();
}
@ -152,6 +157,22 @@ public:
static io_slots *read_slots;
static io_slots *write_slots;
/**
Statistics for asynchronous I/O
@param[in] op operation type (aio_opcode::AIO_PREAD or aio_opcode::AIO_PWRITE)
@param[in] stats pointer to the structure to fill
*/
void innodb_io_slots_stats(tpool::aio_opcode op, innodb_async_io_stats_t *stats)
{
io_slots *slots= op == tpool::aio_opcode::AIO_PREAD? read_slots : write_slots;
stats->pending_ops = slots->pending_io_count();
stats->slot_wait_time_sec=
std::chrono::duration_cast<std::chrono::duration<float>>(
slots->wait_time()).count();
slots->task_group().get_stats(&stats->completion_stats);
}
/** Number of retries for partial I/O's */
constexpr ulint NUM_RETRIES_ON_PARTIAL_IO = 10;
@ -3623,9 +3644,8 @@ more concurrent threads via thread_group setting.
int os_aio_resize(ulint n_reader_threads, ulint n_writer_threads)
{
/* Lock the slots, and wait until all current IOs finish.*/
auto &lk_read= read_slots->mutex(), &lk_write= write_slots->mutex();
mysql_mutex_lock(&lk_read);
mysql_mutex_lock(&lk_write);
std::unique_lock<std::mutex> lk_read(read_slots->mutex()),
lk_write(write_slots->mutex());
read_slots->wait(lk_read);
write_slots->wait(lk_write);
@ -3653,9 +3673,6 @@ int os_aio_resize(ulint n_reader_threads, ulint n_writer_threads)
read_slots->resize(max_read_events, static_cast<int>(n_reader_threads));
write_slots->resize(max_write_events, static_cast<int>(n_writer_threads));
}
mysql_mutex_unlock(&lk_read);
mysql_mutex_unlock(&lk_write);
return ret;
}
@ -3693,10 +3710,8 @@ void os_aio_wait_until_no_pending_writes(bool declare)
/** @return number of pending reads */
size_t os_aio_pending_reads()
{
mysql_mutex_lock(&read_slots->mutex());
size_t pending= read_slots->pending_io_count();
mysql_mutex_unlock(&read_slots->mutex());
return pending;
std::lock_guard<std::mutex> lock(read_slots->mutex());
return read_slots->pending_io_count();
}
/** @return approximate number of pending reads */
@ -3708,10 +3723,8 @@ size_t os_aio_pending_reads_approx()
/** @return number of pending writes */
size_t os_aio_pending_writes()
{
mysql_mutex_lock(&write_slots->mutex());
size_t pending= write_slots->pending_io_count();
mysql_mutex_unlock(&write_slots->mutex());
return pending;
std::lock_guard<std::mutex> lock(write_slots->mutex());
return write_slots->pending_io_count();
}
/** Wait until all pending asynchronous reads have completed.

View file

@ -813,6 +813,9 @@ srv_printf_innodb_monitor(
return(ret);
}
void innodb_io_slots_stats(tpool::aio_opcode op,
innodb_async_io_stats_t *stats);
/******************************************************************//**
Function to pass InnoDB status variables to MySQL */
void
@ -824,6 +827,10 @@ srv_export_innodb_status(void)
if (!srv_read_only_mode) {
fil_crypt_total_stat(&crypt_stat);
}
innodb_io_slots_stats(tpool::aio_opcode::AIO_PREAD,
&export_vars.async_read_stats);
innodb_io_slots_stats(tpool::aio_opcode::AIO_PWRITE,
&export_vars.async_write_stats);
#ifdef BTR_CUR_HASH_ADAPT
export_vars.innodb_ahi_hit = btr_cur_n_sea;

View file

@ -42,6 +42,8 @@ namespace tpool
:
m_queue(8),
m_mtx(),
m_total_tasks(0),
m_total_enqueues(0),
m_tasks_running(),
m_max_concurrent_tasks(max_concurrency),
m_enable_task_release(enable_task_release)
@ -59,6 +61,7 @@ namespace tpool
{
/* Queue for later execution by another thread.*/
m_queue.push(t);
m_total_enqueues++;
return;
}
m_tasks_running++;
@ -72,7 +75,7 @@ namespace tpool
t->release();
}
lk.lock();
m_total_tasks++;
if (m_queue.empty())
break;
t = m_queue.front();
@ -96,6 +99,15 @@ namespace tpool
}
}
void task_group::get_stats(group_stats *stats)
{
std::lock_guard<std::mutex> lk(m_mtx);
stats->tasks_running= m_tasks_running;
stats->queue_size= m_queue.size();
stats->total_tasks_executed= m_total_tasks;
stats->total_tasks_enqueued= m_total_enqueues;
}
task_group::~task_group()
{
std::unique_lock<std::mutex> lk(m_mtx);

View file

@ -57,6 +57,18 @@ typedef void (*callback_func)(void *);
typedef void (*callback_func_np)(void);
class task;
struct group_stats
{
/** Current number of running tasks*/
size_t tasks_running;
/** Current number of tasks in the queue*/
size_t queue_size;
/** Total number of tasks executed */
unsigned long long total_tasks_executed;
/** Total number of tasks enqueued */
unsigned long long total_tasks_enqueued;
};
/** A class that can be used e.g. for
restricting concurrency for specific class of tasks. */
@ -66,6 +78,8 @@ private:
circular_queue<task*> m_queue;
std::mutex m_mtx;
std::condition_variable m_cv;
unsigned long long m_total_tasks;
unsigned long long m_total_enqueues;
unsigned int m_tasks_running;
unsigned int m_max_concurrent_tasks;
const bool m_enable_task_release;
@ -75,6 +89,7 @@ public:
void set_max_tasks(unsigned int max_concurrent_tasks);
void execute(task* t);
void cancel_pending(task *t);
void get_stats(group_stats* stats);
~task_group();
};

View file

@ -20,7 +20,9 @@ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02111 - 1301 USA*/
#include <stack>
#include <assert.h>
#include <algorithm>
#include <chrono>
#include <condition_variable>
#include <mutex>
/* Suppress TSAN warnings, that we believe are not critical. */
#if defined(__has_feature)
#define TPOOL_HAS_FEATURE(...) __has_feature(__VA_ARGS__)
@ -36,10 +38,6 @@ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02111 - 1301 USA*/
#define TPOOL_SUPPRESS_TSAN
#endif
#ifdef HAVE_PSI_INTERFACE
typedef unsigned int mysql_pfs_key_t;
extern mysql_pfs_key_t tpool_cache_mutex_key;
#endif
namespace tpool
{
@ -56,16 +54,16 @@ namespace tpool
We assume that put() will only put back the elements that
were retrieved previously with get().
*/
template<typename T> class cache
template<typename T, bool timed=false> class cache
{
/** Protects updates of m_pos and m_cache members */
mysql_mutex_t m_mtx;
std::mutex m_mtx;
/**
Notify waiting threads about "cache full" or "cache not empty" conditions
@see get() and wait()
*/
pthread_cond_t m_cv;
std::condition_variable m_cv;
/** Cached items vector.Does not change after construction */
std::vector<T> m_base;
@ -84,6 +82,12 @@ template<typename T> class cache
/** Current cache size. Protected by m_mtx*/
size_t m_pos;
/**
Total time spent waiting on entries in cache, inside get()
Only valid if timed template parameter is true.
*/
std::chrono::duration<float> m_wait_time;
private:
inline size_t capacity()
@ -107,27 +111,33 @@ private:
return m_pos == capacity();
}
/**
Wait on condition. Instrumented (wait time is recorded),
if timed template parameter is true.
*/
void condition_wait(std::unique_lock<std::mutex>& lock)
{
if (timed)
{
auto start= std::chrono::high_resolution_clock::now();
m_cv.wait(lock);
m_wait_time+= std::chrono::high_resolution_clock::now() - start;
}
else
m_cv.wait(lock);
}
public:
/**
Constructor
@param size - maximum number of items in cache
*/
cache(size_t size) : m_base(size), m_cache(size),
m_waiters(), m_pos(0)
cache(size_t size) :m_mtx(), m_cv(), m_base(size), m_cache(size),
m_waiters(), m_pos(0), m_wait_time()
{
mysql_mutex_init(tpool_cache_mutex_key, &m_mtx, nullptr);
pthread_cond_init(&m_cv, nullptr);
for(size_t i= 0 ; i < size; i++)
m_cache[i]= &m_base[i];
}
~cache()
{
mysql_mutex_destroy(&m_mtx);
pthread_cond_destroy(&m_cv);
}
/**
Retrieve an item from cache. Waits for free item, if cache is
currently empty.
@ -135,17 +145,16 @@ public:
*/
T* get()
{
mysql_mutex_lock(&m_mtx);
std::unique_lock<std::mutex> lock(m_mtx);
while (is_empty())
my_cond_wait(&m_cv, &m_mtx.m_mutex);
condition_wait(lock);
assert(m_pos < capacity());
// return last element
T *t= m_cache[m_pos++];
mysql_mutex_unlock(&m_mtx);
return t;
}
mysql_mutex_t &mutex() { return m_mtx; }
std::mutex &mutex() { return m_mtx; }
/**
Put back an element to cache.
@ -153,15 +162,14 @@ public:
*/
void put(T *ele)
{
mysql_mutex_lock(&m_mtx);
std::unique_lock<std::mutex> lock(m_mtx);
assert(!is_full());
const bool was_empty= is_empty();
// put element to the logical end of the array
m_cache[--m_pos] = ele;
if (was_empty || (is_full() && m_waiters))
pthread_cond_broadcast(&m_cv);
mysql_mutex_unlock(&m_mtx);
m_cv.notify_all();
}
/** Check if pointer represents cached element */
@ -172,22 +180,20 @@ public:
}
/** Wait until cache is full
@param m cache mutex (locked) */
void wait(mysql_mutex_t &m)
@param lock */
void wait(std::unique_lock<std::mutex> &lk)
{
mysql_mutex_assert_owner(&m);
m_waiters++;
while (!is_full())
my_cond_wait(&m_cv, &m.m_mutex);
m_cv.wait(lk);
m_waiters--;
}
/* Wait until cache is full.*/
void wait()
{
mysql_mutex_lock(&m_mtx);
wait(m_mtx);
mysql_mutex_unlock(&m_mtx);
std::unique_lock<std::mutex> lock(m_mtx);
wait(lock);
}
/**
@ -199,9 +205,13 @@ public:
return m_pos;
}
TPOOL_SUPPRESS_TSAN std::chrono::duration<float> wait_time()
{
return m_wait_time;
}
void resize(size_t count)
{
mysql_mutex_assert_owner(&m_mtx);
assert(is_full());
m_base.resize(count);
m_cache.resize(count);