Merge 10.5 into 10.6

This commit is contained in:
Marko Mäkelä 2021-03-29 16:16:12 +03:00
commit 2ad61c6782
10 changed files with 180 additions and 167 deletions

View File

@ -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.

View File

@ -0,0 +1 @@
--innodb-force-recovery=2

View File

@ -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.

View File

@ -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);
}

View File

@ -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)
{

View File

@ -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;
}

View File

@ -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));
}
/** 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)
{
mysql_mutex_assert_owner(&log_requests.mutex);
ut_ad(pending == log_requests.start);
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;
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);
/*****************************************************************//**
Handle a commit checkpoint request from server layer.
if (!last)
return false;
/* 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);
/* 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);
return true;
}
/** 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);
}
}
/** 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)
static void innodb_log_flush_request(void *cookie)
{
ib_uint64_t lsn;
ib_uint64_t flush_lsn;
struct pending_checkpoint * entry;
const lsn_t lsn= log_sys.get_lsn();
lsn_t flush_lsn= log_sys.get_flushed_lsn();
/* 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;
}
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;
entry->next = NULL;
entry->hton = hton;
entry->cookie = cookie;
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);
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 */
{
struct pending_checkpoint * pending;
struct pending_checkpoint * entry;
struct pending_checkpoint * last_ready;
/* 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;
}
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;
}
mysql_mutex_unlock(&pending_checkpoint_mutex);
if (!last_ready)
return;
/* Now that we have released the lock, notify upper layer about all
commit checkpoints that have now completed. */
for (;;) {
entry = pending;
pending = pending->next;
commit_checkpoint_notify_ha(entry->hton, entry->cookie);
my_free(entry);
if (entry == last_ready)
break;
}
/* 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);
}
/*****************************************************************//**

View File

@ -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'.

View File

@ -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.

View File

@ -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);
}
}