diff options
Diffstat (limited to 'storage/innobase/buf')
-rw-r--r-- | storage/innobase/buf/buf0buf.cc | 360 | ||||
-rw-r--r-- | storage/innobase/buf/buf0dblwr.cc | 11 | ||||
-rw-r--r-- | storage/innobase/buf/buf0dump.cc | 22 | ||||
-rw-r--r-- | storage/innobase/buf/buf0flu.cc | 144 | ||||
-rw-r--r-- | storage/innobase/buf/buf0lru.cc | 39 | ||||
-rw-r--r-- | storage/innobase/buf/buf0rea.cc | 6 |
6 files changed, 497 insertions, 85 deletions
diff --git a/storage/innobase/buf/buf0buf.cc b/storage/innobase/buf/buf0buf.cc index 8ef18ee0..23b5b776 100644 --- a/storage/innobase/buf/buf0buf.cc +++ b/storage/innobase/buf/buf0buf.cc @@ -404,7 +404,7 @@ static bool buf_page_decrypt_after_read(buf_page_t *bpage, if (id.space() == SRV_TMP_SPACE_ID && innodb_encrypt_temporary_tables) { - slot = buf_pool.io_buf_reserve(); + slot = buf_pool.io_buf_reserve(false); slot->allocate(); bool ok = buf_tmp_page_decrypt(slot->crypt_buf, dst_frame); slot->release(); @@ -426,7 +426,7 @@ decompress: return false; } - slot = buf_pool.io_buf_reserve(); + slot = buf_pool.io_buf_reserve(false); slot->allocate(); decompress_with_slot: @@ -449,7 +449,7 @@ decrypt_failed: return false; } - slot = buf_pool.io_buf_reserve(); + slot = buf_pool.io_buf_reserve(false); slot->allocate(); /* decrypt using crypt_buf to dst_frame */ @@ -742,6 +742,205 @@ bool buf_page_is_corrupted(bool check_lsn, const byte *read_buf, #ifndef UNIV_INNOCHECKSUM +#ifdef __linux__ +#include <poll.h> +#include <sys/eventfd.h> +#include <fstream> + +/** Memory Pressure + +based off https://www.kernel.org/doc/html/latest/accounting/psi.html#pressure-interface +and https://www.kernel.org/doc/html/latest/admin-guide/cgroup-v2.html#memory */ +class mem_pressure +{ + /* triggers + eventfd */ + struct pollfd m_fds[3]; + nfds_t m_num_fds; + int m_event_fd= -1; + Atomic_relaxed<bool> m_abort= false; + + std::thread m_thd; + /* mem pressure garbage collection restricted to interval */ + static constexpr ulonglong max_interval_us= 60*1000000; + +public: + mem_pressure() : m_num_fds(0) {} + + bool setup() + { + static_assert(array_elements(m_fds) == (array_elements(m_triggers) + 1), + "insufficient fds"); + std::string memcgroup{"/sys/fs/cgroup"}; + std::string cgroup; + { + std::ifstream selfcgroup("/proc/self/cgroup"); + std::getline(selfcgroup, cgroup, '\n'); + } + + cgroup.erase(0, 3); // Remove "0::" + memcgroup+= cgroup + "/memory.pressure"; + + m_num_fds= 0; + for (auto trig= std::begin(m_triggers); trig!= std::end(m_triggers); ++trig) + { + if ((m_fds[m_num_fds].fd= + open(memcgroup.c_str(), O_RDWR | O_NONBLOCK | O_CLOEXEC)) < 0) + { + /* User can't do anything about it, no point giving warning */ + shutdown(); + return false; + } + my_register_filename(m_fds[m_num_fds].fd, memcgroup.c_str(), FILE_BY_OPEN, 0, MYF(0)); + ssize_t slen= strlen(*trig); + if (write(m_fds[m_num_fds].fd, *trig, slen) < slen) + { + /* we may fail this one, but continue to the next */ + my_close(m_fds[m_num_fds].fd, MYF(MY_WME)); + continue; + } + m_fds[m_num_fds].events= POLLPRI; + m_num_fds++; + } + if (m_num_fds < 1) + return false; + + if ((m_event_fd= eventfd(0, EFD_CLOEXEC|EFD_NONBLOCK)) == -1) + { + /* User can't do anything about it, no point giving warning */ + shutdown(); + return false; + } + my_register_filename(m_event_fd, "mem_pressure_eventfd", FILE_BY_DUP, 0, MYF(0)); + m_fds[m_num_fds].fd= m_event_fd; + m_fds[m_num_fds].events= POLLIN; + m_num_fds++; + m_thd= std::thread(pressure_routine, this); + sql_print_information("InnoDB: Initialized memory pressure event listener"); + return true; + } + + void shutdown() + { + /* m_event_fd is in this list */ + while (m_num_fds) + { + m_num_fds--; + my_close(m_fds[m_num_fds].fd, MYF(MY_WME)); + m_fds[m_num_fds].fd= -1; + } + } + + static void pressure_routine(mem_pressure *m); + +#ifdef UNIV_DEBUG + void trigger_collection() + { + uint64_t u= 1; + if (m_event_fd >=0 && write(m_event_fd, &u, sizeof(uint64_t)) != sizeof(uint64_t)) + sql_print_information("InnoDB: (Debug) Failed to trigger memory pressure"); + else /* assumed failed to meet intialization criteria, so trigger directy */ + buf_pool.garbage_collect(); + } +#endif + + void quit() + { + uint64_t u= 1; + m_abort= true; +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-result" + /* return result ignored, cannot do anything with it */ + write(m_event_fd, &u, sizeof(uint64_t)); +#pragma GCC diagnostic pop + } + + void join() + { + if (m_thd.joinable()) + { + quit(); + m_thd.join(); + } + } + + static const char* const m_triggers[2]; +}; + + +/* + ref: https://docs.kernel.org/accounting/psi.html + maximum window size (second number) 10 seconds. + window size in multiples of 2 second interval required (for Unprivileged) + Time is in usec. +*/ +const char* const mem_pressure::m_triggers[]= + {"some 5000000 10000000", /* 5s out of 10s */ + "full 10000 2000000"}; /* 10ms out of 2s */ + +static mem_pressure mem_pressure_obj; + +void mem_pressure::pressure_routine(mem_pressure *m) +{ + DBUG_ASSERT(m == &mem_pressure_obj); + if (my_thread_init()) + { + m->shutdown(); + return; + } + + ulonglong last= microsecond_interval_timer() - max_interval_us; + while (!m->m_abort) + { + if (poll(&m->m_fds[0], m->m_num_fds, -1) < 0) + { + if (errno == EINTR) + continue; + else + break; + } + if (!m->m_abort) + break; + + for (pollfd &p : st_::span<pollfd>(m->m_fds, m->m_num_fds)) + { + if (p.revents & POLLPRI) + { + ulonglong now= microsecond_interval_timer(); + if ((now - last) > max_interval_us) + { + last= now; + buf_pool.garbage_collect(); + } + } + +#ifdef UNIV_DEBUG + if (p.revents & POLLIN) + { + uint64_t u; + /* we haven't aborted, so this must be a debug trigger */ + if (read(p.fd, &u, sizeof(u)) >=0) + buf_pool.garbage_collect(); + } +#endif + } + } + m->shutdown(); + + my_thread_end(); +} + +/** Initialize mem pressure. */ +ATTRIBUTE_COLD void buf_mem_pressure_detect_init() +{ + mem_pressure_obj.setup(); +} + +ATTRIBUTE_COLD void buf_mem_pressure_shutdown() +{ + mem_pressure_obj.join(); +} +#endif /* __linux__ */ + #if defined(DBUG_OFF) && defined(HAVE_MADVISE) && defined(MADV_DODUMP) /** Enable buffers to be dumped to core files @@ -1099,6 +1298,11 @@ bool buf_pool_t::create() chunk_t::map_ref= chunk_t::map_reg; buf_LRU_old_ratio_update(100 * 3 / 8, false); btr_search_sys_create(); + +#ifdef __linux__ + if (srv_operation == SRV_OPERATION_NORMAL) + buf_mem_pressure_detect_init(); +#endif ut_ad(is_initialised()); return false; } @@ -1300,14 +1504,17 @@ void buf_pool_t::io_buf_t::close() n_slots= 0; } -buf_tmp_buffer_t *buf_pool_t::io_buf_t::reserve() +buf_tmp_buffer_t *buf_pool_t::io_buf_t::reserve(bool wait_for_reads) { for (;;) { for (buf_tmp_buffer_t *s= slots, *e= slots + n_slots; s != e; s++) if (s->acquire()) return s; + buf_dblwr.flush_buffered_writes(); os_aio_wait_until_no_pending_writes(true); + if (!wait_for_reads) + continue; for (buf_tmp_buffer_t *s= slots, *e= slots + n_slots; s != e; s++) if (s->acquire()) return s; @@ -1536,6 +1743,7 @@ struct find_interesting_trx inline void buf_pool_t::resize() { ut_ad(this == &buf_pool); + ut_ad(srv_shutdown_state < SRV_SHUTDOWN_CLEANUP); bool warning = false; @@ -1878,6 +2086,100 @@ calc_buf_pool_size: return; } +#ifdef __linux__ +inline void buf_pool_t::garbage_collect() +{ + mysql_mutex_lock(&mutex); + size_t freed= 0; + +#ifdef BTR_CUR_HASH_ADAPT + /* buf_LRU_free_page() will temporarily release and reacquire + buf_pool.mutex for invoking btr_search_drop_page_hash_index(). Thus, + we must protect ourselves with the hazard pointer. */ +rescan: +#else + lru_hp.set(nullptr); +#endif + for (buf_page_t *bpage= UT_LIST_GET_LAST(LRU), *prev; bpage; bpage= prev) + { + prev= UT_LIST_GET_PREV(LRU, bpage); +#ifdef BTR_CUR_HASH_ADAPT + lru_hp.set(prev); +#endif + auto state= bpage->state(); + ut_ad(state >= buf_page_t::FREED); + ut_ad(bpage->in_LRU_list); + + /* We try to free any pages that can be freed without writing out + anything. */ + switch (bpage->oldest_modification()) { + case 0: + try_to_evict: + if (buf_LRU_free_page(bpage, true)) + { + evicted: + freed++; +#ifdef BTR_CUR_HASH_ADAPT + bpage= prev; + prev= lru_hp.get(); + if (!prev && bpage) + goto rescan; +#endif + } + continue; + case 1: + break; + default: + if (state >= buf_page_t::UNFIXED) + continue; + } + + if (state < buf_page_t::READ_FIX && bpage->lock.u_lock_try(true)) + { + ut_ad(!bpage->is_io_fixed()); + lsn_t oldest_modification= bpage->oldest_modification(); + switch (oldest_modification) { + case 1: + mysql_mutex_lock(&flush_list_mutex); + oldest_modification= bpage->oldest_modification(); + if (oldest_modification) + { + ut_ad(oldest_modification == 1); + delete_from_flush_list(bpage); + } + mysql_mutex_unlock(&flush_list_mutex); + /* fall through */ + case 0: + bpage->lock.u_unlock(true); + goto try_to_evict; + default: + if (bpage->state() < buf_page_t::UNFIXED && + oldest_modification <= log_sys.get_flushed_lsn()) + { + release_freed_page(bpage); + goto evicted; + } + else + bpage->lock.u_unlock(true); + } + } + } + +#if defined MADV_FREE + /* FIXME: Issue fewer calls for larger contiguous blocks of + memory. For now, we assume that this is acceptable, because this + code should be executed rarely. */ + for (buf_page_t *bpage= UT_LIST_GET_FIRST(free); bpage; + bpage= UT_LIST_GET_NEXT(list, bpage)) + madvise(bpage->frame, srv_page_size, MADV_FREE); +#endif + mysql_mutex_unlock(&mutex); + sql_print_information("InnoDB: Memory pressure event freed %zu pages", + freed); + return; +} +#endif /* __linux__ */ + /** Thread pool task invoked by innodb_buffer_pool_size changes. */ static void buf_resize_callback(void *) { @@ -1906,12 +2208,23 @@ static tpool::waitable_task buf_resize_task(buf_resize_callback, void buf_resize_start() { - srv_thread_pool->submit_task(&buf_resize_task); +#if !defined(DBUG_OFF) && defined(__linux__) + DBUG_EXECUTE_IF("trigger_garbage_collection", + { + mem_pressure_obj.trigger_collection(); + } + ); +#endif + + srv_thread_pool->submit_task(&buf_resize_task); } void buf_resize_shutdown() { - buf_resize_task.wait(); +#ifdef __linux__ + buf_mem_pressure_shutdown(); +#endif + buf_resize_task.wait(); } @@ -2220,14 +2533,21 @@ lookup: if (discard_attempted || !bpage->frame) { - /* Even when we are holding a hash_lock, it should be - acceptable to wait for a page S-latch here, because - buf_page_t::read_complete() will not wait for buf_pool.mutex, - and because S-latch would not conflict with a U-latch - that would be protecting buf_page_t::write_complete(). */ - bpage->lock.s_lock(); + const bool got_s_latch= bpage->lock.s_lock_try(); hash_lock.unlock_shared(); - break; + if (UNIV_LIKELY(got_s_latch)) + break; + /* We may fail to acquire bpage->lock because + buf_page_t::read_complete() may be invoking + buf_pool_t::corrupted_evict() on this block, which it would + hold an exclusive latch on. + + Let us aqcuire and release buf_pool.mutex to ensure that any + buf_pool_t::corrupted_evict() will proceed before we reacquire + the hash_lock that it could be waiting for. */ + mysql_mutex_lock(&buf_pool.mutex); + mysql_mutex_unlock(&buf_pool.mutex); + goto lookup; } hash_lock.unlock_shared(); @@ -2246,7 +2566,6 @@ lookup: ut_ad(s < buf_page_t::READ_FIX || s >= buf_page_t::WRITE_FIX); } - bpage->set_accessed(); buf_page_make_young_if_needed(bpage); #ifdef UNIV_DEBUG @@ -2873,18 +3192,6 @@ get_latch_valid: ut_ad(page_id_t(page_get_space_id(block->page.frame), page_get_page_no(block->page.frame)) == page_id); - - if (mode == BUF_GET_POSSIBLY_FREED - || mode == BUF_PEEK_IF_IN_POOL) { - return block; - } - - const bool not_first_access{block->page.set_accessed()}; - buf_page_make_young_if_needed(&block->page); - if (!not_first_access) { - buf_read_ahead_linear(page_id, block->zip_size(), - ibuf_inside(mtr)); - } } return block; @@ -3057,7 +3364,6 @@ bool buf_page_optimistic_get(ulint rw_latch, buf_block_t *block, block->page.fix(); ut_ad(!block->page.is_read_fixed()); - block->page.set_accessed(); buf_page_make_young_if_needed(&block->page); mtr->memo_push(block, mtr_memo_type_t(rw_latch)); } diff --git a/storage/innobase/buf/buf0dblwr.cc b/storage/innobase/buf/buf0dblwr.cc index e9aea355..e2702adc 100644 --- a/storage/innobase/buf/buf0dblwr.cc +++ b/storage/innobase/buf/buf0dblwr.cc @@ -336,11 +336,14 @@ func_exit: os_file_flush(file); } else - for (ulint i= 0; i < size * 2; i++, page += srv_page_size) - if (mach_read_from_8(my_assume_aligned<8>(page + FIL_PAGE_LSN))) - /* Each valid page header must contain a nonzero FIL_PAGE_LSN field. */ + { + alignas(8) char checkpoint[8]; + mach_write_to_8(checkpoint, log_sys.next_checkpoint_lsn); + for (auto i= size * 2; i--; page += srv_page_size) + if (memcmp_aligned<8>(page + FIL_PAGE_LSN, checkpoint, 8) >= 0) + /* Valid pages are not older than the log checkpoint. */ recv_sys.dblwr.add(page); - + } err= DB_SUCCESS; goto func_exit; } diff --git a/storage/innobase/buf/buf0dump.cc b/storage/innobase/buf/buf0dump.cc index 957632db..cc51f8c6 100644 --- a/storage/innobase/buf/buf0dump.cc +++ b/storage/innobase/buf/buf0dump.cc @@ -33,7 +33,7 @@ Created April 08, 2011 Vasil Dimov #include "buf0rea.h" #include "buf0dump.h" -#include "dict0dict.h" +#include "dict0load.h" #include "os0file.h" #include "srv0srv.h" #include "srv0start.h" @@ -180,7 +180,7 @@ static void buf_dump_generate_path(char *path, size_t path_size) char buf[FN_REFLEN]; mysql_mutex_lock(&LOCK_global_system_variables); - snprintf(buf, sizeof buf, "%s/%s", get_buf_dump_dir(), + snprintf(buf, sizeof buf, "%s" FN_ROOTDIR "%s", get_buf_dump_dir(), srv_buf_dump_filename); mysql_mutex_unlock(&LOCK_global_system_variables); @@ -214,7 +214,7 @@ static void buf_dump_generate_path(char *path, size_t path_size) format = "%s%s"; break; default: - format = "%s/%s"; + format = "%s" FN_ROOTDIR "%s"; } snprintf(path, path_size, format, @@ -562,6 +562,22 @@ buf_load() if (!SHUTTING_DOWN()) { std::sort(dump, dump + dump_n); + std::set<uint32_t> missing; + for (const page_id_t id : st_::span<const page_id_t> + (dump, dump_n)) { + missing.emplace(id.space()); + } + for (std::set<uint32_t>::iterator i = missing.begin(); + i != missing.end(); ) { + auto j = i++; + if (fil_space_t* space = fil_space_t::get(*j)) { + space->release(); + missing.erase(j); + } + } + if (!missing.empty()) { + dict_check_tablespaces_and_store_max_id(&missing); + } } /* Avoid calling the expensive fil_space_t::get() for each diff --git a/storage/innobase/buf/buf0flu.cc b/storage/innobase/buf/buf0flu.cc index b6357989..d4628985 100644 --- a/storage/innobase/buf/buf0flu.cc +++ b/storage/innobase/buf/buf0flu.cc @@ -655,7 +655,7 @@ static byte *buf_page_encrypt(fil_space_t* space, buf_page_t* bpage, byte* s, ut_ad(!bpage->zip_size() || !page_compressed); /* Find free slot from temporary memory array */ - *slot= buf_pool.io_buf_reserve(); + *slot= buf_pool.io_buf_reserve(true); ut_a(*slot); (*slot)->allocate(); @@ -754,16 +754,20 @@ bool buf_page_t::flush(bool evict, fil_space_t *space) ut_ad(space->referenced()); const auto s= state(); - ut_a(s >= FREED); + + const lsn_t lsn= + mach_read_from_8(my_assume_aligned<8> + (FIL_PAGE_LSN + (zip.data ? zip.data : frame))); + ut_ad(lsn + ? lsn >= oldest_modification() || oldest_modification() == 2 + : space->purpose != FIL_TYPE_TABLESPACE); if (s < UNFIXED) { + ut_a(s >= FREED); if (UNIV_LIKELY(space->purpose == FIL_TYPE_TABLESPACE)) { - const lsn_t lsn= - mach_read_from_8(my_assume_aligned<8> - (FIL_PAGE_LSN + (zip.data ? zip.data : frame))); - ut_ad(lsn >= oldest_modification()); + freed: if (lsn > log_sys.get_flushed_lsn()) { mysql_mutex_unlock(&buf_pool.mutex); @@ -775,6 +779,12 @@ bool buf_page_t::flush(bool evict, fil_space_t *space) return false; } + if (UNIV_UNLIKELY(lsn < space->get_create_lsn())) + { + ut_ad(space->purpose == FIL_TYPE_TABLESPACE); + goto freed; + } + ut_d(const auto f=) zip.fix.fetch_add(WRITE_FIX - UNFIXED); ut_ad(f >= UNFIXED); ut_ad(f < READ_FIX); @@ -869,15 +879,9 @@ bool buf_page_t::flush(bool evict, fil_space_t *space) if ((s & LRU_MASK) == REINIT || !space->use_doublewrite()) { - if (UNIV_LIKELY(space->purpose == FIL_TYPE_TABLESPACE)) - { - const lsn_t lsn= - mach_read_from_8(my_assume_aligned<8>(FIL_PAGE_LSN + - (write_frame ? write_frame - : frame))); - ut_ad(lsn >= oldest_modification()); + if (UNIV_LIKELY(space->purpose == FIL_TYPE_TABLESPACE) && + lsn > log_sys.get_flushed_lsn()) log_write_up_to(lsn, true); - } space->io(IORequest{type, this, slot}, physical_offset(), size, write_frame, this); } @@ -1057,11 +1061,25 @@ static ulint buf_flush_try_neighbors(fil_space_t *space, bool contiguous, bool evict, ulint n_flushed, ulint n_to_flush) { - mysql_mutex_unlock(&buf_pool.mutex); - ut_ad(space->id == page_id.space()); ut_ad(bpage->id() == page_id); + { + const lsn_t lsn= + mach_read_from_8(my_assume_aligned<8> + (FIL_PAGE_LSN + + (bpage->zip.data ? bpage->zip.data : bpage->frame))); + ut_ad(lsn >= bpage->oldest_modification()); + if (UNIV_UNLIKELY(lsn < space->get_create_lsn())) + { + ut_a(!bpage->flush(evict, space)); + mysql_mutex_unlock(&buf_pool.mutex); + return 0; + } + } + + mysql_mutex_unlock(&buf_pool.mutex); + ulint count= 0; page_id_t id= page_id; page_id_t high= buf_flush_check_neighbors(*space, id, contiguous, evict); @@ -1741,6 +1759,28 @@ ulint buf_flush_LRU(ulint max_n, bool evict) buf_pool.try_LRU_scan= true; pthread_cond_broadcast(&buf_pool.done_free); } + else if (!pages && !buf_pool.try_LRU_scan && + !buf_pool.LRU_warned.test_and_set(std::memory_order_acquire)) + { + /* For example, with the minimum innodb_buffer_pool_size=5M and + the default innodb_page_size=16k there are only a little over 316 + pages in the buffer pool. The buffer pool can easily be exhausted + by a workload of some dozen concurrent connections. The system could + reach a deadlock like the following: + + (1) Many threads are waiting in buf_LRU_get_free_block() + for buf_pool.done_free. + (2) Some threads are waiting for a page latch which is held by + another thread that is waiting in buf_LRU_get_free_block(). + (3) This thread is the only one that could make progress, but + we fail to do so because all the pages that we scanned are + buffer-fixed or latched by some thread. */ + sql_print_warning("InnoDB: Could not free any blocks in the buffer pool!" + " %zu blocks are in use and %zu free." + " Consider increasing innodb_buffer_pool_size.", + UT_LIST_GET_LEN(buf_pool.LRU), + UT_LIST_GET_LEN(buf_pool.free)); + } return pages; } @@ -2124,6 +2164,8 @@ ATTRIBUTE_COLD void buf_flush_ahead(lsn_t lsn, bool furious) limit= lsn; buf_pool.page_cleaner_set_idle(false); pthread_cond_signal(&buf_pool.do_flush_list); + if (furious) + log_sys.set_check_for_checkpoint(); } mysql_mutex_unlock(&buf_pool.flush_list_mutex); } @@ -2371,11 +2413,19 @@ func_exit: goto func_exit; } +TPOOL_SUPPRESS_TSAN +bool buf_pool_t::need_LRU_eviction() const +{ + /* try_LRU_scan==false means that buf_LRU_get_free_block() is waiting + for buf_flush_page_cleaner() to evict some blocks */ + return UNIV_UNLIKELY(!try_LRU_scan || + (UT_LIST_GET_LEN(LRU) > BUF_LRU_MIN_LEN && + UT_LIST_GET_LEN(free) < srv_LRU_scan_depth / 2)); +} + #if defined __aarch64__&&defined __GNUC__&&__GNUC__==4&&!defined __clang__ -/* Avoid GCC 4.8.5 internal compiler error "could not split insn". -We would only need this for buf_flush_page_cleaner(), -but GCC 4.8.5 does not support pop_options. */ -# pragma GCC optimize ("O0") +/* Avoid GCC 4.8.5 internal compiler error "could not split insn". */ +__attribute__((optimize(0))) #endif /** page_cleaner thread tasked with flushing dirty pages from the buffer pools. As of now we'll have only one coordinator. */ @@ -2409,21 +2459,24 @@ static void buf_flush_page_cleaner() } mysql_mutex_lock(&buf_pool.flush_list_mutex); - if (buf_pool.ran_out()) - goto no_wait; - else if (srv_shutdown_state > SRV_SHUTDOWN_INITIATED) - break; + if (!buf_pool.need_LRU_eviction()) + { + if (srv_shutdown_state > SRV_SHUTDOWN_INITIATED) + break; - if (buf_pool.page_cleaner_idle() && - (!UT_LIST_GET_LEN(buf_pool.flush_list) || - srv_max_dirty_pages_pct_lwm == 0.0)) - /* We are idle; wait for buf_pool.page_cleaner_wakeup() */ - my_cond_wait(&buf_pool.do_flush_list, - &buf_pool.flush_list_mutex.m_mutex); - else - my_cond_timedwait(&buf_pool.do_flush_list, - &buf_pool.flush_list_mutex.m_mutex, &abstime); - no_wait: + if (buf_pool.page_cleaner_idle() && + (!UT_LIST_GET_LEN(buf_pool.flush_list) || + srv_max_dirty_pages_pct_lwm == 0.0)) + { + buf_pool.LRU_warned.clear(std::memory_order_release); + /* We are idle; wait for buf_pool.page_cleaner_wakeup() */ + my_cond_wait(&buf_pool.do_flush_list, + &buf_pool.flush_list_mutex.m_mutex); + } + else + my_cond_timedwait(&buf_pool.do_flush_list, + &buf_pool.flush_list_mutex.m_mutex, &abstime); + } set_timespec(abstime, 1); lsn_limit= buf_flush_sync_lsn; @@ -2445,9 +2498,9 @@ static void buf_flush_page_cleaner() do { - DBUG_EXECUTE_IF("ib_log_checkpoint_avoid", continue;); - DBUG_EXECUTE_IF("ib_log_checkpoint_avoid_hard", continue;); - + IF_DBUG(if (_db_keyword_(nullptr, "ib_log_checkpoint_avoid", 1) || + _db_keyword_(nullptr, "ib_log_checkpoint_avoid_hard", 1)) + continue,); if (!recv_recovery_is_on() && !srv_startup_is_before_trx_rollback_phase && srv_operation <= SRV_OPERATION_EXPORT_RESTORED) @@ -2455,7 +2508,7 @@ static void buf_flush_page_cleaner() } while (false); - if (!buf_pool.ran_out()) + if (!buf_pool.need_LRU_eviction()) continue; mysql_mutex_lock(&buf_pool.flush_list_mutex); oldest_lsn= buf_pool.get_oldest_modification(0); @@ -2484,7 +2537,7 @@ static void buf_flush_page_cleaner() if (oldest_lsn >= soft_lsn_limit) buf_flush_async_lsn= soft_lsn_limit= 0; } - else if (buf_pool.ran_out()) + else if (buf_pool.need_LRU_eviction()) { buf_pool.page_cleaner_set_idle(false); buf_pool.n_flush_inc(); @@ -2549,10 +2602,11 @@ static void buf_flush_page_cleaner() else { maybe_unemployed: - const bool below{dirty_pct < pct_lwm}; - pct_lwm= 0.0; - if (below) + if (dirty_pct < pct_lwm) + { + pct_lwm= 0.0; goto possibly_unemployed; + } } } else if (dirty_pct < srv_max_buf_pool_modified_pct) @@ -2598,9 +2652,13 @@ static void buf_flush_page_cleaner() MONITOR_FLUSH_ADAPTIVE_PAGES, n_flushed); } - else if (buf_flush_async_lsn <= oldest_lsn) + else if (buf_flush_async_lsn <= oldest_lsn && + !buf_pool.need_LRU_eviction()) goto check_oldest_and_set_idle; + else + mysql_mutex_lock(&buf_pool.mutex); + n= srv_max_io_capacity; n= n >= n_flushed ? n - n_flushed : 0; goto LRU_flush; } diff --git a/storage/innobase/buf/buf0lru.cc b/storage/innobase/buf/buf0lru.cc index 65ee8fa3..2a8d6ff2 100644 --- a/storage/innobase/buf/buf0lru.cc +++ b/storage/innobase/buf/buf0lru.cc @@ -60,10 +60,6 @@ static constexpr ulint BUF_LRU_OLD_TOLERANCE = 20; frames in the buffer pool, we set this to TRUE */ static bool buf_lru_switched_on_innodb_mon = false; -/** True if diagnostic message about difficult to find free blocks -in the buffer bool has already printed. */ -static bool buf_lru_free_blocks_error_printed; - /******************************************************************//** These statistics are not 'of' LRU but 'for' LRU. We keep count of I/O and page_zip_decompress() operations. Based on the statistics, @@ -408,6 +404,7 @@ got_mutex: buf_LRU_check_size_of_non_data_objects(); buf_block_t* block; + IF_DBUG(static bool buf_lru_free_blocks_error_printed,); DBUG_EXECUTE_IF("ib_lru_force_no_free_page", if (!buf_lru_free_blocks_error_printed) { n_iterations = 21; @@ -417,9 +414,25 @@ retry: /* If there is a block in the free list, take it */ if ((block = buf_LRU_get_free_only()) != nullptr) { got_block: + const ulint LRU_size = UT_LIST_GET_LEN(buf_pool.LRU); + const ulint available = UT_LIST_GET_LEN(buf_pool.free); + const ulint scan_depth = srv_LRU_scan_depth / 2; + ut_ad(LRU_size <= BUF_LRU_MIN_LEN || available >= scan_depth + || buf_pool.need_LRU_eviction()); + if (!have_mutex) { mysql_mutex_unlock(&buf_pool.mutex); } + + if (UNIV_UNLIKELY(available < scan_depth) + && LRU_size > BUF_LRU_MIN_LEN) { + mysql_mutex_lock(&buf_pool.flush_list_mutex); + if (!buf_pool.page_cleaner_active()) { + buf_pool.page_cleaner_wakeup(true); + } + mysql_mutex_unlock(&buf_pool.flush_list_mutex); + } + block->page.zip.clear(); return block; } @@ -445,10 +458,11 @@ got_block: if ((block = buf_LRU_get_free_only()) != nullptr) { goto got_block; } + const bool wake = buf_pool.need_LRU_eviction(); mysql_mutex_unlock(&buf_pool.mutex); mysql_mutex_lock(&buf_pool.flush_list_mutex); const auto n_flush = buf_pool.n_flush(); - if (!buf_pool.try_LRU_scan) { + if (wake && !buf_pool.page_cleaner_active()) { buf_pool.page_cleaner_wakeup(true); } mysql_mutex_unlock(&buf_pool.flush_list_mutex); @@ -467,9 +481,10 @@ not_found: MONITOR_INC( MONITOR_LRU_GET_FREE_WAITS ); } - if (n_iterations == 21 && !buf_lru_free_blocks_error_printed - && srv_buf_pool_old_size == srv_buf_pool_size) { - buf_lru_free_blocks_error_printed = true; + if (n_iterations == 21 + && srv_buf_pool_old_size == srv_buf_pool_size + && buf_pool.LRU_warned.test_and_set(std::memory_order_acquire)) { + IF_DBUG(buf_lru_free_blocks_error_printed = true,); mysql_mutex_unlock(&buf_pool.mutex); ib::warn() << "Difficult to find free blocks in the buffer pool" " (" << n_iterations << " search iterations)! " @@ -787,6 +802,14 @@ void buf_page_make_young(buf_page_t *bpage) mysql_mutex_unlock(&buf_pool.mutex); } +bool buf_page_make_young_if_needed(buf_page_t *bpage) +{ + const bool not_first{bpage->set_accessed()}; + if (UNIV_UNLIKELY(buf_page_peek_if_too_old(bpage))) + buf_page_make_young(bpage); + return not_first; +} + /** Try to free a block. If bpage is a descriptor of a compressed-only ROW_FORMAT=COMPRESSED page, the buf_page_t object will be freed as well. The caller must hold buf_pool.mutex. diff --git a/storage/innobase/buf/buf0rea.cc b/storage/innobase/buf/buf0rea.cc index c4f07738..9041c6a2 100644 --- a/storage/innobase/buf/buf0rea.cc +++ b/storage/innobase/buf/buf0rea.cc @@ -597,6 +597,12 @@ failed: uint32_t prev= mach_read_from_4(my_assume_aligned<4>(f + FIL_PAGE_PREV)); uint32_t next= mach_read_from_4(my_assume_aligned<4>(f + FIL_PAGE_NEXT)); hash_lock.unlock_shared(); + /* The underlying file page of this buffer pool page could actually + be marked as freed, or a read of the page into the buffer pool might + be in progress. We may read uninitialized data here. + Suppress warnings of comparing uninitialized values. */ + MEM_MAKE_DEFINED(&prev, sizeof prev); + MEM_MAKE_DEFINED(&next, sizeof next); if (prev == FIL_NULL || next == FIL_NULL) goto fail; page_id_t id= page_id; |