Commit 1ef50a34 authored by Teemu Ollakka's avatar Teemu Ollakka Committed by Jan Lindström

10.4 wsrep group commit fixes (#1224)

* 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.
parent b234f810
......@@ -82,6 +82,7 @@ extern struct wsrep_service_st {
my_bool (*wsrep_thd_skip_locking_func)(const MYSQL_THD thd);
const char* (*wsrep_get_sr_table_name_func)();
my_bool (*wsrep_get_debug_func)();
void (*wsrep_commit_ordered_func)(MYSQL_THD thd);
} *wsrep_service;
#define MYSQL_SERVICE_WSREP_INCLUDED
......@@ -120,6 +121,7 @@ extern struct wsrep_service_st {
#define wsrep_thd_skip_locking(T) wsrep_service->wsrep_thd_skip_locking_func(T)
#define wsrep_get_sr_table_name() wsrep_service->wsrep_get_sr_table_name_func()
#define wsrep_get_debug() wsrep_service->wsrep_get_debug_func()
#define wsrep_commit_ordered(T) wsrep_service->wsrep_commit_ordered(T)
#else
......@@ -209,5 +211,8 @@ extern const char* wsrep_sr_table_name_full;
extern "C" const char* wsrep_get_sr_table_name();
extern "C" my_bool wsrep_get_debug();
extern "C" void wsrep_commit_ordered(MYSQL_THD thd);
#endif
#endif /* MYSQL_SERVICE_WSREP_INCLUDED */
connection node_2;
connection node_1;
CREATE TABLE t1 (f1 INT PRIMARY KEY) ENGINE=InnoDB;
SET SESSION wsrep_sync_wait = 0;
connection node_1;
SET SESSION wsrep_sync_wait = 0;
SET DEBUG_SYNC = "wsrep_before_commit_order_leave SIGNAL bcol_reached WAIT_FOR bcol_continue";
SET DEBUG_SYNC = "wsrep_after_commit_order_leave SIGNAL acol_reached WAIT_FOR acol_continue";
SET DEBUG_SYNC = "after_group_after_commit SIGNAL after_group_reached WAIT_FOR after_group_continue";
INSERT INTO t1 VALUES (1);
connection ctrl;
SET DEBUG_SYNC = "now WAIT_FOR bcol_reached";
wsrep_last_seen_gtid_match
1
SELECT * FROM t1;
f1
1
SET DEBUG_SYNC = "now SIGNAL bcol_continue";
SET DEBUG_SYNC = "now WAIT_FOR acol_reached";
wsrep_last_seen_gtid_match
1
SELECT * FROM t1;
f1
1
SET DEBUG_SYNC = "now SIGNAL acol_continue";
SET DEBUG_SYNC = "now WAIT_FOR after_group_reached";
wsrep_last_seen_gtid_do_not_match
1
SET DEBUG_SYNC = "now SIGNAL after_group_continue";
connection node_1;
SET SESSION wsrep_sync_wait = 0;
connection ctrl;
connection node_1;
SET DEBUG_SYNC = "wsrep_before_commit_order_leave SIGNAL bcol_reached_1 WAIT_FOR bcol_continue_1";
SET DEBUG_SYNC = "wsrep_after_commit_order_leave SIGNAL acol_reached_1 WAIT_FOR acol_continue_1";
SET DEBUG_SYNC = "after_group_after_commit SIGNAL agac_reached_1 WAIT_FOR agac_continue_1";
INSERT INTO t1 VALUES (2);;
connection ctrl;
SET DEBUG_SYNC = "now WAIT_FOR bcol_reached_1";
wsrep_last_seen_gtid_match
1
connection node_1a;
SET DEBUG_SYNC = "wsrep_before_commit_order_leave SIGNAL bcol_reached_2 WAIT_FOR bcol_continue_2";
SET DEBUG_SYNC = "wsrep_after_commit_order_leave SIGNAL acol_reached_2 WAIT_FOR acol_continue_2";
SET DEBUG_SYNC = "after_group_after_commit SIGNAL agac_reached_2 WAIT_FOR agac_continue_2";
INSERT INTO t1 VALUES (3);;
connection ctrl;
SET DEBUG_SYNC = "now SIGNAL bcol_continue_1";
SET DEBUG_SYNC = "now WAIT_FOR acol_reached_1";
SET DEBUG_SYNC = "now WAIT_FOR bcol_reached_2";
wsrep_last_seen_gtid_match
1
SET DEBUG_SYNC = "now SIGNAL bcol_continue_2";
SET DEBUG_SYNC = "now WAIT_FOR acol_reached_2";
wsrep_last_seen_gtid_match
1
SET DEBUG_SYNC = "now SIGNAL acol_continue_1";
SET DEBUG_SYNC = "now WAIT_FOR agac_reached_1";
wsrep_last_seen_gtid_no_match
1
SET DEBUG_SYNC = "now SIGNAL acol_continue_2";
SET DEBUG_SYNC = "now WAIT_FOR agac_reached_2";
wsrep_last_seen_gtid_no_match
1
SET DEBUG_SYNC = "now SIGNAL agac_continue_1";
SET DEBUG_SYNC = "now SIGNAL agac_continue_2";
connection node_1;
connection node_1a;
connection ctrl;
SET DEBUG_SYNC = "RESET";
DROP TABLE t1;
disconnect ctrl;
disconnect node_1a;
disconnect node_2;
disconnect node_1;
connection node_2;
connection node_1;
CREATE TABLE t1 (f1 INT PRIMARY KEY);
connection node_1;
SET DEBUG_SYNC = "wsrep_after_commit_order_leave SIGNAL acol_reached WAIT_FOR acol_continue";
INSERT INTO t1 VALUES (1);
connection ctrl;
SET DEBUG_SYNC = "now WAIT_FOR acol_reached";
connection node_1_sr;
SET SESSION wsrep_sync_wait = 0;
SET SESSION wsrep_trx_fragment_unit = 'rows';
SET SESSION wsrep_trx_fragment_size = 1;
START TRANSACTION;
INSERT INTO t1 VALUES (2);
connection ctrl;
SET DEBUG_SYNC = "now SIGNAL acol_continue";
connection node_1;
connection node_1_sr;
ROLLBACK;
connection ctrl;
SET DEBUG_SYNC = "RESET";
disconnect ctrl;
disconnect node_1_sr;
connection node_1;
DROP TABLE t1;
disconnect node_2;
disconnect node_1;
#
# Test various executions which go through binlog group commit
#
--source include/galera_cluster.inc
--source include/have_debug_sync.inc
CREATE TABLE t1 (f1 INT PRIMARY KEY) ENGINE=InnoDB;
--let $galera_connection_name = ctrl
--let $galera_server_number = 1
--source include/galera_connect.inc
# Scenario 1: Block INSERT after commit order release after queued for
# group commit. Verify that
#
# - WSREP_LAST_SEEN_GTID is not advanced before commit finishes
# - The INSERT does not become visible before commit finishes
# Turn off sync wait to avoid blocking and use WSREP_LAST_SEEN_GTID()
# to observe gtid position.
SET SESSION wsrep_sync_wait = 0;
--let $last_seen_gtid_prev = `SELECT WSREP_LAST_SEEN_GTID()`
--connection node_1
SET SESSION wsrep_sync_wait = 0;
# Set up sync points
SET DEBUG_SYNC = "wsrep_before_commit_order_leave SIGNAL bcol_reached WAIT_FOR bcol_continue";
SET DEBUG_SYNC = "wsrep_after_commit_order_leave SIGNAL acol_reached WAIT_FOR acol_continue";
SET DEBUG_SYNC = "after_group_after_commit SIGNAL after_group_reached WAIT_FOR after_group_continue";
# Send insert which will block in the sync points above
--send INSERT INTO t1 VALUES (1)
--connection ctrl
# INSERT has gone through wsrep_ordered_commit() and the transaction is
# committed in memory.
SET DEBUG_SYNC = "now WAIT_FOR bcol_reached";
--disable_query_log
--eval SELECT WSREP_LAST_SEEN_GTID() = '$last_seen_gtid_prev' AS wsrep_last_seen_gtid_match
--enable_query_log
SELECT * FROM t1;
SET DEBUG_SYNC = "now SIGNAL bcol_continue";
# SE commit finished but wsrep_after_commit() has not called yet.
SET DEBUG_SYNC = "now WAIT_FOR acol_reached";
--disable_query_log
--eval SELECT WSREP_LAST_SEEN_GTID() = '$last_seen_gtid_prev' AS wsrep_last_seen_gtid_match
--enable_query_log
SELECT * FROM t1;
SET DEBUG_SYNC = "now SIGNAL acol_continue";
SET DEBUG_SYNC = "now WAIT_FOR after_group_reached";
--disable_query_log
--eval SELECT WSREP_LAST_SEEN_GTID() != '$last_seen_gtid_prev' AS wsrep_last_seen_gtid_do_not_match
--enable_query_log
SET DEBUG_SYNC = "now SIGNAL after_group_continue";
--connection node_1
--reap
#
# Scenario 2: Verify that two INSERTs from two different connections
# queue for commit.
#
--let $galera_connection_name = node_1a
--let $galera_server_number = 1
--source include/galera_connect.inc
SET SESSION wsrep_sync_wait = 0;
--connection ctrl
--let $last_seen_gtid_prev = `SELECT WSREP_LAST_SEEN_GTID()`
--connection node_1
SET DEBUG_SYNC = "wsrep_before_commit_order_leave SIGNAL bcol_reached_1 WAIT_FOR bcol_continue_1";
SET DEBUG_SYNC = "wsrep_after_commit_order_leave SIGNAL acol_reached_1 WAIT_FOR acol_continue_1";
SET DEBUG_SYNC = "after_group_after_commit SIGNAL agac_reached_1 WAIT_FOR agac_continue_1";
--send INSERT INTO t1 VALUES (2);
--connection ctrl
SET DEBUG_SYNC = "now WAIT_FOR bcol_reached_1";
--disable_query_log
--eval SELECT WSREP_LAST_SEEN_GTID() = '$last_seen_gtid_prev' AS wsrep_last_seen_gtid_match
--enable_query_log
--connection node_1a
SET DEBUG_SYNC = "wsrep_before_commit_order_leave SIGNAL bcol_reached_2 WAIT_FOR bcol_continue_2";
SET DEBUG_SYNC = "wsrep_after_commit_order_leave SIGNAL acol_reached_2 WAIT_FOR acol_continue_2";
SET DEBUG_SYNC = "after_group_after_commit SIGNAL agac_reached_2 WAIT_FOR agac_continue_2";
--send INSERT INTO t1 VALUES (3);
# Now INSERTs are queued, node_1 waiting after releasing commit order,
# node_1a waiting before releasing commit order.
--connection ctrl
SET DEBUG_SYNC = "now SIGNAL bcol_continue_1";
SET DEBUG_SYNC = "now WAIT_FOR acol_reached_1";
SET DEBUG_SYNC = "now WAIT_FOR bcol_reached_2";
--disable_query_log
--eval SELECT WSREP_LAST_SEEN_GTID() = '$last_seen_gtid_prev' AS wsrep_last_seen_gtid_match
--enable_query_log
SET DEBUG_SYNC = "now SIGNAL bcol_continue_2";
SET DEBUG_SYNC = "now WAIT_FOR acol_reached_2";
--disable_query_log
--eval SELECT WSREP_LAST_SEEN_GTID() = '$last_seen_gtid_prev' AS wsrep_last_seen_gtid_match
--enable_query_log
# Last seen GTIDs are incremented one by one once after_group_after_commit
# is reached.
SET DEBUG_SYNC = "now SIGNAL acol_continue_1";
SET DEBUG_SYNC = "now WAIT_FOR agac_reached_1";
--disable_query_log
--eval SELECT WSREP_LAST_SEEN_GTID() != '$last_seen_gtid_prev' AS wsrep_last_seen_gtid_no_match
--enable_query_log
--let $last_seen_gtid_prev = `SELECT WSREP_LAST_SEEN_GTID()`
SET DEBUG_SYNC = "now SIGNAL acol_continue_2";
SET DEBUG_SYNC = "now WAIT_FOR agac_reached_2";
--disable_query_log
--eval SELECT WSREP_LAST_SEEN_GTID() != '$last_seen_gtid_prev' AS wsrep_last_seen_gtid_no_match
--enable_query_log
SET DEBUG_SYNC = "now SIGNAL agac_continue_1";
SET DEBUG_SYNC = "now SIGNAL agac_continue_2";
--connection node_1
--reap
--connection node_1a
--reap
--connection ctrl
SET DEBUG_SYNC = "RESET";
DROP TABLE t1;
--disconnect ctrl
--disconnect node_1a
--source include/galera_end.inc
#
# This test checks that innodb_disallow_writes works as expected
#
# Note that we need to enable binlog for this test: If the commit
# to InnoDB is done in one phase, 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. On the other hand, if the
# commit is two phase (as it is with binlog), the transaction
# will be blocked in prepare phase.
#
--source include/galera_cluster.inc
--source include/have_innodb.inc
--source include/have_log_bin.inc
# Open a separate connection to be used to run SHOW PROCESSLIST
--let $galera_connection_name = node_1a
......
#
# Test scenario:
#
# Run an autocommit INSERT and stop the execution after the INSERT
# has released commit order critical section. On another connection,
# run SR transaction which will store one fragment into streaming log.
# If the bug is present, the fragment streaming log commit may
# out of order, and completing INSERT may cause assertion in debug build.
#
# Note that due to nature of this bug, it is may not be possible
# to construct fully deterministic test case which will crash the
# server each time if the bug is present, but will work with fix.
#
--source include/galera_cluster.inc
--source include/have_log_bin.inc
--source include/have_debug_sync.inc
CREATE TABLE t1 (f1 INT PRIMARY KEY);
# Control connection for controlling node_1 debug sync points
--let $galera_connection_name = ctrl
--let $galera_server_number = 1
--source include/galera_connect.inc
# Another connection for SR transaction
--let $galera_connection_name = node_1_sr
--let $galera_server_number = 1
--source include/galera_connect.inc
# Set up sync point and send INSERT
--connection node_1
SET DEBUG_SYNC = "wsrep_after_commit_order_leave SIGNAL acol_reached WAIT_FOR acol_continue";
--send INSERT INTO t1 VALUES (1)
# Wait until INSERT releases commit order
--connection ctrl
SET DEBUG_SYNC = "now WAIT_FOR acol_reached";
# Streaming transaction, will replicate fragment for each row separately.
--connection node_1_sr
SET SESSION wsrep_sync_wait = 0;
SET SESSION wsrep_trx_fragment_unit = 'rows';
SET SESSION wsrep_trx_fragment_size = 1;
START TRANSACTION;
--send INSERT INTO t1 VALUES (2)
--connection ctrl
# Now let the thread node_1 continue after a one second sleep.
# The sleep while not completely deterministic, will allow the SR
# insert to complete the commit out of order in most of the cases if
# the bug is present, leading to assertion in debug build.
--sleep 1
SET DEBUG_SYNC = "now SIGNAL acol_continue";
--connection node_1
--reap
--connection node_1_sr
--reap
ROLLBACK;
--connection ctrl
SET DEBUG_SYNC = "RESET";
--disconnect ctrl
--disconnect node_1_sr
--connection node_1
DROP TABLE t1;
--source include/galera_end.inc
--- r/wsrep-recover.result 2019-01-11 16:22:46.329012579 +0200
+++ r/wsrep-recover.reject 2019-01-11 16:23:55.313137675 +0200
@@ -48,19 +48,17 @@
SET DEBUG_SYNC = "now WAIT_FOR before_commit_order_reached_1";
connection default;
# Kill the server
-Expect seqno 9
-9
+Expect seqno 7
+7
disconnect con1;
disconnect con2;
disconnect con_ctrl;
connection default;
-SELECT VARIABLE_VALUE `expect 10` FROM INFORMATION_SCHEMA.GLOBAL_STATUS WHERE VARIABLE_NAME = 'wsrep_last_committed';
-expect 10
-10
-Expect rows 5, 9, 10
+SELECT VARIABLE_VALUE `expect 8` FROM INFORMATION_SCHEMA.GLOBAL_STATUS WHERE VARIABLE_NAME = 'wsrep_last_committed';
+expect 8
+8
+Expect row 5
SELECT * FROM t1;
f1
5
-9
-10
DROP TABLE t1;
--- r/wsrep-recover-v25.result 2019-02-05 12:33:42.681586950 +0200
+++ r/wsrep-recover-v25.reject 2019-02-05 12:34:41.661752903 +0200
@@ -18,11 +18,10 @@
connection default;
SET DEBUG_SYNC = "now WAIT_FOR after_prepare_reached";
# Kill the server
-Expect seqno 7
-7
-Expect 5 7
--- r/wsrep-recover-v25.result 2019-02-28 09:20:56.153775856 +0200
+++ r/wsrep-recover-v25.reject 2019-02-28 09:22:16.578113115 +0200
@@ -12,4 +12,16 @@
SELECT VARIABLE_VALUE `expect 6` FROM INFORMATION_SCHEMA.GLOBAL_STATUS WHERE VARIABLE_NAME = 'wsrep_last_committed';
expect 6
6
+connect con1, localhost, root;
+SET DEBUG_SYNC = "ha_commit_trans_after_prepare SIGNAL after_prepare_reached WAIT_FOR continue";
+INSERT INTO t1 VALUES (7);
+connection default;
+SET DEBUG_SYNC = "now WAIT_FOR after_prepare_reached";
+# Kill the server
+Expect seqno 6
+6
+Expect 5
SELECT * FROM t1;
f1
5
-7
+SELECT * FROM t1;
+f1
+5
DROP TABLE t1;
......@@ -12,17 +12,4 @@ Expect seqno 5
SELECT VARIABLE_VALUE `expect 6` FROM INFORMATION_SCHEMA.GLOBAL_STATUS WHERE VARIABLE_NAME = 'wsrep_last_committed';
expect 6
6
connect con1, localhost, root;
SET DEBUG_SYNC = "ha_commit_trans_after_prepare SIGNAL after_prepare_reached WAIT_FOR continue";
INSERT INTO t1 VALUES (7);
connection default;
SET DEBUG_SYNC = "now WAIT_FOR after_prepare_reached";
# Kill the server
Expect seqno 7
7
Expect 5 7
SELECT * FROM t1;
f1
5
7
DROP TABLE t1;
......@@ -48,19 +48,17 @@ SET DEBUG_SYNC = "now SIGNAL continue_after_certification";
SET DEBUG_SYNC = "now WAIT_FOR before_commit_order_reached_1";
connection default;
# Kill the server
Expect seqno 9
9
Expect seqno 7
7
disconnect con1;
disconnect con2;
disconnect con_ctrl;
connection default;
SELECT VARIABLE_VALUE `expect 10` FROM INFORMATION_SCHEMA.GLOBAL_STATUS WHERE VARIABLE_NAME = 'wsrep_last_committed';
expect 10
10
Expect rows 5, 9, 10
SELECT VARIABLE_VALUE `expect 8` FROM INFORMATION_SCHEMA.GLOBAL_STATUS WHERE VARIABLE_NAME = 'wsrep_last_committed';
expect 8
8
Expect row 5
SELECT * FROM t1;
f1
5
9
10
DROP TABLE t1;
......@@ -86,36 +86,29 @@ SELECT VARIABLE_VALUE `expect 6` FROM INFORMATION_SCHEMA.GLOBAL_STATUS WHERE VAR
#
# The INSERT gets prepared but not committed.
#
# If binlog is off, the expected outcome is that the INSERT gets committed
# since it is already committed in the cluster. If binlog is on, the INSERT
# This scenario is not applicable if binlog is not on since the
# commit is not 2PC.
#
# If binlog is on, the INSERT
# should be rolled back during recovery phase since it has not yet
# been logged into binlog.
#
--connect con1, localhost, root
SET DEBUG_SYNC = "ha_commit_trans_after_prepare SIGNAL after_prepare_reached WAIT_FOR continue";
--send INSERT INTO t1 VALUES (7)
--connection default
SET DEBUG_SYNC = "now WAIT_FOR after_prepare_reached";
--source include/kill_mysqld.inc
--source wsrep-recover-step.inc
if ($log_bin) {
--echo Expect seqno 6
}
if (!$log_bin) {
--echo Expect seqno 7
}
--echo $wsrep_recover_start_position_seqno
--let $restart_parameters = --wsrep-start-position=$wsrep_recover_start_position_uuid:$wsrep_recover_start_position_seqno
--source include/start_mysqld.inc
--source include/wait_wsrep_ready.inc
--connect con1, localhost, root
SET DEBUG_SYNC = "ha_commit_trans_after_prepare SIGNAL after_prepare_reached WAIT_FOR continue";
--send INSERT INTO t1 VALUES (7)
if ($log_bin) {
--echo Expect 5
}
if (!$log_bin) {
--echo Expect 5 7
--connection default
SET DEBUG_SYNC = "now WAIT_FOR after_prepare_reached";
--source include/kill_mysqld.inc
--source wsrep-recover-step.inc
--echo Expect seqno 6
--echo $wsrep_recover_start_position_seqno
--let $restart_parameters = --wsrep-start-position=$wsrep_recover_start_position_uuid:$wsrep_recover_start_position_seqno
--source include/start_mysqld.inc
--source include/wait_wsrep_ready.inc
--echo Expect 5
SELECT * FROM t1;
}
SELECT * FROM t1;
DROP TABLE t1;
......@@ -22,7 +22,6 @@
--source include/have_wsrep_provider.inc
--source include/have_debug_sync.inc
#
# Binlog option for recovery run. This must be set in the test because
# combinations file causes log-bin option to be set from command line,
# not via my.cnf.
......@@ -135,17 +134,15 @@ SELECT VARIABLE_VALUE `expect 7` FROM INFORMATION_SCHEMA.GLOBAL_STATUS WHERE VAR
#
# This scenario will run two INSERTs in parallel so that they are
# prepared out of order. The execution is stopped before commit
# and the server is killed. The outcome of this scenario depends
# on binlog settings:
#
# If binlog is off, the transactions will be recovered from InnoDB and
# committed during recovery. The expected recovered seqno is 9, the
# expected wsrep_last_committed after server is restarted is 10.
# and the server is killed.
#
# If binlog is on, the transactions will be recovered from InnoDB but
# will be rolled back since they are not logged yet in binlog. The
# expected recovered seqno is 7, the expected wsrep_last_committed
# after server is restarted is 8.
# The transactions will be recovered from InnoDB but
# will be rolled back:
# - If binlog is on, the binlog acts as a transaction coordinator.
# The transaction is not logged into binlog, so the transaction is rolled
# back.
# - If binlog is not on, the transaction is 1PC and the wsrep XID
# is not persisted before commit is complete.
#
--connect con1, localhost, root
......@@ -168,12 +165,8 @@ SET DEBUG_SYNC = "now WAIT_FOR before_commit_order_reached_1";
--connection default
--source include/kill_mysqld.inc
--source wsrep-recover-step.inc
if ($log_bin) {
--echo Expect seqno 7
}
if (!$log_bin) {
--echo Expect seqno 9
}
--echo Expect seqno 7
--echo $wsrep_recover_start_position_seqno
--let $restart_parameters = --wsrep-start-position=$wsrep_recover_start_position_uuid:$wsrep_recover_start_position_seqno
--source include/start_mysqld.inc
......@@ -184,21 +177,13 @@ if (!$log_bin) {
--disconnect con_ctrl
--connection default
if ($log_bin) {
SELECT VARIABLE_VALUE `expect 8` FROM INFORMATION_SCHEMA.GLOBAL_STATUS WHERE VARIABLE_NAME = 'wsrep_last_committed';
}
if (!$log_bin) {
SELECT VARIABLE_VALUE `expect 10` FROM INFORMATION_SCHEMA.GLOBAL_STATUS WHERE VARIABLE_NAME = 'wsrep_last_committed';
}
SELECT VARIABLE_VALUE `expect 8` FROM INFORMATION_SCHEMA.GLOBAL_STATUS WHERE VARIABLE_NAME = 'wsrep_last_committed';
#
# Final sanity check: The successful inserts into t1 should result
if ($log_bin) {
--echo Expect row 5
}
if (!$log_bin) {
--echo Expect rows 5, 9, 10
}
# Final sanity check: The successful inserts into t1 should result single row
#
--echo Expect row 5
SELECT * FROM t1;
DROP TABLE t1;
......@@ -2202,7 +2202,19 @@ void MYSQL_BIN_LOG::set_write_error(THD *thd, bool is_transactional)
{
my_error(ER_ERROR_ON_WRITE, MYF(0), name, errno);
}
#ifdef WITH_WSREP
/* If wsrep transaction is active and binlog emulation is on,
binlog write error may leave transaction without any registered
htons. This makes wsrep rollback hooks to be skipped and the
transaction will remain alive in wsrep world after rollback.
Register binlog hton here to ensure that rollback happens in full. */
if (WSREP_EMULATE_BINLOG(thd))
{
if (is_transactional)
trans_register_ha(thd, TRUE, binlog_hton);
trans_register_ha(thd, FALSE, binlog_hton);
}
#endif /* WITH_WSREP */
DBUG_VOID_RETURN;
}
......@@ -5676,7 +5688,18 @@ THD::binlog_start_trans_and_stmt()
this->binlog_set_stmt_begin();
bool mstmt_mode= in_multi_stmt_transaction_mode();
#ifdef WITH_WSREP
/* Write Gtid
/*
With wsrep binlog emulation we can skip the rest because the
binlog cache will not be written into binlog. Note however that
because of this the hton callbacks will not get called to clean
up the cache, so this must be done explicitly when the transaction
terminates.
*/
if (WSREP_EMULATE_BINLOG_NNULL(this))
{
DBUG_VOID_RETURN;
}
/* Write Gtid
Get domain id only when gtid mode is set
If this event is replicate through a master then ,
we will forward the same gtid another nodes
......@@ -7686,9 +7709,24 @@ MYSQL_BIN_LOG::write_transaction_to_binlog_events(group_commit_entry *entry)
{
int is_leader= queue_for_group_commit(entry);
#ifdef WITH_WSREP
if (wsrep_run_commit_hook(entry->thd, true) && is_leader >= 0 &&
wsrep_ordered_commit(entry->thd, entry->all, wsrep_apply_error()))
return true;
if (wsrep_is_active(entry->thd) &&
wsrep_run_commit_hook(entry->thd, entry->all))
{
/*
Release commit order and if leader, wait for prior commit to
complete. This establishes total order for group leaders.
*/
if (wsrep_ordered_commit(entry->thd, entry->all, wsrep_apply_error()))
{
entry->thd->wakeup_subsequent_commits(1);
return 1;
}
if (is_leader)
{
if (entry->thd->wait_for_prior_commit())
return 1;
}
}
#endif /* WITH_WSREP */
/*
The first in the queue handles group commit for all; the others just wait
......
......@@ -20,6 +20,7 @@
#include "wsrep_trans_observer.h"
#include "sql_class.h"
#include "debug_sync.h"
#include "log.h"
extern "C" my_bool wsrep_on(const THD *thd)
{
......@@ -259,3 +260,13 @@ extern "C" int wsrep_thd_append_key(THD *thd,
}
return ret;
}
extern "C" void wsrep_commit_ordered(THD *thd)
{
if (wsrep_is_active(thd) &&
thd->wsrep_trx().state() == wsrep::transaction::s_committing &&
!wsrep_commit_will_write_binlog(thd))
{
thd->wsrep_cs().ordered_commit();
}
}
......@@ -682,7 +682,9 @@ THD::THD(my_thread_id id, bool is_wsrep_applier, bool skip_global_sys_var_lock)
m_wsrep_cond,
Wsrep_server_state::instance(),
m_wsrep_client_service,
wsrep::client_id(thread_id))
wsrep::client_id(thread_id)),
wsrep_applier_service(NULL),
wsrep_wfc()
#endif /*WITH_WSREP */
{
ulong tmp;
......
......@@ -4933,6 +4933,8 @@ class THD: public THD_count, /* this must be first */
be able to delete applier service object in case of background
rollback. */
Wsrep_applier_service* wsrep_applier_service;
/* wait_for_commit struct for binlog group commit */
wait_for_commit wsrep_wfc;
#endif /* WITH_WSREP */
/* Handling of timeouts for commands */
......
......@@ -171,7 +171,8 @@ static struct wsrep_service_st wsrep_handler = {
wsrep_handle_SR_rollback,
wsrep_thd_skip_locking,
wsrep_get_sr_table_name,
wsrep_get_debug
wsrep_get_debug,
wsrep_commit_ordered
};
static struct thd_specifics_service_st thd_specifics_handler=
......
......@@ -18,6 +18,7 @@
#include "wsrep_binlog.h"
#include "wsrep_priv.h"
#include "log.h"
#include "slave.h"
#include "log_event.h"
#include "wsrep_applier.h"
......@@ -371,3 +372,66 @@ int wsrep_write_dummy_event(THD *orig_thd, const char *msg)
{
return 0;
}
bool wsrep_commit_will_write_binlog(THD *thd)
{
return (!wsrep_emulate_bin_log && /* binlog enabled*/
(wsrep_thd_is_local(thd) || /* local thd*/
(thd->wsrep_applier_service && /* applier and log-slave-updates */
opt_log_slave_updates)));
}
/*
The last THD/commit_for_wait registered for group commit.
*/
static wait_for_commit *commit_order_tail= NULL;
void wsrep_register_for_group_commit(THD *thd)
{
DBUG_ENTER("wsrep_register_for_group_commit");
if (wsrep_emulate_bin_log)
{
/* Binlog is off, no need to maintain group commit queue */
DBUG_VOID_RETURN;
}
DBUG_ASSERT(thd->wsrep_trx().state() == wsrep::transaction::s_committing);
wait_for_commit *wfc= thd->wait_for_commit_ptr= &thd->wsrep_wfc;
mysql_mutex_lock(&LOCK_wsrep_group_commit);
if (commit_order_tail)
{
wfc->register_wait_for_prior_commit(commit_order_tail);
}
commit_order_tail= thd->wait_for_commit_ptr;
mysql_mutex_unlock(&LOCK_wsrep_group_commit);
/*
Now we have queued for group commit. If the commit will go
through TC log_and_order(), the commit ordering is done
by TC group commit. Otherwise the wait for prior
commits to complete is done in ha_commit_one_phase().
*/
DBUG_VOID_RETURN;
}
void wsrep_unregister_from_group_commit(THD *thd)
{
DBUG_ASSERT(thd->wsrep_trx().state() == wsrep::transaction::s_ordered_commit);
wait_for_commit *wfc= thd->wait_for_commit_ptr;
if (wfc)
{
mysql_mutex_lock(&LOCK_wsrep_group_commit);
wfc->unregister_wait_for_prior_commit();
thd->wakeup_subsequent_commits(0);
/* The last one queued for group commit has completed commit, it is
safe to set tail to NULL. */
if (wfc == commit_order_tail)
commit_order_tail= NULL;
mysql_mutex_unlock(&LOCK_wsrep_group_commit);
thd->wait_for_commit_ptr= NULL;
}
}
......@@ -74,4 +74,32 @@ int wsrep_write_dummy_event(THD* thd, const char *msg);
void wsrep_register_binlog_handler(THD *thd, bool trx);
/**
Return true if committing THD will write to binlog during commit.
This is the case for:
- Local THD, binlog is open
- Replaying THD, binlog is open
- Applier THD, log-slave-updates is enabled
*/
bool wsrep_commit_will_write_binlog(THD *thd);
/**
Register THD for group commit. The wsrep_trx must be in committing state,
i.e. the call must be done after wsrep_before_commit() but before
commit order is released.
This call will release commit order critical section if it is
determined that the commit will go through binlog group commit.
*/
void wsrep_register_for_group_commit(THD *thd);
/**
Deregister THD from group commit. The wsrep_trx must be in committing state,
as for wsrep_register_for_group_commit() above.
This call must be used only for THDs which will not go through
binlog group commit.
*/
void wsrep_unregister_from_group_commit(THD *thd);
#endif /* WSREP_BINLOG_H */
......@@ -132,3 +132,6 @@ const char* wsrep_get_sr_table_name()
my_bool wsrep_get_debug()
{ return 0;}
void wsrep_commit_ordered(THD* )
{ }
......@@ -448,6 +448,7 @@ Wsrep_applier_service::~Wsrep_applier_service()
m_thd->wsrep_cs().after_command_after_result();
m_thd->wsrep_cs().close();
m_thd->wsrep_cs().cleanup();
m_thd->wsrep_applier_service= NULL;
}
int Wsrep_applier_service::apply_write_set(const wsrep::ws_meta& ws_meta,
......
......@@ -148,6 +148,7 @@ mysql_cond_t COND_wsrep_slave_threads;
mysql_mutex_t LOCK_wsrep_cluster_config;
mysql_mutex_t LOCK_wsrep_desync;
mysql_mutex_t LOCK_wsrep_config_state;
mysql_mutex_t LOCK_wsrep_group_commit;
mysql_mutex_t LOCK_wsrep_SR_pool;
mysql_mutex_t LOCK_wsrep_SR_store;
......@@ -161,6 +162,7 @@ PSI_mutex_key
key_LOCK_wsrep_sst_thread, key_LOCK_wsrep_sst_init,
key_LOCK_wsrep_slave_threads, key_LOCK_wsrep_desync,
key_LOCK_wsrep_config_state, key_LOCK_wsrep_cluster_config,
key_LOCK_wsrep_group_commit,
key_LOCK_wsrep_SR_pool,
key_LOCK_wsrep_SR_store,
key_LOCK_wsrep_thd_queue;
......@@ -185,6 +187,7 @@ static PSI_mutex_info wsrep_mutexes[]=
{ &key_LOCK_wsrep_cluster_config, "LOCK_wsrep_cluster_config", PSI_FLAG_GLOBAL},
{ &key_LOCK_wsrep_desync, "LOCK_wsrep_desync", PSI_FLAG_GLOBAL},
{ &key_LOCK_wsrep_config_state, "LOCK_wsrep_config_state", PSI_FLAG_GLOBAL},
{ &key_LOCK_wsrep_group_commit, "LOCK_wsrep_group_commit", PSI_FLAG_GLOBAL},
{ &key_LOCK_wsrep_SR_pool, "LOCK_wsrep_SR_pool", PSI_FLAG_GLOBAL},
{ &key_LOCK_wsrep_SR_store, "LOCK_wsrep_SR_store", PSI_FLAG_GLOBAL}
};
......@@ -766,6 +769,7 @@ void wsrep_thr_init()
mysql_mutex_init(key_LOCK_wsrep_cluster_config, &LOCK_wsrep_cluster_config, MY_MUTEX_INIT_FAST);
mysql_mutex_init(key_LOCK_wsrep_desync, &LOCK_wsrep_desync, MY_MUTEX_INIT_FAST);
mysql_mutex_init(key_LOCK_wsrep_config_state, &LOCK_wsrep_config_state, MY_MUTEX_INIT_FAST);
mysql_mutex_init(key_LOCK_wsrep_group_commit, &LOCK_wsrep_group_commit, MY_MUTEX_INIT_FAST);
mysql_mutex_init(key_LOCK_wsrep_SR_pool,
&LOCK_wsrep_SR_pool, MY_MUTEX_INIT_FAST);
mysql_mutex_init(key_LOCK_wsrep_SR_store,
......@@ -870,6 +874,7 @@ void wsrep_thr_deinit()
mysql_mutex_destroy(&LOCK_wsrep_cluster_config);
mysql_mutex_destroy(&LOCK_wsrep_desync);
mysql_mutex_destroy(&LOCK_wsrep_config_state);
mysql_mutex_destroy(&LOCK_wsrep_group_commit);
mysql_mutex_destroy(&LOCK_wsrep_SR_pool);
mysql_mutex_destroy(&LOCK_wsrep_SR_store);
......@@ -2456,20 +2461,6 @@ bool wsrep_provider_is_SR_capable()
return Wsrep_server_state::has_capability(wsrep::provider::capability::streaming);
}
int wsrep_ordered_commit_if_no_binlog(THD* thd, bool all)
{
if (((wsrep_thd_is_local(thd) &&
(WSREP_EMULATE_BINLOG(thd) || !thd->variables.sql_log_bin)) ||
(wsrep_thd_is_applying(thd) && !opt_log_slave_updates))
&& wsrep_thd_trx_seqno(thd) > 0)
{
wsrep_apply_error unused;
return wsrep_ordered_commit(thd, all, unused);
}
return 0;
}
int wsrep_thd_retry_counter(const THD *thd)
{
return thd->wsrep_retry_counter;
......
......@@ -306,6 +306,7 @@ extern mysql_mutex_t LOCK_wsrep_desync;
extern mysql_mutex_t LOCK_wsrep_SR_pool;
extern mysql_mutex_t LOCK_wsrep_SR_store;
extern mysql_mutex_t LOCK_wsrep_config_state;
extern mysql_mutex_t LOCK_wsrep_group_commit;
extern my_bool wsrep_emulate_bin_log;
extern int wsrep_to_isolation;
#ifdef GTID_SUPPORT
......@@ -433,19 +434,6 @@ bool wsrep_node_is_synced();
*/
bool wsrep_provider_is_SR_capable();
/**
* Mark current commit ordered if binlogging is not enabled.
*
* The purpose of this function is to leave commit order critical
* section if binlog is not enabled.
*
* The function can be called from inside storage engine during commit.
* Binlog options are checked inside the function.
*
* @return Zero in case of success, non-zero in case of failure.
*/
int wsrep_ordered_commit_if_no_binlog(THD*, bool);
/**
* Initialize WSREP server instance.
*
......
......@@ -16,6 +16,7 @@
#include "my_global.h"
#include "wsrep_api.h"
#include "wsrep_server_state.h"
#include "wsrep_binlog.h" /* init/deinit group commit */
mysql_mutex_t LOCK_wsrep_server_state;
mysql_cond_t COND_wsrep_server_state;
......@@ -45,9 +46,10 @@ Wsrep_server_state::Wsrep_server_state(const std::string& name,
, m_mutex(LOCK_wsrep_server_state)
, m_cond(COND_wsrep_server_state)
, m_service(*this)
{
{ }
}
Wsrep_server_state::~Wsrep_server_state()
{ }
void Wsrep_server_state::init_once(const std::string& name,
const std::string& incoming_address,
......
......@@ -49,6 +49,7 @@ class Wsrep_server_state : public wsrep::server_state
{
return (get_provider().capabilities() & capability);
}
private:
Wsrep_server_state(const std::string& name,
const std::string& incoming_address,
......@@ -56,11 +57,12 @@ class Wsrep_server_state : public wsrep::server_state
const std::string& working_dir,
const wsrep::gtid& initial_position,
int max_protocol_version);
~Wsrep_server_state();
Wsrep_mutex m_mutex;
Wsrep_condition_variable m_cond;
Wsrep_server_service m_service;
static Wsrep_server_state* m_instance;
};
#endif // WSREP_SERVER_STATE_H
......@@ -21,7 +21,7 @@
#include "wsrep_applier.h" /* wsrep_apply_error */
#include "wsrep_xid.h"
#include "wsrep_thd.h"
#include "wsrep_binlog.h" /* register/deregister group commit */
#include "my_dbug.h"
class THD;
......@@ -243,7 +243,6 @@ static inline int wsrep_after_prepare(THD* thd, bool all)
DBUG_RETURN(ret);
}
/*
Called before the transaction is committed.
......@@ -265,6 +264,7 @@ static inline int wsrep_before_commit(THD* thd, bool all)
DBUG_ASSERT(!thd->wsrep_trx().ws_meta().gtid().is_undefined());
wsrep_xid_init(&thd->wsrep_xid,
thd->wsrep_trx().ws_meta().gtid());
wsrep_register_for_group_commit(thd);
}
DBUG_RETURN(ret);
}
......@@ -305,10 +305,14 @@ static inline int wsrep_after_commit(THD* thd, bool all)
(long long)wsrep_thd_trx_seqno(thd),
wsrep_has_changes(thd));
DBUG_ASSERT(wsrep_run_commit_hook(thd, all));
DBUG_RETURN((thd->wsrep_trx().state() == wsrep::transaction::s_committing
? thd->wsrep_cs().ordered_commit() : 0) ||
(thd->wsrep_xid.null(),
thd->wsrep_cs().after_commit()));
int ret= 0;
if (thd->wsrep_trx().state() == wsrep::transaction::s_committing)
{
ret= thd->wsrep_cs().ordered_commit();
}
wsrep_unregister_from_group_commit(thd);
thd->wsrep_xid.null();
DBUG_RETURN(ret || thd->wsrep_cs().after_commit());
}
/*
......
......@@ -1568,6 +1568,16 @@ trx_commit(
}
trx_commit_low(trx, mtr);
#ifdef WITH_WSREP
/* Serialization history has been written and the
transaction is committed in memory, which makes
this commit ordered. Release commit order critical
section. */
if (wsrep_on(trx->mysql_thd))
{
wsrep_commit_ordered(trx->mysql_thd);
}
#endif /* WITH_WSREP */
}
/****************************************************************//**
......
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment