summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorMarko Mäkelä <marko.makela@mariadb.com>2021-03-29 15:16:23 +0300
committerMarko Mäkelä <marko.makela@mariadb.com>2021-03-29 15:16:23 +0300
commite8b7fceb82292e881d2a8fb1ba402f47f6f1c973 (patch)
treeb470948935aef5c0c6c685b536decf8c79e951cc
parent8e2d69f7b8425c9cd9546cb45c16c492d5aa5b0a (diff)
downloadmariadb-git-e8b7fceb82292e881d2a8fb1ba402f47f6f1c973.tar.gz
MDEV-24302: RESET MASTER hangs
Starting with MariaDB 10.5, roughly after MDEV-23855 was fixed, we are observing sporadic hangs during the execution of the RESET MASTER statement. We are hoping to fix the hangs with these changes, but due to the rather infrequent occurrence of the hangs and our inability to reliably reproduce the hangs, we cannot be sure of this. What we do know is that innodb_force_recovery=2 (or a larger setting) will prevent srv_master_callback (the former srv_master_thread) from running. In that mode, periodic log flushes would never occur and RESET MASTER could hang indefinitely. That is demonstrated by the new test case that was developed by Andrei Elkin. We fix this case by implementing a special case for it. This also includes some code cleanup and renames of misleadingly named code. The interface has nothing to do with log checkpoints in the storage engine; it is only about requesting log writes to be persistent. handlerton::commit_checkpoint_request, commit_checkpoint_notify_ha(): Remove the unused parameter hton. log_requests.start: Replaces pending_checkpoint_list. log_requests.end: Replaces pending_checkpoint_list_end. log_requests.mutex: Replaces pending_checkpoint_mutex. log_flush_notify_and_unlock(), log_flush_notify(): Replaces innobase_mysql_log_notify(). The new implementation should be functionally equivalent to the old one. innodb_log_flush_request(): Replaces innobase_checkpoint_request(). Implement a fast path for common cases, and reduce the mutex hold time. POSSIBLE FIX OF THE HANG: We will invoke commit_checkpoint_notify_ha() for the current request if it is already satisfied, as well as invoke log_flush_notify_and_unlock() for any satisfied requests. log_write(): Invoke log_flush_notify() when the write is already durable. This was missing WITH_PMEM when the log is in persistent memory. Reviewed by: Vladislav Vaintroub
-rw-r--r--mysql-test/suite/innodb/r/group_commit_force_recovery.result5
-rw-r--r--mysql-test/suite/innodb/t/group_commit_force_recovery-master.opt1
-rw-r--r--mysql-test/suite/innodb/t/group_commit_force_recovery.test21
-rw-r--r--sql/handler.cc7
-rw-r--r--sql/handler.h4
-rw-r--r--storage/innobase/handler/ha_innodb.cc278
-rw-r--r--storage/innobase/include/ha_prototypes.h10
-rw-r--r--storage/innobase/log/log0log.cc10
-rw-r--r--storage/rocksdb/ha_rocksdb.cc5
9 files changed, 176 insertions, 165 deletions
diff --git a/mysql-test/suite/innodb/r/group_commit_force_recovery.result b/mysql-test/suite/innodb/r/group_commit_force_recovery.result
new file mode 100644
index 00000000000..f177d96c744
--- /dev/null
+++ b/mysql-test/suite/innodb/r/group_commit_force_recovery.result
@@ -0,0 +1,5 @@
+CREATE TABLE t1(a int) ENGINE=InnoDB;
+INSERT INTO t1 SET a=1;
+RESET MASTER;
+DROP TABLE t1;
+End of the tests.
diff --git a/mysql-test/suite/innodb/t/group_commit_force_recovery-master.opt b/mysql-test/suite/innodb/t/group_commit_force_recovery-master.opt
new file mode 100644
index 00000000000..65470e63e44
--- /dev/null
+++ b/mysql-test/suite/innodb/t/group_commit_force_recovery-master.opt
@@ -0,0 +1 @@
+--innodb-force-recovery=2
diff --git a/mysql-test/suite/innodb/t/group_commit_force_recovery.test b/mysql-test/suite/innodb/t/group_commit_force_recovery.test
new file mode 100644
index 00000000000..0e4767476ef
--- /dev/null
+++ b/mysql-test/suite/innodb/t/group_commit_force_recovery.test
@@ -0,0 +1,21 @@
+# MDEV-24302 RESET MASTER hangs as Innodb does not report on binlog checkpoint
+# Testing binlog checkpoint notification works under stringent condition
+# set by innodb_force_recovery = 2.
+
+--source include/have_innodb.inc
+--source include/have_binlog_format_mixed.inc
+
+# Binlog checkpoint notification consumers such as RESET MASTER
+# receive one when lsn_0 at the time of the request is finally gets flushed
+# flush_lsn >= lsn_0
+# The bug situation was that when lsn_0 reflects a write of an internal innodb trx
+# and RESET MASTER was not followed by any more user transaction
+# it would hang.
+
+CREATE TABLE t1(a int) ENGINE=InnoDB;
+INSERT INTO t1 SET a=1;
+RESET MASTER;
+
+# final cleanup
+DROP TABLE t1;
+--echo End of the tests.
diff --git a/sql/handler.cc b/sql/handler.cc
index 751a9cd0a97..7e3b46cab04 100644
--- a/sql/handler.cc
+++ b/sql/handler.cc
@@ -1,5 +1,5 @@
/* Copyright (c) 2000, 2016, Oracle and/or its affiliates.
- Copyright (c) 2009, 2020, MariaDB Corporation.
+ Copyright (c) 2009, 2021, 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
@@ -861,7 +861,7 @@ static my_bool commit_checkpoint_request_handlerton(THD *unused1, plugin_ref plu
void *cookie= st->cookie;
if (st->pre_hook)
(*st->pre_hook)(cookie);
- (*hton->commit_checkpoint_request)(hton, cookie);
+ (*hton->commit_checkpoint_request)(cookie);
}
return FALSE;
}
@@ -2437,8 +2437,7 @@ int ha_recover(HASH *commit_list)
Called by engine to notify TC that a new commit checkpoint has been reached.
See comments on handlerton method commit_checkpoint_request() for details.
*/
-void
-commit_checkpoint_notify_ha(handlerton *hton, void *cookie)
+void commit_checkpoint_notify_ha(void *cookie)
{
tc_log->commit_checkpoint_notify(cookie);
}
diff --git a/sql/handler.h b/sql/handler.h
index 4c013733833..f976ac50ab8 100644
--- a/sql/handler.h
+++ b/sql/handler.h
@@ -1471,7 +1471,7 @@ struct handlerton
recovery. It uses that to reduce the work needed for any subsequent XA
recovery process.
*/
- void (*commit_checkpoint_request)(handlerton *hton, void *cookie);
+ void (*commit_checkpoint_request)(void *cookie);
/*
"Disable or enable checkpointing internal to the storage engine. This is
used for FLUSH TABLES WITH READ LOCK AND DISABLE CHECKPOINT to ensure that
@@ -5211,7 +5211,7 @@ void trans_register_ha(THD *thd, bool all, handlerton *ht,
const char *get_canonical_filename(handler *file, const char *path,
char *tmp_path);
-void commit_checkpoint_notify_ha(handlerton *hton, void *cookie);
+void commit_checkpoint_notify_ha(void *cookie);
inline const LEX_CSTRING *table_case_name(HA_CREATE_INFO *info, const LEX_CSTRING *name)
{
diff --git a/storage/innobase/handler/ha_innodb.cc b/storage/innobase/handler/ha_innodb.cc
index 700ac73b9cf..683dc92edd6 100644
--- a/storage/innobase/handler/ha_innodb.cc
+++ b/storage/innobase/handler/ha_innodb.cc
@@ -158,9 +158,6 @@ void close_thread_tables(THD* thd);
#include "wsrep_sst.h"
#endif /* WITH_WSREP */
-/** to force correct commit order in binlog */
-static mysql_mutex_t pending_checkpoint_mutex;
-
#define INSIDE_HA_INNOBASE_CC
#define EQ_CURRENT_THD(thd) ((thd) == current_thd)
@@ -513,15 +510,12 @@ const struct _ft_vft_ext ft_vft_ext_result = {innobase_fts_get_version,
performance schema */
static mysql_pfs_key_t pending_checkpoint_mutex_key;
-static PSI_mutex_info all_pthread_mutexes[] = {
- PSI_KEY(pending_checkpoint_mutex),
-};
-
# ifdef UNIV_PFS_MUTEX
/* all_innodb_mutexes array contains mutexes that are
performance schema instrumented if "UNIV_PFS_MUTEX"
is defined */
static PSI_mutex_info all_innodb_mutexes[] = {
+ PSI_KEY(pending_checkpoint_mutex),
PSI_KEY(buf_pool_mutex),
PSI_KEY(dict_foreign_err_mutex),
PSI_KEY(dict_sys_mutex),
@@ -1169,7 +1163,32 @@ innobase_release_savepoint(
savepoint should be released */
void* savepoint); /*!< in: savepoint data */
-static void innobase_checkpoint_request(handlerton *hton, void *cookie);
+/** Request notification of log writes */
+static void innodb_log_flush_request(void *cookie);
+
+/** Requests for log flushes */
+struct log_flush_request
+{
+ /** earlier request (for a smaller LSN) */
+ log_flush_request *next;
+ /** parameter provided to innodb_log_flush_request() */
+ void *cookie;
+ /** log sequence number that is being waited for */
+ lsn_t lsn;
+};
+
+/** Buffer of pending innodb_log_flush_request() */
+MY_ALIGNED(CPU_LEVEL1_DCACHE_LINESIZE) static
+struct
+{
+ /** first request */
+ Atomic_relaxed<log_flush_request*> start;
+ /** last request */
+ log_flush_request *end;
+ /** mutex protecting this object */
+ mysql_mutex_t mutex;
+}
+log_requests;
/** @brief Adjust some InnoDB startup parameters based on file contents
or innodb_page_size. */
@@ -3902,7 +3921,7 @@ static int innodb_init(void* p)
innobase_hton->recover = innobase_xa_recover;
innobase_hton->commit_by_xid = innobase_commit_by_xid;
innobase_hton->rollback_by_xid = innobase_rollback_by_xid;
- innobase_hton->commit_checkpoint_request=innobase_checkpoint_request;
+ innobase_hton->commit_checkpoint_request = innodb_log_flush_request;
innobase_hton->create = innobase_create_handler;
innobase_hton->drop_database = innobase_drop_database;
@@ -3971,9 +3990,6 @@ static int innodb_init(void* p)
/* Register keys with MySQL performance schema */
int count;
- count = array_elements(all_pthread_mutexes);
- mysql_mutex_register("innodb", all_pthread_mutexes, count);
-
# ifdef UNIV_PFS_MUTEX
count = array_elements(all_innodb_mutexes);
mysql_mutex_register("innodb", all_innodb_mutexes, count);
@@ -4020,7 +4036,7 @@ static int innodb_init(void* p)
ibuf_max_size_update(srv_change_buffer_max_size);
mysql_mutex_init(pending_checkpoint_mutex_key,
- &pending_checkpoint_mutex,
+ &log_requests.mutex,
MY_MUTEX_INIT_FAST);
#ifdef MYSQL_DYNAMIC_PLUGIN
if (innobase_hton != p) {
@@ -4083,7 +4099,7 @@ innobase_end(handlerton*, ha_panic_function)
innodb_shutdown();
- mysql_mutex_destroy(&pending_checkpoint_mutex);
+ mysql_mutex_destroy(&log_requests.mutex);
}
DBUG_RETURN(0);
@@ -4442,144 +4458,118 @@ innobase_rollback_trx(
0, trx->mysql_thd));
}
-
-struct pending_checkpoint {
- struct pending_checkpoint *next;
- handlerton *hton;
- void *cookie;
- ib_uint64_t lsn;
-};
-static struct pending_checkpoint *pending_checkpoint_list;
-static struct pending_checkpoint *pending_checkpoint_list_end;
-
-/*****************************************************************//**
-Handle a commit checkpoint request from server layer.
-We put the request in a queue, so that we can notify upper layer about
-checkpoint complete when we have flushed the redo log.
-If we have already flushed all relevant redo log, we notify immediately.*/
-static
-void
-innobase_checkpoint_request(
- handlerton *hton,
- void *cookie)
-{
- ib_uint64_t lsn;
- ib_uint64_t flush_lsn;
- struct pending_checkpoint * entry;
-
- /* Do the allocation outside of lock to reduce contention. The normal
- case is that not everything is flushed, so we will need to enqueue. */
- entry = static_cast<struct pending_checkpoint *>
- (my_malloc(PSI_INSTRUMENT_ME, sizeof(*entry), MYF(MY_WME)));
- if (!entry) {
- sql_print_error("Failed to allocate %u bytes."
- " Commit checkpoint will be skipped.",
- static_cast<unsigned>(sizeof(*entry)));
- return;
- }
-
- entry->next = NULL;
- entry->hton = hton;
- entry->cookie = cookie;
-
- mysql_mutex_lock(&pending_checkpoint_mutex);
- lsn = log_get_lsn();
- flush_lsn = log_get_flush_lsn();
- if (lsn > flush_lsn) {
- /* Put the request in queue.
- When the log gets flushed past the lsn, we will remove the
- entry from the queue and notify the upper layer. */
- entry->lsn = lsn;
- if (pending_checkpoint_list_end) {
- pending_checkpoint_list_end->next = entry;
- /* There is no need to order the entries in the list
- by lsn. The upper layer can accept notifications in
- any order, and short delays in notifications do not
- significantly impact performance. */
- } else {
- pending_checkpoint_list = entry;
- }
- pending_checkpoint_list_end = entry;
- entry = NULL;
- }
- mysql_mutex_unlock(&pending_checkpoint_mutex);
-
- if (entry) {
- /* We are already flushed. Notify the checkpoint immediately. */
- commit_checkpoint_notify_ha(entry->hton, entry->cookie);
- my_free(entry);
- }
-}
-
-/*****************************************************************//**
-Log code calls this whenever log has been written and/or flushed up
-to a new position. We use this to notify upper layer of a new commit
-checkpoint when necessary.*/
-UNIV_INTERN
-void
-innobase_mysql_log_notify(
-/*======================*/
- ib_uint64_t flush_lsn) /*!< in: LSN flushed to disk */
+/** Invoke commit_checkpoint_notify_ha() on completed log flush requests.
+@param pending log_requests.start
+@param lsn log_sys.get_flushed_lsn()
+@return whether something was notified (and log_requests.mutex released) */
+static bool log_flush_notify_and_unlock(log_flush_request *pending, lsn_t lsn)
{
- struct pending_checkpoint * pending;
- struct pending_checkpoint * entry;
- struct pending_checkpoint * last_ready;
+ mysql_mutex_assert_owner(&log_requests.mutex);
+ ut_ad(pending == log_requests.start);
- /* It is safe to do a quick check for NULL first without lock.
- Even if we should race, we will at most skip one checkpoint and
- take the next one, which is harmless. */
- if (!pending_checkpoint_list)
- return;
-
- mysql_mutex_lock(&pending_checkpoint_mutex);
- pending = pending_checkpoint_list;
- if (!pending)
- {
- mysql_mutex_unlock(&pending_checkpoint_mutex);
- return;
- }
-
- last_ready = NULL;
- for (entry = pending; entry != NULL; entry = entry -> next)
- {
- /* Notify checkpoints up until the first entry that has not
- been fully flushed to the redo log. Since we do not maintain
- the list ordered, in principle there could be more entries
- later than were also flushed. But there is no harm in
- delaying notifications for those a bit. And in practise, the
- list is unlikely to have more than one element anyway, as we
- flush the redo log at least once every second. */
- if (entry->lsn > flush_lsn)
- break;
- last_ready = entry;
- }
+ log_flush_request *entry= pending, *last= nullptr;
+ /* Process the first requests that have been completed. Since
+ the list is not necessarily in ascending order of LSN, we may
+ miss to notify some requests that have already been completed.
+ But there is no harm in delaying notifications for those a bit.
+ And in practise, the list is unlikely to have more than one
+ element anyway, because the redo log would be flushed every
+ srv_flush_log_at_timeout seconds (1 by default). */
+ for (; entry && entry->lsn <= lsn; last= entry, entry= entry->next);
- if (last_ready)
- {
- /* We found some pending checkpoints that are now flushed to
- disk. So remove them from the list. */
- pending_checkpoint_list = entry;
- if (!entry)
- pending_checkpoint_list_end = NULL;
- }
+ if (!last)
+ return false;
- mysql_mutex_unlock(&pending_checkpoint_mutex);
+ /* Detach the head of the list that corresponds to persisted log writes. */
+ log_requests.start= entry;
+ if (!entry)
+ log_requests.end= nullptr;
+ mysql_mutex_unlock(&log_requests.mutex);
- if (!last_ready)
- return;
+ /* Now that we have released the mutex, notify the submitters
+ and free the head of the list. */
+ do
+ {
+ entry= pending;
+ pending= pending->next;
+ commit_checkpoint_notify_ha(entry->cookie);
+ my_free(entry);
+ }
+ while (entry != last);
- /* Now that we have released the lock, notify upper layer about all
- commit checkpoints that have now completed. */
- for (;;) {
- entry = pending;
- pending = pending->next;
+ return true;
+}
- commit_checkpoint_notify_ha(entry->hton, entry->cookie);
+/** Invoke commit_checkpoint_notify_ha() to notify that outstanding
+log writes have been completed. */
+void log_flush_notify(lsn_t flush_lsn)
+{
+ if (log_requests.start)
+ {
+ mysql_mutex_lock(&log_requests.mutex);
+ if (log_flush_request *pending= log_requests.start)
+ if (log_flush_notify_and_unlock(pending, flush_lsn))
+ return;
+ mysql_mutex_unlock(&log_requests.mutex);
+ }
+}
- my_free(entry);
- if (entry == last_ready)
- break;
- }
+/** Handle a commit checkpoint request from server layer.
+We put the request in a queue, so that we can notify upper layer about
+checkpoint complete when we have flushed the redo log.
+If we have already flushed all relevant redo log, we notify immediately.*/
+static void innodb_log_flush_request(void *cookie)
+{
+ const lsn_t lsn= log_sys.get_lsn();
+ lsn_t flush_lsn= log_sys.get_flushed_lsn();
+
+ if (flush_lsn >= lsn)
+ /* All log is already persistent. */;
+ else if (UNIV_UNLIKELY(srv_force_recovery >= SRV_FORCE_NO_BACKGROUND))
+ /* Normally, srv_master_callback() should periodically invoke
+ srv_sync_log_buffer_in_background(), which should initiate a log
+ flush about once every srv_flush_log_at_timeout seconds. But,
+ starting with the innodb_force_recovery=2 level, that background
+ task will not run. */
+ log_write_up_to(flush_lsn= lsn, true);
+ else if (log_flush_request *req= static_cast<log_flush_request*>
+ (my_malloc(PSI_INSTRUMENT_ME, sizeof *req, MYF(MY_WME))))
+ {
+ req->next= nullptr;
+ req->cookie= cookie;
+ req->lsn= lsn;
+
+ mysql_mutex_lock(&log_requests.mutex);
+ auto old_end= log_requests.end;
+ log_requests.end= req;
+ if (old_end)
+ {
+ /* Append the entry to the list. Because we determined req->lsn before
+ acquiring the mutex, this list may not be ordered by req->lsn,
+ even though log_flush_notify_and_unlock() assumes so. */
+ old_end->next= req;
+ /* This hopefully addresses the hang that was reported in MDEV-24302.
+ Upon receiving a new request, we will notify old requests of
+ completion. */
+ if (log_flush_notify_and_unlock(log_requests.start, flush_lsn))
+ return;
+ }
+ else
+ log_requests.start= req;
+ mysql_mutex_unlock(&log_requests.mutex);
+ return;
+ }
+ else
+ sql_print_error("Failed to allocate %zu bytes."
+ " Commit checkpoint will be skipped.", sizeof *req);
+
+ /* This hopefully addresses the hang that was reported in MDEV-24302.
+ Upon receiving a new request to notify of log writes becoming
+ persistent, we will notify old requests of completion. Note:
+ log_flush_notify() may skip some notifications because it is
+ basically assuming that the list is in ascending order of LSN. */
+ log_flush_notify(flush_lsn);
+ commit_checkpoint_notify_ha(cookie);
}
/*****************************************************************//**
diff --git a/storage/innobase/include/ha_prototypes.h b/storage/innobase/include/ha_prototypes.h
index 559de415b01..453f9e028e9 100644
--- a/storage/innobase/include/ha_prototypes.h
+++ b/storage/innobase/include/ha_prototypes.h
@@ -145,16 +145,6 @@ innobase_mysql_print_thd(
uint max_query_len); /*!< in: max query length to print, or 0 to
use the default max length */
-/*****************************************************************//**
-Log code calls this whenever log has been written and/or flushed up
-to a new position. We use this to notify upper layer of a new commit
-checkpoint when necessary.*/
-UNIV_INTERN
-void
-innobase_mysql_log_notify(
-/*======================*/
- ib_uint64_t flush_lsn); /*!< in: LSN flushed to disk */
-
/** Converts a MySQL type to an InnoDB type. Note that this function returns
the 'mtype' of InnoDB. InnoDB differentiates between MySQL's old <= 4.1
VARCHAR and the new true VARCHAR in >= 5.0.3 by the 'prtype'.
diff --git a/storage/innobase/log/log0log.cc b/storage/innobase/log/log0log.cc
index 6dfeec0f925..71b0c37877e 100644
--- a/storage/innobase/log/log0log.cc
+++ b/storage/innobase/log/log0log.cc
@@ -657,6 +657,10 @@ log_buffer_switch()
log_sys.buf_next_to_write = log_sys.buf_free;
}
+/** Invoke commit_checkpoint_notify_ha() to notify that outstanding
+log writes have been completed. */
+void log_flush_notify(lsn_t flush_lsn);
+
/**
Writes log buffer to disk
which is the "write" part of log_write_up_to().
@@ -759,8 +763,10 @@ static void log_write(bool rotate_key)
start_offset - area_start);
srv_stats.log_padded.add(pad_size);
log_sys.write_lsn = write_lsn;
- if (log_sys.log.writes_are_durable())
+ if (log_sys.log.writes_are_durable()) {
log_sys.set_flushed_lsn(write_lsn);
+ log_flush_notify(write_lsn);
+ }
return;
}
@@ -823,7 +829,7 @@ void log_write_up_to(lsn_t lsn, bool flush_to_disk, bool rotate_key)
log_write_flush_to_disk_low(flush_lsn);
flush_lock.release(flush_lsn);
- innobase_mysql_log_notify(flush_lsn);
+ log_flush_notify(flush_lsn);
}
/** write to the log file up to the last log entry.
diff --git a/storage/rocksdb/ha_rocksdb.cc b/storage/rocksdb/ha_rocksdb.cc
index e7a3c6b3494..0e63ea50809 100644
--- a/storage/rocksdb/ha_rocksdb.cc
+++ b/storage/rocksdb/ha_rocksdb.cc
@@ -4111,15 +4111,14 @@ static int rocksdb_recover(handlerton* hton, XID* xid_list, uint len)
MariaRocks just flushes everything right away ATM
*/
-static void rocksdb_checkpoint_request(handlerton *hton,
- void *cookie)
+static void rocksdb_checkpoint_request(void *cookie)
{
const rocksdb::Status s= rdb->SyncWAL();
//TODO: what to do on error?
if (s.ok())
{
rocksdb_wal_group_syncs++;
- commit_checkpoint_notify_ha(hton, cookie);
+ commit_checkpoint_notify_ha(cookie);
}
}