summaryrefslogtreecommitdiff
path: root/storage/innobase/buf/buf0buf.c
diff options
context:
space:
mode:
Diffstat (limited to 'storage/innobase/buf/buf0buf.c')
-rw-r--r--storage/innobase/buf/buf0buf.c591
1 files changed, 98 insertions, 493 deletions
diff --git a/storage/innobase/buf/buf0buf.c b/storage/innobase/buf/buf0buf.c
index 52358c52853..33b4cd40215 100644
--- a/storage/innobase/buf/buf0buf.c
+++ b/storage/innobase/buf/buf0buf.c
@@ -1,6 +1,6 @@
/*****************************************************************************
-Copyright (c) 1995, 2010, Innobase Oy. All Rights Reserved.
+Copyright (c) 1995, 2011, Oracle and/or its affiliates. All Rights Reserved.
Copyright (c) 2008, Google Inc.
Portions of this file contain modifications contributed and copyrighted by
@@ -400,6 +400,7 @@ buf_get_total_stat(
tot_stat->n_pages_read += buf_stat->n_pages_read;
tot_stat->n_pages_written += buf_stat->n_pages_written;
tot_stat->n_pages_created += buf_stat->n_pages_created;
+ tot_stat->n_ra_pages_read_rnd += buf_stat->n_ra_pages_read_rnd;
tot_stat->n_ra_pages_read += buf_stat->n_ra_pages_read;
tot_stat->n_ra_pages_evicted += buf_stat->n_ra_pages_evicted;
tot_stat->n_pages_made_young += buf_stat->n_pages_made_young;
@@ -1099,70 +1100,6 @@ buf_chunk_not_freed(
return(NULL);
}
-/*********************************************************************//**
-Checks that all blocks in the buffer chunk are in BUF_BLOCK_NOT_USED state.
-@return TRUE if all freed */
-static
-ibool
-buf_chunk_all_free(
-/*===============*/
- const buf_chunk_t* chunk) /*!< in: chunk being checked */
-{
- const buf_block_t* block;
- ulint i;
-
- block = chunk->blocks;
-
- for (i = chunk->size; i--; block++) {
-
- if (buf_block_get_state(block) != BUF_BLOCK_NOT_USED) {
-
- return(FALSE);
- }
- }
-
- return(TRUE);
-}
-
-/********************************************************************//**
-Frees a chunk of buffer frames. */
-static
-void
-buf_chunk_free(
-/*===========*/
- buf_pool_t* buf_pool, /*!< in: buffer pool instance */
- buf_chunk_t* chunk) /*!< out: chunk of buffers */
-{
- buf_block_t* block;
- const buf_block_t* block_end;
-
- ut_ad(buf_pool_mutex_own(buf_pool));
-
- block_end = chunk->blocks + chunk->size;
-
- for (block = chunk->blocks; block < block_end; block++) {
- ut_a(buf_block_get_state(block) == BUF_BLOCK_NOT_USED);
- ut_a(!block->page.zip.data);
-
- ut_ad(!block->page.in_LRU_list);
- ut_ad(!block->in_unzip_LRU_list);
- ut_ad(!block->page.in_flush_list);
- /* Remove the block from the free list. */
- ut_ad(block->page.in_free_list);
- UT_LIST_REMOVE(list, buf_pool->free, (&block->page));
-
- /* Free the latches. */
- mutex_free(&block->mutex);
- rw_lock_free(&block->lock);
-#ifdef UNIV_SYNC_DEBUG
- rw_lock_free(&block->debug_latch);
-#endif /* UNIV_SYNC_DEBUG */
- UNIV_MEM_UNDESC(block);
- }
-
- os_mem_free_large(chunk->mem, chunk->mem_size);
-}
-
/********************************************************************//**
Set buffer pool size variables after resizing it */
static
@@ -1272,8 +1209,6 @@ buf_pool_free_instance(
chunk = chunks + buf_pool->n_chunks;
while (--chunk >= chunks) {
- /* Bypass the checks of buf_chunk_free(), since they
- would fail at shutdown. */
os_mem_free_large(chunk->mem, chunk->mem_size);
}
@@ -1533,281 +1468,6 @@ buf_relocate(
}
/********************************************************************//**
-Shrinks a buffer pool instance. */
-static
-void
-buf_pool_shrink_instance(
-/*=====================*/
- buf_pool_t* buf_pool, /*!< in: buffer pool instance */
- ulint chunk_size) /*!< in: number of pages to remove */
-{
- buf_chunk_t* chunks;
- buf_chunk_t* chunk;
- ulint max_size;
- ulint max_free_size;
- buf_chunk_t* max_chunk;
- buf_chunk_t* max_free_chunk;
-
- ut_ad(!buf_pool_mutex_own(buf_pool));
-
-try_again:
- btr_search_disable(); /* Empty the adaptive hash index again */
- buf_pool_mutex_enter(buf_pool);
-
-shrink_again:
- if (buf_pool->n_chunks <= 1) {
-
- /* Cannot shrink if there is only one chunk */
- goto func_done;
- }
-
- /* Search for the largest free chunk
- not larger than the size difference */
- chunks = buf_pool->chunks;
- chunk = chunks + buf_pool->n_chunks;
- max_size = max_free_size = 0;
- max_chunk = max_free_chunk = NULL;
-
- while (--chunk >= chunks) {
- if (chunk->size <= chunk_size
- && chunk->size > max_free_size) {
- if (chunk->size > max_size) {
- max_size = chunk->size;
- max_chunk = chunk;
- }
-
- if (buf_chunk_all_free(chunk)) {
- max_free_size = chunk->size;
- max_free_chunk = chunk;
- }
- }
- }
-
- if (!max_free_size) {
-
- ulint dirty = 0;
- ulint nonfree = 0;
- buf_block_t* block;
- buf_block_t* bend;
-
- /* Cannot shrink: try again later
- (do not assign srv_buf_pool_old_size) */
- if (!max_chunk) {
-
- goto func_exit;
- }
-
- block = max_chunk->blocks;
- bend = block + max_chunk->size;
-
- /* Move the blocks of chunk to the end of the
- LRU list and try to flush them. */
- for (; block < bend; block++) {
- switch (buf_block_get_state(block)) {
- case BUF_BLOCK_NOT_USED:
- continue;
- case BUF_BLOCK_FILE_PAGE:
- break;
- default:
- nonfree++;
- continue;
- }
-
- mutex_enter(&block->mutex);
- /* The following calls will temporarily
- release block->mutex and buf_pool->mutex.
- Therefore, we have to always retry,
- even if !dirty && !nonfree. */
-
- if (!buf_flush_ready_for_replace(&block->page)) {
-
- buf_LRU_make_block_old(&block->page);
- dirty++;
- } else if (buf_LRU_free_block(&block->page, TRUE)
- != BUF_LRU_FREED) {
- nonfree++;
- }
-
- mutex_exit(&block->mutex);
- }
-
- buf_pool_mutex_exit(buf_pool);
-
- /* Request for a flush of the chunk if it helps.
- Do not flush if there are non-free blocks, since
- flushing will not make the chunk freeable. */
- if (nonfree) {
- /* Avoid busy-waiting. */
- os_thread_sleep(100000);
- } else if (dirty
- && buf_flush_LRU(buf_pool, dirty)
- == ULINT_UNDEFINED) {
-
- buf_flush_wait_batch_end(buf_pool, BUF_FLUSH_LRU);
- }
-
- goto try_again;
- }
-
- max_size = max_free_size;
- max_chunk = max_free_chunk;
-
- buf_pool->old_pool_size = buf_pool->curr_pool_size;
-
- /* Rewrite buf_pool->chunks. Copy everything but max_chunk. */
- chunks = mem_alloc((buf_pool->n_chunks - 1) * sizeof *chunks);
- memcpy(chunks, buf_pool->chunks,
- (max_chunk - buf_pool->chunks) * sizeof *chunks);
- memcpy(chunks + (max_chunk - buf_pool->chunks),
- max_chunk + 1,
- buf_pool->chunks + buf_pool->n_chunks
- - (max_chunk + 1));
- ut_a(buf_pool->curr_size > max_chunk->size);
- buf_pool->curr_size -= max_chunk->size;
- buf_pool->curr_pool_size = buf_pool->curr_size * UNIV_PAGE_SIZE;
- chunk_size -= max_chunk->size;
- buf_chunk_free(buf_pool, max_chunk);
- mem_free(buf_pool->chunks);
- buf_pool->chunks = chunks;
- buf_pool->n_chunks--;
-
- /* Allow a slack of one megabyte. */
- if (chunk_size > 1048576 / UNIV_PAGE_SIZE) {
-
- goto shrink_again;
- }
- goto func_exit;
-
-func_done:
- buf_pool->old_pool_size = buf_pool->curr_pool_size;
-func_exit:
- buf_pool_mutex_exit(buf_pool);
- btr_search_enable();
-}
-
-/********************************************************************//**
-Shrinks the buffer pool. */
-static
-void
-buf_pool_shrink(
-/*============*/
- ulint chunk_size) /*!< in: number of pages to remove */
-{
- ulint i;
-
- for (i = 0; i < srv_buf_pool_instances; i++) {
- buf_pool_t* buf_pool;
- ulint instance_chunk_size;
-
- instance_chunk_size = chunk_size / srv_buf_pool_instances;
- buf_pool = buf_pool_from_array(i);
- buf_pool_shrink_instance(buf_pool, instance_chunk_size);
- }
-
- buf_pool_set_sizes();
-}
-
-/********************************************************************//**
-Rebuild buf_pool->page_hash for a buffer pool instance. */
-static
-void
-buf_pool_page_hash_rebuild_instance(
-/*================================*/
- buf_pool_t* buf_pool) /*!< in: buffer pool instance */
-{
- ulint i;
- buf_page_t* b;
- buf_chunk_t* chunk;
- ulint n_chunks;
- hash_table_t* zip_hash;
- hash_table_t* page_hash;
-
- buf_pool_mutex_enter(buf_pool);
-
- /* Free, create, and populate the hash table. */
- hash_table_free(buf_pool->page_hash);
- buf_pool->page_hash = page_hash = hash_create(2 * buf_pool->curr_size);
- zip_hash = hash_create(2 * buf_pool->curr_size);
-
- HASH_MIGRATE(buf_pool->zip_hash, zip_hash, buf_page_t, hash,
- BUF_POOL_ZIP_FOLD_BPAGE);
-
- hash_table_free(buf_pool->zip_hash);
- buf_pool->zip_hash = zip_hash;
-
- /* Insert the uncompressed file pages to buf_pool->page_hash. */
-
- chunk = buf_pool->chunks;
- n_chunks = buf_pool->n_chunks;
-
- for (i = 0; i < n_chunks; i++, chunk++) {
- ulint j;
- buf_block_t* block = chunk->blocks;
-
- for (j = 0; j < chunk->size; j++, block++) {
- if (buf_block_get_state(block)
- == BUF_BLOCK_FILE_PAGE) {
- ut_ad(!block->page.in_zip_hash);
- ut_ad(block->page.in_page_hash);
-
- HASH_INSERT(buf_page_t, hash, page_hash,
- buf_page_address_fold(
- block->page.space,
- block->page.offset),
- &block->page);
- }
- }
- }
-
- /* Insert the compressed-only pages to buf_pool->page_hash.
- All such blocks are either in buf_pool->zip_clean or
- in buf_pool->flush_list. */
-
- for (b = UT_LIST_GET_FIRST(buf_pool->zip_clean); b;
- b = UT_LIST_GET_NEXT(list, b)) {
- ut_a(buf_page_get_state(b) == BUF_BLOCK_ZIP_PAGE);
- ut_ad(!b->in_flush_list);
- ut_ad(b->in_LRU_list);
- ut_ad(b->in_page_hash);
- ut_ad(!b->in_zip_hash);
-
- HASH_INSERT(buf_page_t, hash, page_hash,
- buf_page_address_fold(b->space, b->offset), b);
- }
-
- buf_flush_list_mutex_enter(buf_pool);
- for (b = UT_LIST_GET_FIRST(buf_pool->flush_list); b;
- b = UT_LIST_GET_NEXT(list, b)) {
- ut_ad(b->in_flush_list);
- ut_ad(b->in_LRU_list);
- ut_ad(b->in_page_hash);
- ut_ad(!b->in_zip_hash);
-
- switch (buf_page_get_state(b)) {
- case BUF_BLOCK_ZIP_DIRTY:
- HASH_INSERT(buf_page_t, hash, page_hash,
- buf_page_address_fold(b->space,
- b->offset), b);
- break;
- case BUF_BLOCK_FILE_PAGE:
- /* uncompressed page */
- break;
- case BUF_BLOCK_ZIP_FREE:
- case BUF_BLOCK_ZIP_PAGE:
- case BUF_BLOCK_NOT_USED:
- case BUF_BLOCK_READY_FOR_USE:
- case BUF_BLOCK_MEMORY:
- case BUF_BLOCK_REMOVE_HASH:
- ut_error;
- break;
- }
- }
-
- buf_flush_list_mutex_exit(buf_pool);
- buf_pool_mutex_exit(buf_pool);
-}
-
-/********************************************************************
Determine if a block is a sentinel for a buffer pool watch.
@return TRUE if a sentinel for a buffer pool watch, FALSE if not */
UNIV_INTERN
@@ -1913,123 +1573,6 @@ buf_pool_watch_set(
return(NULL);
}
-/********************************************************************//**
-Rebuild buf_pool->page_hash. */
-static
-void
-buf_pool_page_hash_rebuild(void)
-/*============================*/
-{
- ulint i;
-
- for (i = 0; i < srv_buf_pool_instances; i++) {
- buf_pool_page_hash_rebuild_instance(buf_pool_from_array(i));
- }
-}
-
-/********************************************************************//**
-Increase the buffer pool size of one buffer pool instance. */
-static
-void
-buf_pool_increase_instance(
-/*=======================*/
- buf_pool_t* buf_pool, /*!< in: buffer pool instane */
- ulint change_size) /*!< in: new size of the pool */
-{
- buf_chunk_t* chunks;
- buf_chunk_t* chunk;
-
- buf_pool_mutex_enter(buf_pool);
- chunks = mem_alloc((buf_pool->n_chunks + 1) * sizeof *chunks);
-
- memcpy(chunks, buf_pool->chunks, buf_pool->n_chunks * sizeof *chunks);
-
- chunk = &chunks[buf_pool->n_chunks];
-
- if (!buf_chunk_init(buf_pool, chunk, change_size)) {
- mem_free(chunks);
- } else {
- buf_pool->old_pool_size = buf_pool->curr_pool_size;
- buf_pool->curr_size += chunk->size;
- buf_pool->curr_pool_size = buf_pool->curr_size * UNIV_PAGE_SIZE;
- mem_free(buf_pool->chunks);
- buf_pool->chunks = chunks;
- buf_pool->n_chunks++;
- }
-
- buf_pool_mutex_exit(buf_pool);
-}
-
-/********************************************************************//**
-Increase the buffer pool size. */
-static
-void
-buf_pool_increase(
-/*==============*/
- ulint change_size)
-{
- ulint i;
-
- for (i = 0; i < srv_buf_pool_instances; i++) {
- buf_pool_increase_instance(
- buf_pool_from_array(i),
- change_size / srv_buf_pool_instances);
- }
-
- buf_pool_set_sizes();
-}
-
-/********************************************************************//**
-Resizes the buffer pool. */
-UNIV_INTERN
-void
-buf_pool_resize(void)
-/*=================*/
-{
- ulint change_size;
- ulint min_change_size = 1048576 * srv_buf_pool_instances;
-
- buf_pool_mutex_enter_all();
-
- if (srv_buf_pool_old_size == srv_buf_pool_size) {
-
- buf_pool_mutex_exit_all();
-
- return;
-
- } else if (srv_buf_pool_curr_size + min_change_size
- > srv_buf_pool_size) {
-
- change_size = (srv_buf_pool_curr_size - srv_buf_pool_size)
- / UNIV_PAGE_SIZE;
-
- buf_pool_mutex_exit_all();
-
- /* Disable adaptive hash indexes and empty the index
- in order to free up memory in the buffer pool chunks. */
- buf_pool_shrink(change_size);
-
- } else if (srv_buf_pool_curr_size + min_change_size
- < srv_buf_pool_size) {
-
- /* Enlarge the buffer pool by at least one megabyte */
-
- change_size = srv_buf_pool_size - srv_buf_pool_curr_size;
-
- buf_pool_mutex_exit_all();
-
- buf_pool_increase(change_size);
- } else {
- srv_buf_pool_size = srv_buf_pool_old_size;
-
- buf_pool_mutex_exit_all();
-
- return;
- }
-
- buf_pool_page_hash_rebuild();
-}
-
/****************************************************************//**
Remove the sentinel block for the watch before replacing it with a real block.
buf_page_watch_clear() or buf_page_watch_occurred() will notice that
@@ -2365,7 +1908,7 @@ err_exit:
mutex_enter(block_mutex);
/* Discard the uncompressed page frame if possible. */
- if (buf_LRU_free_block(bpage, FALSE) == BUF_LRU_FREED) {
+ if (buf_LRU_free_block(bpage, FALSE)) {
mutex_exit(block_mutex);
goto lookup;
@@ -2768,12 +2311,8 @@ loop:
if (block) {
/* If the guess is a compressed page descriptor that
- has been allocated by buf_buddy_alloc(), it may have
- been invalidated by buf_buddy_relocate(). In that
- case, block could point to something that happens to
- contain the expected bits in block->page. Similarly,
- the guess may be pointing to a buffer pool chunk that
- has been released when resizing the buffer pool. */
+ has been allocated by buf_page_alloc_descriptor(),
+ it may have been freed by buf_relocate(). */
if (!buf_block_is_uncompressed(buf_pool, block)
|| offset != block->page.offset
@@ -2820,6 +2359,9 @@ loop2:
}
if (buf_read_page(space, zip_size, offset)) {
+ buf_read_ahead_random(space, zip_size, offset,
+ ibuf_inside(mtr));
+
retries = 0;
} else if (retries < BUF_PAGE_READ_MAX_RETRIES) {
++retries;
@@ -2951,8 +2493,10 @@ wait_until_unfixed:
if (buf_page_get_state(&block->page)
== BUF_BLOCK_ZIP_PAGE) {
+#if defined UNIV_DEBUG || defined UNIV_BUF_DEBUG
UT_LIST_REMOVE(list, buf_pool->zip_clean,
&block->page);
+#endif /* UNIV_DEBUG || UNIV_BUF_DEBUG */
ut_ad(!block->page.in_flush_list);
} else {
/* Relocate buf_pool->flush_list. */
@@ -2978,11 +2522,10 @@ wait_until_unfixed:
mutex_exit(&buf_pool->zip_mutex);
buf_pool->n_pend_unzip++;
- bpage->state = BUF_BLOCK_ZIP_FREE;
- buf_buddy_free(buf_pool, bpage, sizeof *bpage);
-
buf_pool_mutex_exit(buf_pool);
+ buf_page_free_descriptor(bpage);
+
/* Decompress the page and apply buffered operations
while not holding buf_pool->mutex or block->mutex. */
success = buf_zip_decompress(block, srv_use_checksums);
@@ -3028,7 +2571,7 @@ wait_until_unfixed:
/* Try to evict the block from the buffer pool, to use the
insert buffer (change buffer) as much as possible. */
- if (buf_LRU_free_block(&block->page, TRUE) == BUF_LRU_FREED) {
+ if (buf_LRU_free_block(&block->page, TRUE)) {
mutex_exit(&block->mutex);
if (mode == BUF_GET_IF_IN_POOL_OR_WATCH) {
/* Set the watch, as it would have
@@ -3472,19 +3015,20 @@ buf_page_init_low(
/********************************************************************//**
Inits a page to the buffer buf_pool. */
-static
+static __attribute__((nonnull))
void
buf_page_init(
/*==========*/
+ buf_pool_t* buf_pool,/*!< in/out: buffer pool */
ulint space, /*!< in: space id */
ulint offset, /*!< in: offset of the page within space
in units of a page */
ulint fold, /*!< in: buf_page_address_fold(space,offset) */
- buf_block_t* block) /*!< in: block to init */
+ buf_block_t* block) /*!< in/out: block to init */
{
buf_page_t* hash_page;
- buf_pool_t* buf_pool = buf_pool_get(space, offset);
+ ut_ad(buf_pool == buf_pool_get(space, offset));
ut_ad(buf_pool_mutex_own(buf_pool));
ut_ad(mutex_own(&(block->mutex)));
ut_a(buf_block_get_state(block) != BUF_BLOCK_FILE_PAGE);
@@ -3643,7 +3187,7 @@ err_exit:
ut_ad(buf_pool_from_bpage(bpage) == buf_pool);
- buf_page_init(space, offset, fold, block);
+ buf_page_init(buf_pool, space, offset, fold, block);
/* The block must be put to the LRU list, to the old blocks */
buf_LRU_add_block(bpage, TRUE/* to old blocks */);
@@ -3687,20 +3231,11 @@ err_exit:
mutex_exit(&block->mutex);
} else {
- /* Defer buf_buddy_alloc() until after the block has
- been found not to exist. The buf_buddy_alloc() and
- buf_buddy_free() calls may be expensive because of
- buf_buddy_relocate(). */
-
/* The compressed page must be allocated before the
control block (bpage), in order to avoid the
invocation of buf_buddy_relocate_block() on
uninitialized data. */
data = buf_buddy_alloc(buf_pool, zip_size, &lru);
- bpage = buf_buddy_alloc(buf_pool, sizeof *bpage, &lru);
-
- /* Initialize the buf_pool pointer. */
- bpage->buf_pool_index = buf_pool_index(buf_pool);
/* If buf_buddy_alloc() allocated storage from the LRU list,
it released and reacquired buf_pool->mutex. Thus, we must
@@ -3716,8 +3251,6 @@ err_exit:
/* The block was added by some other thread. */
watch_page = NULL;
- bpage->state = BUF_BLOCK_ZIP_FREE;
- buf_buddy_free(buf_pool, bpage, sizeof *bpage);
buf_buddy_free(buf_pool, data, zip_size);
bpage = NULL;
@@ -3725,6 +3258,11 @@ err_exit:
}
}
+ bpage = buf_page_alloc_descriptor();
+
+ /* Initialize the buf_pool pointer. */
+ bpage->buf_pool_index = buf_pool_index(buf_pool);
+
page_zip_des_init(&bpage->zip);
page_zip_set_size(&bpage->zip, zip_size);
bpage->zip.data = data;
@@ -3739,7 +3277,6 @@ err_exit:
bpage->space = space;
bpage->offset = offset;
-
#ifdef UNIV_DEBUG
bpage->in_page_hash = FALSE;
bpage->in_zip_hash = FALSE;
@@ -3764,7 +3301,9 @@ err_exit:
/* The block must be put to the LRU list, to the old blocks */
buf_LRU_add_block(bpage, TRUE/* to old blocks */);
+#if defined UNIV_DEBUG || defined UNIV_BUF_DEBUG
buf_LRU_insert_zip_clean(bpage);
+#endif /* UNIV_DEBUG || UNIV_BUF_DEBUG */
buf_page_set_io_fix(bpage, BUF_IO_READ);
@@ -3852,7 +3391,7 @@ buf_page_create(
mutex_enter(&block->mutex);
- buf_page_init(space, offset, fold, block);
+ buf_page_init(buf_pool, space, offset, fold, block);
/* The block must be put to the LRU list */
buf_LRU_add_block(&block->page, FALSE);
@@ -3936,6 +3475,53 @@ buf_page_create(
}
/********************************************************************//**
+Mark a table with the specified space pointed by bpage->space corrupted.
+Also remove the bpage from LRU list.
+@return TRUE if successful */
+static
+ibool
+buf_mark_space_corrupt(
+/*===================*/
+ buf_page_t* bpage) /*!< in: pointer to the block in question */
+{
+ buf_pool_t* buf_pool = buf_pool_from_bpage(bpage);
+ const ibool uncompressed = (buf_page_get_state(bpage)
+ == BUF_BLOCK_FILE_PAGE);
+ ulint space = bpage->space;
+ ibool ret = TRUE;
+
+ /* First unfix and release lock on the bpage */
+ buf_pool_mutex_enter(buf_pool);
+ mutex_enter(buf_page_get_mutex(bpage));
+ ut_ad(buf_page_get_io_fix(bpage) == BUF_IO_READ);
+ ut_ad(bpage->buf_fix_count == 0);
+
+ /* Set BUF_IO_NONE before we remove the block from LRU list */
+ buf_page_set_io_fix(bpage, BUF_IO_NONE);
+
+ if (uncompressed) {
+ rw_lock_x_unlock_gen(
+ &((buf_block_t*) bpage)->lock,
+ BUF_IO_READ);
+ }
+
+ /* Find the table with specified space id, and mark it corrupted */
+ if (dict_set_corrupted_by_space(space)) {
+ buf_LRU_free_one_page(bpage);
+ } else {
+ ret = FALSE;
+ }
+
+ ut_ad(buf_pool->n_pend_reads > 0);
+ buf_pool->n_pend_reads--;
+
+ mutex_exit(buf_page_get_mutex(bpage));
+ buf_pool_mutex_exit(buf_pool);
+
+ return(ret);
+}
+
+/********************************************************************//**
Completes an asynchronous read or write request of a file page to or from
the buffer pool. */
UNIV_INTERN
@@ -4060,10 +3646,19 @@ corrupt:
"InnoDB: about forcing recovery.\n", stderr);
if (srv_force_recovery < SRV_FORCE_IGNORE_CORRUPT) {
- fputs("InnoDB: Ending processing because of"
- " a corrupt database page.\n",
- stderr);
- exit(1);
+ /* If page space id is larger than TRX_SYS_SPACE
+ (0), we will attempt to mark the corresponding
+ table as corrupted instead of crashing server */
+ if (bpage->space > TRX_SYS_SPACE
+ && buf_mark_space_corrupt(bpage)) {
+ return;
+ } else {
+ fputs("InnoDB: Ending processing"
+ " because of"
+ " a corrupt database page.\n",
+ stderr);
+ ut_error;
+ }
}
}
@@ -4847,6 +4442,7 @@ buf_stats_aggregate_pool_info(
total_info->n_pages_created += pool_info->n_pages_created;
total_info->n_pages_written += pool_info->n_pages_written;
total_info->n_page_gets += pool_info->n_page_gets;
+ total_info->n_ra_pages_read_rnd += pool_info->n_ra_pages_read_rnd;
total_info->n_ra_pages_read += pool_info->n_ra_pages_read;
total_info->n_ra_pages_evicted += pool_info->n_ra_pages_evicted;
total_info->page_made_young_rate += pool_info->page_made_young_rate;
@@ -4859,6 +4455,7 @@ buf_stats_aggregate_pool_info(
total_info->page_read_delta += pool_info->page_read_delta;
total_info->young_making_delta += pool_info->young_making_delta;
total_info->not_young_making_delta += pool_info->not_young_making_delta;
+ total_info->pages_readahead_rnd_rate += pool_info->pages_readahead_rnd_rate;
total_info->pages_readahead_rate += pool_info->pages_readahead_rate;
total_info->pages_evicted_rate += pool_info->pages_evicted_rate;
total_info->unzip_lru_len += pool_info->unzip_lru_len;
@@ -4936,6 +4533,7 @@ buf_stats_get_pool_info(
pool_info->n_page_gets = buf_pool->stat.n_page_gets;
+ pool_info->n_ra_pages_read_rnd = buf_pool->stat.n_ra_pages_read_rnd;
pool_info->n_ra_pages_read = buf_pool->stat.n_ra_pages_read;
pool_info->n_ra_pages_evicted = buf_pool->stat.n_ra_pages_evicted;
@@ -4975,6 +4573,10 @@ buf_stats_get_pool_info(
buf_pool->stat.n_pages_not_made_young
- buf_pool->old_stat.n_pages_not_made_young;
}
+ pool_info->pages_readahead_rnd_rate =
+ (buf_pool->stat.n_ra_pages_read_rnd
+ - buf_pool->old_stat.n_ra_pages_read_rnd) / time_elapsed;
+
pool_info->pages_readahead_rate =
(buf_pool->stat.n_ra_pages_read
@@ -5060,9 +4662,12 @@ buf_print_io_instance(
/* Statistics about read ahead algorithm */
fprintf(file, "Pages read ahead %.2f/s,"
- " evicted without access %.2f/s\n",
+ " evicted without access %.2f/s,"
+ " Random read ahead %.2f/s\n",
+
pool_info->pages_readahead_rate,
- pool_info->pages_evicted_rate);
+ pool_info->pages_evicted_rate,
+ pool_info->pages_readahead_rnd_rate);
/* Print some values to help us with visualizing what is
happening with LRU eviction. */