lock_sec_rec_some_has_impl(): Remove a harmful condition that caused the
performance regression and should not have been added in
commit b6e41e3872 in the first place.
Locking transactions that have not modified any persistent tables
can carry the transaction identifier 0.
trx_t::max_inactive_id: A cache for trx_sys_t::find_same_or_older().
The value is not reset on transaction commit so that previous results
can be reused for subsequent transactions. The smallest active
transaction ID can only increase over time, not decrease.
trx_sys_t::find_same_or_older(): Remember the maximum previous id for which
rw_trx_hash.iterate() returned false, to avoid redundant iterations.
lock_sec_rec_read_check_and_lock(): Add an early return in case we are
already holding a covering table lock.
lock_rec_convert_impl_to_expl(): Add a template parameter to avoid
a redundant run-time check on whether the index is secondary.
lock_rec_convert_impl_to_expl_for_trx(): Move some code from
lock_rec_convert_impl_to_expl(), to reduce code duplication due
to the added template parameter.
Reviewed by: Vladislav Lesin
Tested by: Matthias Leich
The initial issue was in assertion failure, which checked the equality
of lock to cancel with trx->lock.wait_lock in lock_sys_t::cancel().
If we analyze lock_sys_t::cancel() code from the perspective of
trx->lock.wait_lock racing, we won't find the error there, except the
cases when we need to reload it after the corresponding latches
acquiring.
So the fix is just to remove the assertion and reload
trx->lock.wait_lock after acquiring necessary latches.
Reviewed by: Marko Mäkelä <marko.makela@mariadb.com>
If two high priority threads have lock conflict, we look at the
order of these transactions and honor the earlier transaction.
for_locking parameter in lock_rec_has_to_wait() has become
obsolete and it is now removed from the code .
Reviewed-by: Jan Lindström <jan.lindstrom@mariadb.com>
ha_innobase::referenced_by_foreign_key(): Protect the check with
dict_sys.freeze(), to prevent races with TRUNCATE TABLE.
The test innodb.instant_alter_crash has been adjusted for this
additional locking.
dict_table_is_referenced_by_foreign_key(): Removed (merged to
the only caller).
create_table_info_t::create_table(): Ignore missing indexes for
FOREIGN KEY constraints if foreign_key_checks=0.
create_table_info_t::create_table_update_dict(): Rewritten as
a static function. Do not return any error.
ha_innobase::create(): When trx!=nullptr and we are operating
on a persistent table, do not rollback, commit, or release the
data dictionary latch.
ha_innobase::truncate(): Protect the entire critical section
with an exclusive dict_sys.latch, so that
ha_innobase::referenced_by_foreign_key() on referenced tables
will return a consistent result. In case of a failure,
invoke dict_load_foreigns() to restore also any FOREIGN KEY
constraints.
ha_innobase::free_foreign_key_create_info(): Define inline.
lock_release(): Disregard innodb_evict_tables_on_commit_debug=ON
when dict_sys.locked() holds. It would hold when fts_load_stopword()
is invoked by create_table_info_t::create_table_update_dict().
dict_sys_t::locked(): Return whether the current thread is holding
the exclusive dict_sys.latch.
dict_sys_t::frozen_not_locked(): Return whether any thread is
holding a shared dict_sys.latch.
In the test main.mysql_upgrade, the InnoDB persistent statistics
will no longer be recalculated in ha_innobase::open() as part of
CHECK TABLE ... FOR UPGRADE. They were deleted earlier in the test.
Tested by: Matthias Leich
Post-push fix. The flag of transaction which indicates that it's necessary
to forbid gap lock inheritance after XA PREPARE could be inverted if
lock_release_on_prepare_try() is invoked several times. The fix is to
toggle it on lock_release_on_prepare() exit.
1. The merge aeccbbd926 has overwritten
lock0lock.cc, and the changes of MDEV-29622 and MDEV-29635 were
partially lost, this commit restores the changes.
2. innodb.deadlock_wait_thr_race test:
The following hang was found during testing.
There is deadlock_report_before_lock_releasing sync point in
Deadlock::report(), which is waiting for sel_cont signal under lock_sys_t
lock. The signal must be issued after "UPDATE t SET b = 100" rollback,
and that rollback is executing undo record, which is blocked
on dict_sys latch request. dict_sys is locked by the thread of statistics
update(dict_stats_save()), and during that update lock_sys lock is
requested, and can't be acquired as Deadlock::report() holds it. We have
to disable statistics update to make the test stable.
But even if statistics update is disabled, and transaction with consistent
snapshot is started at the very beginning of the test to prevent purging,
the purge can still be invoked for system tables, and it tries to open
system table by id, what causes dict_sys.freeze() call and dict_sys
latching. What, in combination with lock_sys::xx_lock() causes the same
deadlock as described above. We need to disable purging globally for the
test as well.
All the above is applicable to innodb.deadlock_wait_lock_race test also.
To prevent ASAN heap-use-after-poison in the MDEV-16549 part of
./mtr --repeat=6 main.derived
the initialization of Name_resolution_context was cleaned up.
The lock is created during page splitting after moving records and
locks(lock_move_rec_list_(start|end)()) to the new page, and inheriting
the locks to the supremum of left page from the successor of the infimum
on right page.
There is no need in such inheritance for READ COMMITTED isolation level
and not-gap locks, so the fix is to add the corresponding condition in
gap lock inheritance function.
One more fix is to forbid gap lock inheritance if XA was prepared. Use the
most significant bit of trx_t::n_ref to indicate that gap lock inheritance
is forbidden. This fix is based on
mysql/mysql-server@b063e52a83
Suppose we have two transactions, trx 1 and trx 2.
trx 2 does deadlock resolving from lock_wait(), it sets
victim->lock.was_chosen_as_deadlock_victim=true for trx 1, but has not
yet invoked lock_cancel_waiting_and_release().
trx 1 checks the flag in lock_trx_handle_wait(), and starts rollback
from row_mysql_handle_errors(). It can change trx->lock.wait_thr and
trx->state as it holds trx_t::mutex, but trx 2 has not yet requested it,
as lock_cancel_waiting_and_release() has not yet been called.
After that trx 1 tries to release locks in trx_t::rollback_low(),
invoking trx_t::rollback_finish(). lock_release() is blocked on try to
acquire lock_sys.rd_lock(SRW_LOCK_CALL) in lock_release_try(), as
lock_sys is blocked by trx 2, as deadlock resolution works under
lock_sys.wr_lock(SRW_LOCK_CALL), see Deadlock::report() for details.
trx 2 executes lock_cancel_waiting_and_release() for deadlock victim, i.
e. for trx 1. lock_cancel_waiting_and_release() contains some
trx->lock.wait_thr and trx->state assertions, which will fail, because
trx 1 has changed them during rollback execution.
So, according to the above scenario, it's legal to have
trx->lock.wait_thr==0 and trx->state!=TRX_STATE_ACTIVE in
lock_cancel_waiting_and_release(), if it was invoked from
Deadlock::report(), and the fix is just in the assertion conditions
changing.
The fix is just in changing assertion condition.
There is also lock_wait() cleanup around trx->error_state.
If trx->error_state can be changed not by the owned thread, it must be
protected with lock_sys.wait_mutex, as lock_wait() uses trx->lock.cond
along with that mutex.
Also if trx->error_state was changed before lock_sys.wait_mutex
acquision, then it could be reset with the following code, what is
wrong. Also we need to check trx->error_state before entering waiting
loop, otherwise it can be the case when trx->error_state was set before
lock_sys.wait_mutex acquision, but the thread will be waiting on
trx->lock.cond.
Returning DB_SUCCESS unconditionally if !trx->lock.wait_lock in
lock_trx_handle_wait() is wrong. Because even if
trx->lock.was_chosen_as_deadlock_victim was not set before the first check
in lock_trx_handle_wait(), it can be set after
the check, and trx->lock.wait_lock can be reset by another thread from
lock_reset_lock_and_trx_wait() if the transaction was chosen as deadlock
victim. In this case lock_trx_handle_wait() will return DB_SUCCESS even
the transaction was marked as deadlock victim, and continue execution
instead of rolling back.
The fix is to check trx->lock.was_chosen_as_deadlock_victim once more if
trx->lock.wait_lock is reset, as trx->lock.wait_lock can be reset only
after trx->lock.was_chosen_as_deadlock_victim was set if the transaction
was chosen as deadlock victim.
Until now, the attribute EXTENDED of CHECK TABLE was ignored by InnoDB,
and InnoDB only counted the records in each index according
to the current read view. Unless the attribute QUICK was specified, the
function btr_validate_index() would be invoked to validate the B-tree
structure (the sibling and child links between index pages).
The EXTENDED check will not only count all index records according to the
current read view, but also ensure that any delete-marked records in the
clustered index are waiting for the purge of history, and that all
secondary index records point to a version of the clustered index record
that is waiting for the purge of history. In other words, no index may
contain orphan records. Normal MVCC reads and the non-EXTENDED version
of CHECK TABLE would ignore these orphans.
Unpurged records merely result in warnings (at most one per index),
not errors, and no indexes will be flagged as corrupted due to such
garbage. It will remain possible to SELECT data from such indexes or
tables (which will skip such records) or to rebuild the table to
reclaim some space.
We introduce purge_sys.end_view that will be (almost) a copy of
purge_sys.view at the end of a batch of purging committed transaction
history. It is not an exact copy, because if the size of a purge batch
is limited by innodb_purge_batch_size, some records that
purge_sys.view would allow to be purged will be left over for
subsequent batches.
The purge_sys.view is relevant in the purge of committed transaction
history, to determine if records are safe to remove. The new
purge_sys.end_view is relevant in MVCC operations and in
CHECK TABLE ... EXTENDED. It tells which undo log records are
safe to access (have not been discarded at the end of a purge batch).
purge_sys.clone_oldest_view<true>(): In trx_lists_init_at_db_start(),
clone the oldest read view similar to purge_sys_t::clone_end_view()
so that CHECK TABLE ... EXTENDED will not report bogus failures between
InnoDB restart and the completed purge of committed transaction history.
purge_sys_t::is_purgeable(): Replaces purge_sys_t::changes_visible()
in the case that purge_sys.latch will not be held by the caller.
Among other things, this guards access to BLOBs. It is not safe to
dereference any BLOBs of a delete-marked purgeable record, because
they may have already been freed.
purge_sys_t::view_guard::view(): Return a reference to purge_sys.view
that will be protected by purge_sys.latch, held by purge_sys_t::view_guard.
purge_sys_t::end_view_guard::view(): Return a reference to
purge_sys.end_view while it is protected by purge_sys.end_latch.
Whenever a thread needs to retrieve an older version of a clustered
index record, it will hold a page latch on the clustered index page
and potentially also on a secondary index page that points to the
clustered index page. If these pages contain purgeable records that
would be accessed by a currently running purge batch, the progress of
the purge batch would be blocked by the page latches. Hence, it is
safe to make a copy of purge_sys.end_view while holding an index page
latch, and consult the copy of the view to determine whether a record
should already have been purged.
btr_validate_index(): Remove a redundant check.
row_check_index_match(): Check if a secondary index record and a
version of a clustered index record match each other.
row_check_index(): Replaces row_scan_index_for_mysql().
Count the records in each index directly, duplicating the relevant
logic from row_search_mvcc(). Initialize check_table_extended_view
for CHECK ... EXTENDED while holding an index leaf page latch.
If we encounter an orphan record, the copy of purge_sys.end_view that
we make is safe for visibility checks, and trx_undo_get_undo_rec() will
check for the safety to access each undo log record. Should that check
fail, we should return DB_MISSING_HISTORY to report a corrupted index.
The EXTENDED check tries to match each secondary index record with
every available clustered index record version, by duplicating the logic
of row_vers_build_for_consistent_read() and invoking
trx_undo_prev_version_build() directly.
Before invoking row_check_index_match() on delete-marked clustered index
record versions, we will consult purge_sys.is_purgeable() in order to
avoid accessing freed BLOBs.
We will always check that the DB_TRX_ID or PAGE_MAX_TRX_ID does not
exceed the global maximum. Orphan secondary index records will be
flagged only if everything up to PAGE_MAX_TRX_ID has been purged.
We warn also about clustered index records whose nonzero DB_TRX_ID
should have been reset in purge or rollback.
trx_set_rw_mode(): Move an assertion from ReadView::set_creator_trx_id().
trx_undo_prev_version_build(): Remove two debug-only parameters,
and return an error code instead of a Boolean.
trx_undo_get_undo_rec(): Return a pointer to the undo log record,
or nullptr if one cannot be retrieved. Instead of consulting the
purge_sys.view, consult the purge_sys.end_view to determine which
records can be accessed.
trx_undo_get_rec_if_purgeable(): A variant of trx_undo_get_undo_rec()
that will consult purge_sys.view instead of purge_sys.end_view.
TRX_UNDO_CHECK_PURGEABILITY: A new parameter to
trx_undo_prev_version_build(), passed by row_vers_old_has_index_entry()
so that purge_sys.view instead of purge_sys.end_view will be consulted
to determine whether a secondary index record may be safely purged.
row_upd_changes_disowned_external(): Remove. This should be more
expensive than briefly latching purge_sys in trx_undo_prev_version_build()
(which may make use of transactional memory).
row_sel_reset_old_vers_heap(): New function, split from
row_sel_build_prev_vers_for_mysql().
row_sel_build_prev_vers_for_mysql(): Reorder some parameters
to simplify the call to row_sel_reset_old_vers_heap().
row_search_for_mysql(): Replaced with direct calls to row_search_mvcc().
sel_node_get_nth_plan(): Define inline in row0sel.h
open_step(): Define at the call site, in simplified form.
sel_node_reset_cursor(): Merged with the only caller open_step().
---
ReadViewBase::check_trx_id_sanity(): Remove.
Let us handle "future" DB_TRX_ID in a more meaningful way:
row_sel_clust_sees(): Return DB_SUCCESS if the record is visible,
DB_SUCCESS_LOCKED_REC if it is invisible, and DB_CORRUPTION if
the DB_TRX_ID is in the future.
row_undo_mod_must_purge(), row_undo_mod_clust(): Silently ignore
corrupted DB_TRX_ID. We are in ROLLBACK, and we should have noticed
that corruption when we were about to modify the record in the first
place (leading us to refuse the operation).
row_vers_build_for_consistent_read(): Return DB_CORRUPTION if
DB_TRX_ID is in the future.
Tested by: Matthias Leich
Reviewed by: Vladislav Lesin
The test is unstable because 'UPDATE t SET b = 100' latches a page and
waits for 'upd_cont' signal in lock_trx_handle_wait_enter sync point, then
purge requests RW_X_LATCH on the same page, and then 'SELECT * FROM t
WHERE a = 10 FOR UPDATE' requests RW_S_LATCH, waiting for RW_X_LATCH
requested by purge. 'UPDATE t SET b = 100' can't release page latch as
it waits for upd_cont signal, which must be emitted after 'SELECT * FROM
t WHERE a = 10 FOR UPDATE' acquired RW_S_LATCH. So we have a deadlock,
which is resolved by finishing the debug sync point wait by timeout, and
the 'UPDATE t SET b = 100' releases it's record locks rolling back the
transaction, and 'SELECT * FROM t WHERE a = 10 FOR UPDATE' is finished
successfully instead of finishing by lock wait timeout.
The fix is to forbid purging during the test by opening read view in a
separate connection before the first insert into the table.
Besides, 'lock_wait_end' syncpoint is not needed, as it enough to wait
the end of the SELECT execution to let the UPDATE to continue.
lock_place_prdt_page_lock(): Do not place locks on temporary tables.
Temporary tables can only be accessed from one connection, so
it does not make any sense to acquire any transactional locks on them.
The issue is that trx_t::lock.was_chosen_as_deadlock_victim can be reset
before the transaction check it and set trx_t::error_state.
The fix is to reset trx_t::lock.was_chosen_as_deadlock_victim only in
trx_t::commit_in_memory(), which is invoked on full rollback. There is
also no need to have separate bit in
trx_t::lock.was_chosen_as_deadlock_victim to flag transaction it was
chosen as a victim of Galera conflict resolution, the same variable can be
used for both cases except debug build. For debug build we need to
distinguish deadlock and Galera's abort victims for debug checks. Also
there is no need to check for deadlock in lock_table_enqueue_waiting() for
Galera as the coresponding check presents in lock_wait().
Local variable "error_state" in lock_wait() was replaced with
trx->error_state, because before the replace
lock_sys_t::cancel<false>(trx, lock) and lock_sys.deadlock_check() could
change trx->error_state, which then could be overwritten with the local
"error_state" variable value.
The lock_wait_suspend_thread_enter DEBUG_SYNC point name is misleading,
because lock_wait_suspend_thread was eliminated in e71e613. It was renamed
to lock_wait_start.
Reviewed by: Marko Mäkelä, Jan Lindström.
The function rec_get_offsets_func() used to hit ut_error
due to an invalid rec_get_status() value of a
ROW_FORMAT!=REDUNDANT record. This fix is twofold:
We will not only avoid a crash on corruption in this case,
but we will also make more effort to validate each record
every time we are iterating over index page records.
rec_get_offsets_func(): Do not crash on a corrupted record.
page_rec_get_nth(): Return nullptr on error.
page_dir_slot_get_rec_validate(): Like page_dir_slot_get_rec(),
but validate the pointer and return nullptr on error.
page_cur_search_with_match(), page_cur_search_with_match_bytes(),
page_dir_split_slot(), page_cur_move_to_next():
Indicate failure in a return value.
page_cur_search(): Replaced with page_cur_search_with_match().
rec_get_next_ptr_const(), rec_get_next_ptr(): Replaced with
page_rec_get_next_low().
TODO: rtr_page_split_initialize_nodes(), rtr_update_mbr_field(),
and possibly other SPATIAL INDEX functions fail to properly handle
errors.
Reviewed by: Thirunarayanan Balathandayuthapani
Tested by: Matthias Leich
Performance tested by: Axel Schwenke
SHOW ENGINE INNODB STATUS and SHOW GLOBAL VARIABLES were blocking on the locks used
to access the history length in MDEV-29141. While the reason for the blockage
was elsewhere, we should make these monitoring commands less blocking as there
is a trx_sys.history_size_approx function that can be used.
SHOW ENGINE INNODB STATUS and SHOW GLOBAL STATUS LIKE
'innodb_history_list_length' and Innodb Monitors can use
trx_sys.history_size_approx().
Table_cache_instance: Define the structure aligned at
the CPU cache line, and remove a pad[] data member.
Krunal Bauskar reported this to improve performance on ARMv8.
aligned_malloc(): Wrapper for the Microsoft _aligned_malloc()
and the ISO/IEC 9899:2011 <stdlib.h> aligned_alloc().
Note: The parameters are in the Microsoft order (size, alignment),
opposite of aligned_alloc(alignment, size).
Note: The standard defines that size must be an integer multiple
of alignment. It is enforced by AddressSanitizer but not by GNU libc
on Linux.
aligned_free(): Wrapper for the Microsoft _aligned_free() and
the standard free().
HAVE_ALIGNED_ALLOC: A new test. Unfortunately, support for
aligned_alloc() may still be missing on some platforms.
We will fall back to posix_memalign() for those cases.
HAVE_MEMALIGN: Remove, along with any use of the nonstandard memalign().
PFS_ALIGNEMENT (sic): Removed; we will use CPU_LEVEL1_DCACHE_LINESIZE.
PFS_ALIGNED: Defined using the C++11 keyword alignas.
buf_pool_t::page_hash_table::create(),
lock_sys_t::hash_table::create():
lock_sys_t::hash_table::resize(): Pad the allocation size to an
integer multiple of the alignment.
Reviewed by: Vladislav Vaintroub
The approach to handling corruption that was chosen by Oracle in
commit 177d8b0c12
is not really useful. Not only did it actually fail to prevent InnoDB
from crashing, but it is making things worse by blocking attempts to
rescue data from or rebuild a partially readable table.
We will try to prevent crashes in a different way: by propagating
errors up the call stack. We will never mark the clustered index
persistently corrupted, so that data recovery may be attempted by
reading from the table, or by rebuilding the table.
This should also fix MDEV-13680 (crash on btr_page_alloc() failure);
it was extensively tested with innodb_file_per_table=0 and a
non-autoextend system tablespace.
We should now avoid crashes in many cases, such as when a page
cannot be read or allocated, or an inconsistency is detected when
attempting to update multiple pages. We will not crash on double-free,
such as on the recovery of DDL in system tablespace in case something
was corrupted.
Crashes on corrupted data are still possible. The fault injection mechanism
that is introduced in the subsequent commit may help catch more of them.
buf_page_import_corrupt_failure: Remove the fault injection, and instead
corrupt some pages using Perl code in the tests.
btr_cur_pessimistic_insert(): Always reserve extents (except for the
change buffer), in order to prevent a subsequent allocation failure.
btr_pcur_open_at_rnd_pos(): Merged to the only caller ibuf_merge_pages().
btr_assert_not_corrupted(), btr_corruption_report(): Remove.
Similar checks are already part of btr_block_get().
FSEG_MAGIC_N_BYTES: Replaces FSEG_MAGIC_N_VALUE.
dict_hdr_get(), trx_rsegf_get_new(), trx_undo_page_get(),
trx_undo_page_get_s_latched(): Replaced with error-checking calls.
trx_rseg_t::get(mtr_t*): Replaces trx_rsegf_get().
trx_rseg_header_create(): Let the caller update the TRX_SYS page if needed.
trx_sys_create_sys_pages(): Merged with trx_sysf_create().
dict_check_tablespaces_and_store_max_id(): Do not access
DICT_HDR_MAX_SPACE_ID, because it was already recovered in dict_boot().
Merge dict_check_sys_tables() with this function.
dir_pathname(): Replaces os_file_make_new_pathname().
row_undo_ins_remove_sec(): Do not modify the undo page by adding
a terminating NUL byte to the record.
btr_decryption_failed(): Report decryption failures
dict_set_corrupted_by_space(), dict_set_encrypted_by_space(),
dict_set_corrupted_index_cache_only(): Remove.
dict_set_corrupted(): Remove the constant parameter dict_locked=false.
Never flag the clustered index corrupted in SYS_INDEXES, because
that would deny further access to the table. It might be possible to
repair the table by executing ALTER TABLE or OPTIMIZE TABLE, in case
no B-tree leaf page is corrupted.
dict_table_skip_corrupt_index(), dict_table_next_uncorrupted_index(),
row_purge_skip_uncommitted_virtual_index(): Remove, and refactor
the callers to read dict_index_t::type only once.
dict_table_is_corrupted(): Remove.
dict_index_t::is_btree(): Determine if the index is a valid B-tree.
BUF_GET_NO_LATCH, BUF_EVICT_IF_IN_POOL: Remove.
UNIV_BTR_DEBUG: Remove. Any inconsistency will no longer trigger
assertion failures, but error codes being returned.
buf_corrupt_page_release(): Replaced with a direct call to
buf_pool.corrupted_evict().
fil_invalid_page_access_msg(): Never crash on an invalid read;
let the caller of buf_page_get_gen() decide.
btr_pcur_t::restore_position(): Propagate failure status to the caller
by returning CORRUPTED.
opt_search_plan_for_table(): Simplify the code.
row_purge_del_mark(), row_purge_upd_exist_or_extern_func(),
row_undo_ins_remove_sec_rec(), row_undo_mod_upd_del_sec(),
row_undo_mod_del_mark_sec(): Avoid mem_heap_create()/mem_heap_free()
when no secondary indexes exist.
row_undo_mod_upd_exist_sec(): Simplify the code.
row_upd_clust_step(), dict_load_table_one(): Return DB_TABLE_CORRUPT
if the clustered index (and therefore the table) is corrupted, similar
to what we do in row_insert_for_mysql().
fut_get_ptr(): Replace with buf_page_get_gen() calls.
buf_page_get_gen(): Return nullptr and *err=DB_CORRUPTION
if the page is marked as freed. For other modes than
BUF_GET_POSSIBLY_FREED or BUF_PEEK_IF_IN_POOL this will
trigger a debug assertion failure. For BUF_GET_POSSIBLY_FREED,
we will return nullptr for freed pages, so that the callers
can be simplified. The purge of transaction history will be
a new user of BUF_GET_POSSIBLY_FREED, to avoid crashes on
corrupted data.
buf_page_get_low(): Never crash on a corrupted page, but simply
return nullptr.
fseg_page_is_allocated(): Replaces fseg_page_is_free().
fts_drop_common_tables(): Return an error if the transaction
was rolled back.
fil_space_t::set_corrupted(): Report a tablespace as corrupted if
it was not reported already.
fil_space_t::io(): Invoke fil_space_t::set_corrupted() to report
out-of-bounds page access or other errors.
Clean up mtr_t::page_lock()
buf_page_get_low(): Validate the page identifier (to check for
recently read corrupted pages) after acquiring the page latch.
buf_page_t::read_complete(): Flag uninitialized (all-zero) pages
with DB_FAIL. Return DB_PAGE_CORRUPTED on page number mismatch.
mtr_t::defer_drop_ahi(): Renamed from mtr_defer_drop_ahi().
recv_sys_t::free_corrupted_page(): Only set_corrupt_fs()
if any log records exist for the page. We do not mind if read-ahead
produces corrupted (or all-zero) pages that were not actually needed
during recovery.
recv_recover_page(): Return whether the operation succeeded.
recv_sys_t::recover_low(): Simplify the logic. Check for recovery error.
Thanks to Matthias Leich for testing this extensively and to the
authors of https://rr-project.org for making it easy to diagnose
and fix any failures that were found during the testing.
srv_do_purge(): In commit edde1f6e0d
when the de-facto 32-bit trx_sys_t::history_size() was replaced with
32-bit trx_sys.rseg_history_len, some more variables were changed
from ulint (size_t) to uint32_t.
The history list length is the number of committed transactions whose
undo logs are waiting to be purged. Each TRX_RSEG_HISTORY list is
storing the number of entries in a 32-bit field and each transaction
will occupy at least one undo log page. It is thinkable that the
length of each TRX_RSEG_HISTORY list may approach the maximum
representable number. The number cannot be exceeded, because the
rollback segment header is allocated from the same tablespace as
the undo log header pages it is pointing to, and because the page
numbers of a tablespace are stored in 32 bits. In any case, it is
possible that the total number of unpurged committed transactions
cannot be represented in 32 but 39 bits (corresponding to
128 rollback segments and undo tablespaces).
lock_validate() accumulates page ids under locked lock_sys->mutex, then
releases the latch, and invokes lock_rec_block_validate() for each page.
Some other thread has ability to add/remove locks and change pages
between releasing the latch in lock_validate() and acquiring it in
lock_rec_validate_page().
lock_rec_validate_page() can invoke lock_rec_queue_validate() for
non-locked supremum, what can cause ut_ad(page_rec_is_leaf(rec)) failure
in lock_rec_queue_validate().
The fix is to invoke lock_rec_queue_validate() only for locked records
in lock_rec_validate_page().
The error message in lock_rec_block_validate() is not necessary as
BUF_GET_POSSIBLY_FREED mode is used to get block from buffer pool, and
this is not error if a block was evicted.
The test case would require new debug sync point. I think it's not
necessary as the fixed code is debug-only.
lock_sec_rec_read_check_and_lock(): Remove a redundant check
for trx_sys.get_min_trx_id(). This would be checked in
lock_sec_rec_some_has_impl() anyway. Also, test the
cheapest conditions first.
lock_sec_rec_some_has_impl(): Replace trx_sys.get_min_trx_id()
with trx_sys.find_same_or_older() that is much easier to evaluate.
Inspired by mysql/mysql-server@0a0c0db97e
btr_insert_into_right_sibling(): Inherit any gap lock from the
left sibling to the right sibling before inserting the record
to the right sibling and updating the node pointer(s).
lock_update_node_pointer(): Update locks in case a node pointer
will move.
Based on mysql/mysql-server@c7d93c274f
This follows up the previous fix in
commit c3c53926c4 (MDEV-26554).
ha_innobase::delete_table(): Work around the insufficient
metadata locking (MDL) during DML operations by acquiring exclusive
InnoDB table locks on all child tables. Previously, this was only
done on TRUNCATE and ALTER.
ibuf_delete_rec(), btr_cur_optimistic_delete(): Do not invoke
lock_update_delete() during change buffer operations.
The revised trx_t::commit(std::vector<pfs_os_file_t>&) will
hold exclusive lock_sys.latch while invoking fil_delete_tablespace(),
which in turn may invoke ibuf_delete_rec().
dict_index_t::has_locking(): A new predicate, replacing the dummy
!dict_table_is_locking_disabled(index->table). Used for skipping lock
operations during ibuf_delete_rec().
trx_t::commit(std::vector<pfs_os_file_t>&): Release the locks
and remove the table from the cache while holding exclusive
lock_sys.latch.
trx_t::commit_in_memory(): Skip release_locks() if dict_operation holds.
trx_t::commit(): Reset dict_operation before invoking commit_in_memory()
via commit_persist().
lock_release_on_drop(): Release locks while lock_sys.latch is
exclusively locked.
lock_table(): Add a parameter for a pointer to the table.
We must not dereference the table before a lock_sys.latch has
been acquired. If the pointer to the table does not match the table
at that point, the table is invalid and DB_DEADLOCK will be returned.
row_ins_foreign_check_on_constraint(): Improve the checks.
Remove a bogus DB_LOCK_WAIT_TIMEOUT return that was needed
before commit c5fd9aa562 (MDEV-25919).
row_upd_check_references_constraints(),
wsrep_row_upd_check_foreign_constraints(): Simplify checks.
The fix in commit 6e390a62ba (MDEV-26772)
was a step to the right direction, but implemented incorrectly.
When an InnoDB persistent statistics table cannot be locked immediately,
we must not let row_mysql_handle_errors() to roll back the transaction.
lock_table_for_trx(): Add the parameter no_wait (default false)
for an immediate return of DB_LOCK_WAIT in case of a conflict.
ha_innobase::delete_table(), ha_innobase::rename_table():
Pass no_wait=true to lock_table_for_trx() when needed,
instead of temporarily setting THDVAR(thd, lock_wait_timeout) to 0.
MDEV-27025 allows to insert records before the record on which DELETE is
locked, as a result the DELETE misses those records, what causes serious ACID
violation.
Revert MDEV-27025, MDEV-27550. The test which shows the scenario of ACID
violation is added.
The code was backported from 10.6 bd03c0e516
commit. See that commit message for details.
Apart from the above commit trx_lock_t::wait_trx was also backported from
MDEV-24738. trx_lock_t::wait_trx is protected with lock_sys.wait_mutex
in 10.6, but that mutex was implemented only in MDEV-24789. As there is no
need to backport MDEV-24789 for MDEV-27025,
trx_lock_t::wait_trx is protected with the same mutexes as
trx_lock_t::wait_lock.
This fix should not break innodb-lock-schedule-algorithm=VATS. This
algorithm uses an Eldest-Transaction-First (ETF) heuristic, which prefers
older transactions over new ones. In this fix we just insert granted lock
just before the last granted lock of the same transaction, what does not
change transactions execution order.
The changes in lock_rec_create_low() should not break Galera Cluster,
there is a big "if" branch for WSREP. This branch is necessary to provide
the correct transactions execution order, and should not be changed for
the current bug fix.
When lock is checked for conflict, ignore other locks on the record if
they wait for the requesting transaction.
lock_rec_has_to_wait_in_queue() iterates not all locks for
the page, but only the locks located before the waiting lock in the
queue. So there is some invariant - any lock in the queue can wait only
lock which is located before the waiting lock in the queue.
In the case when conflicting lock waits for the transaction of
requesting lock, we need to place the requesting lock before the waiting
lock in the queue to preserve the invariant. That is why we are looking
for the first waiting for requesting transation lock and place the new
lock just after the last granted requesting transaction lock before the
first waiting for requesting transaction lock.
Example:
trx1 waiting lock, trx1 granted lock, ..., trx2 lock - waiting for trx1
place new lock here -----------------^
There are also implicit locks which are lazily converted to explicit
ones, and we need to place the newly created explicit lock to the correct
place in a queue. All explicit locks converted from implicit ones are
placed just after the last non-waiting lock of the same transaction before
the first waiting for the transaction lock.
Code review and cleanup was made by Marko Mäkelä.