Mutex order violation when wsrep bf thread kills a conflicting trx,
the stack is
wsrep_thd_LOCK()
wsrep_kill_victim()
lock_rec_other_has_conflicting()
lock_clust_rec_read_check_and_lock()
row_search_mvcc()
ha_innobase::index_read()
ha_innobase::rnd_pos()
handler::ha_rnd_pos()
handler::rnd_pos_by_record()
handler::ha_rnd_pos_by_record()
Rows_log_event::find_row()
Update_rows_log_event::do_exec_row()
Rows_log_event::do_apply_event()
Log_event::apply_event()
wsrep_apply_events()
and mutexes are taken in the order
lock_sys->mutex -> victim_trx->mutex -> victim_thread->LOCK_thd_data
When a normal KILL statement is executed, the stack is
innobase_kill_query()
kill_handlerton()
plugin_foreach_with_mask()
ha_kill_query()
THD::awake()
kill_one_thread()
and mutexes are
victim_thread->LOCK_thd_data -> lock_sys->mutex -> victim_trx->mutex
This patch is the plan D variant for fixing potetial mutex locking
order exercised by BF aborting and KILL command execution.
In this approach, KILL command is replicated as TOI operation.
This guarantees total isolation for the KILL command execution
in the first node: there is no concurrent replication applying
and no concurrent DDL executing. Therefore there is no risk of
BF aborting to happen in parallel with KILL command execution
either. Potential mutex deadlocks between the different mutex
access paths with KILL command execution and BF aborting cannot
therefore happen.
TOI replication is used, in this approach, purely as means
to provide isolated KILL command execution in the first node.
KILL command should not (and must not) be applied in secondary
nodes. In this patch, we make this sure by skipping KILL
execution in secondary nodes, in applying phase, where we
bail out if applier thread is trying to execute KILL command.
This is effective, but skipping the applying of KILL command
could happen much earlier as well.
This also fixed unprotected calls to wsrep_thd_abort
that will use wsrep_abort_transaction. This is fixed
by holding THD::LOCK_thd_data while we abort transaction.
Reviewed-by: Jan Lindström <jan.lindstrom@mariadb.com>
In a rebase of the merge, two preceding commits were accidentally reverted:
commit 112b23969a (MDEV-26308)
commit ac2857a5fb (MDEV-25717)
Thanks to Daniele Sciascia for noticing this.
Victim threads which are in currently in process of aborting or already
aborted should be skipped for another kill process.
Reviewed-by: Jan Lindström <jan.lindstrom@mariadb.com>
Incorrect processing of an auto-incrementing field in the
WSREP-related code during applying transactions results in
a duplicate key being created. This is due to the fact that
at the beginning of the write_row() and update_row() functions,
the values of the auto-increment parameters are used, which
are read from the parameters of the current thread, but further
along the code other values are used, which are read from global
variables (when applying a transaction). This can happen when
the cluster configuration has changed while applying a transaction
(for example in the high_priority_service mode for Galera 4).
Further during IST processing duplicating key is detected, and
processing of the DB_DUPLICATE_KEY return code (inside innodb,
in the write_row() handler) results in a call to the
wsrep_thd_self_abort() function.
wsrep_cluster_address_update() causes LOCK_wsrep_slave_threads
to be locked under LOCK_wsrep_cluster_config, while normally
the order should be the opposite.
Fix: don't protect @@wsrep_cluster_address value with the
LOCK_wsrep_cluster_config, LOCK_global_system_variables is enough.
Only protect wsrep reinitialization with the LOCK_wsrep_cluster_config.
And make it use a local copy of the global @@wsrep_cluster_address.
Also, introduce a helper function that checks whether
wsrep_cluster_address is set and also asserts that it can be safely
read by the caller.
Some DML operations on tables having unique secondary keys cause scanning
in the secondary index, for instance to find potential unique key violations
in the seconday index. This scanning may involve GAP locking in the index.
As this locking happens also when applying replication events in high priority
applier threads, there is a probabality for lock conflicts between two wsrep
high priority threads.
This PR avoids lock conflicts of high priority wsrep threads, which do
secondary index scanning e.g. for duplicate key detection.
The actual fix is the patch in sql_class.cc:thd_need_ordering_with(), where
we allow relaxed GAP locking protocol between wsrep high priority threads.
wsrep high priority threads (replication appliers, replayers and TOI processors)
are ordered by the replication provider, and they will not need serializability
support gained by secondary index GAP locks.
PR contains also a mtr test, which exercises a scenario where two replication
applier threads have a false positive conflict in GAP of unique secondary index.
The conflicting local committing transaction has to replay, and the test verifies
also that the replaying phase will not conflict with the latter repllication applier.
Commit also contains new test scenario for galera.galera_UK_conflict.test,
where replayer starts applying after a slave applier thread, with later seqno,
has advanced to commit phase. The applier and replayer have false positive GAP
lock conflict on secondary unique index, and replayer should ignore this.
This test scenario caused crash with earlier version in this PR, and to fix this,
the secondary index uniquenes checking has been relaxed even further.
Now innodb trx_t structure has new member: bool wsrep_UK_scan, which is set to
true, when high priority thread is performing unique secondary index scanning.
The member trx_t::wsrep_UK_scan is defined inside WITH_WSREP directive, to make
it possible to prepare a MariaDB build where this additional trx_t member is
not present and is not used in the code base. trx->wsrep_UK_scan is set to true
only for the duration of function call for: lock_rec_lock() trx->wsrep_UK_scan
is used only in lock_rec_has_to_wait() function to relax the need to wait if
wsrep_UK_scan is set and conflicting transaction is also high priority.
Reviewed-by: Jan Lindström <jan.lindstrom@mariadb.com>
If log_slave_updates==OFF, wsrep applier threads used to be configured
with option: thd->variables.option_bits&= ~(OPTION_BIN_LOG);
(i.e. like sql_log_bin=ON). And this was regardless of log-bin configuration.
With this, having configuration of: --log-bin && --log-slave-updates=OFF,
local threads used binlogging, but applier threads did not. And further:
local threads went through binlog group commit, while applier threads did
direct commits. This resulted in situation, where applier threads entered
earlier in wsrep XID checkpointing, and could sync their wsrep XID out of order.
Later local thread commit would see that higher seqno was already checkpointed,
and fire an assert because of this.
As a fix, applier threads are now forced to enable binlogging regardless of
log-slave-updates configuration.
This PR comes with new mtr test: galera.MDEV-24327, which causes a scenario
where applier transaction is applied and committed while earlier local transaction
is parked before commit order monitor enter. A buggy mariadb versoin would fail
for assertion because of wsrep XID checkpoint order violation.
Reviewed-by: Jan Lindström <jan.lindstrom@mariadb.com>
The reason for the failure is that
thd->mdl_context.release_transactional_locks()
was called after commit & rollback even in cases where the current
transaction is still active.
For 10.2, 10.3 and 10.4 the fix is simple:
- Replace all calls to thd->mdl_context.release_transactional_locks() with
thd->release_transactional_locks(). The thd function will only call
the mdl_context function if there are no active transactional locks.
In 10.6 we will better fix where we will change the return value for
some trans_xxx() functions to indicate if transaction did close the
transaction or not. This will avoid the need of the indirect call.
Other things:
- trans_xa_commit() and trans_xa_rollback() will automatically
call release_transactional_locks() if the transaction is closed.
- We can't do that for the other functions as the caller of many of these
are doing additional work (like close_thread_tables) before calling
release_transactional_locks().
- Added missing abort_result_set() and missing DBUG_RETURN in
select_create::send_eof()
- Fixed wrong indentation in injector::transaction::commit()
Remove incorrect BF (brute force) handling from lock_rec_has_to_wait_in_queue
and move condition to correct callers. Add a function to report
BF lock waits and assert if incorrect BF-BF lock wait happens.
wsrep_report_bf_lock_wait
Add a new function to report BF lock wait.
wsrep_assert_no_bf_bf_wait
Add a new function to check do we have a
BF-BF wait and if we have report this case
and assert as it is a bug.
lock_rec_has_to_wait
Use new wsrep_assert_bf_wait to check BF-BF wait.
lock_rec_create_low
lock_table_create
Use new function to report BF lock waits.
lock_rec_insert_by_trx_age
lock_grant_and_move_on_page
lock_grant_and_move_on_rec
Assert that trx is not Galera as VATS is not compatible
with Galera.
lock_rec_add_to_queue
If there is conflicting lock in a queue make sure that
transaction is BF.
lock_rec_has_to_wait_in_queue
Remove incorrect BF handling. If there is conflicting
locks in a queue all transactions must wait.
lock_rec_dequeue_from_page
lock_rec_unlock
If there is conflicting lock make sure it is not
BF-BF case.
lock_rec_queue_validate
Add Galera record locking rules comment and use
new function to report BF lock waits.
All attempts to reproduce the original assertion have been
failed. Therefore, there is no test case on this commit.
Backported the support for aborting and replaying stored procedure and fix for trigger
key assigments from 10.4 version.
Backported also two mtr tests: wsrep_sp_bf_abort and MDEV-20225
If async replication slave thread conflicts with cluster replication,
then the async slave transaction should be BF aborted, and depending on the
state of async slave transaction execution, potentially also replayed.
There were problems in such BF abort implementation and the replaying was not
started.
This pull request contains fixes which make sure that if async slave thread is
marked to abort and replay, it will complete carry out the rollback and
release all locks and resources before starting the replaying. After replaying,
async slave transactions is treated as successful, so the slave thread will
continue as usual, handling next replication event.
There is also new mtr test: galera.galera_slave_replay, which stresses both a
certification failure for async slave thread and a successful BF abort
followed by replaying.
Test galera_sr_ddl_master would sometimes fail due to leftover
streaming replication fragments. Rollbacker thread would attempt to
open streaming_log table to remove the fragments, but would fail in
check_stack_overrun(). Ultimately the check_stack_overrun() failure
was caused by rollbacker missing to switch the victim's THD thread
stack to rollbacker's thread stack.
Also in this patch:
- Remove duplicate functionality in rollbacker helper functions,
and extract rollbacker fragment removal into function
wsrep_remove_streaming_fragments()
- Reuse open_for_write() in wsrep_schema::remove_fragments
- Partially revert changes to galera_sr_ddl_master test from
commit 44a11a7c08. Removed unnecessary
wait condition and isolation level setting
This patch contains two fixes:
* wsrep_handle_mdl_conflict(): handle the case where SR transaction
is in aborting state. Previously, a BF-BF conflict was reported, and
the process would abort.
* wsrep_thd_bf_abort(): do not restore thread vars after calling
wsrep_bf_abort(). Thread vars are already restored in wsrep-lib if
necessary. This also removes the assumption that the caller of
wsrep_thd_bf_abort() is the given bf_thd, which is not the case.
Also in this patch:
* Remove unnecessary check for active victim transaction in
wsrep_thd_bf_abort(): the exact same check is performed later in
wsrep_bf_abort().
* Make wsrep_thd_bf_abort() and wsrep_log_thd() const-correct.
* Change signature of wsrep_abort_thd() to take THD pointers instead
of void pointers.
For release builds, do not declare unused variables.
unpack_row(): Omit a debug-only variable from WSREP diagnostic message.
create_wsrep_THD(): Fix -Wmaybe-uninitialized for the PSI_thread_key.
MariaDB 10.4 was crashing when thread-handling was set to
pool-of-threads and wsrep was enabled.
There were two apparent reasons for the crash:
- Connection handling in threadpool_common.cc was missing calls to
control wsrep client state.
- Thread specific storage which contains thread variables (THR_KEY_mysys)
was not handled appropriately by wsrep patch when pool-of-threads
was configured.
This patch addresses the above issues in the following way:
- Wsrep client state open/close was moved in thd_prepare_connection() and
end_connection() to have common handling for one-thread-per-connection
and pool-of-threads.
- Thread local storage handling in wsrep patch was reworked by introducing
set of wsrep_xxx_threadvars() calls which replace calls to
THD store_globals()/reset_globals() and deal with thread handling
specifics internally.
Wsrep-lib was updated to version which relaxes internal concurrency
related sanity checks.
Rollback code from wsrep_rollback_process() was extracted to separate calls
for better readability.
Post rollback thread was removed as it was completely unused.
The MDEV-20265 commit e746f451d5
introduces DBUG_ASSERT(right_op == r_tbl) in
st_select_lex::add_cross_joined_table(), and that assertion would
fail in several tests that exercise joins. That commit was skipped
in this merge, and a separate fix of MDEV-20265 will be necessary in 10.4.
Problem was that wsrep thread argument was deleted on wrong
place. Furthermore, scan method incorrectly used unsafe c_ptr().
Finally, fixed wsrep thread initialization to correctly set
up thread_id and pass correct argument to functions and
fix signess problem causing compiler errors.
Galera threads were not registered to performance schema and
used pthread_create when mysql_thread_create should have been
used.
Added test case to verify current galera performance schema
instrumentation does work.
Problem was that tests select INFORMATION_SCHEMA.PROCESSLIST processes
from user system user and empty state. Thus, there is not clear
state for slave threads.
Changes:
- Added new status variables that store current amount of applier threads
(wsrep_applier_thread_count) and rollbacker threads
(wsrep_rollbacker_thread_count). This will make clear how many slave threads
of certain type there is.
- Added THD state "wsrep applier idle" when applier slave thread is
waiting for work. This makes finding slave/applier threads easier.
- Added force-restart option for mtr to always restart servers between tests
to avoid race on start of the test
- Added wait_condition_with_debug to wait until the passed statement returns
true, or the operation times out. If operation times out, the additional error
statement will be executed
Changes to be committed:
new file: mysql-test/include/force_restart.inc
new file: mysql-test/include/wait_condition_with_debug.inc
modified: mysql-test/mysql-test-run.pl
modified: mysql-test/suite/galera/disabled.def
modified: mysql-test/suite/galera/r/MW-336.result
modified: mysql-test/suite/galera/r/galera_kill_applier.result
modified: mysql-test/suite/galera/r/galera_var_slave_threads.result
new file: mysql-test/suite/galera/t/MW-336.cnf
modified: mysql-test/suite/galera/t/MW-336.test
modified: mysql-test/suite/galera/t/galera_kill_applier.test
modified: mysql-test/suite/galera/t/galera_parallel_autoinc_largetrx.test
modified: mysql-test/suite/galera/t/galera_parallel_autoinc_manytrx.test
modified: mysql-test/suite/galera/t/galera_var_slave_threads.test
modified: mysql-test/suite/wsrep/disabled.def
modified: mysql-test/suite/wsrep/r/variables.result
modified: mysql-test/suite/wsrep/t/variables.test
modified: sql/mysqld.cc
modified: sql/wsrep_mysqld.cc
modified: sql/wsrep_mysqld.h
modified: sql/wsrep_thd.cc
modified: sql/wsrep_var.cc