buf_flush_buffer_pool(): Wait for any pending asynchronous reads
to complete. This assertion failed in a run where buf_read_ahead_linear()
had been triggered in an SQL statement that was executed right
before shutdown.
Reviewed by: Debarun Banerjee
btr_cur_t::search_leaf(): When the index root page is also a leaf page,
we may need to upgrade our existing shared root page latch into an
exclusive latch. Even if we end up waiting, the root page won't be able
to go away while we hold an index()->lock. The index page may be split;
that is all.
btr_latch_prev(): Acquire the page latch while holding a buffer-fix
and an index tree latch. Merge the change buffer if needed. Use
buf_pool_t::page_fix() for this special case instead of complicating
buf_page_get_low() and buf_page_get_gen().
row_merge_read_clustered_index(): Remove some code that does not seem
to be useful. No difference was observed with regard to removing this
code when a CREATE INDEX or OPTIMIZE TABLE statement was run concurrently
with sysbench oltp_update_index --tables=1 --table_size=1000 --threads=16.
buf_pool_t::unzip(): Decompress a ROW_FORMAT=COMPRESSED page.
buf_pool_t::page_fix(): Handle also ROW_FORMAT=COMPRESSED pages
as well as change buffer merge. Optionally return an error.
Add a flag for suppressing a page latch wait and a special return
value -1 to indicate that the call would block.
This is the preferred way of buffer-fixing blocks.
The functions buf_page_get_gen() and buf_page_get_low() are only being
invoked with rw_latch=RW_NO_LATCH in operations on SPATIAL INDEX.
buf_page_t: Define some static functions for interpreting state().
buf_page_get_zip(), buf_read_page(),
buf_read_ahead_random(), buf_read_ahead_linear():
Remove the redundant parameter zip_size. We must look up the
tablespace and can invoke fil_space_t::zip_size() on it.
buf_page_get_low(): Require mtr!=nullptr.
buf_page_get_gen(): Implement some lock downgrading during recovery.
ibuf_page_low(): Use buf_pool_t::page_fix() in a debug check.
We do wait for a page read here, because otherwise a debug assertion in
buf_page_get_low() in the test innodb.ibuf_delete could occasionally fail.
PageConverter::operator(): Invoke buf_pool_t::page_fix() in order
to possibly evict a block. This allows us to remove some
special case code from buf_page_get_low().
In commit fa8a46eb68 (MDEV-33613)
the parameter innodb_lru_flush_size ceased to have any effect.
Let us declare the parameter as deprecated and additionally as
MARIADB_REMOVED_OPTION, so that there will be a warning written
to the error log in case the option is specified in the command line.
Let us also do the same for the parameter
innodb_purge_rseg_truncate_frequency
that was deprecated&ignored earlier in MDEV-32050.
Reviewed by: Debarun Banerjee
In a Sysbench oltp_update_index workload that involves 1 table,
a serious contention between the workload and the purge of history
was observed. This was the worst when the table contained only 1 record.
This turned out to be fixed by setting innodb_purge_batch_size=128,
which corresponds to the number of usable persistent rollback segments.
When we go above that, there would be contention between row_purge_poss_sec()
and the workload, typically on the clustered index page latch, sometimes
also on a secondary index page latch. It might be that with smaller
batches, trx_sys.history_size() will end up pausing all concurrent
transaction start/commit frequently enough so that purge will be able
to make some progress, so that there would be less contention on the
index page latches between purge and SQL execution.
In commit aa719b5010 (part of MDEV-32050)
the interpretation of the parameter innodb_purge_batch_size was slightly
changed. It would correspond to the maximum desired size of the
purge_sys.pages cache. Before that change, the parameter was referring to
a number of undo log pages, but the accounting might have been inaccurate.
To avoid a regression, we will reduce the default value to
innodb_purge_batch_size=127, which will also be compatible with
innodb_undo_tablespaces>1 (which will disable rollback segment 0).
Additionally, some logic in the purge and MVCC checks is simplified.
The purge tasks will make use of purge_sys.pages when accessing undo
log pages to find out if a secondary index record can be removed.
If an undo page needs to be looked up in buf_pool.page_hash, we will
merely buffer-fix it. This is correct, because the undo pages are
append-only in nature. Holding purge_sys.latch or purge_sys.end_latch
or the fact that the current thread is executing as a part of an
in-progress purge batch will prevent the contents of the undo page from
being freed and subsequently reused. The buffer-fix will prevent the
page from being evicted form the buffer pool. Thanks to this logic,
we can refer to the undo log record directly in the buffer pool page
and avoid copying the record.
buf_pool_t::page_fix(): Look up and buffer-fix a page. This is useful
for accessing undo log pages, which are append-only by nature.
There will be no need to deal with change buffer or ROW_FORMAT=COMPRESSED
in that case.
purge_sys_t::view_guard::view_guard(): Allow the type of guard to be
acquired: end_latch, latch, or no latch (in case we are a purge thread).
purge_sys_t::view_guard::get(): Read-only accessor to purge_sys.pages.
purge_sys_t::get_page(): Invoke buf_pool_t::page_fix().
row_vers_old_has_index_entry(): Replaced with row_purge_is_unsafe()
and row_undo_mod_sec_unsafe().
trx_undo_get_undo_rec(): Merged to trx_undo_prev_version_build().
row_purge_poss_sec(): Add the parameter mtr and remove redundant
or unused parameters sec_pcur, sec_mtr, is_tree. We will use the
caller's mtr object but release any acquired page latches before
returning.
btr_cur_get_page(), page_cur_get_page(): Do not invoke page_align().
row_purge_remove_sec_if_poss_leaf(): Return the value of PAGE_MAX_TRX_ID
to be checked against the page in row_purge_remove_sec_if_poss_tree().
If the secondary index page was not changed meanwhile, it will be
unnecessary to invoke row_purge_poss_sec() again.
trx_undo_prev_version_build(): Access any undo log pages using
the caller's mini-transaction object.
row_purge_vc_matches_cluster(): Moved to the only compilation unit that
needs it.
Reviewed by: Debarun Banerjee
buf_page_ibuf_merge_try(): A new, separate function for invoking
ibuf_merge_or_delete_for_page() when needed. Use the already requested
page latch for determining if the call is necessary. If it is and
if we are currently holding rw_latch==RW_S_LATCH, upgrading to an exclusive
latch may involve waiting that another thread acquires and releases
a U or X latch on the page. If we have to wait, we must recheck if the
call to ibuf_merge_or_delete_for_page() is still needed. If the page
turns out to be corrupted, we will release and fail the operation.
Finally, the exclusive page latch will be downgraded to the originally
requested latch.
ssux_lock_impl::rd_u_upgrade_try(): Attempt to upgrade a shared lock to
an update lock.
sux_lock::s_x_upgrade_try(): Attempt to upgrade a shared lock to
exclusive.
sux_lock::s_x_upgrade(): Upgrade a shared lock to exclusive.
Return whether a wait was elided.
ssux_lock_impl::u_rd_downgrade(), sux_lock::u_s_downgrade():
Downgrade an update lock to shared.
When SUX_LOCK_GENERIC is defined, the srw_mutex, srw_lock, sux_lock are
implemented based on pthread_mutex_t and pthread_cond_t. This is the
only option for systems that lack a futex-like system call.
In the SUX_LOCK_GENERIC mode, if pthread_mutex_init() is allocating
some resources that need to be freed by pthread_mutex_destroy(),
a memory leak could occur when we are repeatedly invoking
pthread_mutex_init() without a pthread_mutex_destroy() in between.
pthread_mutex_wrapper::initialized: A debug field to track whether
pthread_mutex_init() has been invoked. This also helps find bugs
like the one that was fixed by
commit 1c8af2ae53 (MDEV-34422);
one simply needs to add -DSUX_LOCK_GENERIC to the CMAKE_CXX_FLAGS
to catch that particular bug on the initial server bootstrap.
buf_block_init(), buf_page_init_for_read(): Invoke block_lock::init()
because buf_page_t::init() will no longer do that.
buf_page_t::init(): Instead of invoking lock.init(), assert that it
has already been invoked (the lock is vacant).
add_fts_index(), build_fts_hidden_table(): Explicitly invoke
index_lock::init() in order to avoid a pthread_mutex_destroy()
invocation on an uninitialized object.
srw_lock_debug::destroy(): Invoke readers_lock.destroy().
trx_sys_t::create(): Invoke trx_rseg_t::init() on all rollback segments
in order to guarantee a deterministic state for shutdown, even if
InnoDB fails to start up.
trx_rseg_array_init(), trx_temp_rseg_create(), trx_rseg_create():
Invoke trx_rseg_t::destroy() before trx_rseg_t::init() in order to
balance pthread_mutex_init() and pthread_mutex_destroy() calls.
buf_page_get_gen(): Relax the assertion once more.
The LSN may grow by invoking ibuf_upgrade(), that is,
when upgrading files where innodb_change_buffering!=none was used.
The LSN may also have been recovered from a log that needs
to be upgraded to the current format.
Problem:
========
- During shutdown, InnoDB tries to free the asynchronous
I/O slots and hangs. The reason is that InnoDB disables
asynchronous I/O before waiting for pending
asynchronous I/O to finish.
buf_load(): InnoDB aborts the buffer pool load due to
user requested shutdown and doesn't wait for the asynchronous
read to get completed. This could lead to debug assertion
in buf_flush_buffer_pool() during shutdown
Fix:
===
os_aio_free(): Should wait all read_slots and write_slots
to finish before disabling the aio.
buf_load(): Should wait for pending read request to complete
even though it was aborted.
Problem:
========
- innodb.temp_truncate_freed fails with ER_WRONG_ARGUMENTS and
states that another buffer pool resize is already in progress.
Test case has wait condition to ensure that buffer pool
resize is completed. There is a possibility that wait condition
check could get false impression that InnoDB buffer pool resize
completed due to previous buffer pool resize.
Fix:
===
Add more elaborate wait_condition to ensure the current buffer
pool resize completed.
buf_pool_t::resize(): Set the buffer pool resize status only
after setting previous buffer pool size to current buffer pool
size. This should help the test case to make reliable.
The symptoms were: take a server with no activity and a table that's
not in the buffer pool. Run a query that reads the whole table and
observe that r_engine_stats.pages_read_count shows about 2% of the table
was read. Who reads the rest?
The cause was that page prefetching done inside InnoDB was not counted.
This counts page prefetch requests made in buf_read_ahead_random() and
buf_read_ahead_linear() and makes them visible in:
- ANALYZE: r_engine_stats.pages_prefetch_read_count
- Slow Query Log: Pages_prefetched:
This patch intentionally doesn't attempt to count the time to read the
prefetched pages:
* there's no obvious place where one can do it
* prefetch reads may be done in parallel (right?), it is not clear how
to count the time in this case.
The performance regression seen while loading BP is caused by the
deadlock fix given in MDEV-33543. The area of impact is wider but is
more visible when BP is being loaded initially via DMLs. Specifically
the response time could be impacted in DML doing pessimistic operation
on index(split/merge) and the leaf pages are not found in buffer pool.
It is more likely to occur with small BP size.
The origin of the issue dates back to MDEV-30400 that introduced
btr_cur_t::search_leaf() replacing btr_cur_search_to_nth_level() for
leaf page searches. In btr_latch_prev, we use RW_NO_LATCH to get the
previous page fixed in BP without latching. When the page is not in BP,
we try to acquire and wait for S latch violating the latching order.
This deadlock was analyzed in MDEV-33543 and fixed by using the already
present wait logic in buf_page_get_gen() instead of waiting for latch.
The wait logic is inferior to usual S latch wait and is simply a
repeated sleep 100 of micro-sec (The actual sleep time could be more
depending on platforms). The bug was seen with "change-buffering" code
path and the idea was that this path should be less exercised. The
judgement was not correct and the path is actually quite frequent and
does impact performance when pages are not in BP and being loaded by
DML expanding/shrinking large data.
Fix: While trying to get a page with RW_NO_LATCH and we are attempting
"out of order" latch, return from buf_page_get_gen immediately instead
of waiting and follow the ordered latching path.
As part of commit 685d958e38 (MDEV-14425)
the parameter innodb_log_write_ahead_size was removed, because it was
thought that determining the physical block size would be a sufficient
replacement.
However, we can only determine the physical block size on Linux or
Microsoft Windows. On some file systems, the physical block size
is not relevant. For example, XFS uses a block size of 4096 bytes
even if the underlying block size may be smaller.
On Linux, we failed to determine the physical block size if
innodb_log_file_buffered=OFF was not requested or possible.
This will be fixed.
log_sys.write_size: The value of the reintroduced parameter
innodb_log_write_ahead_size. To keep it simple, this is read-only
and a power of two between 512 and 4096 bytes, so that the previous
alignment guarantees are fulfilled. This will replace the previous
log_sys.get_block_size().
log_sys.block_size, log_t::get_block_size(): Remove.
log_t::set_block_size(): Ensure that write_size will not be less
than the physical block size. There is no point to invoke this
function with 512 or less, because that is the minimum value of
write_size.
innodb_params_adjust(): Add some disabled code for adjusting
the minimum value and default value of innodb_log_write_ahead_size
to reflect the log_sys.write_size.
log_t::set_recovered(): Mark the recovery completed. This is the
place to adjust some things if we want to allow write_size>4096.
log_t::resize_write_buf(): Refer to write_size.
log_t::resize_start(): Refer to write_size instead of get_block_size().
log_write_buf(): Simplify some arithmetics and remove a goto.
log_t::write_buf(): Refer to write_size. If we are writing less than
that, do not switch buffers, but keep writing to the same buffer.
Move some code to improve the locality of reference.
recv_scan_log(): Refer to write_size instead of get_block_size().
os_file_create_func(): For type==OS_LOG_FILE on Linux, always invoke
os_file_log_maybe_unbuffered(), so that log_sys.set_block_size() will
be invoked even if we are not attempting to use O_DIRECT.
recv_sys_t::find_checkpoint(): Read the entire log header
in a single 12 KiB request into log_sys.buf.
Tested with:
./mtr --loose-innodb-log-write-ahead-size=4096
./mtr --loose-innodb-log-write-ahead-size=2048
BTR_MODIFY_PREV: Remove. This mode was only used by the change buffer,
which commit f27e9c8947 (MDEV-29694)
removed.
buf_page_get_gen(): Revert the change that was made in
commit 90b95c6149 (MDEV-33543)
because it is not applicable after MDEV-29694. This fixes the
performance regression that Vladislav Vaintroub reported.
This is a 11.x specific fix; this needs to be fixed differently
in older major versions where the change buffer is present.
In commit 99bd22605938c42d876194f2ec75b32e658f00f5 (MDEV-31558)
we wrongly thought that there would be minimal overhead for accessing
a thread-local variable mariadb_stats.
It turns out that in C++11, each access to an extern thread_local
variable requires conditionally invoking an initialization function.
In fact, the initializer expression of mariadb_stats is dynamic, and
those calls were actually unavoidable.
In C++20, one could declare constinit thread_local variables, but
the address of a thread_local variable (&mariadb_dummy_stats) is not
a compile-time constant. We did not want to declare mariadb_dummy_stats
without thread_local, because then the dummy accesses could lead to
cache line contention between threads.
mariadb_stats: Declare as __thread or __declspec(thread) so that
there will be no dynamic initialization, but zero-initialization.
mariadb_dummy_stats: Remove. It is a lesser evil to let
the environment perform zero-initialization and check if
!mariadb_stats.
Reviewed by: Sergei Petrunia
buf_pool_invalidate(): Properly wait for
os_aio_wait_until_no_pending_writes() to ensure so that there
are no pending buf_page_t::write_complete() or buf_page_write_complete()
operations. This will avoid a failure of buf_pool.assert_all_freed().
This bug should affect debug builds only. At this point, the
buf_pool.flush_list should be clear and all changes should have
been written out. The loop around buf_LRU_scan_and_free_block() should
have eventually completed and freed all pages as soon as
buf_page_t::write_complete() had a chance to release the page latches.
It is worth noting that buf_flush_wait() is working as intended.
As soon as buf_flush_page_cleaner() invokes
buf_pool.get_oldest_modification() it will observe that
buf_page_t::write_complete() had assigned oldest_modification_ to 1,
and remove such blocks from buf_pool.flush_list. Upon reaching
buf_pool.flush_list.count=0 the buf_flush_page_cleaner() will mark
itself idle and wake buf_flush_wait() by broadcasting
buf_pool.done_flush_list.
This regression was introduced in
commit a55b951e60 (MDEV-26827).
Reviewed by: Debarun Banerjee
When checkpoint age goes beyond the sync flush threshold and
buf_flush_sync_lsn is set, page cleaner enters into "furious flush"
stage to aggressively flush dirty pages from flush list and pull
checkpoint LSN above safe margin. In this stage, page cleaner skips
doing LRU flush and eviction.
In 10.6, all other threads entirely rely on page cleaner to generate
free pages. If free pages get over while page cleaner is busy in
"furious flush" stage, a session thread could wait for free page in the
middle of a min-transaction(mtr) while holding latches on other pages.
It, in turn, can prevent page cleaner to flush such pages preventing
checkpoint LSN to move forward creating a deadlock situation. Even
otherwise, it could create a stall and hang like situation for large BP
with plenty of dirty pages to flush before the stage could finish.
Fix: During furious flush, check and evict LRU pages after each flush
iteration.
trx_undo_mem_create_at_db_start(): Invoke recv_sys_t::recover()
instead of buf_page_get_gen(), so that all undo log pages will be
recovered correctly. Failure to do this could prevent InnoDB from
starting up due to "Data structure corruption", or it could
potentially lead to a situation where InnoDB starts up but some
transactions were recovered incorrectly.
recv_sys_t::recover(): Only acquire a buffer-fix on the pages,
not a shared latch. This is adequate protection, because this function
is only being invoked during early startup when no "users" are modifying
buffer pool pages. The only writes are due to server bootstrap
(the data files being created) or crash recovery (changes from
ib_logfile0 being applied).
buf_page_get_gen(): Assert that the function is not invoked while crash
recovery is in progress, and that the special mode BUF_GET_RECOVER is
only invoked during crash recovery or server bootstrap.
All this should really have been part of
commit 850d61736d (MDEV-32042).
in buf_dblwr_t::init_or_load_pages()
- InnoDB fails to set the TRX_SYS_DOUBLEWRITE_SPACE_ID_STORED
flag in transaction system header page while recreating
the undo log tablespaces
buf_dblwr_t::init_or_load_pages(): Tries to reset the
space id and try to write into doublewrite buffer even
when read_only mode is enabled.
In srv_all_undo_tablespaces_open(), InnoDB should try to
open the extra unused undo tablespaces instead of trying to
creating it.
trx_undo_mem_create_at_db_start(): Invoke recv_sys_t::recover()
instead of buf_page_get_gen(), so that all undo log pages will be
recovered correctly. Failure to do this could prevent InnoDB from
starting up due to "Data structure corruption", or it could
potentially lead to a situation where InnoDB starts up but some
transactions were recovered incorrectly.
recv_sys_t::recover(): Only acquire a buffer-fix on the pages,
not a shared latch. This is adequate protection, because this function
is only being invoked during early startup when no "users" are modifying
buffer pool pages. The only writes are due to server bootstrap
(the data files being created) or crash recovery (changes from
ib_logfile0 being applied).
buf_page_get_gen(): Assert that the function is not invoked while crash
recovery is in progress, and that the special mode BUF_GET_RECOVER is
only invoked during crash recovery or server bootstrap.
All this should really have been part of
commit 850d61736d (MDEV-32042).
in buf_dblwr_t::init_or_load_pages()
- InnoDB fails to set the TRX_SYS_DOUBLEWRITE_SPACE_ID_STORED
flag in transaction system header page while recreating
the undo log tablespaces
buf_dblwr_t::init_or_load_pages(): Tries to reset the
space id and try to write into doublewrite buffer even
when read_only mode is enabled.
In srv_all_undo_tablespaces_open(), InnoDB should try to
open the extra unused undo tablespaces instead of trying to
creating it.
BUF_LRU_MIN_LEN (256) is too high value for low buffer pool(BP) size.
For example, for BP size lower than 80M and 16 K page size, the limit is
more than 5% of total BP and for lowest BP 5M, it is 80% of the BP.
Non-data objects like explicit locks could occupy part of the BP pool
reducing the pages available for LRU. If LRU reaches minimum limit and
if no free pages are available, server would hang with page cleaner not
able to free any more pages.
Fix: To avoid such hang, we adjust the LRU limit lower than the limit
for data objects as checked in buf_LRU_check_size_of_non_data_objects()
i.e. one page less than 5% of BP.
This regression is introduced in 10.6 by following commit.
commit b6a2472489
MDEV-27891: SIGSEGV in InnoDB buffer pool resize
During DML, we check if buffer pool is running out of data pages in
buf_pool_t::running_out. Here is 75% of the buffer pool is occupied by
non-data pages we rollback the current transaction and exit with
ER_LOCK_TABLE_FULL.
The integer division (n_chunks_new / 4) becomes zero whenever the total
number of chunks are < 4 making the check completely ineffective for
such cases. Also the check is inaccurate for larger chunks.
Fix-1: Correct the check in buf_pool_t::running_out.
Fix-2: While waiting for free page, check for
buf_LRU_check_size_of_non_data_objects.
In the merge commit f9807aadef
there were some omissions or errors.
ibuf_remove_free_page(): Return an error if the free list is corrupted
when removing the change buffer on an upgrade. A special 11.0 version of
commit 263932d505 would have been useful.
buf_page_get_gen(): Correctly handle the case that a page was being
concurrently read into the buffer pool and found out to be corrupted.
This was part of commit a4cda66e2d
but had been discarded in the merge.
Because MariaDB Server 11.0 has reached its end of life as of
commit 466ae1cf81 this fix is being applied
to the 11.1 branch.
Issue: When getting a page (buf_page_get_gen) with no latch option
(RW_NO_LATCH), the caller is not expected to follow the B-tree latching
order. However in buf_page_get_low we try to acquire shared page latch
unconditionally to wait for a page that is being loaded by another
thread concurrently. In general it could lead to latch order violation
and deadlock.
Currently it affects the change buffer insert path btr_latch_prev()
which tries to load the previous page out of order with RW_NO_LATCH and
two concurrent inserts into IBUF tree cause deadlock. This problem is
introduced in 10.6 by following commit.
commit 9436c778c3 (MDEV-27058)
Fix: While trying to latch a page with RW_NO_LATCH, always use the
"*lock_try" interface and retry operation on failure after unfixing the
page.
Because the Red Hat Enterprise Linux 8 core repository does not include
libpmem, let us implement the necessary subset ourselves.
pmem_persist(): Implement for 64-bit x86, ARM, POWER, RISC-V, Loongarch
in a way that should be compatible with the https://github.com/pmem/pmdk/
implementation of pmem_persist().
The CMake option WITH_INNODB_PMEM can be used for enabling or disabling
this interface at compile time. By default, it is enabled on all applicable
systems that are covered by our CI system.
Note: libpmem had not been previously enabled for Loongarch in our
Debian packaging. It was enabled for RISC-V, but we will not enable it
by default on RISC-V or Loongarch because we lack CI coverage.
The generated code for x86_64 was reviewed and tested on two
Intel implementations: one that only supports clflush, and
another that supports both clflushopt and clwb.
The generated machine code was also reviewed on https://godbolt.org
using various compiler versions. Godbolt helpfully includes an option
to compile to binary code and display the encoding, which was
useful on POWER.
Reviewed by: Vladislav Vaintroub
In so-called optimistic buffer pool lookups, we must not
dereference a block descriptor before we have made sure that
it is accessible. While buf_pool_t::resize() is running,
block descriptors could become invalid.
The buf::Block_hint class was essentially duplicating
a buf_pool.page_hash lookup that was executed in
buf_page_optimistic_get() anyway. For better locality of
reference, we had better execute that lookup only once.
buf_page_optimistic_fix(): Prepare for buf_page_optimistic_get().
This basically is a simpler version of Buf::Block_hint.
buf_page_optimistic_get(): Assume that buf_page_optimistic_fix()
has been called and the page identifier verified. Should the block
be evicted, the block->modify_clock will be invalidated; we do not
need to check the block->page.id() again. It suffices to check
the block->modify_clock after acquiring the page latch.
btr_pcur_t::old_page_id: Store the expected page identifier
for buf_page_optimistic_fix().
btr_pcur_t::block_when_stored: Remove. This was duplicating
page_cur_t::block.
btr_pcur_optimistic_latch_leaves(): Remove redundant parameters.
First, invoke buf_page_optimistic_fix() on the requested page.
If needed, acquire a latch on the left page. Finally, acquire
a latch on the target page and recheck the block->modify_clock.
If the page had been freed while we were not holding a page latch,
fall back to the slow path. Validate the FIL_PAGE_PREV after
acquiring a latch on the current page. The block->modify_clock
is only being incremented when records are deleted or pages
reorganized or evicted; it does not guard against concurrent
page splits.
Reviewed by: Debarun Banerjee
In commit 24648768b4 (MDEV-30136)
the parameter innodb_flush_method was deprecated, with no direct
replacement for innodb_flush_method=O_DIRECT_NO_FSYNC.
Let us change innodb_doublewrite from Boolean to ENUM that can
be changed while the server is running:
OFF: Assume that writes of innodb_page_size are atomic
ON: Prevent torn writes (the default)
fast: Like ON, but avoid synchronizing writes to data files
The deprecated start-up parameter innodb_flush_method=NO_FSYNC will cause
innodb_doublewrite=ON to be changed to innodb_doublewrite=fast,
which will prevent InnoDB from making any durable writes to data files.
This would normally be done right before the log checkpoint LSN is updated.
Depending on the file systems being used and their configuration,
this may or may not be safe.
The value innodb_doublewrite=fast differs from the previous combination of
innodb_doublewrite=ON and innodb_flush_method=O_DIRECT_NO_FSYNC by always
invoking os_file_flush() on the doublewrite buffer itself
in buf_dblwr_t::flush_buffered_writes_completed(). This should be safer
when there are multiple doublewrite batches between checkpoints.
Typically, once per second, buf_flush_page_cleaner() would write out
up to innodb_io_capacity pages and advance the log checkpoint.
Also typically, innodb_io_capacity>128, which is the size of the
doublewrite buffer in pages. Should os_file_flush_func() not be invoked
between doublewrite batches, writes could be reordered in an unsafe way.
The setting innodb_doublewrite=fast could be safe when the doublewrite
buffer (the first file of the system tablespace) and the data files
reside in the same file system.
This was tested by running "./mtr --rr innodb.alter_kill". On the first
server startup, with innodb_doublewrite=fast, os_file_flush_func()
would only be invoked on the ibdata1 file and possibly ib_logfile0.
On subsequent startups with innodb_doublewrite=OFF, os_file_flush_func()
will be invoked on the individual data files during log_checkpoint().
Note: The setting debug_no_sync (in the code, my_disable_sync) would
disable all durable writes to InnoDB files, which would be much less safe.
IORequest::Type: Introduce special values WRITE_DBL and PUNCH_DBL
for asynchronous writes that are submitted via the doublewrite buffer.
In this way, fil_space_t::use_doublewrite() or buf_dblwr.in_use()
will only be consulted during buf_page_t::flush() and the doublewrite
buffer can be enabled or disabled without any fear of inconsistency.
buf_dblwr_t::block_size: Replaces block_size().
buf_dblwr_t::flush_buffered_writes(): If !in_use() and the doublewrite
buffer is empty, just invoke fil_flush_file_spaces() and return. The
doublewrite buffer could have been disabled while a batch was in
progress.
innodb_init_params(): If innodb_flush_method=O_DIRECT_NO_FSYNC,
set innodb_doublewrite=fast or innodb_doublewrite=fearless.
Thanks to Mark Callaghan for reporting this, and Vladislav Vaintroub
for feedback.
Some fixes related to commit f838b2d799 and
Rows_log_event::do_apply_event() and Update_rows_log_event::do_exec_row()
for system-versioned tables were provided by Nikita Malyavin.
This was required by test versioning.rpl,trx_id,row.
MONITOR_INC_VALUE_CUMULATIVE is a multiline macro, so the second statement
will be executed always, regardless of "if" condition.
These problems first started with
commit b1ab211dee (MDEV-15053).
Thanks to Yury Chaikou from ServiceNow for the report.
By design, InnoDB has always hung when permanently running out of
buffer pool, for example when several threads are waiting to allocate
a block, and all of the buffer pool is buffer-fixed by the active threads.
The hang that we are fixing here occurs when the buffer pool is only
temporarily running out and the situation could be rescued by writing out
some dirty pages or evicting some clean pages.
buf_LRU_get_free_block(): Simplify the way how we wait for
the buf_flush_page_cleaner thread. This fixes occasional hangs
of the test encryption.innochecksum that were introduced by
commit a55b951e60 (MDEV-26827).
To play it safe, we use a timed wait when waiting for the
buf_flush_page_cleaner() thread to perform its job. Should that
thread get stuck, we will invoke buf_pool.LRU_warn() in order to
display a message that pages could not be freed, and keep trying
to wake up the buf_flush_page_cleaner() thread.
The INFORMATION_SCHEMA.INNODB_METRICS counters
buffer_LRU_single_flush_failure_count and
buffer_LRU_get_free_waits will be removed.
The latter is represented by buffer_pool_wait_free.
Also removed will be the message
"InnoDB: Difficult to find free blocks in the buffer pool"
because in d34479dc66 we
introduced a more precise message
"InnoDB: Could not free any blocks in the buffer pool"
in the buf_flush_page_cleaner thread.
buf_pool_t::LRU_warn(): Issue the warning message that we could
not free any blocks in the buffer pool. This may also be invoked
by buf_LRU_get_free_block() if buf_flush_page_cleaner() appears
to be stuck.
buf_pool_t::n_flush_dec(): Remove.
buf_pool_t::n_flush_dec_holding_mutex(): Rename to n_flush_dec().
buf_flush_LRU_list_batch(): Increment the eviction counter for blocks
of temporary, discarded or dropped tablespaces.
buf_flush_LRU(): Make static, and remove the constant parameter
evict=false. The only caller will be the buf_flush_page_cleaner()
thread.
IORequest::is_LRU(): Remove. The only case of evicting pages on
write completion will be when we are writing out pages of the
temporary tablespace. Those pages are not in buf_pool.flush_list,
only in buf_pool.LRU.
buf_page_t::flush(): Remove the parameter evict.
buf_page_t::write_complete(): Change the parameter "bool temporary"
to "bool persistent" and add a parameter for an already read state().
Reviewed by: Debarun Banerjee
The log_sys.lsn_lock is a very contended resource with a small
critical section in log_sys.append_prepare(). On many processor
microarchitectures, replacing the system call based log_sys.lsn_lock
with a pure spin lock would fare worse during high concurrency workloads,
wasting a significant amount of CPU cycles in the spin loop.
On other microarchitectures, we would see a significant amount of time
being spent in native_queued_spin_lock_slowpath() in the Linux kernel,
plus context switching between user and kernel address space. This was
pointed out by Steve Shaw from Intel Corporation.
Depending on the workload and the hardware implementation, it may be
useful to use a pure spin lock in log_sys.append_prepare().
We will introduce a parameter. The statement
SET GLOBAL INNODB_LOG_SPIN_WAIT_DELAY=50;
would enable a spin lock that will execute that many MY_RELAX_CPU()
operations (such as the x86 PAUSE instruction) between successive
attempts of acquiring the spin lock. The use of a system call based
log_sys.lsn_lock (which is the default setting) can be enabled by
SET GLOBAL INNODB_LOG_SPIN_WAIT_DELAY=0;
This patch will also introduce #ifdef LOG_LATCH_DEBUG
(part of cmake -DWITH_INNODB_EXTRA_DEBUG=ON) for more accurate
tracking of log_sys.latch ownership and reorganize the fields of
log_sys to improve the locality of reference and to reduce the
chances of false sharing.
When a spin lock is being used, it will be maintained in the
most significant bit of log_sys.buf_free. This is useful, because that is
one of the fields that is covered by the lock. For IA-32 or AMD64, we
implement the spin lock specially via log_t::lsn_lock_bts(), employing the
i386 LOCK BTS instruction. A straightforward std::atomic::fetch_or() would
translate into an inefficient loop around LOCK CMPXCHG.
mtr_t::spin_wait_delay: The value of innodb_log_spin_wait_delay.
mtr_t::finisher: Pointer to the currently used mtr_t::finish_write()
implementation. This allows to avoid introducing conditional branches.
We no longer invoke log_sys.is_pmem() at the mini-transaction level,
but we would do that in log_write_up_to().
mtr_t::finisher_update(): Update finisher when spin_wait_delay is
changed from or to 0 (the spin lock is changed to log_sys.lsn_lock or
vice versa).
Let us skip the recently added test main.mysql-interactive if
an instrumented ncurses library is not available.
In InnoDB, let us work around an uninstrumented libnuma, by
declaring that the objects returned by numa_get_mems_allowed()
are initialized.
buf_flush_page_cleaner(): Remove a loop that had originally been added
in commit 9d1466522e (MDEV-32029) and made
redundant by commit 5b53342a6a (MDEV-32588).
Starting with commit d34479dc66 (MDEV-33053)
this loop would cause a significant performance regression in workloads
where buf_pool.need_LRU_eviction() constantly holds in
buf_flush_page_cleaner().
Thanks to Steve Shaw of Intel for noticing this.
Reviewed by: Debarun Banerjee
Tested by: Matthias Leich
Ever since commit 412ee0330c
or commit a440d6ed3a
InnoDB should generally not abort when failing to open or create files.
In Datafile::open_or_create() we had failed to set the flag
to avoid abort() on failure, but everywhere else we were setting it.
We may still call abort() via os_file_handle_error().
Reviewed by: Vladislav Vaintroub
buf_read_ahead_linear(): If buf_pool.watch_is_sentinel(*bpage),
do not attempt to read the page frame because the pointer would be null
for the elements of buf_pool.watch[].
Hitting this bug requires the use of a non-default value of
innodb_change_buffering.
A mix of path separators looks odd.
InnoDB: Loading buffer pool(s) from C:\xampp\mysql\data/ib_buffer_pool
This was changed in cf552f5886
Both forward slashes and backward slashes work on Windows. We do not
use \\?\ names.
So we improve the consistent look of it so it doesn't look like a bug.
Normalize, in this case, the path separator to \ for making the filename.
Reported thanks to Github user @celestinoxp.
Closes: https://github.com/ApacheFriends/xampp-build/issues/33
Reviewed by: Marko Mäkelä and Vladislav Vaintroub
In commit a55b951e60 (MDEV-26827)
an error was introduced in a rarely executed code path of the
buf_flush_page_cleaner() thread. As a result, the function
buf_flush_LRU() could be invoked while not holding buf_pool.mutex.
Reviewed by: Debarun Banerjee
buf_flush_LRU(): Display a warning if no pages could be evicted and
no writes initiated.
buf_pool_t::need_LRU_eviction(): Renamed from buf_pool_t::ran_out().
Check if the amount of free pages is smaller than innodb_lru_scan_depth
instead of checking if it is 0.
buf_flush_page_cleaner(): For the final LRU flush after a checkpoint
flush, use a "budget" of innodb_io_capacity_max, like we do in the
case when we are not in "furious" checkpoint flushing.
Co-developed by: Debarun Banerjee
Reviewed by: Debarun Banerjee
Tested by: Matthias Leich
- InnoDB fails to find the space id from the page0 of
the tablespace. In that case, InnoDB can use
doublewrite buffer to recover the page0 and write
into the file.
- buf_dblwr_t::init_or_load_pages(): Loads only the pages
which are valid.(page lsn >= checkpoint). To do that,
InnoDB has to open the redo log before system
tablespace, read the latest checkpoint information.
recv_dblwr_t::find_first_page():
1) Iterate the doublewrite buffer pages and find the 0th page
2) Read the tablespace flags, space id from the 0th page.
3) Read the 1st, 2nd and 3rd page from tablespace file and
compare the space id with the space id which is stored
in doublewrite buffer.
4) If it matches then we can write into the file.
5) Return space which matches the pages from the file.
SysTablespace::read_lsn_and_check_flags(): Remove the
retry logic for validating the first page. After
restoring the first page from doublewrite buffer,
assign tablespace flags by reading the first page.
recv_recovery_read_max_checkpoint(): Reads the maximum
checkpoint information from log file
recv_recovery_from_checkpoint_start(): Avoid reading
the checkpoint header information from log file
Datafile::validate_first_page(): Throw error in case
of first page validation fails.
When innodb_undo_log_truncate=ON causes an InnoDB undo tablespace
to be truncated, we must guarantee that the undo tablespace will
be rebuilt atomically: After mtr_t::commit_shrink() has durably
written the mini-transaction that rebuilds the undo tablespace,
we must not write any old pages to the tablespace.
To guarantee this, in trx_purge_truncate_history() we used to
traverse the entire buf_pool.flush_list in order to acquire
exclusive latches on all pages for the undo tablespace that
reside in the buffer pool, so that those pages cannot be written
and will be evicted during mtr_t::commit_shrink(). But, this
traversal may interfere with the page writing activity of
buf_flush_page_cleaner(). It would be better to lazily discard
the old pages of the truncated undo tablespace.
fil_space_t::is_being_truncated, fil_space_t::clear_stopping(): Remove.
fil_space_t::create_lsn: A new field, identifying the LSN of the
latest rebuild of a tablespace.
buf_page_t::flush(), buf_flush_try_neighbors(): Evict pages whose
FIL_PAGE_LSN is below fil_space_t::create_lsn.
mtr_t::commit_shrink(): Update fil_space_t::create_lsn and
fil_space_t::size right before the log is durably written and the
tablespace file is being truncated.
fsp_page_create(), trx_purge_truncate_history(): Simplify the logic.
Reviewed by: Thirunarayanan Balathandayuthapani, Vladislav Lesin
Performance tested by: Axel Schwenke
Correctness tested by: Matthias Leich
buf_flush_page_cleaner(): A continue or break inside DBUG_EXECUTE_IF
actually is a no-op. Use an explicit call to _db_keyword_() to
actually avoid advancing the checkpoint.
buf_flush_list_now_set(): Invoke os_aio_wait_until_no_pending_writes()
to ensure that the page write to the system tablespace is completed.