summaryrefslogtreecommitdiff
path: root/storage/innobase/log/log0log.cc
diff options
context:
space:
mode:
authorVladislav Vaintroub <wlad@mariadb.com>2022-03-07 10:36:16 +0100
committerVladislav Vaintroub <wlad@mariadb.com>2022-03-11 11:11:45 +0100
commit24541d06140b20e8fcd27ac5c7a38c14c876fc6d (patch)
tree860ea5a4286a913002fb4bcc08c71006fc20b7dd /storage/innobase/log/log0log.cc
parentb95942a2a7b0127e0fc466876974bf48e2c53ed3 (diff)
downloadmariadb-git-bb-10.9-MDEV-26603-async-redo-write.tar.gz
MDEV-26603 asynchronous redo log writebb-10.9-MDEV-26603-async-redo-write
Split redo log write submission and completion. Currently, AIO is not really used much, because it turns out to be 10-15% slower than synchronous IO, in write-heavy benchmarks. It looks like that AIO with callbacks in threadpool currently adds more context switches, if submitting thread waits for it. With this patch, AIO is only used where latency is not that important, in "fire-and-forget" manner, e.g log_write_up_to() retry logic, or prior to LRU flushing,or in srv_master periodic timer task. Still, thanks to AIO, internal group commit logic could be improved and sped up. Apparently, "next groupcommit lead candidate" thread waiting was removed. After some benchmarking, spins in group_commit_lock::acquire() were removed as they did not have any positive effect. So overall the effect of that patch should be positive.
Diffstat (limited to 'storage/innobase/log/log0log.cc')
-rw-r--r--storage/innobase/log/log0log.cc269
1 files changed, 212 insertions, 57 deletions
diff --git a/storage/innobase/log/log0log.cc b/storage/innobase/log/log0log.cc
index 83b78ebf385..f1dd89fd5d2 100644
--- a/storage/innobase/log/log0log.cc
+++ b/storage/innobase/log/log0log.cc
@@ -74,6 +74,38 @@ log_t log_sys;
#define LOG_BUF_FLUSH_MARGIN ((4 * 4096) /* cf. log_t::append_prepare() */ \
+ (4U << srv_page_size_shift))
+/**
+ group commit completion callback used for anything
+ that can run asynchronous
+*/
+static const completion_callback async_io_callback{nullptr, nullptr};
+
+/**
+ group commit completion callback that is forcing synchronous IO
+*/
+static const completion_callback sync_io_callback{nullptr, nullptr};
+
+#ifndef DBUG_OFF
+/**
+ Crashing after disk flush requested via dbug_debug flag.
+ flush can be executed by background thread,
+ where DBUG_EXECUTE_IF() does not work, this the value
+ is passed via global variable.
+*/
+static bool crash_after_flush;
+#endif
+
+static void report_aio_error(const char *text, tpool::aiocb *cb);
+
+/** AIO control block with auxilliary information, for async writing.
+Protected by write_lock.*/
+struct Log_aiocb : tpool::aiocb
+{
+ lsn_t lsn;
+ bool durable;
+};
+static Log_aiocb log_aiocb;
+
void log_t::set_capacity()
{
#ifndef SUX_LOCK_GENERIC
@@ -155,12 +187,29 @@ dberr_t log_file_t::read(os_offset_t offset, span<byte> buf) noexcept
return os_file_read(IORequestRead, m_file, buf.data(), offset, buf.size());
}
-void log_file_t::write(os_offset_t offset, span<const byte> buf) noexcept
+void log_file_t::write(os_offset_t offset, span<const byte> buf,
+ tpool::aiocb *iocb) noexcept
{
ut_ad(is_opened());
- if (dberr_t err= os_file_write_func(IORequestWrite, "ib_logfile0", m_file,
+ if (iocb)
+ {
+ ut_ad(buf.size() < UINT_MAX);
+ iocb->m_fh= m_file;
+ iocb->m_opcode= tpool::aio_opcode::AIO_PWRITE;
+ iocb->m_offset= offset;
+ iocb->m_buffer= (void *) buf.data();
+ iocb->m_len= (unsigned) buf.size();
+ if (srv_thread_pool->submit_io(iocb))
+ {
+ iocb->m_err= IF_WIN(GetLastError(), errno);
+ report_aio_error("submitting asynchronous write to ib_logfile0", iocb);
+ }
+ }
+ else if (dberr_t err= os_file_write(IORequestWrite, "ib_logfile0", m_file,
buf.data(), offset, buf.size()))
+ {
ib::fatal() << "write(\"ib_logfile0\") returned " << err;
+ }
}
#ifdef HAVE_PMEM
@@ -505,10 +554,13 @@ void log_t::resize_abort() noexcept
/** Write an aligned buffer to ib_logfile0.
@param buf buffer to be written
@param len length of data to be written
-@param offset log file offset */
-static void log_write_buf(const byte *buf, size_t len, lsn_t offset)
+@param offset log file offset
+@param cb completion callback */
+static void log_write_buf(const byte *buf, size_t len, lsn_t offset,
+ tpool::aiocb *cb)
{
- ut_ad(write_lock.is_owner());
+ ut_ad(cb ? !write_lock.has_owner() : write_lock.is_owner());
+ ut_ad(write_lock.locked());
ut_ad(!recv_no_log_write);
ut_d(const size_t block_size_1= log_sys.get_block_size() - 1);
ut_ad(!(offset & block_size_1));
@@ -519,7 +571,7 @@ static void log_write_buf(const byte *buf, size_t len, lsn_t offset)
if (UNIV_LIKELY(offset + len <= log_sys.file_size))
{
write:
- log_sys.log.write(offset, {buf, len});
+ log_sys.log.write(offset, {buf, len}, cb);
return;
}
@@ -730,30 +782,110 @@ ATTRIBUTE_COLD void log_t::resize_write_buf(size_t length) noexcept
resize_flush_buf, offset, length) == DB_SUCCESS);
}
+
+static void report_aio_error(const char *text, tpool::aiocb *cb)
+{
+ ib::fatal() << "IO Error "
+ << cb->m_err IF_WIN(, << " " << strerror(cb->m_err)) << " "
+ << text << "," << cb->m_len << " bytes at offset "
+ << cb->m_offset;
+}
+
+/** Ensure that previous log writes are durable.
+@return new durable lsn target
+@retval 0 if caller does not need to call log_write_up_to() again
+
+*/
+static lsn_t log_flush()
+{
+ ut_ad(!log_sys.is_pmem());
+ lsn_t lsn= write_lock.value();
+ ut_a(log_sys.flush(lsn));
+#ifndef DBUG_OFF
+ if (crash_after_flush)
+ DBUG_SUICIDE();
+#endif
+ return flush_lock.release(lsn);
+}
+
+
+/** Complete write_buf().
+@param lsn new value of write_lsn
+@param durable whether the write was durable
+@return new write target
+@retval 0 if there is no need to call log_write_up_to() */
+inline lsn_t log_t::complete_write_buf(lsn_t lsn, bool durable) noexcept
+{
+ ut_ad(write_lock.is_owner());
+ ut_ad(durable == flush_lock.is_owner());
+
+ ut_a(lsn >= write_lsn);
+
+ write_lsn= lsn;
+ lsn_t pending_lsn= write_lock.release(lsn);
+ if (durable)
+ pending_lsn= std::max(pending_lsn, log_flush());
+ return pending_lsn;
+}
+
+static void aio_complete_write_buf(void *p)
+{
+ ut_ad(write_lock.locked());
+
+ Log_aiocb *cb= static_cast<Log_aiocb *>(p);
+ if (cb->m_err)
+ report_aio_error("in asynchronous write to ib_logfile0", cb);
+ const bool durable{cb->durable};
+#ifdef UNIV_DEBUG
+ if (durable)
+ {
+ ut_ad(flush_lock.locked());
+ flush_lock.set_owner();
+ }
+ write_lock.set_owner();
+#endif
+
+ if (lsn_t ret_lsn= log_sys.complete_write_buf(cb->lsn, durable))
+ {
+ /** prevent stalls. Also, force special synchronous callback
+ as optimization. We'll avoid threadpool machinery and context
+ switching (we're already in the background thread here)
+ */
+ log_write_up_to(ret_lsn, durable, &sync_io_callback);
+ }
+}
+
+
/** Write buf to ib_logfile0.
@tparam release_latch whether to invoke latch.wr_unlock()
-@return the current log sequence number */
-template<bool release_latch> inline lsn_t log_t::write_buf() noexcept
+@param durable whether to invoke a durable write
+@param sync whether to invoke a synchronous write
+@return new write target
+@retval 0 if there is no need to call log_write_up_to() */
+template<bool release_latch>
+inline lsn_t log_t::write_buf(bool durable, bool sync) noexcept
{
#ifndef SUX_LOCK_GENERIC
ut_ad(latch.is_write_locked());
#endif
ut_ad(!srv_read_only_mode);
ut_ad(!is_pmem());
+ ut_ad(write_lock.is_owner());
+ ut_ad(durable == flush_lock.is_owner());
const lsn_t lsn{get_lsn(std::memory_order_relaxed)};
-
+ DBUG_EXECUTE_IF("crash_after_log_write_upto", crash_after_flush= true;);
if (write_lsn >= lsn)
{
if (release_latch)
latch.wr_unlock();
- ut_ad(write_lsn == lsn);
+ ut_a(write_lsn == lsn);
+ return complete_write_buf(lsn, durable);
}
else
{
ut_ad(!recv_no_log_write);
- write_lock.set_pending(lsn);
- ut_ad(write_lsn >= get_flushed_lsn());
+ ut_a(write_lsn >= get_flushed_lsn());
const size_t block_size_1{get_block_size() - 1};
lsn_t offset{calc_lsn_offset(write_lsn) & ~lsn_t{block_size_1}};
@@ -804,20 +936,34 @@ template<bool release_latch> inline lsn_t log_t::write_buf() noexcept
"InnoDB log write: " LSN_PF, write_lsn);
}
- /* Do the write to the log file */
- log_write_buf(write_buf, length, offset);
if (UNIV_LIKELY_NULL(resize_buf))
resize_write_buf(length);
- write_lsn= lsn;
- }
- return lsn;
+ /* Do the write to the log file */
+ if (sync)
+ {
+ log_write_buf(write_buf, length, offset, nullptr);
+ return complete_write_buf(lsn, durable);
+ }
+
+ /* Async log IO
+ Note : flush/write lock ownership is going to migrate to a
+ background thread*/
+ ut_d(write_lock.reset_owner());
+ ut_d(if (durable) flush_lock.reset_owner());
+
+ log_aiocb.m_callback= aio_complete_write_buf;
+ log_aiocb.durable= durable;
+ log_aiocb.lsn= lsn;
+ log_write_buf(write_buf, length, offset, &log_aiocb);
+ return 0;
+ }
}
bool log_t::flush(lsn_t lsn) noexcept
{
ut_ad(lsn >= get_flushed_lsn());
- flush_lock.set_pending(lsn);
+ ut_ad(flush_lock.is_owner());
const bool success{srv_file_flush_method == SRV_O_DSYNC || log.flush()};
if (UNIV_LIKELY(success))
{
@@ -827,22 +973,25 @@ bool log_t::flush(lsn_t lsn) noexcept
return success;
}
-/** Ensure that previous log writes are durable.
-@param lsn previously written LSN
-@return new durable lsn target
-@retval 0 if there are no pending callbacks on flush_lock
- or there is another group commit lead.
+/*
+ Decide about whether to do synchronous IO.
+ Async might not make sense because of the higher latency or CPU
+ overhead in threadpool, or because the file is cached,and say libaio
+ can't do AIO on cached files.
+
+ Async IO apparently makes sense always if the waiter does
+ not care about result (i.e callback with NULL function)
+
+ NOTE: currently, async IO is mostly unused, because it turns
+ out to be worse in benchmarks. Perhaps it is just too many threads
+ involved in waking and waiting.
*/
-static lsn_t log_flush(lsn_t lsn)
+static bool use_sync_log_write(bool /* durable */,
+ const completion_callback *cb)
{
- ut_ad(!log_sys.is_pmem());
- ut_a(log_sys.flush(lsn));
- DBUG_EXECUTE_IF("crash_after_log_write_upto", DBUG_SUICIDE(););
- return flush_lock.release(lsn);
+ return !cb || cb->m_callback || cb == &sync_io_callback;
}
-static const completion_callback dummy_callback{[](void *) {},nullptr};
-
/** Ensure that the log has been written to the log file up to a given
log entry (such as that of a transaction commit). Start a new write, or
wait and check if an already running write is covering the request.
@@ -859,7 +1008,7 @@ void log_write_up_to(lsn_t lsn, bool durable,
{
/* A non-final batch of recovery is active no writes to the log
are allowed yet. */
- ut_a(!callback);
+ ut_a(!callback || !callback->m_callback);
return;
}
@@ -868,7 +1017,7 @@ void log_write_up_to(lsn_t lsn, bool durable,
#ifdef HAVE_PMEM
if (log_sys.is_pmem())
{
- ut_ad(!callback);
+ ut_ad(!callback || !callback->m_callback);
if (durable)
log_sys.persist(lsn);
return;
@@ -876,42 +1025,49 @@ void log_write_up_to(lsn_t lsn, bool durable,
#endif
repeat:
- if (durable)
- {
- if (flush_lock.acquire(lsn, callback) != group_commit_lock::ACQUIRED)
- return;
- flush_lock.set_pending(log_sys.get_lsn());
- }
+ if (durable &&
+ flush_lock.acquire(lsn, callback) != group_commit_lock::ACQUIRED)
+ return;
- lsn_t pending_write_lsn= 0, pending_flush_lsn= 0;
+ lsn_t pending_lsn= 0;
if (write_lock.acquire(lsn, durable ? nullptr : callback) ==
group_commit_lock::ACQUIRED)
{
+ const bool sync{use_sync_log_write(durable, callback)};
log_sys.latch.wr_lock(SRW_LOCK_CALL);
- pending_write_lsn= write_lock.release(log_sys.write_buf<true>());
+ pending_lsn= log_sys.write_buf<true>(durable, sync);
+ if (!pending_lsn)
+ return;
}
- if (durable)
+ if (durable && !pending_lsn)
{
- pending_flush_lsn= log_flush(write_lock.value());
+ /* We only get here if flush_lock is acquired, but write_lock
+ is expired, i.e lsn was already written, but not flushed yet.*/
+ pending_lsn= log_flush();
}
- if (pending_write_lsn || pending_flush_lsn)
+ if (pending_lsn)
{
- /* There is no new group commit lead; some async waiters could stall. */
- callback= &dummy_callback;
- lsn= std::max(pending_write_lsn, pending_flush_lsn);
+ /* There is no new group commit lead; some waiters could stall.
+ If special sync_io_callback was used we'll continue to use it
+ as optimization to reduce context switches.
+ */
+ if (callback != &sync_io_callback)
+ callback= &async_io_callback;
+ lsn= pending_lsn;
goto repeat;
}
}
/** Write to the log file up to the last log entry.
@param durable whether to wait for a durable write to complete */
-void log_buffer_flush_to_disk(bool durable)
+void log_buffer_flush_to_disk(bool durable, bool wait)
{
ut_ad(!srv_read_only_mode);
- log_write_up_to(log_sys.get_lsn(std::memory_order_acquire), durable);
+ log_write_up_to(log_sys.get_lsn(std::memory_order_acquire), durable,
+ wait ? nullptr : &async_io_callback);
}
/** Prepare to invoke log_write_and_flush(), before acquiring log_sys.latch. */
@@ -926,20 +1082,19 @@ ATTRIBUTE_COLD void log_write_and_flush_prepare()
group_commit_lock::ACQUIRED);
}
-/** Durably write the log up to log_sys.get_lsn(). */
-ATTRIBUTE_COLD void log_write_and_flush()
+/** Durably write the log up to log_sys.get_lsn().
+@return lsn that log_write_up_to() must be invoked with
+@retval 0 if there is no need to invoke log_write_up_to() */
+ATTRIBUTE_COLD __attribute__((warn_unused_result)) lsn_t log_write_and_flush()
{
ut_ad(!srv_read_only_mode);
if (!log_sys.is_pmem())
- {
- const lsn_t lsn{log_sys.write_buf<false>()};
- write_lock.release(lsn);
- log_flush(lsn);
- }
+ return log_sys.write_buf<false>(true, true);
+
#ifdef HAVE_PMEM
- else
- log_sys.persist(log_sys.get_lsn());
+ log_sys.persist(log_sys.get_lsn());
#endif
+ return 0;
}
/********************************************************************