This patch fixes cases where a transaction caused empty writeset to be
replicated. This could happen in the case where a transaction executes
a statement that initially manages to modify some data and therefore
appended keys some for certification. The statement is however rolled
back at some later stage due to some error (for example, a duplicate
key error). After statement rollback the transaction is still alive,
has no other changes. When committing such transaction, an empty
writeset was replicated through Galera.
The fix is to avoid calling into commit hook only when transaction
has appended one or keys for certification *and* has some data in
binlog cache to replicate. Otherwise, the commit is considered empty,
and goes through usual empty commit path.
Signed-off-by: Julius Goryavsky <julius.goryavsky@mariadb.com>
|| state() == s_prepared || state() == s_committing
|| state() == s_must_abort || state() == s_replaying'
failed.
CACHE INDEX and LOAD INDEX INTO CACHE are local operations.
Therefore, do not replicate them with Galera.
Signed-off-by: Julius Goryavsky <julius.goryavsky@mariadb.com>
Do not start TOI for CREATE TEMPORARY SEQUENCE because
object is local only and not replicated. Similarly,
avoid starting RSU for TEMPORARY SEQUENCEs. Finally,
we need to run commit hooks for TEMPORARY SEQUENCEs
because CREATE TEMPORARY SEQUENCE does implicit
commit for previous changes that need to be replicated
and committed.
Signed-off-by: Julius Goryavsky <julius.goryavsky@mariadb.com>
The problem seems to be a deadlock between KILL command execution
and BF abort issued by an applier, where:
* KILL has locked victim's LOCK_thd_kill and LOCK_thd_data.
* Applier has innodb side global lock mutex and victim trx mutex.
* KILL is calling innobase_kill_query, and is blocked by innodb
global lock mutex.
* Applier is in wsrep_innobase_kill_one_trx and is blocked by
victim's LOCK_thd_kill.
The fix in this commit removes the TOI replication of KILL command
and makes KILL execution less intrusive operation. Aborting the
victim happens now by using awake_no_mutex() and ha_abort_transaction().
If the KILL happens when the transaction is committing, the
KILL operation is postponed to happen after the statement
has completed in order to avoid KILL to interrupt commit
processing.
Notable changes in this commit:
* wsrep client connections's error state may remain sticky after
client connection is closed. This error message will then pop
up for the next client session issuing first SQL statement.
This problem raised with test galera.galera_bf_kill.
The fix is to reset wsrep client error state, before a THD is
reused for next connetion.
* Release THD locks in wsrep_abort_transaction when locking
innodb mutexes. This guarantees same locking order as with applier
BF aborting.
* BF abort from MDL was changed to do BF abort on server/wsrep-lib
side first, and only then do the BF abort on InnoDB side. This
removes the need to call back from InnoDB for BF aborts which originate
from MDL and simplifies the locking.
* Removed wsrep_thd_set_wsrep_aborter() from service_wsrep.h.
The manipulation of the wsrep_aborter can be done solely on
server side. Moreover, it is now debug only variable and
could be excluded from optimized builds.
* Remove LOCK_thd_kill from wsrep_thd_LOCK/UNLOCK to allow more
fine grained locking for SR BF abort which may require locking
of victim LOCK_thd_kill. Added explicit call for
wsrep_thd_kill_LOCK/UNLOCK where appropriate.
* Wsrep-lib was updated to version which allows external
locking for BF abort calls.
Changes to MTR tests:
* Disable galera_bf_abort_group_commit. This test is going to
be removed (MDEV-30855).
* Record galera_gcache_recover_manytrx as result file was incomplete.
Trivial change.
* Make galera_create_table_as_select more deterministic:
Wait until CTAS execution has reached MDL wait for multi-master
conflict case. Expected error from multi-master conflict is
ER_QUERY_INTERRUPTED. This is because CTAS does not yet have open
wsrep transaction when it is waiting for MDL, query gets interrupted
instead of BF aborted. This should be addressed in separate task.
* A new test galera_kill_group_commit to verify correct behavior
when KILL is executed while the transaction is committing.
Co-authored-by: Seppo Jaakola <seppo.jaakola@iki.fi>
Co-authored-by: Jan Lindström <jan.lindstrom@galeracluster.com>
Signed-off-by: Julius Goryavsky <julius.goryavsky@mariadb.com>
This is a backport from 10.5.
The problem seems to be a deadlock between KILL command execution
and BF abort issued by an applier, where:
* KILL has locked victim's LOCK_thd_kill and LOCK_thd_data.
* Applier has innodb side global lock mutex and victim trx mutex.
* KILL is calling innobase_kill_query, and is blocked by innodb
global lock mutex.
* Applier is in wsrep_innobase_kill_one_trx and is blocked by
victim's LOCK_thd_kill.
The fix in this commit removes the TOI replication of KILL command
and makes KILL execution less intrusive operation. Aborting the
victim happens now by using awake_no_mutex() and ha_abort_transaction().
If the KILL happens when the transaction is committing, the
KILL operation is postponed to happen after the statement
has completed in order to avoid KILL to interrupt commit
processing.
Notable changes in this commit:
* wsrep client connections's error state may remain sticky after
client connection is closed. This error message will then pop
up for the next client session issuing first SQL statement.
This problem raised with test galera.galera_bf_kill.
The fix is to reset wsrep client error state, before a THD is
reused for next connetion.
* Release THD locks in wsrep_abort_transaction when locking
innodb mutexes. This guarantees same locking order as with applier
BF aborting.
* BF abort from MDL was changed to do BF abort on server/wsrep-lib
side first, and only then do the BF abort on InnoDB side. This
removes the need to call back from InnoDB for BF aborts which originate
from MDL and simplifies the locking.
* Removed wsrep_thd_set_wsrep_aborter() from service_wsrep.h.
The manipulation of the wsrep_aborter can be done solely on
server side. Moreover, it is now debug only variable and
could be excluded from optimized builds.
* Remove LOCK_thd_kill from wsrep_thd_LOCK/UNLOCK to allow more
fine grained locking for SR BF abort which may require locking
of victim LOCK_thd_kill. Added explicit call for
wsrep_thd_kill_LOCK/UNLOCK where appropriate.
* Wsrep-lib was updated to version which allows external
locking for BF abort calls.
Changes to MTR tests:
* Disable galera_bf_abort_group_commit. This test is going to
be removed (MDEV-30855).
* Record galera_gcache_recover_manytrx as result file was incomplete.
Trivial change.
* Make galera_create_table_as_select more deterministic:
Wait until CTAS execution has reached MDL wait for multi-master
conflict case. Expected error from multi-master conflict is
ER_QUERY_INTERRUPTED. This is because CTAS does not yet have open
wsrep transaction when it is waiting for MDL, query gets interrupted
instead of BF aborted. This should be addressed in separate task.
* A new test galera_kill_group_commit to verify correct behavior
when KILL is executed while the transaction is committing.
Co-authored-by: Seppo Jaakola <seppo.jaakola@iki.fi>
Co-authored-by: Jan Lindström <jan.lindstrom@galeracluster.com>
Signed-off-by: Julius Goryavsky <julius.goryavsky@mariadb.com>
Problem for Galera is the fact that sequences are not really
transactional. Sequence operation is committed immediately
in sql_sequence.cd and later Galera could find out that
we have changes but actual statement is not there anymore.
Therefore, we must make some restrictions what kind
of sequences Galera can support.
(1) Galera cluster supports only sequences implemented
by InnoDB storage engine. This is because Galera replication
supports currently only InnoDB.
(2) We do not allow LOCK TABLE on sequence object and
we do not allow sequence creation under LOCK TABLE, instead
lock is released and we issue warning.
(3) We allow sequences with NOCACHE definition or with
INCREMEMENT BY 0 CACHE=n definition. This makes sure that
sequence values are unique accross Galera cluster.
Signed-off-by: Julius Goryavsky <julius.goryavsky@mariadb.com>
CREATE [TEMPORARY] SEQUENCE is internally CREATE+INSERT (initial value)
and it is replicated using statement based replication. In Galera
we use either TOI or RSU so we should skip commit time hooks
for it.
Signed-off-by: Julius Goryavsky <julius.goryavsky@mariadb.com>
If repl.max_ws_size is set too low following CREATE TABLE could fail
during commit. In this case wsrep_commit_empty should allow rolling
it back if provider state is s_aborted.
Furhermore, original ER_ERROR_DURING_COMMIT does not really tell anything
clear for user. Therefore, this commit adds a new error
ER_TOO_BIG_WRITESET. This will change some test cases output.
This patch fixes a problem that arises when a Galera node acts as a
replica for native replication. When parallel applying is enabled, it
is possible to end up with attempts to write binlog events with gtids
out of order. This happens because when multiple events are delivered
from the native replication stream and applied in concurrently, it is
for them to be replicated to the Galera cluster in an order which is
different from the original order in which they were committed in the
aync replication master.
To correct this behavior we now wait_for_prior_commit() before
replicating changes though galera. As a consequence, parallel appliers
may apply events in parallel until the galera replication step, which
is now serialized.
MDEV-22617 Galera node crashes when trying to log to slow_log table in
streaming replication mode
Other things:
- Changed name of wsrep_after_row(two arguments) to
wsrep_after_row_internal(one argument) to not depended on the
function signature with unused arguments.
Reviewed-by: Jan Lindström <jan.lindstrom@mariadb.com>
Added test case
This patch changes statement rollback for streaming replication.
Previously, a statement rollback was turned into full transaction
rollback in the case where the transaction had already replicated a
fragment. This was introduced in the initial implementation of
streaming replication due to the fact that we do not have a mechanism
to perform a statement rollback on the applying side.
This policy is however overly pessimistic, causing full rollbacks even
in cases where a local statement rollback, would not require a
statement rollback on the applying side. This happens to be case when
the statement itself has not replicated any fragments.
So the patch changes the condition that determines if a statement
rollback should be turned into a full rollback accordingly.
Reviewed-by: Jan Lindström <jan.lindstrom@mariadb.com>
* Disallow setting wsrep_on = 1 if wsrep_provider is unset. Also, move
wsrep_on_basic from sys_vars to wsrep suite: this test now requires
to run with wsrep_provider set
* Disallow setting @@session.wsrep_on = 1 when @@global.wsrep_on = 0
* Handle the case where a new connection turns @@global.wsrep_on from
off to on. In this case we would miss a call to wsrep_open, causing
unexpected states in wsrep::client_state (causing assertions).
* Disable wsrep.MDEV-22443 because it is no longer possible to enable
wsrep_on, if server is started with wsrep_provider='none'
Reviewed-by: Jan Lindström <jan.lindstrom@mariadb.com>
Assert was:
mariadbd: /my/maria-10.6/wsrep-lib/src/client_state.cpp:256:
int wsrep::client_state::after_statement(): Assertion `state() == s_exec'
The reason was because of two faults:
- A missing test for WSREP(thd) when checking wsrep_after_statement(()
- THD->wsrep_cs().state was set to s_idle instead of s_none
This patch makes the following changes around variable wsrep_on:
1) Variable wsrep_on can no longer be updated from a session that has
an active transaction running. The original behavior allowed cases
like this:
BEGIN;
INSERT INTO t1 VALUES (1);
SET SESSION wsrep_on = OFF;
INSERT INTO t1 VALUES (2);
COMMIT;
With regular transactions this would result in no replication
events (not even value 1). With streaming replication it would be
unnecessarily complex to achieve the same behavior. In the above
example, it would be possible for value 1 to be already replicated if
it happened to fill a separate fragment, while value 2 wouldn't.
2) Global variable wsrep_on no longer affects current sessions, only
subsequent ones. This is to avoid a similar case to the above, just
using just by using global wsrep_on instead session wsrep_on:
--connection conn_1
BEGIN;
INSERT INTO t1 VALUES(1);
--connection conn_2
SET GLOBAL wsrep_on = OFF;
--connection conn_1
INSERT INTO t1 VALUES(2);
COMMIT;
The above example results in the transaction to be replicated, as
global wsrep_on will only affect the session wsrep_on of new
connections.
Reviewed-by: Jan Lindström <jan.lindstrom@mariadb.com>
Under ps-protocol, commandsl like COM_STMT_FETCH, COM_STMT_CLOSE and
COM_STMT_SEND_LONG_DATA are not supposed to return errors. Therefore,
if a transaction is BF aborted and the client is processing one of
those commands, then we should not return a deadlock error
immediately. Instead wait for the a subsequent client interaction
which permits errors to be returned. To handle this,
wsrep_before_command() now accepts parameter keep_command_error. If
set true, keep_command_error will cause wsrep-lib side to skip result
handling, and to keep the current error for the next interaction with
the client.
Reviewed-by: Jan Lindström <jan.lindstrom@mariadb.com>
The optional AND CHAIN clause is a convenience for initiating a new
transaction as soon as the old transaction terminates. Therefore,
do not start new transaction if it is already started
at wsrep_start_transaction.
MDEV-22617 Galera node crashes when trying to log to slow_log table in
streaming replication mode
Other things:
- Changed name of wsrep_after_row(two arguments) to
wsrep_after_row_internal(one argument) to not depended on the
function signature with unused arguments.
All changes (except one) is of type
thd->transaction. -> thd->transaction->
thd->transaction points by default to 'thd->default_transaction'
This allows us to 'easily' have multiple active transactions for a
THD object, like when reading data from the mysql.proc table
The reason why we have wsrep_on() at all is that the macro WSREP(thd)
depends on the definition of THD, and that is intentionally an opaque
data type for InnoDB. So, we cannot avoid invoking wsrep_on(), but
we can evaluate the less expensive conditions thd && WSREP_ON before
calling the function.
Global_read_lock: Use WSREP_NNULL(thd) instead of wsrep_on(thd)
because we not only know the definition of THD but also that
the pointer is not null.
wsrep_open(): Use WSREP(thd) instead of wsrep_on(thd).
InnoDB: Replace thd && wsrep_on(thd) with wsrep_on(thd), now that
the condition has been merged to the definition of the macro
wsrep_on().
* Remove dead code
* MDEV-21675 Data inconsistency after multirow insert rollback
This patch fixes data inconsistencies that happen after rollback of
multirow inserts, with binlog disabled.
For example, statements such as `INSERT INTO t1 VALUES (1,'a'),(1,'b')`
that fail with duplicate key error. In such cases the whole statement
is rolled back. However, with wsrep_emulate_binlog in effect, the
IO_CACHE would not be truncated, and the pending rows events would be
replicated to the rest of the cluster. In the above example, it would
result in row (1,'a') being replicated, whereas locally the statement
is rolled back entirely. Making the cluster inconsistent.
The patch changes the code so that prior to statement rollback,
pending rows event are removed and the stmt cache reset.
That patch also introduces MTR tests that excercise multirow insert
statements for regular, and streaming replication.
e.g.
- dont -> don't
- occurence -> occurrence
- succesfully -> successfully
- easyly -> easily
Also remove trailing space in selected files.
These changes span:
- server core
- Connect and Innobase storage engine code
- OQgraph, Sphinx and TokuDB storage engines
Related to MDEV-21769.
Support for galera GTID consistency thru cluster. All nodes in cluster
should have same GTID for replicated events which are originating from cluster.
Cluster originating commands need to contain sequential WSREP GTID seqno
Ignore manual setting of gtid_seq_no=X.
In master-slave scenario where master is non galera node replicated GTID is
replicated and is preserved in all nodes.
To have this - domain_id, server_id and seqnos should be same on all nodes.
Node which bootstraps the cluster, to achieve this, sends domain_id and
server_id to other nodes and this combination is used to write GTID for events
that are replicated inside cluster.
Cluster nodes that are executing non replicated events are going to have different
GTID than replicated ones, difference will be visible in domain part of gtid.
With wsrep_gtid_domain_id you can set domain_id for WSREP cluster.
Functions WSREP_LAST_WRITTEN_GTID, WSREP_LAST_SEEN_GTID and
WSREP_SYNC_WAIT_UPTO_GTID now works with "native" GTID format.
Fixed galera tests to reflect this chances.
Add variable to manually update WSREP GTID seqno in cluster
Add variable to manipulate and change WSREP GTID seqno. Next command
originating from cluster and on same thread will have set seqno and
cluster should change their internal counter to it's value.
Behavior is same as using @@gtid_seq_no for non WSREP transaction.
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.
* Collect and pass apply error data to provider
* Rollback failed transaction and continue operation if provider returns
SUCCESS
* MTR tests for inconsistency voting
The extensive usage of stack space, especially when used with ASan
(AdressSanitizer) of mysql_execute_command caused the test
rpl.rpl_row_sp011 to fail because it did run out of stack. In this
test case mysql_execute_command is called recursively for each
function all.
Changes done:
- Changed a few functions that used big local variables to be marked
__attribute__ ((noinline))
- Moved sub parts that used big local variables to external functions.
- Changed wsrep_commit_empty() from inline to normal function as this used
more than 1K of stack space and because there is no reason for this
rarely used function to be inline.
End result (with gcc 7.4.1 on Intel Xeon):
Starting point for stack space usage:
gcc -O: 7800
gcc with -fsanitize=address -O (ASan) : 27240
After this patch:
gcc -O: 1160
gcc -O0 (debug build) 1584
gcc with -fsanitize=address -O (ASan): 4424
gcc with -fsanitize=address -O2 (ASan): 3874
A 6x improvement and will allow us to run all mtr tests with ASan.
- Changed replaying to always allocate a separate THD object
for applying log events. This is to avoid tampering original
THD state during replay process.
- Return success from sp_instr_stmt::exec_core() if replaying
succeeds.
- Do not push warnings/errors into diagnostics area if the
transaction must be replayed. This is to avoid reporting
transient errors to the client.
Added two tests galera_sp_bf_abort, galera_sp_insert_parallel.
Wsrep-lib position updated.
* MDEV-16509 Improve wsrep commit performance with binlog disabled
Release commit order critical section early after trx_commit_low() if
binlog is not transaction coordinator. In order to avoid two phase commit,
binlog_hton is not registered for THD during IO_CACHE population.
Implemented a test which verifies that the transactions release
commit order early.
This optimization will change behavior during recovery as the commit
is not two phase when binlog is off. Fixed and recorded wsrep-recover-v25
and wsrep-recover to match the behavior.
* MDEV-18730 Ordering for wsrep binlog group commit
Previously out of order execution was allowed for wsrep commits.
Established proper ordering by populating wait_for_commit
for every wsrep THD and making group commit leader to wait for
prior commits before proceeding to trx_group_commit_leader().
* MDEV-18730 Added a test case to verify correct commit ordering
* MDEV-16509, MDEV-18730 Review fixes
Use WSREP_EMULATE_BINLOG() macro to decide if the binlog_hton
should be registered. Whitespace/syntax fixes and cleanups.
* MDEV-16509 Require binlog for galera_var_innodb_disallow_writes test
If the commit to InnoDB is done in one phase, the native InnoDB behavior
is that the transaction is committed in memory before it is persisted to
disk. This means that the innodb_disallow_writes=ON may not prevent
transaction to become visible to other readers before commit is completely
over. On the other hand, if the commit is two phase (as it is with binlog),
the transaction will be blocked in prepare phase.
Fixed the test to use binlog, which enforces two phase commit, which
in turn makes commit to block before the changes become visible to
other connections. This guarantees that the test produces expected
result.
With wsrep_gtid_mode=ON, the appropriate commit hooks were not
called in all cases for applied streaming transactions.
As a fix, removed all special handling of commit order critical
section from Wsrep_high_priority_service and Wsrep_storage_service.
Now commit order critical section is always entered in ha_commit_trans().
Check for wsrep_run_commit_hook is now done in handler.cc, log.cc.
This makes it explicit that the transaction is an active wsrep
transaction which must go through commit hooks.
Make sure that the Annotate_rows_log_events is written into
binlog only for the first fragment of the current statement.
Also avoid flusing pending rows event when calculating bytes
generated by the transaction.
Added and recorded a test which verifies that the binlog
contains only one Annotate_rows_log_event per statement
with various SR settings. Recrded mysql-wsrep-features#136
which produced different output with excession log events
suppressed.