During insertion of clustered index, InnoDB does the check for foreign key
constraints. Problem is that it uses the clustered index entry to search
indexes of referenced tables and it could lead to unexpected result
when there is no foreign index.
Solution:
========
Rebuild the tuple based on foreign column names before searching it
on reference index when there is no foreign index.
Build failure was:
storage/innobase/os/os0proc.cc:144:3: error: use of undeclared identifier 'MEM_UNDEFINED'
MEM_UNDEFINED(ptr, size);
Assumed to be introduced in MDEV-20377
commit: c36834c832
The InnoDB index fields store bytes, not characters.
Remove some unnecessary conversions from characters to bytes.
This also fixes MDEV-20422 and the wrong-result bug MDEV-12486.
Add a proper error handling of innobase_get_computed_value results in
row_upd_store_row/row_upd_store_v_row.
Also add an assertion in row_vers_build_clust_v_col to fail during row
purge.
Add one more assertion in row_sel_sec_rec_is_for_clust_rec for possible
future catches.
The problem was in improper error handling behavior in
`row_upd_build_difference_binary`:
`innobase_free_row_for_vcol` wasn't called.
To eliminate this problem in all potential places, a refactoring has been
made:
* class ib_vcol_row is added. It owns VCOL_STORAGE and heap and maintains
it in RAII manner
* all innobase_allocate_row_for_vcol/innobase_free_row_for_vcol pairs are
substituted
with ib_vcol_row usage
* row_merge_buf_add is only left untouched because it doesn't own vheap
passed as an argument
* innobase_allocate_row_for_vcol does not allocate VCOL_STORAGE anymore and
accepts it as an argument -- this reduces a number of memory allocations
* move rec_printer out of `#ifndef DBUG_OFF` and mark it cold
This commit contains a fix and extended test case for a ASAN failure
reported during galera.fk mtr testing.
The reported heap buffer overflow happens in test case where a cascading
foreign key constraint is defined for a column of varchar type, and
galera.fk.test has such vulnerable test scenario.
Troubleshoting revealed that erlier fix for MDEV-19660 has made a fix
for cascading delete handling to append wsrep keys from pcur->old_rec,
in row_ins_foreign_check_on_constraint(). And, the ASAN failuer comes from
later scanning of this old_rec reference.
The fix in this commit, moves the call for wsrep_append_foreign_key() to happen
somewhat earlier, and inside ongoing mtr, and using clust_rec which is set
earlier in the same mtr for both update and delete cascade operations.
for wsrep key populating, it does not matter when the keys are populated,
all keys just have to be appended before wsrep transaction replicates.
Note that I also tried similar fix for earlier wsrep key append, but using
the old implementation with pcur->old_rec (instead of clust_rec), and same
ASAN failure was reported. So it appears that pcur->old_rec is not properly
set, to be used for wsrep key appending.
galera.galera_fk_cascade_delete test has been extended by two new test scenarios:
* FK cascade on varchar column.
This test case reproduces same scenario as galera.fk, and this test scenario
will also trigger ASAN failure with non fixed MariaDB versions.
* multi-master conflict with FK cascading.
this scenario causes a conflict between a replicated FK cascading transaction
and local transaction trying to modify the cascaded child table row.
Local transaction should be aborted and get deadlock error.
This test scenario is passing both with old MariaDB version and with this
commit as well.
This bug was originally repeated on 10.4 after defining a UNIQUE KEY
on a TEXT column, which is implemented by MDEV-371 by creating the
index on a hidden virtual column.
While row_vers_vc_matches_cluster() is executing in a purge thread
to find out if an index entry may be removed in a secondary index
that comprises a virtual column, another purge thread may process
the undo log record that this check is interested in, and write
a null BLOB pointer in that record. This would trip the assertion.
To prevent this from occurring, we must propagate the 'missing BLOB'
error up the call stack.
row_upd_ext_fetch(): Return NULL when the error occurs.
row_upd_index_replace_new_col_val(): Return whether the previous
version was built successfully.
row_upd_index_replace_new_col_vals_index_pos(): Check the error
result. Yes, we would intentionally crash on this error if it
occurs outside the purge thread.
row_upd_index_replace_new_col_vals(): Check for the error condition,
and simplify the logic.
trx_undo_prev_version_build(): Check for the error condition.
In trx_free() we used to declare the entire trx_t unaccessible
and then declare that some data members are accessible.
This involves a race condition with other threads that may concurrently
access the data members that must remain accessible.
One type of error is "AddressSanitizer: unknown-crash", whose
exact cause we have not determined.
Another type of error (reported in MDEV-23472) is "use-after-poison",
where the reported shadow bytes would in fact be 00, indicating that
the memory was no longer poisoned. The poison-access-unpoison race
condition was confirmed by "rr replay".
We eliminate the race condition by invoking MEM_NOACCESS on each
individual data member of trx_t before freeing the memory to the pool.
The memory would not be unpoisoned until the pool is freed
or the memory is being reused for another allocation.
trx_t::free(): Replaces trx_free().
trx_t::active_commit_ordered: Changed to bool, so that MEM_NOACCESS
can be invoked. Removed some accessor functions.
Pool: Remove all MEM_ instrumentation.
TrxFactory: Move the MEM_ instrumentation from Pool.
TrxFactory::debug(): Removed. Moved to trx_t::free(). Because
the memory was already marked unaccessible in trx_t::free(), the
Factory::debug() call in Pool::putl() would be unable to access it.
trx_allocate_for_background(): Replaces trx_create_low().
trx_t::free(): Perform all consistency checks while avoiding
duplication, and declare most data members unaccessible.
in buf_page_set_sticky
commit a1f899a8ab (MDEV-23233) added the
code to make page sticky. So that InnoDB can't allow the page to
be grabbed by other thread while doing lazy drop of ahi.
But the block could be in flush list and it could have io_fix value
as BUF_IO_WRITE. It could lead to the failure in buf_page_set_sticky().
buf_page_create(): If btr_search_drop_page_hash_index() must be invoked,
take x-latch on the block. If the block io_fix value is other than
BUF_IO_NONE, release the buffer pool mutex and page hash lock and
wait for I/O to complete.
Since commit 1509363970 (MDEV-23484)
the rollback of InnoDB transactions is no longer protected by
dict_operation_lock. Removing that protection revealed a race
condition between transaction rollback and the rollback of an
online table-rebuilding operation (OPTIMIZE TABLE, or any online
ALTER TABLE that is rebuilding the table).
row_undo_mod_clust(): Re-check dict_index_is_online_ddl() after
acquiring index->lock, similar to how row_undo_ins_remove_clust_rec()
is doing it. Because innobase_online_rebuild_log_free() is holding
exclusive index->lock while invoking row_log_free(), this re-check
will ensure that row_log_table_low() will not be invoked when
index->online_log=NULL.
A different race condition is possible between the rollback of a
recovered transaction and the start of online secondary index creation.
Because prepare_inplace_alter_table_dict() is not acquiring an InnoDB
table lock in this case, and because recovered transactions are not
covered by metadata locks (MDL), the dict_table_t::indexes could be
modified by prepare_inplace_alter_table_dict() while the rollback of
a recovered transaction is being executed. Normal transactions would
be covered by MDL, and during prepare_inplace_alter_table_dict() we
do hold MDL_EXCLUSIVE, that is, an online ALTER TABLE operation may
not execute concurrently with other transactions that have accessed
the table.
row_undo(): To prevent a race condition with
prepare_inplace_alter_table_dict(), acquire dict_operation_lock
for all recovered transactions. Before MDEV-23484 we used to acquire
it for all transactions, not only recovered ones.
Note: row_merge_drop_indexes() would not invoke
dict_index_remove_from_cache() while transactional locks
exist on the table, or while any thread is holding an open table handle.
OK, it does that for FULLTEXT INDEX, but ADD FULLTEXT INDEX is not
supported as an online operation, and therefore
prepare_inplace_alter_table_dict() would acquire a table S lock,
which cannot succeed as long as recovered transactions on the table
exist, because they would hold a conflicting IX lock on the table.
In commit fe39d02f51 (MDEV-20638)
we removed some wake-up signaling of the master thread that should
have been there, to ensure a steady log checkpointing workload.
Common sense suggests that the commit omitted some necessary calls
to srv_inc_activity_count(). But, an attempt to add the call to
trx_flush_log_if_needed_low() as well as to reinstate the function
innobase_active_small() did not restore the performance for the
case where sync_binlog=1 is set.
Therefore, we will revert the entire commit in MariaDB Server 10.2.
In MariaDB Server 10.5, adding a srv_inc_activity_count() call to
trx_flush_log_if_needed_low() did restore the performance, so we
will not revert MDEV-20638 across all versions.
InnoDB transaction rollback includes an unnecessary work-around for
a data corruption bug that was fixed by me in MySQL 5.6.12
mysql/mysql-server@935ba09d52
and ported to MariaDB 10.0.8 by
commit c291ddfdf7
in 2013 and 2014, respectively.
By acquiring and releasing dict_operation_lock in shared mode,
row_undo() hopes to prevent the table from being dropped while
the undo log record is being rolled back. But, thanks to mentioned fix,
debug assertions (that we are adding) show that the rollback is
protected by transactional locks (table IX lock, in addition to
implicit or explicit exclusive locks on the records that had been modified).
Because row_drop_table_for_mysql() would invoke
row_add_table_to_background_drop_list() if any locks exist on the table,
the mere existence of locks (which is guaranteed during ROLLBACK) is
enough to protect the table from disappearing. Hence, acquiring and
releasing dict_operation_lock for every row that is being rolled back is
unnecessary.
row_undo(): Remove the unnecessary acquisition and release of
dict_operation_lock.
Note: row_add_table_to_background_drop_list() is mostly working around
bugs outside InnoDB:
MDEV-21175 (insufficient MDL protection of FOREIGN KEY operations)
MDEV-21602 (incorrect error handling of CREATE TABLE...SELECT).
Regretfully, the parameter innodb_log_checksums was introduced
in MySQL 5.7.9 (the first GA release of that series) by
mysql/mysql-server@af0acedd88
which partly replaced a parameter that had been introduced in 5.7.8
mysql/mysql-server@22ba38218e
as innodb_log_checksum_algorithm.
Given that the CRC-32C operations are accelerated on many processor
implementations (AMD64 with SSE4.2; since MDEV-22669 also on IA-32
with SSE4.2, POWER 8 and later, ARMv8 with some extensions)
and by lookup tables when only generic SISD instructions are available,
there should be no valid reason to disable checksums.
In MariaDB 10.5.2, as a preparation for MDEV-12353, MDEV-19543 deprecated
and ignored the parameter innodb_log_checksums altogether. This should
imply that after a clean shutdown with innodb_log_checksums=OFF one
cannot upgrade to MariaDB Server 10.5 at all.
Due to these problems, let us deprecate the parameter innodb_log_checksums
and honor it only during server startup.
The command SET GLOBAL innodb_log_checksums will always set the
parameter to ON.
Problem:
=======
InnoDB drops the column which has foreign key relations on it. So it
tries to load the foreign key during rename process of copy algorithm
even though the foreign_key_check is disabled.
Solution:
========
During alter copy algorithm, InnoDB ignores the error while loading
the foreign key constraint if foreign key check is disabled. It
should throw the warning about failure of the foreign key constraint
when foreign key check is disabled.
This problem is caused by 6697135c6d
(MDEV-21572). During recovery, InnoDB prefetches the siblings of
change buffer index leaf page. It does asynchronous page read
and recovery scenario wasn't handled in buf_read_page_background().
It leads to the refusal of startup of the server.
Solution:
=========
InnoDB shouldn't allow the change buffer index page siblings
to be prefetched.
btr_validate_index(): do not stop checking after some level failed.
That way it'll become possible to see errors in leaf pages even when
uppers layers are corrupted too.
page_validate(): check info_bits and status_bits more
fil_page_decompress(): Remove a rather useless debug check.
We should have test coverage for reading page_compressed pages
from files, either due to buffer pool page eviction or due to
server restarts.
A similar check was removed from fil_space_encrypt() in
commit 0b36c27e0c (MDEV-20307).
The usage message for the innodb_compression_algorithm system variable
did not list snappy, which was added as an optional compression algorithm
in MariaDB 10.1.3 and might actually work since
commit 90c52e5291 (MDEV-12615)
in MariaDB 10.1.24.
Unfortunately, we will include also unavailable compression algorithms
in the list, because ENUM parameters allow numeric values, and we do
not want innodb_compression_algorithm=3 to change meaning depending on
the way how the source code was compiled.
InnoDB only reserves 13 bits for the heap number in the record header,
limiting the heap number to be at most 8191. But, when using
innodb_page_size=64k and secondary index records of 7 bytes each,
it is possible to exceed the maximum heap number.
btr_cur_optimistic_insert(): Let the operation fail if the
maximum number of records would be exceeded.
page_mem_alloc_heap(): Move to the same compilation unit with the
only caller, and let the operation fail if the maximum heap number
has been allocated already.
The debug assertion is bogus, and we had removed it in
commit b1ab211dee (MDEV-15053)
in the MariaDB Server 10.5 branch.
For a small data file, fil_space_extend_must_retry() would always
allocate a minimum size of 4*innodb_page_size.
It is possible that random read-ahead will be triggered for
a smaller file than this. In the observed case, the read-ahead
was triggered for a 6-page file that used ROW_FORMAT=COMPRESSED
with 8KiB page size. So, the desired file size was 49152 bytes,
but the actual size was 65536 bytes.
innobase_pk_order_preserved(): Treat an added AUTO_INCREMENT
column in the same way as an added existing column.
In either case, the column values are not guaranteed to
be constant, and thus the ordering may change if such a column
is added before any existing PRIMARY KEY columns.
prepare_inplace_alter_table_dict(): Initialize
dict_table_t::persistent_autoinc before invoking
innobase_pk_order_preserved().
fil_system_t::keyrotate_next(): If space && space->is_in_rotation_list
does not hold, iterate from the start of the list.
In debug builds, we would typically have hit SIGSEGV because the
iterator would have wrapped a null pointer. It might also be that
we are dereferencing a stale pointer.
There is no test case, because the encryption is very nondeterministic
in nature, due to the use of background threads.
This scenario can be hit by setting the following:
SET GLOBAL innodb_encryption_threads=5;
SET GLOBAL innodb_encryption_rotate_key_age=0;
The test encryption.create_or_replace would occasionally fail,
because some fil_space_t::n_pending_ops would never be decremented.
fil_crypt_find_space_to_rotate(): If rotate_thread_t::should_shutdown()
holds due to innodb_encryption_threads having been reduced, do
release the reference.
fil_space_remove_from_keyrotation(), fil_space_next(): Declare the
functions static, simplify a little, and define in the same compilation
unit with the only caller, fil_crypt_find_space_to_rotate().
fil_crypt_key_mutex: Remove (unused).
lock_rec_has_to_wait_in_queue(): Remove an obviously redundant assertion
that was added in commit a8ec45863b
and also enclose a Galera-specific condition in #ifdef WITH_WSREP.
lock_rec_has_to_wait
wsrep_kill_victim
lock_rec_create_low
lock_rec_add_to_queue
DeadlockChecker::select_victim()
THD can't change from normal transaction to BF (brute force) transaction
here, thus there is no need to syncronize access in wsrep_thd_is_BF
function.
lock_rec_has_to_wait_in_queue
Add condition that lock is not NULL and add assertions if we are in
strong state.
The purpose of the InnoDB doublewrite buffer is to make InnoDB
tolerant against cases where the server was killed in the middle
of a page write. (In Linux, killing a process may interrupt a
write system call, typically on a 4096-byte boundary.)
There may exist multiple copies of a page number in the doublewrite
buffer. Recovery should choose the latest valid copy of the page.
By design, the FIL_PAGE_LSN must not precede the latest checkpoint LSN
nor be later than the end of the recovered log.
For page_compressed and encrypted pages, we were missing proper
consistency checks. In the 10.4 data set generated for in MDEV-23231,
the data file contained a valid page_compressed page, and an
identical copy of that page was also present in the doublewrite
buffer. But, recovery would incorrectly consider the page invalid
and restore an uncompressed copy of the same page that had been
written before the log checkpoint. (In fact, no redo log was to
be applied to that page.)
buf_dblwr_process(): Validate the FIL_PAGE_LSN in the doublewrite
buffer pages, and always skip page 0, because those pages should
have been recovered by Datafile::restore_from_doublewrite() if
necessary.
Datafile::restore_from_doublewrite(): Choose the latest applicable
page from the doublewrite buffer.
recv_dblwr_t::find_page(): Also validate encrypted or
page_compressed pages.
recv_dblwr_t::validate_page(): New function to validate a page,
either a copy in a data file or in the doublewrite buffer.
Also validate encrypted or page_compressed pages.
This is joint work with Thirunarayanan Balathandayuthapani.
row_vers_impl_x_locked_low(): clust_offsets may point to memory
that is allocated by mem_heap_alloc() and may have been freed.
For initializing clust_offsets, try to use the stack-allocated
buffer instead of a pointer that may point to freed memory.
This fixes a regression that was introduced in
commit f0aa073f2b (MDEV-20950).
rec_get_nth_cfield(): Remove a bogus debug assertion.
The function may be invoked by innobase_rec_to_mysql()
for reporting a duplicate key error during CREATE UNIQUE INDEX
or ALTER TABLE...ADD UNIQUE KEY, and in that case the record
will be missing the 5-byte or 6-byte fixed header.
It turns out that in every other code path leading to
rec_get_nth_cfield() we either invoked rec_get_offsets()
ourselves or asserted rec_offs_validate(). So, we can
safely remove the assertion and make debug builds
smaller and faster.
Problem:
========
In row_merge_drop_indexes(), InnoDB drops only the index from
dictionary and frees the index pages but it maintains the index
object if the table is being used by other DML threads. It sets
the online status of the index to ONLINE_INDEX_ABORTED_DROPPED.
Removing the index from dictionary doesn't remove the
corressponding ahi entries of the index. When block is being
reused, InnoDB tries to remove ahi entries for the block and
it fails if index online status is ONLINE_INDEX_ABORTED_DROPPED.
Fix:
====
MDEV-22456 allows the index ahi entries to be dropped lazily.
so checking online status in btr_search_drop_page_hash_index()
is meaningless and should be removed.
mysql/mysql-server@e00ad49edc
which introduced WL#6326 to MySQL 5.7.2 added a buffer page
acquisition to CHECK TABLE code (solely for the purpose of
obeying the changed latching order), but failed to check that
a parent page actually exists. It would not necessarily exist in a
corrupted index where a parent page is missing pointer records
to child pages.
commit ad6171b91c (MDEV-22456)
introduced code to buf_page_create() that would lazily drop
adaptive hash index entries for an index that has been
evicted from the data dictionary cache.
Unfortunately, that call was missing adequate protection.
While the btr_search_drop_page_hash_index(block) was executing,
the block could be reused for something else.
buf_page_create(): If btr_search_drop_page_hash_index() must be
invoked, pin the block before releasing the buf_pool->page_hash lock,
so that the block cannot be grabbed by other threads.
Problem:
=======
In buf_cur_optimistic_latch_leaves(), requesting a left block with BTR_GET
after releasing current block. But there is no guarantee that left block
could be still available.
Fix:
====
(1) In btr_cur_optimistic_latch_leaves(), replace the BUF_GET with
BUF_GET_POSSIBLY_FREED for fetching left block.
(2) Once InnoDB acquires left block, it should check FIL_PAGE_NEXT with
current block page number. If not, release cursor->left_block and return
false.
THD proc info was assigned from stack allocated temporary buffer
which went out of scope immediately after assignment.
Fixed by removing the use of temp buffer and assign proc info
from string literal.
Problem:
========
dict_load_table_one() doesn't handle the scenario where clustered
index page is FIL_NULL when DICT_ERR_IGNORE_INDEX_ROOT mode
is set.
Fix:
====
InnoDB should set the file_unreadable when it can't find the
clustered index root page.
The fix of MDEV-13654 (commit ff81faf670)
wrongly caused ADD PRIMARY KEY to ignore duplicate PRIMARY KEY values
caused by concurrent DML transactions that had been started before the
ALTER TABLE operation (but did not access the table before the ALTER TABLE
started).
row_ins_duplicate_online(): Always report a duplicate key error
if DB_TRX_ID had been reset (it belongs to a transaction that had
started before the ALTER TABLE operation).
Problem:
=======
fts_cache_append_deleted_doc_ids() holds the deleted_lock and tries to
access size of deleted_doc_ids. In the meantime, fts_cache_clear()
clears the sync_heap before clearing deleted_doc_ids. It leads to
invalid access of deleted_doc_ids.
Fix:
===
fts_cache_clear() should free the sync_heap after clearing
deleted_doc_ids.
- Due to commit fe95cb2e40 (MDEV-16125),
InnoDB master thread does not need to call srv_resume_thread()
and therefore there is no need to wake up the thread.
Due to the above patch, InnoDB should remove the following dead code.
srv_check_activity(): Makes the parameter as in,out and returns the
recent activity value
innobase_active_small(): Removed
srv_active_wake_master_thread(): Removed
srv_wake_master_thread(): Removed
srv_active_wake_master_thread_low(): Removed
Simplify srv_master_thread() and remove switch cases, added the assert.
Replace srv_wake_master_thread() with srv_inc_activity_count()
INNOBASE_WAKE_INTERVAL: Removed
The srv_monitor_event and the srv_monitor_thread would not be
created when InnoDB is in read-only mode. Yet, some code would
unconditionally invoke os_event_set(srv_monitor_event).
__pthread_cond_timedwait() in page cleaner hangs if os time moved
backwards.Workaround could be waking up the page cleaner thread in
logs_empty_and_mark_files_at_shutdown(). But there is possibility that
server could hang when server is running. So InnoDB should wake up page
cleaner thread periodically in srv_master_do_idle_tasks().
There can be multiple MLOG_CHECKPOINT record for the same checkpoint.
During recovery, InnoDB could encounter the previous MLOG_CHECKPOINT
for the checkpoint lsn. So the assertion
mlog_checkpoint_lsn <= recovered_lsn is wrong.
trx_update_mod_tables_timestamp(): When implementing
innodb_evict_tables_on_commit_debug, do not evict tables
on which transactional locks exist.
This debug variable was broken since its introduction in
commit 947b0b5722.
Fix stale virtual field value in 4 cases: when virtual field depends
on row_start/row_end in timestamp/trx_id versioned table. row_start
dep is recalculated in vers_update_fields() (SQL and InnoDB
layer). row_end dep is recalculated on history row insert.
make_versioned_helper() appended new update field unconditionally
while it should check if this field already exists in update vector.
Misc renames to conform versioning prefix. vers_update_fields() name
conforms with sql layer TABLE::vers_update_fields().
When InnoDB is extending a data file, it is updating the FSP_SIZE
field in the first page of the data file.
In commit 8451e09073 (MDEV-11556)
we removed a work-around for this bug and made recovery stricter,
by making it track changes to FSP_SIZE via redo log records, and
extend the data files before any changes are being applied to them.
It turns out that the function fsp_fill_free_list() is not crash-safe
with respect to this when it is initializing the change buffer bitmap
page (page 1, or generally, N*innodb_page_size+1). It uses a separate
mini-transaction that is committed (and will be written to the redo
log file) before the mini-transaction that actually extended the data
file. Hence, recovery can observe a reference to a page that is
beyond the current end of the data file.
fsp_fill_free_list(): Initialize the change buffer bitmap page in
the same mini-transaction.
The rest of the changes are fixing a bug that the use of the separate
mini-transaction was attempting to work around. Namely, we must ensure
that no other thread will access the change buffer bitmap page before
our mini-transaction has been committed and all page latches have been
released.
That is, for read-ahead as well as neighbour flushing, we must avoid
accessing pages that might not yet be durably part of the tablespace.
fil_space_t::committed_size: The size of the tablespace
as persisted by mtr_commit().
fil_space_t::max_page_number_for_io(): Limit the highest page
number for I/O batches to committed_size.
MTR_MEMO_SPACE_X_LOCK: Replaces MTR_MEMO_X_LOCK for fil_space_t::latch.
mtr_x_space_lock(): Replaces mtr_x_lock() for fil_space_t::latch.
mtr_memo_slot_release_func(): When releasing MTR_MEMO_SPACE_X_LOCK,
copy space->size to space->committed_size. In this way, read-ahead
or flushing will never be invoked on pages that do not yet exist
according to FSP_SIZE.
In commit 0f90728bc0 (MDEV-16809)
we introduced the configuration option innodb_log_optimize_ddl
for controlling whether native index creation or table-rebuild
in InnoDB should avoid writing full redo log.
Fungo Wang reported that this option is causing occasional failures.
The reason is that pages may be written to data files in an
inconsistent state. Applying log records to such inconsistent pages
may fail.
The solution is to always invoke PageBulk::finish() before page latches
may be released, to ensure that the page contents is in a consistent
state.
Something similar was implemented in MySQL 8.0.13:
mysql/mysql-server@d1254b9473
buf_block_t::skip_flush_check: Remove. Suppressing consistency checks
is a bad idea.
PageBulk::needs_finish(): New predicate: Determine whether
PageBulk::finish() must fix up the page.
PageBulk::init(): Clear PAGE_DIRECTION to ensure that needs_finish()
will hold. We change the field from PAGE_NO_DIRECTION to 0
and back without writing redo log. This trick avoids the need
to introduce any new data member to PageBulk.
PageBulk::insert(): Replace some high-level accessors to bypass
debug assertions related to PAGE_HEAP_TOP that we will be violating
until finish() has been executed.
PageBulk::finish(): Tolerate m_rec_no==0. We must invoke this also
on an empty page, to ensure that PAGE_HEAP_TOP is initialized.
PageBulk::commit(): Always invoke finish().
PageBulk::release(), BtrBulk::pageSplit(), BtrBulk::storeExt(),
BtrBulk::finish(): Invoke PageBulk::finish().
This issue was originally reported by Fungo Wang, along with a fix, as
MySQL Bug #98990.
His suggested fix was applied as part of
mysql/mysql-server@a003fc373d
and released in MySQL 5.7.31.
i_s_metrics_fill(): Add the missing call to Field::set_notnull(),
and simplify some code.
Problem:
=======
- Read operations are always allowed to hold a secondary index leaf
latch and then look up the corresponding clustered index record.
Flush table operation acquires secondary index latch while holding
a clustered index latch. It leads to deadlock violation.
Fix:
====
- Flush table operation should acquire secondary index before taking
clustered index to avoid deadlock violation with select operation.
commit 484931325e included a
workaround for a 10.5 merge issue that should now be properly
addressed in commit ab4069909d.
buf_chunk_init(): Remove an unnecessary MEM_MAKE_ADDRESSABLE().
We might invoke MEM_UNDEFINED() here, but actually the allocated
memory ought to be guaranteed to be zero-initialized.
commit 484931325e was a necessary
fix for the buffer pool resizing tests in 10.5 in
AddressSanitizer. However, that change would break the tests
innodb.innodb_buffer_pool_resize and
innodb.innodb_buffer_pool_resize_with_chunks
when run in MemorySanitizer, or presumably in Valgrind as well.
(Those tests run "forever" in Valgrind.)
buf_pool_resize(): Cancel the effect of MEM_NOACCESS() in Valgrind
and ASAN. In MSAN, MEM_NOACCESS() is a no-op, and hence we must do
nothing special here.
MEM_MAKE_ADDRESSABLE() would declare the memory contents undefined.
In this particular case, we must actually declare the contents
defined for Valgrind.
In AddressSanitizer, we only want memory poisoning to happen
in connection with custom memory allocation or freeing.
The primary use of MEM_UNDEFINED is for declaring memory uninitialized
in Valgrind or MemorySanitizer. We do not want MEM_UNDEFINED to
have the unwanted side effect that AddressSanitizer would no longer
be able to complain about accessing unallocated memory.
MEM_UNDEFINED(): Define as no-op for AddressSanitizer.
MEM_MAKE_ADDRESSABLE(): Define as MEM_UNDEFINED() or
ASAN_UNPOISON_MEMORY_REGION().
MEM_CHECK_ADDRESSABLE(): Wrap also __asan_region_is_poisoned().
- Some of the bug fixes are backports from 10.5!
- The fix in innobase/fil/fil0fil.cc is just a backport to get less
error messages in mysqld.1.err when running with valgrind.
- Renamed HAVE_valgrind_or_MSAN to HAVE_valgrind
MemorySanitizer (clang -fsanitize=memory) requires that all code
be compiled with instrumentation enabled. The only exception is the
C runtime library. Failure to use instrumented libraries will cause
bogus messages about memory being uninitialized.
In WITH_MSAN builds, we must avoid calling getservbyname(),
because even though it is a standard library function, it is
not instrumented, not even in clang 10.
Note: Before MariaDB Server 10.5, ./mtr will typically fail
due to the old PCRE library, which was updated in MDEV-14024.
The following cmake options were tested on 10.5
in commit 94d0bb4dbe:
cmake \
-DCMAKE_C_FLAGS='-march=native -O2' \
-DCMAKE_CXX_FLAGS='-stdlib=libc++ -march=native -O2' \
-DWITH_EMBEDDED_SERVER=OFF -DWITH_UNIT_TESTS=OFF -DCMAKE_BUILD_TYPE=Debug \
-DWITH_INNODB_{BZIP2,LZ4,LZMA,LZO,SNAPPY}=OFF \
-DPLUGIN_{ARCHIVE,TOKUDB,MROONGA,OQGRAPH,ROCKSDB,CONNECT,SPIDER}=NO \
-DWITH_SAFEMALLOC=OFF \
-DWITH_{ZLIB,SSL,PCRE}=bundled \
-DHAVE_LIBAIO_H=0 \
-DWITH_MSAN=ON
MEM_MAKE_DEFINED(): An alias for VALGRIND_MAKE_MEM_DEFINED()
and __msan_unpoison().
MEM_GET_VBITS(), MEM_SET_VBITS(): Aliases for
VALGRIND_GET_VBITS(), VALGRIND_SET_VBITS(), __msan_copy_shadow().
InnoDB: Replace the UNIV_MEM_ macros with corresponding MEM_ macros.
ut_crc32_8_hw(), ut_crc32_64_low_hw(): Use the compiler built-in
functions instead of inline assembler when building WITH_MSAN.
This will require at least -msse4.2 when building for IA-32 or AMD64.
The inline assembler would not be instrumented, and would thus cause
bogus failures.
This function is very common in a debug build. I can even see it in
profiler.
This patch reduces execution time of fil_validate() from
8948ns
8367ns
8650ns
8906ns
8448ns
to
260ns
232ns
403ns
275ns
169ns
in my environment.
The trick is a faster fil_space_t iteration. Hash table
is typically initialized with a size of 50,000. And looping through
it is slow. Slower, than iterating an exact amount of fil_space_t
which is typically less than ten.
Only debug builds are affected.
This issue is pretty much the same as MDEV-20213.
The fix is similar to:
3c238ac51c52c4abbff2
Check::validate(): fix a debug assertion
SysTablespace::open_or_create(): protect assigning to a shared
variable with a mutex
follow up
fil_system.sys_space is a shared variable between the thread
which assigns a value to it, and the thread which does Check::validate()
SysTablespace::open_or_create(): protect a shared variable with
a mutex to avoid any data race surprises.
Check::validate(): Relax a debug assertion. TRX_SYS_SPACE fil_space_t
can be created and became visible to this assertion before
fil_system.sys_space becomes initialized
Problem:
========
- InnoDB clears the fts resource when last FTS index is being dropped
if the table has user defined FTS_DOC_ID. While creating the new fts
index, InnoDB expects to have FTS resources.
Fix:
===
fts_drop_index(): Removed the fts resource clear.
fts_clear_all(): Clear the fts resource when there are no new fts
index to be added.
commit_cache_norebuild(), row_merge_drop_indexes():
Tries to call fts resource after removing associated fts index
from table object
When MDEV-22769 introduced srv_shutdown_state=SRV_SHUTDOWN_INITIATED in
commit efc70da5fd
we forgot to adjust a few checks for SRV_SHUTDOWN_NONE.
In the initial shutdown step, we are waiting for the background
DROP TABLE queue to be processed or discarded. At that time,
some background tasks (such as buffer pool resizing or dumping
or encryption key rotation) may be terminated, but others must
remain running normally.
srv_purge_coordinator_suspend(), srv_purge_coordinator_thread(),
srv_start_wait_for_purge_to_start(): Treat SRV_SHUTDOWN_NONE
and SRV_SHUTDOWN_INITIATED equally.
lock_check_trx_id_sanity(): Because the argument of UNIV_LIKELY
or __builtin_expect() can be less than sizeof(trx_id_t) on 32-bit
systems, it cannot reliably perform an implicit comparison to 0.
page_zip_fields_decode(): Do not dereference index=NULL.
Instead, return NULL early. The only caller does not care
about the values of output parameters in that case.
This bug was introduced in MySQL 5.7.6 by
mysql/mysql-server@9eae0edb7a
and in MariaDB 10.2.2 by
commit 2e814d4702.
Thanks to my son for pointing this out after investigating
the output of a static analysis tool.
The background drop table queue in InnoDB is a work-around for
cases where the SQL layer is requesting DDL on tables on which
transactional locks exist.
One such case are XA transactions. Our test case exploits the
fact that the recovery of XA PREPARE transactions will
only resurrect InnoDB table locks, but not MDL that should
block any concurrent DDL.
srv_shutdown_t: Introduce the srv_shutdown_state=SRV_SHUTDOWN_INITIATED
for the initial part of shutdown, to wait for the background drop
table queue to be emptied.
srv_shutdown_bg_undo_sources(): Assign
srv_shutdown_state=SRV_SHUTDOWN_INITIATED
before waiting for the background drop table queue to be emptied.
row_drop_tables_for_mysql_in_background(): On slow shutdown, if
no active transactions exist (excluding ones that are in
XA PREPARE state), skip any tables on which locks exist.
row_drop_table_for_mysql(): Do not unnecessarily attempt to
drop InnoDB persistent statistics for tables that have
already been added to the background drop table queue.
row_mysql_close(): Relax an assertion, and free all memory
even if innodb_force_recovery=2 would prevent the background
drop table queue from being emptied.
This race condition was introduced by
commit ad6171b91c (MDEV-22456).
In the observed case, two threads were executing
btr_search_drop_page_hash_index() on the same block,
to free a stale entry that was attached to a dropped index.
Both threads were holding an S latch on the block.
We must prevent the double-free of block->index by holding
block->lock in exclusive mode.
btr_search_guess_on_hash(): Do not invoke
btr_search_drop_page_hash_index(block) to get rid of
stale entries, because we are not necessarily holding
an exclusive block->lock here.
buf_defer_drop_ahi(): New function, to safely drop stale
entries in buf_page_mtr_lock(). We will skip the call to
btr_search_drop_page_hash_index(block) when only requesting
bufferfixing (no page latch), because in that case, we should
not be accessing the adaptive hash index, and we might get
a deadlock if we acquired the page latch.
We are supposed to commit and restart the mini-transaction
between records. There is no point to store and restore the
persistent cursor position otherwise.
If buf_page_optimistic_get() is patched to always fail, the
debug build would fail to start up due to trying to re-acquire
an already S-latched block.
This bug (which should not have visible impact to users, because
the code is only executed during startup, while no other threads
are accessing B-trees or causing pages to be evicted from the
buffer pool) was caught as part of a debugging effort for
something else.
The debugging approach was: Make buf_page_optimistic_get()
always return FALSE, and add ut_a(block->lock.lock_word == X_LOCK_DECR)
to both buf_LRU_get_free_only() and buf_LRU_block_free_non_file_page().
This would catch misuse of the buffer pool. If it were not for
buf_page_optimistic_get(), no buf_block_t::lock of any BUF_BLOCK_NOT_USED
block would ever be acquired.
Introduce a new ATTRIBUTE_NOINLINE to
ib::logger member functions, and add UNIV_UNLIKELY hints to callers.
Also, remove some crash reporting output. If needed, the
information will be available using debugging tools.
Furthermore, remove some fts_enable_diag_print output that included
indexed words in raw form. The code seemed to assume that words are
NUL-terminated byte strings. It is not clear whether a NUL terminator
is always guaranteed to be present. Also, UCS2 or UTF-16 strings would
typically contain many NUL bytes.
Problem:
========
During buffer pool resizing, InnoDB recreates the dictionary hash
tables. Dictionary hash table reuses the heap of AHI hash tables.
It leads to memory corruption.
Fix:
====
- While disabling AHI, free the heap and AHI hash tables. Recreate the
AHI hash tables and assign new heap when AHI is enabled.
- btr_blob_free() access invalid page if page was reallocated during
buffer poolresizing. So btr_blob_free() should get the page from
buf_pool instead of using existing block.
- btr_search_enabled and block->index should be checked after
acquiring the btr_search_sys latch
- Moved the buffer_pool_scan debug sync to earlier before accessing the
btr_search_sys latches to avoid the hang of truncate_purge_debug
test case
- srv_printf_innodb_monitor() should acquire btr_search_sys latches
before AHI hash tables.
commit f74023b955 (MDEV-15090)
inadvertently removed a mtr_t::commit() call from
trx_undo_report_rename(), causing an InnoDB hang if
we failed to log a RENAME operation.
It is unclear whether this condition is possible in practice.
The test case involved SET GLOBAL innodb_trx_rseg_n_slots_debug=1
and a failed CREATE TABLE...SELECT, whose error handling would
internally invoke RENAME in InnoDB.
Problem:
=======
While evicting the uncompressed page from buffer pool, InnoDB writes
the checksum for the compressed page in buf_LRU_free_page().
So while flushing the compressed page, checksum validation fails
when innodb_checksum_algorithm variable changed to strict_none.
Solution:
========
- Calculate the checksum only during flushing of page. Removed the
checksum write in buf_LRU_free_page().
namespace intrusive: removed
split class into two: ilist<T> and sized_ilist<T> which has a size field.
ilist<T> no more NULLify pointers to bring a slignly better performance.
As a consequence, fil_space_t::is_in_unflushed_spaces and
fil_space_t::is_in_rotation_list boolean members are needed now.
- During column reorder table rebuild, rollback of insert fails.
Reason is that InnoDB tries to fetch the column position from
new clustered index and it exceeds default column value tuple fields.
So InnoDB should use the table column position while searching for
defaults column value.
There was a race condition where the connection of the
victim of a KILL statement is disconnected while the
KILL statement is executing.
As a side effect of this fix, we will make XA PREPARE
transactions immune to KILL statements.
Starting with MariaDB 10.2, we have a pool of trx_t objects.
trx_free() would only free memory to the pool. We poison the
contents of freed objects in the pool in order to catch misuse.
trx_free(): Unpoison also trx->mysql_thd and trx->state.
This is to counter the poisoning of *trx in trx_pools->mem_free().
Unpoison only on AddressSanitizer or Valgrind, but not on MemorySanitizer.
Pool: Unpoison allocated objects only on AddressSanitizer or
Valgrind, but not on MemorySanitizer.
innobase_kill_query(): Properly protect trx, acquiring also
trx_sys_t::mutex and checking trx_t::mysql_thd and trx_t::state.
MDEV-21398 Deadlock (server hang) or assertion failure in
Diagnostics_area::set_error_status upon ALTER under lock
This failure could only happen if one locked the same table
multiple times and then did an ALTER TABLE on the table.
Major change is to change all instances of
table->m_needs_reopen= true;
to
table->mark_table_for_reopen();
The main fix for the problem was to ensure that we mark all
instances of the table in the locked_table_list and when we
reopen the tables, we first close all tables before reopening
and locking them.
Other things:
- Don't call thd->locked_tables_list.reopen_tables if there
are no tables marked for reopen. (performance)
ins_node_create_entry_list(): Create dummy empty tuples for
corrupted or incomplete indexes, to avoid dereferencing a NULL
dict_field_t::col pointer in row_build_index_entry_low().
This issue was found by a crash in the test gcol.innodb_virtual_basic
when merging the fix to 10.5.
prepare_inplace_alter_table_dict(): In the error handling, relax
a debug assertion for the case that we did not execute
dict_stats_wait_bg_to_stop_using_table() yet.
For no good reason, innodb_encryption_threads was limited to
4,294,967,295. Expectedly, the server would crash if such an
insane value was specified. Let us limit the maximum to 255.
The encryption threads are not doing much useful work.
They are basically only dirtying pages by performing
dummy writes via the redo log. The encryption key rotation
or the in-place addition or removal of encryption
will take place in the page cleaner.
In a quick test on a 20-core CPU (40 threads in total),
the sweet spot on an otherwise idle server seemed to be
innodb_encryption_threads=16 for the test
encryption.encrypt_and_grep. The new limit 255 should be
more than enough for even bigger servers.
This is a regression due to MDEV-16376
commit 8dc70c862b.
To make dict_index_t::detach_columns() idempotent,
we cleared dict_index_t::n_fields. But, this could
cause trouble with purge after a secondary index
creation failed (not even involving virtual columns).
A better way is to clear the dict_field_t::col pointers
that point to virtual columns that are being freed
due to aborting index creation on an index that depends
on a virtual column.
Note: the v_cols[] of an existing dict_table_t object will
never be modified. If any virtual columns are added or removed,
ha_innobase::commit_inplace_alter_table() would invoke
dict_table_remove_from_cache() and reload the table to dict_sys.
Index creation is a special case where the dict_index_t points
to virtual columns that do not yet exist in dict_table_t.
This is a regression due to the cleanup
commit 12f804acfa.
row_sel_open_pcur(): Remove the unnecessary parameter.
It suffices for us to acquire the adaptive hash index latch
only when btr_search_guess_on_hash() is called by
btr_cur_search_to_nth_level_func(), in
btr_pcur_open_with_no_init().
This code seems to be a relic from the times when there was
only one btr_search_latch, which was held in shared mode
for longer periods of time. Another relic of that era was
removed in commit e5980bf1b1.
This clean-up was missed when the btr_search_latch was split in
mysql/mysql-server/commit@ab17ab91ce18a47bb6c5c49e4dc0505ad488a448
(MySQL 5.7.8).
error is logged
The fix is to set flag in ib::error::~error() and check it in
mariabackup.
ib::error::error() is replaced with ib::warn::warn() in
AIO::linux_create_io_ctx() because of two reasons:
1) if we leave it as is, then mariabackup MTR tests will fail with --mem
option, because Linux AIO can not be used on tmpfs,
2) when Linux AIO can not be initialized, InnoDB falls back to simulated
AIO, so such sutiation is not fatal error, it should be treated as warning.
lock_table_locks_lookup(): Relax the assertion.
Locks must not exist while online secondary index creation is
in progress. However, if CREATE UNIQUE INDEX has not been committed
yet, but the index creation has been completed, concurrent DML
transactions may acquire record locks on the index. Furthermore,
such concurrent DML may cause duplicate key violation, causing
the DDL operation to be rolled back. After that, the online_status
may be ONLINE_INDEX_ABORTED or ONLINE_INDEX_ABORTED_DROPPED.
So, the debug assertion may only forbid the state ONLINE_INDEX_CREATION.
In commit ad6171b91c (MDEV-22456)
we removed the acquisition of the adaptive hash index latch
from the caller of btr_search_update_hash_ref().
The tests innodb.innodb_buffer_pool_resize_with_chunks
and innodb.innodb_buffer_pool_resize
would occasionally fail starting with 10.3,
due to MDEV-12288 causing more purge activity during the test.
btr_search_update_hash_ref(): After acquiring the adaptive hash index
latch, check that the adaptive hash index is still enabled on the page.
Problem:
=======
- During alter rebuild, document read from old table is tokenzied
parallelly by innodb_ft_sort_pll_degree threads and stores it
in respective merge files. While doing the parallel merge, InnoDB
wrongly skips the root level selection of merging buffer records.
So it leads to insertion of merge records in non-ascending order.
Solution:
==========
Build selection tree for the root level also. So that root of
selection tree can always contain sorted buffer.
The rw_lock_stats were incorrectly updated.
While global statistics have limited usefulness, we cannot
remove them from a GA version. This contribution is slightly
improving performance in write workloads.
If the InnoDB buffer pool contains many pages for a table or index
that is being dropped or rebuilt, and if many of such pages are
pointed to by the adaptive hash index, dropping the adaptive hash index
may consume a lot of time.
The time-consuming operation of dropping the adaptive hash index entries
is being executed while the InnoDB data dictionary cache dict_sys is
exclusively locked.
It is not actually necessary to drop all adaptive hash index entries
at the time a table or index is being dropped or rebuilt. We can let
the LRU replacement policy of the buffer pool take care of this gradually.
For this to work, we must detach the dict_table_t and dict_index_t
objects from the main dict_sys cache, and once the last
adaptive hash index entry for the detached table is removed
(when the garbage page is evicted from the buffer pool) we can free
the dict_table_t and dict_index_t object.
Related to this, in MDEV-16283, we made ALTER TABLE...DISCARD TABLESPACE
skip both the buffer pool eviction and the drop of the adaptive hash index.
We shifted the burden to ALTER TABLE...IMPORT TABLESPACE or DROP TABLE.
We can remove the eviction from DROP TABLE. We must retain the eviction
in the ALTER TABLE...IMPORT TABLESPACE code path, so that in case the
discarded table is being re-imported with the same tablespace identifier,
the fresh data from the imported tablespace will replace any stale pages
in the buffer pool.
rpl.rpl_failed_drop_tbl_binlog: Remove the test. DROP TABLE can
no longer be interrupted inside InnoDB.
fseg_free_page(), fseg_free_step(), fseg_free_step_not_header(),
fseg_free_page_low(), fseg_free_extent(): Remove the parameter
that specifies whether the adaptive hash index should be dropped.
btr_search_lazy_free(): Lazily free an index when the last
reference to it is dropped from the adaptive hash index.
buf_pool_clear_hash_index(): Declare static, and move to the
same compilation unit with the bulk of the adaptive hash index
code.
dict_index_t::clone(), dict_index_t::clone_if_needed():
Clone an index that is being rebuilt while adaptive hash index
entries exist. The original index will be inserted into
dict_table_t::freed_indexes and dict_index_t::set_freed()
will be called.
dict_index_t::set_freed(), dict_index_t::freed(): Note that
or check whether the index has been freed. We will use the
impossible page number 1 to denote this condition.
dict_index_t::n_ahi_pages(): Replaces btr_search_info_get_ref_count().
dict_index_t::detach_columns(): Move the assignment n_fields=0
to ha_innobase_inplace_ctx::clear_added_indexes().
We must have access to the columns when freeing the
adaptive hash index. Note: dict_table_t::v_cols[] will remain
valid. If virtual columns are dropped or added, the table
definition will be reloaded in ha_innobase::commit_inplace_alter_table().
buf_page_mtr_lock(): Drop a stale adaptive hash index if needed.
We will also reduce the number of btr_get_search_latch() calls
and enclose some more code inside #ifdef BTR_CUR_HASH_ADAPT
in order to benefit cmake -DWITH_INNODB_AHI=OFF.
On a checksum failure of a ROW_FORMAT=COMPRESSED page,
buf_LRU_free_one_page() would invoke buf_LRU_block_remove_hashed()
which will read the uncompressed page frame, although it would not
be initialized. With bad enough luck, fil_page_get_type(page)
could return an unrecognized value and cause the server to abort.
buf_page_io_complete(): On the corruption of a ROW_FORMAT=COMPRESSED
page, zerofill the uncompressed page frame.
This essentially reverts commit b393e2cb0c.
The leak might have been fixed, but because the
DEBUG_SYNC instrumentation for InnoDB purge threads was reverted
in 10.5 commit 5e62b6a5e0
as part of introducing a thread pool, it is easiest to revert
the entire change.
- There are multiple inconsistency and incorrect way in which rw-lock
stats are calculated.
- shared rw-lock stats:
"rounds" counter is incremented only once for N rounds done
in spin-cycle.
- all rw-lock stats:
If the spin-cycle is short-circuited then attempts are re-counted.
[If spin-cycle is interrupted, before it completes
srv_n_spin_wait_rounds (default 30) rounds, spin_count is incremented
to consider this. If thread resumes spin-cycle (due to unavailability
of the locks) and is again interrupted or completed, spin_count
is again incremented with the total count, failing to adjust the
previous attempt increment].
- s/x rw-lock stats:
spin_loop counter is not incremented at-all instead it is projected
as 0 (in show engine output) and division to calculate spin-round per
spin-loop is adjusted.
As per the original semantics spin_loop counter should be incremented
once per spin_loop execution.
- sx rw-lock stats:
sx locks increments spin_loop counter but instead of incrementing it
once for a spin_loop invocation it does it multiple times based on how
many time spin_loop flow is repeated for same instance post os-wait.
innobase_get_charset(), innobase_get_stmt_safe(): Remove.
It is more efficient and readable to invoke thd_charset()
and thd_query_safe() directly, without a non-inlined wrapper function.
As part of the SPATIAL INDEX implementation in InnoDB,
dict_index_t was expanded by a rtr_ssn_t field. There are only
3 operations for this field, all protected by rtr_ssn_t::mutex:
* btr_cur_search_to_nth_level() stores the least significant 32 bits
of the 64-bit value that is stored in the index root page.
(This would better be done when the table is opened for the
very first time.)
* rtr_get_new_ssn_id() increments the value by 1.
* rtr_get_current_ssn_id() reads the current value.
All these operations can be implemented equally safely by using
atomic memory access operations.
Let us limit the maximum value of the debug parameter
innodb_data_file_size to 256 MiB. It is only being used
in the test innodb.log_data_file_size, and the size
of the system tablespace should never exceed some 70 MiB
in ./mtr. Thus, 256 MiB should be a reasonable limit.
The fact that negative values that are passed to unsigned parameters
wrap around to the maximum value appears to be a regression due to
commit 18ef02b04d
and has been filed as bug MDEV-22219.
The test was incompatible with ./mtr --repeat=2 until
commit 2d6719d7ee
fixed that.
It turns out that the failing assertion that we disabled in
commit 3db94d2403
is bogus and can fail when the change buffer is emptied
during the last batch of crash recovery. The reason for this
is the condition around the page_create_empty() call in
page_cur_delete_rec(). The condition was removed in MariaDB
Server 10.5 as part of MDEV-12353, in
commit 7ae21b18a6 and
commit f8a9f90667.
The bug that the assertion aimed to catch is MDEV-22497, which
was fixed in commit 26aab96ecf.
The InnoDB insert buffer was upgraded in MySQL 5.5 into a change
buffer that also covers delete-mark and delete (purge) operations.
There is an important constraint for delete operations: a B-tree
leaf page must not become empty unless the entire tree becomes empty,
consisting of an empty root page. Because change buffer merges only
occur on a single leaf page at a time, delete operations must not be
buffered if it is possible that the last record of the page could be
deleted. (In that case, we would refuse to use the change buffer, and
if we really delete the last record, we would shrink the index tree.)
The function ibuf_get_volume_buffered_hash() is part of our insurance
that the page would not become empty. It is supposed to map each
buffered INSERT or DELETE_MARK record payload into a hash value.
We will only count each such record as a distinct key if there is no
hash collision. DELETE operations will always decrement the predicted
number fo records in the page.
Due to a bug in the function, we would actually compute the hash value
not only on the record payload, but also on some following bytes,
in case the record contains NULL values. In MySQL Bug #61104, we had
some examples of this dating back to 2012. But back then, we failed to
reproduce the bug, and in commit d84c95579b
we simply demoted the hard assertion to a message printout and a debug
assertion failure.
ibuf_get_volume_buffered_hash(): Correctly compute the hash value
of the payload bytes only. Note: we will consider
('foo','bar'),(NULL,'foobar'),('foob','ar') to be equal, but this
is not a problem, because in case of a hash collision, we could
also consider ('boo','far') to be equal, and underestimate the number
of records in the page, leading to refusing to buffer a DELETE.
This is a partial backport of
commit 5e7e7153b4 from 10.4.
assert_trx_is_free(): Assert !is_wsrep().
trx_init(): Do not initialize trx->wsrep, because it must have been
initialized already.
trx_commit_in_memory(): Invoke wsrep_commit_ordered(). This call
was being skipped, because the transaction object had already been
freed to the pool.
trx_rollback_for_mysql(), innobase_commit_low(),
innobase_rollback_trx(): Always reset trx->wsrep.
- Instant alter should change the metadata alone when table is
discarded. It shouldn't try to add metadata record in clustered index.
Also make the clustered index to non-instant format.
FOREIGN_KEY_CHECKS is disabled
- Referenced index can be null While renaming the referenced column name.
In that case, rename the referenced column name in dict_foreign_t and
find the equivalent referenced index.
Maybe this patch will help catch problems like buffer overflow.
log_t::first_in_use: removed
log_t::buf: this is where mtr_t are supposed to append data
log_t::flush_buf: this is from server writes to a file
Those two buffers are std::swap()ped when some thread is gonna write
to a file
During the UPDATE of PRIMARY KEY columns, we may miscalculate the
size of the clustered index record.
row_upd_clust_rec_by_insert(): Pass the total number of off-page columns,
which may include such columns that were inherited from the record
and not created as part of the UPDATE operation.
This is based on
mysql/mysql-server@490c45e8c8
which is a follow-up to
mysql/mysql-server@1fa475b85d
which we filed and fixed as MDEV-21511.
No test case was provided by Oracle.
Several MYSQL_SYSVAR_STR parameters that employ both a validate
function callback fail to copy the string for saving the
validated value. The affected variables include the following:
innodb_ft_aux_table
innodb_ft_server_stopword_table
innodb_ft_user_stopword_table
innodb_buffer_pool_filename
The test case is an enhanced version of
mysql/mysql-server@0b0c30641f
and the code changes are inspired by their fixes.
We are also importing and adjusting the test innodb_fts.stopword
to get coverage for the variable innodb_ft_user_stopword_table.
buf_dump(), buf_load(): Protect srv_buf_dump_filename with
LOCK_global_system_variables.
fts_load_user_stopword(): Minor cleanup
fts_load_stopword(): Remove the parameter global_stopword_table.
innobase_fts_load_stopword(): Protect innodb_server_stopword_table
against concurrent SET GLOBAL.
innodb_buffer_pool_evict_uncompressed(): Restart the loop when
prev_block might not enjoy mutex protection.
This is based on
mysql/mysql-server@eccaecac07
- Inplace alter shouldn't set default date column as '0000-00-00' when
table is not empty. So mysql_inplace_alter_table() copied
alter_ctx.error_if_not_empty to a new field of Alter_inplace_info.
In ha_innobase::check_if_supported_inplace_alter() should check the
error_if_not_empty flag and return INPLACE_NOT_SUPPORTED if the table
is not empty
dict_stats_update_if_needed(): Replace the parameter THD*
with const trx_t& so that trx_t::is_wsrep() can be invoked
instead of the more expensive wsrep_on().
Replace also other occurrences of wsrep_on() with trx_t::is_wsrep().
The function wsrep_on() was being called rather frequently
in InnoDB and XtraDB. Let us cache it in trx_t and invoke
trx_t::is_wsrep() instead.
innobase_trx_init(): Cache trx->wsrep = wsrep_on(thd).
ha_innobase::write_row(): Replace many repeated calls to current_thd,
and test the cheapest condition first.
row_vers_vc_matches_cluster(): Remove the parameter in_purge,
which was always passed as in_purge=true.
This parameter became constant in
mysql/mysql-server@1dec14d346
and it always was constant in MariaDB starting from the
introduction of the function in
commit 2e814d4702 (MariaDB 10.2.2).
row_prebuilt_free(): Do not attempt to drop orphan indexes
that might have been left behind by a failed ADD UNIQUE INDEX.
This avoids the execution of unwanted transactions during shutdown.
After MDEV-12353, the consistency check that I originally added for
commit 1b9fe0bbac
(InnoDB Plugin for MySQL 5.1) started randomly failing.
It turns out that the IMPORT TABLESPACE code was always incorrect:
it did not update the (redundantly stored) tablespace ID
in index tree root pages. It only does that for page headers
and BLOB pointers.
PageConverter::update_index_page(): Update the tablespace ID
in the BTR_SEG_TOP and BTR_SEG_LEAF of index root pages.
This is a backport of commit b8b3edff13.
was restored.
Optionally rollback prepared XA's on "mariabackup --prepare".
The fix MUST NOT be ported on 10.5+, as MDEV-742 fix solves the issue for
slaves.
Improve the test that was imported and adapted for MariaDB in
commit fb217449dc.
row_undo_step(): Move the DEBUG_SYNC point from trx_rollback_for_mysql().
This DEBUG_SYNC point is executed after rolling back one row.
trx_rollback_for_mysql(): Clarify the comments that describe the scenario,
and remove the DEBUG_SYNC point.
If the statement "if (trx->has_logged_persistent())" and its body are
removed from trx_rollback_for_mysql(), then the test
innodb.xa_recovery_debug will fail because the transaction would still
exist in the XA PREPARE state. If we allow the XA COMMIT statement
to succeed in the test, we would observe an incorrect state of the
XA transaction where the table would contain row (1,NULL). Depending
on whether the XA transaction was committed, the table should either
be empty or contain the record (1,1). The intermediate state of
(1,NULL) should never be observed after completed recovery.
If a table is altered using the MDEV-11369/MDEV-15562/MDEV-13134
ALGORITHM=INSTANT, it can force the table to use a non-canonical
format:
* A hidden metadata record at the start of the clustered index
is used to store each column's DEFAULT value. This makes it possible
to add new columns that have default values without rebuilding the table.
* Starting with MDEV-15562 in MariaDB Server 10.4, a BLOB in the
hidden metadata record is used to store column mappings. This makes
it possible to drop or reorder columns without rebuilding the table.
This also makes it possible to add columns to any position or drop
columns from any position in the table without rebuilding the table.
If a column is dropped without rebuilding the table, old records
will contain garbage in that column's former position, and new records
will be written with NULL values, empty strings, or dummy values.
This is generally not a problem. However, there may be cases where
users may want to avoid putting a table into this format.
For example, users may want to ensure that future UPDATE operations
after an ADD COLUMN will be performed in-place, to reduce write
amplification. (Instantly added columns are essentially always
variable-length.) Users might also want to avoid bugs similar to
MDEV-19916, or they may want to be able to export tables to
older versions of the server.
We will introduce the option innodb_instant_alter_column_allowed,
with the following values:
* never (0): Do not allow instant add/drop/reorder,
to maintain format compatibility with MariaDB 10.x and MySQL 5.x.
If the table (or partition) is not in the canonical format, then
any ALTER TABLE (even one that does not involve instant column
operations) will force a table rebuild.
* add_last (1, default in 10.3): Store a hidden metadata record that
allows columns to be appended to the table instantly (MDEV-11369).
In 10.4 or later, if the table (or partition) is not in this format,
then any ALTER TABLE (even one that does not involve column changes)
will force a table rebuild.
Starting with 10.4:
* add_drop_reorder (2, default): Like 'add_last', but allow the
metadata record to store a column map, to support instant
add/drop/reorder of columns (MDEV-15562).
partition does rebuild
- In ha_innobase::commit_inplace_alter_table() assumes that all partition
should do the same kind of alter operations. During DDL, if one partition
requires table rebuild and other partition doesn't need rebuild
then all partition should be forced to rebuild.
FOREIGN_KEY_CHECKS is disabled
- dict_foreign_find_index() can return NULL if InnoDB already dropped
the foreign index when FOREIGN_KEY_CHECKS is disabled.
redo log during recovery
- InnoDB unnecessarily reads the page even though it has fully initialized
buffered redo log records. Allow the page initialization redo log to
apply for the page in buf_page_get_gen() during recovery.
- Renamed buf_page_get_gen() to buf_page_get_low()
- Newly added buf_page_get_gen() will check for buffered redo log for
the particular page id during recovery
- Added new function buf_page_mtr_lock() which basically latches the page
for the given latch type.
- recv_recovery_create_page() is inline function which creates a page
if it has page initialization redo log records.
Making a linked list of dtuple_t is needed only for inserting
records. It's better to store tuples in a non-intrusive
container to not affect all other use cases of dtuple_t
dtuple_t::tuple_list: removed, it was 2 * sizeof(void*) bytes
ins_node_t::entry_list: now it's std::vector<dtuple_t*>
ins_node_t::entry: now it's std::vector<dtuple_t*>::iterator
DBUG_EXECUTE_IF("row_ins_skip_sec": this dead code removed
move span.h to a proper place to make it available for the whole server
Reformat it.
Constuctors from a contigous container are fixed
to use cont.data() instead of cont.begin()
span<>::index_type is replaced with span<>::size_type
In my micro-benchmarks memcmp(4196) 3 times faster than old
implementation. Also, it's generally better to use as less
reinterpret_casts<> as possible.
buf_is_zeroes(): renamed from buf_page_is_zeroes() and
argument changed to span<> for convenience.
st_::span<T>::const_iterator: fixed
page_zip-verify_checksum(): make argument byte* instead of void*
After MDEV-12353, the consistency check that I originally added for
commit 1b9fe0bbac
(InnoDB Plugin for MySQL 5.1) started randomly failing.
It turns out that the IMPORT TABLESPACE code was always incorrect:
it did not update the (redundantly stored) tablespace ID
in index tree root pages. It only does that for page headers
and BLOB pointers.
PageConverter::update_index_page(): Update the tablespace ID
in the BTR_SEG_TOP and BTR_SEG_LEAF of index root pages.
fil_delete_tablespace(): Remove the unused parameter drop_ahi,
and add the parameter if_exists=false. We want to suppress
error messages if we know that the tablespace has been discarded.
dict_table_rename_in_cache(): Pass the new parameter to
fil_delete_tablespace(), that is, do not complain about
missing tablespace if the tablespace has been discarded.
row_make_new_pathname(): Declare as static.
row_drop_table_for_mysql(): Tolerate !table->data_dir_path
when the tablespace has been discarded.
row_rename_table_for_mysql(): Skip part of the RENAME TABLE
when fil_space_get_first_path() returns NULL.
buf_pool_resize(): Simplify the fault injection
for innodb.buf_pool_resize_oom.
innodb.buf_pool_resize_oom: Use a small buffer pool.
innodb.innodb_buffer_pool_load_now: Make use of the sequence engine,
to avoid creating explicit InnoDB record locks. Clean up the
accesses to information_schema.innodb_buffer_page_lru.
- This issue is caused by MDEV-19176
(bba59abb03).
- Problem is that there is miscalculation of available memory during
recovery if innodb_buffer_pool_instances > 1.
- Ignore the buffer pool instance while calculating available_memory
- Removed recv_n_pool_free_frames variable and use buf_pool_get_n_pages()
instead.
Temporary tables are typically short-lived, and temporary tables
are assumed to be accessed only by the thread that is handling
the owning connection. Hence, they must not be subject to
defragmenting.
ha_innobase::optimize(): Do not add temporary tables to
the defragment_table() queue.
All tablespace metadata is buffered in fil_system. There is a LRU
mechanism, but that only controls the opening and closing of
fil_node_t::handle.
It is much more efficient and less error-prone to access data file names
by looking up the fil_space_t object rather than by essentially joining
each row with an access to SYS_DATAFILES via the InnoDB internal SQL parser.
dict_get_first_path(): Declare static. The function may only be needed
when loading or updating the data dictionary. Also, change a condition
in order to avoid a bogus GCC 10 -Wstringop-overflow warning for
mem_strdupl() about len==ULINT_UNDEFINED.
i_s_sys_tablespaces_fill_table(): Do not access other InnoDB internal
dictionary tables than SYS_TABLESPACES.
actually, page_zip_verify_checksum() generally allows all-zeroes
checksums because our CRC32 checksum is something like
crc_1 ^ crc_2 ^ crc_3
Also, all zeroes page is considered correct.
As a side effect fix nasty reinterpret_cast<> UB
Also, since c0f47a4a58 innodb_checksum_algorithm=full_crc32
exists which computes CRC32 in one go (without bitwise arithmetic)
- n_ext value may be less than dtuple_get_n_ext(dtuple) when PK is being
updated and new record inherits the externally stored fields from
delete mark old record.
Problem:
=======
After discarding the table, fts_optimize_thread aborts during shutdown.
InnoDB fails to remove the table from fts_optimize_wq and it leads to
the fts_optimize_thread to lookup for the auxiliary table and fails.
Fix:
====
While discarding the fts table, remove the table from fts_optimize_wq.
The column INFORMATION_SCHEMA.INNODB_MUTEXES.NAME is not populated ever since
commit 2e814d4702 applied the InnoDB changes from
MySQL 5.7.9 to MariaDB Server 10.2.2.
Since the same commit, the view is only providing information about
rw_lock_t, not any mutexes.
For now, let us convert the source code file name and line number of
the rw_lock_t creation into a name. A better option in the future might
be to store the information somewhere where it can be looked up by
mysql_pfs_key_t, and possibly to remove the CREATE_FILE and CREATE_LINE
columns.
fil_space_encrypt(): Remove the debug check that decrypts the
just encrypted page. We are exercising the decryption of encrypted
pages enough via --suite=encryption,mariabackup. It is a waste of
computing resources to decrypt every page immediately after encrypting it.
The redundant check had been added in
commit 2bedc3978b (MDEV-9931).
Problem:
=======
The problem is that InnoDB doesn't add the table in fts slots if drop table fails. InnoDB marks the table is in fts slots while processing sync message. So the consecutive alter statement assumes that table is in queue and tries to remove it. But InnoDB can't find the table in fts_slots.
Solution:
=========
i) Removal of in_queue in fts_t while processing the fts sync message.
ii) Add the table to fts_slots when drop table fails.
cmake -DWITH_INNODB_EXTRA_DEBUG:BOOL=ON
was broken ever since commit 8777458a6e
(MDEV-6076 Persistent AUTO_INCREMENT for InnoDB).
There is a race condition between page reads that call
page_zip_validate() (while holding clustered index root page S-latch)
and writes that update PAGE_ROOT_AUTO_INC
(with buf_block_t::lock SX-latch, compatible with S-latch).
page_zip_validate_low(): Skip the PAGE_ROOT_AUTO_INC field on
clustered index root pages in order to avoid false positives.
The only change is a change of the version number.
In MySQL 5.6.46, the copyright comments in a number of files were changed
in mysql/mysql-server@f1a006ece7
but there was no functional change to InnoDB code.
This was also reflected by XtraDB. We are not changing the copyright
comments in MariaDB Server for now.
Between MySQL 5.6.46 and 5.6.47, InnoDB was not changed at all.
Actually, we had forgotten to update the InnoDB version number to
5.6.46. With this change, we are updating InnoDB
from 5.6.45 to 5.6.47 and XtraDB from 5.6.45-86.1 to 5.6.46-86.2.
- Fixed a warning visible in optimized build related to calling
memcpy with length parameters larger than ptrdiff_t max.
rb#23333 approved by Annamalai Gurusami <annamalai.gurusami@oracle.com>
IndexPurge::next(): Replace btr_pcur_move_to_next_user_rec()
with some equivalent code that performs sanity checks without
killing the server. Perform some additional sanity checks as well.
This change is motivated by
mysql/mysql-server@48de4d74f4
which unnecessarily introduces storage overhead to btr_pcur_t
and uses a test case that injects a fault somewhere else,
not in the code path that was modified.
MySQL 5.7.29 includes the following fix:
Bug #30287668 INNODB: A LONG SEMAPHORE WAIT
mysql/mysql-server@5cdbb22b51
There is no test case. It seems that the problem could occur when
a spatial index is large and peculiar enough so that multiple R-tree
leaf pages will have the exactly same maximum bounding rectangle (MBR).
The commit message suggests that the hang can occur when R-tree
non-leaf pages are being merged, which should only be possible
during transaction rollback or the purge of transaction history,
when the R-tree index is at least 2 levels high and very many records
are being deleted. The message says that a comparison result that two
spatial index node pointer records are equal will cause an infinite loop
in rtr_page_copy_rec_list_end_no_locks(). Hence, we must include the
child page number in the comparison to be consistent with
mysql/mysql-server@2e11fe0e15.
We fix this bug in a simpler way, involving fewer code changes.
cmp_rec_rec(): Renamed from cmp_rec_rec_with_match().
Assert that rec2 always resides in an index page.
Treat non-leaf spatial index pages specially.
Now that we will be invoking dtuple_get_n_ext() instead of
letting btr_push_update_extern_fields() update an already
calculated value, it is unnecessary to calculate the n_ext
upfront.
row_rec_to_index_entry(), row_rec_to_index_entry_low():
Remove the output parameter n_ext.
During update, rollback, or MVCC read, we may miscalculate
the number of off-page columns, and thus the size of the
clustered index record. The function btr_push_update_extern_fields()
is mostly redundant, because the off-page columns would also be
moved by row_upd_index_replace_new_col_val(), which is invoked
via row_upd_index_replace_new_col_vals().
btr_push_update_extern_fields(): Remove.
This is based on
mysql/mysql-server@1fa475b85d
which refines a fix for a recovery bug fix
mysql/mysql-server@ce0a1e85e2
in MySQL 5.7.5.
No test case was provided by Oracle.
Some of the changed code is being covered by the existing test
innodb.blob-crash.
WL#6326 in MariaDB 10.2.2 introduced a potential hang on purge or rollback
when an index tree is being shrunk by multiple levels.
This fix is based on
mysql/mysql-server@f2c5852630
with the main difference that our version of the test case uses
DEBUG_SYNC instrumentation on ROLLBACK, not on purge.
btr_cur_will_modify_tree(): Simplify the check further.
This is the actual bug fix.
row_undo_mod_remove_clust_low(), row_undo_mod_clust(): Add DEBUG_SYNC
instrumentation for the test case.
The write-heavy test innodb_zip.wl6501_scale_1 timed out on
10.2 60d7011c5f for me.
Out of os_aio_n_segments=6, 5 are waiting for an event in
os_aio_simulated_handler(). One thread is waiting for a
write to complete in buf_dblwr_add_to_batch(), but that
would never happen, because nothing is waking up the simulated AIO
handler threads.
This hang appears to have been introduced in MySQL 5.6.12
in mysql/mysql-server@26cfde776c.