When SUX_LOCK_GENERIC is defined, the srw_mutex, srw_lock, sux_lock are
implemented based on pthread_mutex_t and pthread_cond_t. This is the
only option for systems that lack a futex-like system call.
In the SUX_LOCK_GENERIC mode, if pthread_mutex_init() is allocating
some resources that need to be freed by pthread_mutex_destroy(),
a memory leak could occur when we are repeatedly invoking
pthread_mutex_init() without a pthread_mutex_destroy() in between.
pthread_mutex_wrapper::initialized: A debug field to track whether
pthread_mutex_init() has been invoked. This also helps find bugs
like the one that was fixed by
commit 1c8af2ae53 (MDEV-34422);
one simply needs to add -DSUX_LOCK_GENERIC to the CMAKE_CXX_FLAGS
to catch that particular bug on the initial server bootstrap.
buf_block_init(), buf_page_init_for_read(): Invoke block_lock::init()
because buf_page_t::init() will no longer do that.
buf_page_t::init(): Instead of invoking lock.init(), assert that it
has already been invoked (the lock is vacant).
add_fts_index(), build_fts_hidden_table(): Explicitly invoke
index_lock::init() in order to avoid a pthread_mutex_destroy()
invocation on an uninitialized object.
srw_lock_debug::destroy(): Invoke readers_lock.destroy().
trx_sys_t::create(): Invoke trx_rseg_t::init() on all rollback segments
in order to guarantee a deterministic state for shutdown, even if
InnoDB fails to start up.
trx_rseg_array_init(), trx_temp_rseg_create(), trx_rseg_create():
Invoke trx_rseg_t::destroy() before trx_rseg_t::init() in order to
balance pthread_mutex_init() and pthread_mutex_destroy() calls.
buf_page_get_gen(): Relax the assertion once more.
The LSN may grow by invoking ibuf_upgrade(), that is,
when upgrading files where innodb_change_buffering!=none was used.
The LSN may also have been recovered from a log that needs
to be upgraded to the current format.
int wsrep_thd_append_key(THD*, const wsrep_key*, int, Wsrep_service_key_type)
CREATE TABLE [SELECT|REPLACE SELECT] is CTAS and idea was that
we force ROW format. However, it was not correctly enforced
and keys were appended before wsrep transaction was started.
At THD::decide_logging_format we should force used stmt binlog
format to ROW in CTAS case and produce a warning if used
binlog format was not ROW.
At ha_innodb::update_row we should not append keys similarly
as in ha_innodb::write_row if sql_command is SQLCOM_CREATE_TABLE.
Improved error logging on ::write_row, ::update_row and ::delete_row
if wsrep key append fails.
Signed-off-by: Julius Goryavsky <julius.goryavsky@mariadb.com>
Assertion `table->field[0]->ptr >= table->record[0] &&
table->field[0]->ptr <= table->record[0] + table->s->reclength' failed in
handler::assert_icp_limitations.
table->move_fields has some limitations:
1. It cannot be used in cascade
2. It should always have a restoring pair.
Rule 1 is covered by assertions in handler::assert_icp_limitations
and handler::ptr_in_record (commit 30894fe9a9).
Rule 2 should be manually maintained with care. Hopefully, the rule 1 assertions
may sometimes help as well.
In ha_myisam::repair, both rules are broken. table->move_fields is used
asymmetrically there: it is set on every param->fix_record call
(i.e. in compute_vcols) but is restored only once, in the end of repair.
The reason to updating field ptr's for every call is that compute_vcols can
(supposedly) be called in parallel, that is, with the same table, but different
records.
The condition to "unmove" the pointers in ha_myisam::restore_vcos_after_repair
is incorrect, when stored vcols are available, and myisam stores a VIRTUAL field
if it's the only field in the table (the record cannot be of zero length).
This patch solves the problem by "unmoving" the pointers symmetrically, in
compute_vcols. That is, both rules will be preserved maintained.
Issue: During mtr_t:commit, if there is not enough space available in
redo log buffer, we flush the buffer. During flush, the LSN lock is
released allowing other concurrent mtr to commit. After flush we
reacquire the lock but use the old LSN obtained before check. It could
lead to redo log corruption. As the LSN moves backwards with the
possibility of data loss and unrecoverable server if the server aborts
for any reason or if server is shutdown with innodb_fast_shutdown=2.
With normal shutdown, recovery fails to map the checkpoint LSN to
correct offset.
In debug mode it hits log0log.cc:863: lsn_t log_t::write_buf()
Assertion `new_buf_free == ((lsn - first_lsn) & write_size_1)' failed.
In release mode, after normal shutdown, restart fails.
[ERROR] InnoDB: Missing FILE_CHECKPOINT(8416546) at 8416546
[ERROR] InnoDB: Log scan aborted at LSN 8416546
Backup fails reading the corrupt redo log.
[00] 2024-07-31 20:59:10 Retrying read of log at LSN=7334851
[00] FATAL ERROR: 2024-07-31 20:59:11 Was only able to copy log from
7334851 to 7334851, not 8416446; try increasing innodb_log_file_size
Unless a backup is tried or the server is shutdown or killed
immediately, the corrupt redo part is eventually truncated and there
may not be any visible issues seen in release mode.
This issue was introduced by the following commit.
commit a635c40648
MDEV-27774 Reduce scalability bottlenecks in mtr_t::commit()
Fix: If we need to release latch and flush redo before writing mtr
logs, make sure to get the latest system LSN after reacquiring the
redo system latch.
Reason:
======
- InnoDB fails to load the instant alter table metadata from
clustered index while loading the table definition.
The reason is that InnoDB metadata blob has the column length
exceeds maximum fixed length column size.
Fix:
===
- InnoDB should treat the long fixed length column as variable
length fields that needs external storage while initializing
the field map for instant alter operation
Problem:
========
- After the commit ada1074bb1 (MDEV-14398)
fil_crypt_set_encrypt_tables() iterates through all tablespaces to
fill the default_encrypt tables list. This was a trigger to
encrypt or decrypt when key rotation age is set to 0. But import
tablespace does call fil_crypt_set_encrypt_tables() unnecessarily.
The motivation for the call is to signal the encryption threads.
Fix:
====
ha_innobase::discard_or_import_tablespace: Remove the
fil_crypt_set_encrypt_tables() and add the import tablespace
to the default encrypt list if necessary
During InnoDB root page split, InnoDB does the following
1) First move the root records to the new page(p1)
2) Empty the root, insert the node pointer to the root page
3) Split the new page and make it as child nodes.
4) Finds the split record, allocate another new page(p2)
to the index
5) InnoDB stores the record(ret) predecessor to the supremum
record of the page (p2).
6) In page_copy_rec_list_start(), move the records from p1 to p2
upto the split record
6) Given table is a compressed row format page, InnoDB attempts to
compress the page p2 and failed (due to innodb_compression_level = 0)
7) Since the compression fails, InnoDB gets the number of preceding
records(ret_pos) of a record (ret) on the page (p2)
8) Page (p2) is a new page, ret points to infimum record.
ret_pos can be 0. InnoDB have wrong condition that ret_pos shouldn't
be 0 and returns corruption. InnoDB has similar wrong check in
page_copy_rec_list_end()
In commit bf0b82d24b (MDEV-33515)
the function log_t::init_lsn_lock() was removed. This was fine on
those platforms where InnoDB uses futex-based mutexes (Linux, FreeBSD,
OpenBSD, NetBSD, DragonflyBSD).
Dave Gosselin debugged this on Apple macOS and submitted a fix where
pthread_mutex_wrapper::pthread_mutex_wrapper() would invoke init().
We do not really need that; we only need to invoke lsn_lock.init()
like we used to do before commit bf0b82d24b.
This should be a no-op for the futex based mutexes, which intentionally
rely on zero initialization.
The missing pthread_mutex_init() call would cause race conditions
and corruption of log_sys.buf because multiple threads could
apparently hold log_sys.lsn_lock concurrently in
log_t::append_prepare(). The error would be caught by a debug
assertion in log_t::write_buf(), or in non-debug builds by the
fact that the server cannot be restarted due to an apparently
missing FILE_CHECKPOINT record (because it had been written
to wrong offset in log_sys.buf).
The failure in log_t::append_prepare() was caught on Microsoft Windows
after enabling SUX_LOCK_GENERIC and therefore forcing the use of
pthread_mutex_wrapper for the log_sys.lsn_lock. It appears to be fine
to omit the pthread_mutex_init() call on GNU/Linux.
log_t::create(): Invoke lsn_lock.init().
log_t::close(): Invoke lsn_lock.destroy().
To better catch this kind of issues in the future by simply defining
SUX_LOCK_GENERIC on any platform, a separate debug instrumentation patch
will be applied to the 10.6 branch later.
Reviewed by: Debarun Banerjee
- commit 85db534731 (MDEV-33400)
retains the instantness in the table definition after discard
tablespace. So there is no need to assign n_core_null_bytes
during instant table preparation unless they are not
initialized.
- During copy algorithm, InnoDB should use bulk insert operation
for row by row insert operation. By doing this, copy algorithm
can effectively build indexes. This optimization is disabled
for temporary table, versioning table and table which has
foreign key relation.
Introduced the variable innodb_alter_copy_bulk to allow
the bulk insert operation for copy alter operation
inside InnoDB. This is enabled by default
ha_innobase::extra(): HA_EXTRA_END_ALTER_COPY mode tries to apply
the buffered bulk insert operation, updates the non-persistent
table stats.
row_merge_bulk_t::write_to_index(): Update stat_n_rows after
applying the bulk insert operation
row_ins_clust_index_entry_low(): In case of copy algorithm,
switch to bulk insert operation.
copy_data_error_ignore(): Handles the error while copying
the data from source to target file.
(With trivial fixes by sergey@mariadb.com)
Added option fix_innodb_cardinality to optimizer_adjust_secondary_key_costs
Using fix_innodb_cardinality disables the 'divide by 2' of rec_per_key_int
in InnoDB that in effect doubles the Cardinality for secondary keys.
This has the biggest effect for indexes where a few rows has the same key
value. Using this may also cause table scans for very small tables (which
in some cases may be better than an index scan).
The user visible effect is that 'SHOW INDEX FROM table_name' will for
InnoDB show the true Cardinality (and not 2x the real value). It will
also allow the optimizer to chose a better index in some cases as the
division by 2 could have a bad effect for tables with 2-5 identical values
per key.
A few notes about using fix_innodb_cardinality:
- It has direct affect for SHOW INDEX FROM table_name. SHOW INDEX
will also update the statistics in table share.
- The effect of fix_innodb_cardinality for query plans or EXPLAIN
is only visible after first open of the table. This is why one must
do a flush tables or use SHOW INDEX for the option to take effect.
- Using fix_innodb_cardinality can thus affect all user in their query
plans if they are using the same tables.
Because of this, it is strongly recommended that one uses
optimizer_adjust_secondary_key_costs=fix_innodb_cardinality mainly
in configuration files to not cause issues for other users.
We need to work around deficiencies of Valgrind, and apparently
the previous work-around attempts
(such as d247d64988) do not work
anymore, definitely not on recent clang-based compilers.
MemorySanitizer should be fine; unfortunately we set HAVE_valgrind for it
as well.
In commit 2f6df93748
we fixed an observed case of the bug by removing
some code related to the no longer needed
BTR_MODIFY_PREV mode.
In commit 73ad436e16
an alternative fix was applied that also fixes the
BTR_SEARCH_PREV case.
Let us clean up some implicit references to BTR_MODIFY_PREV
that were missed in 2f6df93748.
btr_pcur_move_backward_from_page(): Assume that the latch mode was
BTR_SEARCH_LEAF.
btr_pcur_move_to_prev(): Assert that the latch mode is BTR_SEARCH_LEAF.
This function is mostly invoked in row0sel.cc for read operations,
as well as in row0merge.cc for reading from the clustered index.
All callers indeed use a cursor in the BTR_SEARCH_LEAF mode.
Item_exists_subselect::fix_length_and_dec() sets explicit_limit to 1.
In the exists2in transformation it resets select_limit to NULL. For
consistency we should reset explicity_limit too.
This fixes a bug where spider table returns wrong results for queries
that gets through exists2in transformation when semijoin is off.
Distinguish them in two place:
when constructing connection key in create_conn_key and
spider_create_conn for both ordinary queries and spider_direct_sql
For spider_create_conn and spider_udf_direct_sql_create_conn, the
created conn gets assigned a field from the source object if and only
if source->field is non-null.
For spider_create_conn_keys and spider_udf_direct_sql_create_conn_key,
we update the encoding so that absence of keyword and keyword with an
empty value result in different keys. More specifically, if the i-th
keyword has a value, the corresponding part in the conn key begins
with the char \i, followed by the possibly empty value and ends with a
\0. If the i-th keyword is not specified, then it does not get a
mention in the conn key.
As part of this change, we also update table param / option parsing to
recognise a singleton empty string when creating an string list,
instead of writing it off as NULL.
ENGINE_SUBSTITUTION only applies to CREATE TABLE and ALTER TABLE, and
Storage_engine_name::resolve_storage_engine_with_error() could be
called when executing any sql command.
SPIDER_CONN::loop_check_meraged_first is useless, because all
SPIDER_CONN_LOOP_CHECKs are in SPIDER_CONN::loop_check_queue, which in
spider_db_conn::fin_loop_check() is iterated over.
This fixes the use-after-free issue when there are three spider tables
sharing the same remote, and their corresponding
SPIDER_CONN_LOOP_CHECKs getting merged in
spider_conn_queue_and_merge_loop_check()
This also fixes MDEV-34555
Same cause as MDEV-31996. This prevents reuse of conns freed
previously, e.g. during the commit of previous statement.
It does not occur in 10.4 because of the commit for MDEV-19002 removed
the call to spider_check_trx_and_get_conn().
This allows creation of SPIDER_CONN on demand, if the previous one was
freed.
Also, the first_link_idx's are reset during
spider_check_trx_and_get_conn(), which in the case of remote HANDLER
commands, might not match the link to use correct first_link_idx for
remote HANDLER statement that was later set in
ha_spider::rnd_handler_init() (causing testing regressions in the
spider/handler suite). Therefore we fix the first_link_idx there.
A SPIDER_TRX_HA associated with a SPIDER_TRX could have longer
lifetime than its associated SPIDER_SHARE. And it is identified with
the associated table name. When the SPIDER_SHARE no longer valid, e.g.
when the associated spider table has been dropped and recreated, the
SPIDER_TRX_HA should be reset too.
Since spider could create a new SPIDER_SHARE with the exact same
address of a freed SPIDER_SHARE, we try to mark all SPIDER_TRX_HAs
associated with a SPIDER_SHARE invalid when the SPIDER_SHARE is about
to be freed.
- During XA PREPARE, InnoDB releases the non-exclusive locks.
But it fails to remove the non-exclusive table lock from the
transaction table locks. In the mean time, main thread evicts
the table from the LRU cache. While rollbacking the XA transaction,
InnoDB iterates through the table locks to check whether it
holds lock on any system tables and wrongly assumes the
evicted table as system table since the table id is 0
Fix:
===
During XA PREPARE, remove the table locks of the transaction while
releasing the non-exclusive locks.
Problem:
========
- During shutdown, InnoDB tries to free the asynchronous
I/O slots and hangs. The reason is that InnoDB disables
asynchronous I/O before waiting for pending
asynchronous I/O to finish.
buf_load(): InnoDB aborts the buffer pool load due to
user requested shutdown and doesn't wait for the asynchronous
read to get completed. This could lead to debug assertion
in buf_flush_buffer_pool() during shutdown
Fix:
===
os_aio_free(): Should wait all read_slots and write_slots
to finish before disabling the aio.
buf_load(): Should wait for pending read request to complete
even though it was aborted.