summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorMarko Mäkelä <marko.makela@mariadb.com>2021-03-29 16:16:12 +0300
committerMarko Mäkelä <marko.makela@mariadb.com>2021-03-29 16:16:12 +0300
commit2ad61c678243dec2a808b1f67b5760a725163052 (patch)
tree91e5af40cea2f5b2bd56406224963ea6fbe9a2a8
parent0f6f72965bc5743961725ec3bf7b541130427bc7 (diff)
parente8b7fceb82292e881d2a8fb1ba402f47f6f1c973 (diff)
downloadmariadb-git-2ad61c678243dec2a808b1f67b5760a725163052.tar.gz
Merge 10.5 into 10.6
-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--sql/sql_prepare.cc8
-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
10 files changed, 181 insertions, 168 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 8792c02194d..b312635c8ee 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 26c183be4c7..a2dd747a4d7 100644
--- a/sql/handler.h
+++ b/sql/handler.h
@@ -1477,7 +1477,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
@@ -5255,7 +5255,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/sql/sql_prepare.cc b/sql/sql_prepare.cc
index bba182c1b9c..24e0d8abf27 100644
--- a/sql/sql_prepare.cc
+++ b/sql/sql_prepare.cc
@@ -3418,15 +3418,17 @@ static void mysql_stmt_execute_common(THD *thd,
if (!(stmt= find_prepared_statement(thd, stmt_id)))
{
char llbuf[22];
+ size_t length;
/*
Did not find the statement with the provided stmt_id.
Set thd->query_string with the stmt_id so the
audit plugin gets the meaningful notification.
*/
- if (alloc_query(thd, llbuf, sizeof(llbuf)))
+ length= (size_t) (longlong10_to_str(stmt_id, llbuf, 10) - llbuf);
+ if (alloc_query(thd, llbuf, length + 1))
thd->set_query(0, 0);
- my_error(ER_UNKNOWN_STMT_HANDLER, MYF(0), static_cast<int>(sizeof(llbuf)),
- llstr(stmt_id, llbuf), "mysqld_stmt_execute");
+ my_error(ER_UNKNOWN_STMT_HANDLER, MYF(0), (int) length, llbuf,
+ "mysqld_stmt_execute");
DBUG_VOID_RETURN;
}
diff --git a/storage/innobase/handler/ha_innodb.cc b/storage/innobase/handler/ha_innodb.cc
index 26cbabf4e8e..4eeecd374c4 100644
--- a/storage/innobase/handler/ha_innodb.cc
+++ b/storage/innobase/handler/ha_innodb.cc
@@ -156,9 +156,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)
@@ -511,10 +508,6 @@ 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
mysql_pfs_key_t buf_pool_mutex_key;
mysql_pfs_key_t dict_foreign_err_mutex_key;
@@ -559,6 +552,7 @@ mysql_pfs_key_t read_view_mutex_key;
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),
@@ -1213,7 +1207,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. */
@@ -3660,7 +3679,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;
@@ -3733,9 +3752,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);
@@ -3782,7 +3798,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) {
@@ -3847,7 +3863,7 @@ innobase_end(handlerton*, ha_panic_function)
#ifdef WITH_INNODB_DISALLOW_WRITES
pthread_cond_destroy(&allow_writes_cond);
#endif /* WITH_INNODB_DISALLOW_WRITES */
- mysql_mutex_destroy(&pending_checkpoint_mutex);
+ mysql_mutex_destroy(&log_requests.mutex);
}
DBUG_RETURN(0);
@@ -4206,144 +4222,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 d3ff3a23f38..f3004c40b1a 100644
--- a/storage/innobase/include/ha_prototypes.h
+++ b/storage/innobase/include/ha_prototypes.h
@@ -136,16 +136,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 f5da279f676..bfb7b4dfb7f 100644
--- a/storage/innobase/log/log0log.cc
+++ b/storage/innobase/log/log0log.cc
@@ -654,6 +654,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().
@@ -756,8 +760,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;
}
@@ -824,7 +830,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 88b6b4bdec4..289a1ac07b9 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);
}
}