Let the purge_coordinator_task acquire purge_sys.rseg->latch
less frequently and hold it longer at a time. This may throttle
concurrent DML and prevent purge lag a little.
Remove an unnecessary std::this_thread::yield(), because the
trx_purge_attach_undo_recs() is supposed to terminate the scan
when running out of undo log records. Ultimately, this will
result in purge_coordinator_state::do_purge() and
purge_coordinator_callback() returning control to the thread pool.
Reviewed by: Vladislav Lesin and Vladislav Vaintroub
purge_sys_t::wake_if_not_active(): Replaces
srv_wake_purge_thread_if_not_active().
innodb_ddl_recovery_done(): Move the wakeup call to
srv_init_purge_tasks().
purge_coordinator_timer: Remove. The srv_master_callback() already
invokes purge_sys.wake_if_not_active() once per second.
Reviewed by: Vladislav Lesin and Vladislav Vaintroub
The motivation of introducing the parameter
innodb_purge_rseg_truncate_frequency in
mysql/mysql-server@28bbd66ea5 and
mysql/mysql-server@8fc2120fed
seems to have been to avoid stalls due to freeing undo log pages
or truncating undo log tablespaces. In MariaDB Server,
innodb_undo_log_truncate=ON should be a much lighter operation
than in MySQL, because it will not involve any log checkpoint.
Another source of performance stalls should be
trx_purge_truncate_rseg_history(), which is shrinking the history list
by freeing the undo log pages whose undo records have been purged.
To alleviate that, we will introduce a purge_truncation_task that will
offload this from the purge_coordinator_task. In that way, the next
innodb_purge_batch_size pages may be parsed and purged while the pages
from the previous batch are being freed and the history list being shrunk.
The processing of innodb_undo_log_truncate=ON will still remain the
responsibility of the purge_coordinator_task.
purge_coordinator_state::count: Remove. We will ignore
innodb_purge_rseg_truncate_frequency, and act as if it had been
set to 1 (the maximum shrinking frequency).
purge_coordinator_state::do_purge(): Invoke an asynchronous task
purge_truncation_callback() to free the undo log pages.
purge_sys_t::iterator::free_history(): Free those undo log pages
that have been processed. This used to be a part of
trx_purge_truncate_history().
purge_sys_t::clone_end_view(): Take a new value of purge_sys.head
as a parameter, so that it will be updated while holding exclusive
purge_sys.latch. This is needed for race-free access to the field
in purge_truncation_callback().
Reviewed by: Vladislav Lesin
UndorecApplier::assign_rec(): Remove. We will pass the undo record to
UndorecApplier::apply_undo_rec(). There is no need to copy the
undo record, because nothing else can write to the undo log pages
that belong to an active or incomplete transaction.
trx_t::apply_log(): Buffer-fix the undo page across mini-transaction
boundary in order to avoid repeated page lookups.
Reviewed by: Vladislav Lesin
purge_node_t, undo_node_t: Change the type of rec_type and cmpl_info
to byte, because this data is being extracted from a single byte.
UndoRecApplier: Change type and cmpl_info to be of type byte, and
move them next to the 16-bit offset field to minimize alignment bloat.
row_purge_parse_undo_rec(): Remove some redundant code. Purge will
be started by innodb_ddl_recovery_done(), at which point all
necessary subsystems will have been initialized.
trx_purge_rec_t::undo_rec: Point to const.
Reviewed by: Vladislav Lesin
undo_node_t::state: Replaced with bool is_temp.
row_undo_rec_get(): Do not copy the undo log record.
The motivation of the copying was to not hold latches on the undo pages
and therefore to avoid deadlocks due to lock order inversion a.k.a.
latching order violation: It is not allowed to wait for an index page latch
while holding an undo page latch, because MVCC reads would first acquire
an index page latch and then an undo page latch. But, in rollback, we
do not actually need any latch on our own undo pages. The transaction
that is being rolled back is the exclusive owner of its undo log records.
They cannot be overwritten by other threads until the rollback is complete.
Therefore, a buffer fix will protect the undo log record just fine,
by preventing page eviction. We still must initially acquire a shared latch
on each undo page, to avoid a race condition like the one that was fixed in
commit b102872ad5.
row_undo_ins_parse_undo_rec(): The first two bytes of the undo log record
now are the pointer to the next record within the page, not a length.
Reviewed by: Vladislav Lesin
After acquiring lock_sys.latch, always load trx->lock.wait_lock.
It could have changed by another thread that did lock_rec_move()
and released lock_sys.latch right before lock_sys.wr_lock_try()
succeeded.
This regression was introduced in
commit e039720bf3 (MDEV-32096).
Reviewed by: Vladislav Lesin
buf_page_free(): Flag the freed page as modified if it is found in
the buffer pool.
buf_flush_page(): If the page has been freed, ensure that the log
for it has been durably written, before removing the page
from buf_pool.flush_list.
FindBlockX: Find also MTR_MEMO_PAGE_X_MODIFY in order to avoid an
occasional failure of innodb.innodb_defrag_concurrent, which involves
freeing and reallocating pages in the same mini-transaction.
This fixes a regression that was introduced in
commit a35b4ae898 (MDEV-15528).
This logic was tested by commenting out the $shutdown_timeout line
from a test and running the following:
./mtr --rr innodb.scrub
rr replay var/log/mysqld.1.rr/mariadbd-0
A breakpoint in the modified buf_flush_page() was hit, and the
FIL_PAGE_LSN of that page had been last modified during the
mtr_t::commit() of a mini-transaction where buf_page_free()
had been executed on that page.
(Variant#3: Allow cross-charset comparisons, use a special
CHARSET_INFO to create lookup keys. Review input addressed.)
Equalities that compare utf8mb{3,4}_general_ci strings, like:
WHERE ... utf8mb3_key_col=utf8mb4_value (MB3-4-CMP)
can now be used to construct ref[const] access and also participate
in multiple-equalities.
This means that utf8mb3_key_col can be used for key-lookups when
compared with an utf8mb4 constant, field or expression using '=' or
'<=>' comparison operators.
This is controlled by optimizer_switch='cset_narrowing=on', which is
OFF by default.
IMPLEMENTATION
Item value comparison in (MB3-4-CMP) is done using utf8mb4_general_ci.
This is valid as any utf8mb3 value is also an utf8mb4 value.
When making index lookup value for utf8mb3_key_col, we do "Charset
Narrowing": characters that are in the Basic Multilingual Plane (=BMP) are
copied as-is, as they can be represented in utf8mb3. Characters that are
outside the BMP cannot be represented in utf8mb3 and are replaced
with U+FFFD, the "Replacement Character".
In utf8mb4_general_ci, the Replacement Character compares as equal to any
character that's not in BMP. Because of this, the constructed lookup value
will find all index records that would be considered equal by the original
condition (MB3-4-CMP).
Approved-by: Monty <monty@mariadb.org>
Problem was that JOIN_TAB::cleanup() was not run because
JOIN::top_join_tab_count was not set in case of early errors.
Fixed by setting JOIN::tab_join_tab_count when JOIN_TAB's are allocated.
Something that should eventually be fixed:
- Cleaning up JOIN_TAB's is now done in 3 different loops.
JOIN_TAB::cleanup() is only doing a partial cleanup. Other cleanups
are done outside of JOIN_TAB::cleanup().
The above should be fixed so that JOIN_TAB::cleanup() is freeing
everything related to it's own memory, including all its sub JOIN_ TAB's.
JOIN::cleanup() should only loop over all it's top JOIN_TAB's and call
JOIN_TAB::cleanup() on these.
This will greatly simplify and speedup the current code (as we now do some
cleanup's twice).
Other usage if persistent statistics is checking 'stats_is_read' in
caller, which is why this was not noticed earlier.
Other things:
- Simplified no_stat_values_provided
srv_all_undo_tablespaces_open(): While opening the extra unused
undo tablespaces, InnoDB should use ULINT_UNDEFINED instead of
SRV_SPACE_ID_UPPER_BOUND.
recv_recovery_from_checkpoint_start(): InnoDB should add the
redo log block header + trailer size while checking the log
sequence number in log file with log sequence number in the
system tablespace first page.
fil_aio_callback(): Invoke fil_node_t::complete_write() before
releasing any page latch, so that in case a log checkpoint is
executed roughly concurrently with the first write into a file
since the previous checkpoint, we will not miss a fdatasync()
or fsync() call to make the write durable.
In MemorySanitizer builds of 10.10 and 10.11, we would rather often
have the assertion fail in innodb_init() during mariadb-backup --prepare.
The assertion could also fail during InnoDB startup, but less often.
Before commit 685d958e38 in 10.8 the
log file cleanup after a successfully applied backup is different,
and the os_aio_pending_writes() assertion is in srv0start.cc.
IORequest::write_complete(): Invoke node->complete_write() before
releasing the page latch, so that a log checkpoint that is about to
execute concurrently will not miss a fdatasync() or fsync() on the
file, in case this was the first write since the last such call.
create_log_file(), srv_start(): Replace the debug assertion with
a debug check. For all intents and purposes, all writes could have
been completed but some write_io_callback() may not have invoked
io_slots::release() yet.
Problem:
========
- InnoDB fails to open undo tablespace when page0 is corrupted
and fails to throw error.
Solution:
=========
- InnoDB throws DB_CORRUPTION error when InnoDB encounters
page0 corruption of undo tablespace.
- InnoDB restores the page0 of undo tablespace from
doublewrite buffer if it encounters page corruption
- Moved Datafile::restore_from_doublewrite() to
recv_dblwr_t::restore_first_page(). So that undo
tablespace and system tablespace can use this function
instead of duplicating the code
srv_undo_tablespace_open(): Returns 0 if file doesn't exist
or ULINT_UNDEFINED if page0 is corrupted.
- InnoDB fails to check the overflow buffer while applying
the operation to the table that was rebuilt. This is caused
by commit 3cef4f8f0f (MDEV-515).
Fixed missing initialization of Alter_info()
This could cause crashes in some create table like scenarios
where some generated indexes where automatically dropped.
I also added a test that we do not try to drop from index_stats for
temporary tables.
The intentention was always to not create histograms for single value
unique keys (as histograms is not useful in this case), but because of
a bug in the code this was still done.
The changes in the test cases was mainly because hist_size is now NULL
for these kind of columns.
This reverts commit 03c9a4ef4a.
The fix is wrong. It was doing this: if the uninitialized
wait->m_class has some specific value, then don't initialize it.
The error is caused by MDEV-30165 fix with the following commit:
d13a57ae81
There is logical error in lock_release_on_prepare_try():
if (supremum_bit)
lock_rec_unlock_supremum(*cell, lock);
else
lock_rec_dequeue_from_page(lock, false);
Because there can be other bits set in the lock's bitmap, and the lock
type can be suitable for releasing criteria, but the above logic
releases only supremum bit of the lock.
The fix is to release lock if it suits for releasing criteria and unlock
supremum if supremum is locked otherwise.
Tere is also the test for the case, which was reported by QA team. I
placed it in a separate files, because it requires debug build.
Reviewed by: Marko Mäkelä
- InnoDB fails to report the error when encryption configuration
wasn't passed. This patch addresses the issue by adding
the error while loading the tablespace and deferring the
tablespace creation.
There are many filesystem related errors that can occur with
MariaBackup. These already outputed to stderr with a good description of
the error. Many of these are permission or resource (file descriptor)
limits where the assertion and resulting core crash doesn't offer
developers anything more than the log message. To the user, assertions
and core crashes come across as poor error handling.
As such we return an error and handle this all the way up the stack.
There where several reasons why the test failed:
- Constructors for Field_double and Field_float changed an argument
to the constructor instead of a the correct class variable.
- gcc 7.5.0 produced wrong code when inlining Field_double constructor
into Field_test_double constructor.
Fixed by changing the correct class variable and make the constructors
not inline to go around the gcc bug.
In commit 5ea5291 @sanja-byelkin for unknown reason switched the file mode
for 3 Galera tzinfo related test files from 644 -> 755. This exists only
from branch 10.6 onward:
$ git checkout 10.5
$ find mysql-test -executable -name *.test -or -executable -name *.result
(no results)
$ git checkout 10.6
$ find mysql-test -executable -name *.test -or -executable -name *.result
mysql-test/suite/galera/t/mysql_tzmysql-test/suite/galera/t/mysql_tzinfo_to_sql.test
mysql-test/suite/galera/t/mariadb_tzinfo_to_sql.test
mysql-test/suite/galera/r/mariadb_tzinfo_to_sql.resultinfo_to_sql.test
mysql-test/suite/galera/t/mariadb_tzinfo_to_sql.test
mysql-test/suite/galera/r/mariadb_tzinfo_to_sql.result
No test file nor test result file should be executable, so run chmod -x
on them.
All new code of the whole pull request, including one or several files
that are either new files or modified ones, are contributed under the
BSD-new license. I am contributing on behalf of my employer Amazon Web
Services, Inc.
The bug was in the test case.
The problem was that maria_empty_logs.inc deleted aria log files before
the server was properly shutdown.
Fixed by waiting for pid file to disappear before starting to delete log
files.
Other things:
- Fixed that translog_purge_at_flush() will not stop deleting files even
if one file could not be deleted.
The problem was that sometimes InnoDB returned sligtly wrong record count
for table, which causes the optimizer to disregard the result from
the range optimizer. The end result was that the optimizer choosed a
ref access instead of a range access which caused errors in buildbot.
Fixed by adding more rows to the table to ensure that table scan is
more costly than range scan of the given interval.
copy_back(): Also copy the dummy empty ib_logfile0 so that
MariaDB Server 10.8 or later can be started after
--copy-back or --move-back.
Thanks to Daniel Black for reporting this.
This is a 10.5 version of
commit ebf3649259
Add threadpool functionality to restrict concurrency during "batch"
periods (where tasks are added in rapid succession).
This will throttle thread creation more agressively than usual, while
keeping performance at least on-par.
One of these cases is bufferpool load, where async read IOs are executed
without any throttling. There can be as much as 650K read IOs for
loading 10GB buffer pool.
Another one is recovery, where "fake read" IOs are executed.
Why there are more threads than we expect?
Worker threads are not be recognized as idle, until they return to the
standby list, and to return to that list, they need to acquire
mutex currently held in the submit_task(). In those cases, submit_task()
has no worker to wake, and would create threads until default concurrency
level (2*ncpus) is satisfied. Only after that throttling would happen.
The problem was that we did not handle errors properly in
JOIN::get_best_combination. In case an early error, JOIN->join_tab would
contain unintialized values, which would cause errors on cleanup().
The error in question was reported earlier, but not noticed until later.
One cause of this is that most of the sql_select.cc code just checks
thd->fatal_error and not thd->is_error().
Fixed by changing of checks of fatal_error to is_error().
This allows a user to to change the default value of MAX_SEL_ARGS (16000)
in the rare case where they neeed more generated SEL_ARGS (as part of
the range optimizer)