Commit 2f9e2647 authored by Marko Mäkelä's avatar Marko Mäkelä

MDEV-29911 InnoDB recovery and mariadb-backup --prepare fail to report detailed progress

The progress reporting of InnoDB crash recovery was rather intermittent.
Nothing was reported during the single-threaded log record parsing, which
could consume minutes when parsing a large log. During log application,
there only was progress reporting in background threads that would be
invoked on data page read completion.

The progress reporting here will be detailed like this:

InnoDB: Starting crash recovery from checkpoint LSN=503549688
InnoDB: Parsed redo log up to LSN=1990840177; to recover: 124806 pages
InnoDB: Parsed redo log up to LSN=2729777071; to recover: 186123 pages
InnoDB: Parsed redo log up to LSN=3488599173; to recover: 248397 pages
InnoDB: Parsed redo log up to LSN=4177856618; to recover: 306469 pages
InnoDB: Multi-batch recovery needed at LSN 4189599815
InnoDB: End of log at LSN=4483551634
InnoDB: To recover: LSN 4189599815/4483551634; 307490 pages
InnoDB: To recover: LSN 4189599815/4483551634; 197159 pages
InnoDB: To recover: LSN 4189599815/4483551634; 67623 pages
InnoDB: Parsed redo log up to LSN=4353924218; to recover: 102083 pages
...
InnoDB: log sequence number 4483551634 ...

The previous messages "Starting a batch to recover" or
"Starting a final batch to recover" will be replaced by
"To recover: ... pages" messages.

If a batch lasts longer than 15 seconds, then there will be
progress reports every 15 seconds, showing the number of remaining pages.
For the non-final batch, the "To recover:" message includes two end LSN:
that of the batch, and of the recovered log. This is the primary measure
of progress. The batch will end once the number of pages to recover
reaches 0.

If recovery is possible in a single batch, the output will look like this,
with a shorter "To recover:" message that counts only the remaining pages:

InnoDB: Starting crash recovery from checkpoint LSN=503549688
InnoDB: Parsed redo log up to LSN=1998701027; to recover: 125560 pages
InnoDB: Parsed redo log up to LSN=2734136874; to recover: 186446 pages
InnoDB: Parsed redo log up to LSN=3499505504; to recover: 249378 pages
InnoDB: Parsed redo log up to LSN=4183247844; to recover: 306964 pages
InnoDB: End of log at LSN=4483551634
...
InnoDB: To recover: 331797 pages
...
InnoDB: log sequence number 4483551634 ...

We will also speed up recovery by improving the memory management and
implementing multi-threaded recovery of data pages that will not need
to be read into the buffer pool ("fake read"). Log application in the
"fake read" threads will be protected by an atomic being_recovered field
and exclusive buf_page_t::latch.

Recovery will reserve for data pages two thirds of the buffer pool,
or 256 pages, whichever is smaller. Previously, we could only use at most
one third of the buffer pool for buffered log records. This would typically
mean that with large buffer pools, recovery unnecessary consisted of
multiple batches.

If recovery runs out of memory, it will "roll back" or "rewind" the current
mini-transaction. The recv_sys.lsn and recv_sys.pages will correspond
to the "out of memory LSN", at the end of the previous complete
mini-transaction.

If recovery runs out of memory while executing the final recovery batch,
we can simply invoke recv_sys.apply(false) to make room, and resume
parsing.

If recovery runs out of memory before the final batch, we will scan
the redo log to the end (recv_sys.scanned_lsn) and check for any missing
or inconsistent files. If recv_init_crash_recovery_spaces() does not
report any potentially missing tablespaces, we can make use of the
already stored recv_sys.pages and only rewind to the "out of memory LSN".
Else, we must keep parsing and invoking recv_validate_tablespace()
until an error has been found or everything has been resolved, and
ultimatily rewind to to the checkpoint LSN.

recv_sys_t::pages_it: A cached iterator to recv_sys.pages

recv_sys_t::parse_mtr(): Remove an ATTRIBUTE_NOINLINE that would
prevent tail call optimization in recv_sys_t::parse_pmem().

recv_sys_t::parse(), recv_sys_t::parse_mtr(), recv_sys_t::parse_pmem():
Add template<bool store> parameter. Redo log record parsing
(store=false) is better specialized from store=true
(with bool if_exists) so that we can avoid some conditional branches
in frequently invoked low-level code.

recv_sys_t::is_memory_exhausted(): Remove. The special parse() status
GOT_OOM will report out-of-memory situation at the low level.

recv_sys_t::rewind(), page_recv_t::recs_t::rewind():
Remove all log starting with a specific LSN.

