summaryrefslogtreecommitdiff
path: root/storage/innobase/include/buf0buf.h
diff options
context:
space:
mode:
Diffstat (limited to 'storage/innobase/include/buf0buf.h')
-rw-r--r--storage/innobase/include/buf0buf.h249
1 files changed, 107 insertions, 142 deletions
diff --git a/storage/innobase/include/buf0buf.h b/storage/innobase/include/buf0buf.h
index 04d74b6e36f..df52d6b572c 100644
--- a/storage/innobase/include/buf0buf.h
+++ b/storage/innobase/include/buf0buf.h
@@ -39,7 +39,6 @@ Created 11/5/1995 Heikki Tuuri
#include "hash0hash.h"
#include "ut0byte.h"
#include "page0types.h"
-#include "ut0rbt.h"
#include "log0log.h"
#include "srv0srv.h"
#include <ostream>
@@ -105,10 +104,6 @@ struct buf_pool_info_t
ulint n_pend_reads; /*!< buf_pool.n_pend_reads, pages
pending read */
ulint n_pending_flush_lru; /*!< Pages pending flush in LRU */
- ulint n_pending_flush_single_page;/*!< Pages pending to be
- flushed as part of single page
- flushes issued by various user
- threads */
ulint n_pending_flush_list; /*!< Pages pending flush in FLUSH
LIST */
ulint n_pages_made_young; /*!< number of pages made young */
@@ -340,10 +335,11 @@ FILE_PAGE (the other is buf_page_get_gen).
@param[in] offset offset of the tablespace
@param[in] zip_size ROW_FORMAT=COMPRESSED page size, or 0
@param[in,out] mtr mini-transaction
+@param[in,out] free_block pre-allocated buffer block
@return pointer to the block, page bufferfixed */
buf_block_t*
buf_page_create(fil_space_t *space, uint32_t offset,
- ulint zip_size, mtr_t *mtr);
+ ulint zip_size, mtr_t *mtr, buf_block_t *free_block);
/********************************************************************//**
Releases a compressed-only page acquired with buf_page_get_zip(). */
@@ -818,15 +814,11 @@ public:
state() == BUF_BLOCK_NOT_USED: buf_pool.free or buf_pool.withdraw
- state() == BUF_BLOCK_FILE_PAGE ||
- (state() == BUF_BLOCK_ZIP_PAGE && !oldest_modification()):
+ in_file() && oldest_modification():
buf_pool.flush_list (protected by buf_pool.flush_list_mutex)
- state() == BUF_BLOCK_ZIP_PAGE && !oldest_modification(): buf_pool.zip_clean
-
- The contents is undefined if
- !oldest_modification() && state() == BUF_BLOCK_FILE_PAGE,
- or if state() is not any of the above. */
+ The contents is undefined if in_file() && !oldest_modification(),
+ or if state() is BUF_BLOCK_MEMORY or BUF_BLOCK_REMOVE_HASH. */
UT_LIST_NODE_T(buf_page_t) list;
private:
@@ -986,6 +978,15 @@ public:
return zip.ssize ? (UNIV_ZIP_SIZE_MIN >> 1) << zip.ssize : 0;
}
+ /** @return the byte offset of the page within a file */
+ os_offset_t physical_offset() const
+ {
+ os_offset_t o= id().page_no();
+ return zip.ssize
+ ? o << (zip.ssize + (UNIV_ZIP_SIZE_SHIFT_MIN - 1))
+ : o << srv_page_size_shift;
+ }
+
/** @return whether the block is mapped to a data file */
bool in_file() const
{
@@ -1043,7 +1044,7 @@ struct buf_block_t{
is of size srv_page_size, and
aligned to an address divisible by
srv_page_size */
- BPageLock lock; /*!< read-write lock of the buffer
+ rw_lock_t lock; /*!< read-write lock of the buffer
frame */
#ifdef UNIV_DEBUG
/** whether page.list is in buf_pool.withdraw
@@ -1219,13 +1220,13 @@ public:
virtual ~HazardPointer() {}
/** @return current value */
- buf_page_t *get() const { ut_ad(mutex_own(m_mutex)); return m_hp; }
+ buf_page_t *get() const { mysql_mutex_assert_owner(m_mutex); return m_hp; }
/** Set current value
@param bpage buffer block to be set as hp */
void set(buf_page_t *bpage)
{
- ut_ad(mutex_own(m_mutex));
+ mysql_mutex_assert_owner(m_mutex);
ut_ad(!bpage || bpage->in_file());
m_hp= bpage;
}
@@ -1234,7 +1235,7 @@ public:
@param bpage buffer block to be compared
@return true if it is hp */
bool is_hp(const buf_page_t *bpage) const
- { ut_ad(mutex_own(m_mutex)); return bpage == m_hp; }
+ { mysql_mutex_assert_owner(m_mutex); return bpage == m_hp; }
/** Adjust the value of hp. This happens when some
other thread working on the same list attempts to
@@ -1243,7 +1244,7 @@ public:
#ifdef UNIV_DEBUG
/** mutex that protects access to the m_hp. */
- const ib_mutex_t *m_mutex= nullptr;
+ const mysql_mutex_t *m_mutex= nullptr;
#endif /* UNIV_DEBUG */
protected:
@@ -1466,21 +1467,7 @@ public:
/** @return whether resize() is in progress */
bool resize_in_progress() const
{
- return UNIV_UNLIKELY(resizing.load(std::memory_order_relaxed) ||
- withdrawing.load(std::memory_order_relaxed));
- }
-
- /** @return the withdraw_clock */
- ulint withdraw_clock() const
- { return withdraw_clock_.load(std::memory_order_relaxed); }
-
- /** Verify the possibility that a stored page is not in buffer pool.
- @param withdraw_clock the withdraw clock of the page
- @return whether the page might be relocated */
- bool is_obsolete(ulint withdraw_clock) const
- {
- return UNIV_UNLIKELY(withdrawing.load(std::memory_order_relaxed) ||
- this->withdraw_clock() != withdraw_clock);
+ return UNIV_UNLIKELY(resizing.load(std::memory_order_relaxed));
}
/** @return the current size in blocks */
@@ -1499,7 +1486,10 @@ public:
bool will_be_withdrawn(const byte *ptr) const
{
ut_ad(curr_size < old_size);
- ut_ad(!resizing.load(std::memory_order_relaxed) || mutex_own(&mutex));
+#ifdef SAFE_MUTEX
+ if (resizing.load(std::memory_order_relaxed))
+ mysql_mutex_assert_owner(&mutex);
+#endif /* SAFE_MUTEX */
for (const chunk_t *chunk= chunks + n_chunks_new,
* const echunk= chunks + n_chunks;
@@ -1516,7 +1506,10 @@ public:
bool will_be_withdrawn(const buf_page_t &bpage) const
{
ut_ad(curr_size < old_size);
- ut_ad(!resizing.load(std::memory_order_relaxed) || mutex_own(&mutex));
+#ifdef SAFE_MUTEX
+ if (resizing.load(std::memory_order_relaxed))
+ mysql_mutex_assert_owner(&mutex);
+#endif /* SAFE_MUTEX */
for (const chunk_t *chunk= chunks + n_chunks_new,
* const echunk= chunks + n_chunks;
@@ -1529,7 +1522,10 @@ public:
/** Release and evict a corrupted page.
@param bpage page that was being read */
- void corrupted_evict(buf_page_t *bpage);
+ ATTRIBUTE_COLD void corrupted_evict(buf_page_t *bpage);
+
+ /** Release a memory block to the buffer pool. */
+ ATTRIBUTE_COLD void free_block(buf_block_t *block);
#ifdef UNIV_DEBUG
/** Find a block that points to a ROW_FORMAT=COMPRESSED page
@@ -1538,7 +1534,7 @@ public:
@retval nullptr if not found */
const buf_block_t *contains_zip(const void *data) const
{
- ut_ad(mutex_own(&mutex));
+ mysql_mutex_assert_owner(&mutex);
for (const chunk_t *chunk= chunks, * const end= chunks + n_chunks;
chunk != end; chunk++)
if (const buf_block_t *block= chunk->contains_zip(data))
@@ -1561,22 +1557,30 @@ public:
inline buf_block_t *block_from_ahi(const byte *ptr) const;
#endif /* BTR_CUR_HASH_ADAPT */
- bool is_block_lock(const BPageLock *l) const
- { return is_block_field(reinterpret_cast<const void*>(l)); }
+ bool is_block_lock(const rw_lock_t *l) const
+ { return is_block_field(static_cast<const void*>(l)); }
/**
@return the smallest oldest_modification lsn for any page
- @retval 0 if all modified persistent pages have been flushed */
- lsn_t get_oldest_modification();
+ @retval empty_lsn if all modified persistent pages have been flushed */
+ lsn_t get_oldest_modification(lsn_t empty_lsn)
+ {
+ mysql_mutex_assert_owner(&flush_list_mutex);
+ const buf_page_t *bpage= UT_LIST_GET_LAST(flush_list);
+#if 1 /* MDEV-12227 FIXME: remove this loop */
+ for (; bpage && fsp_is_system_temporary(bpage->id().space());
+ bpage= UT_LIST_GET_PREV(list, bpage))
+ ut_ad(bpage->oldest_modification());
+#endif
+ return bpage ? bpage->oldest_modification() : empty_lsn;
+ }
/** Determine if a buffer block was created by chunk_t::create().
@param block block descriptor (not dereferenced)
@return whether block has been created by chunk_t::create() */
bool is_uncompressed(const buf_block_t *block) const
{
- /* The pointer should be aligned. */
- return !(size_t(block) % sizeof *block) &&
- is_block_field(reinterpret_cast<const void*>(block));
+ return is_block_field(reinterpret_cast<const void*>(block));
}
/** Get the page_hash latch for a page */
@@ -1593,7 +1597,10 @@ public:
buf_page_t *page_hash_get_low(const page_id_t id, const ulint fold)
{
ut_ad(id.fold() == fold);
- ut_ad(mutex_own(&mutex) || page_hash.lock_get(fold)->is_locked());
+#ifdef SAFE_MUTEX
+ DBUG_ASSERT(mysql_mutex_is_owner(&mutex) ||
+ page_hash.lock_get(fold)->is_locked());
+#endif /* SAFE_MUTEX */
buf_page_t *bpage;
/* Look for the page in the hash table */
HASH_SEARCH(hash, &page_hash, fold, buf_page_t*, bpage,
@@ -1660,7 +1667,10 @@ public:
@return whether bpage a sentinel for a buffer pool watch */
bool watch_is_sentinel(const buf_page_t &bpage)
{
- ut_ad(mutex_own(&mutex) || hash_lock_get(bpage.id())->is_locked());
+#ifdef SAFE_MUTEX
+ DBUG_ASSERT(mysql_mutex_is_owner(&mutex) ||
+ hash_lock_get(bpage.id())->is_locked());
+#endif /* SAFE_MUTEX */
ut_ad(bpage.in_file());
if (&bpage < &watch[0] || &bpage >= &watch[UT_ARR_SIZE(watch)])
@@ -1717,7 +1727,7 @@ public:
HASH_DELETE(buf_page_t, hash, &page_hash, fold, watch);
hash_lock->write_unlock();
// Now that the watch is detached from page_hash, release it to watch[].
- mutex_enter(&mutex);
+ mysql_mutex_lock(&mutex);
/* It is possible that watch_remove() already removed the watch. */
if (watch->id_ == id)
{
@@ -1725,7 +1735,7 @@ public:
ut_ad(watch->state() == BUF_BLOCK_ZIP_PAGE);
watch->set_state(BUF_BLOCK_NOT_USED);
}
- mutex_exit(&mutex);
+ mysql_mutex_unlock(&mutex);
}
else
hash_lock->write_unlock();
@@ -1758,14 +1768,13 @@ public:
@return the predecessor in the LRU list */
buf_page_t *LRU_remove(buf_page_t *bpage)
{
- ut_ad(mutex_own(&mutex));
+ mysql_mutex_assert_owner(&mutex);
ut_ad(bpage->in_LRU_list);
ut_ad(bpage->in_page_hash);
ut_ad(!bpage->in_zip_hash);
ut_ad(bpage->in_file());
lru_hp.adjust(bpage);
lru_scan_itr.adjust(bpage);
- single_scan_itr.adjust(bpage);
ut_d(bpage->in_LRU_list= false);
buf_page_t *prev= UT_LIST_GET_PREV(LRU, bpage);
UT_LIST_REMOVE(LRU, bpage);
@@ -1775,9 +1784,19 @@ public:
/** Number of pages to read ahead */
static constexpr uint32_t READ_AHEAD_PAGES= 64;
+ /** Buffer pool mutex */
+ mysql_mutex_t mutex;
+ /** Number of pending LRU flush. */
+ Atomic_counter<ulint> n_flush_LRU;
+ /** broadcast when n_flush_LRU reaches 0; protected by mutex */
+ mysql_cond_t done_flush_LRU;
+ /** Number of pending flush_list flush. */
+ Atomic_counter<ulint> n_flush_list;
+ /** broadcast when n_flush_list reaches 0; protected by mutex */
+ mysql_cond_t done_flush_list;
+
/** @name General fields */
/* @{ */
- BufPoolMutex mutex; /*!< Buffer pool mutex */
ulint curr_pool_size; /*!< Current pool size in bytes */
ulint LRU_old_ratio; /*!< Reserve this much of the buffer
pool for "old" blocks */
@@ -1908,50 +1927,23 @@ public:
/* @} */
- /** @name Page flushing algorithm fields */
+ /** @name Page flushing algorithm fields */
+ /* @{ */
- /* @{ */
+ /** mutex protecting flush_list, buf_page_t::set_oldest_modification()
+ and buf_page_t::list pointers when !oldest_modification() */
+ mysql_mutex_t flush_list_mutex;
+ /** "hazard pointer" for flush_list scans; protected by flush_list_mutex */
+ FlushHp flush_hp;
+ /** modified blocks (a subset of LRU) */
+ UT_LIST_BASE_NODE_T(buf_page_t) flush_list;
+
+ /** signalled to wake up the page_cleaner; protected by flush_list_mutex */
+ mysql_cond_t do_flush_list;
+
+ // n_flush_LRU + n_flush_list is approximately COUNT(io_fix()==BUF_IO_WRITE)
+ // in flush_list
- FlushListMutex flush_list_mutex;/*!< mutex protecting the
- flush list access. This mutex
- protects flush_list, flush_rbt
- and bpage::list pointers when
- the bpage is on flush_list. It
- also protects writes to
- bpage::oldest_modification and
- flush_list_hp */
- FlushHp flush_hp;/*!< "hazard pointer"
- used during scan of flush_list
- while doing flush list batch.
- Protected by flush_list_mutex */
- UT_LIST_BASE_NODE_T(buf_page_t) flush_list;
- /*!< base node of the modified block
- list */
- /** set if a flush of the type is being initialized */
- Atomic_relaxed<bool> init_flush[3];
- /** Number of pending writes of a flush type.
- The sum of these is approximately the sum of BUF_IO_WRITE blocks. */
- Atomic_counter<ulint> n_flush[3];
- os_event_t no_flush[3];
- /*!< this is in the set state
- when there is no flush batch
- of the given type running;
- os_event_set() and os_event_reset()
- are protected by buf_pool_t::mutex */
- ib_rbt_t* flush_rbt; /*!< a red-black tree is used
- exclusively during recovery to
- speed up insertions in the
- flush_list. This tree contains
- blocks in order of
- oldest_modification LSN and is
- kept in sync with the
- flush_list.
- Each member of the tree MUST
- also be on the flush_list.
- This tree is relevant only in
- recovery and is set to NULL
- once the recovery is over.
- Protected by flush_list_mutex */
unsigned freed_page_clock;/*!< a sequence number used
to count the number of buffer
blocks removed from the end of
@@ -1997,10 +1989,6 @@ public:
replacable victim. Protected by buf_pool_t::mutex. */
LRUItr lru_scan_itr;
- /** Iterator used to scan the LRU list when searching for
- single page flushing victim. Protected by buf_pool_t::mutex. */
- LRUItr single_scan_itr;
-
UT_LIST_BASE_NODE_T(buf_page_t) LRU;
/*!< base node of the LRU list */
@@ -2024,22 +2012,11 @@ public:
unzip_LRU list */
/* @} */
- /** @name Buddy allocator fields
- The buddy allocator is used for allocating compressed page
- frames and buf_page_t descriptors of blocks that exist
- in the buffer pool only in compressed form. */
- /* @{ */
-#ifdef UNIV_DEBUG
- /** unmodified ROW_FORMAT=COMPRESSED pages;
- protected by buf_pool.mutex */
- UT_LIST_BASE_NODE_T(buf_page_t) zip_clean;
-#endif /* UNIV_DEBUG */
- UT_LIST_BASE_NODE_T(buf_buddy_free_t) zip_free[BUF_BUDDY_SIZES_MAX];
- /*!< buddy free lists */
+ /** free ROW_FORMAT=COMPRESSED page frames */
+ UT_LIST_BASE_NODE_T(buf_buddy_free_t) zip_free[BUF_BUDDY_SIZES_MAX];
#if BUF_BUDDY_LOW > UNIV_ZIP_SIZE_MIN
# error "BUF_BUDDY_LOW > UNIV_ZIP_SIZE_MIN"
#endif
- /* @} */
/** Sentinels to detect if pages are read into the buffer pool while
a delete-buffering operation is pending. Protected by mutex. */
@@ -2050,16 +2027,12 @@ public:
/** @return whether any I/O is pending */
bool any_io_pending() const
{
- return n_pend_reads ||
- n_flush[IORequest::LRU] || n_flush[IORequest::FLUSH_LIST] ||
- n_flush[IORequest::SINGLE_PAGE];
+ return n_pend_reads || n_flush_LRU || n_flush_list;
}
/** @return total amount of pending I/O */
ulint io_pending() const
{
- return n_pend_reads +
- n_flush[IORequest::LRU] + n_flush[IORequest::FLUSH_LIST] +
- n_flush[IORequest::SINGLE_PAGE];
+ return n_pend_reads + n_flush_LRU + n_flush_list;
}
private:
/** Temporary memory for page_compressed and encrypted I/O */
@@ -2102,13 +2075,6 @@ private:
/** whether resize() is in the critical path */
std::atomic<bool> resizing;
-
- /** whether withdrawing buffer pool pages might cause page relocation */
- std::atomic<bool> withdrawing;
-
- /** a counter that is incremented every time a pointer to a page may
- become obsolete */
- std::atomic<ulint> withdraw_clock_;
};
/** The InnoDB buffer pool */
@@ -2116,7 +2082,7 @@ extern buf_pool_t buf_pool;
inline void page_hash_latch::read_lock()
{
- ut_ad(!mutex_own(&buf_pool.mutex));
+ mysql_mutex_assert_not_owner(&buf_pool.mutex);
if (!read_trylock())
read_lock_wait();
}
@@ -2129,19 +2095,19 @@ inline void page_hash_latch::write_lock()
inline void buf_page_t::add_buf_fix_count(uint32_t count)
{
- ut_ad(mutex_own(&buf_pool.mutex));
+ mysql_mutex_assert_owner(&buf_pool.mutex);
buf_fix_count_+= count;
}
inline void buf_page_t::set_buf_fix_count(uint32_t count)
{
- ut_ad(mutex_own(&buf_pool.mutex));
+ mysql_mutex_assert_owner(&buf_pool.mutex);
buf_fix_count_= count;
}
inline void buf_page_t::set_state(buf_page_state state)
{
- ut_ad(mutex_own(&buf_pool.mutex));
+ mysql_mutex_assert_owner(&buf_pool.mutex);
#ifdef UNIV_DEBUG
switch (state) {
case BUF_BLOCK_REMOVE_HASH:
@@ -2170,7 +2136,7 @@ inline void buf_page_t::set_state(buf_page_state state)
inline void buf_page_t::set_io_fix(buf_io_fix io_fix)
{
- ut_ad(mutex_own(&buf_pool.mutex));
+ mysql_mutex_assert_owner(&buf_pool.mutex);
io_fix_= io_fix;
}
@@ -2196,7 +2162,7 @@ inline void buf_page_t::set_corrupt_id()
/** Set oldest_modification when adding to buf_pool.flush_list */
inline void buf_page_t::set_oldest_modification(lsn_t lsn)
{
- ut_ad(mutex_own(&buf_pool.flush_list_mutex));
+ mysql_mutex_assert_owner(&buf_pool.flush_list_mutex);
ut_ad(!oldest_modification());
oldest_modification_= lsn;
}
@@ -2204,7 +2170,7 @@ inline void buf_page_t::set_oldest_modification(lsn_t lsn)
/** Clear oldest_modification when removing from buf_pool.flush_list */
inline void buf_page_t::clear_oldest_modification()
{
- ut_ad(mutex_own(&buf_pool.flush_list_mutex));
+ mysql_mutex_assert_owner(&buf_pool.flush_list_mutex);
ut_d(const auto state= state_);
ut_ad(state == BUF_BLOCK_FILE_PAGE || state == BUF_BLOCK_ZIP_PAGE ||
state == BUF_BLOCK_REMOVE_HASH);
@@ -2215,7 +2181,7 @@ inline void buf_page_t::clear_oldest_modification()
/** @return whether the block is modified and ready for flushing */
inline bool buf_page_t::ready_for_flush() const
{
- ut_ad(mutex_own(&buf_pool.mutex));
+ mysql_mutex_assert_owner(&buf_pool.mutex);
ut_ad(in_LRU_list);
ut_a(in_file());
return oldest_modification() && io_fix_ == BUF_IO_NONE;
@@ -2225,7 +2191,7 @@ inline bool buf_page_t::ready_for_flush() const
The block can be dirty, but it must not be I/O-fixed or bufferfixed. */
inline bool buf_page_t::can_relocate() const
{
- ut_ad(mutex_own(&buf_pool.mutex));
+ mysql_mutex_assert_owner(&buf_pool.mutex);
ut_ad(in_file());
ut_ad(in_LRU_list);
return io_fix_ == BUF_IO_NONE && !buf_fix_count_;
@@ -2234,7 +2200,7 @@ inline bool buf_page_t::can_relocate() const
/** @return whether the block has been flagged old in buf_pool.LRU */
inline bool buf_page_t::is_old() const
{
- ut_ad(mutex_own(&buf_pool.mutex));
+ mysql_mutex_assert_owner(&buf_pool.mutex);
ut_ad(in_file());
ut_ad(in_LRU_list);
return old;
@@ -2243,7 +2209,7 @@ inline bool buf_page_t::is_old() const
/** Set whether a block is old in buf_pool.LRU */
inline void buf_page_t::set_old(bool old)
{
- ut_ad(mutex_own(&buf_pool.mutex));
+ mysql_mutex_assert_owner(&buf_pool.mutex);
ut_ad(in_LRU_list);
#ifdef UNIV_LRU_DEBUG
@@ -2270,14 +2236,14 @@ inline void buf_page_t::set_old(bool old)
#ifdef UNIV_DEBUG
/** Forbid the release of the buffer pool mutex. */
-# define buf_pool_mutex_exit_forbid() do { \
- ut_ad(mutex_own(&buf_pool.mutex)); \
- buf_pool.mutex_exit_forbidden++; \
+# define buf_pool_mutex_exit_forbid() do { \
+ mysql_mutex_assert_owner(&buf_pool.mutex); \
+ buf_pool.mutex_exit_forbidden++; \
} while (0)
/** Allow the release of the buffer pool mutex. */
# define buf_pool_mutex_exit_allow() do { \
- ut_ad(mutex_own(&buf_pool.mutex)); \
- ut_ad(buf_pool.mutex_exit_forbidden--); \
+ mysql_mutex_assert_owner(&buf_pool.mutex); \
+ ut_ad(buf_pool.mutex_exit_forbidden--); \
} while (0)
#else
/** Forbid the release of the buffer pool mutex. */
@@ -2295,8 +2261,7 @@ MEMORY: is not in free list, LRU list, or flush list, nor page
hash table
FILE_PAGE: space and offset are defined, is in page hash table
if io_fix == BUF_IO_WRITE,
- pool: no_flush[flush_type] is in reset state,
- pool: n_flush[flush_type] > 0
+ buf_pool.n_flush_LRU > 0 || buf_pool.n_flush_list > 0
(1) if buf_fix_count == 0, then
is in LRU list, not in free list
@@ -2333,7 +2298,7 @@ of the LRU list.
@return buf_page_t from where to start scan. */
inline buf_page_t *LRUItr::start()
{
- ut_ad(mutex_own(m_mutex));
+ mysql_mutex_assert_owner(m_mutex);
if (!m_hp || m_hp->old)
m_hp= UT_LIST_GET_LAST(buf_pool.LRU);