fil_space_t::flush_freed(): Renamed from buf_flush_freed_pages();
this is a backport of aa45850687 from 10.6.
Invoke log_write_up_to() on last_freed_lsn, instead of avoiding
the operation when the log has not yet been written.
A more costly alternative would be that log_checkpoint() would invoke
this function on every affected tablespace.
handle_slave_io(), handle_slave_sql(), os_thread_exit():
Remove a redundant pthread_exit(nullptr) call, because it
would cause SIGSEGV.
mysql_print_status(): Add MEM_MAKE_DEFINED() to work around
some missing instrumentation around mallinfo2().
que_graph_free_stat_list(): Invoke que_node_get_next(node) before
que_graph_free_recursive(node). That is the logical and
MSAN_OPTIONS=poison_in_dtor=1 compatible way of freeing memory.
ins_node_t::~ins_node_t(): Invoke mem_heap_free(entry_sys_heap).
que_graph_free_recursive(): Rely on ins_node_t::~ins_node_t().
fts_t::~fts_t(): Invoke mem_heap_free(fts_heap).
fts_free(): Replace with direct calls to fts_t::~fts_t().
The failures in free_root() due to MSAN_OPTIONS=poison_in_dtor=1
will be covered in MDEV-30942.
MySQL 5.7.41 includes one InnoDB change
mysql/mysql-server@d2d6b2dd00
that seems to be applicable to MariaDB Server 10.3 and 10.4.
Even though commit 5b9ee8d819
seems to have fixed sporadic failures on our CI systems, it is
theoretically possible that another race condition remained.
buf_flush_page_cleaner_coordinator(): In the final loop,
wait also for buf_get_n_pending_read_ios() to reach 0.
In this way, if a secondary index leaf page was read into the
buffer pool and ibuf_merge_or_delete_for_page() modified that
page or some change buffer pages, the flush loop would execute
until the buffer pool really is in a clean state.
This potential data corruption bug does not affect MariaDB Server 10.5
or later, thanks to commit b42294bc64
which removed change buffer merges that are not explicitly requested.
The InnoDB write-ahead log ib_logfile0 is of fixed size,
specified by innodb_log_file_size. If the tail of the log
manages to overwrite the head (latest checkpoint) of the log,
crash recovery will be broken.
Let us clarify the messages about this, including adding
a message on the completion of a log checkpoint that notes
that the dangerous situation is over.
To reproduce the dangerous scenario, we will introduce the
debug injection label ib_log_checkpoint_avoid_hard, which will
avoid log checkpoints even harder than the previous
ib_log_checkpoint_avoid.
log_t::overwrite_warned: The first known dangerous log sequence number.
Set in log_close() and cleared in log_write_checkpoint_info(),
which will output a "Crash recovery was broken" message.
Let us use the normal platform-specific preprocessor symbols
__linux__, __sun__, _AIX instead of some homebrew ones.
The preprocessor symbol UNIV_HPUX must have lost its meaning
by f6deb00a56 (note: the symbol
UNIV_HPUX10 is being checked for, but only UNIV_HPUX is defined).
In commit 73fee39ea6 (MDEV-27985)
a regression was introduced that would cause bpage=nullptr to
be referenced.
buf_flush_LRU_list_batch(): Always terminate the loop upon
encountering a null pointer.
buf_flush_page(): Never wait for a page latch, even in checkpoint
flushing (flush_type == BUF_FLUSH_LIST), to prevent a hang of the
page cleaner threads when a large number of pages is latched.
In mysql/mysql-server@9542f3015b
it was claimed that such a hang only affects CREATE FULLTEXT INDEX.
Their fix was to retain buffer-fix but release exclusive latch
on non-leaf pages, and subsequently write to those pages while
they are not associated with the mini-transaction, which would
trip a debug assertion in the MariaDB version of
mtr_t::memo_modify_page() and cause potential corruption
when using the default MariaDB setting innodb_log_optimize_ddl=OFF.
This change essentially backports a small part of
commit 7cffb5f6e8 (MDEV-23399)
from MariaDB Server 10.5.7.
This is a backport of commit 4489a89c71
in order to remove the test innodb.redo_log_during_checkpoint
that would cause trouble in the DBUG subsystem invoked by
safe_mutex_lock() via log_checkpoint(). Before
commit 7cffb5f6e8
these mutexes were of different type.
The following options were introduced in
commit 2e814d4702 (mariadb-10.2.2)
and have little use:
innodb_disable_resize_buffer_pool_debug had no effect even in
MariaDB 10.2.2 or MySQL 5.7.9. It was introduced in
mysql/mysql-server@5c4094cf49
to work around a problem that was fixed in
mysql/mysql-server@2957ae4f99
(but the parameter was not removed).
innodb_page_cleaner_disabled_debug and innodb_master_thread_disabled_debug
are only used by the test innodb.redo_log_during_checkpoint
that will be removed as part of this commit.
innodb_dict_stats_disabled_debug is only used by that test,
and it is redundant because one could simply use
innodb_stats_persistent=OFF or the STATS_PERSISTENT=0 attribute
of the table in the test to achieve the same effect.
The comparison on the checkpoint age (number of log bytes
written since the previous checkpoint) is inaccurate, because
the previous FILE_CHECKPOINT record could span two 512-byte
log blocks, which will cause the LSN to increase by the size of the
log block header and footer.
We will still generate a redudant checkpoint if the previous
checkpoint wrote some FILE_MODIFY records before the FILE_CHECKPOINT
record.
Only one checkpoint may be in progress at a time.
The counter log_sys.n_pending_checkpoint_writes
was being protected by log_sys.mutex.
Let us replace it with the Boolean log_sys.checkpoint_pending.
buf_flush_freed_pages(): Assert that neither buf_pool.mutex
nor buf_pool.flush_list_mutex are held. Simplify the loops.
Return the tablespace and the number of pages written or punched.
buf_flush_LRU_list_batch(), buf_do_flush_list_batch():
Release buf_pool.mutex before invoking buf_flush_space().
buf_flush_list_space(): Acquire the mutexes only after invoking
buf_flush_freed_pages().
Reviewed by: Thirunarayanan Balathandayuthapani
Problem:
=======
InnoDB ran out of memory during recovery and it fails to
flush the dirty LRU blocks. The reason is that buffer pool
can ran out before the LRU list length reaches
BUF_LRU_OLD_MIN_LEN(256) threshold.
Fix:
====
During recovery, InnoDB should write out and evict all
dirty blocks.
.. to be the same as startup.
In resolving MDEV-27461, BUF_LRU_MIN_LEN (256) is the minimum number of
pages for the innodb buffer pool size. Obviously we need more than just
flushing pages. Taking the 16k page size and its default minimum, an
extra 25% is needed on top of the flushing pages to make a workable buffer
pool.
The minimum innodb_buffer_pool_chunk_size (1M) restricts the minimum
otherwise we'd have a pool made up of different chunk sizes.
The resulting minimum innodb buffer pool sizes are:
Page Size, Previously minimum (startup), with change.
4k 5M 2M
8k 5M 3M
16k 5M 5M
32k 24M 10M
64k 24M 20M
With this patch, SET GLOBAL innodb_buffer_pool_size minimums are
enforced.
The evident minimum system variable size for innodb_buffer_pool_size
is 2M, however this is only setable if using 4k page size. As
the order of the page_size and buffer_pool_size aren't fixed, we can't
hide this change.
Subsequent changes:
* innodb_buffer_pool_resize_with_chunks.test - raised of pool resize due to new
minimums. Chunk size also needed increase as the test was for
pool_size < chunk_size to generate a warning.
* Removed srv_buf_pool_min_size and replaced use with MYSQL_SYSVAR_NAME(buffer_pool_size).min_val
* Removed srv_buf_pool_def_size and replaced constant defination in
MYSQL_SYSVAR_LONGLONG(buffer_pool_size)
* Reordered ha_innodb to allow for direct use of MYSQL_SYSVAR_NAME(buffer_pool_size).min_val
* Moved buf_pool_size_align into ha_innodb to access to MYSQL_SYSVAR_NAME(buffer_pool_size).min_val
* loose-innodb_disable_resize_buffer_pool_debug is needed in the
innodb.restart.opt test so that under debug mode, resizing of the
innodb buffer pool can occur.
In commit 4c3ad24413 (MDEV-27416)
an unnecessarily strict wait condition was introduced in the
function buf_flush_wait(). Most callers actually only care that
the pages have been flushed, not that a checkpoint has completed.
Only in the buf_flush_sync() call for log resizing, we might care
about the log checkpoint. But, in fact,
srv_prepare_to_delete_redo_log_file() is explicitly disabling
checkpoints. So, we can simply remove the unnecessary wait loop.
Thanks to Krunal Bauskar for reporting this performance regression
that we failed to repeat in our testing.
InnoDB could sometimes hang when triggering a log checkpoint. This is
due to commit 7b1252c03d (MDEV-24278),
which introduced an untimed wait to buf_flush_page_cleaner().
The hang was noticed by occasional failures of IMPORT TABLESPACE tests,
such as innodb.innodb-wl5522, which would (unnecessarily) invoke
log_make_checkpoint() from row_import_cleanup().
The reason of the hang was that buf_flush_page_cleaner() would enter
untimed sleep despite buf_flush_sync_lsn being set. The exact failure
scenario is unclear, because buf_flush_sync_lsn should actually be
protected by buf_pool.flush_list_mutex. We prevent the hang by
invoking buf_pool.page_cleaner_set_idle(false) whenever we are
setting buf_flush_sync_lsn and signaling buf_pool.do_flush_list.
The bulk of these changes was originally developed as a preparation
for MDEV-26827, to invoke buf_flush_list() from fewer threads,
and tested on 10.6 by Matthias Leich.
This fix was tested by running 100 repetitions of 100 concurrent instances
of the test innodb.innodb-wl5522 on a RelWithDebInfo build, using ext4fs
and innodb_flush_method=O_DIRECT on a SATA SSD with 4096-byte block size.
During the test, the call to log_make_checkpoint() in row_import_cleanup()
was present.
buf_flush_list(): Make static.
buf_flush_wait(): Wait for buf_pool.get_oldest_modification()
to reach a target, by work done in the buf_flush_page_cleaner.
If buf_flush_sync_lsn is going to be set, we will invoke
buf_pool.page_cleaner_set_idle(false).
buf_flush_ahead(): If buf_flush_sync_lsn or buf_flush_async_lsn
is going to be set and the page cleaner woken up, we will invoke
buf_pool.page_cleaner_set_idle(false).
buf_flush_wait_flushed(): Invoke buf_flush_wait().
buf_flush_sync(): Invoke recv_sys.apply() at the start in case
crash recovery is active. Invoke buf_flush_wait().
buf_flush_sync_batch(): A lower-level variant of buf_flush_sync()
that is only called by recv_sys_t::apply().
buf_flush_sync_for_checkpoint(): Do not trigger log apply
or checkpoint during recovery.
buf_dblwr_t::create(): Only initiate a buffer pool flush, not
a checkpoint.
row_import_cleanup(): Do not unnecessarily invoke log_make_checkpoint().
Invoking buf_flush_list_space() before starting to generate redo log
for the imported tablespace should suffice.
srv_prepare_to_delete_redo_log_file():
Set recv_sys.recovery_on in order to prevent
buf_flush_sync_for_checkpoint() from initiating a checkpoint
while the log is inaccessible. Remove a wait loop that is already
part of buf_flush_sync().
Do not invoke fil_names_clear() if the log is being upgraded,
because the FILE_MODIFY record is specific to the latest format.
create_log_file(): Clear recv_sys.recovery_on only after calling
log_make_checkpoint(), to prevent buf_flush_page_cleaner from
invoking a checkpoint.
innodb_shutdown(): Simplify the logic in mariadb-backup --prepare.
os_aio_wait_until_no_pending_writes(): Update the function comment.
Apart from row_quiesce_table_start() during FLUSH TABLES...FOR EXPORT,
this is being called by buf_flush_list_space(), which is invoked
by ALTER TABLE...IMPORT TABLESPACE as well as some encryption operations.
buf_flush_check_neighbors(): Relax a debug assertion that
could fail for the very last page(s) of a ROW_FORMAT=COMPRESSED tables using a 1024-byte or 2048-byte page size.
This assertion started to fail after
commit d09426f9e6 (MDEV-26537)
modified the .ibd file extension to occur in steps of 4096 bytes.
innodb_max_dirty_pages_pct_update(),
innodb_max_dirty_pages_pct_lwm_update():
Invoke buf_pool.page_cleaner_wakeup() in order to wake up
buf_flush_page_cleaner. This allows the test innodb.page_cleaner
to run without any occasional timeouts.
The occasional hangs were introduced by
commit 7b1252c03d (MDEV-24278).
trx_purge_truncate_history(): Do not force a write of the undo tablespace
that is being truncated. Instead, prevent page writes by acquiring
an exclusive latch on all dirty pages of the tablespace.
fseg_create(): Relax an assertion that could fail if a dirty undo page
is being initialized during undo tablespace truncation (and
trx_purge_truncate_history() already acquired an exclusive latch on it).
fsp_page_create(): If we are truncating a tablespace, try to reuse
a page that we may have already latched exclusively (because it was
in buf_pool.flush_list). To some extent, this helps the test
innodb.undo_truncate,16k to avoid running out of buffer pool.
mtr_t::commit_shrink(): Mark as clean all pages that are outside the
new bounds of the tablespace, and only add the newly reinitialized pages
to the buf_pool.flush_list.
buf_page_create(): Do not unnecessarily invoke change buffer merge on
undo tablespaces.
buf_page_t::clear_oldest_modification(bool temporary): Move some
assertions to the caller buf_page_write_complete().
innodb.undo_truncate: Use a bigger innodb_buffer_pool_size=24M.
On my system, it would otherwise hang 1 out of 1547 attempts
(on the 40th repeat of innodb.undo_truncate,16k).
Other page sizes were not affected.
While the redo log is being resized in srv_start(),
we must not write checkpoint information to the old log.
Thanks to Matthias Leich for noticing this.
buf_flush_page_cleaner(): Always try to advance the log checkpoint,
even when no pages were flushed during the latest batch.
Maybe, since the previous batch, there was an LRU flush that
removed the last dirty pages.
Failure to advance the log checkpoint will cause unnecessary work
in Mariabackup and on crash recovery.
The lazy deletion of clean blocks from buf_pool.flush_list that was
introduced in commit 6441bc614a (MDEV-25113)
introduced a race condition around the function
buf_flush_relocate_on_flush_list().
The test innodb_zip.wl5522_debug_zip as well as the buffer pool
resizing tests would occasionally fail in debug builds due to
buf_pool.flush_list.count disagreeing with the actual length of the
doubly-linked list.
The safe procedure for relocating a block in buf_pool.flush_list should be
as follows, now that we implement lazy deletion from buf_pool.flush_list:
1. Acquire buf_pool.mutex.
2. Acquire the exclusive buf_pool.page_hash.latch.
3. Acquire buf_pool.flush_list_mutex.
4. Copy the block descriptor.
5. Invoke buf_flush_relocate_on_flush_list().
6. Release buf_pool.flush_list_mutex.
buf_flush_relocate_on_flush_list(): Assert that
buf_pool.flush_list_mutex is being held. Invoke
buf_page_t::oldest_modification() only once, using
std::memory_order_relaxed, now that the mutex protects us.
buf_LRU_free_page(), buf_LRU_block_remove_hashed(): Avoid
an unlock-lock cycle on hash_lock. (We must not acquire hash_lock
while already holding buf_pool.flush_list_mutex, because that
could lead to a deadlock due to latching order violation.)
buf_flush_relocate_on_flush_list(): Use dpage->physical_size()
because bpage->zip.ssize may already have been zeroed in
page_zip_set_size() invoked by buf_pool_t::realloc().
This would cause occasional failures of the test
innodb.innodb_buffer_pool_resize, which creates a
ROW_FORMAT=COMPRESSED table.
buf_flush_relocate_on_flush_list(): If we are removing the block from
buf_pool.flush_list, subtract its size from buf_pool.stat.flush_list_bytes.
This fixes a regression that was introduced in
commit 22b62edaed (MDEV-25113).
In commit 5f22511e35 we depend on
Total Store Ordering. For correct operation on ISAs that implement
weaker memory ordering, we must explicitly use release/acquire stores
and loads on buf_page_t::oldest_modification_ to prevent a race condition
when buf_page_t::list does not happen to be on the same cache line.
buf_page_t::clear_oldest_modification(): Assert that the block is
not in buf_pool.flush_list, and use std::memory_order_release.
buf_page_t::oldest_modification_acquire(): Read oldest_modification_
with std::memory_order_acquire. In this way, if the return value is 0,
the caller may safely assume that it will not observe the buf_page_t
as being in buf_pool.flush_list, even if it is not holding
buf_pool.flush_list_mutex.
buf_flush_relocate_on_flush_list(), buf_LRU_free_page():
Invoke buf_page_t::oldest_modification_acquire().
In commit 22b62edaed (MDEV-25113)
we introduced a race condition. buf_LRU_free_page() would read
buf_page_t::oldest_modification() as 0 and assume that
buf_page_t::list can be used (for attaching the block to the
buf_pool.free list). In the observed race condition,
buf_pool_t::delete_from_flush_list() had cleared the field,
and buf_pool_t::delete_from_flush_list_low() was executing
concurrently with buf_LRU_block_free_non_file_page(),
which resulted in buf_pool.flush_list.end becoming corrupted.
buf_pool_t::delete_from_flush_list(), buf_flush_relocate_on_flush_list():
First remove the block from buf_pool.flush_list, and only then
invoke buf_page_t::clear_oldest_modification(), to ensure that
reading oldest_modification()==0 really implies that the block
no longer is in buf_pool.flush_list.
buf_LRU_get_free_block(): Initially wait for a single block to be
freed, signaled by buf_pool.done_free. Only if that fails and no
LRU eviction flushing batch is already running, we initiate a
flushing batch that should serve all threads that are currently
waiting in buf_LRU_get_free_block().
Note: In an extreme case, this may introduce a performance regression
at larger numbers of connections. We observed this in sysbench
oltp_update_index with 512MiB buffer pool, 4GiB of data on fast NVMe,
and 1000 concurrent connections, on a 20-thread CPU. The contention point
appears to be buf_pool.mutex, and the improvement would turn into a
regression somewhere beyond 32 concurrent connections.
On slower storage, such regression was not observed; instead, the
throughput was improving and maximum latency was reduced.
The excessive waits were pointed out by Vladislav Vaintroub.
MDEV-23855 changed the way how the page cleaner is signaled by
user threads. If a threshold is exceeded, a mini-transaction commit
would invoke buf_flush_ahead() in order to initiate page flushing
before all writers would eventually grind to halt in
log_free_check(), waiting for the checkpoint age to reduce.
However, buf_flush_ahead() would always initiate 'furious flushing',
making the buf_flush_page_cleaner thread write innodb_io_capacity_max
pages per batch, and sleeping no time between batches, until the
limit LSN is reached. Because this could saturate the I/O subsystem,
system throughput could significantly reduce during these
'furious flushing' spikes.
With this change, we introduce a gentler version of flush-ahead,
which would write innodb_io_capacity_max pages per second until
the 'soft limit' is reached.
buf_flush_ahead(): Add a parameter to specify whether furious flushing
is requested.
buf_flush_async_lsn: Similar to buf_flush_sync_lsn, a limit for
the less intrusive flushing.
buf_flush_page_cleaner(): Keep working until buf_flush_async_lsn
has been reached.
log_close(): Suppress a warning message in the event that a new log
is being created during startup, when old logs did not exist.
Return what type of page cleaning will be needed.
mtr_t::finish_write(): Also when m_log.is_small(), invoke log_close().
Return what type of page cleaning will be needed.
mtr_t::commit(): Invoke buf_flush_ahead() based on the return value of
mtr_t::finish_write().
buf_page_write_complete(): Reduce the buf_pool.mutex hold time,
and do not acquire buf_pool.flush_list_mutex at all.
Instead, mark blocks clean by setting oldest_modification to 1.
Dirty pages of temporary tables will be identified by the special
value 2 instead of the previous special value 1.
(By design of the ib_logfile0 format, actual LSN values smaller
than 2048 are not possible.)
buf_LRU_free_page(), buf_pool_t::get_oldest_modification()
and many other functions will remove the garbage (clean blocks)
from buf_pool.flush_list while holding buf_pool.flush_list_mutex.
buf_pool_t::n_flush_LRU, buf_pool_t::n_flush_list:
Replaced with non-atomic variables, protected by buf_pool.mutex,
to avoid unnecessary synchronization when modifying the counts.
export_vars: Remove unnecessary indirection for
innodb_pages_created, innodb_pages_read, innodb_pages_written.
In commit 7cffb5f6e8 (MDEV-23399)
the implementation of buf_flush_dirty_pages() was replaced with
a slow one, which would perform excessive scans of the
buf_pool.flush_list and make little progress.
buf_flush_list(), buf_flush_LRU(): Split from buf_flush_lists().
Vladislav Vaintroub noticed that we will not need to invoke
log_flush_task.wait() for the LRU eviction flushing.
buf_flush_list_space(): Replaces buf_flush_dirty_pages().
This is like buf_flush_list(), but operating on a single
tablespace at a time. Writes at most innodb_io_capacity
pages. Returns whether some of the tablespace might remain
in the buffer pool.
Vladislav Vaintroub suggested that invoking log_flush_up_to()
for every page could perform better than invoking a log write
between buf_pool.flush_list batches, like we started doing in
commit 3a9a3be1c6 (MDEV-23855).
This could depend on the sequence in which pages are being
modified. The buf_pool.flush_list is ordered by
oldest_modification, while the FIL_PAGE_LSN of the pages is
theoretically independent of that. In the pathological case,
we will wait for a log write before writing each individual page.
It turns out that we can defer the call to log_flush_up_to()
until just before submitting the page write. If the doublewrite
buffer is being used, we can submit a write batch of "future" pages
to the doublewrite buffer, and only wait for the log write right
before we are writing an already doublewritten page.
The next doublewrite batch will not be initiated before the last
page write from the current batch has completed.
When a future version introduces asynchronous writes if the log,
we could initiate a write at the start of a flushing batch, to
reduce waiting further.
It turns out that we had some unnecessary waits for no outstanding
write requests to exist. They were basically working around a
bug that was fixed in MDEV-25953.
On write completion callback, blocks will be marked clean.
So, it is sufficient to consult buf_pool.flush_list to determine
which writes have not been completed yet.
On FLUSH TABLES...FOR EXPORT we must still wait for all pending
asynchronous writes to complete, because buf_flush_file_space()
would merely guarantee that writes will have been initiated.
innodb_adaptive_flushing_lwm is hit. (possible regression)
adaptive flushing should kick in if
a. dirty_pct (dirty pages in buffer pool) > innodb_max_dirty_pages_pct_lwm
OR
b. innodb_adaptive_flushing_lwm limit is reached (default to 10%)
both conditions are mutually exclusive and whichever is first to evaluate
true should kick-start the adaptive flushing.
After recent changes to simplify the flushing algorithm logic, (b) got ignored
that introduced the said regression.
On startup, InnoDB tried to increase the priority of the page cleaner
thread. This would usually fail, resulting in a server log message.
When the page cleaner code was refactored in MDEV-23855, the throughput
and latency were greatly improved even when the
buf_flush_page_cleaner thread always ran the same priority with other
threads in our benchmarks.
So, this code should have no benefit in MariaDB Server 10.5.7 or later.
Besides, starting with MariaDB Server 10.5, the executable is called
mariadbd, not mysqld.
As pointed out by Andrei Elkin, the previous fix did not fix one
race condition that may have caused the observed hang.
innodb_log_flush_request(): If we are enqueueing the very first
request at the same time the log write is being completed,
we must ensure that a near-concurrent call to log_flush_notify()
will not result in a missed notification. We guarantee this by
release-acquire operations on log_requests.start and
log_sys.flushed_to_disk_lsn.
log_flush_notify_and_unlock(): Cleanup: Always release the mutex.
log_sys_t::get_flushed_lsn(): Use acquire memory order.
log_sys_t::set_flushed_lsn(): Use release memory order.
log_sys_t::set_lsn(): Use release memory order.
log_sys_t::get_lsn(): Use relaxed memory order by default, and
allow the caller to specify acquire memory order explicitly.
Whenever the log_sys.mutex is being held or when log writes are
prohibited during startup, we can use a relaxed load. Likewise,
in some assertions where reading a stale value of log_sys.lsn
should not matter, we can use a relaxed load.
This will cause some additional instructions to be emitted on
architectures that do not implement Total Store Ordering (TSO),
such as POWER, ARM, and RISC-V Weak Memory Ordering (RVWMO).
- Currently page cleaner thread will stop flushing if
dirty_pct < innodb_max_dirty_pages_pct_lwm.
- If the server is not performing any activity then said resources/time
could be used to flush the pending dirty pages and keep buffer pool
clean for the next burst of the cycle. This flushing is called idle flushing.
- flushing logic underwent a complete revamp in 10.5.7/8
and as part of the revamp idle flushing logic got removed.
- New proposed logic of idle flushing is based on updated logic of the
page cleaner that will enable idle flushing if
- buf page cleaner is idle
- there are dirty pages (< innodb_max_dirty_pages_pct_lwm)
- server is not performing any activity
Logic will kickstart the idle flushing bounded by innodb_io_capacity.
(Thanks to Marko Makela for reviewing the patch and idea
right from the its inception).
commit a993310593 (MDEV-24537)
introduced the regression that the page cleaner will keep sleeping
even if there is work to do.
innodb_max_dirty_pages_pct_update(): Always wake up the page cleaner
on any SET GLOBAL innodb_max_dirty_pages_pct= assignment.
buf_flush_page_cleaner(): If innodb_max_dirty_pages_pct is nonzero,
consult only that parameter when determining whether there is work
to do. Else, consult innodb_max_dirty_pages.
fsp_free_page() writes MLOG_INIT_FREE_PAGE, but does not update page
type. But fil_crypt_rotate_page() checks the type to understand if the
page is freshly initialized, and writes dummy record(updates space id)
to force rotation during recovery. This dummy record causes assertion
crash when the page is flushed after recovery, as it's supposed
that pages LSN is 0 for freshly initialized pages.
The bug is similiar to MDEV-24695, the difference is that in 10.5 the
assertion crashes during log record applying, but in 10.4 it crashes
during page flushing.
The fix could be in marking page as freed and not writing dummy record
during keys rotation procedure for such marked pages. But
bpage->file_page_was_freed is not consistent enough for release builds in
10.4, and the issue is fixed in 10.5 and does not exist in 10.[23] as
MLOG_INIT_FREE_PAGE was introduced since 10.4.
So the better solution is just to relax the assertion and implement some
additional property for freshly allocated pages, and check this property
during pages flushing.
The test is copied from MDEV-24695, the only change is in forcing pages
flushing after each server start to cause crash in non-fixed code.
There is no need to merge it to 10.5+, as the bug is already fixed by
MDEV-24695.
The condition variables that were introduced in
commit 7cffb5f6e8 (MDEV-23399)
are never instrumented with PERFORMANCE_SCHEMA.
Let us avoid the storage overhead and dead code.
In commit 3a9a3be1c6 (MDEV-23855)
some previous logic was replaced with the condition
dirty_pct < srv_max_dirty_pages_pct_lwm, which caused
the default value of the parameter innodb_max_dirty_pages_pct_lwm=0
to lose its special meaning: 'refer to innodb_max_dirty_pages_pct instead'.
This implicit special meaning was visible in the function
af_get_pct_for_dirty(), which was removed in
commit f0c295e2de (MDEV-24369).
page_cleaner_flush_pages_recommendation(): Restore the special
meaning that was removed in MDEV-24369.
buf_flush_page_cleaner(): If srv_max_dirty_pages_pct_lwm==0.0,
refer to srv_max_buf_pool_modified_pct. This fixes the observed
performance regression due to excessive page flushing.
buf_pool_t::page_cleaner_wakeup(): Revise the wakeup condition.
innodb_init(): Do initialize srv_max_io_capacity in Mariabackup.
It was previously constantly 0, which caused mariadb-backup --prepare
to hang in buf_flush_sync(), making no progress.
Since commit ea21d630be we
conditionally define a variable that only plays a role on
systems that support hole-punching (explicit creation of sparse files).
However, that broke debug builds on such systems.
It turns out that the debug_dbug label "ignore_punch_hole" is
not at all used in MariaDB server. It would be covered by
the MySQL 5.7 test innodb.table_compress. (Note: MariaDB 10.1
implemented page_compressed tables before something comparable
appeared in MySQL 5.7.)