summaryrefslogtreecommitdiff
path: root/storage/xtradb/buf/buf0flu.cc
diff options
context:
space:
mode:
Diffstat (limited to 'storage/xtradb/buf/buf0flu.cc')
-rw-r--r--storage/xtradb/buf/buf0flu.cc435
1 files changed, 256 insertions, 179 deletions
diff --git a/storage/xtradb/buf/buf0flu.cc b/storage/xtradb/buf/buf0flu.cc
index a7f55eb9c79..9e92cf321a7 100644
--- a/storage/xtradb/buf/buf0flu.cc
+++ b/storage/xtradb/buf/buf0flu.cc
@@ -59,8 +59,12 @@ need to protect it by a mutex. It is only ever read by the thread
doing the shutdown */
UNIV_INTERN ibool buf_page_cleaner_is_active = FALSE;
+/** Flag indicating if the lru_manager is in active state. */
+UNIV_INTERN bool buf_lru_manager_is_active = false;
+
#ifdef UNIV_PFS_THREAD
UNIV_INTERN mysql_pfs_key_t buf_page_cleaner_thread_key;
+UNIV_INTERN mysql_pfs_key_t buf_lru_manager_thread_key;
#endif /* UNIV_PFS_THREAD */
/** If LRU list of a buf_pool is less than this size then LRU eviction
@@ -503,15 +507,15 @@ buf_flush_ready_for_replace(
#ifdef UNIV_DEBUG
buf_pool_t* buf_pool = buf_pool_from_bpage(bpage);
ut_ad(mutex_own(&buf_pool->LRU_list_mutex));
-#endif
+#endif /* UNIV_DEBUG */
ut_ad(mutex_own(buf_page_get_mutex(bpage)));
ut_ad(bpage->in_LRU_list);
- if (UNIV_LIKELY(buf_page_in_file(bpage))) {
+ if (buf_page_in_file(bpage)) {
return(bpage->oldest_modification == 0
- && buf_page_get_io_fix(bpage) == BUF_IO_NONE
- && bpage->buf_fix_count == 0);
+ && bpage->buf_fix_count == 0
+ && buf_page_get_io_fix(bpage) == BUF_IO_NONE);
}
ut_print_timestamp(stderr);
@@ -552,13 +556,8 @@ buf_flush_ready_for_flush(
case BUF_FLUSH_LIST:
case BUF_FLUSH_LRU:
case BUF_FLUSH_SINGLE_PAGE:
- /* Because any thread may call single page flush, even
- when owning locks on pages, to avoid deadlocks, we must
- make sure that the that it is not buffer fixed.
- The same holds true for LRU flush because a user thread
- may end up waiting for an LRU flush to end while
- holding locks on other pages. */
- return(bpage->buf_fix_count == 0);
+ return(true);
+
case BUF_FLUSH_N_TYPES:
break;
}
@@ -982,9 +981,12 @@ Writes a flushable page asynchronously from the buffer pool to a file.
NOTE: in simulated aio we must call
os_aio_simulated_wake_handler_threads after we have posted a batch of
writes! NOTE: buf_page_get_mutex(bpage) must be held upon entering this
-function, and it will be released by this function. */
+function, and it will be released by this function if it returns true.
+LRU_list_mutex must be held iff performing a single page flush and will be
+released by the function if it returns true.
+@return TRUE if the page was flushed */
UNIV_INTERN
-void
+bool
buf_flush_page(
/*===========*/
buf_pool_t* buf_pool, /*!< in: buffer pool instance */
@@ -992,111 +994,98 @@ buf_flush_page(
buf_flush_t flush_type, /*!< in: type of flush */
bool sync) /*!< in: true if sync IO request */
{
- ib_mutex_t* block_mutex;
- ibool is_uncompressed;
-
ut_ad(flush_type < BUF_FLUSH_N_TYPES);
- ut_ad(!mutex_own(&buf_pool->LRU_list_mutex));
+ /* Hold the LRU list mutex iff called for a single page LRU
+ flush. A single page LRU flush is already non-performant, and holding
+ the LRU list mutex allows us to avoid having to store the previous LRU
+ list page or to restart the LRU scan in
+ buf_flush_single_page_from_LRU(). */
+ ut_ad(flush_type == BUF_FLUSH_SINGLE_PAGE ||
+ !mutex_own(&buf_pool->LRU_list_mutex));
+ ut_ad(flush_type != BUF_FLUSH_SINGLE_PAGE ||
+ mutex_own(&buf_pool->LRU_list_mutex));
ut_ad(buf_page_in_file(bpage));
ut_ad(!sync || flush_type == BUF_FLUSH_SINGLE_PAGE);
- block_mutex = buf_page_get_mutex(bpage);
+ ib_mutex_t* block_mutex = buf_page_get_mutex(bpage);
+
ut_ad(mutex_own(block_mutex));
ut_ad(buf_flush_ready_for_flush(bpage, flush_type));
- mutex_enter(&buf_pool->flush_state_mutex);
-
- buf_page_set_io_fix(bpage, BUF_IO_WRITE);
+ bool is_uncompressed;
- buf_page_set_flush_type(bpage, flush_type);
-
- if (buf_pool->n_flush[flush_type] == 0) {
+ is_uncompressed = (buf_page_get_state(bpage) == BUF_BLOCK_FILE_PAGE);
+ ut_ad(is_uncompressed == (block_mutex != &buf_pool->zip_mutex));
- os_event_reset(buf_pool->no_flush[flush_type]);
- }
+ ibool flush;
+ rw_lock_t* rw_lock;
+ bool no_fix_count = bpage->buf_fix_count == 0;
- buf_pool->n_flush[flush_type]++;
+ if (!is_uncompressed) {
+ flush = TRUE;
+ rw_lock = NULL;
- mutex_exit(&buf_pool->flush_state_mutex);
+ } else if (!(no_fix_count || flush_type == BUF_FLUSH_LIST)) {
+ /* This is a heuristic, to avoid expensive S attempts. */
+ flush = FALSE;
+ } else {
- is_uncompressed = (buf_page_get_state(bpage) == BUF_BLOCK_FILE_PAGE);
- ut_ad(is_uncompressed == (block_mutex != &buf_pool->zip_mutex));
+ rw_lock = &reinterpret_cast<buf_block_t*>(bpage)->lock;
- switch (flush_type) {
- ibool is_s_latched;
- case BUF_FLUSH_LIST:
- /* If the simulated aio thread is not running, we must
- not wait for any latch, as we may end up in a deadlock:
- if buf_fix_count == 0, then we know we need not wait */
-
- is_s_latched = (bpage->buf_fix_count == 0);
- if (is_s_latched && is_uncompressed) {
- rw_lock_s_lock_gen(&((buf_block_t*) bpage)->lock,
- BUF_IO_WRITE);
+ if (flush_type != BUF_FLUSH_LIST) {
+ flush = rw_lock_s_lock_gen_nowait(
+ rw_lock, BUF_IO_WRITE);
+ } else {
+ /* Will S lock later */
+ flush = TRUE;
}
+ }
- mutex_exit(block_mutex);
+ if (flush) {
- /* Even though bpage is not protected by any mutex at
- this point, it is safe to access bpage, because it is
- io_fixed and oldest_modification != 0. Thus, it
- cannot be relocated in the buffer pool or removed from
- flush_list or LRU_list. */
+ /* We are committed to flushing by the time we get here */
- if (!is_s_latched) {
- buf_dblwr_flush_buffered_writes();
+ mutex_enter(&buf_pool->flush_state_mutex);
- if (is_uncompressed) {
- rw_lock_s_lock_gen(&((buf_block_t*) bpage)
- ->lock, BUF_IO_WRITE);
- }
- }
+ buf_page_set_io_fix(bpage, BUF_IO_WRITE);
- break;
+ buf_page_set_flush_type(bpage, flush_type);
- case BUF_FLUSH_LRU:
- case BUF_FLUSH_SINGLE_PAGE:
- /* VERY IMPORTANT:
- Because any thread may call single page flush, even when
- owning locks on pages, to avoid deadlocks, we must make
- sure that the s-lock is acquired on the page without
- waiting: this is accomplished because
- buf_flush_ready_for_flush() must hold, and that requires
- the page not to be bufferfixed.
- The same holds true for LRU flush because a user thread
- may end up waiting for an LRU flush to end while
- holding locks on other pages. */
-
- if (is_uncompressed) {
- rw_lock_s_lock_gen(&((buf_block_t*) bpage)->lock,
- BUF_IO_WRITE);
+ if (buf_pool->n_flush[flush_type] == 0) {
+
+ os_event_reset(buf_pool->no_flush[flush_type]);
}
- /* Note that the s-latch is acquired before releasing the
- buf_page_get_mutex() mutex: this ensures that the latch is
- acquired immediately. */
+ ++buf_pool->n_flush[flush_type];
+
+ mutex_exit(&buf_pool->flush_state_mutex);
mutex_exit(block_mutex);
- break;
- default:
- ut_error;
- }
+ if (flush_type == BUF_FLUSH_SINGLE_PAGE)
+ mutex_exit(&buf_pool->LRU_list_mutex);
+
+ if (flush_type == BUF_FLUSH_LIST
+ && is_uncompressed
+ && !rw_lock_s_lock_gen_nowait(rw_lock, BUF_IO_WRITE)) {
+ /* avoiding deadlock possibility involves doublewrite
+ buffer, should flush it, because it might hold the
+ another block->lock. */
+ buf_dblwr_flush_buffered_writes();
- /* Even though bpage is not protected by any mutex at this
- point, it is safe to access bpage, because it is io_fixed and
- oldest_modification != 0. Thus, it cannot be relocated in the
- buffer pool or removed from flush_list or LRU_list. */
+ rw_lock_s_lock_gen(rw_lock, BUF_IO_WRITE);
+ }
-#ifdef UNIV_DEBUG
- if (buf_debug_prints) {
- fprintf(stderr,
- "Flushing %u space %u page %u\n",
- flush_type, bpage->space, bpage->offset);
- }
-#endif /* UNIV_DEBUG */
- buf_flush_write_block_low(bpage, flush_type, sync);
+ /* Even though bpage is not protected by any mutex at this
+ point, it is safe to access bpage, because it is io_fixed and
+ oldest_modification != 0. Thus, it cannot be relocated in the
+ buffer pool or removed from flush_list or LRU_list. */
+
+ buf_flush_write_block_low(bpage, flush_type, sync);
+ }
+
+ return(flush);
}
# if defined UNIV_DEBUG || defined UNIV_IBUF_DEBUG
@@ -1115,15 +1104,16 @@ buf_flush_page_try(
{
ut_ad(buf_block_get_state(block) == BUF_BLOCK_FILE_PAGE);
ut_ad(mutex_own(&block->mutex));
+ ut_ad(mutex_own(&buf_pool->LRU_list_mutex));
if (!buf_flush_ready_for_flush(&block->page, BUF_FLUSH_SINGLE_PAGE)) {
return(FALSE);
}
- /* The following call will release the buffer pool and
- block mutex. */
- buf_flush_page(buf_pool, &block->page, BUF_FLUSH_SINGLE_PAGE, true);
- return(TRUE);
+ /* The following call will release the LRU list and
+ block mutex if successful. */
+ return(buf_flush_page(
+ buf_pool, &block->page, BUF_FLUSH_SINGLE_PAGE, true));
}
# endif /* UNIV_DEBUG || UNIV_IBUF_DEBUG */
/***********************************************************//**
@@ -1199,7 +1189,6 @@ buf_flush_try_neighbors(
ulint i;
ulint low;
ulint high;
- ulint count = 0;
buf_pool_t* buf_pool = buf_pool_get(space, offset);
ut_ad(flush_type == BUF_FLUSH_LRU || flush_type == BUF_FLUSH_LIST);
@@ -1257,9 +1246,10 @@ buf_flush_try_neighbors(
high = fil_space_get_size(space);
}
+ ulint count = 0;
+
for (i = low; i < high; i++) {
- buf_page_t* bpage;
prio_rw_lock_t* hash_lock;
ib_mutex_t* block_mutex;
@@ -1281,10 +1271,10 @@ buf_flush_try_neighbors(
buf_pool = buf_pool_get(space, i);
/* We only want to flush pages from this buffer pool. */
- bpage = buf_page_hash_get_s_locked(buf_pool, space, i,
- &hash_lock);
+ buf_page_t* bpage = buf_page_hash_get_s_locked(buf_pool,
+ space, i, &hash_lock);
- if (!bpage) {
+ if (bpage == NULL) {
continue;
}
@@ -1305,19 +1295,12 @@ buf_flush_try_neighbors(
|| buf_page_is_old(bpage)) {
if (buf_flush_ready_for_flush(bpage, flush_type)
- && (i == offset || !bpage->buf_fix_count)) {
- /* We only try to flush those
- neighbors != offset where the buf fix
- count is zero, as we then know that we
- probably can latch the page without a
- semaphore wait. Semaphore waits are
- expensive because we must flush the
- doublewrite buffer before we start
- waiting. */
-
- buf_flush_page(buf_pool, bpage, flush_type, false);
- ut_ad(!mutex_own(block_mutex));
- count++;
+ && (i == offset || bpage->buf_fix_count == 0)
+ && buf_flush_page(
+ buf_pool, bpage, flush_type, false)) {
+
+ ++count;
+
continue;
}
}
@@ -1358,8 +1341,8 @@ buf_flush_page_and_try_neighbors(
ulint* count) /*!< in/out: number of pages
flushed */
{
+ ibool flushed;
ib_mutex_t* block_mutex = NULL;
- ibool flushed = FALSE;
#ifdef UNIV_DEBUG
buf_pool_t* buf_pool = buf_pool_from_bpage(bpage);
#endif /* UNIV_DEBUG */
@@ -1374,21 +1357,10 @@ buf_flush_page_and_try_neighbors(
mutex_enter(block_mutex);
}
- if (UNIV_UNLIKELY(buf_page_get_state(bpage)
- == BUF_BLOCK_REMOVE_HASH)) {
-
- /* In case we don't hold the LRU list mutex, we may see a page
- that is about to be relocated on the flush list. Do not
- attempt to flush it. */
- ut_ad(flush_type == BUF_FLUSH_LIST);
- return (flushed);
- }
-
- ut_a(buf_page_in_file(bpage));
+ ut_a(buf_page_in_file(bpage)
+ || buf_page_get_state(bpage) == BUF_BLOCK_REMOVE_HASH);
if (buf_flush_ready_for_flush(bpage, flush_type)) {
- ulint space;
- ulint offset;
buf_pool_t* buf_pool;
buf_pool = buf_pool_from_bpage(bpage);
@@ -1399,8 +1371,10 @@ buf_flush_page_and_try_neighbors(
/* These fields are protected by the buf_page_get_mutex()
mutex. */
- space = buf_page_get_space(bpage);
- offset = buf_page_get_page_no(bpage);
+ /* Read the fields directly in order to avoid asserting on
+ BUF_BLOCK_REMOVE_HASH pages. */
+ ulint space = bpage->space;
+ ulint offset = bpage->offset;
if (flush_type == BUF_FLUSH_LRU) {
mutex_exit(block_mutex);
@@ -1409,11 +1383,8 @@ buf_flush_page_and_try_neighbors(
}
/* Try to flush also all the neighbors */
- *count += buf_flush_try_neighbors(space,
- offset,
- flush_type,
- *count,
- n_to_flush);
+ *count += buf_flush_try_neighbors(
+ space, offset, flush_type, *count, n_to_flush);
if (flush_type == BUF_FLUSH_LRU) {
mutex_enter(&buf_pool->LRU_list_mutex);
@@ -1421,8 +1392,12 @@ buf_flush_page_and_try_neighbors(
buf_flush_list_mutex_enter(buf_pool);
}
flushed = TRUE;
+
} else if (flush_type == BUF_FLUSH_LRU) {
mutex_exit(block_mutex);
+ flushed = FALSE;
+ } else {
+ flushed = FALSE;
}
ut_ad((flush_type == BUF_FLUSH_LRU
@@ -1574,6 +1549,7 @@ buf_flush_LRU_list_batch(
of the flushed pages then the scan becomes
O(n*n). */
if (evict) {
+
if (buf_LRU_free_page(bpage, true)) {
mutex_exit(block_mutex);
@@ -1588,19 +1564,42 @@ buf_flush_LRU_list_batch(
}
} else if (UNIV_LIKELY(!failed_acquire)) {
+ ulint space;
+ ulint offset;
+ buf_page_t* prev_bpage;
+
+ prev_bpage = UT_LIST_GET_PREV(LRU, bpage);
+
+ /* Save the previous bpage */
+
+ if (prev_bpage != NULL) {
+ space = prev_bpage->space;
+ offset = prev_bpage->offset;
+ } else {
+ space = ULINT_UNDEFINED;
+ offset = ULINT_UNDEFINED;
+ }
+
if (buf_flush_page_and_try_neighbors(
bpage,
BUF_FLUSH_LRU, max, &n->flushed)) {
- lru_position = 0;
-
/* LRU list mutex was released.
- Restart the scan. */
- bpage = UT_LIST_GET_LAST(buf_pool->LRU);
- } else {
+ reposition the iterator. Note: the
+ prev block could have been repositioned
+ too but that should be rare. */
- bpage = UT_LIST_GET_PREV(LRU, bpage);
+ if (prev_bpage != NULL) {
+
+ ut_ad(space != ULINT_UNDEFINED);
+ ut_ad(offset != ULINT_UNDEFINED);
+
+ prev_bpage = buf_page_hash_get(
+ buf_pool, space, offset);
+ }
}
+
+ bpage = prev_bpage;
}
free_len = UT_LIST_GET_LEN(buf_pool->free);
@@ -1912,7 +1911,7 @@ buf_flush_wait_batch_end(
}
} else {
thd_wait_begin(NULL, THD_WAIT_DISKIO);
- os_event_wait(buf_pool->no_flush[type]);
+ os_event_wait(buf_pool->no_flush[type]);
thd_wait_end(NULL);
}
}
@@ -2101,9 +2100,7 @@ buf_flush_single_page_from_LRU(
{
ulint scanned;
buf_page_t* bpage;
- ib_mutex_t* block_mutex;
- ibool freed;
- bool evict_zip;
+ ibool flushed = FALSE;
mutex_enter(&buf_pool->LRU_list_mutex);
@@ -2111,18 +2108,30 @@ buf_flush_single_page_from_LRU(
bpage != NULL;
bpage = UT_LIST_GET_PREV(LRU, bpage), ++scanned) {
- block_mutex = buf_page_get_mutex(bpage);
+ ib_mutex_t* block_mutex = buf_page_get_mutex(bpage);
+
mutex_enter(block_mutex);
- if (buf_flush_ready_for_flush(bpage,
- BUF_FLUSH_SINGLE_PAGE)) {
- /* buf_flush_page() will release the block
- mutex */
- break;
+
+ if (buf_flush_ready_for_flush(bpage, BUF_FLUSH_SINGLE_PAGE)) {
+
+ /* The following call will release the LRU list
+ and block mutex. */
+
+ flushed = buf_flush_page(buf_pool, bpage,
+ BUF_FLUSH_SINGLE_PAGE, true);
+
+ if (flushed) {
+ /* buf_flush_page() will release the
+ block mutex */
+ break;
+ }
}
+
mutex_exit(block_mutex);
}
- mutex_exit(&buf_pool->LRU_list_mutex);
+ if (!flushed)
+ mutex_exit(&buf_pool->LRU_list_mutex);
MONITOR_INC_VALUE_CUMULATIVE(
MONITOR_LRU_SINGLE_FLUSH_SCANNED,
@@ -2130,13 +2139,13 @@ buf_flush_single_page_from_LRU(
MONITOR_LRU_SINGLE_FLUSH_SCANNED_PER_CALL,
scanned);
- if (!bpage) {
+ if (bpage == NULL) {
/* Can't find a single flushable page. */
return(FALSE);
}
- /* The following call will release the buf_page_get_mutex() mutex. */
- buf_flush_page(buf_pool, bpage, BUF_FLUSH_SINGLE_PAGE, true);
+
+ ibool freed = FALSE;
/* At this point the page has been written to the disk.
As we are not holding LRU list or buf_page_get_mutex() mutex therefore
@@ -2151,30 +2160,30 @@ buf_flush_single_page_from_LRU(
bpage != NULL;
bpage = UT_LIST_GET_PREV(LRU, bpage)) {
- ibool ready;
+ ib_mutex_t* block_mutex = buf_page_get_mutex(bpage);
- block_mutex = buf_page_get_mutex(bpage);
mutex_enter(block_mutex);
- ready = buf_flush_ready_for_replace(bpage);
+
+ ibool ready = buf_flush_ready_for_replace(bpage);
+
if (ready) {
+ bool evict_zip;
+
+ evict_zip = !buf_LRU_evict_from_unzip_LRU(buf_pool);;
+
+ freed = buf_LRU_free_page(bpage, evict_zip);
+
+ mutex_exit(block_mutex);
+
break;
}
- mutex_exit(block_mutex);
- }
+ mutex_exit(block_mutex);
- if (!bpage) {
- /* Can't find a single replaceable page. */
- mutex_exit(&buf_pool->LRU_list_mutex);
- return(FALSE);
}
- evict_zip = !buf_LRU_evict_from_unzip_LRU(buf_pool);;
-
- freed = buf_LRU_free_page(bpage, evict_zip);
if (!freed)
mutex_exit(&buf_pool->LRU_list_mutex);
- mutex_exit(block_mutex);
return(freed);
}
@@ -2626,7 +2635,7 @@ page_cleaner_adapt_flush_sleep_time(void)
/******************************************************************//**
page_cleaner thread tasked with flushing dirty pages from the buffer
-pools. As of now we'll have only one instance of this thread.
+pool flush lists. As of now we'll have only one instance of this thread.
@return a dummy parameter */
extern "C" UNIV_INTERN
os_thread_ret_t
@@ -2639,7 +2648,7 @@ DECLARE_THREAD(buf_flush_page_cleaner_thread)(
ulint next_loop_time = ut_time_ms() + 1000;
ulint n_flushed = 0;
ulint last_activity = srv_get_activity_count();
- ulint lru_sleep_time = srv_cleaner_max_lru_time;
+ ulint last_activity_time = ut_time_ms();
ut_ad(!srv_read_only_mode);
@@ -2660,8 +2669,8 @@ DECLARE_THREAD(buf_flush_page_cleaner_thread)(
while (srv_shutdown_state == SRV_SHUTDOWN_NONE) {
- ulint flush_sleep_time;
ulint page_cleaner_sleep_time;
+ ibool server_active;
srv_current_thread_priority = srv_cleaner_thread_priority;
@@ -2674,20 +2683,20 @@ DECLARE_THREAD(buf_flush_page_cleaner_thread)(
page_cleaner_sleep_if_needed(next_loop_time);
}
- page_cleaner_adapt_lru_sleep_time(&lru_sleep_time);
-
- flush_sleep_time = page_cleaner_adapt_flush_sleep_time();
-
- page_cleaner_sleep_time = ut_min(lru_sleep_time,
- flush_sleep_time);
+ page_cleaner_sleep_time
+ = page_cleaner_adapt_flush_sleep_time();
next_loop_time = ut_time_ms() + page_cleaner_sleep_time;
- /* Flush pages from end of LRU if required */
- n_flushed = buf_flush_LRU_tail();
+ server_active = srv_check_activity(last_activity);
+ if (server_active
+ || ut_time_ms() - last_activity_time < 1000) {
- if (srv_check_activity(last_activity)) {
- last_activity = srv_get_activity_count();
+ if (server_active) {
+
+ last_activity = srv_get_activity_count();
+ last_activity_time = ut_time_ms();
+ }
/* Flush pages from flush_list if required */
n_flushed += page_cleaner_flush_pages_if_needed();
@@ -2778,6 +2787,74 @@ thread_exit:
OS_THREAD_DUMMY_RETURN;
}
+/******************************************************************//**
+lru_manager thread tasked with performing LRU flushes and evictions to refill
+the buffer pool free lists. As of now we'll have only one instance of this
+thread.
+@return a dummy parameter */
+extern "C" UNIV_INTERN
+os_thread_ret_t
+DECLARE_THREAD(buf_flush_lru_manager_thread)(
+/*==========================================*/
+ void* arg __attribute__((unused)))
+ /*!< in: a dummy parameter required by
+ os_thread_create */
+{
+ ulint next_loop_time = ut_time_ms() + 1000;
+ ulint lru_sleep_time = srv_cleaner_max_lru_time;
+
+#ifdef UNIV_PFS_THREAD
+ pfs_register_thread(buf_lru_manager_thread_key);
+#endif /* UNIV_PFS_THREAD */
+
+ srv_lru_manager_tid = os_thread_get_tid();
+
+ os_thread_set_priority(srv_lru_manager_tid,
+ srv_sched_priority_cleaner);
+
+#ifdef UNIV_DEBUG_THREAD_CREATION
+ fprintf(stderr, "InnoDB: lru_manager thread running, id %lu\n",
+ os_thread_pf(os_thread_get_curr_id()));
+#endif /* UNIV_DEBUG_THREAD_CREATION */
+
+ buf_lru_manager_is_active = true;
+
+ /* On server shutdown, the LRU manager thread runs through cleanup
+ phase to provide free pages for the master and purge threads. */
+ while (srv_shutdown_state == SRV_SHUTDOWN_NONE
+ || srv_shutdown_state == SRV_SHUTDOWN_CLEANUP) {
+
+ ulint n_flushed_lru;
+
+ srv_current_thread_priority = srv_cleaner_thread_priority;
+
+ page_cleaner_sleep_if_needed(next_loop_time);
+
+ page_cleaner_adapt_lru_sleep_time(&lru_sleep_time);
+
+ next_loop_time = ut_time_ms() + lru_sleep_time;
+
+ n_flushed_lru = buf_flush_LRU_tail();
+
+ if (n_flushed_lru) {
+
+ MONITOR_INC_VALUE_CUMULATIVE(
+ MONITOR_FLUSH_BACKGROUND_TOTAL_PAGE,
+ MONITOR_FLUSH_BACKGROUND_COUNT,
+ MONITOR_FLUSH_BACKGROUND_PAGES,
+ n_flushed_lru);
+ }
+ }
+
+ buf_lru_manager_is_active = false;
+
+ /* 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(NULL);
+
+ OS_THREAD_DUMMY_RETURN;
+}
+
#if defined UNIV_DEBUG || defined UNIV_BUF_DEBUG
/** Functor to validate the flush list. */