summaryrefslogtreecommitdiff
path: root/storage/innobase/buf/buf0flu.cc
diff options
context:
space:
mode:
authorMarko Mäkelä <marko.makela@mariadb.com>2020-10-26 16:35:47 +0200
committerMarko Mäkelä <marko.makela@mariadb.com>2020-10-26 17:09:01 +0200
commit3a9a3be1c64b14c05648e87ebe0f1dd96457de41 (patch)
treee31bb55c89395aea801dde608c8002d62b385a10 /storage/innobase/buf/buf0flu.cc
parentbd67cb92846cfac49c5e4afe72ed9884176854dc (diff)
downloadmariadb-git-3a9a3be1c64b14c05648e87ebe0f1dd96457de41.tar.gz
MDEV-23855: Improve InnoDB log checkpoint performance
After MDEV-15053, MDEV-22871, MDEV-23399 shifted the scalability bottleneck, log checkpoints became a new bottleneck. If innodb_io_capacity is set low or innodb_max_dirty_pct_lwm is set high and the workload fits in the buffer pool, the page cleaner thread will perform very little flushing. When we reach the capacity of the circular redo log file ib_logfile0 and must initiate a checkpoint, some 'furious flushing' will be necessary. (If innodb_flush_sync=OFF, then flushing would continue at the innodb_io_capacity rate, and writers would be throttled.) We have the best chance of advancing the checkpoint LSN immediately after a page flush batch has been completed. Hence, it is best to perform checkpoints after every batch in the page cleaner thread, attempting to run once per second. By initiating high-priority flushing in the page cleaner as early as possible, we aim to make the throughput more stable. The function buf_flush_wait_flushed() used to sleep for 10ms, hoping that the page cleaner thread would do something during that time. The observed end result was that a large number of threads that call log_free_check() would end up sleeping while nothing useful is happening. We will revise the design so that in the default innodb_flush_sync=ON mode, buf_flush_wait_flushed() will wake up the page cleaner thread to perform the necessary flushing, and it will wait for a signal from the page cleaner thread. If innodb_io_capacity is set to a low value (causing the page cleaner to throttle its work), a write workload would initially perform well, until the capacity of the circular ib_logfile0 is reached and log_free_check() will trigger checkpoints. At that point, the extra waiting in buf_flush_wait_flushed() will start reducing throughput. The page cleaner thread will also initiate log checkpoints after each buf_flush_lists() call, because that is the best point of time for the checkpoint LSN to advance by the maximum amount. Even in 'furious flushing' mode we invoke buf_flush_lists() with innodb_io_capacity_max pages at a time, and at the start of each batch (in the log_flush() callback function that runs in a separate task) we will invoke os_aio_wait_until_no_pending_writes(). This tweak allows the checkpoint to advance in smaller steps and significantly reduces the maximum latency. On an Intel Optane 960 NVMe SSD on Linux, it reduced from 4.6 seconds to 74 milliseconds. On Microsoft Windows with a slower SSD, it reduced from more than 180 seconds to 0.6 seconds. We will make innodb_adaptive_flushing=OFF simply flush innodb_io_capacity per second whenever the dirty proportion of buffer pool pages exceeds innodb_max_dirty_pages_pct_lwm. For innodb_adaptive_flushing=ON we try to make page_cleaner_flush_pages_recommendation() more consistent and predictable: if we are below innodb_adaptive_flushing_lwm, let us flush pages according to the return value of af_get_pct_for_dirty(). innodb_max_dirty_pages_pct_lwm: Revert the change of the default value that was made in MDEV-23399. The value innodb_max_dirty_pages_pct_lwm=0 guarantees that a shutdown of an idle server will be fast. Users might be surprised if normal shutdown suddenly became slower when upgrading within a GA release series. innodb_checkpoint_usec: Remove. The master task will no longer perform periodic log checkpoints. It is the duty of the page cleaner thread. log_sys.max_modified_age: Remove. The current span of the buf_pool.flush_list expressed in LSN only matters for adaptive flushing (outside the 'furious flushing' condition). For the correctness of checkpoints, the only thing that matters is the checkpoint age (log_sys.lsn - log_sys.last_checkpoint_lsn). This run-time constant was also reported as log_max_modified_age_sync. log_sys.max_checkpoint_age_async: Remove. This does not serve any purpose, because the checkpoints will now be triggered by the page cleaner thread. We will retain the log_sys.max_checkpoint_age limit for engaging 'furious flushing'. page_cleaner.slot: Remove. It turns out that page_cleaner_slot.flush_list_time was duplicating page_cleaner.slot.flush_time and page_cleaner.slot.flush_list_pass was duplicating page_cleaner.flush_pass. Likewise, there were some redundant monitor counters, because the page cleaner thread no longer performs any buf_pool.LRU flushing, and because there only is one buf_flush_page_cleaner thread. buf_flush_sync_lsn: Protect writes by buf_pool.flush_list_mutex. buf_pool_t::get_oldest_modification(): Add a parameter to specify the return value when no persistent data pages are dirty. Require the caller to hold buf_pool.flush_list_mutex. log_buf_pool_get_oldest_modification(): Take the fall-back LSN as a parameter. All callers will also invoke log_sys.get_lsn(). log_preflush_pool_modified_pages(): Replaced with buf_flush_wait_flushed(). buf_flush_wait_flushed(): Implement two limits. If not enough buffer pool has been flushed, signal the page cleaner (unless innodb_flush_sync=OFF) and wait for the page cleaner to complete. If the page cleaner thread is not running (which can be the case durign shutdown), initiate the flush and wait for it directly. buf_flush_ahead(): If innodb_flush_sync=ON (the default), submit a new buf_flush_sync_lsn target for the page cleaner but do not wait for the flushing to finish. log_get_capacity(), log_get_max_modified_age_async(): Remove, to make it easier to see that af_get_pct_for_lsn() is not acquiring any mutexes. page_cleaner_flush_pages_recommendation(): Protect all access to buf_pool.flush_list with buf_pool.flush_list_mutex. Previously there were some race conditions in the calculation. buf_flush_sync_for_checkpoint(): New function to process buf_flush_sync_lsn in the page cleaner thread. At the end of each batch, we try to wake up any blocked buf_flush_wait_flushed(). If everything up to buf_flush_sync_lsn has been flushed, we will reset buf_flush_sync_lsn=0. The page cleaner thread will keep 'furious flushing' until the limit is reached. Any threads that are waiting in buf_flush_wait_flushed() will be able to resume as soon as their own limit has been satisfied. buf_flush_page_cleaner: Prioritize buf_flush_sync_lsn and do not sleep as long as it is set. Do not update any page_cleaner statistics for this special mode of operation. In the normal mode (buf_flush_sync_lsn is not set for innodb_flush_sync=ON), try to wake up once per second. No longer check whether srv_inc_activity_count() has been called. After each batch, try to perform a log checkpoint, because the best chances for the checkpoint LSN to advance by the maximum amount are upon completing a flushing batch. log_t: Move buf_free, max_buf_free possibly to the same cache line with log_sys.mutex. log_margin_checkpoint_age(): Simplify the logic, and replace a 0.1-second sleep with a call to buf_flush_wait_flushed() to initiate flushing. Moved to the same compilation unit with the only caller. log_close(): Clean up the calculations. (Should be no functional change.) Return whether flush-ahead is needed. Moved to the same compilation unit with the only caller. mtr_t::finish_write(): Return whether flush-ahead is needed. mtr_t::commit(): Invoke buf_flush_ahead() when needed. Let us avoid external calls in mtr_t::commit() and make the logic easier to follow by having related code in a single compilation unit. Also, we will invoke srv_stats.log_write_requests.inc() only once per mini-transaction commit, while not holding mutexes. log_checkpoint_margin(): Only care about log_sys.max_checkpoint_age. Upon reaching log_sys.max_checkpoint_age where we must wait to prevent the log from getting corrupted, let us wait for at most 1MiB of LSN at a time, before rechecking the condition. This should allow writers to proceed even if the redo log capacity has been reached and 'furious flushing' is in progress. We no longer care about log_sys.max_modified_age_sync or log_sys.max_modified_age_async. The log_sys.max_modified_age_sync could be a relic from the time when there was a srv_master_thread that wrote dirty pages to data files. Also, we no longer have any log_sys.max_checkpoint_age_async limit, because log checkpoints will now be triggered by the page cleaner thread upon completing buf_flush_lists(). log_set_capacity(): Simplify the calculations of the limit (no functional change). log_checkpoint_low(): Split from log_checkpoint(). Moved to the same compilation unit with the caller. log_make_checkpoint(): Only wait for everything to be flushed until the current LSN. create_log_file(): After checkpoint, invoke log_write_up_to() to ensure that the FILE_CHECKPOINT record has been written. This avoids ut_ad(!srv_log_file_created) in create_log_file_rename(). srv_start(): Do not call recv_recovery_from_checkpoint_start() if the log has just been created. Set fil_system.space_id_reuse_warned before dict_boot() has been executed, and clear it after recovery has finished. dict_boot(): Initialize fil_system.max_assigned_id. srv_check_activity(): Remove. The activity count is counting transaction commits and therefore mostly interesting for the purge of history. BtrBulk::insert(): Do not explicitly wake up the page cleaner, but do invoke srv_inc_activity_count(), because that counter is still being used in buf_load_throttle_if_needed() for some heuristics. (It might be cleaner to execute buf_load() in the page cleaner thread!) Reviewed by: Vladislav Vaintroub
Diffstat (limited to 'storage/innobase/buf/buf0flu.cc')
-rw-r--r--storage/innobase/buf/buf0flu.cc809
1 files changed, 485 insertions, 324 deletions
diff --git a/storage/innobase/buf/buf0flu.cc b/storage/innobase/buf/buf0flu.cc
index da25b825e7e..b69026ef990 100644
--- a/storage/innobase/buf/buf0flu.cc
+++ b/storage/innobase/buf/buf0flu.cc
@@ -26,6 +26,7 @@ Created 11/11/1995 Heikki Tuuri
*******************************************************/
#include "univ.i"
+#include <my_service_manager.h>
#include <mysql/service_thd_wait.h>
#include <sql_class.h>
@@ -52,10 +53,6 @@ Created 11/11/1995 Heikki Tuuri
# include "snappy-c.h"
#endif
-/** Sleep time in microseconds for loop waiting for the oldest
-modification lsn */
-static constexpr ulint buf_flush_wait_flushed_sleep_time = 10000;
-
/** Number of pages flushed via LRU. Protected by buf_pool.mutex.
Also included in buf_flush_page_count. */
ulint buf_lru_flush_page_count;
@@ -68,41 +65,27 @@ bool buf_page_cleaner_is_active;
/** Factor for scan length to determine n_pages for intended oldest LSN
progress */
-static ulint buf_flush_lsn_scan_factor = 3;
+static constexpr ulint buf_flush_lsn_scan_factor = 3;
/** Average redo generation rate */
static lsn_t lsn_avg_rate = 0;
-/** Target oldest LSN for the requested flush_sync */
-static std::atomic<lsn_t> buf_flush_sync_lsn;
+/** Target oldest_modification for the page cleaner; writes are protected by
+buf_pool.flush_list_mutex */
+static Atomic_relaxed<lsn_t> buf_flush_sync_lsn;
#ifdef UNIV_PFS_THREAD
mysql_pfs_key_t page_cleaner_thread_key;
#endif /* UNIV_PFS_THREAD */
-/** Page cleaner request state for buf_pool */
-struct page_cleaner_slot_t {
- ulint n_flushed_list;
- /*!< number of flushed pages
- by flush_list flushing */
- ulint flush_list_time;
- /*!< elapsed time for flush_list
- flushing */
- ulint flush_list_pass;
- /*!< count to attempt flush_list
- flushing */
-};
-
/** Page cleaner structure */
-struct page_cleaner_t {
- ulint flush_time; /*!< elapsed time to flush
- requests for all slots */
- ulint flush_pass; /*!< count to finish to flush
- requests for all slots */
- page_cleaner_slot_t slot;
-};
-
-static page_cleaner_t page_cleaner;
+static struct
+{
+ /** total elapsed time in adaptive flushing, in seconds */
+ ulint flush_time;
+ /** number of adaptive flushing passes */
+ ulint flush_pass;
+} page_cleaner;
#ifdef UNIV_DEBUG
my_bool innodb_page_cleaner_disabled_debug;
@@ -257,7 +240,7 @@ ulint buf_flush_dirty_pages(ulint id)
}
mysql_mutex_unlock(&buf_pool.flush_list_mutex);
if (n)
- buf_flush_lists(ULINT_UNDEFINED, LSN_MAX);
+ buf_flush_lists(srv_max_io_capacity, LSN_MAX);
return n;
}
@@ -1449,6 +1432,12 @@ static std::atomic_flag log_flush_pending;
/** Advance log_sys.get_flushed_lsn() */
static void log_flush(void *)
{
+ /* Between batches, we try to prevent I/O stalls by these calls.
+ This should not be needed for correctness. */
+ os_aio_wait_until_no_pending_writes();
+ fil_flush_file_spaces();
+
+ /* Guarantee progress for buf_flush_lists(). */
log_write_up_to(log_sys.get_lsn(), true);
log_flush_pending.clear();
}
@@ -1515,65 +1504,199 @@ ulint buf_flush_lists(ulint max_n, lsn_t lsn)
return n_flushed;
}
-/** Request IO burst and wake up the page_cleaner.
-@param lsn desired lower bound of oldest_modification */
-static void buf_flush_request_force(lsn_t lsn)
+
+/** Initiate a log checkpoint, discarding the start of the log.
+@param oldest_lsn the checkpoint LSN
+@param end_lsn log_sys.get_lsn()
+@return true if success, false if a checkpoint write was already running */
+static bool log_checkpoint_low(lsn_t oldest_lsn, lsn_t end_lsn)
{
- lsn+= lsn_avg_rate * 3;
+ ut_ad(!srv_read_only_mode);
+ ut_ad(log_mutex_own());
+ ut_ad(oldest_lsn <= end_lsn);
+ ut_ad(end_lsn == log_sys.get_lsn());
+ ut_ad(!recv_no_log_write);
+
+ ut_ad(oldest_lsn >= log_sys.last_checkpoint_lsn);
+
+ if (oldest_lsn > log_sys.last_checkpoint_lsn + SIZE_OF_FILE_CHECKPOINT)
+ /* Some log has been written since the previous checkpoint. */;
+ else if (srv_shutdown_state > SRV_SHUTDOWN_INITIATED)
+ /* MariaDB startup expects the redo log file to be logically empty
+ (not even containing a FILE_CHECKPOINT record) after a clean shutdown.
+ Perform an extra checkpoint at shutdown. */;
+ else
+ {
+ /* Do nothing, because nothing was logged (other than a
+ FILE_CHECKPOINT record) since the previous checkpoint. */
+ log_mutex_exit();
+ return true;
+ }
- lsn_t o= 0;
+ /* Repeat the FILE_MODIFY records after the checkpoint, in case some
+ log records between the checkpoint and log_sys.lsn need them.
+ Finally, write a FILE_CHECKPOINT record. Redo log apply expects to
+ see a FILE_CHECKPOINT after the checkpoint, except on clean
+ shutdown, where the log will be empty after the checkpoint.
- while (!buf_flush_sync_lsn.compare_exchange_weak(o, lsn,
- std::memory_order_acquire,
- std::memory_order_relaxed))
- if (lsn > o)
- break;
+ It is important that we write out the redo log before any further
+ dirty pages are flushed to the tablespace files. At this point,
+ because we hold log_sys.mutex, mtr_t::commit() in other threads will
+ be blocked, and no pages can be added to the flush lists. */
+ lsn_t flush_lsn= oldest_lsn;
- mysql_cond_signal(&buf_pool.do_flush_list);
+ if (fil_names_clear(flush_lsn, oldest_lsn != end_lsn ||
+ srv_shutdown_state <= SRV_SHUTDOWN_INITIATED))
+ {
+ flush_lsn= log_sys.get_lsn();
+ ut_ad(flush_lsn >= end_lsn + SIZE_OF_FILE_CHECKPOINT);
+ log_mutex_exit();
+ log_write_up_to(flush_lsn, true, true);
+ log_mutex_enter();
+ if (log_sys.last_checkpoint_lsn >= oldest_lsn)
+ {
+ log_mutex_exit();
+ return true;
+ }
+ }
+ else
+ ut_ad(oldest_lsn >= log_sys.last_checkpoint_lsn);
+
+ ut_ad(log_sys.get_flushed_lsn() >= flush_lsn);
+
+ if (log_sys.n_pending_checkpoint_writes)
+ {
+ /* A checkpoint write is running */
+ log_mutex_exit();
+ return false;
+ }
+
+ log_sys.next_checkpoint_lsn= oldest_lsn;
+ log_write_checkpoint_info(end_lsn);
+ ut_ad(!log_mutex_own());
+
+ return true;
}
-/** Wait until a flush batch of the given lsn ends
-@param[in] new_oldest target oldest_modified_lsn to wait for */
-void buf_flush_wait_flushed(lsn_t new_oldest)
+/** Make a checkpoint. Note that this function does not flush dirty
+blocks from the buffer pool: it only checks what is lsn of the oldest
+modification in the pool, and writes information about the lsn in
+log file. Use log_make_checkpoint() to flush also the pool.
+@retval true if the checkpoint was or had been made
+@retval false if a checkpoint write was already running */
+static bool log_checkpoint()
{
- ut_ad(new_oldest);
+ if (recv_recovery_is_on())
+ recv_sys.apply(true);
- if (srv_flush_sync) {
- /* wake page cleaner for IO burst */
- buf_flush_request_force(new_oldest);
- }
+ switch (srv_file_flush_method) {
+ case SRV_NOSYNC:
+ case SRV_O_DIRECT_NO_FSYNC:
+ break;
+ default:
+ fil_flush_file_spaces();
+ }
- for (;;) {
- /* We don't need to wait for fsync of the flushed
- blocks, because anyway we need fsync to make chekpoint.
- So, we don't need to wait for the batch end here. */
-
- mysql_mutex_lock(&buf_pool.flush_list_mutex);
-
- buf_page_t* bpage;
- /* FIXME: Keep temporary tablespace pages in a separate flush
- list. We would only need to write out temporary pages if the
- page is about to be evicted from the buffer pool, and the page
- contents is still needed (the page has not been freed). */
- for (bpage = UT_LIST_GET_LAST(buf_pool.flush_list);
- bpage && fsp_is_system_temporary(bpage->id().space());
- bpage = UT_LIST_GET_PREV(list, bpage)) {
- ut_ad(bpage->oldest_modification());
- }
+ log_mutex_enter();
+ const lsn_t end_lsn= log_sys.get_lsn();
+ log_flush_order_mutex_enter();
+ mysql_mutex_lock(&buf_pool.flush_list_mutex);
+ const lsn_t oldest_lsn= buf_pool.get_oldest_modification(end_lsn);
+ mysql_mutex_unlock(&buf_pool.flush_list_mutex);
+ log_flush_order_mutex_exit();
+ return log_checkpoint_low(oldest_lsn, end_lsn);
+}
- lsn_t oldest = bpage ? bpage->oldest_modification() : 0;
+/** Make a checkpoint. */
+ATTRIBUTE_COLD void log_make_checkpoint()
+{
+ buf_flush_wait_flushed(log_sys.get_lsn());
+ while (!log_checkpoint());
+}
- mysql_mutex_unlock(&buf_pool.flush_list_mutex);
+/** Wait until all persistent pages are flushed up to a limit.
+@param sync_lsn buf_pool.get_oldest_modification(LSN_MAX) to wait for */
+ATTRIBUTE_COLD void buf_flush_wait_flushed(lsn_t sync_lsn)
+{
+ ut_ad(sync_lsn);
+ ut_ad(sync_lsn < LSN_MAX);
+ ut_ad(!log_mutex_own());
+ ut_ad(!srv_read_only_mode);
- if (oldest == 0 || oldest >= new_oldest) {
- break;
- }
+ if (recv_recovery_is_on())
+ recv_sys.apply(true);
- /* sleep and retry */
- os_thread_sleep(buf_flush_wait_flushed_sleep_time);
+ mysql_mutex_lock(&buf_pool.flush_list_mutex);
- MONITOR_INC(MONITOR_FLUSH_SYNC_WAITS);
- }
+#if 1 /* FIXME: remove this, and guarantee that the page cleaner serves us */
+ if (UNIV_UNLIKELY(!buf_page_cleaner_is_active)
+ ut_d(|| innodb_page_cleaner_disabled_debug))
+ {
+ for (;;)
+ {
+ const lsn_t lsn= buf_pool.get_oldest_modification(sync_lsn);
+ mysql_mutex_unlock(&buf_pool.flush_list_mutex);
+ if (lsn >= sync_lsn)
+ return;
+ ulint n_pages= buf_flush_lists(srv_max_io_capacity, sync_lsn);
+ buf_flush_wait_batch_end_acquiring_mutex(false);
+ if (n_pages)
+ {
+ MONITOR_INC_VALUE_CUMULATIVE(MONITOR_FLUSH_SYNC_TOTAL_PAGE,
+ MONITOR_FLUSH_SYNC_COUNT,
+ MONITOR_FLUSH_SYNC_PAGES, n_pages);
+ log_checkpoint();
+ }
+ MONITOR_INC(MONITOR_FLUSH_SYNC_WAITS);
+ mysql_mutex_lock(&buf_pool.flush_list_mutex);
+ }
+ return;
+ }
+ else if (UNIV_LIKELY(srv_flush_sync))
+#endif
+ {
+ if (buf_flush_sync_lsn < sync_lsn)
+ {
+ buf_flush_sync_lsn= sync_lsn;
+ mysql_cond_signal(&buf_pool.do_flush_list);
+ }
+ }
+
+ while (buf_pool.get_oldest_modification(sync_lsn) < sync_lsn)
+ {
+ tpool::tpool_wait_begin();
+ thd_wait_begin(nullptr, THD_WAIT_DISKIO);
+ mysql_cond_wait(&buf_pool.done_flush_list, &buf_pool.flush_list_mutex);
+ thd_wait_end(nullptr);
+ tpool::tpool_wait_end();
+
+ MONITOR_INC(MONITOR_FLUSH_SYNC_WAITS);
+ }
+
+ mysql_mutex_unlock(&buf_pool.flush_list_mutex);
+}
+
+/** If innodb_flush_sync=ON, initiate a furious flush.
+@param lsn buf_pool.get_oldest_modification(LSN_MAX) target */
+void buf_flush_ahead(lsn_t lsn)
+{
+ ut_ad(!log_mutex_own());
+ ut_ad(!srv_read_only_mode);
+
+ if (recv_recovery_is_on())
+ recv_sys.apply(true);
+
+ if (buf_flush_sync_lsn < lsn &&
+ UNIV_LIKELY(srv_flush_sync) && UNIV_LIKELY(buf_page_cleaner_is_active))
+ {
+ mysql_mutex_lock(&buf_pool.flush_list_mutex);
+ if (buf_flush_sync_lsn < lsn)
+ {
+ buf_flush_sync_lsn= lsn;
+ mysql_cond_signal(&buf_pool.do_flush_list);
+ }
+ mysql_mutex_unlock(&buf_pool.flush_list_mutex);
+ }
}
/** Wait for pending flushes to complete. */
@@ -1587,29 +1710,84 @@ void buf_flush_wait_batch_end_acquiring_mutex(bool lru)
}
}
+/** Conduct checkpoint-related flushing for innodb_flush_sync=ON,
+and try to initiate checkpoints until the target is met.
+@param lsn minimum value of buf_pool.get_oldest_modification(LSN_MAX) */
+ATTRIBUTE_COLD static void buf_flush_sync_for_checkpoint(lsn_t lsn)
+{
+ ut_ad(!srv_read_only_mode);
+
+ for (;;)
+ {
+ mysql_mutex_unlock(&buf_pool.flush_list_mutex);
+
+ if (ulint n_flushed= buf_flush_lists(srv_max_io_capacity, lsn))
+ {
+ MONITOR_INC_VALUE_CUMULATIVE(MONITOR_FLUSH_SYNC_TOTAL_PAGE,
+ MONITOR_FLUSH_SYNC_COUNT,
+ MONITOR_FLUSH_SYNC_PAGES, n_flushed);
+ }
+
+ /* Attempt to perform a log checkpoint upon completing each batch. */
+ if (recv_recovery_is_on())
+ recv_sys.apply(true);
+
+ switch (srv_file_flush_method) {
+ case SRV_NOSYNC:
+ case SRV_O_DIRECT_NO_FSYNC:
+ break;
+ default:
+ fil_flush_file_spaces();
+ }
+
+ log_mutex_enter();
+ const lsn_t newest_lsn= log_sys.get_lsn();
+ log_flush_order_mutex_enter();
+ mysql_mutex_lock(&buf_pool.flush_list_mutex);
+ lsn_t measure= buf_pool.get_oldest_modification(0);
+ log_flush_order_mutex_exit();
+ const lsn_t checkpoint_lsn= measure ? measure : newest_lsn;
+
+ if (checkpoint_lsn > log_sys.last_checkpoint_lsn + SIZE_OF_FILE_CHECKPOINT)
+ {
+ mysql_mutex_unlock(&buf_pool.flush_list_mutex);
+ log_checkpoint_low(checkpoint_lsn, newest_lsn);
+ mysql_mutex_lock(&buf_pool.flush_list_mutex);
+ measure= buf_pool.get_oldest_modification(LSN_MAX);
+ }
+ else
+ {
+ log_mutex_exit();
+ if (!measure)
+ measure= LSN_MAX;
+ }
+
+ ut_ad(!log_mutex_own());
+
+ /* After attempting log checkpoint, check if we have reached our target. */
+ const lsn_t target= buf_flush_sync_lsn;
+
+ if (measure >= target)
+ buf_flush_sync_lsn= 0;
+
+ /* wake up buf_flush_wait_flushed() */
+ mysql_cond_broadcast(&buf_pool.done_flush_list);
+
+ lsn= std::max(lsn, target);
+
+ if (measure >= lsn)
+ return;
+ }
+}
+
/*********************************************************************//**
Calculates if flushing is required based on number of dirty pages in
the buffer pool.
+@param dirty_pct 100*flush_list.count / (LRU.count + free.count)
@return percent of io_capacity to flush to manage dirty page ratio */
-static
-ulint
-af_get_pct_for_dirty()
+static ulint af_get_pct_for_dirty(double dirty_pct)
{
- const ulint dirty = UT_LIST_GET_LEN(buf_pool.flush_list);
- if (!dirty) {
- /* No pages modified */
- return 0;
- }
-
- /* 1 + is there to avoid division by zero (in case the buffer
- pool (including the flush_list) was emptied while we are
- looking at it) */
- double dirty_pct = 100 * static_cast<double>(dirty)
- / static_cast<double>(1 + UT_LIST_GET_LEN(buf_pool.LRU)
- + UT_LIST_GET_LEN(buf_pool.free));
-
- ut_a(srv_max_dirty_pages_pct_lwm
- <= srv_max_buf_pool_modified_pct);
+ ut_ad(srv_max_dirty_pages_pct_lwm <= srv_max_buf_pool_modified_pct);
if (srv_max_dirty_pages_pct_lwm == 0) {
/* The user has not set the option to preflush dirty
@@ -1620,7 +1798,7 @@ af_get_pct_for_dirty()
innodb_io_capacity. */
return(100);
}
- } else if (dirty_pct >= srv_max_dirty_pages_pct_lwm) {
+ } else {
/* We should start flushing pages gradually. */
return(static_cast<ulint>((dirty_pct * 100)
/ (srv_max_buf_pool_modified_pct + 1)));
@@ -1638,30 +1816,16 @@ af_get_pct_for_lsn(
/*===============*/
lsn_t age) /*!< in: current age of LSN. */
{
- lsn_t max_async_age;
- lsn_t lsn_age_factor;
lsn_t af_lwm = static_cast<lsn_t>(
srv_adaptive_flushing_lwm
- * static_cast<double>(log_get_capacity()) / 100);
+ * static_cast<double>(log_sys.log_capacity) / 100);
if (age < af_lwm) {
/* No adaptive flushing. */
return(0);
}
- max_async_age = log_get_max_modified_age_async();
-
- if (age < max_async_age && !srv_adaptive_flushing) {
- /* We have still not reached the max_async point and
- the user has disabled adaptive flushing. */
- return(0);
- }
-
- /* If we are here then we know that either:
- 1) User has enabled adaptive flushing
- 2) User may have disabled adaptive flushing but we have reached
- max_async_age. */
- lsn_age_factor = (age * 100) / max_async_age;
+ lsn_t lsn_age_factor = (age * 100) / log_sys.max_modified_age_async;
ut_ad(srv_max_io_capacity >= srv_io_capacity);
return static_cast<ulint>(
@@ -1671,46 +1835,40 @@ af_get_pct_for_lsn(
/ 7.5));
}
-/*********************************************************************//**
-This function is called approximately once every second by the
-page_cleaner thread. Based on various factors it decides if there is a
-need to do flushing.
+/** This function is called approximately once every second by the
+page_cleaner thread if innodb_adaptive_flushing=ON.
+Based on various factors it decides if there is a need to do flushing.
@return number of pages recommended to be flushed
-@param last_pages_in the number of pages flushed by the last flush_list
- flushing. */
-static
-ulint
-page_cleaner_flush_pages_recommendation(ulint last_pages_in)
+@param last_pages_in number of pages flushed in previous batch
+@param oldest_lsn buf_pool.get_oldest_modification(0)
+@param dirty_pct 100*flush_list.count / (LRU.count + free.count) */
+static ulint page_cleaner_flush_pages_recommendation(ulint last_pages_in,
+ lsn_t oldest_lsn,
+ double dirty_pct)
{
static lsn_t prev_lsn = 0;
static ulint sum_pages = 0;
static ulint avg_page_rate = 0;
static ulint n_iterations = 0;
static time_t prev_time;
- lsn_t oldest_lsn;
- lsn_t age;
lsn_t lsn_rate;
ulint n_pages = 0;
- ulint pct_for_dirty = 0;
- ulint pct_for_lsn = 0;
- ulint pct_total = 0;
const lsn_t cur_lsn = log_sys.get_lsn();
+ ulint pct_for_dirty = af_get_pct_for_dirty(dirty_pct);
+ ut_ad(oldest_lsn <= cur_lsn);
+ ulint pct_for_lsn = af_get_pct_for_lsn(cur_lsn - oldest_lsn);
+ time_t curr_time = time(nullptr);
- if (prev_lsn == 0) {
- /* First time around. */
+ if (!prev_lsn || !pct_for_lsn) {
+ prev_time = curr_time;
prev_lsn = cur_lsn;
- prev_time = time(NULL);
- return(0);
- }
-
- if (prev_lsn == cur_lsn) {
- return(0);
+ return ulint(double(pct_for_dirty) / 100.0
+ * double(srv_io_capacity));
}
sum_pages += last_pages_in;
- time_t curr_time = time(NULL);
double time_elapsed = difftime(curr_time, prev_time);
/* We update our variables every srv_flushing_avg_loops
@@ -1740,37 +1898,12 @@ page_cleaner_flush_pages_recommendation(ulint last_pages_in)
page_cleaner.flush_time = 0;
page_cleaner.flush_pass = 0;
- ulint list_tm = page_cleaner.slot.flush_list_time;
- ulint list_pass = page_cleaner.slot.flush_list_pass;
- page_cleaner.slot.flush_list_time = 0;
- page_cleaner.slot.flush_list_pass = 0;
-
- /* minimum values are 1, to avoid dividing by zero. */
- if (list_tm < 1) {
- list_tm = 1;
- }
- if (flush_tm < 1) {
- flush_tm = 1;
+ if (flush_pass) {
+ flush_tm /= flush_pass;
}
- if (list_pass < 1) {
- list_pass = 1;
- }
- if (flush_pass < 1) {
- flush_pass = 1;
- }
-
- MONITOR_SET(MONITOR_FLUSH_ADAPTIVE_AVG_TIME_SLOT,
- list_tm / list_pass);
-
- MONITOR_SET(MONITOR_FLUSH_ADAPTIVE_AVG_TIME_THREAD,
- list_tm / flush_pass);
- MONITOR_SET(MONITOR_FLUSH_ADAPTIVE_AVG_TIME_EST,
- flush_tm / flush_pass);
- MONITOR_SET(MONITOR_FLUSH_AVG_TIME, flush_tm / flush_pass);
-
- MONITOR_SET(MONITOR_FLUSH_ADAPTIVE_AVG_PASS, list_pass);
- MONITOR_SET(MONITOR_FLUSH_AVG_PASS, flush_pass);
+ MONITOR_SET(MONITOR_FLUSH_ADAPTIVE_AVG_TIME, flush_tm);
+ MONITOR_SET(MONITOR_FLUSH_ADAPTIVE_AVG_PASS, flush_pass);
prev_lsn = cur_lsn;
prev_time = curr_time;
@@ -1780,30 +1913,24 @@ page_cleaner_flush_pages_recommendation(ulint last_pages_in)
sum_pages = 0;
}
- oldest_lsn = buf_pool.get_oldest_modification();
-
- ut_ad(oldest_lsn <= log_get_lsn());
-
- age = cur_lsn > oldest_lsn ? cur_lsn - oldest_lsn : 0;
-
- pct_for_dirty = af_get_pct_for_dirty();
- pct_for_lsn = af_get_pct_for_lsn(age);
+ mysql_mutex_lock(&buf_pool.flush_list_mutex);
- pct_total = ut_max(pct_for_dirty, pct_for_lsn);
+ ulint pct_total = std::max(pct_for_dirty, pct_for_lsn);
/* Estimate pages to be flushed for the lsn progress */
lsn_t target_lsn = oldest_lsn
+ lsn_avg_rate * buf_flush_lsn_scan_factor;
ulint pages_for_lsn = 0;
- mysql_mutex_lock(&buf_pool.flush_list_mutex);
for (buf_page_t* b = UT_LIST_GET_LAST(buf_pool.flush_list);
b != NULL;
b = UT_LIST_GET_PREV(list, b)) {
if (b->oldest_modification() > target_lsn) {
break;
}
- ++pages_for_lsn;
+ if (++pages_for_lsn >= srv_max_io_capacity) {
+ break;
+ }
}
mysql_mutex_unlock(&buf_pool.flush_list_mutex);
@@ -1812,11 +1939,6 @@ page_cleaner_flush_pages_recommendation(ulint last_pages_in)
pages_for_lsn = 1;
}
- /* Cap the maximum IO capacity that we are going to use by
- max_io_capacity. Limit the value to avoid too quick increase */
- pages_for_lsn = std::min<ulint>(
- pages_for_lsn, srv_max_io_capacity * 2);
-
n_pages = (ulint(double(srv_io_capacity) * double(pct_total) / 100.0)
+ avg_page_rate + pages_for_lsn) / 3;
@@ -1836,183 +1958,222 @@ page_cleaner_flush_pages_recommendation(ulint last_pages_in)
return(n_pages);
}
-/** Initiate a flushing batch.
-@param max_n maximum mumber of blocks flushed
-@param lsn oldest_modification limit
-@return ut_time_ms() at the start of the wait */
-static ulint pc_request_flush_slot(ulint max_n, lsn_t lsn)
-{
- ut_ad(max_n);
- ut_ad(lsn);
-
- const ulint flush_start_tm= ut_time_ms();
- page_cleaner.slot.n_flushed_list= buf_flush_lists(max_n, lsn);
- page_cleaner.slot.flush_list_time+= ut_time_ms() - flush_start_tm;
- page_cleaner.slot.flush_list_pass++;
- return flush_start_tm;
-}
-
-#ifdef UNIV_DEBUG
-/** Loop used to disable the page cleaner thread. */
-static void buf_flush_page_cleaner_disabled_loop()
-{
- while (innodb_page_cleaner_disabled_debug
- && srv_shutdown_state == SRV_SHUTDOWN_NONE) {
- os_thread_sleep(100000);
- }
-}
-#endif /* UNIV_DEBUG */
-
/******************************************************************//**
page_cleaner thread tasked with flushing dirty pages from the buffer
pools. As of now we'll have only one coordinator.
@return a dummy parameter */
static os_thread_ret_t DECLARE_THREAD(buf_flush_page_cleaner)(void*)
{
- my_thread_init();
+ my_thread_init();
#ifdef UNIV_PFS_THREAD
- pfs_register_thread(page_cleaner_thread_key);
+ pfs_register_thread(page_cleaner_thread_key);
#endif /* UNIV_PFS_THREAD */
- ut_ad(!srv_read_only_mode);
- ut_ad(buf_page_cleaner_is_active);
+ ut_ad(!srv_read_only_mode);
+ ut_ad(buf_page_cleaner_is_active);
#ifdef UNIV_DEBUG_THREAD_CREATION
- ib::info() << "page_cleaner thread running, id "
- << os_thread_pf(os_thread_get_curr_id());
+ ib::info() << "page_cleaner thread running, id "
+ << os_thread_pf(os_thread_get_curr_id());
#endif /* UNIV_DEBUG_THREAD_CREATION */
#ifdef UNIV_LINUX
- /* linux might be able to set different setting for each thread.
- worth to try to set high priority for the page cleaner thread */
- const pid_t tid= static_cast<pid_t>(syscall(SYS_gettid));
- setpriority(PRIO_PROCESS, tid, -20);
- if (getpriority(PRIO_PROCESS, tid) != -20) {
- ib::info() << "If the mysqld execution user is authorized,"
- " page cleaner thread priority can be changed."
- " See the man page of setpriority().";
- }
+ /* linux might be able to set different setting for each thread.
+ worth to try to set high priority for the page cleaner thread */
+ const pid_t tid= static_cast<pid_t>(syscall(SYS_gettid));
+ setpriority(PRIO_PROCESS, tid, -20);
+ if (getpriority(PRIO_PROCESS, tid) != -20)
+ ib::info() << "If the mysqld execution user is authorized,"
+ " page cleaner thread priority can be changed."
+ " See the man page of setpriority().";
#endif /* UNIV_LINUX */
- ulint curr_time = ut_time_ms();
- ulint n_flushed = 0;
- ulint last_activity = srv_get_activity_count();
- ulint last_pages = 0;
-
- for (ulint next_loop_time = curr_time + 1000;
- srv_shutdown_state <= SRV_SHUTDOWN_INITIATED;
- curr_time = ut_time_ms()) {
- bool sleep_timeout;
-
- /* The page_cleaner skips sleep if the server is
- idle and there are no pending IOs in the buffer pool
- and there is work to do. */
- if (next_loop_time <= curr_time) {
- sleep_timeout = true;
- } else if (!n_flushed || !buf_pool.n_pend_reads
- || srv_check_activity(&last_activity)) {
- const ulint sleep_ms = std::min<ulint>(next_loop_time
- - curr_time,
- 1000);
- timespec abstime;
- set_timespec_nsec(abstime, 1000000ULL * sleep_ms);
- mysql_mutex_lock(&buf_pool.flush_list_mutex);
- const auto error = mysql_cond_timedwait(
- &buf_pool.do_flush_list,
- &buf_pool.flush_list_mutex,
- &abstime);
- mysql_mutex_unlock(&buf_pool.flush_list_mutex);
- sleep_timeout = error == ETIMEDOUT || error == ETIME;
- if (srv_shutdown_state > SRV_SHUTDOWN_INITIATED) {
- break;
- }
- } else {
- sleep_timeout = false;
- }
+ ulint last_pages= 0;
+ timespec abstime;
+ set_timespec(abstime, 1);
- if (sleep_timeout) {
- /* no activity, slept enough */
- n_flushed = buf_flush_lists(srv_io_capacity, LSN_MAX);
- last_pages = n_flushed;
+ mysql_mutex_lock(&buf_pool.flush_list_mutex);
- if (n_flushed) {
- MONITOR_INC_VALUE_CUMULATIVE(
- MONITOR_FLUSH_BACKGROUND_TOTAL_PAGE,
- MONITOR_FLUSH_BACKGROUND_COUNT,
- MONITOR_FLUSH_BACKGROUND_PAGES,
- n_flushed);
+ lsn_t lsn_limit;
- }
- } else if (lsn_t lsn_limit = buf_flush_sync_lsn.exchange(
- 0, std::memory_order_release)) {
- page_cleaner.flush_time += ut_time_ms()
- - pc_request_flush_slot(ULINT_MAX, lsn_limit);
- page_cleaner.flush_pass++;
- n_flushed = page_cleaner.slot.n_flushed_list;
-
- if (n_flushed) {
- MONITOR_INC_VALUE_CUMULATIVE(
- MONITOR_FLUSH_SYNC_TOTAL_PAGE,
- MONITOR_FLUSH_SYNC_COUNT,
- MONITOR_FLUSH_SYNC_PAGES,
- n_flushed);
- }
- } else if (!srv_check_activity(&last_activity)) {
- /* no activity, but woken up by event */
- n_flushed = 0;
- } else if (ulint n= page_cleaner_flush_pages_recommendation(
- last_pages)) {
- /* Estimate pages from flush_list to be flushed */
- ulint tm= pc_request_flush_slot(n, LSN_MAX);
-
- page_cleaner.flush_time += ut_time_ms() - tm;
- page_cleaner.flush_pass++ ;
-
- n_flushed = page_cleaner.slot.n_flushed_list;
-
- if (n_flushed) {
- MONITOR_INC_VALUE_CUMULATIVE(
- MONITOR_FLUSH_ADAPTIVE_TOTAL_PAGE,
- MONITOR_FLUSH_ADAPTIVE_COUNT,
- MONITOR_FLUSH_ADAPTIVE_PAGES,
- n_flushed);
- }
- } else {
- n_flushed = 0;
- }
+ for (;;)
+ {
+ lsn_limit= buf_flush_sync_lsn;
- if (!n_flushed) {
- next_loop_time = curr_time + 1000;
- }
+ if (UNIV_UNLIKELY(lsn_limit != 0))
+ {
+furious_flush:
+ buf_flush_sync_for_checkpoint(lsn_limit);
+ last_pages= 0;
+ set_timespec(abstime, 1);
+ continue;
+ }
- ut_d(buf_flush_page_cleaner_disabled_loop());
- }
+ if (srv_shutdown_state > SRV_SHUTDOWN_INITIATED)
+ break;
- if (srv_fast_shutdown != 2) {
- buf_flush_wait_batch_end_acquiring_mutex(true);
- buf_flush_wait_batch_end_acquiring_mutex(false);
- }
+ mysql_cond_timedwait(&buf_pool.do_flush_list, &buf_pool.flush_list_mutex,
+ &abstime);
+ set_timespec(abstime, 1);
- mysql_mutex_lock(&buf_pool.flush_list_mutex);
- buf_page_cleaner_is_active = false;
- mysql_cond_broadcast(&buf_pool.done_flush_list);
- mysql_mutex_unlock(&buf_pool.flush_list_mutex);
+ lsn_limit= buf_flush_sync_lsn;
- my_thread_end();
- /* We count the number of threads in os_thread_exit(). A created
- thread should always use that to exit and not use return() to exit. */
- os_thread_exit();
+ if (UNIV_UNLIKELY(lsn_limit != 0))
+ goto furious_flush;
- OS_THREAD_DUMMY_RETURN;
-}
+ if (srv_shutdown_state > SRV_SHUTDOWN_INITIATED)
+ break;
+
+ const ulint dirty_blocks= UT_LIST_GET_LEN(buf_pool.flush_list);
+
+ if (!dirty_blocks)
+ continue;
+
+ /* We perform dirty reads of the LRU+free list lengths here.
+ Division by zero is not possible, because buf_pool.flush_list is
+ guaranteed to be nonempty, and it is a subset of buf_pool.LRU. */
+ const double dirty_pct= double(dirty_blocks) * 100.0 /
+ double(UT_LIST_GET_LEN(buf_pool.LRU) + UT_LIST_GET_LEN(buf_pool.free));
+
+ if (dirty_pct < srv_max_dirty_pages_pct_lwm)
+ continue;
+
+ const lsn_t oldest_lsn= buf_pool.get_oldest_modification(0);
+
+ mysql_mutex_unlock(&buf_pool.flush_list_mutex);
+
+ ulint n_flushed;
+
+ if (!srv_adaptive_flushing)
+ {
+ n_flushed= buf_flush_lists(srv_io_capacity, LSN_MAX);
+
+ if (n_flushed)
+ {
+ MONITOR_INC_VALUE_CUMULATIVE(MONITOR_FLUSH_BACKGROUND_TOTAL_PAGE,
+ MONITOR_FLUSH_BACKGROUND_COUNT,
+ MONITOR_FLUSH_BACKGROUND_PAGES,
+ n_flushed);
+do_checkpoint:
+ /* The periodic log_checkpoint() call here makes it harder to
+ reproduce bugs in crash recovery or mariabackup --prepare, or
+ in code that writes the redo log records. Omitting the call
+ here should not affect correctness, because log_free_check()
+ should still be invoking checkpoints when needed. */
+ DBUG_EXECUTE_IF("ib_log_checkpoint_avoid", goto next;);
+
+ if (!recv_recovery_is_on() && srv_operation == SRV_OPERATION_NORMAL)
+ log_checkpoint();
+ }
+ }
+ else if (ulint n= page_cleaner_flush_pages_recommendation(last_pages,
+ oldest_lsn,
+ dirty_pct))
+ {
+ page_cleaner.flush_pass++;
+ const ulint tm= ut_time_ms();
+ last_pages= n_flushed= buf_flush_lists(n, LSN_MAX);
+ page_cleaner.flush_time+= ut_time_ms() - tm;
+
+ if (n_flushed)
+ {
+ MONITOR_INC_VALUE_CUMULATIVE(MONITOR_FLUSH_ADAPTIVE_TOTAL_PAGE,
+ MONITOR_FLUSH_ADAPTIVE_COUNT,
+ MONITOR_FLUSH_ADAPTIVE_PAGES,
+ n_flushed);
+ goto do_checkpoint;
+ }
+ }
+
+#ifdef UNIV_DEBUG
+ while (innodb_page_cleaner_disabled_debug && !buf_flush_sync_lsn &&
+ srv_shutdown_state == SRV_SHUTDOWN_NONE)
+ os_thread_sleep(100000);
+#endif /* UNIV_DEBUG */
+
+#ifndef DBUG_OFF
+next:
+#endif /* !DBUG_OFF */
+ mysql_mutex_lock(&buf_pool.flush_list_mutex);
+ }
+
+ mysql_mutex_unlock(&buf_pool.flush_list_mutex);
+ if (srv_fast_shutdown != 2)
+ {
+ buf_flush_wait_batch_end_acquiring_mutex(true);
+ buf_flush_wait_batch_end_acquiring_mutex(false);
+ }
+
+ log_flush_task.wait();
+
+ mysql_mutex_lock(&buf_pool.flush_list_mutex);
+ lsn_limit= buf_flush_sync_lsn;
+ if (UNIV_UNLIKELY(lsn_limit != 0))
+ goto furious_flush;
+ buf_page_cleaner_is_active= false;
+ mysql_cond_broadcast(&buf_pool.done_flush_list);
+ mysql_mutex_unlock(&buf_pool.flush_list_mutex);
+
+ my_thread_end();
+ /* We count the number of threads in os_thread_exit(). A created
+ thread should always use that to exit and not use return() to exit. */
+ os_thread_exit();
+
+ OS_THREAD_DUMMY_RETURN;
+}
/** Initialize page_cleaner. */
-void buf_flush_page_cleaner_init()
+ATTRIBUTE_COLD void buf_flush_page_cleaner_init()
{
ut_ad(!buf_page_cleaner_is_active);
+ ut_ad(srv_operation == SRV_OPERATION_NORMAL ||
+ srv_operation == SRV_OPERATION_RESTORE ||
+ srv_operation == SRV_OPERATION_RESTORE_EXPORT);
+ buf_flush_sync_lsn= 0;
buf_page_cleaner_is_active= true;
os_thread_create(buf_flush_page_cleaner);
}
+/** @return the number of dirty pages in the buffer pool */
+static ulint buf_flush_list_length()
+{
+ mysql_mutex_lock(&buf_pool.flush_list_mutex);
+ const ulint len= UT_LIST_GET_LEN(buf_pool.flush_list);
+ mysql_mutex_unlock(&buf_pool.flush_list_mutex);
+ return len;
+}
+
+/** Flush the buffer pool on shutdown. */
+ATTRIBUTE_COLD void buf_flush_buffer_pool()
+{
+ ut_ad(!buf_page_cleaner_is_active);
+ ut_ad(!buf_flush_sync_lsn);
+
+ service_manager_extend_timeout(INNODB_EXTEND_TIMEOUT_INTERVAL,
+ "Waiting to flush the buffer pool");
+
+ while (buf_pool.n_flush_list || buf_flush_list_length())
+ {
+ buf_flush_lists(srv_max_io_capacity, LSN_MAX);
+ timespec abstime;
+
+ if (buf_pool.n_flush_list)
+ {
+ service_manager_extend_timeout(INNODB_EXTEND_TIMEOUT_INTERVAL,
+ "Waiting to flush " ULINTPF " pages",
+ buf_flush_list_length());
+ set_timespec(abstime, INNODB_EXTEND_TIMEOUT_INTERVAL / 2);
+ mysql_mutex_lock(&buf_pool.mutex);
+ while (buf_pool.n_flush_list)
+ mysql_cond_timedwait(&buf_pool.done_flush_list, &buf_pool.mutex,
+ &abstime);
+ mysql_mutex_unlock(&buf_pool.mutex);
+ }
+ }
+
+ ut_ad(!buf_pool.any_io_pending());
+ log_flush_task.wait();
+}
+
/** Synchronously flush dirty blocks.
NOTE: The calling thread is not allowed to hold any buffer page latches! */
void buf_flush_sync()
@@ -2021,7 +2182,7 @@ void buf_flush_sync()
for (;;)
{
- const ulint n_flushed= buf_flush_lists(ULINT_UNDEFINED, LSN_MAX);
+ const ulint n_flushed= buf_flush_lists(srv_max_io_capacity, LSN_MAX);
buf_flush_wait_batch_end_acquiring_mutex(false);
if (!n_flushed)
{