When srv_page_size and innodb_page_size were introduced,
the functions page_align() and page_offset() got more expensive.
Let us try to replace such calls with simpler pointer arithmetics
with respect to the buffer page frame.
page_rec_get_next_non_del_marked(): Add a page frame as a parameter,
and template<bool comp>.
page_rec_next_get(): A more efficient variant of page_rec_get_next(),
with template<bool comp> and const page_t* parameters.
lock_get_heap_no(): Replaces page_rec_get_heap_no() outside debug checks.
fseg_free_step(), fseg_free_step_not_header(): Take the header block
as a parameter.
Reviewed by: Vladislav Lesin
Problem:
========
From 10.6.13(86767bcc0f) version,
InnoDB purge thread does free the TRX_UNDO_CACHED undo segment.
Pre-10.6.13 version data directory can contain TRX_UNDO_CACHED
undo segment in system tablespace even though it
has external undo tablespace.
During slow shutdown, InnoDB collects the segment from tables
that exist in system tablespace and cached undo segment in
the system tablespace as used segment exist in system tablespace.
While shrinking the system tablespace, last used extent can be
used by undo cached segment. This extent blocks the shrinking of
system tablespace in a effective way.
Solution:
========
While freeing the unused segment, InnoDB should free the cached
undo segment header page exists in system tablespace and
reset the TRX_RSEG_UNDO_SLOTS to 0xff for the rollback segment
header page exists in system tablespace. This could improve
the shrinking of system tablespace further.
- InnoDB fails to shrink the system tablespace when it contains
the leaked undo log pages caused by MDEV-30671.
- InnoDB does free the unused segment in system tablespace
before shrinking the tablespace. InnoDB fails to free
the unused segment if XA PREPARE transaction exist or
if the previous shutdown was not with innodb_fast_shutdown=0
inode_info: Structure to store the inode page and offsets.
fil_space_t::garbage_collect(): Frees the system tablespace
unused segment
fsp_get_sys_used_segment(): Iterates through all default
file segment and index segment present in system tablespace.
trx_sys_t::is_xa_exist(): Returns true if the XA transaction
exist in the undo logs
fseg_inode_free(): Frees the extents, fragment pages for the
given index node and ignores any error similar to
trx_purge_free_segment()
trx_sys_t::reset_page(): Retain the TRX_SYS_FSEG_HEADER value
in trx_sys page while resetting the page.
purge_sys_t::get_page(): Avoid accessing a freed reference to pages[id]
after pages.erase(id). This heap-use-after-free would sometimes be
caught by AddressSanitizer.
purge_sys_t::iterator::free_history_rseg(): Do not crash if undo=nullptr
(the database is corrupted).
Reviewed by: Debarun Banerjee
Before this patch, the InnoDB purge coordinator task submitted
innodb_purge_threads-1 tasks even if there was not sufficient amount
of work for all of them. For example, if there are undo log records
only for 1 table, only 1 task can be employed, and that task had better
be the purge coordinator.
srv_purge_worker_task_low(): Split from purge_worker_callback().
trx_purge_attach_undo_recs(): Remove the parameter n_purge_threads,
and add the parameter n_work_items, to keep track of the amount of
work.
trx_purge(): Launch purge worker tasks only if necessary. The work of
one thread will be executed by this purge coordinator thread.
que_fork_scheduler_round_robin(): Merged to trx_purge().
Thanks to Vladislav Vaintroub for supplying a prototype of this.
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
There were two separate Atomic_counter<uint32_t>, purge_sys.m_SYS_paused
and purge_sys.m_FTS_paused. In purge_sys.wait_FTS() we have to read both
atomically. We used to use an overkill solution for this, acquiring
purge_sys.latch and waiting 10 milliseconds between samples. To make
matters worse, the 10-millisecond wait was unconditional, which would
unnecessarily suspend the purge_coordinator_task every now and then.
It turns out that we can fold both "reference counts" into a single
Atomic_relaxed<uint32_t> and avoid the purge_sys.latch.
To assess whether std::memory_order_relaxed is acceptable, we should
consider the operations that read these "reference counts", that is,
purge_sys_t::wait_FTS(bool) and purge_sys_t::must_wait_FTS().
Outside debug assertions, purge_sys.must_wait_FTS() is only invoked in
trx_purge_table_acquire(), which is covered by a shared dict_sys.latch.
We would increment the counter as part of a DDL operation, but before
acquiring an exclusive dict_sys.latch. So, a
purge_sys_t::close_and_reopen() loop could be triggered slightly
prematurely, before a problematic DDL operation is actually executed.
Decrementing the counter is less of an issue; purge_sys.resume_FTS()
or purge_sys.resume_SYS() would mostly be invoked while holding an
exclusive dict_sys.latch; ha_innobase::delete_table() does it outside
that critical section. Still, this would only cause some extra wait in
the purge_coordinator_task, just like at the start of a DDL operation.
There are two calls to purge_sys_t::wait_FTS(bool): in the above mentioned
purge_sys_t::close_and_reopen() and in purge_sys_t::clone_oldest_view(),
both invoked by the purge_coordinator_task. There is also a
purge_sys.clone_oldest_view<true>() call at startup when no DDL operation
can be in progress.
purge_sys_t::m_SYS_paused: Merged into m_FTS_paused, using a new
multiplier PAUSED_SYS = 65536.
purge_sys_t::wait_FTS(): Remove an unnecessary sleep as well as the
access to purge_sys.latch. It suffices to poll purge_sys.m_FTS_paused.
purge_sys_t::stop_FTS(): Do not acquire purge_sys.latch.
Reviewed by: Debarun Banerjee
flst_read_addr(): Remove assertions. Instead, we will check these
conditions in the callers and avoid a crash in case of corruption.
We will check the conditions more carefully, because the callers
know more exact bounds for the page numbers and the byte offsets
withing pages.
flst_remove(), flst_add_first(), flst_add_last(): Add a parameter
for passing fil_space_t::free_limit. None of the lists may point to
pages that are beyond the current initialized length of the
tablespace.
trx_rseg_mem_restore(): Access the first page of the tablespace,
so that we will correctly recover rseg->space->free_limit
in case some log based recovery is pending.
ibuf_remove_free_page(): Only look up the root page once, and
validate the last page number.
Reviewed by: Debarun Banerjee
In commit aa719b5010 (part of MDEV-32050)
a bug was introduced in the function purge_sys_t::choose_next_log(),
which reimplements some logic that previously was part of
trx_purge_read_undo_rec(). We must invoke trx_undo_get_first_rec()
with the page number and offset of the undo log header, but we were
incorrectly invoking it on the current undo page number, which caused
us to parse undo records starting at an incorrect offset.
purge_sys_t::choose_next_log(): Pass the correct parameter to
trx_undo_page_get_first_rec().
trx_undo_page_get_next_rec(), trx_undo_page_get_first_rec(),
trx_undo_page_get_last_rec(): Add debug assertions and make the
code more robust by returning nullptr on corruption. Should we
detect any corrupted undo logs during the purge of committed
transaction history, the sanest thing to do is to pretend that
the end of an undo log was reached. If any garbage is left in
the tables, it will be ignored by anything else than
CHECK TABLE ... EXTENDED, and it can be removed by OPTIMIZE TABLE.
Thanks to Matthias Leich for providing an "rr replay" trace where
this bug could be found.
Reviewed by: Vladislav Lesin
Post-push fix: purge queue array can't be fixed size, because the elements
of the array is the analogue of undo logs, which must be processed in
the order of transaction commits, and the array can contain more
elements, than trx_sys.rseg_array. Also it's necessary to maintain
min-heap property by the trx_no of transaction, which produced the first
non-purged undo log in all rsegs. That's why the element of purge queue
aray must contain not only trx_sys.rseg_array index, but also trx_no of
committed transacion, i.e. the pair (trx_no, trx_sys.rseg_array index),
which is encoded as uint64_t((trx_no << 8) | (trx_sys.rseg_array index)).
Reviewed by: Marko Mäkelä
TrxUndoRsegs is wrapper for vector of trx_rseg_t*. It has two
constructors, both initialize the vector with only one element. And they
are used to push transactions rseg(the singular) to purge queue. There is
no function to add elements to the vector. The default constructor is used
only for declaration of NullElement.
The TrxUndoRsegs was introduced in WL#6915 in MySQL 5.7 and. MySQL 5.7
would unnecessarily let the purge of history parse the
temporary undo records, and then look up the table (via a global hash
table), and only at the point of processing the parsed undo log record
determine that the table is a temporary table and the undo record must be
thrown away.
In MariaDB 10.2 we have two disjoint sets of rollback segments (128 for
persistent, 128 for temporary), and purge does not even see the temporary
tables. The only reason why temporary tables are visible to other threads
is a SQL layer bug (MDEV-17805).
purge_sys_t::choose_next_log(): merge the relevant part
of TrxUndoRsegsIterator::set_next() to the start of
purge_sys_t::choose_next_log().
purge_sys_t::rseg_get_next_history_log(): add a tail call of
purge_sys_t::choose_next_log() and adjust the callers, to simplify the
control flow further.
purge_sys.pq_mutex and purge_sys.purge_queue: make it private by adding
some simple accessor function.
trx_purge_cleanse_purge_queue(): make it a member of purge_sys_t to have
have access to private purge_sys.pq_mutex and purge_sys.purge_queue,
simplify the code with using simple array copy and clearing purge queue
instead of poping each purge queue element.
rseg_t::last_commit_and_offset: exchange trx_no and offset bits to avoid
bitwise operations during pushing to/popping from purge queue.
Thanks Marko Mäkelä for historical overview of TrxUndoRsegs development.
Reviewed by: Marko Mäkelä
purge_sys_t::truncating_tablespace(): Clamp the
innodb_max_undo_log_size to the maximum number of pages
before converting the result into a 32-bit unsigned integer.
This fixes up commit f8c88d905b (MDEV-33213).
In later major versions, we would use 32-bit unsigned integer here
due to commit ca501ffb04
and the code would crash also on 64-bit processors.
Reviewed by: Debarun Banerjee
While the index_lock and block_lock include debug instrumentation
to keep track of shared lock holders, such instrumentation was
never part of the simpler srw_lock, and therefore some users of the
class implemented a limited form of bookkeeping.
srw_lock_debug encapsulates srw_lock and adds the data members
writer, readers_lock, and readers to keep track of the threads that
hold the exclusive latch or any shared latches. The debug checks
are available also with SUX_LOCK_GENERIC (in environments that do not
implement a futex-like system call).
dict_sys_t::latch: Use srw_lock_debug in debug builds.
This makes the debug fields latch_ex, latch_readers redundant.
fil_space_t::latch: Use srw_lock_debug in debug builds.
This makes the debug field latch_count redundant.
The field latch_owner must be preserved, because
fil_space_t::is_owner() is being used in all builds.
lock_sys_t::latch: Use srw_lock_debug in debug builds.
This makes the debug fields writer, readers redundant.
lock_sys_t::is_holder(): A new debug predicate to check if
the current thread is holding lock_sys.latch in any mode.
trx_rseg_t::latch: Use srw_lock_debug in debug builds.
The parameter innodb_undo_log_truncate=ON enables a multi-phased logic:
1. Any "producers" (new starting transactions) are prohibited
from using the rollback segments that reside in the undo tablespace.
2. Any transactions that use any of the rollback segments must be
committed or aborted.
3. The purge of committed transaction history must process all the
rollback segments.
4. The undo tablespace is truncated and rebuilt.
5. The rollback segments are re-enabled for new transactions.
There was one flaw in this logic: The first step was not being invoked
as often as it could be, and therefore innodb_undo_log_truncate=ON
would have no chance to work during a heavy write workload.
Independent of innodb_undo_log_truncate, even after
commit 86767bcc0f
we are missing some chances to free processed undo log pages.
If we prohibited the creation of new transactions in one busy
rollback segment at a time, we would be eventually guaranteed
to be able to free such pages.
purge_sys_t::skipped_rseg: The current candidate rollback segment
for shrinking the history independent of innodb_undo_log_truncate.
purge_sys_t::iterator::free_history_rseg(): Renamed from
trx_purge_truncate_rseg_history(). Implement the logic
around purge_sys.m_skipped_rseg.
purge_sys_t::truncate_undo_space: Renamed from truncate.
purge_sys.truncate_undo_space.last: Changed the type to integer
to get rid of some pointer dereferencing and conditional branches.
purge_sys_t::truncating_tablespace(), purge_sys_t::undo_truncate_try():
Refactored from trx_purge_truncate_history().
Set purge_sys.truncate_undo_space.current if applicable,
or return an already set purge_sys.truncate_undo_space.current.
purge_coordinator_state::do_purge(): Invoke
purge_sys_t::truncating_tablespace() as part of the normal work loop,
to implement innodb_undo_log_truncate=ON as often as possible.
trx_purge_truncate_rseg_history(): Remove a redundant parameter.
trx_undo_truncate_start(): Replace dead code with a debug assertion.
Correctness tested by: Matthias Leich
Performance tested by: Axel Schwenke
Reviewed by: Debarun Banerjee
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
trx_purge_free_segment(), trx_purge_truncate_rseg_history():
Do not claim that the blocks will be modified in the mini-transaction,
because that will not always be the case. Whenever there is a
modification, mtr_t::set_modified() will flag it.
The debug assertion that failed in recovery is checking that all
changes to data pages are covered by log records. Due to these
incorrect calls, we would unnecessarily write unmodified data pages,
which is something that commit 05fa4558e0
aims to avoid.
The incorrect calls had originally been added in
commit de31ca6a21 (MDEV-32820) and
commit 86767bcc0f (MDEV-29593).
Reviewed by: Vladislav Lesin
Tested by: Elena Stepanova
trx_purge_truncate_history(): Release buf_pool.flush_list_mutex
and 'dummily' acquire and release buf_pool.mutex before starting
a rescan of buf_pool.flush_list, to ensure that
the buf_flush_page_cleaner thread (which may be holding buf_pool.mutex)
will be able to proceed.
This fixes up commit 5dbe7a8c9a (MDEV-32757).
Tested by: Axel Schwenke (on Ubuntu 18.04 and Ubuntu 20.04)
Reviewed by: Vladislav Lesin
trx_purge_truncate_history(): Release buf_pool.flush_list_mutex
before starting a rescan of buf_pool.flush_list, to ensure that
the buf_flush_page_cleaner thread (which may be holding buf_pool.mutex)
will be able to proceed.
This fixes up commit a0f02f7438 (MDEV-32757).
Tested by: Axel Schwenke
Reviewed by: Vladislav Lesin
trx_purge_free_segment(): If fseg_free_step_not_header() needs to be
called multiple times, acquire an exclusive latch on the
rollback segment header page after restarting the mini-transaction
so that the rest of this function cannot execute concurrently
with trx_undo_create() on the same rollback segment.
This fixes a regression that was introduced in
commit c14a39431b (MDEV-30753).
Note: The buffer-fixes that we are holding across the mini-transaction
restart will prevent the pages from being evicted from the buffer pool.
They may be accessed by other threads or written back to data files
while we are not holding exclusive latches.
Reviewed by: Vladislav Lesin
trx_purge_truncate_history(): Do not prematurely mark dirty pages
as clean. This will be done in mtr_t::commit_shrink() as part of
Shrink::operator()(mtr_memo_slot_t*). Also, register each dirty page
only once in the mini-transaction.
fsp_page_create(): Adjust and simplify the page creation during
undo tablespace truncation. We can directly reuse pages that are
already in buf_pool.page_hash.
This fixes a regression that was caused by
commit f5794e1dc6 (MDEV-26445).
Tested by: Matthias Leich
Reviewed by: Thirunarayanan Balathandayuthapani