recv_scan_log(): Separate some code for only parsing, not storing log.
In rewound_lsn, remember the LSN at which last_phase=false recovery
ran out of memory. This is where the next call to recv_scan_log()
will resume storing the log. This replaces recv_sys.last_stored_lsn.

recv_sys_t::parse(): Evaluate the template parameter store in a few more
cases, to allow dead code to be eliminated at compile time.

recv_sys_t::scanned_lsn: The end of the log found by recv_scan_log().
The special value 1 means that recv_sys has been initialized but
no log has been parsed.

IORequest::write_complete(), IORequest::read_complete():
Replaces fil_aio_callback().

read_io_callback(), write_io_callback(): Replaces io_callback().

IORequest::fake_read_complete(), fake_io_callback(), os_fake_read():
Process a "fake read" request for concurrent recovery.

recv_sys_t::apply_batch(): Choose a number of successive pages
for a recovery batch.

recv_sys_t::erase(recv_sys_t::map::iterator): Remove log records for a
page whose recovery is not in progress. Log application threads
will not invoke this; they will only set being_recovered=-1 to indicate
that the entry is no longer needed.

recv_sys_t::garbage_collect(): Remove all being_recovered=-1 entries.

recv_sys_t::wait_for_pool(): Wait for some space to become available
in the buffer pool.

mlog_init_t::mark_ibuf_exist(): Avoid calls to
recv_sys::recover_low() via ibuf_page_exists() and buf_page_get_low().
Such calls would lead to double locking of recv_sys.mutex, which
depending on implementation could cause a deadlock. We will use
lower-level calls to look up index pages.

buf_LRU_block_remove_hashed(): Disable consistency checks for freed
ROW_FORMAT=COMPRESSED pages. Their contents could be uninitialized garbage.
This fixes an occasional failure of the test
innodb.innodb_bulk_create_index_debug.

