summaryrefslogtreecommitdiff
path: root/storage/innobase/buf/buf0buf.cc
diff options
context:
space:
mode:
Diffstat (limited to 'storage/innobase/buf/buf0buf.cc')
-rw-r--r--storage/innobase/buf/buf0buf.cc2729
1 files changed, 1181 insertions, 1548 deletions
diff --git a/storage/innobase/buf/buf0buf.cc b/storage/innobase/buf/buf0buf.cc
index 83eee85d749..462b1eb634a 100644
--- a/storage/innobase/buf/buf0buf.cc
+++ b/storage/innobase/buf/buf0buf.cc
@@ -48,7 +48,6 @@ Created 11/5/1995 Heikki Tuuri
#include "buf0buddy.h"
#include "buf0dblwr.h"
#include "lock0lock.h"
-#include "sync0rw.h"
#include "btr0sea.h"
#include "ibuf0ibuf.h"
#include "trx0undo.h"
@@ -64,7 +63,6 @@ Created 11/5/1995 Heikki Tuuri
#include "fil0pagecompress.h"
#endif /* !UNIV_INNOCHECKSUM */
#include "page0zip.h"
-#include "sync0sync.h"
#include "buf0dump.h"
#include <map>
#include <sstream>
@@ -222,11 +220,11 @@ buf_pool.LRU.
The chains of free memory blocks (buf_pool.zip_free[]) are used by
the buddy allocator (buf0buddy.cc) to keep track of currently unused
-memory blocks of size sizeof(buf_page_t)..srv_page_size / 2. These
-blocks are inside the srv_page_size-sized memory blocks of type
+memory blocks of size 1024..innodb_page_size / 2. These
+blocks are inside the memory blocks of size innodb_page_size and type
BUF_BLOCK_MEMORY that the buddy allocator requests from the buffer
-pool. The buddy allocator is solely used for allocating control
-blocks for compressed pages (buf_page_t) and compressed page frames.
+pool. The buddy allocator is solely used for allocating
+ROW_FORMAT=COMPRESSED page frames.
Loading a file page
-------------------
@@ -234,9 +232,9 @@ blocks for compressed pages (buf_page_t) and compressed page frames.
First, a victim block for replacement has to be found in the
buf_pool. It is taken from the free list or searched for from the
end of the LRU-list. An exclusive lock is reserved for the frame,
-the io_fix field is set in the block fixing the block in buf_pool,
+the io_fix is set in the block fixing the block in buf_pool,
and the io-operation for loading the page is queued. The io-handler thread
-releases the X-lock on the frame and resets the io_fix field
+releases the X-lock on the frame and releases the io_fix
when the io operation completes.
A thread may request the above operation using the function
@@ -280,18 +278,19 @@ the read requests for the whole area.
*/
#ifndef UNIV_INNOCHECKSUM
+# ifdef SUX_LOCK_GENERIC
void page_hash_latch::read_lock_wait()
{
/* First, try busy spinning for a while. */
for (auto spin= srv_n_spin_wait_rounds; spin--; )
{
- ut_delay(srv_spin_wait_delay);
+ LF_BACKOFF();
if (read_trylock())
return;
}
/* Fall back to yielding to other threads. */
do
- os_thread_yield();
+ std::this_thread::yield();
while (!read_trylock());
}
@@ -304,18 +303,16 @@ void page_hash_latch::write_lock_wait()
{
if (write_lock_poll())
return;
- ut_delay(srv_spin_wait_delay);
+ LF_BACKOFF();
}
/* Fall back to yielding to other threads. */
do
- os_thread_yield();
+ std::this_thread::yield();
while (!write_lock_poll());
}
+# endif
-/** Value in microseconds */
-constexpr int WAIT_FOR_READ= 100;
-constexpr int WAIT_FOR_WRITE= 100;
/** Number of attempts made to read in a page in the buffer pool */
constexpr ulint BUF_PAGE_READ_MAX_RETRIES= 100;
/** The maximum portion of the buffer pool that can be used for the
@@ -336,16 +333,13 @@ buf_pool_t::chunk_t::map *buf_pool_t::chunk_t::map_ref;
#ifdef UNIV_DEBUG
/** This is used to insert validation operations in execution
in the debug version */
-static ulint buf_dbg_counter;
+static Atomic_counter<size_t> buf_dbg_counter;
#endif /* UNIV_DEBUG */
/** Macro to determine whether the read of write counter is used depending
on the io_type */
-#define MONITOR_RW_COUNTER(io_type, counter) \
- ((io_type == BUF_IO_READ) \
- ? (counter##_READ) \
- : (counter##_WRITTEN))
-
+#define MONITOR_RW_COUNTER(read, counter) \
+ (read ? (counter##_READ) : (counter##_WRITTEN))
/** Decrypt a page for temporary tablespace.
@param[in,out] tmp_frame Temporary buffer
@@ -399,8 +393,7 @@ static bool buf_page_decrypt_after_read(buf_page_t *bpage,
ut_ad(node.space->id == bpage->id().space());
const auto flags = node.space->flags;
- byte* dst_frame = bpage->zip.data ? bpage->zip.data :
- ((buf_block_t*) bpage)->frame;
+ byte* dst_frame = bpage->zip.data ? bpage->zip.data : bpage->frame;
bool page_compressed = node.space->is_compressed()
&& buf_page_is_compressed(dst_frame, flags);
const page_id_t id(bpage->id());
@@ -410,28 +403,21 @@ static bool buf_page_decrypt_after_read(buf_page_t *bpage,
return (true);
}
- if (node.space->purpose == FIL_TYPE_TEMPORARY
+ buf_tmp_buffer_t* slot;
+
+ if (id.space() == SRV_TMP_SPACE_ID
&& innodb_encrypt_temporary_tables) {
- buf_tmp_buffer_t* slot = buf_pool.io_buf_reserve();
- ut_a(slot);
+ slot = buf_pool.io_buf_reserve();
slot->allocate();
-
- if (!buf_tmp_page_decrypt(slot->crypt_buf, dst_frame)) {
- slot->release();
- ib::error() << "Encrypted page " << id
- << " in file " << node.name;
- return false;
- }
-
+ bool ok = buf_tmp_page_decrypt(slot->crypt_buf, dst_frame);
slot->release();
- return true;
+ return ok;
}
/* Page is encrypted if encryption information is found from
tablespace and page contains used key_version. This is true
also for pages first compressed and then encrypted. */
- buf_tmp_buffer_t* slot;
uint key_version = buf_page_get_key_version(dst_frame, flags);
if (page_compressed && !key_version) {
@@ -444,17 +430,12 @@ decompress:
}
slot = buf_pool.io_buf_reserve();
- ut_a(slot);
slot->allocate();
decompress_with_slot:
- ut_d(fil_page_type_validate(node.space, dst_frame));
-
ulint write_size = fil_page_decompress(
slot->crypt_buf, dst_frame, flags);
slot->release();
- ut_ad(!write_size
- || fil_page_type_validate(node.space, dst_frame));
ut_ad(node.space->referenced());
return write_size != 0;
}
@@ -472,9 +453,7 @@ decrypt_failed:
}
slot = buf_pool.io_buf_reserve();
- ut_a(slot);
slot->allocate();
- ut_d(fil_page_type_validate(node.space, dst_frame));
/* decrypt using crypt_buf to dst_frame */
if (!fil_space_decrypt(node.space, slot->crypt_buf, dst_frame)) {
@@ -482,8 +461,6 @@ decrypt_failed:
goto decrypt_failed;
}
- ut_d(fil_page_type_validate(node.space, dst_frame));
-
if ((fil_space_t::full_crc32(flags) && page_compressed)
|| fil_page_get_type(dst_frame)
== FIL_PAGE_PAGE_COMPRESSED_ENCRYPTED) {
@@ -506,6 +483,7 @@ decrypt_failed:
@param[in] checksum_field1 new checksum field
@param[in] checksum_field2 old checksum field
@return true if the page is in crc32 checksum format. */
+static
bool
buf_page_is_checksum_valid_crc32(
const byte* read_buf,
@@ -515,8 +493,9 @@ buf_page_is_checksum_valid_crc32(
const uint32_t crc32 = buf_calc_page_crc32(read_buf);
#ifdef UNIV_INNOCHECKSUM
- if (log_file
- && srv_checksum_algorithm == SRV_CHECKSUM_ALGORITHM_STRICT_CRC32) {
+ extern FILE* log_file;
+ extern uint32_t cur_page_num;
+ if (log_file) {
fprintf(log_file, "page::" UINT32PF ";"
" crc32 calculated = " UINT32PF ";"
" recorded checksum field1 = " ULINTPF " recorded"
@@ -532,132 +511,6 @@ buf_page_is_checksum_valid_crc32(
return checksum_field1 == crc32;
}
-/** Checks if the page is in innodb checksum format.
-@param[in] read_buf database page
-@param[in] checksum_field1 new checksum field
-@param[in] checksum_field2 old checksum field
-@return true if the page is in innodb checksum format. */
-bool
-buf_page_is_checksum_valid_innodb(
- const byte* read_buf,
- ulint checksum_field1,
- ulint checksum_field2)
-{
- /* There are 2 valid formulas for
- checksum_field2 (old checksum field) which algo=innodb could have
- written to the page:
-
- 1. Very old versions of InnoDB only stored 8 byte lsn to the
- start and the end of the page.
-
- 2. Newer InnoDB versions store the old formula checksum
- (buf_calc_page_old_checksum()). */
-
- ulint old_checksum = buf_calc_page_old_checksum(read_buf);
- ulint new_checksum = buf_calc_page_new_checksum(read_buf);
-
-#ifdef UNIV_INNOCHECKSUM
- if (log_file
- && srv_checksum_algorithm == SRV_CHECKSUM_ALGORITHM_INNODB) {
- fprintf(log_file, "page::" UINT32PF ";"
- " old style: calculated ="
- " " ULINTPF "; recorded = " ULINTPF "\n",
- cur_page_num, old_checksum,
- checksum_field2);
- fprintf(log_file, "page::" UINT32PF ";"
- " new style: calculated ="
- " " ULINTPF "; crc32 = " UINT32PF "; recorded = " ULINTPF "\n",
- cur_page_num, new_checksum,
- buf_calc_page_crc32(read_buf), checksum_field1);
- }
-
- if (log_file
- && srv_checksum_algorithm == SRV_CHECKSUM_ALGORITHM_STRICT_INNODB) {
- fprintf(log_file, "page::" UINT32PF ";"
- " old style: calculated ="
- " " ULINTPF "; recorded checksum = " ULINTPF "\n",
- cur_page_num, old_checksum,
- checksum_field2);
- fprintf(log_file, "page::" UINT32PF ";"
- " new style: calculated ="
- " " ULINTPF "; recorded checksum = " ULINTPF "\n",
- cur_page_num, new_checksum,
- checksum_field1);
- }
-#endif /* UNIV_INNOCHECKSUM */
-
-
- if (checksum_field2 != mach_read_from_4(read_buf + FIL_PAGE_LSN)
- && checksum_field2 != old_checksum) {
- DBUG_LOG("checksum",
- "Page checksum crc32 not valid"
- << " field1 " << checksum_field1
- << " field2 " << checksum_field2
- << " crc32 " << buf_calc_page_old_checksum(read_buf)
- << " lsn " << mach_read_from_4(
- read_buf + FIL_PAGE_LSN));
- return(false);
- }
-
- /* old field is fine, check the new field */
-
- /* InnoDB versions < 4.0.14 and < 4.1.1 stored the space id
- (always equal to 0), to FIL_PAGE_SPACE_OR_CHKSUM */
-
- if (checksum_field1 != 0 && checksum_field1 != new_checksum) {
- DBUG_LOG("checksum",
- "Page checksum crc32 not valid"
- << " field1 " << checksum_field1
- << " field2 " << checksum_field2
- << " crc32 " << buf_calc_page_new_checksum(read_buf)
- << " lsn " << mach_read_from_4(
- read_buf + FIL_PAGE_LSN));
- return(false);
- }
-
- return(true);
-}
-
-/** Checks if the page is in none checksum format.
-@param[in] read_buf database page
-@param[in] checksum_field1 new checksum field
-@param[in] checksum_field2 old checksum field
-@return true if the page is in none checksum format. */
-bool
-buf_page_is_checksum_valid_none(
- const byte* read_buf,
- ulint checksum_field1,
- ulint checksum_field2)
-{
-#ifndef DBUG_OFF
- if (checksum_field1 != checksum_field2
- && checksum_field1 != BUF_NO_CHECKSUM_MAGIC) {
- DBUG_LOG("checksum",
- "Page checksum crc32 not valid"
- << " field1 " << checksum_field1
- << " field2 " << checksum_field2
- << " crc32 " << BUF_NO_CHECKSUM_MAGIC
- << " lsn " << mach_read_from_4(read_buf
- + FIL_PAGE_LSN));
- }
-#endif /* DBUG_OFF */
-
-#ifdef UNIV_INNOCHECKSUM
- if (log_file
- && srv_checksum_algorithm == SRV_CHECKSUM_ALGORITHM_STRICT_NONE) {
- fprintf(log_file,
- "page::" UINT32PF "; none checksum: calculated"
- " = %lu; recorded checksum_field1 = " ULINTPF
- " recorded checksum_field2 = " ULINTPF "\n",
- cur_page_num, BUF_NO_CHECKSUM_MAGIC,
- checksum_field1, checksum_field2);
- }
-#endif /* UNIV_INNOCHECKSUM */
-
- return(checksum_field1 == checksum_field2
- && checksum_field1 == BUF_NO_CHECKSUM_MAGIC);
-}
-
/** Checks whether the lsn present in the page is lesser than the
peek current lsn.
@param[in] check_lsn lsn to check
@@ -717,9 +570,6 @@ buf_page_is_corrupted(
const byte* read_buf,
ulint fsp_flags)
{
-#ifndef UNIV_INNOCHECKSUM
- DBUG_EXECUTE_IF("buf_page_import_corrupt_failure", return(true); );
-#endif
if (fil_space_t::full_crc32(fsp_flags)) {
bool compressed = false, corrupted = false;
const uint size = buf_page_full_crc32_size(
@@ -764,8 +614,6 @@ buf_page_is_corrupted(
return false;
}
- size_t checksum_field1 = 0;
- size_t checksum_field2 = 0;
const ulint zip_size = fil_space_t::zip_size(fsp_flags);
const uint16_t page_type = fil_page_get_type(read_buf);
@@ -804,21 +652,14 @@ buf_page_is_corrupted(
/* Check whether the checksum fields have correct values */
- const srv_checksum_algorithm_t curr_algo =
- static_cast<srv_checksum_algorithm_t>(srv_checksum_algorithm);
-
- if (curr_algo == SRV_CHECKSUM_ALGORITHM_NONE) {
- return(false);
- }
-
if (zip_size) {
return !page_zip_verify_checksum(read_buf, zip_size);
}
- checksum_field1 = mach_read_from_4(
+ const uint32_t checksum_field1 = mach_read_from_4(
read_buf + FIL_PAGE_SPACE_OR_CHKSUM);
- checksum_field2 = mach_read_from_4(
+ const uint32_t checksum_field2 = mach_read_from_4(
read_buf + srv_page_size - FIL_PAGE_END_LSN_OLD_CHKSUM);
static_assert(FIL_PAGE_LSN % 8 == 0, "alignment");
@@ -853,47 +694,22 @@ buf_page_is_corrupted(
}
}
- switch (curr_algo) {
+#ifndef UNIV_INNOCHECKSUM
+ switch (srv_checksum_algorithm) {
case SRV_CHECKSUM_ALGORITHM_STRICT_FULL_CRC32:
case SRV_CHECKSUM_ALGORITHM_STRICT_CRC32:
+#endif /* !UNIV_INNOCHECKSUM */
return !buf_page_is_checksum_valid_crc32(
read_buf, checksum_field1, checksum_field2);
- case SRV_CHECKSUM_ALGORITHM_STRICT_INNODB:
- return !buf_page_is_checksum_valid_innodb(
- read_buf, checksum_field1, checksum_field2);
- case SRV_CHECKSUM_ALGORITHM_STRICT_NONE:
- return !buf_page_is_checksum_valid_none(
- read_buf, checksum_field1, checksum_field2);
- case SRV_CHECKSUM_ALGORITHM_NONE:
- /* should have returned false earlier */
- break;
- case SRV_CHECKSUM_ALGORITHM_FULL_CRC32:
- case SRV_CHECKSUM_ALGORITHM_CRC32:
- case SRV_CHECKSUM_ALGORITHM_INNODB:
- const uint32_t crc32 = buf_calc_page_crc32(read_buf);
-
- if (buf_page_is_checksum_valid_none(read_buf,
- checksum_field1, checksum_field2)) {
-#ifdef UNIV_INNOCHECKSUM
- if (log_file) {
- fprintf(log_file, "page::" UINT32PF ";"
- " old style: calculated = %u;"
- " recorded = " ULINTPF ";\n",
- cur_page_num,
- buf_calc_page_old_checksum(read_buf),
- checksum_field2);
- fprintf(log_file, "page::" UINT32PF ";"
- " new style: calculated = " UINT32PF ";"
- " crc32 = " UINT32PF "; recorded = " ULINTPF ";\n",
- cur_page_num,
- buf_calc_page_new_checksum(read_buf),
- crc32,
- checksum_field1);
- }
-#endif /* UNIV_INNOCHECKSUM */
+#ifndef UNIV_INNOCHECKSUM
+ default:
+ if (checksum_field1 == BUF_NO_CHECKSUM_MAGIC
+ && checksum_field2 == BUF_NO_CHECKSUM_MAGIC) {
return false;
}
+ const uint32_t crc32 = buf_calc_page_crc32(read_buf);
+
/* Very old versions of InnoDB only stored 8 byte lsn to the
start and the end of the page. */
@@ -921,20 +737,13 @@ buf_page_is_corrupted(
switch (checksum_field1) {
case 0:
case BUF_NO_CHECKSUM_MAGIC:
- break;
- default:
- if ((checksum_field1 != crc32
- || checksum_field2 != crc32)
- && checksum_field1
- != buf_calc_page_new_checksum(read_buf)) {
- return true;
- }
+ return false;
}
-
- break;
+ return (checksum_field1 != crc32 || checksum_field2 != crc32)
+ && checksum_field1
+ != buf_calc_page_new_checksum(read_buf);
}
-
- return false;
+#endif /* !UNIV_INNOCHECKSUM */
}
#ifndef UNIV_INNOCHECKSUM
@@ -1029,28 +838,23 @@ buf_block_init(buf_block_t* block, byte* frame)
buf_pool.resize(). Either way, adaptive hash index must not exist. */
assert_block_ahi_empty_on_init(block);
- block->frame = frame;
+ block->page.frame = frame;
- block->modify_clock = 0;
- block->page.init(BUF_BLOCK_NOT_USED, page_id_t(~0ULL));
+ MEM_MAKE_DEFINED(&block->modify_clock, sizeof block->modify_clock);
+ ut_ad(!block->modify_clock);
+ MEM_MAKE_DEFINED(&block->page.lock, sizeof block->page.lock);
+ block->page.init(buf_page_t::NOT_USED, page_id_t(~0ULL));
#ifdef BTR_CUR_HASH_ADAPT
- block->index = NULL;
+ MEM_MAKE_DEFINED(&block->index, sizeof block->index);
+ ut_ad(!block->index);
#endif /* BTR_CUR_HASH_ADAPT */
ut_d(block->in_unzip_LRU_list = false);
ut_d(block->in_withdraw_list = false);
page_zip_des_init(&block->page.zip);
- ut_d(block->debug_latch = (rw_lock_t *) ut_malloc_nokey(sizeof(rw_lock_t)));
-
- rw_lock_create(PFS_NOT_INSTRUMENTED, &block->lock, SYNC_LEVEL_VARYING);
-
- ut_d(rw_lock_create(PFS_NOT_INSTRUMENTED, block->debug_latch,
- SYNC_LEVEL_VARYING));
-
- block->lock.is_block_lock = 1;
-
- ut_ad(rw_lock_validate(&(block->lock)));
+ MEM_MAKE_DEFINED(&block->page.hash, sizeof block->page.hash);
+ ut_ad(!block->page.hash);
}
/** Allocate a chunk of buffer frames.
@@ -1123,7 +927,7 @@ inline bool buf_pool_t::chunk_t::create(size_t bytes)
for (auto i= size; i--; ) {
buf_block_init(block, frame);
- MEM_UNDEFINED(block->frame, srv_page_size);
+ MEM_UNDEFINED(block->page.frame, srv_page_size);
/* Add the block to the free list */
UT_LIST_ADD_LAST(buf_pool.free, &block->page);
@@ -1146,18 +950,11 @@ inline const buf_block_t *buf_pool_t::chunk_t::not_freed() const
buf_block_t *block= blocks;
for (auto i= size; i--; block++)
{
- switch (block->page.state()) {
- case BUF_BLOCK_ZIP_PAGE:
+ if (block->page.in_file())
+ {
/* The uncompressed buffer pool should never
contain ROW_FORMAT=COMPRESSED block descriptors. */
- ut_error;
- break;
- case BUF_BLOCK_NOT_USED:
- case BUF_BLOCK_MEMORY:
- case BUF_BLOCK_REMOVE_HASH:
- /* Skip blocks that are not being used for file pages. */
- break;
- case BUF_BLOCK_FILE_PAGE:
+ ut_ad(block->page.frame);
const lsn_t lsn= block->page.oldest_modification();
if (srv_read_only_mode)
@@ -1166,8 +963,6 @@ inline const buf_block_t *buf_pool_t::chunk_t::not_freed() const
can be dirtied, so all of them must be clean. */
ut_ad(lsn == 0 || lsn == recv_sys.recovered_lsn ||
srv_force_recovery == SRV_FORCE_NO_LOG_REDO);
- ut_ad(!block->page.buf_fix_count());
- ut_ad(block->page.io_fix() == BUF_IO_NONE);
break;
}
@@ -1188,24 +983,16 @@ inline const buf_block_t *buf_pool_t::chunk_t::not_freed() const
}
#endif /* UNIV_DEBUG */
-/** Free the synchronization objects of a buffer pool block descriptor
-@param[in,out] block buffer pool block descriptor */
-static void buf_block_free_mutexes(buf_block_t* block)
-{
- rw_lock_free(&block->lock);
- ut_d(rw_lock_free(block->debug_latch));
- ut_d(ut_free(block->debug_latch));
-}
-
/** Create the hash table.
@param n the lower bound of n_cells */
void buf_pool_t::page_hash_table::create(ulint n)
{
n_cells= ut_find_prime(n);
- const size_t size= pad(n_cells) * sizeof *array;
- void* v= aligned_malloc(size, CPU_LEVEL1_DCACHE_LINESIZE);
- memset(v, 0, size);
- array= static_cast<hash_cell_t*>(v);
+ const size_t size= MY_ALIGN(pad(n_cells) * sizeof *array,
+ CPU_LEVEL1_DCACHE_LINESIZE);
+ void *v= aligned_malloc(size, CPU_LEVEL1_DCACHE_LINESIZE);
+ memset_aligned<CPU_LEVEL1_DCACHE_LINESIZE>(v, 0, size);
+ array= static_cast<hash_chain*>(v);
}
/** Create the buffer pool.
@@ -1253,7 +1040,7 @@ bool buf_pool_t::create()
buf_block_t* block= chunk->blocks;
for (auto i= chunk->size; i--; block++)
- buf_block_free_mutexes(block);
+ block->page.lock.free();
allocator.deallocate_large_dodump(chunk->mem, &chunk->mem_pfx);
}
@@ -1283,7 +1070,6 @@ bool buf_pool_t::create()
for (size_t i= 0; i < UT_ARR_SIZE(zip_free); ++i)
UT_LIST_INIT(zip_free[i], &buf_buddy_free_t::list);
ulint s= curr_size;
- old_size= s;
s/= BUF_READ_AHEAD_PORTION;
read_ahead_area= s >= READ_AHEAD_PAGES
? READ_AHEAD_PAGES
@@ -1351,8 +1137,11 @@ void buf_pool_t::close()
? (oldest == 0 || oldest == 2)
: oldest <= 1 || srv_is_being_started || srv_fast_shutdown == 2);
- if (bpage->state() != BUF_BLOCK_FILE_PAGE)
- buf_page_free_descriptor(bpage);
+ if (UNIV_UNLIKELY(!bpage->frame))
+ {
+ bpage->lock.free();
+ ut_free(bpage);
+ }
}
for (auto chunk= chunks + n_chunks; --chunk >= chunks; )
@@ -1360,7 +1149,7 @@ void buf_pool_t::close()
buf_block_t *block= chunk->blocks;
for (auto i= chunk->size; i--; block++)
- buf_block_free_mutexes(block);
+ block->page.lock.free();
allocator.deallocate_large_dodump(chunk->mem, &chunk->mem_pfx);
}
@@ -1390,7 +1179,8 @@ inline bool buf_pool_t::realloc(buf_block_t *block)
buf_block_t* new_block;
mysql_mutex_assert_owner(&mutex);
- ut_ad(block->page.state() == BUF_BLOCK_FILE_PAGE);
+ ut_ad(block->page.in_file());
+ ut_ad(block->page.frame);
new_block = buf_LRU_get_free_only();
@@ -1401,15 +1191,24 @@ inline bool buf_pool_t::realloc(buf_block_t *block)
return(false); /* free list was not enough */
}
- const page_id_t id(block->page.id());
- page_hash_latch* hash_lock = hash_lock_get(id);
- hash_lock->write_lock();
+ const page_id_t id{block->page.id()};
+ hash_chain& chain = page_hash.cell_get(id.fold());
+ page_hash_latch& hash_lock = page_hash.lock_get(chain);
+ /* It does not make sense to use transactional_lock_guard
+ here, because copying innodb_page_size (4096 to 65536) bytes
+ as well as other changes would likely make the memory
+ transaction too large. */
+ hash_lock.lock();
if (block->page.can_relocate()) {
memcpy_aligned<OS_FILE_LOG_BLOCK_SIZE>(
- new_block->frame, block->frame, srv_page_size);
+ new_block->page.frame, block->page.frame,
+ srv_page_size);
mysql_mutex_lock(&buf_pool.flush_list_mutex);
+ const auto frame = new_block->page.frame;
+ new_block->page.lock.free();
new (&new_block->page) buf_page_t(block->page);
+ new_block->page.frame = frame;
/* relocate LRU list */
if (buf_page_t* prev_b = buf_pool.LRU_remove(&block->page)) {
@@ -1447,23 +1246,20 @@ inline bool buf_pool_t::realloc(buf_block_t *block)
}
/* relocate page_hash */
- ut_ad(block->page.in_page_hash);
- ut_ad(new_block->page.in_page_hash);
- const ulint fold = id.fold();
- ut_ad(&block->page == page_hash_get_low(id, fold));
- ut_d(block->page.in_page_hash = false);
- HASH_REPLACE(buf_page_t, hash, &page_hash, fold,
- &block->page, &new_block->page);
-
+ hash_chain& chain = page_hash.cell_get(id.fold());
+ ut_ad(&block->page == page_hash.get(id, chain));
+ buf_pool.page_hash.replace(chain, &block->page,
+ &new_block->page);
buf_block_modify_clock_inc(block);
static_assert(FIL_PAGE_OFFSET % 4 == 0, "alignment");
- memset_aligned<4>(block->frame + FIL_PAGE_OFFSET, 0xff, 4);
+ memset_aligned<4>(block->page.frame
+ + FIL_PAGE_OFFSET, 0xff, 4);
static_assert(FIL_PAGE_ARCH_LOG_NO_OR_SPACE_ID % 4 == 2,
"not perfect alignment");
- memset_aligned<2>(block->frame
+ memset_aligned<2>(block->page.frame
+ FIL_PAGE_ARCH_LOG_NO_OR_SPACE_ID, 0xff, 4);
- MEM_UNDEFINED(block->frame, srv_page_size);
- block->page.set_state(BUF_BLOCK_REMOVE_HASH);
+ MEM_UNDEFINED(block->page.frame, srv_page_size);
+ block->page.set_state(buf_page_t::REMOVE_HASH);
if (!fsp_is_system_temporary(id.space())) {
buf_flush_relocate_on_flush_list(&block->page,
&new_block->page);
@@ -1484,16 +1280,51 @@ inline bool buf_pool_t::realloc(buf_block_t *block)
new_block->n_fields = 1;
new_block->left_side = TRUE;
#endif /* BTR_CUR_HASH_ADAPT */
- ut_d(block->page.set_state(BUF_BLOCK_MEMORY));
+ ut_d(block->page.set_state(buf_page_t::MEMORY));
/* free block */
new_block = block;
}
- hash_lock->write_unlock();
+ hash_lock.unlock();
buf_LRU_block_free_non_file_page(new_block);
return(true); /* free_list was enough */
}
+void buf_pool_t::io_buf_t::create(ulint n_slots)
+{
+ this->n_slots= n_slots;
+ slots= static_cast<buf_tmp_buffer_t*>
+ (ut_malloc_nokey(n_slots * sizeof *slots));
+ memset((void*) slots, 0, n_slots * sizeof *slots);
+}
+
+void buf_pool_t::io_buf_t::close()
+{
+ for (buf_tmp_buffer_t *s= slots, *e= slots + n_slots; s != e; s++)
+ {
+ aligned_free(s->crypt_buf);
+ aligned_free(s->comp_buf);
+ }
+ ut_free(slots);
+ slots= nullptr;
+ n_slots= 0;
+}
+
+buf_tmp_buffer_t *buf_pool_t::io_buf_t::reserve()
+{
+ for (;;)
+ {
+ for (buf_tmp_buffer_t *s= slots, *e= slots + n_slots; s != e; s++)
+ if (s->acquire())
+ return s;
+ os_aio_wait_until_no_pending_writes();
+ for (buf_tmp_buffer_t *s= slots, *e= slots + n_slots; s != e; s++)
+ if (s->acquire())
+ return s;
+ os_aio_wait_until_no_pending_reads();
+ }
+}
+
/** Sets the global variable that feeds MySQL's innodb_buffer_pool_resize_status
to the specified string. The format and the following parameters are the
same as the ones used for printf(3).
@@ -1529,17 +1360,13 @@ inline bool buf_pool_t::withdraw_blocks()
ib::info() << "start to withdraw the last "
<< withdraw_target << " blocks";
- /* Minimize zip_free[i] lists */
- mysql_mutex_lock(&mutex);
- buf_buddy_condense_free();
- mysql_mutex_unlock(&mutex);
-
while (UT_LIST_GET_LEN(withdraw) < withdraw_target) {
/* try to withdraw from free_list */
ulint count1 = 0;
mysql_mutex_lock(&mutex);
+ buf_buddy_condense_free();
block = reinterpret_cast<buf_block_t*>(
UT_LIST_GET_FIRST(free));
while (block != NULL
@@ -1564,68 +1391,57 @@ inline bool buf_pool_t::withdraw_blocks()
block = next_block;
}
- mysql_mutex_unlock(&mutex);
/* reserve free_list length */
if (UT_LIST_GET_LEN(withdraw) < withdraw_target) {
- ulint n_flushed = buf_flush_LRU(
+ buf_flush_LRU(
std::max<ulint>(withdraw_target
- UT_LIST_GET_LEN(withdraw),
- srv_LRU_scan_depth));
- buf_flush_wait_batch_end_acquiring_mutex(true);
-
- if (n_flushed) {
- MONITOR_INC_VALUE_CUMULATIVE(
- MONITOR_LRU_BATCH_FLUSH_TOTAL_PAGE,
- MONITOR_LRU_BATCH_FLUSH_COUNT,
- MONITOR_LRU_BATCH_FLUSH_PAGES,
- n_flushed);
- }
+ srv_LRU_scan_depth),
+ true);
+ mysql_mutex_unlock(&buf_pool.mutex);
+ buf_dblwr.flush_buffered_writes();
+ mysql_mutex_lock(&buf_pool.flush_list_mutex);
+ buf_flush_wait_LRU_batch_end();
+ mysql_mutex_unlock(&buf_pool.flush_list_mutex);
+ mysql_mutex_lock(&buf_pool.mutex);
}
/* relocate blocks/buddies in withdrawn area */
ulint count2 = 0;
- mysql_mutex_lock(&mutex);
- buf_page_t* bpage;
- bpage = UT_LIST_GET_FIRST(LRU);
- while (bpage != NULL) {
- buf_page_t* next_bpage = UT_LIST_GET_NEXT(LRU, bpage);
- if (bpage->zip.data != NULL
+ buf_pool_mutex_exit_forbid();
+ for (buf_page_t* bpage = UT_LIST_GET_FIRST(LRU), *next_bpage;
+ bpage; bpage = next_bpage) {
+ ut_ad(bpage->in_file());
+ next_bpage = UT_LIST_GET_NEXT(LRU, bpage);
+ if (UNIV_LIKELY_NULL(bpage->zip.data)
&& will_be_withdrawn(bpage->zip.data)
&& bpage->can_relocate()) {
- buf_pool_mutex_exit_forbid();
if (!buf_buddy_realloc(
bpage->zip.data,
page_zip_get_size(&bpage->zip))) {
/* failed to allocate block */
- buf_pool_mutex_exit_allow();
break;
}
- buf_pool_mutex_exit_allow();
count2++;
+ if (bpage->frame) {
+ goto realloc_frame;
+ }
}
- if (bpage->state() == BUF_BLOCK_FILE_PAGE
- && will_be_withdrawn(*bpage)) {
- if (bpage->can_relocate()) {
- buf_pool_mutex_exit_forbid();
- if (!realloc(
- reinterpret_cast<buf_block_t*>(
- bpage))) {
- /* failed to allocate block */
- buf_pool_mutex_exit_allow();
- break;
- }
- buf_pool_mutex_exit_allow();
- count2++;
+ if (bpage->frame && will_be_withdrawn(*bpage)
+ && bpage->can_relocate()) {
+realloc_frame:
+ if (!realloc(reinterpret_cast<buf_block_t*>(
+ bpage))) {
+ /* failed to allocate block */
+ break;
}
- /* NOTE: if the page is in use,
- not relocated yet */
+ count2++;
}
-
- bpage = next_bpage;
}
+ buf_pool_mutex_exit_allow();
mysql_mutex_unlock(&mutex);
buf_resize_status(
@@ -1655,7 +1471,7 @@ inline bool buf_pool_t::withdraw_blocks()
* const echunk = chunks + n_chunks; chunk != echunk; chunk++) {
block = chunk->blocks;
for (ulint j = chunk->size; j--; block++) {
- ut_a(block->page.state() == BUF_BLOCK_NOT_USED);
+ ut_a(block->page.state() == buf_page_t::NOT_USED);
ut_ad(block->in_withdraw_list);
}
}
@@ -1672,7 +1488,7 @@ inline void buf_pool_t::page_hash_table::write_lock_all()
{
for (auto n= pad(n_cells) & ~ELEMENTS_PER_LATCH;; n-= ELEMENTS_PER_LATCH + 1)
{
- reinterpret_cast<page_hash_latch&>(array[n]).write_lock();
+ reinterpret_cast<page_hash_latch&>(array[n]).lock();
if (!n)
break;
}
@@ -1683,7 +1499,7 @@ inline void buf_pool_t::page_hash_table::write_unlock_all()
{
for (auto n= pad(n_cells) & ~ELEMENTS_PER_LATCH;; n-= ELEMENTS_PER_LATCH + 1)
{
- reinterpret_cast<page_hash_latch&>(array[n]).write_unlock();
+ reinterpret_cast<page_hash_latch&>(array[n]).unlock();
if (!n)
break;
}
@@ -1701,7 +1517,7 @@ struct find_interesting_trx
return;
if (trx.mysql_thd == nullptr)
return;
- if (withdraw_started <= trx.start_time)
+ if (withdraw_started <= trx.start_time_micro)
return;
if (!found)
@@ -1719,8 +1535,9 @@ struct find_interesting_trx
}
bool &found;
- time_t withdraw_started;
- time_t current_time;
+ /** microsecond_interval_timer() */
+ const ulonglong withdraw_started;
+ const my_hrtime_t current_time;
};
} // namespace
@@ -1744,28 +1561,13 @@ inline void buf_pool_t::resize()
srv_buf_pool_old_size, srv_buf_pool_size,
srv_buf_pool_chunk_unit);
- mysql_mutex_lock(&mutex);
- ut_ad(curr_size == old_size);
- ut_ad(n_chunks_new == n_chunks);
- ut_ad(UT_LIST_GET_LEN(withdraw) == 0);
-
- n_chunks_new = (new_instance_size << srv_page_size_shift)
- / srv_buf_pool_chunk_unit;
- curr_size = n_chunks_new * chunks->size;
- mysql_mutex_unlock(&mutex);
-
#ifdef BTR_CUR_HASH_ADAPT
/* disable AHI if needed */
- const bool btr_search_disabled = btr_search_enabled;
-
buf_resize_status("Disabling adaptive hash index.");
btr_search_s_lock_all();
- if (btr_search_disabled) {
- btr_search_s_unlock_all();
- } else {
- btr_search_s_unlock_all();
- }
+ const bool btr_search_disabled = btr_search_enabled;
+ btr_search_s_unlock_all();
btr_search_disable();
@@ -1774,7 +1576,16 @@ inline void buf_pool_t::resize()
}
#endif /* BTR_CUR_HASH_ADAPT */
- if (curr_size < old_size) {
+ mysql_mutex_lock(&mutex);
+ ut_ad(n_chunks_new == n_chunks);
+ ut_ad(UT_LIST_GET_LEN(withdraw) == 0);
+
+ n_chunks_new = (new_instance_size << srv_page_size_shift)
+ / srv_buf_pool_chunk_unit;
+ curr_size = n_chunks_new * chunks->size;
+ mysql_mutex_unlock(&mutex);
+
+ if (is_shrinking()) {
/* set withdraw target */
size_t w = 0;
@@ -1789,13 +1600,13 @@ inline void buf_pool_t::resize()
buf_resize_status("Withdrawing blocks to be shrunken.");
- time_t withdraw_started = time(NULL);
- double message_interval = 60;
+ ulonglong withdraw_started = microsecond_interval_timer();
+ ulonglong message_interval = 60ULL * 1000 * 1000;
ulint retry_interval = 1;
withdraw_retry:
/* wait for the number of blocks fit to the new size (if needed)*/
- bool should_retry_withdraw = curr_size < old_size
+ bool should_retry_withdraw = is_shrinking()
&& withdraw_blocks();
if (srv_shutdown_state != SRV_SHUTDOWN_NONE) {
@@ -1806,30 +1617,33 @@ withdraw_retry:
/* abort buffer pool load */
buf_load_abort();
- const time_t current_time = time(NULL);
+ const ulonglong current_time = microsecond_interval_timer();
if (should_retry_withdraw
- && difftime(current_time, withdraw_started) >= message_interval) {
+ && current_time - withdraw_started >= message_interval) {
- if (message_interval > 900) {
- message_interval = 1800;
+ if (message_interval > 900000000) {
+ message_interval = 1800000000;
} else {
message_interval *= 2;
}
- lock_mutex_enter();
- bool found = false;
- trx_sys.trx_list.for_each(find_interesting_trx{
- found, withdraw_started, current_time});
- lock_mutex_exit();
-
+ bool found= false;
+ find_interesting_trx f
+ {found, withdraw_started, my_hrtime_coarse()};
withdraw_started = current_time;
+
+ /* This is going to exceed the maximum size of a
+ memory transaction. */
+ LockMutexGuard g{SRW_LOCK_CALL};
+ trx_sys.trx_list.for_each(f);
}
if (should_retry_withdraw) {
ib::info() << "Will retry to withdraw " << retry_interval
<< " seconds later.";
- os_thread_sleep(retry_interval * 1000000);
+ std::this_thread::sleep_for(
+ std::chrono::seconds(retry_interval));
if (retry_interval > 5) {
retry_interval = 10;
@@ -1850,7 +1664,9 @@ withdraw_retry:
should_wait = false;
DBUG_EXECUTE_IF(
"ib_buf_pool_resize_wait_before_resize",
- should_wait = true; os_thread_sleep(10000););
+ should_wait = true;
+ std::this_thread::sleep_for(
+ std::chrono::milliseconds(10)););
}
}
#endif /* !DBUG_OFF */
@@ -1873,7 +1689,7 @@ withdraw_retry:
ULINTPF " to " ULINTPF ".",
n_chunks, n_chunks_new);
- if (n_chunks_new < n_chunks) {
+ if (is_shrinking()) {
/* delete chunks */
chunk_t* chunk = chunks + n_chunks_new;
const chunk_t* const echunk = chunks + n_chunks;
@@ -1897,7 +1713,7 @@ withdraw_retry:
buf_block_t* block = chunk->blocks;
for (ulint j = chunk->size; j--; block++) {
- buf_block_free_mutexes(block);
+ block->page.lock.free();
}
allocator.deallocate_large_dodump(
@@ -1937,8 +1753,7 @@ withdraw_retry:
goto calc_buf_pool_size;
}
- ulint n_chunks_copy = ut_min(n_chunks_new,
- n_chunks);
+ ulint n_chunks_copy = ut_min(n_chunks_new, n_chunks);
memcpy(new_chunks, chunks,
n_chunks_copy * sizeof *new_chunks);
@@ -2005,14 +1820,14 @@ calc_buf_pool_size:
/* set size */
ut_ad(UT_LIST_GET_LEN(withdraw) == 0);
ulint s= curr_size;
- old_size= s;
s/= BUF_READ_AHEAD_PORTION;
read_ahead_area= s >= READ_AHEAD_PAGES
? READ_AHEAD_PAGES
: my_round_up_to_next_power(static_cast<uint32_t>(s));
curr_pool_size= n_chunks * srv_buf_pool_chunk_unit;
srv_buf_pool_curr_size= curr_pool_size;/* FIXME: remove*/
- innodb_set_buf_pool_size(buf_pool_size_align(srv_buf_pool_curr_size));
+ extern ulonglong innobase_buffer_pool_size;
+ innobase_buffer_pool_size= buf_pool_size_align(srv_buf_pool_curr_size);
const bool new_size_too_diff
= srv_buf_pool_base_size > srv_buf_pool_size * 2
@@ -2118,22 +1933,28 @@ void buf_resize_shutdown()
/** Relocate a ROW_FORMAT=COMPRESSED block in the LRU list and
buf_pool.page_hash.
The caller must relocate bpage->list.
-@param bpage BUF_BLOCK_ZIP_PAGE block
+@param bpage ROW_FORMAT=COMPRESSED only block
@param dpage destination control block */
static void buf_relocate(buf_page_t *bpage, buf_page_t *dpage)
{
- const ulint fold= bpage->id().fold();
- ut_ad(bpage->state() == BUF_BLOCK_ZIP_PAGE);
+ const page_id_t id{bpage->id()};
+ buf_pool_t::hash_chain &chain= buf_pool.page_hash.cell_get(id.fold());
+ ut_ad(!bpage->frame);
mysql_mutex_assert_owner(&buf_pool.mutex);
- ut_ad(buf_pool.hash_lock_get(bpage->id())->is_write_locked());
- ut_a(bpage->io_fix() == BUF_IO_NONE);
- ut_a(!bpage->buf_fix_count());
- ut_ad(bpage == buf_pool.page_hash_get_low(bpage->id(), fold));
+ ut_ad(buf_pool.page_hash.lock_get(chain).is_write_locked());
+ ut_ad(bpage == buf_pool.page_hash.get(id, chain));
ut_ad(!buf_pool.watch_is_sentinel(*bpage));
- ut_ad(bpage->state() == BUF_BLOCK_ZIP_PAGE);
+ ut_d(const auto state= bpage->state());
+ ut_ad(state >= buf_page_t::FREED);
+ ut_ad(state <= buf_page_t::READ_FIX);
+ ut_ad(bpage->lock.is_write_locked());
+ const auto frame= dpage->frame;
+ dpage->lock.free();
new (dpage) buf_page_t(*bpage);
+ dpage->frame= frame;
+
/* Important that we adjust the hazard pointer before
removing bpage from LRU list. */
if (buf_page_t *b= buf_pool.LRU_remove(bpage))
@@ -2163,39 +1984,28 @@ static void buf_relocate(buf_page_t *bpage, buf_page_t *dpage)
ut_d(CheckInLRUList::validate());
- /* relocate buf_pool.page_hash */
- ut_ad(bpage->in_page_hash);
- ut_ad(dpage->in_page_hash);
- ut_d(bpage->in_page_hash= false);
- HASH_REPLACE(buf_page_t, hash, &buf_pool.page_hash, fold, bpage, dpage);
+ buf_pool.page_hash.replace(chain, bpage, dpage);
}
-/** Register a watch for a page identifier. The caller must hold an
-exclusive page hash latch. The *hash_lock may be released,
-relocated, and reacquired.
-@param id page identifier
-@param hash_lock exclusively held page_hash latch
-@return a buffer pool block corresponding to id
-@retval nullptr if the block was not present, and a watch was installed */
-inline buf_page_t *buf_pool_t::watch_set(const page_id_t id,
- page_hash_latch **hash_lock)
+buf_page_t *buf_pool_t::watch_set(const page_id_t id,
+ buf_pool_t::hash_chain &chain)
{
- const ulint fold= id.fold();
- ut_ad(*hash_lock == page_hash.lock_get(fold));
- ut_ad((*hash_lock)->is_write_locked());
+ ut_ad(&chain == &page_hash.cell_get(id.fold()));
+ page_hash.lock_get(chain).lock();
-retry:
- if (buf_page_t *bpage= page_hash_get_low(id, fold))
+ buf_page_t *bpage= page_hash.get(id, chain);
+
+ if (bpage)
{
- if (!watch_is_sentinel(*bpage))
- /* The page was loaded meanwhile. */
- return bpage;
- /* Add to an existing watch. */
+got_block:
bpage->fix();
- return nullptr;
+ if (watch_is_sentinel(*bpage))
+ bpage= nullptr;
+ page_hash.lock_get(chain).unlock();
+ return bpage;
}
- (*hash_lock)->write_unlock();
+ page_hash.lock_get(chain).unlock();
/* Allocate a watch[] and then try to insert it into the page_hash. */
mysql_mutex_lock(&mutex);
@@ -2208,93 +2018,102 @@ retry:
ut_ad(!w->oldest_modification());
ut_ad(!w->zip.data);
ut_ad(!w->in_zip_hash);
- if (w->state() == BUF_BLOCK_ZIP_PAGE)
+ static_assert(buf_page_t::NOT_USED == 0, "efficiency");
+ if (ut_d(auto s=) w->state())
+ {
/* This watch may be in use for some other page. */
+ ut_ad(s >= buf_page_t::UNFIXED);
continue;
- ut_ad(w->state() == BUF_BLOCK_NOT_USED);
- ut_ad(!w->buf_fix_count());
+ }
/* w is pointing to watch[], which is protected by mutex.
Normally, buf_page_t::id for objects that are reachable by
- page_hash_get_low(id, fold) are protected by hash_lock. */
- w->set_state(BUF_BLOCK_ZIP_PAGE);
+ page_hash.get(id, chain) are protected by hash_lock. */
+ w->set_state(buf_page_t::UNFIXED + 1);
w->id_= id;
- *hash_lock= page_hash.lock_get(fold);
-
- buf_page_t *bpage= page_hash_get_low(id, fold);
+ page_hash.lock_get(chain).lock();
+ bpage= page_hash.get(id, chain);
if (UNIV_LIKELY_NULL(bpage))
{
- w->set_state(BUF_BLOCK_NOT_USED);
- *hash_lock= page_hash.lock_get(fold);
- (*hash_lock)->write_lock();
+ w->set_state(buf_page_t::NOT_USED);
mysql_mutex_unlock(&mutex);
- goto retry;
+ goto got_block;
}
- (*hash_lock)->write_lock();
- ut_ad(!w->buf_fix_count_);
- w->buf_fix_count_= 1;
- ut_ad(!w->in_page_hash);
- ut_d(w->in_page_hash= true);
- HASH_INSERT(buf_page_t, hash, &page_hash, fold, w);
+ ut_ad(w->state() == buf_page_t::UNFIXED + 1);
+ buf_pool.page_hash.append(chain, w);
mysql_mutex_unlock(&mutex);
+ page_hash.lock_get(chain).unlock();
return nullptr;
}
ut_error;
- mysql_mutex_unlock(&mutex);
- return nullptr;
}
/** Stop watching whether a page has been read in.
watch_set(id) must have returned nullptr before.
-@param id page identifier */
-void buf_pool_t::watch_unset(const page_id_t id)
+@param id page identifier
+@param chain unlocked hash table chain */
+TRANSACTIONAL_TARGET
+void buf_pool_t::watch_unset(const page_id_t id, buf_pool_t::hash_chain &chain)
{
mysql_mutex_assert_not_owner(&mutex);
- const ulint fold= id.fold();
- page_hash_latch *hash_lock= page_hash.lock<true>(fold);
- /* The page must exist because watch_set() increments buf_fix_count. */
- buf_page_t *w= page_hash_get_low(id, fold);
- ut_ad(w->in_page_hash);
- const bool must_remove= watch_is_sentinel(*w) && w->buf_fix_count() == 1;
- if (!must_remove)
- w->unfix();
- hash_lock->write_unlock();
-
- if (must_remove)
+ buf_page_t *w;
{
- const auto old= w;
- /* The following is based on buf_pool_t::watch_remove(). */
- mysql_mutex_lock(&mutex);
- w= page_hash_get_low(id, fold);
- page_hash_latch *hash_lock= buf_pool.page_hash.lock_get(fold);
- hash_lock->write_lock();
- if (w->unfix() == 0 && w == old)
+ transactional_lock_guard<page_hash_latch> g{page_hash.lock_get(chain)};
+ /* The page must exist because watch_set() did fix(). */
+ w= page_hash.get(id, chain);
+ ut_ad(w->in_page_hash);
+ if (!watch_is_sentinel(*w))
{
- ut_ad(w->in_page_hash);
- ut_d(w->in_page_hash= false);
- HASH_DELETE(buf_page_t, hash, &page_hash, fold, w);
- // Now that the watch is detached from page_hash, release it to watch[].
+ no_watch:
+ w->unfix();
+ w= nullptr;
+ }
+ else
+ {
+ const auto state= w->state();
+ ut_ad(~buf_page_t::LRU_MASK & state);
+ ut_ad(state >= buf_page_t::UNFIXED + 1);
+ if (state != buf_page_t::UNFIXED + 1)
+ goto no_watch;
+ }
+ }
+
+ if (!w)
+ return;
+
+ const auto old= w;
+ /* The following is based on buf_pool_t::watch_remove(). */
+ mysql_mutex_lock(&mutex);
+ w= page_hash.get(id, chain);
+
+ {
+ transactional_lock_guard<page_hash_latch> g
+ {buf_pool.page_hash.lock_get(chain)};
+ auto f= w->unfix();
+ ut_ad(f < buf_page_t::READ_FIX || w != old);
+
+ if (f == buf_page_t::UNFIXED && w == old)
+ {
+ page_hash.remove(chain, w);
+ // Now that w is detached from page_hash, release it to watch[].
ut_ad(w->id_ == id);
- ut_ad(!w->buf_fix_count());
- ut_ad(w->state() == BUF_BLOCK_ZIP_PAGE);
- w->set_state(BUF_BLOCK_NOT_USED);
+ ut_ad(!w->frame);
+ ut_ad(!w->zip.data);
+ w->set_state(buf_page_t::NOT_USED);
}
- hash_lock->write_unlock();
- mysql_mutex_unlock(&mutex);
}
+
+ mysql_mutex_unlock(&mutex);
}
-/** Mark the page status as FREED for the given tablespace id and
-page number. If the page is not in buffer pool then ignore it.
+/** Mark the page status as FREED for the given tablespace and page number.
@param[in,out] space tablespace
@param[in] page page number
-@param[in,out] mtr mini-transaction
-@param[in] file file name
-@param[in] line line where called */
-void buf_page_free(fil_space_t *space, uint32_t page, mtr_t *mtr,
- const char *file, unsigned line)
+@param[in,out] mtr mini-transaction */
+TRANSACTIONAL_TARGET
+void buf_page_free(fil_space_t *space, uint32_t page, mtr_t *mtr)
{
ut_ad(mtr);
ut_ad(mtr->is_active());
@@ -2306,124 +2125,138 @@ void buf_page_free(fil_space_t *space, uint32_t page, mtr_t *mtr,
)
mtr->add_freed_offset(space, page);
- buf_pool.stat.n_page_gets++;
+ ++buf_pool.stat.n_page_gets;
const page_id_t page_id(space->id, page);
- const ulint fold= page_id.fold();
- page_hash_latch *hash_lock= buf_pool.page_hash.lock<false>(fold);
- if (buf_block_t *block= reinterpret_cast<buf_block_t*>
- (buf_pool.page_hash_get_low(page_id, fold)))
+ buf_pool_t::hash_chain &chain= buf_pool.page_hash.cell_get(page_id.fold());
+ uint32_t fix;
+ buf_block_t *block;
{
- if (block->page.state() != BUF_BLOCK_FILE_PAGE)
- /* FIXME: convert, but avoid buf_zip_decompress() */;
- else
- {
- buf_block_buf_fix_inc(block, file, line);
- ut_ad(block->page.buf_fix_count());
- hash_lock->read_unlock();
+ transactional_shared_lock_guard<page_hash_latch> g
+ {buf_pool.page_hash.lock_get(chain)};
+ block= reinterpret_cast<buf_block_t*>
+ (buf_pool.page_hash.get(page_id, chain));
+ if (!block || !block->page.frame)
+ /* FIXME: convert ROW_FORMAT=COMPRESSED, without buf_zip_decompress() */
+ return;
+ /* To avoid a deadlock with buf_LRU_free_page() of some other page
+ and buf_page_write_complete() of this page, we must not wait for a
+ page latch while holding a page_hash latch. */
+ fix= block->page.fix();
+ }
- mtr->memo_push(block, MTR_MEMO_PAGE_X_FIX);
- rw_lock_x_lock_inline(&block->lock, 0, file, line);
- buf_block_dbg_add_level(block, SYNC_NO_ORDER_CHECK);
+ if (UNIV_UNLIKELY(fix < buf_page_t::UNFIXED))
+ {
+ block->page.unfix();
+ return;
+ }
+ block->page.lock.x_lock();
#ifdef BTR_CUR_HASH_ADAPT
- if (block->index)
- btr_search_drop_page_hash_index(block, false);
+ if (block->index)
+ btr_search_drop_page_hash_index(block, false);
#endif /* BTR_CUR_HASH_ADAPT */
-
- block->page.status= buf_page_t::FREED;
- return;
- }
- }
-
- hash_lock->read_unlock();
+ block->page.set_freed(block->page.state());
+ mtr->memo_push(block, MTR_MEMO_PAGE_X_MODIFY);
}
/** Get read access to a compressed page (usually of type
FIL_PAGE_TYPE_ZBLOB or FIL_PAGE_TYPE_ZBLOB2).
-The page must be released with buf_page_release_zip().
+The page must be released with unfix().
NOTE: the page is not protected by any latch. Mutual exclusion has to
be implemented at a higher level. In other words, all possible
accesses to a given page through this function must be protected by
the same set of mutexes or latches.
-@param[in] page_id page id
-@param[in] zip_size ROW_FORMAT=COMPRESSED page size
-@return pointer to the block */
+@param page_id page identifier
+@param zip_size ROW_FORMAT=COMPRESSED page size in bytes
+@return pointer to the block, s-latched */
+TRANSACTIONAL_TARGET
buf_page_t* buf_page_get_zip(const page_id_t page_id, ulint zip_size)
{
ut_ad(zip_size);
ut_ad(ut_is_2pow(zip_size));
- buf_pool.stat.n_page_gets++;
+ ++buf_pool.stat.n_page_gets;
- bool discard_attempted= false;
- const ulint fold= page_id.fold();
+ buf_pool_t::hash_chain &chain= buf_pool.page_hash.cell_get(page_id.fold());
+ page_hash_latch &hash_lock= buf_pool.page_hash.lock_get(chain);
buf_page_t *bpage;
- page_hash_latch *hash_lock;
- for (;;)
- {
lookup:
- bpage= buf_pool.page_hash_get_locked<false>(page_id, fold, &hash_lock);
- if (bpage)
- break;
-
- dberr_t err= buf_read_page(page_id, zip_size);
-
- if (UNIV_UNLIKELY(err != DB_SUCCESS))
+ for (bool discard_attempted= false;;)
+ {
+#ifndef NO_ELISION
+ if (xbegin())
{
- ib::error() << "Reading compressed page " << page_id
- << " failed with error: " << err;
- goto err_exit;
+ if (hash_lock.is_locked())
+ xabort();
+ bpage= buf_pool.page_hash.get(page_id, chain);
+ if (!bpage || buf_pool.watch_is_sentinel(*bpage))
+ {
+ xend();
+ goto must_read_page;
+ }
+ if (!bpage->zip.data)
+ {
+ /* There is no ROW_FORMAT=COMPRESSED page. */
+ xend();
+ return nullptr;
+ }
+ if (discard_attempted || !bpage->frame)
+ {
+ if (!bpage->lock.s_lock_try())
+ xabort();
+ xend();
+ break;
+ }
+ xend();
}
+ else
+#endif
+ {
+ hash_lock.lock_shared();
+ bpage= buf_pool.page_hash.get(page_id, chain);
+ if (!bpage || buf_pool.watch_is_sentinel(*bpage))
+ {
+ hash_lock.unlock_shared();
+ goto must_read_page;
+ }
-#ifdef UNIV_DEBUG
- if (!(++buf_dbg_counter % 5771)) buf_pool.validate();
-#endif /* UNIV_DEBUG */
- }
-
- ut_ad(hash_lock->is_read_locked());
+ ut_ad(bpage->in_file());
+ ut_ad(page_id == bpage->id());
- if (!bpage->zip.data)
- {
- /* There is no compressed page. */
-err_exit:
- hash_lock->read_unlock();
- return nullptr;
- }
+ if (!bpage->zip.data)
+ {
+ /* There is no ROW_FORMAT=COMPRESSED page. */
+ hash_lock.unlock_shared();
+ return nullptr;
+ }
- ut_ad(!buf_pool.watch_is_sentinel(*bpage));
+ 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();
+ hash_lock.unlock_shared();
+ break;
+ }
- switch (bpage->state()) {
- case BUF_BLOCK_ZIP_PAGE:
- bpage->fix();
- goto got_block;
- case BUF_BLOCK_FILE_PAGE:
- /* Discard the uncompressed page frame if possible. */
- if (!discard_attempted)
- {
- discard_attempted= true;
- hash_lock->read_unlock();
- mysql_mutex_lock(&buf_pool.mutex);
- if (buf_page_t *bpage= buf_pool.page_hash_get_low(page_id, fold))
- buf_LRU_free_page(bpage, false);
- mysql_mutex_unlock(&buf_pool.mutex);
- goto lookup;
+ hash_lock.unlock_shared();
}
- buf_block_buf_fix_inc(reinterpret_cast<buf_block_t*>(bpage),
- __FILE__, __LINE__);
- goto got_block;
- default:
- break;
+ discard_attempted= true;
+ mysql_mutex_lock(&buf_pool.mutex);
+ if (buf_page_t *bpage= buf_pool.page_hash.get(page_id, chain))
+ buf_LRU_free_page(bpage, false);
+ mysql_mutex_unlock(&buf_pool.mutex);
}
- ut_error;
- goto err_exit;
-
-got_block:
- bool must_read= bpage->io_fix() == BUF_IO_READ;
- hash_lock->read_unlock();
-
- DBUG_ASSERT(bpage->status != buf_page_t::FREED);
+ {
+ ut_d(const auto s=) bpage->fix();
+ ut_ad(s >= buf_page_t::UNFIXED);
+ ut_ad(s < buf_page_t::READ_FIX || s >= buf_page_t::WRITE_FIX);
+ }
bpage->set_accessed();
buf_page_make_young_if_needed(bpage);
@@ -2431,15 +2264,18 @@ got_block:
#ifdef UNIV_DEBUG
if (!(++buf_dbg_counter % 5771)) buf_pool.validate();
#endif /* UNIV_DEBUG */
- ut_ad(bpage->buf_fix_count());
- ut_ad(bpage->in_file());
-
- if (must_read)
- /* Let us wait until the read operation completes */
- while (bpage->io_fix() == BUF_IO_READ)
- os_thread_sleep(WAIT_FOR_READ);
-
return bpage;
+
+must_read_page:
+ switch (dberr_t err= buf_read_page(page_id, zip_size)) {
+ case DB_SUCCESS:
+ case DB_SUCCESS_LOCKED_REC:
+ goto lookup;
+ default:
+ ib::error() << "Reading compressed page " << page_id
+ << " failed with error: " << err;
+ return nullptr;
+ }
}
/********************************************************************//**
@@ -2495,15 +2331,9 @@ buf_zip_decompress(
<< block->page.id() << ": stored: "
<< mach_read_from_4(frame + FIL_PAGE_SPACE_OR_CHKSUM)
<< ", crc32: "
- << page_zip_calc_checksum(
- frame, size, SRV_CHECKSUM_ALGORITHM_CRC32)
- << " innodb: "
- << page_zip_calc_checksum(
- frame, size, SRV_CHECKSUM_ALGORITHM_INNODB)
- << ", none: "
- << page_zip_calc_checksum(
- frame, size, SRV_CHECKSUM_ALGORITHM_NONE)
- << " (algorithm: " << srv_checksum_algorithm << ")";
+ << page_zip_calc_checksum(frame, size, false)
+ << " adler32: "
+ << page_zip_calc_checksum(frame, size, true);
goto err_exit;
}
@@ -2511,7 +2341,8 @@ buf_zip_decompress(
case FIL_PAGE_INDEX:
case FIL_PAGE_RTREE:
if (page_zip_decompress(&block->page.zip,
- block->frame, TRUE)) {
+ block->page.frame, TRUE)) {
+func_exit:
if (space) {
space->release();
}
@@ -2530,12 +2361,8 @@ buf_zip_decompress(
case FIL_PAGE_TYPE_ZBLOB:
case FIL_PAGE_TYPE_ZBLOB2:
/* Copy to uncompressed storage. */
- memcpy(block->frame, frame, block->zip_size());
- if (space) {
- space->release();
- }
-
- return(TRUE);
+ memcpy(block->page.frame, frame, block->zip_size());
+ goto func_exit;
}
ib::error() << "Unknown compressed page type "
@@ -2550,91 +2377,19 @@ err_exit:
}
if (space) {
- if (encrypted) {
- dict_set_encrypted_by_space(space);
- } else {
- dict_set_corrupted_by_space(space);
- }
-
space->release();
}
return(FALSE);
}
-/** Wait for the block to be read in.
-@param[in] block The block to check */
-static
-void
-buf_wait_for_read(
- buf_block_t* block)
-{
- /* Note:
-
- We are using the block->lock to check for IO state.
- We set the IO_READ state under the protection of the hash_lock.
- This is safe because another thread can only
- access the block (and check for IO state) after the block has been
- added to the page hashtable. */
-
- while (block->page.io_fix() == BUF_IO_READ) {
- rw_lock_s_lock(&block->lock);
- rw_lock_s_unlock(&block->lock);
- }
-}
-
-/** Lock the page with the given latch type.
-@param[in,out] block block to be locked
-@param[in] rw_latch RW_S_LATCH, RW_X_LATCH, RW_NO_LATCH
-@param[in] mtr mini-transaction
-@param[in] file file name
-@param[in] line line where called
-@return pointer to locked block */
-static buf_block_t* buf_page_mtr_lock(buf_block_t *block,
- ulint rw_latch,
- mtr_t* mtr,
- const char *file,
- unsigned line)
-{
- mtr_memo_type_t fix_type;
- switch (rw_latch)
- {
- case RW_NO_LATCH:
- fix_type= MTR_MEMO_BUF_FIX;
- goto done;
- case RW_S_LATCH:
- rw_lock_s_lock_inline(&block->lock, 0, file, line);
- fix_type= MTR_MEMO_PAGE_S_FIX;
- break;
- case RW_SX_LATCH:
- rw_lock_sx_lock_inline(&block->lock, 0, file, line);
- fix_type= MTR_MEMO_PAGE_SX_FIX;
- break;
- default:
- ut_ad(rw_latch == RW_X_LATCH);
- rw_lock_x_lock_inline(&block->lock, 0, file, line);
- fix_type= MTR_MEMO_PAGE_X_FIX;
- break;
- }
-
-#ifdef BTR_CUR_HASH_ADAPT
- btr_search_drop_page_hash_index(block, true);
-#endif /* BTR_CUR_HASH_ADAPT */
-
-done:
- mtr_memo_push(mtr, block, fix_type);
- return block;
-}
-
/** Low level function used to get access to a database page.
@param[in] page_id page id
@param[in] zip_size ROW_FORMAT=COMPRESSED page size, or 0
@param[in] rw_latch RW_S_LATCH, RW_X_LATCH, RW_NO_LATCH
@param[in] guess guessed block or NULL
@param[in] mode BUF_GET, BUF_GET_IF_IN_POOL,
-BUF_PEEK_IF_IN_POOL, BUF_GET_NO_LATCH, or BUF_GET_IF_IN_POOL_OR_WATCH
-@param[in] file file name
-@param[in] line line where called
+BUF_PEEK_IF_IN_POOL, or BUF_GET_IF_IN_POOL_OR_WATCH
@param[in] mtr mini-transaction
@param[out] err DB_SUCCESS or error code
@param[in] allow_ibuf_merge Allow change buffer merge to happen
@@ -2642,6 +2397,7 @@ while reading the page from file
then it makes sure that it does merging of change buffer changes while
reading the page from file.
@return pointer to the block or NULL */
+TRANSACTIONAL_TARGET
buf_block_t*
buf_page_get_low(
const page_id_t page_id,
@@ -2649,28 +2405,19 @@ buf_page_get_low(
ulint rw_latch,
buf_block_t* guess,
ulint mode,
- const char* file,
- unsigned line,
mtr_t* mtr,
dberr_t* err,
bool allow_ibuf_merge)
{
- buf_block_t* block;
unsigned access_time;
ulint retries = 0;
- const ulint fold = page_id.fold();
- ut_ad((mtr == NULL) == (mode == BUF_EVICT_IF_IN_POOL));
ut_ad(!mtr || mtr->is_active());
+ ut_ad(mtr || mode == BUF_PEEK_IF_IN_POOL);
ut_ad((rw_latch == RW_S_LATCH)
|| (rw_latch == RW_X_LATCH)
|| (rw_latch == RW_SX_LATCH)
|| (rw_latch == RW_NO_LATCH));
- ut_ad(!allow_ibuf_merge
- || mode == BUF_GET
- || mode == BUF_GET_POSSIBLY_FREED
- || mode == BUF_GET_IF_IN_POOL
- || mode == BUF_GET_IF_IN_POOL_OR_WATCH);
if (err) {
*err = DB_SUCCESS;
@@ -2678,26 +2425,18 @@ buf_page_get_low(
#ifdef UNIV_DEBUG
switch (mode) {
- case BUF_EVICT_IF_IN_POOL:
- /* After DISCARD TABLESPACE, the tablespace would not exist,
- but in IMPORT TABLESPACE, PageConverter::operator() must
- replace any old pages, which were not evicted during DISCARD.
- Skip the assertion on space_page_size. */
+ default:
+ ut_ad(!allow_ibuf_merge);
+ ut_ad(mode == BUF_PEEK_IF_IN_POOL);
break;
- case BUF_PEEK_IF_IN_POOL:
+ case BUF_GET_POSSIBLY_FREED:
case BUF_GET_IF_IN_POOL:
/* The caller may pass a dummy page size,
because it does not really matter. */
break;
- default:
- ut_error;
- case BUF_GET_POSSIBLY_FREED:
- break;
- case BUF_GET_NO_LATCH:
- ut_ad(rw_latch == RW_NO_LATCH);
- /* fall through */
case BUF_GET:
case BUF_GET_IF_IN_POOL_OR_WATCH:
+ ut_ad(!mtr->is_freeing_tree());
fil_space_t* s = fil_space_get(page_id.space());
ut_ad(s);
ut_ad(s->zip_size() == zip_size);
@@ -2705,487 +2444,459 @@ buf_page_get_low(
#endif /* UNIV_DEBUG */
ut_ad(!mtr || !ibuf_inside(mtr)
- || ibuf_page_low(page_id, zip_size, FALSE, file, line, NULL));
+ || ibuf_page_low(page_id, zip_size, FALSE, NULL));
- buf_pool.stat.n_page_gets++;
-loop:
- buf_block_t* fix_block;
- block = guess;
+ ++buf_pool.stat.n_page_gets;
- page_hash_latch* hash_lock = buf_pool.page_hash.lock<false>(fold);
+ auto& chain= buf_pool.page_hash.cell_get(page_id.fold());
+ page_hash_latch& hash_lock = buf_pool.page_hash.lock_get(chain);
+loop:
+ buf_block_t* block = guess;
+ uint32_t state;
if (block) {
-
- /* If the guess is a compressed page descriptor that
- has been allocated by buf_page_alloc_descriptor(),
- it may have been freed by buf_relocate(). */
-
- if (!buf_pool.is_uncompressed(block)
- || page_id != block->page.id()
- || block->page.state() != BUF_BLOCK_FILE_PAGE) {
- /* Our guess was bogus or things have changed
- since. */
- guess = nullptr;
- goto lookup;
- } else {
+ transactional_shared_lock_guard<page_hash_latch> g{hash_lock};
+ if (buf_pool.is_uncompressed(block)
+ && page_id == block->page.id()) {
ut_ad(!block->page.in_zip_hash);
- }
- } else {
-lookup:
- block = reinterpret_cast<buf_block_t*>(
- buf_pool.page_hash_get_low(page_id, fold));
- }
-
- if (!block || buf_pool.watch_is_sentinel(block->page)) {
- hash_lock->read_unlock();
- block = nullptr;
- }
-
- if (UNIV_UNLIKELY(!block)) {
- /* Page not in buf_pool: needs to be read from file */
- if (mode == BUF_GET_IF_IN_POOL_OR_WATCH) {
- hash_lock = buf_pool.page_hash.lock<true>(fold);
-
- if (buf_page_t *bpage= buf_pool.watch_set(
- page_id, &hash_lock)) {
- /* We can release hash_lock after we
- increment the fix count to make
- sure that no state change takes place. */
- bpage->fix();
- hash_lock->write_unlock();
- block = reinterpret_cast<buf_block_t*>(bpage);
- fix_block = block;
+ state = block->page.state();
+ /* Ignore guesses that point to read-fixed blocks.
+ We can only avoid a race condition by
+ looking up the block via buf_pool.page_hash. */
+ if ((state >= buf_page_t::FREED
+ && state < buf_page_t::READ_FIX)
+ || state >= buf_page_t::WRITE_FIX) {
+ state = block->page.fix();
goto got_block;
}
-
- hash_lock->write_unlock();
}
+ }
- switch (mode) {
- case BUF_GET_IF_IN_POOL:
- case BUF_GET_IF_IN_POOL_OR_WATCH:
- case BUF_PEEK_IF_IN_POOL:
- case BUF_EVICT_IF_IN_POOL:
- return(NULL);
- }
-
- /* The call path is buf_read_page() ->
- buf_read_page_low() (fil_space_t::io()) ->
- buf_page_read_complete() ->
- buf_decrypt_after_read(). Here fil_space_t* is used
- and we decrypt -> buf_page_check_corrupt() where page
- checksums are compared. Decryption, decompression as
- well as error handling takes place at a lower level.
- Here we only need to know whether the page really is
- corrupted, or if an encrypted page with a valid
- checksum cannot be decypted. */
-
- dberr_t local_err = buf_read_page(page_id, zip_size);
-
- if (local_err == DB_SUCCESS) {
- buf_read_ahead_random(page_id, zip_size,
- ibuf_inside(mtr));
-
- retries = 0;
- } else if (mode == BUF_GET_POSSIBLY_FREED) {
- if (err) {
- *err = local_err;
- }
- return NULL;
- } else if (retries < BUF_PAGE_READ_MAX_RETRIES) {
- ++retries;
-
- DBUG_EXECUTE_IF(
- "innodb_page_corruption_retries",
- retries = BUF_PAGE_READ_MAX_RETRIES;
- );
- } else {
- if (err) {
- *err = local_err;
- }
-
- /* Pages whose encryption key is unavailable or used
- key, encryption algorithm or encryption method is
- incorrect are marked as encrypted in
- buf_page_check_corrupt(). Unencrypted page could be
- corrupted in a way where the key_id field is
- nonzero. There is no checksum on field
- FIL_PAGE_FILE_FLUSH_LSN_OR_KEY_VERSION. */
- if (local_err == DB_DECRYPTION_FAILED) {
- return (NULL);
- }
-
- if (local_err == DB_PAGE_CORRUPTED
- && srv_force_recovery) {
- return NULL;
- }
-
- /* Try to set table as corrupted instead of
- asserting. */
- if (page_id.space() == TRX_SYS_SPACE) {
- } else if (page_id.space() == SRV_TMP_SPACE_ID) {
- } else if (fil_space_t* space= fil_space_t::get(
- page_id.space())) {
- bool set = dict_set_corrupted_by_space(space);
- space->release();
- if (set) {
- return NULL;
- }
- }
+ guess = nullptr;
+
+ /* A memory transaction would frequently be aborted here. */
+ hash_lock.lock_shared();
+ block = reinterpret_cast<buf_block_t*>(
+ buf_pool.page_hash.get(page_id, chain));
+ if (UNIV_LIKELY(block
+ && !buf_pool.watch_is_sentinel(block->page))) {
+ state = block->page.fix();
+ hash_lock.unlock_shared();
+ goto got_block;
+ }
+ hash_lock.unlock_shared();
- if (local_err == DB_IO_ERROR) {
- return NULL;
- }
+ /* Page not in buf_pool: needs to be read from file */
+ switch (mode) {
+ case BUF_GET_IF_IN_POOL:
+ case BUF_PEEK_IF_IN_POOL:
+ return nullptr;
+ case BUF_GET_IF_IN_POOL_OR_WATCH:
+ /* Buffer-fixing inside watch_set() will prevent eviction */
+ block = reinterpret_cast<buf_block_t*>
+ (buf_pool.watch_set(page_id, chain));
- ib::fatal() << "Unable to read page " << page_id
- << " into the buffer pool after "
- << BUF_PAGE_READ_MAX_RETRIES
- << ". The most probable cause"
- " of this error may be that the"
- " table has been corrupted."
- " See https://mariadb.com/kb/en/library/innodb-recovery-modes/";
+ if (block) {
+ state = block->page.state();
+ goto got_block_fixed;
}
-#ifdef UNIV_DEBUG
- if (!(++buf_dbg_counter % 5771)) buf_pool.validate();
-#endif /* UNIV_DEBUG */
- goto loop;
- } else {
- fix_block = block;
+ return nullptr;
}
- fix_block->fix();
- hash_lock->read_unlock();
-
-got_block:
- switch (mode) {
- default:
- ut_ad(block->zip_size() == zip_size);
+ /* The call path is buf_read_page() ->
+ buf_read_page_low() (fil_space_t::io()) ->
+ buf_page_t::read_complete() ->
+ buf_decrypt_after_read(). Here fil_space_t* is used
+ and we decrypt -> buf_page_check_corrupt() where page
+ checksums are compared. Decryption, decompression as
+ well as error handling takes place at a lower level.
+ Here we only need to know whether the page really is
+ corrupted, or if an encrypted page with a valid
+ checksum cannot be decypted. */
+
+ switch (dberr_t local_err = buf_read_page(page_id, zip_size)) {
+ case DB_SUCCESS:
+ case DB_SUCCESS_LOCKED_REC:
+ buf_read_ahead_random(page_id, zip_size, ibuf_inside(mtr));
break;
- case BUF_GET_IF_IN_POOL:
- case BUF_PEEK_IF_IN_POOL:
- case BUF_EVICT_IF_IN_POOL:
- if (fix_block->page.io_fix() == BUF_IO_READ) {
- /* The page is being read to buffer pool,
- but we cannot wait around for the read to
- complete. */
- fix_block->unfix();
- return(NULL);
+ default:
+ if (mode != BUF_GET_POSSIBLY_FREED
+ && retries++ < BUF_PAGE_READ_MAX_RETRIES) {
+ DBUG_EXECUTE_IF("intermittent_read_failure",
+ retries = BUF_PAGE_READ_MAX_RETRIES;);
+ }
+ /* fall through */
+ case DB_PAGE_CORRUPTED:
+ if (err) {
+ *err = local_err;
}
+ return nullptr;
}
- switch (UNIV_EXPECT(fix_block->page.state(), BUF_BLOCK_FILE_PAGE)) {
- case BUF_BLOCK_FILE_PAGE:
- if (fsp_is_system_temporary(page_id.space())
- && block->page.io_fix() != BUF_IO_NONE) {
- /* This suggests that the page is being flushed.
- Avoid returning reference to this page.
- Instead wait for the flush action to complete. */
- fix_block->unfix();
- os_thread_sleep(WAIT_FOR_WRITE);
- goto loop;
- }
+ ut_d(if (!(++buf_dbg_counter % 5771)) buf_pool.validate());
+ goto loop;
- if (UNIV_UNLIKELY(mode == BUF_EVICT_IF_IN_POOL)) {
-evict_from_pool:
- ut_ad(!fix_block->page.oldest_modification());
- mysql_mutex_lock(&buf_pool.mutex);
- fix_block->unfix();
+got_block:
+ ut_ad(!block->page.in_zip_hash);
+ state++;
+got_block_fixed:
+ ut_ad(state > buf_page_t::FREED);
- if (!buf_LRU_free_page(&fix_block->page, true)) {
- ut_ad(0);
+ if (state > buf_page_t::READ_FIX && state < buf_page_t::WRITE_FIX) {
+ if (mode == BUF_PEEK_IF_IN_POOL) {
+ignore_block:
+ ut_ad(mode == BUF_GET_POSSIBLY_FREED
+ || mode == BUF_PEEK_IF_IN_POOL);
+ block->unfix();
+ if (err) {
+ *err = DB_CORRUPTION;
}
-
- mysql_mutex_unlock(&buf_pool.mutex);
- return(NULL);
+ return nullptr;
}
- break;
- default:
- ut_error;
- break;
-
- case BUF_BLOCK_ZIP_PAGE:
- if (UNIV_UNLIKELY(mode == BUF_EVICT_IF_IN_POOL)) {
- goto evict_from_pool;
+ if (UNIV_UNLIKELY(!block->page.frame)) {
+ goto wait_for_unzip;
}
+ /* A read-fix is released after block->page.lock
+ in buf_page_t::read_complete() or
+ buf_pool_t::corrupted_evict(), or
+ after buf_zip_decompress() in this function. */
+ block->page.lock.s_lock();
+ state = block->page.state();
+ ut_ad(state < buf_page_t::READ_FIX
+ || state >= buf_page_t::WRITE_FIX);
+ const page_id_t id{block->page.id()};
+ block->page.lock.s_unlock();
+
+ if (UNIV_UNLIKELY(id != page_id)) {
+ ut_ad(id == page_id_t{~0ULL});
+ block->page.unfix();
+ if (++retries < BUF_PAGE_READ_MAX_RETRIES) {
+ goto loop;
+ }
- if (mode == BUF_PEEK_IF_IN_POOL) {
- /* This mode is only used for dropping an
- adaptive hash index. There cannot be an
- adaptive hash index for a compressed-only
- page, so do not bother decompressing the page. */
- fix_block->unfix();
+ if (err) {
+ *err = DB_PAGE_CORRUPTED;
+ }
- return(NULL);
+ return nullptr;
}
+ } else if (mode != BUF_PEEK_IF_IN_POOL) {
+ } else if (!mtr) {
+ ut_ad(!block->page.oldest_modification());
+ mysql_mutex_lock(&buf_pool.mutex);
+ block->unfix();
- buf_page_t* bpage = &block->page;
-
- /* Note: We have already buffer fixed this block. */
- if (bpage->buf_fix_count() > 1
- || bpage->io_fix() != BUF_IO_NONE) {
-
- /* This condition often occurs when the buffer
- is not buffer-fixed, but I/O-fixed by
- buf_page_init_for_read(). */
- fix_block->unfix();
+free_unfixed_block:
+ if (!buf_LRU_free_page(&block->page, true)) {
+ ut_ad(0);
+ }
- /* The block is buffer-fixed or I/O-fixed.
- Try again later. */
- os_thread_sleep(WAIT_FOR_READ);
+ mysql_mutex_unlock(&buf_pool.mutex);
+ return nullptr;
+ } else if (UNIV_UNLIKELY(!block->page.frame)) {
+ /* The BUF_PEEK_IF_IN_POOL mode is mainly used for dropping an
+ adaptive hash index. There cannot be an
+ adaptive hash index for a compressed-only page. */
+ goto ignore_block;
+ }
+ ut_ad(mode == BUF_GET_IF_IN_POOL || mode == BUF_PEEK_IF_IN_POOL
+ || block->zip_size() == zip_size);
+
+ if (UNIV_UNLIKELY(!block->page.frame)) {
+ if (!block->page.lock.x_lock_try()) {
+wait_for_unzip:
+ /* The page is being read or written, or
+ another thread is executing buf_zip_decompress()
+ in buf_page_get_low() on it. */
+ block->page.unfix();
+ std::this_thread::sleep_for(
+ std::chrono::microseconds(100));
goto loop;
}
- /* Buffer-fix the block so that it cannot be evicted
- or relocated while we are attempting to allocate an
- uncompressed page. */
-
- block = buf_LRU_get_free_block(false);
- buf_block_init_low(block);
+ buf_block_t *new_block = buf_LRU_get_free_block(false);
+ buf_block_init_low(new_block);
+wait_for_unfix:
mysql_mutex_lock(&buf_pool.mutex);
- hash_lock = buf_pool.page_hash.lock_get(fold);
-
- hash_lock->write_lock();
-
- /* Buffer-fixing prevents the page_hash from changing. */
- ut_ad(bpage == buf_pool.page_hash_get_low(page_id, fold));
-
- fix_block->unfix(); /* hash_lock protects us after this */
-
- if (bpage->buf_fix_count() || bpage->io_fix() != BUF_IO_NONE) {
- /* The block was buffer-fixed or I/O-fixed while
- buf_pool.mutex was not held by this thread.
- Free the block that was allocated and retry.
- This should be extremely unlikely, for example,
- if buf_page_get_zip() was invoked. */
+ page_hash_latch& hash_lock=buf_pool.page_hash.lock_get(chain);
+
+ /* It does not make sense to use
+ transactional_lock_guard here, because buf_relocate()
+ would likely make a memory transaction too large. */
+ hash_lock.lock();
+
+ /* block->page.lock implies !block->page.can_relocate() */
+ ut_ad(&block->page == buf_pool.page_hash.get(page_id, chain));
+
+ /* Wait for any other threads to release their buffer-fix
+ on the compressed-only block descriptor.
+ FIXME: Never fix() before acquiring the lock.
+ Only in buf_page_get_gen(), buf_page_get_low(), buf_page_free()
+ we are violating that principle. */
+ state = block->page.state();
+
+ switch (state) {
+ case buf_page_t::UNFIXED + 1:
+ case buf_page_t::IBUF_EXIST + 1:
+ case buf_page_t::REINIT + 1:
+ break;
+ default:
+ ut_ad(state < buf_page_t::READ_FIX);
+
+ if (state < buf_page_t::UNFIXED + 1) {
+ ut_ad(state > buf_page_t::FREED);
+ block->page.lock.x_unlock();
+ hash_lock.unlock();
+ buf_LRU_block_free_non_file_page(new_block);
+ mysql_mutex_unlock(&buf_pool.mutex);
+ goto ignore_block;
+ }
- hash_lock->write_unlock();
- buf_LRU_block_free_non_file_page(block);
mysql_mutex_unlock(&buf_pool.mutex);
-
- /* Try again */
- goto loop;
+ hash_lock.unlock();
+ std::this_thread::sleep_for(
+ std::chrono::microseconds(100));
+ goto wait_for_unfix;
}
- fix_block = block;
+ /* Ensure that another buf_page_get_low() will wait for
+ new_block->page.lock.x_unlock(). */
+ block->page.set_state(buf_page_t::READ_FIX);
- /* Move the compressed page from bpage to block,
+ /* Move the compressed page from block->page to new_block,
and uncompress it. */
- /* Note: this is the uncompressed block and it is not
- accessible by other threads yet because it is not in
- any list or hash table */
mysql_mutex_lock(&buf_pool.flush_list_mutex);
- buf_relocate(bpage, &block->page);
+ buf_relocate(&block->page, &new_block->page);
- /* Set after buf_relocate(). */
- block->page.set_buf_fix_count(1);
+ /* X-latch the block for the duration of the decompression. */
+ new_block->page.lock.x_lock();
+ ut_d(block->page.lock.x_unlock());
- buf_flush_relocate_on_flush_list(bpage, &block->page);
+ buf_flush_relocate_on_flush_list(&block->page,
+ &new_block->page);
mysql_mutex_unlock(&buf_pool.flush_list_mutex);
- /* Buffer-fix, I/O-fix, and X-latch the block
- for the duration of the decompression.
- Also add the block to the unzip_LRU list. */
- block->page.set_state(BUF_BLOCK_FILE_PAGE);
-
/* Insert at the front of unzip_LRU list */
- buf_unzip_LRU_add_block(block, FALSE);
+ buf_unzip_LRU_add_block(new_block, FALSE);
- block->page.set_io_fix(BUF_IO_READ);
- rw_lock_x_lock_inline(&block->lock, 0, file, line);
+ mysql_mutex_unlock(&buf_pool.mutex);
+ hash_lock.unlock();
- MEM_UNDEFINED(bpage, sizeof *bpage);
+#if defined SUX_LOCK_GENERIC || defined UNIV_DEBUG
+ block->page.lock.free();
+#endif
+ ut_free(reinterpret_cast<buf_page_t*>(block));
+ block = new_block;
- mysql_mutex_unlock(&buf_pool.mutex);
- hash_lock->write_unlock();
buf_pool.n_pend_unzip++;
access_time = block->page.is_accessed();
if (!access_time && !recv_no_ibuf_operations
- && ibuf_page_exists(block->page.id(), zip_size)) {
- block->page.ibuf_exist = true;
+ && ibuf_page_exists(block->page.id(), block->zip_size())) {
+ state = buf_page_t::IBUF_EXIST + 1;
}
- buf_page_free_descriptor(bpage);
-
/* Decompress the page while not holding
buf_pool.mutex. */
-
- if (!buf_zip_decompress(block, false)) {
- rw_lock_x_unlock(&fix_block->lock);
- fix_block->page.io_unfix();
- fix_block->unfix();
- --buf_pool.n_pend_unzip;
-
+ const auto ok = buf_zip_decompress(block, false);
+ --buf_pool.n_pend_unzip;
+ if (!ok) {
if (err) {
*err = DB_PAGE_CORRUPTED;
}
- return NULL;
+ mysql_mutex_lock(&buf_pool.mutex);
}
+ state = block->page.read_unfix(state);
+ block->page.lock.x_unlock();
- rw_lock_x_unlock(&block->lock);
- fix_block->page.io_unfix();
- --buf_pool.n_pend_unzip;
- break;
+ if (!ok) {
+ goto free_unfixed_block;
+ }
}
- ut_ad(block == fix_block);
- ut_ad(fix_block->page.buf_fix_count());
-
- ut_ad(fix_block->page.state() == BUF_BLOCK_FILE_PAGE);
-
#if defined UNIV_DEBUG || defined UNIV_IBUF_DEBUG
re_evict:
if (mode != BUF_GET_IF_IN_POOL
&& mode != BUF_GET_IF_IN_POOL_OR_WATCH) {
} else if (!ibuf_debug || recv_recovery_is_on()) {
} else if (fil_space_t* space = fil_space_t::get(page_id.space())) {
+ for (ulint i = 0; i < mtr->get_savepoint(); i++) {
+ if (buf_block_t* b = mtr->block_at_savepoint(i)) {
+ if (b->page.oldest_modification() > 2
+ && b->page.lock.have_any()) {
+ /* We are holding a dirty page latch
+ that would hang buf_flush_sync(). */
+ space->release();
+ goto re_evict_fail;
+ }
+ }
+ }
+
/* Try to evict the block from the buffer pool, to use the
insert buffer (change buffer) as much as possible. */
mysql_mutex_lock(&buf_pool.mutex);
- fix_block->unfix();
+ block->unfix();
/* Blocks cannot be relocated or enter or exit the
buf_pool while we are holding the buf_pool.mutex. */
- const bool evicted = buf_LRU_free_page(&fix_block->page, true);
+ const bool evicted = buf_LRU_free_page(&block->page, true);
space->release();
- if (evicted) {
- hash_lock = buf_pool.page_hash.lock_get(fold);
- hash_lock->write_lock();
- mysql_mutex_unlock(&buf_pool.mutex);
- /* We may set the watch, as it would have
- been set if the page were not in the
- buffer pool in the first place. */
- block= reinterpret_cast<buf_block_t*>(
- mode == BUF_GET_IF_IN_POOL_OR_WATCH
- ? buf_pool.watch_set(page_id, &hash_lock)
- : buf_pool.page_hash_get_low(page_id, fold));
- hash_lock->write_unlock();
-
- if (block != NULL) {
- /* Either the page has been read in or
- a watch was set on that in the window
- where we released the buf_pool.mutex
- and before we acquire the hash_lock
- above. Try again. */
- guess = block;
+ if (!evicted) {
+ block->fix();
+ }
- goto loop;
- }
+ mysql_mutex_unlock(&buf_pool.mutex);
+ if (evicted) {
+ if (mode == BUF_GET_IF_IN_POOL_OR_WATCH) {
+ buf_pool.watch_set(page_id, chain);
+ }
return(NULL);
}
- fix_block->fix();
- mysql_mutex_unlock(&buf_pool.mutex);
buf_flush_sync();
- if (fix_block->page.buf_fix_count() == 1
- && !fix_block->page.oldest_modification()) {
+ state = block->page.state();
+
+ if (state == buf_page_t::UNFIXED + 1
+ && !block->page.oldest_modification()) {
goto re_evict;
}
/* Failed to evict the page; change it directly */
}
+re_evict_fail:
#endif /* UNIV_DEBUG || UNIV_IBUF_DEBUG */
- ut_ad(fix_block->page.buf_fix_count());
-
-#ifdef UNIV_DEBUG
- /* We have already buffer fixed the page, and we are committed to
- returning this page to the caller. Register for debugging.
- Avoid debug latching if page/block belongs to system temporary
- tablespace (Not much needed for table with single threaded access.). */
- if (!fsp_is_system_temporary(page_id.space())) {
- ibool ret;
- ret = rw_lock_s_lock_nowait(
- fix_block->debug_latch, file, line);
- ut_a(ret);
- }
-#endif /* UNIV_DEBUG */
-
- /* While tablespace is reinited the indexes are already freed but the
- blocks related to it still resides in buffer pool. Trying to remove
- such blocks from buffer pool would invoke removal of AHI entries
- associated with these blocks. Logic to remove AHI entry will try to
- load the block but block is already in free state. Handle the said case
- with mode = BUF_PEEK_IF_IN_POOL that is invoked from
- "btr_search_drop_page_hash_when_freed". */
- ut_ad(mode == BUF_GET_POSSIBLY_FREED
- || mode == BUF_PEEK_IF_IN_POOL
- || fix_block->page.status != buf_page_t::FREED);
-
- const bool not_first_access = fix_block->page.set_accessed();
-
- if (mode != BUF_PEEK_IF_IN_POOL) {
- buf_page_make_young_if_needed(&fix_block->page);
+ if (UNIV_UNLIKELY(state < buf_page_t::UNFIXED)) {
+ goto ignore_block;
}
+ ut_ad((~buf_page_t::LRU_MASK) & state);
+ ut_ad(state > buf_page_t::WRITE_FIX || state < buf_page_t::READ_FIX);
#ifdef UNIV_DEBUG
if (!(++buf_dbg_counter % 5771)) buf_pool.validate();
#endif /* UNIV_DEBUG */
- ut_ad(fix_block->page.state() == BUF_BLOCK_FILE_PAGE);
+ ut_ad(block->page.frame);
- /* We have to wait here because the IO_READ state was set
- under the protection of the hash_lock and not block->lock. */
- buf_wait_for_read(fix_block);
+ if (state >= buf_page_t::UNFIXED
+ && allow_ibuf_merge
+ && fil_page_get_type(block->page.frame) == FIL_PAGE_INDEX
+ && page_is_leaf(block->page.frame)) {
+ block->page.lock.x_lock();
+ ut_ad(block->page.id() == page_id
+ || (state >= buf_page_t::READ_FIX
+ && state < buf_page_t::WRITE_FIX));
- if (fix_block->page.id() != page_id) {
- fix_block->unfix();
+#ifdef BTR_CUR_HASH_ADAPT
+ btr_search_drop_page_hash_index(block, true);
+#endif /* BTR_CUR_HASH_ADAPT */
-#ifdef UNIV_DEBUG
- if (!fsp_is_system_temporary(page_id.space())) {
- rw_lock_s_unlock(fix_block->debug_latch);
- }
-#endif /* UNIV_DEBUG */
+ dberr_t e;
- if (err) {
- *err = DB_PAGE_CORRUPTED;
+ if (UNIV_UNLIKELY(block->page.id() != page_id)) {
+page_id_mismatch:
+ state = block->page.state();
+ e = DB_CORRUPTION;
+ibuf_merge_corrupted:
+ if (err) {
+ *err = e;
+ }
+
+ if (block->page.id().is_corrupted()) {
+ buf_pool.corrupted_evict(&block->page, state);
+ }
+ return nullptr;
}
- return NULL;
- }
+ state = block->page.state();
+ ut_ad(state < buf_page_t::READ_FIX);
- if (fix_block->page.status != buf_page_t::FREED
- && allow_ibuf_merge
- && fil_page_get_type(fix_block->frame) == FIL_PAGE_INDEX
- && page_is_leaf(fix_block->frame)) {
- rw_lock_x_lock_inline(&fix_block->lock, 0, file, line);
-
- if (fix_block->page.ibuf_exist) {
- fix_block->page.ibuf_exist = false;
- ibuf_merge_or_delete_for_page(fix_block, page_id,
- zip_size);
+ if (state >= buf_page_t::IBUF_EXIST
+ && state < buf_page_t::REINIT) {
+ block->page.clear_ibuf_exist();
+ e = ibuf_merge_or_delete_for_page(block, page_id,
+ block->zip_size());
+ if (UNIV_UNLIKELY(e != DB_SUCCESS)) {
+ goto ibuf_merge_corrupted;
+ }
}
if (rw_latch == RW_X_LATCH) {
- mtr->memo_push(fix_block, MTR_MEMO_PAGE_X_FIX);
+ goto get_latch_valid;
} else {
- rw_lock_x_unlock(&fix_block->lock);
+ block->page.lock.x_unlock();
goto get_latch;
}
} else {
get_latch:
- fix_block = buf_page_mtr_lock(fix_block, rw_latch, mtr,
- file, line);
- }
+ switch (rw_latch) {
+ case RW_NO_LATCH:
+ mtr->memo_push(block, MTR_MEMO_BUF_FIX);
+ return block;
+ case RW_S_LATCH:
+ block->page.lock.s_lock();
+ ut_ad(!block->page.is_read_fixed());
+ if (UNIV_UNLIKELY(block->page.id() != page_id)) {
+ block->page.lock.s_unlock();
+ block->page.lock.x_lock();
+ goto page_id_mismatch;
+ }
+get_latch_valid:
+ mtr->memo_push(block, mtr_memo_type_t(rw_latch));
+#ifdef BTR_CUR_HASH_ADAPT
+ btr_search_drop_page_hash_index(block, true);
+#endif /* BTR_CUR_HASH_ADAPT */
+ break;
+ case RW_SX_LATCH:
+ block->page.lock.u_lock();
+ ut_ad(!block->page.is_io_fixed());
+ if (UNIV_UNLIKELY(block->page.id() != page_id)) {
+ block->page.lock.u_x_upgrade();
+ goto page_id_mismatch;
+ }
+ goto get_latch_valid;
+ default:
+ ut_ad(rw_latch == RW_X_LATCH);
+ if (block->page.lock.x_lock_upgraded()) {
+ ut_ad(block->page.id() == page_id);
+ block->unfix();
+ mtr->page_lock_upgrade(*block);
+ return block;
+ }
+ if (UNIV_UNLIKELY(block->page.id() != page_id)) {
+ goto page_id_mismatch;
+ }
+ goto get_latch_valid;
+ }
- if (!not_first_access && mode != BUF_PEEK_IF_IN_POOL) {
- /* In the case of a first access, try to apply linear
- read-ahead */
+ 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;
+ }
- buf_read_ahead_linear(page_id, zip_size, ibuf_inside(mtr));
+ 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(fix_block);
+ return block;
}
/** Get access to a database page. Buffered redo log may be applied.
@@ -3194,10 +2905,8 @@ get_latch:
@param[in] rw_latch RW_S_LATCH, RW_X_LATCH, RW_NO_LATCH
@param[in] guess guessed block or NULL
@param[in] mode BUF_GET, BUF_GET_IF_IN_POOL,
-BUF_PEEK_IF_IN_POOL, BUF_GET_NO_LATCH, or BUF_GET_IF_IN_POOL_OR_WATCH
-@param[in] file file name
-@param[in] line line where called
-@param[in] mtr mini-transaction
+BUF_PEEK_IF_IN_POOL, or BUF_GET_IF_IN_POOL_OR_WATCH
+@param[in,out] mtr mini-transaction, or NULL
@param[out] err DB_SUCCESS or error code
@param[in] allow_ibuf_merge Allow change buffer merge while
reading the pages from file.
@@ -3209,194 +2918,204 @@ buf_page_get_gen(
ulint rw_latch,
buf_block_t* guess,
ulint mode,
- const char* file,
- unsigned line,
mtr_t* mtr,
dberr_t* err,
bool allow_ibuf_merge)
{
- if (buf_block_t *block= recv_sys.recover(page_id))
+ buf_block_t *block= recv_sys.recover(page_id);
+ if (UNIV_LIKELY(!block))
+ return buf_page_get_low(page_id, zip_size, rw_latch,
+ guess, mode, mtr, err, allow_ibuf_merge);
+ else if (UNIV_UNLIKELY(block == reinterpret_cast<buf_block_t*>(-1)))
{
- block->fix();
- ut_ad(rw_lock_s_lock_nowait(block->debug_latch, file, line));
+ corrupted:
if (err)
- *err= DB_SUCCESS;
- const bool must_merge= allow_ibuf_merge &&
- ibuf_page_exists(page_id, block->zip_size());
- if (block->page.status == buf_page_t::FREED)
- ut_ad(mode == BUF_GET_POSSIBLY_FREED || mode == BUF_PEEK_IF_IN_POOL);
- else if (must_merge && fil_page_get_type(block->frame) == FIL_PAGE_INDEX &&
- page_is_leaf(block->frame))
+ *err= DB_CORRUPTION;
+ return nullptr;
+ }
+ /* Recovery is a special case; we fix() before acquiring lock. */
+ auto s= block->page.fix();
+ ut_ad(s >= buf_page_t::FREED);
+ /* The block may be write-fixed at this point because we are not
+ holding a lock, but it must not be read-fixed. */
+ ut_ad(s < buf_page_t::READ_FIX || s >= buf_page_t::WRITE_FIX);
+ if (err)
+ *err= DB_SUCCESS;
+ const bool must_merge= allow_ibuf_merge &&
+ ibuf_page_exists(page_id, block->zip_size());
+ if (s < buf_page_t::UNFIXED)
+ {
+ got_freed_page:
+ ut_ad(mode == BUF_GET_POSSIBLY_FREED || mode == BUF_PEEK_IF_IN_POOL);
+ mysql_mutex_lock(&buf_pool.mutex);
+ block->page.unfix();
+ buf_LRU_free_page(&block->page, true);
+ mysql_mutex_unlock(&buf_pool.mutex);
+ goto corrupted;
+ }
+ else if (must_merge &&
+ fil_page_get_type(block->page.frame) == FIL_PAGE_INDEX &&
+ page_is_leaf(block->page.frame))
+ {
+ block->page.lock.x_lock();
+ s= block->page.state();
+ ut_ad(s > buf_page_t::FREED);
+ ut_ad(s < buf_page_t::READ_FIX);
+ if (s < buf_page_t::UNFIXED)
{
- rw_lock_x_lock_inline(&block->lock, 0, file, line);
- block->page.ibuf_exist= false;
- ibuf_merge_or_delete_for_page(block, page_id, block->zip_size());
-
- if (rw_latch == RW_X_LATCH)
+ block->page.lock.x_unlock();
+ goto got_freed_page;
+ }
+ else
+ {
+ if (block->page.is_ibuf_exist())
+ block->page.clear_ibuf_exist();
+ if (dberr_t e=
+ ibuf_merge_or_delete_for_page(block, page_id, block->zip_size()))
{
- mtr->memo_push(block, MTR_MEMO_PAGE_X_FIX);
- return block;
+ if (err)
+ *err= e;
+ buf_pool.corrupted_evict(&block->page, s);
+ return nullptr;
}
- rw_lock_x_unlock(&block->lock);
}
- block= buf_page_mtr_lock(block, rw_latch, mtr, file, line);
- return block;
- }
- return buf_page_get_low(page_id, zip_size, rw_latch,
- guess, mode, file, line, mtr, err, allow_ibuf_merge);
+ if (rw_latch == RW_X_LATCH)
+ {
+ mtr->memo_push(block, MTR_MEMO_PAGE_X_FIX);
+ return block;
+ }
+ block->page.lock.x_unlock();
+ }
+ mtr->page_lock(block, rw_latch);
+ return block;
}
/********************************************************************//**
This is the general function used to get optimistic access to a database
page.
@return TRUE if success */
-ibool
-buf_page_optimistic_get(
-/*====================*/
- ulint rw_latch,/*!< in: RW_S_LATCH, RW_X_LATCH */
- buf_block_t* block, /*!< in: guessed buffer block */
- ib_uint64_t modify_clock,/*!< in: modify clock value */
- const char* file, /*!< in: file name */
- unsigned line, /*!< in: line where called */
- mtr_t* mtr) /*!< in: mini-transaction */
+TRANSACTIONAL_TARGET
+bool buf_page_optimistic_get(ulint rw_latch, buf_block_t *block,
+ uint64_t modify_clock, mtr_t *mtr)
{
- ibool success;
-
- ut_ad(block);
- ut_ad(mtr);
- ut_ad(mtr->is_active());
- ut_ad(rw_latch == RW_S_LATCH || rw_latch == RW_X_LATCH);
-
- if (UNIV_UNLIKELY(block->page.state() != BUF_BLOCK_FILE_PAGE
- || block->page.io_fix() != BUF_IO_NONE)) {
- return FALSE;
- }
-
- const page_id_t id(block->page.id());
-
- page_hash_latch *hash_lock = buf_pool.hash_lock_get(id);
- hash_lock->read_lock();
-
- if (UNIV_UNLIKELY(id != block->page.id()
- || block->page.state() != BUF_BLOCK_FILE_PAGE
- || block->page.io_fix() != BUF_IO_NONE)) {
- hash_lock->read_unlock();
- return(FALSE);
- }
-
- buf_block_buf_fix_inc(block, file, line);
- hash_lock->read_unlock();
-
- block->page.set_accessed();
-
- buf_page_make_young_if_needed(&block->page);
-
- ut_ad(!ibuf_inside(mtr) || ibuf_page(id, block->zip_size(), NULL));
-
- mtr_memo_type_t fix_type;
-
- if (rw_latch == RW_S_LATCH) {
- fix_type = MTR_MEMO_PAGE_S_FIX;
- success = rw_lock_s_lock_nowait(&block->lock, file, line);
- } else {
- fix_type = MTR_MEMO_PAGE_X_FIX;
- success = rw_lock_x_lock_func_nowait_inline(
- &block->lock, file, line);
- }
-
- ut_ad(id == block->page.id());
+ ut_ad(block);
+ ut_ad(mtr);
+ ut_ad(mtr->is_active());
+ ut_ad(rw_latch == RW_S_LATCH || rw_latch == RW_X_LATCH);
- if (!success) {
- buf_block_buf_fix_dec(block);
- return(FALSE);
- }
+ if (have_transactional_memory);
+ else if (UNIV_UNLIKELY(!block->page.frame))
+ return false;
+ else
+ {
+ const auto state= block->page.state();
+ if (UNIV_UNLIKELY(state < buf_page_t::UNFIXED ||
+ state >= buf_page_t::READ_FIX))
+ return false;
+ }
- if (modify_clock != block->modify_clock) {
+ bool success;
+ const page_id_t id{block->page.id()};
+ buf_pool_t::hash_chain &chain= buf_pool.page_hash.cell_get(id.fold());
+ bool have_u_not_x= false;
- buf_block_dbg_add_level(block, SYNC_NO_ORDER_CHECK);
+ {
+ transactional_shared_lock_guard<page_hash_latch> g
+ {buf_pool.page_hash.lock_get(chain)};
+ if (UNIV_UNLIKELY(id != block->page.id() || !block->page.frame))
+ return false;
+ const auto state= block->page.state();
+ if (UNIV_UNLIKELY(state < buf_page_t::UNFIXED ||
+ state >= buf_page_t::READ_FIX))
+ return false;
+
+ if (rw_latch == RW_S_LATCH)
+ success= block->page.lock.s_lock_try();
+ else
+ {
+ have_u_not_x= block->page.lock.have_u_not_x();
+ success= have_u_not_x || block->page.lock.x_lock_try();
+ }
+ }
- if (rw_latch == RW_S_LATCH) {
- rw_lock_s_unlock(&block->lock);
- } else {
- rw_lock_x_unlock(&block->lock);
- }
+ if (!success)
+ return false;
- buf_block_buf_fix_dec(block);
- return(FALSE);
- }
+ if (have_u_not_x)
+ {
+ block->page.lock.u_x_upgrade();
+ mtr->page_lock_upgrade(*block);
+ ut_ad(id == block->page.id());
+ ut_ad(modify_clock == block->modify_clock);
+ }
+ else
+ {
+ ut_ad(rw_latch == RW_S_LATCH || !block->page.is_io_fixed());
+ ut_ad(id == block->page.id());
+ ut_ad(!ibuf_inside(mtr) || ibuf_page(id, block->zip_size(), nullptr));
- mtr_memo_push(mtr, block, fix_type);
+ if (modify_clock != block->modify_clock || block->page.is_freed())
+ {
+ if (rw_latch == RW_S_LATCH)
+ block->page.lock.s_unlock();
+ else
+ block->page.lock.x_unlock();
+ return false;
+ }
-#ifdef UNIV_DEBUG
- if (!(++buf_dbg_counter % 5771)) buf_pool.validate();
-#endif /* UNIV_DEBUG */
- ut_ad(block->page.buf_fix_count());
- ut_ad(block->page.state() == BUF_BLOCK_FILE_PAGE);
+ 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));
+ }
- buf_pool.stat.n_page_gets++;
+ ut_d(if (!(++buf_dbg_counter % 5771)) buf_pool.validate());
+ ut_d(const auto state = block->page.state());
+ ut_ad(state > buf_page_t::UNFIXED);
+ ut_ad(state < buf_page_t::READ_FIX || state > buf_page_t::WRITE_FIX);
+ ut_ad(~buf_page_t::LRU_MASK & state);
+ ut_ad(block->page.frame);
- return(TRUE);
+ ++buf_pool.stat.n_page_gets;
+ return true;
}
-/** Given a tablespace id and page number tries to get that page. If the
-page is not in the buffer pool it is not loaded and NULL is returned.
-Suitable for using when holding the lock_sys_t::mutex.
-@param[in] page_id page id
-@param[in] file file name
-@param[in] line line where called
-@param[in] mtr mini-transaction
-@return pointer to a page or NULL */
-buf_block_t*
-buf_page_try_get_func(
- const page_id_t page_id,
- const char* file,
- unsigned line,
- mtr_t* mtr)
+/** Try to S-latch a page.
+Suitable for using when holding the lock_sys latches (as it avoids deadlock).
+@param[in] page_id page identifier
+@param[in,out] mtr mini-transaction
+@return the block
+@retval nullptr if an S-latch cannot be granted immediately */
+TRANSACTIONAL_TARGET
+buf_block_t *buf_page_try_get(const page_id_t page_id, mtr_t *mtr)
{
ut_ad(mtr);
ut_ad(mtr->is_active());
+ buf_pool_t::hash_chain &chain= buf_pool.page_hash.cell_get(page_id.fold());
+ buf_block_t *block;
- page_hash_latch *hash_lock;
- buf_page_t *bpage= buf_pool.page_hash_get_locked<false>(page_id,
- page_id.fold(),
- &hash_lock);
- if (!bpage)
- return nullptr;
- if (bpage->state() != BUF_BLOCK_FILE_PAGE)
{
- hash_lock->read_unlock();
- return nullptr;
- }
-
- buf_block_t *block= reinterpret_cast<buf_block_t*>(bpage);
- buf_block_buf_fix_inc(block, file, line);
- hash_lock->read_unlock();
-
- mtr_memo_type_t fix_type= MTR_MEMO_PAGE_S_FIX;
- if (!rw_lock_s_lock_nowait(&block->lock, file, line))
- {
- /* Let us try to get an X-latch. If the current thread
- is holding an X-latch on the page, we cannot get an S-latch. */
- fix_type= MTR_MEMO_PAGE_X_FIX;
- if (!rw_lock_x_lock_func_nowait_inline(&block->lock, file, line))
- {
- buf_block_buf_fix_dec(block);
+ transactional_shared_lock_guard<page_hash_latch> g
+ {buf_pool.page_hash.lock_get(chain)};
+ block= reinterpret_cast<buf_block_t*>
+ (buf_pool.page_hash.get(page_id, chain));
+ if (!block || !block->page.frame || !block->page.lock.s_lock_try())
return nullptr;
- }
}
- mtr_memo_push(mtr, block, fix_type);
+ block->page.fix();
+ ut_ad(!block->page.is_read_fixed());
+ mtr->memo_push(block, MTR_MEMO_PAGE_S_FIX);
#ifdef UNIV_DEBUG
if (!(++buf_dbg_counter % 5771)) buf_pool.validate();
#endif /* UNIV_DEBUG */
- ut_ad(bpage->buf_fix_count());
- ut_ad(bpage->state() == BUF_BLOCK_FILE_PAGE);
- ut_ad(bpage->id() == page_id);
- buf_block_dbg_add_level(block, SYNC_NO_ORDER_CHECK);
+ ut_ad(block->page.buf_fix_count());
+ ut_ad(block->page.id() == page_id);
- buf_pool.stat.n_page_gets++;
+ ++buf_pool.stat.n_page_gets;
return block;
}
@@ -3407,121 +3126,134 @@ buf_page_try_get_func(
void buf_block_t::initialise(const page_id_t page_id, ulint zip_size,
uint32_t fix)
{
- ut_ad(page.state() != BUF_BLOCK_FILE_PAGE);
+ ut_ad(!page.in_file());
buf_block_init_low(this);
- page.init(page_id, fix);
+ page.init(fix, page_id);
page_zip_set_size(&page.zip, zip_size);
}
-/** Initialize a page in the buffer pool. The page is usually not read
-from a file even if it cannot be found in the buffer buf_pool. This is one
-of the functions which perform to a block a state transition NOT_USED =>
-FILE_PAGE (the other is buf_page_get_gen).
-@param[in,out] space space object
-@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, buf_block_t *free_block)
+TRANSACTIONAL_TARGET
+static buf_block_t *buf_page_create_low(page_id_t page_id, ulint zip_size,
+ mtr_t *mtr, buf_block_t *free_block)
{
- page_id_t page_id(space->id, offset);
ut_ad(mtr->is_active());
ut_ad(page_id.space() != 0 || !zip_size);
- space->free_page(offset, false);
- free_block->initialise(page_id, zip_size, 1);
+ free_block->initialise(page_id, zip_size, buf_page_t::MEMORY);
- const ulint fold= page_id.fold();
+ buf_pool_t::hash_chain &chain= buf_pool.page_hash.cell_get(page_id.fold());
+retry:
mysql_mutex_lock(&buf_pool.mutex);
-loop:
- buf_block_t *block= reinterpret_cast<buf_block_t*>
- (buf_pool.page_hash_get_low(page_id, fold));
+ buf_page_t *bpage= buf_pool.page_hash.get(page_id, chain);
- if (block && block->page.in_file() &&
- !buf_pool.watch_is_sentinel(block->page))
+ if (bpage && !buf_pool.watch_is_sentinel(*bpage))
{
#ifdef BTR_CUR_HASH_ADAPT
const dict_index_t *drop_hash_entry= nullptr;
#endif
- switch (UNIV_EXPECT(block->page.state(), BUF_BLOCK_FILE_PAGE)) {
- default:
- ut_ad(0);
- break;
- case BUF_BLOCK_FILE_PAGE:
- if (!mtr->have_x_latch(*block))
+ bool ibuf_exist= false;
+
+ if (!mtr->have_x_latch(reinterpret_cast<const buf_block_t&>(*bpage)))
+ {
+ const bool got= bpage->lock.x_lock_try();
+ if (!got)
{
- buf_block_buf_fix_inc(block, __FILE__, __LINE__);
- while (!rw_lock_x_lock_nowait(&block->lock))
+ mysql_mutex_unlock(&buf_pool.mutex);
+ bpage->lock.x_lock();
+ const page_id_t id{bpage->id()};
+ if (UNIV_UNLIKELY(id != page_id))
{
- /* Wait for buf_page_write_complete() to release block->lock.
- We must not hold buf_pool.mutex while waiting. */
- timespec abstime;
- set_timespec_nsec(abstime, 1000000);
- my_cond_timedwait(&buf_pool.done_flush_list, &buf_pool.mutex.m_mutex,
- &abstime);
+ ut_ad(id.is_corrupted());
+ bpage->lock.x_unlock();
+ goto retry;
}
- mtr_memo_push(mtr, block, MTR_MEMO_PAGE_X_FIX);
+ mysql_mutex_lock(&buf_pool.mutex);
}
+
+ auto state= bpage->fix();
+ ut_ad(state >= buf_page_t::FREED);
+ ut_ad(state < buf_page_t::READ_FIX);
+
+ if (state < buf_page_t::UNFIXED)
+ bpage->set_reinit(buf_page_t::FREED);
else
{
- ut_ad(!block->page.ibuf_exist);
-#ifdef BTR_CUR_HASH_ADAPT
- ut_ad(!block->index);
-#endif
+ bpage->set_reinit(state & buf_page_t::LRU_MASK);
+ ibuf_exist= (state & buf_page_t::LRU_MASK) == buf_page_t::IBUF_EXIST;
}
+
+ if (UNIV_LIKELY(bpage->frame != nullptr))
+ {
+ mysql_mutex_unlock(&buf_pool.mutex);
+ buf_block_t *block= reinterpret_cast<buf_block_t*>(bpage);
+ mtr->memo_push(block, MTR_MEMO_PAGE_X_FIX);
#ifdef BTR_CUR_HASH_ADAPT
- drop_hash_entry= block->index;
+ drop_hash_entry= block->index;
#endif
- break;
- case BUF_BLOCK_ZIP_PAGE:
- page_hash_latch *hash_lock= buf_pool.page_hash.lock_get(fold);
- hash_lock->write_lock();
- if (block->page.io_fix() != BUF_IO_NONE)
- {
- hash_lock->write_unlock();
- /* Wait for buf_page_write_complete() to release the I/O fix. */
- timespec abstime;
- set_timespec_nsec(abstime, 1000000);
- my_cond_timedwait(&buf_pool.done_flush_list, &buf_pool.mutex.m_mutex,
- &abstime);
- goto loop;
}
+ else
+ {
+ auto state= bpage->state();
+ ut_ad(state >= buf_page_t::FREED);
+ ut_ad(state < buf_page_t::READ_FIX);
+
+ page_hash_latch &hash_lock= buf_pool.page_hash.lock_get(chain);
+ /* It does not make sense to use transactional_lock_guard here,
+ because buf_relocate() would likely make the memory transaction
+ too large. */
+ hash_lock.lock();
+
+ if (state < buf_page_t::UNFIXED)
+ bpage->set_reinit(buf_page_t::FREED);
+ else
+ {
+ bpage->set_reinit(state & buf_page_t::LRU_MASK);
+ ibuf_exist= (state & buf_page_t::LRU_MASK) == buf_page_t::IBUF_EXIST;
+ }
- rw_lock_x_lock(&free_block->lock);
- mysql_mutex_lock(&buf_pool.flush_list_mutex);
- buf_relocate(&block->page, &free_block->page);
- buf_flush_relocate_on_flush_list(&block->page, &free_block->page);
- mysql_mutex_unlock(&buf_pool.flush_list_mutex);
-
- free_block->page.set_state(BUF_BLOCK_FILE_PAGE);
- buf_unzip_LRU_add_block(free_block, FALSE);
- hash_lock->write_unlock();
- buf_page_free_descriptor(&block->page);
- block= free_block;
- buf_block_buf_fix_inc(block, __FILE__, __LINE__);
- mtr_memo_push(mtr, block, MTR_MEMO_PAGE_X_FIX);
- break;
- }
+ mysql_mutex_lock(&buf_pool.flush_list_mutex);
+ buf_relocate(bpage, &free_block->page);
+ free_block->page.lock.x_lock();
+ buf_flush_relocate_on_flush_list(bpage, &free_block->page);
+ mysql_mutex_unlock(&buf_pool.flush_list_mutex);
- mysql_mutex_unlock(&buf_pool.mutex);
+ buf_unzip_LRU_add_block(free_block, FALSE);
+
+ mysql_mutex_unlock(&buf_pool.mutex);
+ hash_lock.unlock();
+#if defined SUX_LOCK_GENERIC || defined UNIV_DEBUG
+ bpage->lock.x_unlock();
+ bpage->lock.free();
+#endif
+ ut_free(bpage);
+ mtr->memo_push(free_block, MTR_MEMO_PAGE_X_FIX);
+ bpage= &free_block->page;
+ }
+ }
+ else
+ {
+ mysql_mutex_unlock(&buf_pool.mutex);
+ ut_ad(bpage->frame);
+#ifdef BTR_CUR_HASH_ADAPT
+ ut_ad(!reinterpret_cast<buf_block_t*>(bpage)->index);
+#endif
+ const auto state= bpage->state();
+ ut_ad(state >= buf_page_t::FREED);
+ bpage->set_reinit(state < buf_page_t::UNFIXED ? buf_page_t::FREED
+ : state & buf_page_t::LRU_MASK);
+ }
#ifdef BTR_CUR_HASH_ADAPT
if (drop_hash_entry)
- btr_search_drop_page_hash_index(block, false);
+ btr_search_drop_page_hash_index(reinterpret_cast<buf_block_t*>(bpage),
+ false);
#endif /* BTR_CUR_HASH_ADAPT */
- if (block->page.ibuf_exist)
- {
- if (!recv_recovery_is_on())
- ibuf_merge_or_delete_for_page(nullptr, page_id, zip_size);
- block->page.ibuf_exist= false;
- }
+ if (ibuf_exist && !recv_recovery_is_on())
+ ibuf_merge_or_delete_for_page(nullptr, page_id, zip_size);
- return block;
+ return reinterpret_cast<buf_block_t*>(bpage);
}
/* If we get here, the page was not in buf_pool: init it there */
@@ -3529,52 +3261,37 @@ loop:
DBUG_PRINT("ib_buf", ("create page %u:%u",
page_id.space(), page_id.page_no()));
- block= free_block;
+ bpage= &free_block->page;
- /* Duplicate buf_block_buf_fix_inc_func() */
- ut_ad(block->page.buf_fix_count() == 1);
- ut_ad(fsp_is_system_temporary(page_id.space()) ||
- rw_lock_s_lock_nowait(block->debug_latch, __FILE__, __LINE__));
+ ut_ad(bpage->state() == buf_page_t::MEMORY);
+ bpage->lock.x_lock();
/* The block must be put to the LRU list */
- buf_LRU_add_block(&block->page, false);
- page_hash_latch *hash_lock= buf_pool.page_hash.lock_get(fold);
- hash_lock->write_lock();
- block->page.set_state(BUF_BLOCK_FILE_PAGE);
- ut_d(block->page.in_page_hash= true);
- HASH_INSERT(buf_page_t, hash, &buf_pool.page_hash, fold, &block->page);
-
- rw_lock_x_lock(&block->lock);
+ buf_LRU_add_block(bpage, false);
+ {
+ transactional_lock_guard<page_hash_latch> g
+ {buf_pool.page_hash.lock_get(chain)};
+ bpage->set_state(buf_page_t::REINIT + 1);
+ buf_pool.page_hash.append(chain, bpage);
+ }
+
if (UNIV_UNLIKELY(zip_size))
{
- /* Prevent race conditions during buf_buddy_alloc(), which may
- release and reacquire buf_pool.mutex, by IO-fixing and X-latching
- the block. */
- block->page.set_io_fix(BUF_IO_READ);
- hash_lock->write_unlock();
-
- /* buf_pool.mutex may be released and reacquired by
- buf_buddy_alloc(). We must defer this operation until
- after the block descriptor has been added to
- buf_pool.LRU and buf_pool.page_hash. */
- block->page.zip.data= buf_buddy_alloc(zip_size);
+ bpage->zip.data= buf_buddy_alloc(zip_size);
/* To maintain the invariant block->in_unzip_LRU_list ==
block->page.belongs_to_unzip_LRU() we have to add this
block to unzip_LRU after block->page.zip.data is set. */
- ut_ad(block->page.belongs_to_unzip_LRU());
- buf_unzip_LRU_add_block(block, FALSE);
-
- block->page.set_io_fix(BUF_IO_NONE);
+ ut_ad(bpage->belongs_to_unzip_LRU());
+ buf_unzip_LRU_add_block(reinterpret_cast<buf_block_t*>(bpage), FALSE);
}
- else
- hash_lock->write_unlock();
+ buf_pool.stat.n_pages_created++;
mysql_mutex_unlock(&buf_pool.mutex);
- mtr->memo_push(block, MTR_MEMO_PAGE_X_FIX);
- block->page.set_accessed();
- buf_pool.stat.n_pages_created++;
+ mtr->memo_push(reinterpret_cast<buf_block_t*>(bpage), MTR_MEMO_PAGE_X_FIX);
+
+ bpage->set_accessed();
/* Delete possible entries for the page from the insert buffer:
such can exist if the page belonged to an index which was dropped */
@@ -3584,8 +3301,8 @@ loop:
ibuf_merge_or_delete_for_page(nullptr, page_id, zip_size);
static_assert(FIL_PAGE_PREV + 4 == FIL_PAGE_NEXT, "adjacent");
- memset_aligned<8>(block->frame + FIL_PAGE_PREV, 0xff, 8);
- mach_write_to_2(block->frame + FIL_PAGE_TYPE, FIL_PAGE_TYPE_ALLOCATED);
+ memset_aligned<8>(bpage->frame + FIL_PAGE_PREV, 0xff, 8);
+ mach_write_to_2(bpage->frame + FIL_PAGE_TYPE, FIL_PAGE_TYPE_ALLOCATED);
/* FIL_PAGE_FILE_FLUSH_LSN_OR_KEY_VERSION is only used on the
following pages:
@@ -3593,30 +3310,57 @@ loop:
(2) FIL_RTREE_SPLIT_SEQ_NUM on R-tree pages
(3) key_version on encrypted pages (not page 0:0) */
- memset(block->frame + FIL_PAGE_FILE_FLUSH_LSN_OR_KEY_VERSION, 0, 8);
- memset_aligned<8>(block->frame + FIL_PAGE_LSN, 0, 8);
+ memset(bpage->frame + FIL_PAGE_FILE_FLUSH_LSN_OR_KEY_VERSION, 0, 8);
+ memset_aligned<8>(bpage->frame + FIL_PAGE_LSN, 0, 8);
#ifdef UNIV_DEBUG
if (!(++buf_dbg_counter % 5771)) buf_pool.validate();
#endif /* UNIV_DEBUG */
- return block;
+ return reinterpret_cast<buf_block_t*>(bpage);
+}
+
+/** Initialize a page in the buffer pool. The page is usually not read
+from a file even if it cannot be found in the buffer buf_pool. This is one
+of the functions which perform to a block a state transition NOT_USED =>
+FILE_PAGE (the other is buf_page_get_gen).
+@param[in,out] space space object
+@param[in] offset offset of the tablespace
+ or deferred space id if space
+ object is null
+@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, buf_block_t *free_block)
+{
+ space->free_page(offset, false);
+ return buf_page_create_low({space->id, offset}, zip_size, mtr, free_block);
+}
+
+/** Initialize a page in buffer pool while initializing the
+deferred tablespace
+@param space_id space identfier
+@param zip_size ROW_FORMAT=COMPRESSED page size or 0
+@param mtr mini-transaction
+@param free_block pre-allocated buffer block
+@return pointer to the block, page bufferfixed */
+buf_block_t* buf_page_create_deferred(uint32_t space_id, ulint zip_size,
+ mtr_t *mtr, buf_block_t *free_block)
+{
+ return buf_page_create_low({space_id, 0}, zip_size, mtr, free_block);
}
/** Monitor the buffer page read/write activity, and increment corresponding
counter value in MONITOR_MODULE_BUF_PAGE.
@param bpage buffer page whose read or write was completed
-@param io_type BUF_IO_READ or BUF_IO_WRITE */
-ATTRIBUTE_COLD __attribute__((nonnull))
-void buf_page_monitor(const buf_page_t *bpage, buf_io_fix io_type)
+@param read true=read, false=write */
+ATTRIBUTE_COLD void buf_page_monitor(const buf_page_t &bpage, bool read)
{
- const byte* frame;
monitor_id_t counter;
- ut_ad(io_type == BUF_IO_READ || io_type == BUF_IO_WRITE);
-
- frame = bpage->zip.data
- ? bpage->zip.data
- : ((buf_block_t*) bpage)->frame;
+ const byte* frame = bpage.zip.data ? bpage.zip.data : bpage.frame;
switch (fil_page_get_type(frame)) {
ulint level;
@@ -3631,135 +3375,74 @@ void buf_page_monitor(const buf_page_t *bpage, buf_io_fix io_type)
== (index_id_t)(DICT_IBUF_ID_MIN + IBUF_SPACE_ID)) {
if (level == 0) {
counter = MONITOR_RW_COUNTER(
- io_type, MONITOR_INDEX_IBUF_LEAF_PAGE);
+ read, MONITOR_INDEX_IBUF_LEAF_PAGE);
} else {
counter = MONITOR_RW_COUNTER(
- io_type,
+ read,
MONITOR_INDEX_IBUF_NON_LEAF_PAGE);
}
} else {
if (level == 0) {
counter = MONITOR_RW_COUNTER(
- io_type, MONITOR_INDEX_LEAF_PAGE);
+ read, MONITOR_INDEX_LEAF_PAGE);
} else {
counter = MONITOR_RW_COUNTER(
- io_type, MONITOR_INDEX_NON_LEAF_PAGE);
+ read, MONITOR_INDEX_NON_LEAF_PAGE);
}
}
break;
case FIL_PAGE_UNDO_LOG:
- counter = MONITOR_RW_COUNTER(io_type, MONITOR_UNDO_LOG_PAGE);
+ counter = MONITOR_RW_COUNTER(read, MONITOR_UNDO_LOG_PAGE);
break;
case FIL_PAGE_INODE:
- counter = MONITOR_RW_COUNTER(io_type, MONITOR_INODE_PAGE);
+ counter = MONITOR_RW_COUNTER(read, MONITOR_INODE_PAGE);
break;
case FIL_PAGE_IBUF_FREE_LIST:
- counter = MONITOR_RW_COUNTER(io_type,
- MONITOR_IBUF_FREELIST_PAGE);
+ counter = MONITOR_RW_COUNTER(read, MONITOR_IBUF_FREELIST_PAGE);
break;
case FIL_PAGE_IBUF_BITMAP:
- counter = MONITOR_RW_COUNTER(io_type,
- MONITOR_IBUF_BITMAP_PAGE);
+ counter = MONITOR_RW_COUNTER(read, MONITOR_IBUF_BITMAP_PAGE);
break;
case FIL_PAGE_TYPE_SYS:
- counter = MONITOR_RW_COUNTER(io_type, MONITOR_SYSTEM_PAGE);
+ counter = MONITOR_RW_COUNTER(read, MONITOR_SYSTEM_PAGE);
break;
case FIL_PAGE_TYPE_TRX_SYS:
- counter = MONITOR_RW_COUNTER(io_type, MONITOR_TRX_SYSTEM_PAGE);
+ counter = MONITOR_RW_COUNTER(read, MONITOR_TRX_SYSTEM_PAGE);
break;
case FIL_PAGE_TYPE_FSP_HDR:
- counter = MONITOR_RW_COUNTER(io_type, MONITOR_FSP_HDR_PAGE);
+ counter = MONITOR_RW_COUNTER(read, MONITOR_FSP_HDR_PAGE);
break;
case FIL_PAGE_TYPE_XDES:
- counter = MONITOR_RW_COUNTER(io_type, MONITOR_XDES_PAGE);
+ counter = MONITOR_RW_COUNTER(read, MONITOR_XDES_PAGE);
break;
case FIL_PAGE_TYPE_BLOB:
- counter = MONITOR_RW_COUNTER(io_type, MONITOR_BLOB_PAGE);
+ counter = MONITOR_RW_COUNTER(read, MONITOR_BLOB_PAGE);
break;
case FIL_PAGE_TYPE_ZBLOB:
- counter = MONITOR_RW_COUNTER(io_type, MONITOR_ZBLOB_PAGE);
+ counter = MONITOR_RW_COUNTER(read, MONITOR_ZBLOB_PAGE);
break;
case FIL_PAGE_TYPE_ZBLOB2:
- counter = MONITOR_RW_COUNTER(io_type, MONITOR_ZBLOB2_PAGE);
+ counter = MONITOR_RW_COUNTER(read, MONITOR_ZBLOB2_PAGE);
break;
default:
- counter = MONITOR_RW_COUNTER(io_type, MONITOR_OTHER_PAGE);
+ counter = MONITOR_RW_COUNTER(read, MONITOR_OTHER_PAGE);
}
MONITOR_INC_NOCHECK(counter);
}
-/** Mark a table corrupted.
-@param[in] bpage corrupted page
-@param[in] space tablespace of the corrupted page */
-ATTRIBUTE_COLD
-static void buf_mark_space_corrupt(buf_page_t* bpage, const fil_space_t& space)
-{
- /* If block is not encrypted find the table with specified
- space id, and mark it corrupted. Encrypted tables
- are marked unusable later e.g. in ::open(). */
- if (!space.crypt_data
- || space.crypt_data->type == CRYPT_SCHEME_UNENCRYPTED) {
- dict_set_corrupted_by_space(&space);
- } else {
- dict_set_encrypted_by_space(&space);
- }
-}
-
-/** Release and evict a corrupted page.
-@param bpage page that was being read */
-ATTRIBUTE_COLD void buf_pool_t::corrupted_evict(buf_page_t *bpage)
-{
- const page_id_t id(bpage->id());
- page_hash_latch *hash_lock= hash_lock_get(id);
-
- mysql_mutex_lock(&mutex);
- hash_lock->write_lock();
-
- ut_ad(bpage->io_fix() == BUF_IO_READ);
- ut_ad(!bpage->oldest_modification());
- bpage->set_corrupt_id();
-
- if (bpage->state() == BUF_BLOCK_FILE_PAGE)
- rw_lock_x_unlock_gen(&reinterpret_cast<buf_block_t*>(bpage)->lock,
- BUF_IO_READ);
-
- bpage->io_unfix();
-
- /* remove from LRU and page_hash */
- buf_LRU_free_one_page(bpage, id, hash_lock);
- mysql_mutex_unlock(&mutex);
-
- ut_d(auto n=) n_pend_reads--;
- ut_ad(n > 0);
-}
-
-/** Mark a table corrupted.
-@param[in] bpage Corrupted page
-@param[in] node data file
-Also remove the bpage from LRU list. */
-ATTRIBUTE_COLD
-static void buf_corrupt_page_release(buf_page_t *bpage, const fil_node_t &node)
-{
- ut_ad(bpage->id().space() == node.space->id);
- buf_pool.corrupted_evict(bpage);
-
- if (!srv_force_recovery)
- buf_mark_space_corrupt(bpage, *node.space);
-}
-
/** Check if the encrypted page is corrupted for the full crc32 format.
@param[in] space_id page belongs to space id
@param[in] d page
@@ -3788,15 +3471,13 @@ or decrypt/decompress just failed.
@retval DB_SUCCESS if page has been read and is not corrupted
@retval DB_PAGE_CORRUPTED if page based on checksum check is corrupted
@retval DB_DECRYPTION_FAILED if page post encryption checksum matches but
-after decryption normal page checksum does not match.
-@retval DB_TABLESPACE_DELETED if accessed tablespace is not found */
+after decryption normal page checksum does not match. */
static dberr_t buf_page_check_corrupt(buf_page_t *bpage,
const fil_node_t &node)
{
ut_ad(node.space->referenced());
- byte* dst_frame = (bpage->zip.data) ? bpage->zip.data :
- ((buf_block_t*) bpage)->frame;
+ byte* dst_frame = bpage->zip.data ? bpage->zip.data : bpage->frame;
dberr_t err = DB_SUCCESS;
uint key_version = buf_page_get_key_version(dst_frame,
node.space->flags);
@@ -3836,129 +3517,103 @@ static dberr_t buf_page_check_corrupt(buf_page_t *bpage,
ib::error()
<< "The page " << bpage->id()
<< " in file '" << node.name
- << "' cannot be decrypted.";
-
- ib::info()
- << "However key management plugin or used key_version "
- << key_version
- << " is not found or"
- " used encryption algorithm or method does not match.";
-
- if (bpage->id().space() != TRX_SYS_SPACE) {
- ib::info()
- << "Marking tablespace as missing."
- " You may drop this table or"
- " install correct key management plugin"
- " and key file.";
- }
+ << "' cannot be decrypted; key_version="
+ << key_version;
}
return (err);
}
-/** Complete a read request of a file page to buf_pool.
-@param bpage recently read page
+/** Complete a read of a page.
@param node data file
@return whether the operation succeeded
-@retval DB_SUCCESS always when writing, or if a read page was OK
-@retval DB_PAGE_CORRUPTED if the checksum fails on a page read
-@retval DB_DECRYPTION_FAILED if the page cannot be decrypted */
-dberr_t buf_page_read_complete(buf_page_t *bpage, const fil_node_t &node)
+@retval DB_PAGE_CORRUPTED if the checksum fails
+@retval DB_DECRYPTION_FAILED if the page cannot be decrypted
+@retval DB_FAIL if the page contains the wrong ID */
+dberr_t buf_page_t::read_complete(const fil_node_t &node)
{
- const page_id_t id(bpage->id());
- ut_ad(bpage->in_file());
- ut_ad(!buf_dblwr.is_inside(id));
- ut_ad(id.space() == node.space->id);
- ut_ad(bpage->zip_size() == node.space->zip_size());
-
- /* We do not need protect io_fix here by mutex to read it because
- this and buf_page_write_complete() are the only functions where we can
- change the value from BUF_IO_READ or BUF_IO_WRITE to some other
- value, and our code ensures that this is the only thread that handles
- the i/o for this block. */
-
- ut_ad(bpage->io_fix() == BUF_IO_READ);
- ut_ad(!!bpage->zip.ssize == !!bpage->zip.data);
- ut_ad(bpage->state() == BUF_BLOCK_FILE_PAGE || bpage->zip.data);
-
- const byte *frame= bpage->zip.data
- ? bpage->zip.data
- : reinterpret_cast<buf_block_t*>(bpage)->frame;
- ut_ad(frame);
+ const page_id_t expected_id{id()};
+ ut_ad(is_read_fixed());
+ ut_ad(!buf_dblwr.is_inside(id()));
+ ut_ad(id().space() == node.space->id);
+ ut_ad(zip_size() == node.space->zip_size());
+ ut_ad(!!zip.ssize == !!zip.data);
+
+ const byte *read_frame= zip.data ? zip.data : frame;
+ ut_ad(read_frame);
dberr_t err;
- if (!buf_page_decrypt_after_read(bpage, node))
+ if (!buf_page_decrypt_after_read(this, node))
{
err= DB_DECRYPTION_FAILED;
goto database_corrupted;
}
- if (bpage->zip.data && bpage->state() == BUF_BLOCK_FILE_PAGE)
+ if (belongs_to_unzip_LRU())
{
buf_pool.n_pend_unzip++;
- auto ok= buf_zip_decompress(reinterpret_cast<buf_block_t*>(bpage), FALSE);
+ auto ok= buf_zip_decompress(reinterpret_cast<buf_block_t*>(this), false);
buf_pool.n_pend_unzip--;
if (!ok)
{
- ib::info() << "Page " << id << " zip_decompress failure.";
+ ib::info() << "Page " << expected_id << " zip_decompress failure.";
err= DB_PAGE_CORRUPTED;
goto database_corrupted;
}
}
{
- const page_id_t read_id(mach_read_from_4(frame + FIL_PAGE_SPACE_ID),
- mach_read_from_4(frame + FIL_PAGE_OFFSET));
+ const page_id_t read_id(mach_read_from_4(read_frame + FIL_PAGE_SPACE_ID),
+ mach_read_from_4(read_frame + FIL_PAGE_OFFSET));
- if (read_id == id);
+ if (read_id == expected_id);
else if (read_id == page_id_t(0, 0))
- /* This is likely an uninitialized page. */;
+ {
+ /* This is likely an uninitialized (all-zero) page. */
+ err= DB_FAIL;
+ goto release_page;
+ }
else if (!node.space->full_crc32() &&
- page_id_t(0, read_id.page_no()) == id)
+ page_id_t(0, read_id.page_no()) == expected_id)
/* FIL_PAGE_SPACE_ID was written as garbage in the system tablespace
before MySQL 4.1.1, which introduced innodb_file_per_table. */;
else if (node.space->full_crc32() &&
*reinterpret_cast<const uint32_t*>
- (&frame[FIL_PAGE_FCRC32_KEY_VERSION]) &&
+ (&read_frame[FIL_PAGE_FCRC32_KEY_VERSION]) &&
node.space->crypt_data &&
node.space->crypt_data->type != CRYPT_SCHEME_UNENCRYPTED)
{
- ib::error() << "Cannot decrypt " << id;
+ ib::error() << "Cannot decrypt " << expected_id;
err= DB_DECRYPTION_FAILED;
goto release_page;
}
else
+ {
ib::error() << "Space id and page no stored in the page, read in are "
- << read_id << ", should be " << id;
+ << read_id << ", should be " << expected_id;
+ err= DB_PAGE_CORRUPTED;
+ goto release_page;
+ }
}
- err= buf_page_check_corrupt(bpage, node);
+ err= buf_page_check_corrupt(this, node);
if (UNIV_UNLIKELY(err != DB_SUCCESS))
{
database_corrupted:
- /* Not a real corruption if it was triggered by error injection */
- DBUG_EXECUTE_IF("buf_page_import_corrupt_failure",
- if (!is_predefined_tablespace(id.space()))
- {
- buf_corrupt_page_release(bpage, node);
- ib::info() << "Simulated IMPORT corruption";
- return err;
- }
- err= DB_SUCCESS;
- goto page_not_corrupt;);
-
- if (bpage->zip.data && bpage->state() == BUF_BLOCK_FILE_PAGE)
- memset(reinterpret_cast<buf_block_t*>(bpage)->frame, 0, srv_page_size);
+ if (belongs_to_unzip_LRU())
+ memset_aligned<UNIV_PAGE_SIZE_MIN>(frame, 0, srv_page_size);
if (err == DB_PAGE_CORRUPTED)
{
ib::error() << "Database page corruption on disk"
" or a failed read of file '"
- << node.name << "' page " << id
+ << node.name << "' page " << expected_id
<< ". You may have to recover from a backup.";
- buf_page_print(frame, bpage->zip_size());
+ buf_page_print(read_frame, zip_size());
+
+ node.space->set_corrupted();
ib::info() << " You can use CHECK TABLE to scan"
" your table for corruption. "
@@ -3966,52 +3621,42 @@ database_corrupted:
}
if (!srv_force_recovery)
- {
- /* If the corruption is in the system tablespace, we will
- intentionally crash the server. */
- if (id.space() == TRX_SYS_SPACE)
- ib::fatal() << "Aborting because of a corrupt database page.";
- buf_corrupt_page_release(bpage, node);
- return err;
- }
+ goto release_page;
}
- DBUG_EXECUTE_IF("buf_page_import_corrupt_failure",
- page_not_corrupt: bpage= bpage; );
-
if (err == DB_PAGE_CORRUPTED || err == DB_DECRYPTION_FAILED)
{
release_page:
- buf_corrupt_page_release(bpage, node);
- if (recv_recovery_is_on())
- recv_sys.free_corrupted_page(id);
+ buf_pool.corrupted_evict(this, buf_page_t::READ_FIX);
return err;
}
- if (recv_recovery_is_on())
- recv_recover_page(node.space, bpage);
+ const bool recovery= recv_recovery_is_on();
- if (bpage->state() == BUF_BLOCK_FILE_PAGE && !recv_no_ibuf_operations &&
- (!id.space() || !is_predefined_tablespace(id.space())) &&
- fil_page_get_type(frame) == FIL_PAGE_INDEX &&
- page_is_leaf(frame))
- bpage->ibuf_exist= true;
+ if (recovery && !recv_recover_page(node.space, this))
+ return DB_PAGE_CORRUPTED;
+
+ const bool ibuf_may_exist= frame && !recv_no_ibuf_operations &&
+ (!expected_id.space() || !is_predefined_tablespace(expected_id.space())) &&
+ fil_page_get_type(read_frame) == FIL_PAGE_INDEX &&
+ page_is_leaf(read_frame);
if (UNIV_UNLIKELY(MONITOR_IS_ON(MONITOR_MODULE_BUF_PAGE)))
- buf_page_monitor(bpage, BUF_IO_READ);
- DBUG_PRINT("ib_buf", ("read page %u:%u",
- id.space(), id.page_no()));
+ buf_page_monitor(*this, true);
+ DBUG_PRINT("ib_buf", ("read page %u:%u", id().space(), id().page_no()));
- /* Because this thread which does the unlocking might not be the same that
- did the locking, we use a pass value != 0 in unlock, which simply
- removes the newest lock debug record, without checking the thread id. */
- if (bpage->state() == BUF_BLOCK_FILE_PAGE)
- rw_lock_x_unlock_gen(&((buf_block_t*) bpage)->lock, BUF_IO_READ);
- bpage->io_unfix();
+ if (!recovery)
+ {
+ ut_d(auto f=) zip.fix.fetch_sub(ibuf_may_exist
+ ? READ_FIX - IBUF_EXIST
+ : READ_FIX - UNFIXED);
+ ut_ad(f >= READ_FIX);
+ ut_ad(f < WRITE_FIX);
+ }
+ else if (ibuf_may_exist)
+ set_ibuf_exist();
- ut_d(auto n=) buf_pool.n_pend_reads--;
- ut_ad(n > 0);
- buf_pool.stat.n_pages_read++;
+ lock.x_unlock(true);
return DB_SUCCESS;
}
@@ -4044,9 +3689,6 @@ void buf_pool_invalidate()
{
mysql_mutex_lock(&buf_pool.mutex);
- buf_flush_wait_batch_end(true);
- buf_flush_wait_batch_end(false);
-
/* It is possible that a write batch that has been posted
earlier is still not complete. For buffer pool invalidation to
proceed we must ensure there is NO write activity happening. */
@@ -4055,16 +3697,17 @@ void buf_pool_invalidate()
ut_d(buf_pool.assert_all_freed());
ut_d(mysql_mutex_lock(&buf_pool.mutex));
- while (buf_LRU_scan_and_free_block());
+ while (UT_LIST_GET_LEN(buf_pool.LRU)) {
+ buf_LRU_scan_and_free_block();
+ }
- ut_ad(UT_LIST_GET_LEN(buf_pool.LRU) == 0);
ut_ad(UT_LIST_GET_LEN(buf_pool.unzip_LRU) == 0);
buf_pool.freed_page_clock = 0;
buf_pool.LRU_old = NULL;
buf_pool.LRU_old_len = 0;
+ buf_pool.stat.init();
- memset(&buf_pool.stat, 0x00, sizeof(buf_pool.stat));
buf_refresh_io_stats();
mysql_mutex_unlock(&buf_pool.mutex);
}
@@ -4085,34 +3728,34 @@ void buf_pool_t::validate()
/* Check the uncompressed blocks. */
for (auto i = n_chunks; i--; chunk++) {
-
- ulint j;
buf_block_t* block = chunk->blocks;
- for (j = chunk->size; j--; block++) {
- switch (block->page.state()) {
- case BUF_BLOCK_ZIP_PAGE:
- /* This kind of block descriptors should
- be allocated by malloc() only. */
- ut_error;
- break;
-
- case BUF_BLOCK_NOT_USED:
+ for (auto j = chunk->size; j--; block++) {
+ ut_ad(block->page.frame);
+ switch (const auto f = block->page.state()) {
+ case buf_page_t::NOT_USED:
n_free++;
break;
- case BUF_BLOCK_MEMORY:
- case BUF_BLOCK_REMOVE_HASH:
+ case buf_page_t::MEMORY:
+ case buf_page_t::REMOVE_HASH:
/* do nothing */
break;
- case BUF_BLOCK_FILE_PAGE:
- const page_id_t id = block->page.id();
- ut_ad(page_hash_get_low(id, id.fold())
+ default:
+ if (f >= buf_page_t::READ_FIX
+ && f < buf_page_t::WRITE_FIX) {
+ /* A read-fixed block is not
+ necessarily in the page_hash yet. */
+ break;
+ }
+ ut_ad(f >= buf_page_t::FREED);
+ const page_id_t id{block->page.id()};
+ ut_ad(page_hash.get(
+ id,
+ page_hash.cell_get(id.fold()))
== &block->page);
n_lru++;
- break;
-
}
}
}
@@ -4122,33 +3765,24 @@ void buf_pool_t::validate()
mysql_mutex_lock(&flush_list_mutex);
for (buf_page_t* b = UT_LIST_GET_FIRST(flush_list); b;
b = UT_LIST_GET_NEXT(list, b)) {
+ ut_ad(b->in_file());
ut_ad(b->oldest_modification());
ut_ad(!fsp_is_system_temporary(b->id().space()));
n_flushing++;
- switch (b->state()) {
- case BUF_BLOCK_ZIP_PAGE:
+ if (UNIV_UNLIKELY(!b->frame)) {
n_lru++;
n_zip++;
- break;
- case BUF_BLOCK_FILE_PAGE:
- /* uncompressed page */
- break;
- case BUF_BLOCK_NOT_USED:
- case BUF_BLOCK_MEMORY:
- case BUF_BLOCK_REMOVE_HASH:
- ut_error;
- break;
}
- const page_id_t id = b->id();
- ut_ad(page_hash_get_low(id, id.fold()) == b);
+ const page_id_t id{b->id()};
+ ut_ad(page_hash.get(id, page_hash.cell_get(id.fold())) == b);
}
ut_ad(UT_LIST_GET_LEN(flush_list) == n_flushing);
mysql_mutex_unlock(&flush_list_mutex);
- if (curr_size == old_size
+ if (n_chunks_new == n_chunks
&& n_lru + n_free > curr_size + n_zip) {
ib::fatal() << "n_LRU " << n_lru << ", n_free " << n_free
@@ -4158,7 +3792,7 @@ void buf_pool_t::validate()
ut_ad(UT_LIST_GET_LEN(LRU) >= n_lru);
- if (curr_size == old_size
+ if (n_chunks_new == n_chunks
&& UT_LIST_GET_LEN(free) != n_free) {
ib::fatal() << "Free list len "
@@ -4204,9 +3838,8 @@ void buf_pool_t::print()
<< ", modified database pages="
<< UT_LIST_GET_LEN(flush_list)
<< ", n pending decompressions=" << n_pend_unzip
- << ", n pending reads=" << n_pend_reads
- << ", n pending flush LRU=" << n_flush_LRU_
- << " list=" << n_flush_list_
+ << ", n pending flush LRU=" << n_flush()
+ << " list=" << os_aio_pending_writes()
<< ", pages made young=" << stat.n_pages_made_young
<< ", not young=" << stat.n_pages_not_made_young
<< ", pages read=" << stat.n_pages_read
@@ -4226,7 +3859,7 @@ void buf_pool_t::print()
ulint n_blocks = chunk->size;
for (; n_blocks--; block++) {
- const buf_frame_t* frame = block->frame;
+ const buf_frame_t* frame = block->page.frame;
if (fil_page_index_page_check(frame)) {
@@ -4288,7 +3921,7 @@ ulint buf_get_latched_pages_number()
for (buf_page_t *b= UT_LIST_GET_FIRST(buf_pool.LRU); b;
b= UT_LIST_GET_NEXT(LRU, b))
- if (b->in_file() && (b->buf_fix_count() || b->io_fix() != BUF_IO_NONE))
+ if (b->state() > buf_page_t::UNFIXED)
fixed_pages_number++;
mysql_mutex_unlock(&buf_pool.mutex);
@@ -4318,13 +3951,13 @@ void buf_stats_get_pool_info(buf_pool_info_t *pool_info)
pool_info->flush_list_len = UT_LIST_GET_LEN(buf_pool.flush_list);
pool_info->n_pend_unzip = UT_LIST_GET_LEN(buf_pool.unzip_LRU);
- mysql_mutex_unlock(&buf_pool.flush_list_mutex);
- pool_info->n_pend_reads = buf_pool.n_pend_reads;
+ pool_info->n_pend_reads = os_aio_pending_reads_approx();
- pool_info->n_pending_flush_lru = buf_pool.n_flush_LRU_;
+ pool_info->n_pending_flush_lru = buf_pool.n_flush();
- pool_info->n_pending_flush_list = buf_pool.n_flush_list_;
+ pool_info->n_pending_flush_list = os_aio_pending_writes();
+ mysql_mutex_unlock(&buf_pool.flush_list_mutex);
current_time = time(NULL);
time_elapsed = 0.001 + difftime(current_time,