Tested by: Matthias Leich
parent 2ec42f79
......@@ -3146,7 +3146,7 @@ static bool xtrabackup_copy_logfile()
if (log_sys.buf[recv_sys.offset] <= 1)
break;
if (recv_sys.parse_mtr(STORE_NO) == recv_sys_t::OK)
if (recv_sys.parse_mtr<false>(false) == recv_sys_t::OK)
{
do
{
......@@ -3156,7 +3156,7 @@ static bool xtrabackup_copy_logfile()
sequence_offset));
*seq= 1;
}
while ((r= recv_sys.parse_mtr(STORE_NO)) == recv_sys_t::OK);
while ((r= recv_sys.parse_mtr<false>(false)) == recv_sys_t::OK);
if (ds_write(dst_log_file, log_sys.buf + start_offset,
recv_sys.offset - start_offset))
......
......@@ -372,7 +372,7 @@ void buf_dblwr_t::recover()
const uint32_t space_id= page_get_space_id(page);
const page_id_t page_id(space_id, page_no);
if (recv_sys.lsn < lsn)
if (recv_sys.scanned_lsn < lsn)
{
ib::info() << "Ignoring a doublewrite copy of page " << page_id
<< " with future log sequence number " << lsn;
......
......@@ -2591,6 +2591,7 @@ ATTRIBUTE_COLD void buf_flush_page_cleaner_init()
/** Flush the buffer pool on shutdown. */
ATTRIBUTE_COLD void buf_flush_buffer_pool()
{
ut_ad(!os_aio_pending_reads());
ut_ad(!buf_page_cleaner_is_active);
ut_ad(!buf_flush_sync_lsn);
......
......@@ -1093,7 +1093,11 @@ static bool buf_LRU_block_remove_hashed(buf_page_t *bpage, const page_id_t id,
ut_a(!zip || !bpage->oldest_modification());
ut_ad(bpage->zip_size());
/* Skip consistency checks if the page was freed.
In recovery, we could get a sole FREE_PAGE record
and nothing else, for a ROW_FORMAT=COMPRESSED page.
Its contents would be garbage. */
if (!bpage->is_freed())
switch (fil_page_get_type(page)) {
case FIL_PAGE_TYPE_ALLOCATED:
case FIL_PAGE_INODE:
......@@ -1224,6 +1228,7 @@ void buf_pool_t::corrupted_evict(buf_page_t *bpage, uint32_t state)
buf_pool_t::hash_chain &chain= buf_pool.page_hash.cell_get(id.fold());
page_hash_latch &hash_lock= buf_pool.page_hash.lock_get(chain);
recv_sys.free_corrupted_page(id);
mysql_mutex_lock(&mutex);
hash_lock.lock();
......@@ -1248,8 +1253,6 @@ void buf_pool_t::corrupted_evict(buf_page_t *bpage, uint32_t state)
buf_LRU_block_free_hashed_page(reinterpret_cast<buf_block_t*>(bpage));
mysql_mutex_unlock(&mutex);
recv_sys.free_corrupted_page(id);
}
/** Update buf_pool.LRU_old_ratio.
......
......@@ -655,60 +655,35 @@ buf_read_ahead_linear(const page_id_t page_id, ulint zip_size, bool ibuf)
return count;
}
/** @return whether a page has been freed */
inline bool fil_space_t::is_freed(uint32_t page)
/** Schedule a page for recovery.
@param space tablespace
@param page_id page identifier
@param recs log records
@param init page initialization, or nullptr if the page needs to be read */
void buf_read_recover(fil_space_t *space, const page_id_t page_id,
page_recv_t &recs, recv_init *init)
{
std::lock_guard<std::mutex> freed_lock(freed_range_mutex);
return freed_ranges.contains(page);
}
/** Issues read requests for pages which recovery wants to read in.
@param space_id tablespace identifier
@param page_nos page numbers to read, in ascending order */
void buf_read_recv_pages(uint32_t space_id, st_::span<uint32_t> page_nos)
{
fil_space_t* space = fil_space_t::get(space_id);
if (!space) {
/* The tablespace is missing or unreadable: do nothing */
return;
}
const ulint zip_size = space->zip_size();
for (ulint i = 0; i < page_nos.size(); i++) {
/* Ignore if the page already present in freed ranges. */
if (space->is_freed(page_nos[i])) {
continue;
}
const page_id_t cur_page_id(space_id, page_nos[i]);
ulint limit = 0;
for (ulint j = 0; j < buf_pool.n_chunks; j++) {
limit += buf_pool.chunks[j].size / 2;
}
ut_ad(space->id == page_id.space());
space->reacquire();
const ulint zip_size= space->zip_size();
if (os_aio_pending_reads() >= limit) {
os_aio_wait_until_no_pending_reads(false);
}
space->reacquire();
switch (buf_read_page_low(space, false, BUF_READ_ANY_PAGE,
cur_page_id, zip_size, true)) {
case DB_SUCCESS: case DB_SUCCESS_LOCKED_REC:
break;
default:
sql_print_error("InnoDB: Recovery failed to read page "
UINT32PF " from %s",
cur_page_id.page_no(),
space->chain.start->name);
}
}
DBUG_PRINT("ib_buf", ("recovery read (%zu pages) for %s",
page_nos.size(), space->chain.start->name));
space->release();
if (init)
{
if (buf_page_t *bpage= buf_page_init_for_read(BUF_READ_ANY_PAGE, page_id,
zip_size, true))
{
ut_ad(bpage->in_file());
os_fake_read(IORequest{bpage, (buf_tmp_buffer_t*) &recs,
UT_LIST_GET_FIRST(space->chain),
IORequest::READ_ASYNC}, ptrdiff_t(init));
}
}
else if (dberr_t err= buf_read_page_low(space, false, BUF_READ_ANY_PAGE,
page_id, zip_size, true))
{
if (err != DB_SUCCESS_LOCKED_REC)
sql_print_error("InnoDB: Recovery failed to read page "
UINT32PF " from %s",
page_id.page_no(), space->chain.start->name);
}
}
......@@ -2775,53 +2775,55 @@ fil_io_t fil_space_t::io(const IORequest &type, os_offset_t offset, size_t len,
#include <tpool.h>
/** Callback for AIO completion */
void fil_aio_callback(const IORequest &request)
void IORequest::write_complete() const
{
ut_ad(fil_validate_skip());
ut_ad(request.node);
ut_ad(node);
ut_ad(is_write());
if (!request.bpage)
if (!bpage)
{
ut_ad(!srv_read_only_mode);
if (request.type == IORequest::DBLWR_BATCH)
buf_dblwr.flush_buffered_writes_completed(request);
if (type == IORequest::DBLWR_BATCH)
buf_dblwr.flush_buffered_writes_completed(*this);
else
ut_ad(request.type == IORequest::WRITE_ASYNC);
write_completed:
request.node->complete_write();
}
else if (request.is_write())
{
buf_page_write_complete(request);
goto write_completed;
ut_ad(type == IORequest::WRITE_ASYNC);
}
else
{
ut_ad(request.is_read());
buf_page_write_complete(*this);
/* IMPORTANT: since i/o handling for reads will read also the insert
buffer in fil_system.sys_space, we have to be very careful not to
introduce deadlocks. We never close fil_system.sys_space data
files and never issue asynchronous reads of change buffer pages. */
const page_id_t id(request.bpage->id());
node->complete_write();
node->space->release();
}
if (dberr_t err= request.bpage->read_complete(*request.node))
{
if (recv_recovery_is_on() && !srv_force_recovery)
{
mysql_mutex_lock(&recv_sys.mutex);
recv_sys.set_corrupt_fs();
mysql_mutex_unlock(&recv_sys.mutex);
}
void IORequest::read_complete() const
{
ut_ad(fil_validate_skip());
ut_ad(node);
ut_ad(is_read());
ut_ad(bpage);
/* IMPORTANT: since i/o handling for reads will read also the insert
buffer in fil_system.sys_space, we have to be very careful not to
introduce deadlocks. We never close fil_system.sys_space data files
and never issue asynchronous reads of change buffer pages. */
const page_id_t id(bpage->id());
if (err != DB_FAIL)
ib::error() << "Failed to read page " << id.page_no()
<< " from file '" << request.node->name << "': " << err;
if (dberr_t err= bpage->read_complete(*node))
{
if (recv_recovery_is_on() && !srv_force_recovery)
{
mysql_mutex_lock(&recv_sys.mutex);
recv_sys.set_corrupt_fs();
mysql_mutex_unlock(&recv_sys.mutex);
}
if (err != DB_FAIL)
ib::error() << "Failed to read page " << id.page_no()
<< " from file '" << node->name << "': " << err;
}
request.node->space->release();
node->space->release();
}
/** Flush to disk the writes in file spaces of the given type
......
......@@ -75,8 +75,7 @@ struct buf_pool_info_t
ulint flush_list_len; /*!< Length of buf_pool.flush_list */
ulint n_pend_unzip; /*!< buf_pool.n_pend_unzip, pages
pending decompress */
ulint n_pend_reads; /*!< buf_pool.n_pend_reads, pages
pending read */
ulint n_pend_reads; /*!< os_aio_pending_reads() */
ulint n_pending_flush_lru; /*!< Pages pending flush in LRU */
ulint n_pending_flush_list; /*!< Pages pending flush in FLUSH
LIST */
......
......@@ -102,10 +102,13 @@ which could result in a deadlock if the OS does not support asynchronous io.
ulint
buf_read_ahead_linear(const page_id_t page_id, ulint zip_size, bool ibuf);
/** Issue read requests for pages that need to be recovered.
@param space_id tablespace identifier
@param page_nos page numbers to read, in ascending order */
void buf_read_recv_pages(uint32_t space_id, st_::span<uint32_t> page_nos);
/** Schedule a page for recovery.
@param space tablespace
@param page_id page identifier
@param recs log records
@param init page initialization, or nullptr if the page needs to be read */
void buf_read_recover(fil_space_t *space, const page_id_t page_id,
page_recv_t &recs, recv_init *init);
/** @name Modes used in read-ahead @{ */
/** read only pages belonging to the insert buffer tree */
......
This diff is collapsed.
......@@ -212,6 +212,10 @@ class IORequest
bool is_LRU() const { return (type & (WRITE_LRU ^ WRITE_ASYNC)) != 0; }
bool is_async() const { return (type & (READ_SYNC ^ READ_ASYNC)) != 0; }
void write_complete() const;
void read_complete() const;
void fake_read_complete(os_offset_t offset) const;
/** If requested, free storage space associated with a section of the file.
@param off byte offset from the start (SEEK_SET)
@param len size of the hole in bytes
......@@ -1040,6 +1044,11 @@ int os_aio_init();
Frees the asynchronous io system. */
void os_aio_free();
/** Submit a fake read request during crash recovery.
@param type fake read request
@param offset additional context */
void os_fake_read(const IORequest &type, os_offset_t offset);
/** Request a read or write.
@param type I/O request
@param buf buffer
......
This diff is collapsed.
......@@ -3411,15 +3411,12 @@ os_file_get_status(
return(ret);
}
extern void fil_aio_callback(const IORequest &request);
static void io_callback(tpool::aiocb *cb)
static void io_callback_errorcheck(const tpool::aiocb *cb)
{
const IORequest &request= *static_cast<const IORequest*>
(static_cast<const void*>(cb->m_userdata));
if (cb->m_err != DB_SUCCESS)
{
const IORequest &request= *static_cast<const IORequest*>
(static_cast<const void*>(cb->m_userdata));
ib::fatal() << "IO Error: " << cb->m_err << " during " <<
(request.is_async() ? "async " : "sync ") <<
(request.is_LRU() ? "lru " : "") <<
......@@ -3427,19 +3424,36 @@ static void io_callback(tpool::aiocb *cb)
" of " << cb->m_len << " bytes, for file " << cb->m_fh << ", returned " <<
cb->m_ret_len;
}
/* Return cb back to cache*/
if (cb->m_opcode == tpool::aio_opcode::AIO_PREAD)
{
ut_ad(read_slots->contains(cb));
fil_aio_callback(request);
read_slots->release(cb);
}
else
{
ut_ad(write_slots->contains(cb));
fil_aio_callback(request);
write_slots->release(cb);
}
}
static void fake_io_callback(void *c)
{
tpool::aiocb *cb= static_cast<tpool::aiocb*>(c);
ut_ad(read_slots->contains(cb));
static_cast<const IORequest*>(static_cast<const void*>(cb->m_userdata))->
fake_read_complete(cb->m_offset);
read_slots->release(cb);
}
static void read_io_callback(void *c)
{
tpool::aiocb *cb= static_cast<tpool::aiocb*>(c);
ut_ad(cb->m_opcode == tpool::aio_opcode::AIO_PREAD);
io_callback_errorcheck(cb);
ut_ad(read_slots->contains(cb));
static_cast<const IORequest*>
(static_cast<const void*>(cb->m_userdata))->read_complete();
read_slots->release(cb);
}
static void write_io_callback(void *c)
{
tpool::aiocb *cb= static_cast<tpool::aiocb*>(c);
ut_ad(cb->m_opcode == tpool::aio_opcode::AIO_PWRITE);
ut_ad(write_slots->contains(cb));
static_cast<const IORequest*>
(static_cast<const void*>(cb->m_userdata))->write_complete();
write_slots->release(cb);
}
#ifdef LINUX_NATIVE_AIO
......@@ -3684,6 +3698,28 @@ void os_aio_wait_until_no_pending_reads(bool declare)
tpool::tpool_wait_end();
}
/** Submit a fake read request during crash recovery.
@param type fake read request
@param offset additional context */
void os_fake_read(const IORequest &type, os_offset_t offset)
{
tpool::aiocb *cb= read_slots->acquire();
cb->m_group= read_slots->get_task_group();
cb->m_fh= type.node->handle.m_file;
cb->m_buffer= nullptr;
cb->m_len= 0;
cb->m_offset= offset;
cb->m_opcode= tpool::aio_opcode::AIO_PREAD;
new (cb->m_userdata) IORequest{type};
cb->m_internal_task.m_func= fake_io_callback;
cb->m_internal_task.m_arg= cb;
cb->m_internal_task.m_group= cb->m_group;
srv_thread_pool->submit_task(&cb->m_internal_task);
}
/** Request a read or write.
@param type I/O request
@param buf buffer
......@@ -3729,23 +3765,32 @@ dberr_t os_aio(const IORequest &type, void *buf, os_offset_t offset, size_t n)
return err;
}
io_slots* slots;
tpool::callback_func callback;
tpool::aio_opcode opcode;
if (type.is_read()) {
++os_n_file_reads;
slots = read_slots;
callback = read_io_callback;
opcode = tpool::aio_opcode::AIO_PREAD;
} else {
++os_n_file_writes;
slots = write_slots;
callback = write_io_callback;
opcode = tpool::aio_opcode::AIO_PWRITE;
}
compile_time_assert(sizeof(IORequest) <= tpool::MAX_AIO_USERDATA_LEN);
io_slots* slots= type.is_read() ? read_slots : write_slots;
tpool::aiocb* cb = slots->acquire();
cb->m_buffer = buf;
cb->m_callback = (tpool::callback_func)io_callback;
cb->m_callback = callback;
cb->m_group = slots->get_task_group();
cb->m_fh = type.node->handle.m_file;
cb->m_len = (int)n;
cb->m_offset = offset;
cb->m_opcode = type.is_read() ? tpool::aio_opcode::AIO_PREAD : tpool::aio_opcode::AIO_PWRITE;
cb->m_opcode = opcode;
new (cb->m_userdata) IORequest{type};
if (srv_thread_pool->submit_io(cb)) {
......@@ -3753,6 +3798,7 @@ dberr_t os_aio(const IORequest &type, void *buf, os_offset_t offset, size_t n)
os_file_handle_error(type.node->name, type.is_read()
? "aio read" : "aio write");
err = DB_IO_ERROR;
type.node->space->release();
}
goto func_exit;
......
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment