Merge MDEV-4506: Parallel replication into 10.0-base.
This commit is contained in:
commit
cb86ce60b9
@ -694,6 +694,41 @@ void *thd_get_ha_data(const MYSQL_THD thd, const struct handlerton *hton);
|
||||
*/
|
||||
void thd_set_ha_data(MYSQL_THD thd, const struct handlerton *hton,
|
||||
const void *ha_data);
|
||||
|
||||
|
||||
/**
|
||||
Signal that the first part of handler commit is finished, and that the
|
||||
committed transaction is now visible and has fixed commit ordering with
|
||||
respect to other transactions. The commit need _not_ be durable yet, and
|
||||
typically will not be when this call makes sense.
|
||||
|
||||
This call is optional, if the storage engine does not call it the upper
|
||||
layer will after the handler commit() method is done. However, the storage
|
||||
engine may choose to call it itself to increase the possibility for group
|
||||
commit.
|
||||
|
||||
In-order parallel replication uses this to apply different transaction in
|
||||
parallel, but delay the commits of later transactions until earlier
|
||||
transactions have committed first, thus achieving increased performance on
|
||||
multi-core systems while still preserving full transaction consistency.
|
||||
|
||||
The storage engine can call this from within the commit() method, typically
|
||||
after the commit record has been written to the transaction log, but before
|
||||
the log has been fsync()'ed. This will allow the next replicated transaction
|
||||
to proceed to commit before the first one has done fsync() or similar. Thus,
|
||||
it becomes possible for multiple sequential replicated transactions to share
|
||||
a single fsync() inside the engine in group commit.
|
||||
|
||||
Note that this method should _not_ be called from within the commit_ordered()
|
||||
method, or any other place in the storage engine. When commit_ordered() is
|
||||
used (typically when binlog is enabled), the transaction coordinator takes
|
||||
care of this and makes group commit in the storage engine possible without
|
||||
any other action needed on the part of the storage engine. This function
|
||||
thd_wakeup_subsequent_commits() is only needed when no transaction
|
||||
coordinator is used, meaning a single storage engine and no binary log.
|
||||
*/
|
||||
void thd_wakeup_subsequent_commits(MYSQL_THD thd, int wakeup_error);
|
||||
|
||||
#ifdef __cplusplus
|
||||
}
|
||||
#endif
|
||||
|
@ -258,6 +258,7 @@ void mysql_query_cache_invalidate4(void* thd,
|
||||
void *thd_get_ha_data(const void* thd, const struct handlerton *hton);
|
||||
void thd_set_ha_data(void* thd, const struct handlerton *hton,
|
||||
const void *ha_data);
|
||||
void thd_wakeup_subsequent_commits(void* thd, int wakeup_error);
|
||||
struct mysql_event_general
|
||||
{
|
||||
unsigned int event_subclass;
|
||||
|
@ -258,6 +258,7 @@ void mysql_query_cache_invalidate4(void* thd,
|
||||
void *thd_get_ha_data(const void* thd, const struct handlerton *hton);
|
||||
void thd_set_ha_data(void* thd, const struct handlerton *hton,
|
||||
const void *ha_data);
|
||||
void thd_wakeup_subsequent_commits(void* thd, int wakeup_error);
|
||||
#include <mysql/plugin_auth_common.h>
|
||||
typedef struct st_plugin_vio_info
|
||||
{
|
||||
|
@ -211,6 +211,7 @@ void mysql_query_cache_invalidate4(void* thd,
|
||||
void *thd_get_ha_data(const void* thd, const struct handlerton *hton);
|
||||
void thd_set_ha_data(void* thd, const struct handlerton *hton,
|
||||
const void *ha_data);
|
||||
void thd_wakeup_subsequent_commits(void* thd, int wakeup_error);
|
||||
enum enum_ftparser_mode
|
||||
{
|
||||
MYSQL_FTPARSER_SIMPLE_MODE= 0,
|
||||
|
@ -63,5 +63,5 @@ if (`SELECT '$binlog_limit' <> ''`)
|
||||
|
||||
--replace_result $MYSQLTEST_VARDIR MYSQLTEST_VARDIR $_binlog_start <binlog_start>
|
||||
--replace_column 2 # 4 # 5 #
|
||||
--replace_regex /\/\* xid=.* \*\//\/* XID *\// /table_id: [0-9]+/table_id: #/ /file_id=[0-9]+/file_id=#/ /block_len=[0-9]+/block_len=#/ /Server ver:.*$/SERVER_VERSION, BINLOG_VERSION/ /GTID [0-9]+-[0-9]+-[0-9]+/GTID #-#-#/ /\[([0-9]-[0-9]-[0-9]+)\]/[#-#-#]/
|
||||
--replace_regex /\/\* xid=.* \*\//\/* XID *\// /table_id: [0-9]+/table_id: #/ /file_id=[0-9]+/file_id=#/ /block_len=[0-9]+/block_len=#/ /Server ver:.*$/SERVER_VERSION, BINLOG_VERSION/ /GTID [0-9]+-[0-9]+-[0-9]+/GTID #-#-#/ /\[([0-9]-[0-9]-[0-9]+)\]/[#-#-#]/ /cid=[0-9]+/cid=#/
|
||||
--eval $_statement
|
||||
|
@ -41,6 +41,17 @@ The following options may be given as the first argument:
|
||||
Type of BINLOG_CHECKSUM_ALG. Include checksum for log
|
||||
events in the binary log. Possible values are NONE and
|
||||
CRC32; default is NONE.
|
||||
--binlog-commit-wait-count=#
|
||||
If non-zero, binlog write will wait at most
|
||||
binlog_commit_wait_usec microseconds for at least this
|
||||
many commits to queue up for group commit to the binlog.
|
||||
This can reduce I/O on the binlog and provide increased
|
||||
opportunity for parallel apply on the slave, but too high
|
||||
a value will decrease commit throughput.
|
||||
--binlog-commit-wait-usec=#
|
||||
Maximum time, in microseconds, to wait for more commits
|
||||
to queue up for binlog group commit. Only takes effect if
|
||||
the value of binlog_commit_wait_count is non-zero.
|
||||
--binlog-direct-non-transactional-updates
|
||||
Causes updates to non-transactional engines using
|
||||
statement format to be written directly to binary log.
|
||||
@ -783,6 +794,16 @@ The following options may be given as the first argument:
|
||||
--slave-net-timeout=#
|
||||
Number of seconds to wait for more data from any
|
||||
master/slave connection before aborting the read
|
||||
--slave-parallel-max-queued=#
|
||||
Limit on how much memory SQL threads should use per
|
||||
parallel replication thread when reading ahead in the
|
||||
relay log looking for opportunities for parallel
|
||||
replication. Only used when --slave-parallel-threads > 0.
|
||||
--slave-parallel-threads=#
|
||||
If non-zero, number of threads to spawn to apply in
|
||||
parallel events on the slave that were group-committed on
|
||||
the master or were logged with GTID in different
|
||||
replication domains.
|
||||
--slave-skip-errors=name
|
||||
Tells the slave thread to continue replication when a
|
||||
query event returns an error from the provided list
|
||||
@ -922,6 +943,8 @@ bind-address (No default value)
|
||||
binlog-annotate-row-events FALSE
|
||||
binlog-cache-size 32768
|
||||
binlog-checksum NONE
|
||||
binlog-commit-wait-count 0
|
||||
binlog-commit-wait-usec 100000
|
||||
binlog-direct-non-transactional-updates FALSE
|
||||
binlog-format STATEMENT
|
||||
binlog-optimize-thread-scheduling TRUE
|
||||
@ -1130,6 +1153,8 @@ slave-compressed-protocol FALSE
|
||||
slave-exec-mode STRICT
|
||||
slave-max-allowed-packet 1073741824
|
||||
slave-net-timeout 3600
|
||||
slave-parallel-max-queued 131072
|
||||
slave-parallel-threads 0
|
||||
slave-skip-errors (No default value)
|
||||
slave-sql-verify-checksum TRUE
|
||||
slave-transaction-retries 10
|
||||
|
@ -31,6 +31,6 @@ a
|
||||
1
|
||||
2
|
||||
3
|
||||
InnoDB: Last MySQL binlog file position 0 922, file name ./master-bin.000001
|
||||
InnoDB: Last MySQL binlog file position 0 926, file name ./master-bin.000001
|
||||
SET DEBUG_SYNC= 'RESET';
|
||||
DROP TABLE t1;
|
||||
|
@ -32,6 +32,6 @@ a
|
||||
1
|
||||
2
|
||||
3
|
||||
InnoDB: Last MySQL binlog file position 0 922, file name ./master-bin.000001
|
||||
InnoDB: Last MySQL binlog file position 0 926, file name ./master-bin.000001
|
||||
SET DEBUG_SYNC= 'RESET';
|
||||
DROP TABLE t1;
|
||||
|
@ -61,6 +61,7 @@ wait/synch/mutex/sql/LOCK_prepared_stmt_count
|
||||
wait/synch/mutex/sql/LOCK_prepare_ordered
|
||||
wait/synch/mutex/sql/LOCK_rpl_gtid_state
|
||||
wait/synch/mutex/sql/LOCK_rpl_status
|
||||
wait/synch/mutex/sql/LOCK_rpl_thread_pool
|
||||
wait/synch/mutex/sql/LOCK_server_started
|
||||
wait/synch/mutex/sql/LOCK_slave_list
|
||||
wait/synch/mutex/sql/LOCK_slave_state
|
||||
@ -87,7 +88,6 @@ wait/synch/mutex/sql/Query_cache::structure_guard_mutex
|
||||
wait/synch/mutex/sql/Relay_log_info::data_lock
|
||||
wait/synch/mutex/sql/Relay_log_info::log_space_lock
|
||||
wait/synch/mutex/sql/Relay_log_info::run_lock
|
||||
wait/synch/mutex/sql/Relay_log_info::sleep_lock
|
||||
wait/synch/mutex/sql/Slave_reporting_capability::err_lock
|
||||
wait/synch/mutex/sql/TABLE_SHARE::LOCK_ha_data
|
||||
wait/synch/mutex/sql/THD::LOCK_thd_data
|
||||
@ -122,8 +122,10 @@ wait/synch/cond/mysys/COND_alarm
|
||||
wait/synch/cond/mysys/my_thread_var::suspend
|
||||
wait/synch/cond/mysys/THR_COND_threads
|
||||
wait/synch/cond/sql/COND_flush_thread_cache
|
||||
wait/synch/cond/sql/COND_prepare_ordered
|
||||
wait/synch/cond/sql/COND_queue_state
|
||||
wait/synch/cond/sql/COND_rpl_status
|
||||
wait/synch/cond/sql/COND_rpl_thread_pool
|
||||
wait/synch/cond/sql/COND_server_started
|
||||
wait/synch/cond/sql/COND_thread_cache
|
||||
wait/synch/cond/sql/COND_thread_count
|
||||
@ -143,7 +145,6 @@ wait/synch/cond/sql/MYSQL_RELAY_LOG::update_cond
|
||||
wait/synch/cond/sql/Query_cache::COND_cache_status_changed
|
||||
wait/synch/cond/sql/Relay_log_info::data_cond
|
||||
wait/synch/cond/sql/Relay_log_info::log_space_cond
|
||||
wait/synch/cond/sql/Relay_log_info::sleep_cond
|
||||
wait/synch/cond/sql/Relay_log_info::start_cond
|
||||
wait/synch/cond/sql/Relay_log_info::stop_cond
|
||||
wait/synch/cond/sql/THD::COND_wakeup_ready
|
||||
|
@ -38,14 +38,14 @@ order by name limit 10;
|
||||
NAME ENABLED TIMED
|
||||
wait/synch/cond/sql/COND_flush_thread_cache YES YES
|
||||
wait/synch/cond/sql/COND_manager YES YES
|
||||
wait/synch/cond/sql/COND_parallel_entry YES YES
|
||||
wait/synch/cond/sql/COND_prepare_ordered YES YES
|
||||
wait/synch/cond/sql/COND_queue_state YES YES
|
||||
wait/synch/cond/sql/COND_rpl_status YES YES
|
||||
wait/synch/cond/sql/COND_rpl_thread YES YES
|
||||
wait/synch/cond/sql/COND_rpl_thread_pool YES YES
|
||||
wait/synch/cond/sql/COND_server_started YES YES
|
||||
wait/synch/cond/sql/COND_thread_cache YES YES
|
||||
wait/synch/cond/sql/COND_thread_count YES YES
|
||||
wait/synch/cond/sql/Delayed_insert::cond YES YES
|
||||
wait/synch/cond/sql/Delayed_insert::cond_client YES YES
|
||||
wait/synch/cond/sql/Event_scheduler::COND_state YES YES
|
||||
select * from performance_schema.setup_instruments
|
||||
where name='Wait';
|
||||
select * from performance_schema.setup_instruments
|
||||
|
@ -8,6 +8,7 @@ a
|
||||
1
|
||||
2
|
||||
3
|
||||
SET GLOBAL debug_dbug= '+d,incident_database_resync_on_replace,*';
|
||||
REPLACE INTO t1 VALUES (4);
|
||||
SELECT * FROM t1;
|
||||
a
|
||||
|
267
mysql-test/suite/rpl/r/rpl_parallel.result
Normal file
267
mysql-test/suite/rpl/r/rpl_parallel.result
Normal file
@ -0,0 +1,267 @@
|
||||
include/rpl_init.inc [topology=1->2]
|
||||
SET @old_parallel_threads=@@GLOBAL.slave_parallel_threads;
|
||||
SET GLOBAL slave_parallel_threads=10;
|
||||
ERROR HY000: This operation cannot be performed as you have a running slave ''; run STOP SLAVE '' first
|
||||
include/stop_slave.inc
|
||||
SET GLOBAL slave_parallel_threads=10;
|
||||
CHANGE MASTER TO master_use_gtid=slave_pos;
|
||||
include/start_slave.inc
|
||||
*** Test long-running query in domain 1 can run in parallel with short queries in domain 0 ***
|
||||
CREATE TABLE t1 (a int PRIMARY KEY) ENGINE=MyISAM;
|
||||
CREATE TABLE t2 (a int PRIMARY KEY) ENGINE=InnoDB;
|
||||
INSERT INTO t1 VALUES (1);
|
||||
INSERT INTO t2 VALUES (1);
|
||||
LOCK TABLE t1 WRITE;
|
||||
SET gtid_domain_id=1;
|
||||
INSERT INTO t1 VALUES (2);
|
||||
SET gtid_domain_id=0;
|
||||
INSERT INTO t2 VALUES (2);
|
||||
INSERT INTO t2 VALUES (3);
|
||||
BEGIN;
|
||||
INSERT INTO t2 VALUES (4);
|
||||
INSERT INTO t2 VALUES (5);
|
||||
COMMIT;
|
||||
INSERT INTO t2 VALUES (6);
|
||||
SELECT * FROM t2 ORDER by a;
|
||||
a
|
||||
1
|
||||
2
|
||||
3
|
||||
4
|
||||
5
|
||||
6
|
||||
SELECT * FROM t1;
|
||||
a
|
||||
1
|
||||
UNLOCK TABLES;
|
||||
SELECT * FROM t1 ORDER BY a;
|
||||
a
|
||||
1
|
||||
2
|
||||
*** Test two transactions in different domains committed in opposite order on slave but in a single group commit. ***
|
||||
include/stop_slave.inc
|
||||
SET sql_log_bin=0;
|
||||
CREATE FUNCTION foo(x INT, d1 VARCHAR(500), d2 VARCHAR(500))
|
||||
RETURNS INT DETERMINISTIC
|
||||
BEGIN
|
||||
RETURN x;
|
||||
END
|
||||
||
|
||||
SET sql_log_bin=1;
|
||||
SET @old_format= @@SESSION.binlog_format;
|
||||
SET binlog_format='statement';
|
||||
SET gtid_domain_id=1;
|
||||
INSERT INTO t2 VALUES (foo(10,
|
||||
'commit_before_enqueue SIGNAL ready1 WAIT_FOR cont1',
|
||||
'commit_after_release_LOCK_prepare_ordered SIGNAL ready2'));
|
||||
FLUSH LOGS;
|
||||
SET sql_log_bin=0;
|
||||
CREATE FUNCTION foo(x INT, d1 VARCHAR(500), d2 VARCHAR(500))
|
||||
RETURNS INT DETERMINISTIC
|
||||
BEGIN
|
||||
IF d1 != '' THEN
|
||||
SET debug_sync = d1;
|
||||
END IF;
|
||||
IF d2 != '' THEN
|
||||
SET debug_sync = d2;
|
||||
END IF;
|
||||
RETURN x;
|
||||
END
|
||||
||
|
||||
SET sql_log_bin=1;
|
||||
SET @old_format=@@GLOBAL.binlog_format;
|
||||
SET GLOBAL binlog_format=statement;
|
||||
SET GLOBAL slave_parallel_threads=0;
|
||||
SET GLOBAL slave_parallel_threads=10;
|
||||
include/start_slave.inc
|
||||
SET debug_sync='now WAIT_FOR ready1';
|
||||
SET gtid_domain_id=2;
|
||||
INSERT INTO t2 VALUES (foo(11,
|
||||
'commit_before_enqueue SIGNAL ready3 WAIT_FOR cont3',
|
||||
'commit_after_release_LOCK_prepare_ordered SIGNAL ready4 WAIT_FOR cont4'));
|
||||
SET gtid_domain_id=0;
|
||||
SELECT * FROM t2 WHERE a >= 10 ORDER BY a;
|
||||
a
|
||||
10
|
||||
11
|
||||
SET debug_sync='now WAIT_FOR ready3';
|
||||
SET debug_sync='now SIGNAL cont3';
|
||||
SET debug_sync='now WAIT_FOR ready4';
|
||||
SET debug_sync='now SIGNAL cont1';
|
||||
SET debug_sync='now WAIT_FOR ready2';
|
||||
SET debug_sync='now SIGNAL cont4';
|
||||
SELECT * FROM t2 WHERE a >= 10 ORDER BY a;
|
||||
a
|
||||
10
|
||||
11
|
||||
show binlog events in 'slave-bin.000002' from <binlog_start>;
|
||||
Log_name Pos Event_type Server_id End_log_pos Info
|
||||
slave-bin.000002 # Binlog_checkpoint # # slave-bin.000002
|
||||
slave-bin.000002 # Gtid # # BEGIN GTID #-#-# cid=#
|
||||
slave-bin.000002 # Query # # use `test`; INSERT INTO t2 VALUES (foo(11,
|
||||
'commit_before_enqueue SIGNAL ready3 WAIT_FOR cont3',
|
||||
'commit_after_release_LOCK_prepare_ordered SIGNAL ready4 WAIT_FOR cont4'))
|
||||
slave-bin.000002 # Xid # # COMMIT /* XID */
|
||||
slave-bin.000002 # Gtid # # BEGIN GTID #-#-# cid=#
|
||||
slave-bin.000002 # Query # # use `test`; INSERT INTO t2 VALUES (foo(10,
|
||||
'commit_before_enqueue SIGNAL ready1 WAIT_FOR cont1',
|
||||
'commit_after_release_LOCK_prepare_ordered SIGNAL ready2'))
|
||||
slave-bin.000002 # Xid # # COMMIT /* XID */
|
||||
FLUSH LOGS;
|
||||
include/stop_slave.inc
|
||||
SET GLOBAL slave_parallel_threads=0;
|
||||
SET GLOBAL slave_parallel_threads=10;
|
||||
SET debug_sync='RESET';
|
||||
include/start_slave.inc
|
||||
*** Test that group-committed transactions on the master can replicate in parallel on the slave. ***
|
||||
FLUSH LOGS;
|
||||
CREATE TABLE t3 (a INT PRIMARY KEY, b INT) ENGINE=InnoDB;
|
||||
INSERT INTO t3 VALUES (1,1), (3,3), (5,5), (7,7);
|
||||
BEGIN;
|
||||
INSERT INTO t3 VALUES (2,102);
|
||||
BEGIN;
|
||||
INSERT INTO t3 VALUES (4,104);
|
||||
SET debug_sync='commit_after_release_LOCK_prepare_ordered SIGNAL master_queued1 WAIT_FOR master_cont1';
|
||||
SET binlog_format=statement;
|
||||
INSERT INTO t3 VALUES (2, foo(12,
|
||||
'commit_after_release_LOCK_prepare_ordered SIGNAL slave_queued1 WAIT_FOR slave_cont1',
|
||||
''));
|
||||
SET debug_sync='now WAIT_FOR master_queued1';
|
||||
SET debug_sync='commit_after_release_LOCK_prepare_ordered SIGNAL master_queued2';
|
||||
SET binlog_format=statement;
|
||||
INSERT INTO t3 VALUES (4, foo(14,
|
||||
'commit_after_release_LOCK_prepare_ordered SIGNAL slave_queued2',
|
||||
''));
|
||||
SET debug_sync='now WAIT_FOR master_queued2';
|
||||
SET debug_sync='commit_after_release_LOCK_prepare_ordered SIGNAL master_queued3';
|
||||
SET binlog_format=statement;
|
||||
INSERT INTO t3 VALUES (6, foo(16,
|
||||
'group_commit_waiting_for_prior SIGNAL slave_queued3',
|
||||
''));
|
||||
SET debug_sync='now WAIT_FOR master_queued3';
|
||||
SET debug_sync='now SIGNAL master_cont1';
|
||||
SELECT * FROM t3 ORDER BY a;
|
||||
a b
|
||||
1 1
|
||||
2 12
|
||||
3 3
|
||||
4 14
|
||||
5 5
|
||||
6 16
|
||||
7 7
|
||||
show binlog events in 'master-bin.000002' from <binlog_start>;
|
||||
Log_name Pos Event_type Server_id End_log_pos Info
|
||||
master-bin.000002 # Binlog_checkpoint # # master-bin.000002
|
||||
master-bin.000002 # Gtid # # GTID #-#-#
|
||||
master-bin.000002 # Query # # use `test`; CREATE TABLE t3 (a INT PRIMARY KEY, b INT) ENGINE=InnoDB
|
||||
master-bin.000002 # Gtid # # BEGIN GTID #-#-#
|
||||
master-bin.000002 # Query # # use `test`; INSERT INTO t3 VALUES (1,1), (3,3), (5,5), (7,7)
|
||||
master-bin.000002 # Xid # # COMMIT /* XID */
|
||||
master-bin.000002 # Gtid # # BEGIN GTID #-#-# cid=#
|
||||
master-bin.000002 # Query # # use `test`; INSERT INTO t3 VALUES (2, foo(12,
|
||||
'commit_after_release_LOCK_prepare_ordered SIGNAL slave_queued1 WAIT_FOR slave_cont1',
|
||||
''))
|
||||
master-bin.000002 # Xid # # COMMIT /* XID */
|
||||
master-bin.000002 # Gtid # # BEGIN GTID #-#-# cid=#
|
||||
master-bin.000002 # Query # # use `test`; INSERT INTO t3 VALUES (4, foo(14,
|
||||
'commit_after_release_LOCK_prepare_ordered SIGNAL slave_queued2',
|
||||
''))
|
||||
master-bin.000002 # Xid # # COMMIT /* XID */
|
||||
master-bin.000002 # Gtid # # BEGIN GTID #-#-# cid=#
|
||||
master-bin.000002 # Query # # use `test`; INSERT INTO t3 VALUES (6, foo(16,
|
||||
'group_commit_waiting_for_prior SIGNAL slave_queued3',
|
||||
''))
|
||||
master-bin.000002 # Xid # # COMMIT /* XID */
|
||||
SET debug_sync='now WAIT_FOR slave_queued3';
|
||||
ROLLBACK;
|
||||
SET debug_sync='now WAIT_FOR slave_queued1';
|
||||
ROLLBACK;
|
||||
SET debug_sync='now WAIT_FOR slave_queued2';
|
||||
SET debug_sync='now SIGNAL slave_cont1';
|
||||
SELECT * FROM t3 ORDER BY a;
|
||||
a b
|
||||
1 1
|
||||
2 12
|
||||
3 3
|
||||
4 14
|
||||
5 5
|
||||
6 16
|
||||
7 7
|
||||
show binlog events in 'slave-bin.000003' from <binlog_start>;
|
||||
Log_name Pos Event_type Server_id End_log_pos Info
|
||||
slave-bin.000003 # Binlog_checkpoint # # slave-bin.000003
|
||||
slave-bin.000003 # Gtid # # GTID #-#-#
|
||||
slave-bin.000003 # Query # # use `test`; CREATE TABLE t3 (a INT PRIMARY KEY, b INT) ENGINE=InnoDB
|
||||
slave-bin.000003 # Gtid # # BEGIN GTID #-#-#
|
||||
slave-bin.000003 # Query # # use `test`; INSERT INTO t3 VALUES (1,1), (3,3), (5,5), (7,7)
|
||||
slave-bin.000003 # Xid # # COMMIT /* XID */
|
||||
slave-bin.000003 # Gtid # # BEGIN GTID #-#-# cid=#
|
||||
slave-bin.000003 # Query # # use `test`; INSERT INTO t3 VALUES (2, foo(12,
|
||||
'commit_after_release_LOCK_prepare_ordered SIGNAL slave_queued1 WAIT_FOR slave_cont1',
|
||||
''))
|
||||
slave-bin.000003 # Xid # # COMMIT /* XID */
|
||||
slave-bin.000003 # Gtid # # BEGIN GTID #-#-# cid=#
|
||||
slave-bin.000003 # Query # # use `test`; INSERT INTO t3 VALUES (4, foo(14,
|
||||
'commit_after_release_LOCK_prepare_ordered SIGNAL slave_queued2',
|
||||
''))
|
||||
slave-bin.000003 # Xid # # COMMIT /* XID */
|
||||
slave-bin.000003 # Gtid # # BEGIN GTID #-#-# cid=#
|
||||
slave-bin.000003 # Query # # use `test`; INSERT INTO t3 VALUES (6, foo(16,
|
||||
'group_commit_waiting_for_prior SIGNAL slave_queued3',
|
||||
''))
|
||||
slave-bin.000003 # Xid # # COMMIT /* XID */
|
||||
*** Test STOP SLAVE in parallel mode ***
|
||||
include/stop_slave.inc
|
||||
SET binlog_direct_non_transactional_updates=0;
|
||||
SET sql_log_bin=0;
|
||||
CALL mtr.add_suppression("Statement is unsafe because it accesses a non-transactional table after accessing a transactional table within the same transaction");
|
||||
SET sql_log_bin=1;
|
||||
BEGIN;
|
||||
INSERT INTO t2 VALUES (20);
|
||||
INSERT INTO t1 VALUES (20);
|
||||
INSERT INTO t2 VALUES (21);
|
||||
INSERT INTO t3 VALUES (20, 20);
|
||||
COMMIT;
|
||||
INSERT INTO t3 VALUES(21, 21);
|
||||
INSERT INTO t3 VALUES(22, 22);
|
||||
SET binlog_format=@old_format;
|
||||
BEGIN;
|
||||
INSERT INTO t2 VALUES (21);
|
||||
START SLAVE;
|
||||
STOP SLAVE;
|
||||
ROLLBACK;
|
||||
include/wait_for_slave_to_stop.inc
|
||||
SELECT * FROM t1 WHERE a >= 20 ORDER BY a;
|
||||
a
|
||||
20
|
||||
SELECT * FROM t2 WHERE a >= 20 ORDER BY a;
|
||||
a
|
||||
20
|
||||
21
|
||||
SELECT * FROM t3 WHERE a >= 20 ORDER BY a;
|
||||
a b
|
||||
20 20
|
||||
include/start_slave.inc
|
||||
SELECT * FROM t1 WHERE a >= 20 ORDER BY a;
|
||||
a
|
||||
20
|
||||
SELECT * FROM t2 WHERE a >= 20 ORDER BY a;
|
||||
a
|
||||
20
|
||||
21
|
||||
SELECT * FROM t3 WHERE a >= 20 ORDER BY a;
|
||||
a b
|
||||
20 20
|
||||
21 21
|
||||
22 22
|
||||
include/stop_slave.inc
|
||||
SET GLOBAL binlog_format=@old_format;
|
||||
SET GLOBAL slave_parallel_threads=0;
|
||||
SET GLOBAL slave_parallel_threads=10;
|
||||
include/start_slave.inc
|
||||
include/stop_slave.inc
|
||||
SET GLOBAL slave_parallel_threads=@old_parallel_threads;
|
||||
include/start_slave.inc
|
||||
DROP function foo;
|
||||
DROP TABLE t1,t2,t3;
|
||||
include/rpl_end.inc
|
@ -1 +0,0 @@
|
||||
--loose-debug=+d,incident_database_resync_on_replace
|
@ -7,12 +7,19 @@ CREATE TABLE t1 (a INT);
|
||||
INSERT INTO t1 VALUES (1),(2),(3);
|
||||
SELECT * FROM t1;
|
||||
|
||||
let $debug_save= `SELECT @@GLOBAL.debug`;
|
||||
SET GLOBAL debug_dbug= '+d,incident_database_resync_on_replace,*';
|
||||
|
||||
# This will generate an incident log event and store it in the binary
|
||||
# log before the replace statement.
|
||||
REPLACE INTO t1 VALUES (4);
|
||||
--save_master_pos
|
||||
SELECT * FROM t1;
|
||||
|
||||
--disable_query_log
|
||||
eval SET GLOBAL debug_dbug= '$debug_save';
|
||||
--enable_query_log
|
||||
|
||||
connection slave;
|
||||
# Wait until SQL thread stops with error LOST_EVENT on master
|
||||
call mtr.add_suppression("Slave SQL.*The incident LOST_EVENTS occured on the master.* 1590");
|
||||
|
353
mysql-test/suite/rpl/t/rpl_parallel.test
Normal file
353
mysql-test/suite/rpl/t/rpl_parallel.test
Normal file
@ -0,0 +1,353 @@
|
||||
--source include/have_innodb.inc
|
||||
--source include/have_debug.inc
|
||||
--source include/have_debug_sync.inc
|
||||
--let $rpl_topology=1->2
|
||||
--source include/rpl_init.inc
|
||||
|
||||
# Test various aspects of parallel replication.
|
||||
|
||||
--connection server_2
|
||||
SET @old_parallel_threads=@@GLOBAL.slave_parallel_threads;
|
||||
--error ER_SLAVE_MUST_STOP
|
||||
SET GLOBAL slave_parallel_threads=10;
|
||||
--source include/stop_slave.inc
|
||||
SET GLOBAL slave_parallel_threads=10;
|
||||
CHANGE MASTER TO master_use_gtid=slave_pos;
|
||||
--source include/start_slave.inc
|
||||
|
||||
|
||||
--echo *** Test long-running query in domain 1 can run in parallel with short queries in domain 0 ***
|
||||
|
||||
--connection server_1
|
||||
CREATE TABLE t1 (a int PRIMARY KEY) ENGINE=MyISAM;
|
||||
CREATE TABLE t2 (a int PRIMARY KEY) ENGINE=InnoDB;
|
||||
INSERT INTO t1 VALUES (1);
|
||||
INSERT INTO t2 VALUES (1);
|
||||
--save_master_pos
|
||||
|
||||
--connection server_2
|
||||
--sync_with_master
|
||||
|
||||
# Block the table t1 to simulate a replicated query taking a long time.
|
||||
--connect (con_temp1,127.0.0.1,root,,test,$SERVER_MYPORT_2,)
|
||||
LOCK TABLE t1 WRITE;
|
||||
|
||||
--connection server_1
|
||||
SET gtid_domain_id=1;
|
||||
# This query will be blocked on the slave until UNLOCK TABLES.
|
||||
INSERT INTO t1 VALUES (2);
|
||||
SET gtid_domain_id=0;
|
||||
# These t2 queries can be replicated in parallel with the prior t1 query, as
|
||||
# they are in a separate replication domain.
|
||||
INSERT INTO t2 VALUES (2);
|
||||
INSERT INTO t2 VALUES (3);
|
||||
BEGIN;
|
||||
INSERT INTO t2 VALUES (4);
|
||||
INSERT INTO t2 VALUES (5);
|
||||
COMMIT;
|
||||
INSERT INTO t2 VALUES (6);
|
||||
|
||||
--connection server_2
|
||||
--let $wait_condition= SELECT COUNT(*) = 6 FROM t2
|
||||
--source include/wait_condition.inc
|
||||
|
||||
SELECT * FROM t2 ORDER by a;
|
||||
|
||||
--connection con_temp1
|
||||
SELECT * FROM t1;
|
||||
UNLOCK TABLES;
|
||||
|
||||
--connection server_2
|
||||
--let $wait_condition= SELECT COUNT(*) = 2 FROM t1
|
||||
--source include/wait_condition.inc
|
||||
|
||||
SELECT * FROM t1 ORDER BY a;
|
||||
|
||||
|
||||
--echo *** Test two transactions in different domains committed in opposite order on slave but in a single group commit. ***
|
||||
--connection server_2
|
||||
--source include/stop_slave.inc
|
||||
|
||||
--connection server_1
|
||||
# Use a stored function to inject a debug_sync into the appropriate THD.
|
||||
# The function does nothing on the master, and on the slave it injects the
|
||||
# desired debug_sync action(s).
|
||||
SET sql_log_bin=0;
|
||||
--delimiter ||
|
||||
CREATE FUNCTION foo(x INT, d1 VARCHAR(500), d2 VARCHAR(500))
|
||||
RETURNS INT DETERMINISTIC
|
||||
BEGIN
|
||||
RETURN x;
|
||||
END
|
||||
||
|
||||
--delimiter ;
|
||||
SET sql_log_bin=1;
|
||||
|
||||
SET @old_format= @@SESSION.binlog_format;
|
||||
SET binlog_format='statement';
|
||||
SET gtid_domain_id=1;
|
||||
INSERT INTO t2 VALUES (foo(10,
|
||||
'commit_before_enqueue SIGNAL ready1 WAIT_FOR cont1',
|
||||
'commit_after_release_LOCK_prepare_ordered SIGNAL ready2'));
|
||||
|
||||
--connection server_2
|
||||
FLUSH LOGS;
|
||||
--source include/wait_for_binlog_checkpoint.inc
|
||||
SET sql_log_bin=0;
|
||||
--delimiter ||
|
||||
CREATE FUNCTION foo(x INT, d1 VARCHAR(500), d2 VARCHAR(500))
|
||||
RETURNS INT DETERMINISTIC
|
||||
BEGIN
|
||||
IF d1 != '' THEN
|
||||
SET debug_sync = d1;
|
||||
END IF;
|
||||
IF d2 != '' THEN
|
||||
SET debug_sync = d2;
|
||||
END IF;
|
||||
RETURN x;
|
||||
END
|
||||
||
|
||||
--delimiter ;
|
||||
SET sql_log_bin=1;
|
||||
SET @old_format=@@GLOBAL.binlog_format;
|
||||
SET GLOBAL binlog_format=statement;
|
||||
# We need to restart all parallel threads for the new global setting to
|
||||
# be copied to the session-level values.
|
||||
SET GLOBAL slave_parallel_threads=0;
|
||||
SET GLOBAL slave_parallel_threads=10;
|
||||
--source include/start_slave.inc
|
||||
|
||||
# First make sure the first insert is ready to commit, but not queued yet.
|
||||
SET debug_sync='now WAIT_FOR ready1';
|
||||
|
||||
--connection server_1
|
||||
SET gtid_domain_id=2;
|
||||
INSERT INTO t2 VALUES (foo(11,
|
||||
'commit_before_enqueue SIGNAL ready3 WAIT_FOR cont3',
|
||||
'commit_after_release_LOCK_prepare_ordered SIGNAL ready4 WAIT_FOR cont4'));
|
||||
SET gtid_domain_id=0;
|
||||
SELECT * FROM t2 WHERE a >= 10 ORDER BY a;
|
||||
|
||||
--connection server_2
|
||||
# Now wait for the second insert to queue itself as the leader, and then
|
||||
# wait for more commits to queue up.
|
||||
SET debug_sync='now WAIT_FOR ready3';
|
||||
SET debug_sync='now SIGNAL cont3';
|
||||
SET debug_sync='now WAIT_FOR ready4';
|
||||
# Now allow the first insert to queue up to participate in group commit.
|
||||
SET debug_sync='now SIGNAL cont1';
|
||||
SET debug_sync='now WAIT_FOR ready2';
|
||||
# Finally allow the second insert to proceed and do the group commit.
|
||||
SET debug_sync='now SIGNAL cont4';
|
||||
|
||||
--let $wait_condition= SELECT COUNT(*) = 2 FROM t2 WHERE a >= 10
|
||||
--source include/wait_condition.inc
|
||||
SELECT * FROM t2 WHERE a >= 10 ORDER BY a;
|
||||
# The two INSERT transactions should have been committed in opposite order,
|
||||
# but in the same group commit (seen by precense of cid=# in the SHOW
|
||||
# BINLOG output).
|
||||
--let $binlog_file= slave-bin.000002
|
||||
--source include/show_binlog_events.inc
|
||||
FLUSH LOGS;
|
||||
--source include/wait_for_binlog_checkpoint.inc
|
||||
|
||||
# Restart all the slave parallel worker threads, to clear all debug_sync actions.
|
||||
--connection server_2
|
||||
--source include/stop_slave.inc
|
||||
SET GLOBAL slave_parallel_threads=0;
|
||||
SET GLOBAL slave_parallel_threads=10;
|
||||
SET debug_sync='RESET';
|
||||
--source include/start_slave.inc
|
||||
|
||||
|
||||
--echo *** Test that group-committed transactions on the master can replicate in parallel on the slave. ***
|
||||
--connection server_1
|
||||
FLUSH LOGS;
|
||||
--source include/wait_for_binlog_checkpoint.inc
|
||||
CREATE TABLE t3 (a INT PRIMARY KEY, b INT) ENGINE=InnoDB;
|
||||
# Create some sentinel rows so that the rows inserted in parallel fall into
|
||||
# separate gaps and do not cause gap lock conflicts.
|
||||
INSERT INTO t3 VALUES (1,1), (3,3), (5,5), (7,7);
|
||||
--save_master_pos
|
||||
--connection server_2
|
||||
--sync_with_master
|
||||
|
||||
# We want to test that the transactions can execute out-of-order on
|
||||
# the slave, but still end up committing in-order, and in a single
|
||||
# group commit.
|
||||
#
|
||||
# The idea is to group-commit three transactions together on the master:
|
||||
# A, B, and C. On the slave, C will execute the insert first, then A,
|
||||
# and then B. But B manages to complete before A has time to commit, so
|
||||
# all three end up committing together.
|
||||
#
|
||||
# So we start by setting up some row locks that will block transactions
|
||||
# A and B from executing, allowing C to run first.
|
||||
|
||||
--connection con_temp1
|
||||
BEGIN;
|
||||
INSERT INTO t3 VALUES (2,102);
|
||||
--connect (con_temp2,127.0.0.1,root,,test,$SERVER_MYPORT_2,)
|
||||
BEGIN;
|
||||
INSERT INTO t3 VALUES (4,104);
|
||||
|
||||
# On the master, queue three INSERT transactions as a single group commit.
|
||||
--connect (con_temp3,127.0.0.1,root,,test,$SERVER_MYPORT_1,)
|
||||
SET debug_sync='commit_after_release_LOCK_prepare_ordered SIGNAL master_queued1 WAIT_FOR master_cont1';
|
||||
SET binlog_format=statement;
|
||||
send INSERT INTO t3 VALUES (2, foo(12,
|
||||
'commit_after_release_LOCK_prepare_ordered SIGNAL slave_queued1 WAIT_FOR slave_cont1',
|
||||
''));
|
||||
|
||||
--connection server_1
|
||||
SET debug_sync='now WAIT_FOR master_queued1';
|
||||
|
||||
--connect (con_temp4,127.0.0.1,root,,test,$SERVER_MYPORT_1,)
|
||||
SET debug_sync='commit_after_release_LOCK_prepare_ordered SIGNAL master_queued2';
|
||||
SET binlog_format=statement;
|
||||
send INSERT INTO t3 VALUES (4, foo(14,
|
||||
'commit_after_release_LOCK_prepare_ordered SIGNAL slave_queued2',
|
||||
''));
|
||||
|
||||
--connection server_1
|
||||
SET debug_sync='now WAIT_FOR master_queued2';
|
||||
|
||||
--connect (con_temp5,127.0.0.1,root,,test,$SERVER_MYPORT_1,)
|
||||
SET debug_sync='commit_after_release_LOCK_prepare_ordered SIGNAL master_queued3';
|
||||
SET binlog_format=statement;
|
||||
send INSERT INTO t3 VALUES (6, foo(16,
|
||||
'group_commit_waiting_for_prior SIGNAL slave_queued3',
|
||||
''));
|
||||
|
||||
--connection server_1
|
||||
SET debug_sync='now WAIT_FOR master_queued3';
|
||||
SET debug_sync='now SIGNAL master_cont1';
|
||||
|
||||
--connection con_temp3
|
||||
REAP;
|
||||
--connection con_temp4
|
||||
REAP;
|
||||
--connection con_temp5
|
||||
REAP;
|
||||
|
||||
--connection server_1
|
||||
SELECT * FROM t3 ORDER BY a;
|
||||
--let $binlog_file= master-bin.000002
|
||||
--source include/show_binlog_events.inc
|
||||
|
||||
# First, wait until insert 3 is ready to queue up for group commit, but is
|
||||
# waiting for insert 2 to commit before it can do so itself.
|
||||
--connection server_2
|
||||
SET debug_sync='now WAIT_FOR slave_queued3';
|
||||
|
||||
# Next, let insert 1 proceed, and allow it to queue up as the group commit
|
||||
# leader, but let it wait for insert 2 to also queue up before proceeding.
|
||||
--connection con_temp1
|
||||
ROLLBACK;
|
||||
--connection server_2
|
||||
SET debug_sync='now WAIT_FOR slave_queued1';
|
||||
|
||||
# Now let insert 2 proceed and queue up.
|
||||
--connection con_temp2
|
||||
ROLLBACK;
|
||||
--connection server_2
|
||||
SET debug_sync='now WAIT_FOR slave_queued2';
|
||||
# And finally, we can let insert 1 proceed and do the group commit with all
|
||||
# three insert transactions together.
|
||||
SET debug_sync='now SIGNAL slave_cont1';
|
||||
|
||||
# Wait for the commit to complete and check that all three transactions
|
||||
# group-committed together (will be seen in the binlog as all three having
|
||||
# cid=# on their GTID event).
|
||||
--let $wait_condition= SELECT COUNT(*) = 3 FROM t3 WHERE a IN (2,4,6)
|
||||
--source include/wait_condition.inc
|
||||
SELECT * FROM t3 ORDER BY a;
|
||||
--let $binlog_file= slave-bin.000003
|
||||
--source include/show_binlog_events.inc
|
||||
|
||||
|
||||
--echo *** Test STOP SLAVE in parallel mode ***
|
||||
--connection server_2
|
||||
--source include/stop_slave.inc
|
||||
|
||||
--connection server_1
|
||||
# Set up a couple of transactions. The first will be blocked halfway
|
||||
# through on a lock, and while it is blocked we initiate STOP SLAVE.
|
||||
# We then test that the halfway-initiated transaction is allowed to
|
||||
# complete, but no subsequent ones.
|
||||
# We have to use statement-based mode and set
|
||||
# binlog_direct_non_transactional_updates=0; otherwise the binlog will
|
||||
# be split into two event groups, one for the MyISAM part and one for the
|
||||
# InnoDB part.
|
||||
SET binlog_direct_non_transactional_updates=0;
|
||||
SET sql_log_bin=0;
|
||||
CALL mtr.add_suppression("Statement is unsafe because it accesses a non-transactional table after accessing a transactional table within the same transaction");
|
||||
SET sql_log_bin=1;
|
||||
BEGIN;
|
||||
INSERT INTO t2 VALUES (20);
|
||||
--disable_warnings
|
||||
INSERT INTO t1 VALUES (20);
|
||||
--disable_warnings
|
||||
INSERT INTO t2 VALUES (21);
|
||||
INSERT INTO t3 VALUES (20, 20);
|
||||
COMMIT;
|
||||
INSERT INTO t3 VALUES(21, 21);
|
||||
INSERT INTO t3 VALUES(22, 22);
|
||||
SET binlog_format=@old_format;
|
||||
--save_master_pos
|
||||
|
||||
# Start a connection that will block the replicated transaction halfway.
|
||||
--connection con_temp1
|
||||
BEGIN;
|
||||
INSERT INTO t2 VALUES (21);
|
||||
|
||||
--connection server_2
|
||||
START SLAVE;
|
||||
# Wait for the MyISAM change to be visible, after which replication will wait
|
||||
# for con_temp1 to roll back.
|
||||
--let $wait_condition= SELECT COUNT(*) = 1 FROM t1 WHERE a=20
|
||||
--source include/wait_condition.inc
|
||||
|
||||
--connection con_temp2
|
||||
# Initiate slave stop. It will have to wait for the current event group
|
||||
# to complete.
|
||||
send STOP SLAVE;
|
||||
|
||||
--connection con_temp1
|
||||
ROLLBACK;
|
||||
|
||||
--connection con_temp2
|
||||
reap;
|
||||
|
||||
--connection server_2
|
||||
--source include/wait_for_slave_to_stop.inc
|
||||
# We should see the first transaction applied, but not the two others.
|
||||
SELECT * FROM t1 WHERE a >= 20 ORDER BY a;
|
||||
SELECT * FROM t2 WHERE a >= 20 ORDER BY a;
|
||||
SELECT * FROM t3 WHERE a >= 20 ORDER BY a;
|
||||
|
||||
--source include/start_slave.inc
|
||||
--sync_with_master
|
||||
SELECT * FROM t1 WHERE a >= 20 ORDER BY a;
|
||||
SELECT * FROM t2 WHERE a >= 20 ORDER BY a;
|
||||
SELECT * FROM t3 WHERE a >= 20 ORDER BY a;
|
||||
|
||||
|
||||
--connection server_2
|
||||
--source include/stop_slave.inc
|
||||
SET GLOBAL binlog_format=@old_format;
|
||||
SET GLOBAL slave_parallel_threads=0;
|
||||
SET GLOBAL slave_parallel_threads=10;
|
||||
--source include/start_slave.inc
|
||||
|
||||
|
||||
--connection server_2
|
||||
--source include/stop_slave.inc
|
||||
SET GLOBAL slave_parallel_threads=@old_parallel_threads;
|
||||
--source include/start_slave.inc
|
||||
|
||||
--connection server_1
|
||||
DROP function foo;
|
||||
DROP TABLE t1,t2,t3;
|
||||
|
||||
--source include/rpl_end.inc
|
@ -0,0 +1,13 @@
|
||||
SET @save_binlog_commit_wait_count= @@GLOBAL.binlog_commit_wait_count;
|
||||
SELECT @@GLOBAL.binlog_commit_wait_count as 'must be zero because of default';
|
||||
must be zero because of default
|
||||
0
|
||||
SELECT @@SESSION.binlog_commit_wait_count as 'no session var';
|
||||
ERROR HY000: Variable 'binlog_commit_wait_count' is a GLOBAL variable
|
||||
SET GLOBAL binlog_commit_wait_count= 0;
|
||||
SET GLOBAL binlog_commit_wait_count= DEFAULT;
|
||||
SET GLOBAL binlog_commit_wait_count= 10;
|
||||
SELECT @@GLOBAL.binlog_commit_wait_count;
|
||||
@@GLOBAL.binlog_commit_wait_count
|
||||
10
|
||||
SET GLOBAL binlog_commit_wait_count = @save_binlog_commit_wait_count;
|
@ -0,0 +1,13 @@
|
||||
SET @save_binlog_commit_wait_usec= @@GLOBAL.binlog_commit_wait_usec;
|
||||
SELECT @@GLOBAL.binlog_commit_wait_usec as 'check default';
|
||||
check default
|
||||
100000
|
||||
SELECT @@SESSION.binlog_commit_wait_usec as 'no session var';
|
||||
ERROR HY000: Variable 'binlog_commit_wait_usec' is a GLOBAL variable
|
||||
SET GLOBAL binlog_commit_wait_usec= 0;
|
||||
SET GLOBAL binlog_commit_wait_usec= DEFAULT;
|
||||
SET GLOBAL binlog_commit_wait_usec= 10000;
|
||||
SELECT @@GLOBAL.binlog_commit_wait_usec;
|
||||
@@GLOBAL.binlog_commit_wait_usec
|
||||
10000
|
||||
SET GLOBAL binlog_commit_wait_usec = @save_binlog_commit_wait_usec;
|
@ -0,0 +1,13 @@
|
||||
SET @save_slave_parallel_max_queued= @@GLOBAL.slave_parallel_max_queued;
|
||||
SELECT @@GLOBAL.slave_parallel_max_queued as 'Check default';
|
||||
Check default
|
||||
131072
|
||||
SELECT @@SESSION.slave_parallel_max_queued as 'no session var';
|
||||
ERROR HY000: Variable 'slave_parallel_max_queued' is a GLOBAL variable
|
||||
SET GLOBAL slave_parallel_max_queued= 0;
|
||||
SET GLOBAL slave_parallel_max_queued= DEFAULT;
|
||||
SET GLOBAL slave_parallel_max_queued= 65536;
|
||||
SELECT @@GLOBAL.slave_parallel_max_queued;
|
||||
@@GLOBAL.slave_parallel_max_queued
|
||||
65536
|
||||
SET GLOBAL slave_parallel_max_queued = @save_slave_parallel_max_queued;
|
@ -0,0 +1,13 @@
|
||||
SET @save_slave_parallel_threads= @@GLOBAL.slave_parallel_threads;
|
||||
SELECT @@GLOBAL.slave_parallel_threads as 'must be zero because of default';
|
||||
must be zero because of default
|
||||
0
|
||||
SELECT @@SESSION.slave_parallel_threads as 'no session var';
|
||||
ERROR HY000: Variable 'slave_parallel_threads' is a GLOBAL variable
|
||||
SET GLOBAL slave_parallel_threads= 0;
|
||||
SET GLOBAL slave_parallel_threads= DEFAULT;
|
||||
SET GLOBAL slave_parallel_threads= 10;
|
||||
SELECT @@GLOBAL.slave_parallel_threads;
|
||||
@@GLOBAL.slave_parallel_threads
|
||||
10
|
||||
SET GLOBAL slave_parallel_threads = @save_slave_parallel_threads;
|
@ -0,0 +1,14 @@
|
||||
--source include/not_embedded.inc
|
||||
|
||||
SET @save_binlog_commit_wait_count= @@GLOBAL.binlog_commit_wait_count;
|
||||
|
||||
SELECT @@GLOBAL.binlog_commit_wait_count as 'must be zero because of default';
|
||||
--error ER_INCORRECT_GLOBAL_LOCAL_VAR
|
||||
SELECT @@SESSION.binlog_commit_wait_count as 'no session var';
|
||||
|
||||
SET GLOBAL binlog_commit_wait_count= 0;
|
||||
SET GLOBAL binlog_commit_wait_count= DEFAULT;
|
||||
SET GLOBAL binlog_commit_wait_count= 10;
|
||||
SELECT @@GLOBAL.binlog_commit_wait_count;
|
||||
|
||||
SET GLOBAL binlog_commit_wait_count = @save_binlog_commit_wait_count;
|
@ -0,0 +1,14 @@
|
||||
--source include/not_embedded.inc
|
||||
|
||||
SET @save_binlog_commit_wait_usec= @@GLOBAL.binlog_commit_wait_usec;
|
||||
|
||||
SELECT @@GLOBAL.binlog_commit_wait_usec as 'check default';
|
||||
--error ER_INCORRECT_GLOBAL_LOCAL_VAR
|
||||
SELECT @@SESSION.binlog_commit_wait_usec as 'no session var';
|
||||
|
||||
SET GLOBAL binlog_commit_wait_usec= 0;
|
||||
SET GLOBAL binlog_commit_wait_usec= DEFAULT;
|
||||
SET GLOBAL binlog_commit_wait_usec= 10000;
|
||||
SELECT @@GLOBAL.binlog_commit_wait_usec;
|
||||
|
||||
SET GLOBAL binlog_commit_wait_usec = @save_binlog_commit_wait_usec;
|
@ -0,0 +1,14 @@
|
||||
--source include/not_embedded.inc
|
||||
|
||||
SET @save_slave_parallel_max_queued= @@GLOBAL.slave_parallel_max_queued;
|
||||
|
||||
SELECT @@GLOBAL.slave_parallel_max_queued as 'Check default';
|
||||
--error ER_INCORRECT_GLOBAL_LOCAL_VAR
|
||||
SELECT @@SESSION.slave_parallel_max_queued as 'no session var';
|
||||
|
||||
SET GLOBAL slave_parallel_max_queued= 0;
|
||||
SET GLOBAL slave_parallel_max_queued= DEFAULT;
|
||||
SET GLOBAL slave_parallel_max_queued= 65536;
|
||||
SELECT @@GLOBAL.slave_parallel_max_queued;
|
||||
|
||||
SET GLOBAL slave_parallel_max_queued = @save_slave_parallel_max_queued;
|
@ -0,0 +1,14 @@
|
||||
--source include/not_embedded.inc
|
||||
|
||||
SET @save_slave_parallel_threads= @@GLOBAL.slave_parallel_threads;
|
||||
|
||||
SELECT @@GLOBAL.slave_parallel_threads as 'must be zero because of default';
|
||||
--error ER_INCORRECT_GLOBAL_LOCAL_VAR
|
||||
SELECT @@SESSION.slave_parallel_threads as 'no session var';
|
||||
|
||||
SET GLOBAL slave_parallel_threads= 0;
|
||||
SET GLOBAL slave_parallel_threads= DEFAULT;
|
||||
SET GLOBAL slave_parallel_threads= 10;
|
||||
SELECT @@GLOBAL.slave_parallel_threads;
|
||||
|
||||
SET GLOBAL slave_parallel_threads = @save_slave_parallel_threads;
|
@ -840,7 +840,6 @@ static int findopt(char *optpat, uint length,
|
||||
{
|
||||
uint count;
|
||||
const struct my_option *opt= *opt_res;
|
||||
my_bool is_prefix= FALSE;
|
||||
DBUG_ENTER("findopt");
|
||||
|
||||
for (count= 0; opt->name; opt++)
|
||||
@ -856,8 +855,6 @@ static int findopt(char *optpat, uint length,
|
||||
/* We only need to know one prev */
|
||||
count= 1;
|
||||
*ffname= opt->name;
|
||||
if (opt->name[length])
|
||||
is_prefix= TRUE;
|
||||
}
|
||||
else if (strcmp(*ffname, opt->name))
|
||||
{
|
||||
|
@ -89,7 +89,7 @@ SET (SQL_SOURCE
|
||||
threadpool_common.cc
|
||||
../sql-common/mysql_async.c
|
||||
my_apc.cc my_apc.h
|
||||
rpl_gtid.cc
|
||||
rpl_gtid.cc rpl_parallel.cc
|
||||
${CMAKE_CURRENT_BINARY_DIR}/sql_builtin.cc
|
||||
${GEN_SOURCES}
|
||||
${MYSYS_LIBWRAP_SOURCE}
|
||||
|
@ -1253,6 +1253,8 @@ int ha_commit_trans(THD *thd, bool all)
|
||||
bool need_prepare_ordered, need_commit_ordered;
|
||||
my_xid xid;
|
||||
DBUG_ENTER("ha_commit_trans");
|
||||
DBUG_PRINT("info",("thd: %p option_bits: %lu all: %d",
|
||||
thd, (ulong) thd->variables.option_bits, all));
|
||||
|
||||
/* Just a random warning to test warnings pushed during autocommit. */
|
||||
DBUG_EXECUTE_IF("warn_during_ha_commit_trans",
|
||||
@ -1312,6 +1314,8 @@ int ha_commit_trans(THD *thd, bool all)
|
||||
/* rw_trans is TRUE when we in a transaction changing data */
|
||||
bool rw_trans= is_real_trans && (rw_ha_count > 0);
|
||||
MDL_request mdl_request;
|
||||
DBUG_PRINT("info", ("is_real_trans: %d rw_trans: %d rw_ha_count: %d",
|
||||
is_real_trans, rw_trans, rw_ha_count));
|
||||
|
||||
if (rw_trans)
|
||||
{
|
||||
@ -1460,8 +1464,11 @@ int ha_commit_one_phase(THD *thd, bool all)
|
||||
transaction.all.ha_list, see why in trans_register_ha()).
|
||||
*/
|
||||
bool is_real_trans=all || thd->transaction.all.ha_list == 0;
|
||||
int res;
|
||||
DBUG_ENTER("ha_commit_one_phase");
|
||||
int res= commit_one_phase_2(thd, all, trans, is_real_trans);
|
||||
if (is_real_trans && (res= thd->wait_for_prior_commit()))
|
||||
DBUG_RETURN(res);
|
||||
res= commit_one_phase_2(thd, all, trans, is_real_trans);
|
||||
DBUG_RETURN(res);
|
||||
}
|
||||
|
||||
@ -1500,7 +1507,10 @@ commit_one_phase_2(THD *thd, bool all, THD_TRANS *trans, bool is_real_trans)
|
||||
}
|
||||
/* Free resources and perform other cleanup even for 'empty' transactions. */
|
||||
if (is_real_trans)
|
||||
{
|
||||
thd->wakeup_subsequent_commits(error);
|
||||
thd->transaction.cleanup();
|
||||
}
|
||||
|
||||
DBUG_RETURN(error);
|
||||
}
|
||||
@ -1575,7 +1585,10 @@ int ha_rollback_trans(THD *thd, bool all)
|
||||
}
|
||||
/* Always cleanup. Even if nht==0. There may be savepoints. */
|
||||
if (is_real_trans)
|
||||
{
|
||||
thd->wakeup_subsequent_commits(error);
|
||||
thd->transaction.cleanup();
|
||||
}
|
||||
if (all)
|
||||
thd->transaction_rollback_request= FALSE;
|
||||
|
||||
|
407
sql/log.cc
407
sql/log.cc
@ -88,6 +88,7 @@ ulong opt_binlog_dbug_fsync_sleep= 0;
|
||||
#endif
|
||||
|
||||
mysql_mutex_t LOCK_prepare_ordered;
|
||||
mysql_cond_t COND_prepare_ordered;
|
||||
mysql_mutex_t LOCK_commit_ordered;
|
||||
|
||||
static ulonglong binlog_status_var_num_commits;
|
||||
@ -5386,7 +5387,7 @@ MYSQL_BIN_LOG::flush_and_set_pending_rows_event(THD *thd,
|
||||
/* Generate a new global transaction ID, and write it to the binlog */
|
||||
bool
|
||||
MYSQL_BIN_LOG::write_gtid_event(THD *thd, bool standalone,
|
||||
bool is_transactional)
|
||||
bool is_transactional, uint64 commit_id)
|
||||
{
|
||||
rpl_gtid gtid;
|
||||
uint32 domain_id= thd->variables.gtid_domain_id;
|
||||
@ -5424,7 +5425,8 @@ MYSQL_BIN_LOG::write_gtid_event(THD *thd, bool standalone,
|
||||
return true;
|
||||
|
||||
Gtid_log_event gtid_event(thd, seq_no, domain_id, standalone,
|
||||
LOG_EVENT_SUPPRESS_USE_F, is_transactional);
|
||||
LOG_EVENT_SUPPRESS_USE_F, is_transactional,
|
||||
commit_id);
|
||||
|
||||
/* Write the event to the binary log. */
|
||||
if (gtid_event.write(&mysql_bin_log.log_file))
|
||||
@ -5706,7 +5708,7 @@ bool MYSQL_BIN_LOG::write(Log_event *event_info, my_bool *with_annotate)
|
||||
my_org_b_tell= my_b_tell(file);
|
||||
mysql_mutex_lock(&LOCK_log);
|
||||
prev_binlog_id= current_binlog_id;
|
||||
if (write_gtid_event(thd, true, using_trans))
|
||||
if (write_gtid_event(thd, true, using_trans, 0))
|
||||
goto err;
|
||||
}
|
||||
else
|
||||
@ -6595,45 +6597,284 @@ MYSQL_BIN_LOG::write_transaction_to_binlog(THD *thd,
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
Put a transaction that is ready to commit in the group commit queue.
|
||||
The transaction is identified by the ENTRY object passed into this function.
|
||||
|
||||
To facilitate group commit for the binlog, we first queue up ourselves in
|
||||
this function. Then later the first thread to enter the queue waits for
|
||||
the LOCK_log mutex, and commits for everyone in the queue once it gets the
|
||||
lock. Any other threads in the queue just wait for the first one to finish
|
||||
the commit and wake them up. This way, all transactions in the queue get
|
||||
committed in a single disk operation.
|
||||
|
||||
The main work in this function is when the commit in one transaction has
|
||||
been marked to wait for the commit of another transaction to happen
|
||||
first. This is used to support in-order parallel replication, where
|
||||
transactions can execute out-of-order but need to be committed in-order with
|
||||
how they happened on the master. The waiting of one commit on another needs
|
||||
to be integrated with the group commit queue, to ensure that the waiting
|
||||
transaction can participate in the same group commit as the waited-for
|
||||
transaction.
|
||||
|
||||
So when we put a transaction in the queue, we check if there were other
|
||||
transactions already prepared to commit but just waiting for the first one
|
||||
to commit. If so, we add those to the queue as well, transitively for all
|
||||
waiters.
|
||||
|
||||
@retval TRUE If queued as the first entry in the queue (meaning this
|
||||
is the leader)
|
||||
@retval FALSE Otherwise
|
||||
*/
|
||||
|
||||
bool
|
||||
MYSQL_BIN_LOG::queue_for_group_commit(group_commit_entry *orig_entry)
|
||||
{
|
||||
group_commit_entry *entry, *orig_queue;
|
||||
wait_for_commit *list, *cur, *last;
|
||||
wait_for_commit *wfc;
|
||||
DBUG_ENTER("MYSQL_BIN_LOG::queue_for_group_commit");
|
||||
|
||||
/*
|
||||
Check if we need to wait for another transaction to commit before us.
|
||||
|
||||
It is safe to do a quick check without lock first in the case where we do
|
||||
not have to wait. But if the quick check shows we need to wait, we must do
|
||||
another safe check under lock, to avoid the race where the other
|
||||
transaction wakes us up between the check and the wait.
|
||||
*/
|
||||
wfc= orig_entry->thd->wait_for_commit_ptr;
|
||||
orig_entry->queued_by_other= false;
|
||||
if (wfc && wfc->waiting_for_commit)
|
||||
{
|
||||
mysql_mutex_lock(&wfc->LOCK_wait_commit);
|
||||
/* Do an extra check here, this time safely under lock. */
|
||||
if (wfc->waiting_for_commit)
|
||||
{
|
||||
/*
|
||||
By setting wfc->opaque_pointer to our own entry, we mark that we are
|
||||
ready to commit, but waiting for another transaction to commit before
|
||||
us.
|
||||
|
||||
This other transaction may then take over the commit process for us to
|
||||
get us included in its own group commit. If this happens, the
|
||||
queued_by_other flag is set.
|
||||
*/
|
||||
wfc->opaque_pointer= orig_entry;
|
||||
DEBUG_SYNC(orig_entry->thd, "group_commit_waiting_for_prior");
|
||||
do
|
||||
{
|
||||
mysql_cond_wait(&wfc->COND_wait_commit, &wfc->LOCK_wait_commit);
|
||||
} while (wfc->waiting_for_commit);
|
||||
wfc->opaque_pointer= NULL;
|
||||
DBUG_PRINT("info", ("After waiting for prior commit, queued_by_other=%d",
|
||||
orig_entry->queued_by_other));
|
||||
}
|
||||
mysql_mutex_unlock(&wfc->LOCK_wait_commit);
|
||||
}
|
||||
|
||||
/*
|
||||
If the transaction we were waiting for has already put us into the group
|
||||
commit queue (and possibly already done the entire binlog commit for us),
|
||||
then there is nothing else to do.
|
||||
*/
|
||||
if (orig_entry->queued_by_other)
|
||||
DBUG_RETURN(false);
|
||||
|
||||
/* Now enqueue ourselves in the group commit queue. */
|
||||
DEBUG_SYNC(orig_entry->thd, "commit_before_enqueue");
|
||||
orig_entry->thd->clear_wakeup_ready();
|
||||
mysql_mutex_lock(&LOCK_prepare_ordered);
|
||||
orig_queue= group_commit_queue;
|
||||
|
||||
/*
|
||||
Iteratively process everything added to the queue, looking for waiters,
|
||||
and their waiters, and so on. If a waiter is ready to commit, we
|
||||
immediately add it to the queue; if not we just wake it up.
|
||||
|
||||
This would be natural to do with recursion, but we want to avoid
|
||||
potentially unbounded recursion blowing the C stack, so we use the list
|
||||
approach instead.
|
||||
|
||||
We keep a list of all the waiters that need to be processed in `list',
|
||||
linked through the next_subsequent_commit pointer. Initially this list
|
||||
contains only the entry passed into this function.
|
||||
|
||||
We process entries in the list one by one. The element currently being
|
||||
processed is pointed to by `cur`, and the element at the end of the list
|
||||
is pointed to by `last` (we do not use NULL to terminate the list).
|
||||
|
||||
As we process an element, it is first added to the group_commit_queue.
|
||||
Then any waiters for that element are added at the end of the list, to
|
||||
be processed in subsequent iterations. This continues until the list
|
||||
is exhausted, with all elements ever added eventually processed.
|
||||
|
||||
The end result is a breath-first traversal of the tree of waiters,
|
||||
re-using the next_subsequent_commit pointers in place of extra stack
|
||||
space in a recursive traversal.
|
||||
|
||||
The temporary list created in next_subsequent_commit is not
|
||||
used by the caller or any other function.
|
||||
*/
|
||||
|
||||
list= wfc;
|
||||
cur= list;
|
||||
last= list;
|
||||
entry= orig_entry;
|
||||
for (;;)
|
||||
{
|
||||
/* Add the entry to the group commit queue. */
|
||||
entry->next= group_commit_queue;
|
||||
group_commit_queue= entry;
|
||||
|
||||
if (entry->cache_mngr->using_xa)
|
||||
{
|
||||
DEBUG_SYNC(entry->thd, "commit_before_prepare_ordered");
|
||||
run_prepare_ordered(entry->thd, entry->all);
|
||||
DEBUG_SYNC(entry->thd, "commit_after_prepare_ordered");
|
||||
}
|
||||
|
||||
if (!cur)
|
||||
break; // Can happen if initial entry has no wait_for_commit
|
||||
|
||||
/*
|
||||
Check if this transaction has other transaction waiting for it to commit.
|
||||
|
||||
If so, process the waiting transactions, and their waiters and so on,
|
||||
transitively.
|
||||
*/
|
||||
if (cur->subsequent_commits_list)
|
||||
{
|
||||
bool have_lock;
|
||||
wait_for_commit *waiter;
|
||||
|
||||
mysql_mutex_lock(&cur->LOCK_wait_commit);
|
||||
have_lock= true;
|
||||
/*
|
||||
Grab the list, now safely under lock, and process it if still
|
||||
non-empty.
|
||||
*/
|
||||
waiter= cur->subsequent_commits_list;
|
||||
cur->subsequent_commits_list= NULL;
|
||||
while (waiter)
|
||||
{
|
||||
wait_for_commit *next= waiter->next_subsequent_commit;
|
||||
group_commit_entry *entry2=
|
||||
(group_commit_entry *)waiter->opaque_pointer;
|
||||
if (entry2)
|
||||
{
|
||||
/*
|
||||
This is another transaction ready to be written to the binary
|
||||
log. We can put it into the queue directly, without needing a
|
||||
separate context switch to the other thread. We just set a flag
|
||||
so that the other thread will know when it wakes up that it was
|
||||
already processed.
|
||||
|
||||
So put it at the end of the list to be processed in a subsequent
|
||||
iteration of the outer loop.
|
||||
*/
|
||||
entry2->queued_by_other= true;
|
||||
last->next_subsequent_commit= waiter;
|
||||
last= waiter;
|
||||
/*
|
||||
As a small optimisation, we do not actually need to set
|
||||
waiter->next_subsequent_commit to NULL, as we can use the
|
||||
pointer `last' to check for end-of-list.
|
||||
*/
|
||||
}
|
||||
else
|
||||
{
|
||||
/*
|
||||
Wake up the waiting transaction.
|
||||
|
||||
For this, we need to set the "wakeup running" flag and release
|
||||
the waitee lock to avoid a deadlock, see comments on
|
||||
THD::wakeup_subsequent_commits2() for details.
|
||||
*/
|
||||
if (have_lock)
|
||||
{
|
||||
have_lock= false;
|
||||
cur->wakeup_subsequent_commits_running= true;
|
||||
mysql_mutex_unlock(&cur->LOCK_wait_commit);
|
||||
}
|
||||
waiter->wakeup(0);
|
||||
}
|
||||
waiter= next;
|
||||
}
|
||||
if (have_lock)
|
||||
mysql_mutex_unlock(&cur->LOCK_wait_commit);
|
||||
}
|
||||
if (cur == last)
|
||||
break;
|
||||
/*
|
||||
Move to the next entry in the flattened list of waiting transactions
|
||||
that still need to be processed transitively.
|
||||
*/
|
||||
cur= cur->next_subsequent_commit;
|
||||
entry= (group_commit_entry *)cur->opaque_pointer;
|
||||
DBUG_ASSERT(entry != NULL);
|
||||
}
|
||||
|
||||
/*
|
||||
Now we need to clear the wakeup_subsequent_commits_running flags.
|
||||
|
||||
We need a full memory barrier between walking the list above, and clearing
|
||||
the flag wakeup_subsequent_commits_running below. This barrier is needed
|
||||
to ensure that no other thread will start to modify the list pointers
|
||||
before we are done traversing the list.
|
||||
|
||||
But wait_for_commit::wakeup(), which was called above for any other thread
|
||||
that might modify the list in parallel, does a full memory barrier already
|
||||
(it locks a mutex).
|
||||
*/
|
||||
if (list)
|
||||
{
|
||||
for (;;)
|
||||
{
|
||||
list->wakeup_subsequent_commits_running= false;
|
||||
if (list == last)
|
||||
break;
|
||||
list= list->next_subsequent_commit;
|
||||
}
|
||||
}
|
||||
|
||||
if (opt_binlog_commit_wait_count > 0)
|
||||
mysql_cond_signal(&COND_prepare_ordered);
|
||||
mysql_mutex_unlock(&LOCK_prepare_ordered);
|
||||
DEBUG_SYNC(orig_entry->thd, "commit_after_release_LOCK_prepare_ordered");
|
||||
|
||||
DBUG_PRINT("info", ("Queued for group commit as %s\n",
|
||||
(orig_queue == NULL) ? "leader" : "participant"));
|
||||
DBUG_RETURN(orig_queue == NULL);
|
||||
}
|
||||
|
||||
bool
|
||||
MYSQL_BIN_LOG::write_transaction_to_binlog_events(group_commit_entry *entry)
|
||||
{
|
||||
/*
|
||||
To facilitate group commit for the binlog, we first queue up ourselves in
|
||||
the group commit queue. Then the first thread to enter the queue waits for
|
||||
the LOCK_log mutex, and commits for everyone in the queue once it gets the
|
||||
lock. Any other threads in the queue just wait for the first one to finish
|
||||
the commit and wake them up.
|
||||
*/
|
||||
|
||||
entry->thd->clear_wakeup_ready();
|
||||
mysql_mutex_lock(&LOCK_prepare_ordered);
|
||||
group_commit_entry *orig_queue= group_commit_queue;
|
||||
entry->next= orig_queue;
|
||||
group_commit_queue= entry;
|
||||
|
||||
if (entry->cache_mngr->using_xa)
|
||||
{
|
||||
DEBUG_SYNC(entry->thd, "commit_before_prepare_ordered");
|
||||
run_prepare_ordered(entry->thd, entry->all);
|
||||
DEBUG_SYNC(entry->thd, "commit_after_prepare_ordered");
|
||||
}
|
||||
mysql_mutex_unlock(&LOCK_prepare_ordered);
|
||||
DEBUG_SYNC(entry->thd, "commit_after_release_LOCK_prepare_ordered");
|
||||
bool is_leader= queue_for_group_commit(entry);
|
||||
|
||||
/*
|
||||
The first in the queue handle group commit for all; the others just wait
|
||||
The first in the queue handles group commit for all; the others just wait
|
||||
to be signalled when group commit is done.
|
||||
*/
|
||||
if (orig_queue != NULL)
|
||||
if (is_leader)
|
||||
trx_group_commit_leader(entry);
|
||||
else if (!entry->queued_by_other)
|
||||
entry->thd->wait_for_wakeup_ready();
|
||||
else
|
||||
trx_group_commit_leader(entry);
|
||||
{
|
||||
/*
|
||||
If we were queued by another prior commit, then we are woken up
|
||||
only when the leader has already completed the commit for us.
|
||||
So nothing to do here then.
|
||||
*/
|
||||
}
|
||||
|
||||
if (!opt_optimize_thread_scheduling)
|
||||
{
|
||||
/* For the leader, trx_group_commit_leader() already took the lock. */
|
||||
if (orig_queue != NULL)
|
||||
if (!is_leader)
|
||||
mysql_mutex_lock(&LOCK_commit_ordered);
|
||||
|
||||
DEBUG_SYNC(entry->thd, "commit_loop_entry_commit_ordered");
|
||||
@ -6652,7 +6893,20 @@ MYSQL_BIN_LOG::write_transaction_to_binlog_events(group_commit_entry *entry)
|
||||
|
||||
if (next)
|
||||
{
|
||||
next->thd->signal_wakeup_ready();
|
||||
/*
|
||||
Wake up the next thread in the group commit.
|
||||
|
||||
The next thread can be waiting in two different ways, depending on
|
||||
whether it put itself in the queue, or if it was put in queue by us
|
||||
because it had to wait for us to commit first.
|
||||
|
||||
So execute the appropriate wakeup, identified by the queued_by_other
|
||||
field.
|
||||
*/
|
||||
if (next->queued_by_other)
|
||||
next->thd->wait_for_commit_ptr->wakeup(entry->error);
|
||||
else
|
||||
next->thd->signal_wakeup_ready();
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -6722,6 +6976,7 @@ MYSQL_BIN_LOG::trx_group_commit_leader(group_commit_entry *leader)
|
||||
group_commit_entry *queue= NULL;
|
||||
bool check_purge= false;
|
||||
ulong binlog_id;
|
||||
uint64 commit_id;
|
||||
DBUG_ENTER("MYSQL_BIN_LOG::trx_group_commit_leader");
|
||||
LINT_INIT(binlog_id);
|
||||
|
||||
@ -6732,12 +6987,18 @@ MYSQL_BIN_LOG::trx_group_commit_leader(group_commit_entry *leader)
|
||||
*/
|
||||
mysql_mutex_lock(&LOCK_log);
|
||||
DEBUG_SYNC(leader->thd, "commit_after_get_LOCK_log");
|
||||
binlog_id= current_binlog_id;
|
||||
|
||||
mysql_mutex_lock(&LOCK_prepare_ordered);
|
||||
if (opt_binlog_commit_wait_count)
|
||||
wait_for_sufficient_commits();
|
||||
/*
|
||||
Note that wait_for_sufficient_commits() may have released and
|
||||
re-acquired the LOCK_log and LOCK_prepare_ordered if it needed to wait.
|
||||
*/
|
||||
current= group_commit_queue;
|
||||
group_commit_queue= NULL;
|
||||
mysql_mutex_unlock(&LOCK_prepare_ordered);
|
||||
binlog_id= current_binlog_id;
|
||||
|
||||
/* As the queue is in reverse order of entering, reverse it. */
|
||||
last_in_queue= current;
|
||||
@ -6756,6 +7017,7 @@ MYSQL_BIN_LOG::trx_group_commit_leader(group_commit_entry *leader)
|
||||
DBUG_ASSERT(is_open());
|
||||
if (likely(is_open())) // Should always be true
|
||||
{
|
||||
commit_id= (last_in_queue == leader ? 0 : (uint64)leader->thd->query_id);
|
||||
/*
|
||||
Commit every transaction in the queue.
|
||||
|
||||
@ -6776,7 +7038,7 @@ MYSQL_BIN_LOG::trx_group_commit_leader(group_commit_entry *leader)
|
||||
*/
|
||||
DBUG_ASSERT(!cache_mngr->stmt_cache.empty() || !cache_mngr->trx_cache.empty());
|
||||
|
||||
if ((current->error= write_transaction_or_stmt(current)))
|
||||
if ((current->error= write_transaction_or_stmt(current, commit_id)))
|
||||
current->commit_errno= errno;
|
||||
|
||||
strmake_buf(cache_mngr->last_commit_pos_file, log_file_name);
|
||||
@ -6936,7 +7198,12 @@ MYSQL_BIN_LOG::trx_group_commit_leader(group_commit_entry *leader)
|
||||
*/
|
||||
next= current->next;
|
||||
if (current != leader) // Don't wake up ourself
|
||||
current->thd->signal_wakeup_ready();
|
||||
{
|
||||
if (current->queued_by_other)
|
||||
current->thd->wait_for_commit_ptr->wakeup(current->error);
|
||||
else
|
||||
current->thd->signal_wakeup_ready();
|
||||
}
|
||||
current= next;
|
||||
}
|
||||
DEBUG_SYNC(leader->thd, "commit_after_group_run_commit_ordered");
|
||||
@ -6951,11 +7218,12 @@ MYSQL_BIN_LOG::trx_group_commit_leader(group_commit_entry *leader)
|
||||
|
||||
|
||||
int
|
||||
MYSQL_BIN_LOG::write_transaction_or_stmt(group_commit_entry *entry)
|
||||
MYSQL_BIN_LOG::write_transaction_or_stmt(group_commit_entry *entry,
|
||||
uint64 commit_id)
|
||||
{
|
||||
binlog_cache_mngr *mngr= entry->cache_mngr;
|
||||
|
||||
if (write_gtid_event(entry->thd, false, entry->using_trx_cache))
|
||||
if (write_gtid_event(entry->thd, false, entry->using_trx_cache, commit_id))
|
||||
return ER_ERROR_ON_WRITE;
|
||||
|
||||
if (entry->using_stmt_cache && !mngr->stmt_cache.empty() &&
|
||||
@ -7023,6 +7291,72 @@ MYSQL_BIN_LOG::write_transaction_or_stmt(group_commit_entry *entry)
|
||||
return 0;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
Wait for sufficient commits to queue up for group commit, according to the
|
||||
values of binlog_commit_wait_count and binlog_commit_wait_usec.
|
||||
|
||||
Note that this function may release and re-acquire LOCK_log and
|
||||
LOCK_prepare_ordered if it needs to wait.
|
||||
*/
|
||||
|
||||
void
|
||||
MYSQL_BIN_LOG::wait_for_sufficient_commits()
|
||||
{
|
||||
size_t count;
|
||||
group_commit_entry *e;
|
||||
group_commit_entry *last_head;
|
||||
struct timespec wait_until;
|
||||
|
||||
mysql_mutex_assert_owner(&LOCK_log);
|
||||
mysql_mutex_assert_owner(&LOCK_prepare_ordered);
|
||||
|
||||
for (e= last_head= group_commit_queue, count= 0; e; e= e->next)
|
||||
if (++count >= opt_binlog_commit_wait_count)
|
||||
return;
|
||||
|
||||
mysql_mutex_unlock(&LOCK_log);
|
||||
set_timespec_nsec(wait_until, (ulonglong)1000*opt_binlog_commit_wait_usec);
|
||||
|
||||
for (;;)
|
||||
{
|
||||
int err;
|
||||
group_commit_entry *head;
|
||||
|
||||
err= mysql_cond_timedwait(&COND_prepare_ordered, &LOCK_prepare_ordered,
|
||||
&wait_until);
|
||||
if (err == ETIMEDOUT)
|
||||
break;
|
||||
head= group_commit_queue;
|
||||
for (e= head; e && e != last_head; e= e->next)
|
||||
++count;
|
||||
if (count >= opt_binlog_commit_wait_count)
|
||||
break;
|
||||
last_head= head;
|
||||
}
|
||||
|
||||
/*
|
||||
We must not wait for LOCK_log while holding LOCK_prepare_ordered.
|
||||
LOCK_log can be held for long periods (eg. we do I/O under it), while
|
||||
LOCK_prepare_ordered must only be held for short periods.
|
||||
|
||||
In addition, waiting for LOCK_log while holding LOCK_prepare_ordered would
|
||||
violate locking order of LOCK_log-before-LOCK_prepare_ordered. This could
|
||||
cause SAFEMUTEX warnings (even if it cannot actually deadlock with current
|
||||
code, as there can be at most one group commit leader thread at a time).
|
||||
|
||||
So release and re-acquire LOCK_prepare_ordered if we need to wait for the
|
||||
LOCK_log.
|
||||
*/
|
||||
if (mysql_mutex_trylock(&LOCK_log))
|
||||
{
|
||||
mysql_mutex_unlock(&LOCK_prepare_ordered);
|
||||
mysql_mutex_lock(&LOCK_log);
|
||||
mysql_mutex_lock(&LOCK_prepare_ordered);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
Wait until we get a signal that the relay log has been updated.
|
||||
|
||||
@ -7565,6 +7899,9 @@ int TC_LOG_MMAP::log_and_order(THD *thd, my_xid xid, bool all,
|
||||
mysql_mutex_unlock(&LOCK_prepare_ordered);
|
||||
}
|
||||
|
||||
if (thd->wait_for_prior_commit())
|
||||
return 0;
|
||||
|
||||
cookie= 0;
|
||||
if (xid)
|
||||
cookie= log_one_transaction(xid);
|
||||
|
21
sql/log.h
21
sql/log.h
@ -45,6 +45,15 @@ class TC_LOG
|
||||
|
||||
virtual int open(const char *opt_name)=0;
|
||||
virtual void close()=0;
|
||||
/*
|
||||
Transaction coordinator 2-phase commit.
|
||||
|
||||
Must invoke the run_prepare_ordered and run_commit_ordered methods, as
|
||||
described below for these methods.
|
||||
|
||||
In addition, must invoke THD::wait_for_prior_commit(), or equivalent
|
||||
wait, to ensure that one commit waits for another if registered to do so.
|
||||
*/
|
||||
virtual int log_and_order(THD *thd, my_xid xid, bool all,
|
||||
bool need_prepare_ordered,
|
||||
bool need_commit_ordered) = 0;
|
||||
@ -76,9 +85,11 @@ protected:
|
||||
prepare_ordered() or commit_ordered() methods.
|
||||
*/
|
||||
extern mysql_mutex_t LOCK_prepare_ordered;
|
||||
extern mysql_cond_t COND_prepare_ordered;
|
||||
extern mysql_mutex_t LOCK_commit_ordered;
|
||||
#ifdef HAVE_PSI_INTERFACE
|
||||
extern PSI_mutex_key key_LOCK_prepare_ordered, key_LOCK_commit_ordered;
|
||||
extern PSI_cond_key key_COND_prepare_ordered;
|
||||
#endif
|
||||
|
||||
class TC_LOG_DUMMY: public TC_LOG // use it to disable the logging
|
||||
@ -397,6 +408,7 @@ private:
|
||||
|
||||
class binlog_cache_mngr;
|
||||
struct rpl_gtid;
|
||||
struct wait_for_commit;
|
||||
class MYSQL_BIN_LOG: public TC_LOG, private MYSQL_LOG
|
||||
{
|
||||
private:
|
||||
@ -445,6 +457,8 @@ class MYSQL_BIN_LOG: public TC_LOG, private MYSQL_LOG
|
||||
group commit, only used when opt_optimize_thread_scheduling is not set.
|
||||
*/
|
||||
bool check_purge;
|
||||
/* Flag used to optimise around wait_for_prior_commit. */
|
||||
bool queued_by_other;
|
||||
ulong binlog_id;
|
||||
};
|
||||
|
||||
@ -525,7 +539,8 @@ class MYSQL_BIN_LOG: public TC_LOG, private MYSQL_LOG
|
||||
int new_file_impl(bool need_lock);
|
||||
void do_checkpoint_request(ulong binlog_id);
|
||||
void purge();
|
||||
int write_transaction_or_stmt(group_commit_entry *entry);
|
||||
int write_transaction_or_stmt(group_commit_entry *entry, uint64 commit_id);
|
||||
bool queue_for_group_commit(group_commit_entry *entry);
|
||||
bool write_transaction_to_binlog_events(group_commit_entry *entry);
|
||||
void trx_group_commit_leader(group_commit_entry *leader);
|
||||
bool is_xidlist_idle_nolock();
|
||||
@ -672,6 +687,7 @@ public:
|
||||
}
|
||||
void set_max_size(ulong max_size_arg);
|
||||
void signal_update();
|
||||
void wait_for_sufficient_commits();
|
||||
void wait_for_update_relay_log(THD* thd);
|
||||
int wait_for_update_bin_log(THD* thd, const struct timespec * timeout);
|
||||
void init(ulong max_size);
|
||||
@ -777,7 +793,8 @@ public:
|
||||
inline uint32 get_open_count() { return open_count; }
|
||||
void set_status_variables(THD *thd);
|
||||
bool is_xidlist_idle();
|
||||
bool write_gtid_event(THD *thd, bool standalone, bool is_transactional);
|
||||
bool write_gtid_event(THD *thd, bool standalone, bool is_transactional,
|
||||
uint64 commit_id);
|
||||
int read_state_from_file();
|
||||
int write_state_to_file();
|
||||
int get_most_recent_gtid_list(rpl_gtid **list, uint32 *size);
|
||||
|
543
sql/log_event.cc
543
sql/log_event.cc
File diff suppressed because it is too large
Load Diff
194
sql/log_event.h
194
sql/log_event.h
@ -1317,9 +1317,9 @@ public:
|
||||
|
||||
@see do_apply_event
|
||||
*/
|
||||
int apply_event(Relay_log_info const *rli)
|
||||
int apply_event(rpl_group_info *rgi)
|
||||
{
|
||||
return do_apply_event(rli);
|
||||
return do_apply_event(rgi);
|
||||
}
|
||||
|
||||
|
||||
@ -1331,9 +1331,9 @@ public:
|
||||
|
||||
@see do_update_pos
|
||||
*/
|
||||
int update_pos(Relay_log_info *rli)
|
||||
int update_pos(rpl_group_info *rgi)
|
||||
{
|
||||
return do_update_pos(rli);
|
||||
return do_update_pos(rgi);
|
||||
}
|
||||
|
||||
/**
|
||||
@ -1342,9 +1342,9 @@ public:
|
||||
|
||||
@see do_shall_skip
|
||||
*/
|
||||
enum_skip_reason shall_skip(Relay_log_info *rli)
|
||||
enum_skip_reason shall_skip(rpl_group_info *rgi)
|
||||
{
|
||||
return do_shall_skip(rli);
|
||||
return do_shall_skip(rgi);
|
||||
}
|
||||
|
||||
|
||||
@ -1352,6 +1352,7 @@ public:
|
||||
Check if an event is non-final part of a stand-alone event group,
|
||||
such as Intvar_log_event (such events should be processed as part
|
||||
of the following event group, not individually).
|
||||
See also is_part_of_group()
|
||||
*/
|
||||
static bool is_part_of_group(enum Log_event_type ev_type)
|
||||
{
|
||||
@ -1375,7 +1376,32 @@ public:
|
||||
return false;
|
||||
}
|
||||
}
|
||||
/*
|
||||
Same as above, but works on the object. In addition this is true for all
|
||||
rows event except the last one.
|
||||
*/
|
||||
virtual bool is_part_of_group() { return 0; }
|
||||
|
||||
static bool is_group_event(enum Log_event_type ev_type)
|
||||
{
|
||||
switch (ev_type)
|
||||
{
|
||||
case START_EVENT_V3:
|
||||
case STOP_EVENT:
|
||||
case ROTATE_EVENT:
|
||||
case SLAVE_EVENT:
|
||||
case FORMAT_DESCRIPTION_EVENT:
|
||||
case INCIDENT_EVENT:
|
||||
case HEARTBEAT_LOG_EVENT:
|
||||
case BINLOG_CHECKPOINT_EVENT:
|
||||
case GTID_LIST_EVENT:
|
||||
return false;
|
||||
|
||||
default:
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
protected:
|
||||
|
||||
/**
|
||||
@ -1388,14 +1414,14 @@ protected:
|
||||
|
||||
A typical usage is:
|
||||
@code
|
||||
enum_skip_reason do_shall_skip(Relay_log_info *rli) {
|
||||
return continue_group(rli);
|
||||
enum_skip_reason do_shall_skip(rpl_group_info *rgi) {
|
||||
return continue_group(rgi);
|
||||
}
|
||||
@endcode
|
||||
|
||||
@return Skip reason
|
||||
*/
|
||||
enum_skip_reason continue_group(Relay_log_info *rli);
|
||||
enum_skip_reason continue_group(rpl_group_info *rgi);
|
||||
|
||||
/**
|
||||
Primitive to apply an event to the database.
|
||||
@ -1412,7 +1438,7 @@ protected:
|
||||
@retval 0 Event applied successfully
|
||||
@retval errno Error code if event application failed
|
||||
*/
|
||||
virtual int do_apply_event(Relay_log_info const *rli)
|
||||
virtual int do_apply_event(rpl_group_info *rgi)
|
||||
{
|
||||
return 0; /* Default implementation does nothing */
|
||||
}
|
||||
@ -1441,7 +1467,7 @@ protected:
|
||||
1). Observe that handler errors are returned by the
|
||||
do_apply_event() function, and not by this one.
|
||||
*/
|
||||
virtual int do_update_pos(Relay_log_info *rli);
|
||||
virtual int do_update_pos(rpl_group_info *rgi);
|
||||
|
||||
|
||||
/**
|
||||
@ -1473,7 +1499,7 @@ protected:
|
||||
The event shall be skipped because the slave skip counter was
|
||||
non-zero. The caller shall decrease the counter by one.
|
||||
*/
|
||||
virtual enum_skip_reason do_shall_skip(Relay_log_info *rli);
|
||||
virtual enum_skip_reason do_shall_skip(rpl_group_info *rgi);
|
||||
#endif
|
||||
};
|
||||
|
||||
@ -1965,11 +1991,11 @@ public:
|
||||
|
||||
public: /* !!! Public in this patch to allow old usage */
|
||||
#if defined(MYSQL_SERVER) && defined(HAVE_REPLICATION)
|
||||
virtual enum_skip_reason do_shall_skip(Relay_log_info *rli);
|
||||
virtual int do_apply_event(Relay_log_info const *rli);
|
||||
virtual int do_update_pos(Relay_log_info *rli);
|
||||
virtual enum_skip_reason do_shall_skip(rpl_group_info *rgi);
|
||||
virtual int do_apply_event(rpl_group_info *rgi);
|
||||
virtual int do_update_pos(rpl_group_info *rgi);
|
||||
|
||||
int do_apply_event(Relay_log_info const *rli,
|
||||
int do_apply_event(rpl_group_info *rgi,
|
||||
const char *query_arg,
|
||||
uint32 q_len_arg);
|
||||
static bool peek_is_commit_rollback(const char *event_start,
|
||||
@ -1997,6 +2023,9 @@ public: /* !!! Public in this patch to allow old usage */
|
||||
!strncasecmp(query, "SAVEPOINT", 9) ||
|
||||
!strncasecmp(query, "ROLLBACK", 8);
|
||||
}
|
||||
bool is_begin() { return !strcmp(query, "BEGIN"); }
|
||||
bool is_commit() { return !strcmp(query, "COMMIT"); }
|
||||
bool is_rollback() { return !strcmp(query, "ROLLBACK"); }
|
||||
};
|
||||
|
||||
|
||||
@ -2083,7 +2112,7 @@ public:
|
||||
|
||||
private:
|
||||
#if defined(MYSQL_SERVER) && defined(HAVE_REPLICATION)
|
||||
virtual int do_apply_event(Relay_log_info const* rli);
|
||||
virtual int do_apply_event(rpl_group_info *rgi);
|
||||
#endif
|
||||
};
|
||||
|
||||
@ -2396,12 +2425,12 @@ public:
|
||||
|
||||
public: /* !!! Public in this patch to allow old usage */
|
||||
#if defined(MYSQL_SERVER) && defined(HAVE_REPLICATION)
|
||||
virtual int do_apply_event(Relay_log_info const* rli)
|
||||
virtual int do_apply_event(rpl_group_info *rgi)
|
||||
{
|
||||
return do_apply_event(thd->slave_net,rli,0);
|
||||
return do_apply_event(thd->slave_net,rgi,0);
|
||||
}
|
||||
|
||||
int do_apply_event(NET *net, Relay_log_info const *rli,
|
||||
int do_apply_event(NET *net, rpl_group_info *rgi,
|
||||
bool use_rli_only_for_errors);
|
||||
#endif
|
||||
};
|
||||
@ -2482,8 +2511,8 @@ public:
|
||||
|
||||
protected:
|
||||
#if defined(MYSQL_SERVER) && defined(HAVE_REPLICATION)
|
||||
virtual int do_apply_event(Relay_log_info const *rli);
|
||||
virtual enum_skip_reason do_shall_skip(Relay_log_info*)
|
||||
virtual int do_apply_event(rpl_group_info *rgi);
|
||||
virtual enum_skip_reason do_shall_skip(rpl_group_info*)
|
||||
{
|
||||
/*
|
||||
Events from ourself should be skipped, but they should not
|
||||
@ -2578,9 +2607,9 @@ public:
|
||||
static bool is_version_before_checksum(const master_version_split *version_split);
|
||||
protected:
|
||||
#if defined(MYSQL_SERVER) && defined(HAVE_REPLICATION)
|
||||
virtual int do_apply_event(Relay_log_info const *rli);
|
||||
virtual int do_update_pos(Relay_log_info *rli);
|
||||
virtual enum_skip_reason do_shall_skip(Relay_log_info *rli);
|
||||
virtual int do_apply_event(rpl_group_info *rgi);
|
||||
virtual int do_update_pos(rpl_group_info *rgi);
|
||||
virtual enum_skip_reason do_shall_skip(rpl_group_info *rgi);
|
||||
#endif
|
||||
};
|
||||
|
||||
@ -2654,12 +2683,13 @@ Intvar_log_event(THD* thd_arg,uchar type_arg, ulonglong val_arg,
|
||||
bool write(IO_CACHE* file);
|
||||
#endif
|
||||
bool is_valid() const { return 1; }
|
||||
bool is_part_of_group() { return 1; }
|
||||
|
||||
private:
|
||||
#if defined(MYSQL_SERVER) && defined(HAVE_REPLICATION)
|
||||
virtual int do_apply_event(Relay_log_info const *rli);
|
||||
virtual int do_update_pos(Relay_log_info *rli);
|
||||
virtual enum_skip_reason do_shall_skip(Relay_log_info *rli);
|
||||
virtual int do_apply_event(rpl_group_info *rgi);
|
||||
virtual int do_update_pos(rpl_group_info *rgi);
|
||||
virtual enum_skip_reason do_shall_skip(rpl_group_info *rgi);
|
||||
#endif
|
||||
};
|
||||
|
||||
@ -2733,12 +2763,13 @@ class Rand_log_event: public Log_event
|
||||
bool write(IO_CACHE* file);
|
||||
#endif
|
||||
bool is_valid() const { return 1; }
|
||||
bool is_part_of_group() { return 1; }
|
||||
|
||||
private:
|
||||
#if defined(MYSQL_SERVER) && defined(HAVE_REPLICATION)
|
||||
virtual int do_apply_event(Relay_log_info const *rli);
|
||||
virtual int do_update_pos(Relay_log_info *rli);
|
||||
virtual enum_skip_reason do_shall_skip(Relay_log_info *rli);
|
||||
virtual int do_apply_event(rpl_group_info *rgi);
|
||||
virtual int do_update_pos(rpl_group_info *rgi);
|
||||
virtual enum_skip_reason do_shall_skip(rpl_group_info *rgi);
|
||||
#endif
|
||||
};
|
||||
|
||||
@ -2785,8 +2816,8 @@ class Xid_log_event: public Log_event
|
||||
|
||||
private:
|
||||
#if defined(MYSQL_SERVER) && defined(HAVE_REPLICATION)
|
||||
virtual int do_apply_event(Relay_log_info const *rli);
|
||||
enum_skip_reason do_shall_skip(Relay_log_info *rli);
|
||||
virtual int do_apply_event(rpl_group_info *rgi);
|
||||
enum_skip_reason do_shall_skip(rpl_group_info *rgi);
|
||||
#endif
|
||||
};
|
||||
|
||||
@ -2854,12 +2885,13 @@ public:
|
||||
void set_deferred(query_id_t qid) { deferred= true; query_id= qid; }
|
||||
#endif
|
||||
bool is_valid() const { return name != 0; }
|
||||
bool is_part_of_group() { return 1; }
|
||||
|
||||
private:
|
||||
#if defined(MYSQL_SERVER) && defined(HAVE_REPLICATION)
|
||||
virtual int do_apply_event(Relay_log_info const *rli);
|
||||
virtual int do_update_pos(Relay_log_info *rli);
|
||||
virtual enum_skip_reason do_shall_skip(Relay_log_info *rli);
|
||||
virtual int do_apply_event(rpl_group_info *rgi);
|
||||
virtual int do_update_pos(rpl_group_info *rgi);
|
||||
virtual enum_skip_reason do_shall_skip(rpl_group_info *rgi);
|
||||
#endif
|
||||
};
|
||||
|
||||
@ -2892,8 +2924,8 @@ public:
|
||||
|
||||
private:
|
||||
#if defined(MYSQL_SERVER) && defined(HAVE_REPLICATION)
|
||||
virtual int do_update_pos(Relay_log_info *rli);
|
||||
virtual enum_skip_reason do_shall_skip(Relay_log_info *rli)
|
||||
virtual int do_update_pos(rpl_group_info *rgi);
|
||||
virtual enum_skip_reason do_shall_skip(rpl_group_info *rgi)
|
||||
{
|
||||
/*
|
||||
Events from ourself should be skipped, but they should not
|
||||
@ -2996,8 +3028,8 @@ public:
|
||||
|
||||
private:
|
||||
#if defined(MYSQL_SERVER) && defined(HAVE_REPLICATION)
|
||||
virtual int do_update_pos(Relay_log_info *rli);
|
||||
virtual enum_skip_reason do_shall_skip(Relay_log_info *rli);
|
||||
virtual int do_update_pos(rpl_group_info *rgi);
|
||||
virtual enum_skip_reason do_shall_skip(rpl_group_info *rgi);
|
||||
#endif
|
||||
};
|
||||
|
||||
@ -3089,6 +3121,7 @@ class Gtid_log_event: public Log_event
|
||||
{
|
||||
public:
|
||||
uint64 seq_no;
|
||||
uint64 commit_id;
|
||||
uint32 domain_id;
|
||||
uchar flags2;
|
||||
|
||||
@ -3096,15 +3129,20 @@ public:
|
||||
|
||||
/* FL_STANDALONE is set when there is no terminating COMMIT event. */
|
||||
static const uchar FL_STANDALONE= 1;
|
||||
/*
|
||||
FL_GROUP_COMMIT_ID is set when event group is part of a group commit on the
|
||||
master. Groups with same commit_id are part of the same group commit.
|
||||
*/
|
||||
static const uchar FL_GROUP_COMMIT_ID= 2;
|
||||
|
||||
#ifdef MYSQL_SERVER
|
||||
Gtid_log_event(THD *thd_arg, uint64 seq_no, uint32 domain_id, bool standalone,
|
||||
uint16 flags, bool is_transactional);
|
||||
uint16 flags, bool is_transactional, uint64 commit_id);
|
||||
#ifdef HAVE_REPLICATION
|
||||
void pack_info(THD *thd, Protocol *protocol);
|
||||
virtual int do_apply_event(Relay_log_info const *rli);
|
||||
virtual int do_update_pos(Relay_log_info *rli);
|
||||
virtual enum_skip_reason do_shall_skip(Relay_log_info *rli);
|
||||
virtual int do_apply_event(rpl_group_info *rgi);
|
||||
virtual int do_update_pos(rpl_group_info *rgi);
|
||||
virtual enum_skip_reason do_shall_skip(rpl_group_info *rgi);
|
||||
#endif
|
||||
#else
|
||||
void print(FILE *file, PRINT_EVENT_INFO *print_event_info);
|
||||
@ -3113,7 +3151,10 @@ public:
|
||||
const Format_description_log_event *description_event);
|
||||
~Gtid_log_event() { }
|
||||
Log_event_type get_type_code() { return GTID_EVENT; }
|
||||
int get_data_size() { return GTID_HEADER_LEN; }
|
||||
int get_data_size()
|
||||
{
|
||||
return GTID_HEADER_LEN + ((flags2 & FL_GROUP_COMMIT_ID) ? 2 : 0);
|
||||
}
|
||||
bool is_valid() const { return seq_no != 0; }
|
||||
#ifdef MYSQL_SERVER
|
||||
bool write(IO_CACHE *file);
|
||||
@ -3232,7 +3273,7 @@ public:
|
||||
#if defined(MYSQL_SERVER) && defined(HAVE_REPLICATION)
|
||||
bool to_packet(String *packet);
|
||||
bool write(IO_CACHE *file);
|
||||
virtual int do_apply_event(Relay_log_info const *rli);
|
||||
virtual int do_apply_event(rpl_group_info *rgi);
|
||||
#endif
|
||||
static bool peek(const char *event_start, uint32 event_len,
|
||||
uint8 checksum_alg,
|
||||
@ -3312,7 +3353,7 @@ public:
|
||||
|
||||
private:
|
||||
#if defined(MYSQL_SERVER) && defined(HAVE_REPLICATION)
|
||||
virtual int do_apply_event(Relay_log_info const *rli);
|
||||
virtual int do_apply_event(rpl_group_info *rgi);
|
||||
#endif
|
||||
};
|
||||
|
||||
@ -3367,7 +3408,7 @@ public:
|
||||
|
||||
private:
|
||||
#if defined(MYSQL_SERVER) && defined(HAVE_REPLICATION)
|
||||
virtual int do_apply_event(Relay_log_info const *rli);
|
||||
virtual int do_apply_event(rpl_group_info *rgi);
|
||||
#endif
|
||||
};
|
||||
|
||||
@ -3408,7 +3449,7 @@ public:
|
||||
|
||||
private:
|
||||
#if defined(MYSQL_SERVER) && defined(HAVE_REPLICATION)
|
||||
virtual int do_apply_event(Relay_log_info const *rli);
|
||||
virtual int do_apply_event(rpl_group_info *rgi);
|
||||
#endif
|
||||
};
|
||||
|
||||
@ -3448,7 +3489,7 @@ public:
|
||||
|
||||
private:
|
||||
#if defined(MYSQL_SERVER) && defined(HAVE_REPLICATION)
|
||||
virtual int do_apply_event(Relay_log_info const *rli);
|
||||
virtual int do_apply_event(rpl_group_info *rgi);
|
||||
#endif
|
||||
};
|
||||
|
||||
@ -3481,7 +3522,7 @@ public:
|
||||
Log_event_type get_type_code() { return BEGIN_LOAD_QUERY_EVENT; }
|
||||
private:
|
||||
#if defined(MYSQL_SERVER) && defined(HAVE_REPLICATION)
|
||||
virtual enum_skip_reason do_shall_skip(Relay_log_info *rli);
|
||||
virtual enum_skip_reason do_shall_skip(rpl_group_info *rgi);
|
||||
#endif
|
||||
};
|
||||
|
||||
@ -3547,7 +3588,7 @@ public:
|
||||
|
||||
private:
|
||||
#if defined(MYSQL_SERVER) && defined(HAVE_REPLICATION)
|
||||
virtual int do_apply_event(Relay_log_info const *rli);
|
||||
virtual int do_apply_event(rpl_group_info *rgi);
|
||||
#endif
|
||||
};
|
||||
|
||||
@ -3603,6 +3644,7 @@ public:
|
||||
virtual int get_data_size();
|
||||
virtual Log_event_type get_type_code();
|
||||
virtual bool is_valid() const;
|
||||
virtual bool is_part_of_group() { return 1; }
|
||||
|
||||
#ifndef MYSQL_CLIENT
|
||||
virtual bool write_data_header(IO_CACHE*);
|
||||
@ -3619,9 +3661,9 @@ public:
|
||||
|
||||
#if !defined(MYSQL_CLIENT) && defined(HAVE_REPLICATION)
|
||||
private:
|
||||
virtual int do_apply_event(Relay_log_info const*);
|
||||
virtual int do_update_pos(Relay_log_info*);
|
||||
virtual enum_skip_reason do_shall_skip(Relay_log_info*);
|
||||
virtual int do_apply_event(rpl_group_info *rgi);
|
||||
virtual int do_update_pos(rpl_group_info *rgi);
|
||||
virtual enum_skip_reason do_shall_skip(rpl_group_info*);
|
||||
#endif
|
||||
|
||||
private:
|
||||
@ -4014,6 +4056,7 @@ public:
|
||||
|
||||
virtual Log_event_type get_type_code() { return TABLE_MAP_EVENT; }
|
||||
virtual bool is_valid() const { return m_memory != NULL; /* we check malloc */ }
|
||||
virtual bool is_part_of_group() { return 1; }
|
||||
|
||||
virtual int get_data_size() { return (uint) m_data_size; }
|
||||
#ifdef MYSQL_SERVER
|
||||
@ -4034,9 +4077,9 @@ public:
|
||||
|
||||
private:
|
||||
#if defined(MYSQL_SERVER) && defined(HAVE_REPLICATION)
|
||||
virtual int do_apply_event(Relay_log_info const *rli);
|
||||
virtual int do_update_pos(Relay_log_info *rli);
|
||||
virtual enum_skip_reason do_shall_skip(Relay_log_info *rli);
|
||||
virtual int do_apply_event(rpl_group_info *rgi);
|
||||
virtual int do_update_pos(rpl_group_info *rgi);
|
||||
virtual enum_skip_reason do_shall_skip(rpl_group_info *rgi);
|
||||
#endif
|
||||
|
||||
#ifdef MYSQL_SERVER
|
||||
@ -4179,6 +4222,7 @@ public:
|
||||
{
|
||||
return m_rows_buf && m_cols.bitmap;
|
||||
}
|
||||
bool is_part_of_group() { return get_flags(STMT_END_F) != 0; }
|
||||
|
||||
uint m_row_count; /* The number of rows added to the event */
|
||||
|
||||
@ -4240,16 +4284,16 @@ protected:
|
||||
uint m_key_nr; /* Key number */
|
||||
|
||||
int find_key(); // Find a best key to use in find_row()
|
||||
int find_row(const Relay_log_info *const);
|
||||
int write_row(const Relay_log_info *const, const bool);
|
||||
int find_row(rpl_group_info *);
|
||||
int write_row(rpl_group_info *, const bool);
|
||||
|
||||
// Unpack the current row into m_table->record[0]
|
||||
int unpack_current_row(const Relay_log_info *const rli)
|
||||
int unpack_current_row(rpl_group_info *rgi)
|
||||
{
|
||||
DBUG_ASSERT(m_table);
|
||||
|
||||
ASSERT_OR_RETURN_ERROR(m_curr_row < m_rows_end, HA_ERR_CORRUPT_EVENT);
|
||||
int const result= ::unpack_row(rli, m_table, m_width, m_curr_row,
|
||||
int const result= ::unpack_row(rgi, m_table, m_width, m_curr_row,
|
||||
m_rows_end, &m_cols,
|
||||
&m_curr_row_end, &m_master_reclength);
|
||||
if (m_curr_row_end > m_rows_end)
|
||||
@ -4262,9 +4306,9 @@ protected:
|
||||
private:
|
||||
|
||||
#if defined(MYSQL_SERVER) && defined(HAVE_REPLICATION)
|
||||
virtual int do_apply_event(Relay_log_info const *rli);
|
||||
virtual int do_update_pos(Relay_log_info *rli);
|
||||
virtual enum_skip_reason do_shall_skip(Relay_log_info *rli);
|
||||
virtual int do_apply_event(rpl_group_info *rgi);
|
||||
virtual int do_update_pos(rpl_group_info *rgi);
|
||||
virtual enum_skip_reason do_shall_skip(rpl_group_info *rgi);
|
||||
|
||||
/*
|
||||
Primitive to prepare for a sequence of row executions.
|
||||
@ -4315,7 +4359,7 @@ private:
|
||||
0 if execution succeeded, 1 if execution failed.
|
||||
|
||||
*/
|
||||
virtual int do_exec_row(const Relay_log_info *const rli) = 0;
|
||||
virtual int do_exec_row(rpl_group_info *rli) = 0;
|
||||
#endif /* defined(MYSQL_SERVER) && defined(HAVE_REPLICATION) */
|
||||
|
||||
friend class Old_rows_log_event;
|
||||
@ -4371,7 +4415,7 @@ private:
|
||||
#if defined(MYSQL_SERVER) && defined(HAVE_REPLICATION)
|
||||
virtual int do_before_row_operations(const Slave_reporting_capability *const);
|
||||
virtual int do_after_row_operations(const Slave_reporting_capability *const,int);
|
||||
virtual int do_exec_row(const Relay_log_info *const);
|
||||
virtual int do_exec_row(rpl_group_info *);
|
||||
#endif
|
||||
};
|
||||
|
||||
@ -4445,7 +4489,7 @@ protected:
|
||||
#if defined(MYSQL_SERVER) && defined(HAVE_REPLICATION)
|
||||
virtual int do_before_row_operations(const Slave_reporting_capability *const);
|
||||
virtual int do_after_row_operations(const Slave_reporting_capability *const,int);
|
||||
virtual int do_exec_row(const Relay_log_info *const);
|
||||
virtual int do_exec_row(rpl_group_info *);
|
||||
#endif /* defined(MYSQL_SERVER) && defined(HAVE_REPLICATION) */
|
||||
};
|
||||
|
||||
@ -4510,7 +4554,7 @@ protected:
|
||||
#if defined(MYSQL_SERVER) && defined(HAVE_REPLICATION)
|
||||
virtual int do_before_row_operations(const Slave_reporting_capability *const);
|
||||
virtual int do_after_row_operations(const Slave_reporting_capability *const,int);
|
||||
virtual int do_exec_row(const Relay_log_info *const);
|
||||
virtual int do_exec_row(rpl_group_info *);
|
||||
#endif
|
||||
};
|
||||
|
||||
@ -4596,7 +4640,7 @@ public:
|
||||
#endif
|
||||
|
||||
#if defined(MYSQL_SERVER) && defined(HAVE_REPLICATION)
|
||||
virtual int do_apply_event(Relay_log_info const *rli);
|
||||
virtual int do_apply_event(rpl_group_info *rgi);
|
||||
#endif
|
||||
|
||||
virtual bool write_data_header(IO_CACHE *file);
|
||||
@ -4682,16 +4726,6 @@ bool event_checksum_test(uchar *buf, ulong event_len, uint8 alg);
|
||||
uint8 get_checksum_alg(const char* buf, ulong len);
|
||||
extern TYPELIB binlog_checksum_typelib;
|
||||
|
||||
#ifndef MYSQL_CLIENT
|
||||
/**
|
||||
The function is called by slave applier in case there are
|
||||
active table filtering rules to force gathering events associated
|
||||
with Query-log-event into an array to execute
|
||||
them once the fate of the Query is determined for execution.
|
||||
*/
|
||||
bool slave_execute_deferred_events(THD *thd);
|
||||
#endif
|
||||
|
||||
/**
|
||||
@} (end of group Replication)
|
||||
*/
|
||||
|
@ -36,12 +36,13 @@
|
||||
|
||||
// Old implementation of do_apply_event()
|
||||
int
|
||||
Old_rows_log_event::do_apply_event(Old_rows_log_event *ev, const Relay_log_info *rli)
|
||||
Old_rows_log_event::do_apply_event(Old_rows_log_event *ev, rpl_group_info *rgi)
|
||||
{
|
||||
DBUG_ENTER("Old_rows_log_event::do_apply_event(st_relay_log_info*)");
|
||||
int error= 0;
|
||||
THD *ev_thd= ev->thd;
|
||||
uchar const *row_start= ev->m_rows_buf;
|
||||
const Relay_log_info *rli= rgi->rli;
|
||||
|
||||
/*
|
||||
If m_table_id == ~0UL, then we have a dummy event that does not
|
||||
@ -57,7 +58,7 @@ Old_rows_log_event::do_apply_event(Old_rows_log_event *ev, const Relay_log_info
|
||||
*/
|
||||
DBUG_ASSERT(ev->get_flags(Old_rows_log_event::STMT_END_F));
|
||||
|
||||
const_cast<Relay_log_info*>(rli)->slave_close_thread_tables(ev_thd);
|
||||
rgi->slave_close_thread_tables(ev_thd);
|
||||
ev_thd->clear_error();
|
||||
DBUG_RETURN(0);
|
||||
}
|
||||
@ -67,7 +68,7 @@ Old_rows_log_event::do_apply_event(Old_rows_log_event *ev, const Relay_log_info
|
||||
do_apply_event(). We still check here to prevent future coding
|
||||
errors.
|
||||
*/
|
||||
DBUG_ASSERT(rli->sql_thd == ev_thd);
|
||||
DBUG_ASSERT(rgi->thd == ev_thd);
|
||||
|
||||
/*
|
||||
If there is no locks taken, this is the first binrow event seen
|
||||
@ -98,7 +99,7 @@ Old_rows_log_event::do_apply_event(Old_rows_log_event *ev, const Relay_log_info
|
||||
*/
|
||||
ev_thd->lex->set_stmt_row_injection();
|
||||
|
||||
if (open_and_lock_tables(ev_thd, rli->tables_to_lock, FALSE, 0))
|
||||
if (open_and_lock_tables(ev_thd, rgi->tables_to_lock, FALSE, 0))
|
||||
{
|
||||
uint actual_error= ev_thd->stmt_da->sql_errno();
|
||||
if (ev_thd->is_slave_error || ev_thd->is_fatal_error)
|
||||
@ -113,7 +114,7 @@ Old_rows_log_event::do_apply_event(Old_rows_log_event *ev, const Relay_log_info
|
||||
"unexpected success or fatal error"));
|
||||
ev_thd->is_slave_error= 1;
|
||||
}
|
||||
const_cast<Relay_log_info*>(rli)->slave_close_thread_tables(thd);
|
||||
rgi->slave_close_thread_tables(thd);
|
||||
DBUG_RETURN(actual_error);
|
||||
}
|
||||
|
||||
@ -126,8 +127,8 @@ Old_rows_log_event::do_apply_event(Old_rows_log_event *ev, const Relay_log_info
|
||||
*/
|
||||
|
||||
{
|
||||
RPL_TABLE_LIST *ptr= rli->tables_to_lock;
|
||||
for (uint i= 0 ; ptr&& (i< rli->tables_to_lock_count);
|
||||
RPL_TABLE_LIST *ptr= rgi->tables_to_lock;
|
||||
for (uint i= 0 ; ptr&& (i< rgi->tables_to_lock_count);
|
||||
ptr= static_cast<RPL_TABLE_LIST*>(ptr->next_global), i++)
|
||||
{
|
||||
DBUG_ASSERT(ptr->m_tabledef_valid);
|
||||
@ -136,7 +137,7 @@ Old_rows_log_event::do_apply_event(Old_rows_log_event *ev, const Relay_log_info
|
||||
ptr->table, &conv_table))
|
||||
{
|
||||
ev_thd->is_slave_error= 1;
|
||||
const_cast<Relay_log_info*>(rli)->slave_close_thread_tables(ev_thd);
|
||||
rgi->slave_close_thread_tables(ev_thd);
|
||||
DBUG_RETURN(Old_rows_log_event::ERR_BAD_TABLE_DEF);
|
||||
}
|
||||
DBUG_PRINT("debug", ("Table: %s.%s is compatible with master"
|
||||
@ -161,15 +162,15 @@ Old_rows_log_event::do_apply_event(Old_rows_log_event *ev, const Relay_log_info
|
||||
Old_rows_log_event, we can invalidate the query cache for the
|
||||
associated table.
|
||||
*/
|
||||
TABLE_LIST *ptr= rli->tables_to_lock;
|
||||
for (uint i=0; ptr && (i < rli->tables_to_lock_count); ptr= ptr->next_global, i++)
|
||||
const_cast<Relay_log_info*>(rli)->m_table_map.set_table(ptr->table_id, ptr->table);
|
||||
TABLE_LIST *ptr= rgi->tables_to_lock;
|
||||
for (uint i=0; ptr && (i < rgi->tables_to_lock_count); ptr= ptr->next_global, i++)
|
||||
rgi->m_table_map.set_table(ptr->table_id, ptr->table);
|
||||
#ifdef HAVE_QUERY_CACHE
|
||||
query_cache.invalidate_locked_for_write(thd, rli->tables_to_lock);
|
||||
query_cache.invalidate_locked_for_write(thd, rgi->tables_to_lock);
|
||||
#endif
|
||||
}
|
||||
|
||||
TABLE* table= const_cast<Relay_log_info*>(rli)->m_table_map.get_table(ev->m_table_id);
|
||||
TABLE* table= rgi->m_table_map.get_table(ev->m_table_id);
|
||||
|
||||
if (table)
|
||||
{
|
||||
@ -205,22 +206,11 @@ Old_rows_log_event::do_apply_event(Old_rows_log_event *ev, const Relay_log_info
|
||||
/* A small test to verify that objects have consistent types */
|
||||
DBUG_ASSERT(sizeof(ev_thd->variables.option_bits) == sizeof(OPTION_RELAXED_UNIQUE_CHECKS));
|
||||
|
||||
/*
|
||||
Now we are in a statement and will stay in a statement until we
|
||||
see a STMT_END_F.
|
||||
|
||||
We set this flag here, before actually applying any rows, in
|
||||
case the SQL thread is stopped and we need to detect that we're
|
||||
inside a statement and halting abruptly might cause problems
|
||||
when restarting.
|
||||
*/
|
||||
const_cast<Relay_log_info*>(rli)->set_flag(Relay_log_info::IN_STMT);
|
||||
|
||||
error= do_before_row_operations(table);
|
||||
while (error == 0 && row_start < ev->m_rows_end)
|
||||
{
|
||||
uchar const *row_end= NULL;
|
||||
if ((error= do_prepare_row(ev_thd, rli, table, row_start, &row_end)))
|
||||
if ((error= do_prepare_row(ev_thd, rgi, table, row_start, &row_end)))
|
||||
break; // We should perform the after-row operation even in
|
||||
// the case of error
|
||||
|
||||
@ -280,7 +270,7 @@ Old_rows_log_event::do_apply_event(Old_rows_log_event *ev, const Relay_log_info
|
||||
rollback at the caller along with sbr.
|
||||
*/
|
||||
ev_thd->reset_current_stmt_binlog_format_row();
|
||||
const_cast<Relay_log_info*>(rli)->cleanup_context(ev_thd, error);
|
||||
rgi->cleanup_context(ev_thd, error);
|
||||
ev_thd->is_slave_error= 1;
|
||||
DBUG_RETURN(error);
|
||||
}
|
||||
@ -953,7 +943,7 @@ int Write_rows_log_event_old::do_after_row_operations(TABLE *table, int error)
|
||||
|
||||
int
|
||||
Write_rows_log_event_old::do_prepare_row(THD *thd_arg,
|
||||
Relay_log_info const *rli,
|
||||
rpl_group_info *rgi,
|
||||
TABLE *table,
|
||||
uchar const *row_start,
|
||||
uchar const **row_end)
|
||||
@ -962,7 +952,7 @@ Write_rows_log_event_old::do_prepare_row(THD *thd_arg,
|
||||
DBUG_ASSERT(row_start && row_end);
|
||||
|
||||
int error;
|
||||
error= unpack_row_old(const_cast<Relay_log_info*>(rli),
|
||||
error= unpack_row_old(rgi,
|
||||
table, m_width, table->record[0],
|
||||
row_start, m_rows_end,
|
||||
&m_cols, row_end, &m_master_reclength,
|
||||
@ -1037,7 +1027,7 @@ int Delete_rows_log_event_old::do_after_row_operations(TABLE *table, int error)
|
||||
|
||||
int
|
||||
Delete_rows_log_event_old::do_prepare_row(THD *thd_arg,
|
||||
Relay_log_info const *rli,
|
||||
rpl_group_info *rgi,
|
||||
TABLE *table,
|
||||
uchar const *row_start,
|
||||
uchar const **row_end)
|
||||
@ -1050,7 +1040,7 @@ Delete_rows_log_event_old::do_prepare_row(THD *thd_arg,
|
||||
*/
|
||||
DBUG_ASSERT(table->s->fields >= m_width);
|
||||
|
||||
error= unpack_row_old(const_cast<Relay_log_info*>(rli),
|
||||
error= unpack_row_old(rgi,
|
||||
table, m_width, table->record[0],
|
||||
row_start, m_rows_end,
|
||||
&m_cols, row_end, &m_master_reclength,
|
||||
@ -1134,7 +1124,7 @@ int Update_rows_log_event_old::do_after_row_operations(TABLE *table, int error)
|
||||
|
||||
|
||||
int Update_rows_log_event_old::do_prepare_row(THD *thd_arg,
|
||||
Relay_log_info const *rli,
|
||||
rpl_group_info *rgi,
|
||||
TABLE *table,
|
||||
uchar const *row_start,
|
||||
uchar const **row_end)
|
||||
@ -1148,14 +1138,14 @@ int Update_rows_log_event_old::do_prepare_row(THD *thd_arg,
|
||||
DBUG_ASSERT(table->s->fields >= m_width);
|
||||
|
||||
/* record[0] is the before image for the update */
|
||||
error= unpack_row_old(const_cast<Relay_log_info*>(rli),
|
||||
error= unpack_row_old(rgi,
|
||||
table, m_width, table->record[0],
|
||||
row_start, m_rows_end,
|
||||
&m_cols, row_end, &m_master_reclength,
|
||||
table->read_set, PRE_GA_UPDATE_ROWS_EVENT);
|
||||
row_start = *row_end;
|
||||
/* m_after_image is the after image for the update */
|
||||
error= unpack_row_old(const_cast<Relay_log_info*>(rli),
|
||||
error= unpack_row_old(rgi,
|
||||
table, m_width, m_after_image,
|
||||
row_start, m_rows_end,
|
||||
&m_cols, row_end, &m_master_reclength,
|
||||
@ -1451,10 +1441,11 @@ int Old_rows_log_event::do_add_row_data(uchar *row_data, size_t length)
|
||||
|
||||
|
||||
#if !defined(MYSQL_CLIENT) && defined(HAVE_REPLICATION)
|
||||
int Old_rows_log_event::do_apply_event(Relay_log_info const *rli)
|
||||
int Old_rows_log_event::do_apply_event(rpl_group_info *rgi)
|
||||
{
|
||||
DBUG_ENTER("Old_rows_log_event::do_apply_event(Relay_log_info*)");
|
||||
int error= 0;
|
||||
Relay_log_info const *rli= rgi->rli;
|
||||
|
||||
/*
|
||||
If m_table_id == ~0UL, then we have a dummy event that does not
|
||||
@ -1470,7 +1461,7 @@ int Old_rows_log_event::do_apply_event(Relay_log_info const *rli)
|
||||
*/
|
||||
DBUG_ASSERT(get_flags(STMT_END_F));
|
||||
|
||||
const_cast<Relay_log_info*>(rli)->slave_close_thread_tables(thd);
|
||||
rgi->slave_close_thread_tables(thd);
|
||||
thd->clear_error();
|
||||
DBUG_RETURN(0);
|
||||
}
|
||||
@ -1480,7 +1471,7 @@ int Old_rows_log_event::do_apply_event(Relay_log_info const *rli)
|
||||
do_apply_event(). We still check here to prevent future coding
|
||||
errors.
|
||||
*/
|
||||
DBUG_ASSERT(rli->sql_thd == thd);
|
||||
DBUG_ASSERT(rgi->thd == thd);
|
||||
|
||||
/*
|
||||
If there is no locks taken, this is the first binrow event seen
|
||||
@ -1498,8 +1489,8 @@ int Old_rows_log_event::do_apply_event(Relay_log_info const *rli)
|
||||
*/
|
||||
lex_start(thd);
|
||||
|
||||
if ((error= lock_tables(thd, rli->tables_to_lock,
|
||||
rli->tables_to_lock_count, 0)))
|
||||
if ((error= lock_tables(thd, rgi->tables_to_lock,
|
||||
rgi->tables_to_lock_count, 0)))
|
||||
{
|
||||
if (thd->is_slave_error || thd->is_fatal_error)
|
||||
{
|
||||
@ -1521,7 +1512,7 @@ int Old_rows_log_event::do_apply_event(Relay_log_info const *rli)
|
||||
"Error in %s event: when locking tables",
|
||||
get_type_str());
|
||||
}
|
||||
const_cast<Relay_log_info*>(rli)->slave_close_thread_tables(thd);
|
||||
rgi->slave_close_thread_tables(thd);
|
||||
DBUG_RETURN(error);
|
||||
}
|
||||
|
||||
@ -1534,8 +1525,8 @@ int Old_rows_log_event::do_apply_event(Relay_log_info const *rli)
|
||||
*/
|
||||
|
||||
{
|
||||
RPL_TABLE_LIST *ptr= rli->tables_to_lock;
|
||||
for (uint i= 0 ; ptr&& (i< rli->tables_to_lock_count);
|
||||
RPL_TABLE_LIST *ptr= rgi->tables_to_lock;
|
||||
for (uint i= 0 ; ptr&& (i< rgi->tables_to_lock_count);
|
||||
ptr= static_cast<RPL_TABLE_LIST*>(ptr->next_global), i++)
|
||||
{
|
||||
TABLE *conv_table;
|
||||
@ -1543,7 +1534,7 @@ int Old_rows_log_event::do_apply_event(Relay_log_info const *rli)
|
||||
ptr->table, &conv_table))
|
||||
{
|
||||
thd->is_slave_error= 1;
|
||||
const_cast<Relay_log_info*>(rli)->slave_close_thread_tables(thd);
|
||||
rgi->slave_close_thread_tables(thd);
|
||||
DBUG_RETURN(ERR_BAD_TABLE_DEF);
|
||||
}
|
||||
ptr->m_conv_table= conv_table;
|
||||
@ -1565,18 +1556,18 @@ int Old_rows_log_event::do_apply_event(Relay_log_info const *rli)
|
||||
Old_rows_log_event, we can invalidate the query cache for the
|
||||
associated table.
|
||||
*/
|
||||
for (TABLE_LIST *ptr= rli->tables_to_lock ; ptr ; ptr= ptr->next_global)
|
||||
for (TABLE_LIST *ptr= rgi->tables_to_lock ; ptr ; ptr= ptr->next_global)
|
||||
{
|
||||
const_cast<Relay_log_info*>(rli)->m_table_map.set_table(ptr->table_id, ptr->table);
|
||||
rgi->m_table_map.set_table(ptr->table_id, ptr->table);
|
||||
}
|
||||
#ifdef HAVE_QUERY_CACHE
|
||||
query_cache.invalidate_locked_for_write(thd, rli->tables_to_lock);
|
||||
query_cache.invalidate_locked_for_write(thd, rgi->tables_to_lock);
|
||||
#endif
|
||||
}
|
||||
|
||||
TABLE*
|
||||
table=
|
||||
m_table= const_cast<Relay_log_info*>(rli)->m_table_map.get_table(m_table_id);
|
||||
m_table= rgi->m_table_map.get_table(m_table_id);
|
||||
|
||||
if (table)
|
||||
{
|
||||
@ -1612,17 +1603,6 @@ int Old_rows_log_event::do_apply_event(Relay_log_info const *rli)
|
||||
/* A small test to verify that objects have consistent types */
|
||||
DBUG_ASSERT(sizeof(thd->variables.option_bits) == sizeof(OPTION_RELAXED_UNIQUE_CHECKS));
|
||||
|
||||
/*
|
||||
Now we are in a statement and will stay in a statement until we
|
||||
see a STMT_END_F.
|
||||
|
||||
We set this flag here, before actually applying any rows, in
|
||||
case the SQL thread is stopped and we need to detect that we're
|
||||
inside a statement and halting abruptly might cause problems
|
||||
when restarting.
|
||||
*/
|
||||
const_cast<Relay_log_info*>(rli)->set_flag(Relay_log_info::IN_STMT);
|
||||
|
||||
if ( m_width == table->s->fields && bitmap_is_set_all(&m_cols))
|
||||
set_flags(COMPLETE_ROWS_F);
|
||||
|
||||
@ -1656,7 +1636,7 @@ int Old_rows_log_event::do_apply_event(Relay_log_info const *rli)
|
||||
if (!table->in_use)
|
||||
table->in_use= thd;
|
||||
|
||||
error= do_exec_row(rli);
|
||||
error= do_exec_row(rgi);
|
||||
|
||||
DBUG_PRINT("info", ("error: %d", error));
|
||||
DBUG_ASSERT(error != HA_ERR_RECORD_DELETED);
|
||||
@ -1695,7 +1675,7 @@ int Old_rows_log_event::do_apply_event(Relay_log_info const *rli)
|
||||
(ulong) m_curr_row, (ulong) m_curr_row_end, (ulong) m_rows_end));
|
||||
|
||||
if (!m_curr_row_end && !error)
|
||||
unpack_current_row(rli);
|
||||
unpack_current_row(rgi);
|
||||
|
||||
// at this moment m_curr_row_end should be set
|
||||
DBUG_ASSERT(error || m_curr_row_end != NULL);
|
||||
@ -1732,7 +1712,7 @@ int Old_rows_log_event::do_apply_event(Relay_log_info const *rli)
|
||||
rollback at the caller along with sbr.
|
||||
*/
|
||||
thd->reset_current_stmt_binlog_format_row();
|
||||
const_cast<Relay_log_info*>(rli)->cleanup_context(thd, error);
|
||||
rgi->cleanup_context(thd, error);
|
||||
thd->is_slave_error= 1;
|
||||
DBUG_RETURN(error);
|
||||
}
|
||||
@ -1761,7 +1741,7 @@ int Old_rows_log_event::do_apply_event(Relay_log_info const *rli)
|
||||
problem. When WL#2975 is implemented, just remove the member
|
||||
Relay_log_info::last_event_start_time and all its occurrences.
|
||||
*/
|
||||
const_cast<Relay_log_info*>(rli)->last_event_start_time= my_time(0);
|
||||
rgi->last_event_start_time= my_time(0);
|
||||
}
|
||||
|
||||
if (get_flags(STMT_END_F))
|
||||
@ -1811,7 +1791,7 @@ int Old_rows_log_event::do_apply_event(Relay_log_info const *rli)
|
||||
*/
|
||||
|
||||
thd->reset_current_stmt_binlog_format_row();
|
||||
const_cast<Relay_log_info*>(rli)->cleanup_context(thd, 0);
|
||||
rgi->cleanup_context(thd, 0);
|
||||
}
|
||||
|
||||
DBUG_RETURN(error);
|
||||
@ -1819,22 +1799,23 @@ int Old_rows_log_event::do_apply_event(Relay_log_info const *rli)
|
||||
|
||||
|
||||
Log_event::enum_skip_reason
|
||||
Old_rows_log_event::do_shall_skip(Relay_log_info *rli)
|
||||
Old_rows_log_event::do_shall_skip(rpl_group_info *rgi)
|
||||
{
|
||||
/*
|
||||
If the slave skip counter is 1 and this event does not end a
|
||||
statement, then we should not start executing on the next event.
|
||||
Otherwise, we defer the decision to the normal skipping logic.
|
||||
*/
|
||||
if (rli->slave_skip_counter == 1 && !get_flags(STMT_END_F))
|
||||
if (rgi->rli->slave_skip_counter == 1 && !get_flags(STMT_END_F))
|
||||
return Log_event::EVENT_SKIP_IGNORE;
|
||||
else
|
||||
return Log_event::do_shall_skip(rli);
|
||||
return Log_event::do_shall_skip(rgi);
|
||||
}
|
||||
|
||||
int
|
||||
Old_rows_log_event::do_update_pos(Relay_log_info *rli)
|
||||
Old_rows_log_event::do_update_pos(rpl_group_info *rgi)
|
||||
{
|
||||
Relay_log_info *rli= rgi->rli;
|
||||
DBUG_ENTER("Old_rows_log_event::do_update_pos");
|
||||
int error= 0;
|
||||
|
||||
@ -1848,7 +1829,7 @@ Old_rows_log_event::do_update_pos(Relay_log_info *rli)
|
||||
Step the group log position if we are not in a transaction,
|
||||
otherwise increase the event log position.
|
||||
*/
|
||||
rli->stmt_done(log_pos, when, thd);
|
||||
rli->stmt_done(log_pos, when, thd, rgi);
|
||||
/*
|
||||
Clear any errors in thd->net.last_err*. It is not known if this is
|
||||
needed or not. It is believed that any errors that may exist in
|
||||
@ -1859,7 +1840,7 @@ Old_rows_log_event::do_update_pos(Relay_log_info *rli)
|
||||
}
|
||||
else
|
||||
{
|
||||
rli->inc_event_relay_log_pos();
|
||||
rgi->inc_event_relay_log_pos();
|
||||
}
|
||||
|
||||
DBUG_RETURN(error);
|
||||
@ -1996,8 +1977,7 @@ void Old_rows_log_event::print_helper(FILE *file,
|
||||
*/
|
||||
|
||||
int
|
||||
Old_rows_log_event::write_row(const Relay_log_info *const rli,
|
||||
const bool overwrite)
|
||||
Old_rows_log_event::write_row(rpl_group_info *rgi, const bool overwrite)
|
||||
{
|
||||
DBUG_ENTER("write_row");
|
||||
DBUG_ASSERT(m_table != NULL && thd != NULL);
|
||||
@ -2014,7 +1994,7 @@ Old_rows_log_event::write_row(const Relay_log_info *const rli,
|
||||
DBUG_RETURN(error);
|
||||
|
||||
/* unpack row into table->record[0] */
|
||||
error= unpack_current_row(rli); // TODO: how to handle errors?
|
||||
error= unpack_current_row(rgi); // TODO: how to handle errors?
|
||||
|
||||
#ifndef DBUG_OFF
|
||||
DBUG_DUMP("record[0]", table->record[0], table->s->reclength);
|
||||
@ -2121,7 +2101,7 @@ Old_rows_log_event::write_row(const Relay_log_info *const rli,
|
||||
if (!get_flags(COMPLETE_ROWS_F))
|
||||
{
|
||||
restore_record(table,record[1]);
|
||||
error= unpack_current_row(rli);
|
||||
error= unpack_current_row(rgi);
|
||||
}
|
||||
|
||||
#ifndef DBUG_OFF
|
||||
@ -2216,7 +2196,7 @@ Old_rows_log_event::write_row(const Relay_log_info *const rli,
|
||||
for any following update/delete command.
|
||||
*/
|
||||
|
||||
int Old_rows_log_event::find_row(const Relay_log_info *rli)
|
||||
int Old_rows_log_event::find_row(rpl_group_info *rgi)
|
||||
{
|
||||
DBUG_ENTER("find_row");
|
||||
|
||||
@ -2229,7 +2209,7 @@ int Old_rows_log_event::find_row(const Relay_log_info *rli)
|
||||
|
||||
// TODO: shall we check and report errors here?
|
||||
prepare_record(table, m_width, FALSE /* don't check errors */);
|
||||
error= unpack_current_row(rli);
|
||||
error= unpack_current_row(rgi);
|
||||
|
||||
#ifndef DBUG_OFF
|
||||
DBUG_PRINT("info",("looking for the following record"));
|
||||
@ -2601,10 +2581,10 @@ Write_rows_log_event_old::do_after_row_operations(const Slave_reporting_capabili
|
||||
|
||||
|
||||
int
|
||||
Write_rows_log_event_old::do_exec_row(const Relay_log_info *const rli)
|
||||
Write_rows_log_event_old::do_exec_row(rpl_group_info *rgi)
|
||||
{
|
||||
DBUG_ASSERT(m_table != NULL);
|
||||
int error= write_row(rli, TRUE /* overwrite */);
|
||||
int error= write_row(rgi, TRUE /* overwrite */);
|
||||
|
||||
if (error && !thd->net.last_errno)
|
||||
thd->net.last_errno= error;
|
||||
@ -2703,12 +2683,12 @@ Delete_rows_log_event_old::do_after_row_operations(const Slave_reporting_capabil
|
||||
}
|
||||
|
||||
|
||||
int Delete_rows_log_event_old::do_exec_row(const Relay_log_info *const rli)
|
||||
int Delete_rows_log_event_old::do_exec_row(rpl_group_info *rgi)
|
||||
{
|
||||
int error;
|
||||
DBUG_ASSERT(m_table != NULL);
|
||||
|
||||
if (!(error= find_row(rli)))
|
||||
if (!(error= find_row(rgi)))
|
||||
{
|
||||
/*
|
||||
Delete the record found, located in record[0]
|
||||
@ -2802,11 +2782,11 @@ Update_rows_log_event_old::do_after_row_operations(const Slave_reporting_capabil
|
||||
|
||||
|
||||
int
|
||||
Update_rows_log_event_old::do_exec_row(const Relay_log_info *const rli)
|
||||
Update_rows_log_event_old::do_exec_row(rpl_group_info *rgi)
|
||||
{
|
||||
DBUG_ASSERT(m_table != NULL);
|
||||
|
||||
int error= find_row(rli);
|
||||
int error= find_row(rgi);
|
||||
if (error)
|
||||
{
|
||||
/*
|
||||
@ -2814,7 +2794,7 @@ Update_rows_log_event_old::do_exec_row(const Relay_log_info *const rli)
|
||||
able to skip to the next pair of updates
|
||||
*/
|
||||
m_curr_row= m_curr_row_end;
|
||||
unpack_current_row(rli);
|
||||
unpack_current_row(rgi);
|
||||
return error;
|
||||
}
|
||||
|
||||
@ -2832,7 +2812,7 @@ Update_rows_log_event_old::do_exec_row(const Relay_log_info *const rli)
|
||||
store_record(m_table,record[1]);
|
||||
|
||||
m_curr_row= m_curr_row_end;
|
||||
error= unpack_current_row(rli); // this also updates m_curr_row_end
|
||||
error= unpack_current_row(rgi); // this also updates m_curr_row_end
|
||||
|
||||
/*
|
||||
Now we have the right row to update. The old row (the one we're
|
||||
|
@ -145,6 +145,7 @@ public:
|
||||
{
|
||||
return m_rows_buf && m_cols.bitmap;
|
||||
}
|
||||
bool is_part_of_group() { return 1; }
|
||||
|
||||
uint m_row_count; /* The number of rows added to the event */
|
||||
|
||||
@ -195,15 +196,15 @@ protected:
|
||||
const uchar *m_curr_row_end; /* One-after the end of the current row */
|
||||
uchar *m_key; /* Buffer to keep key value during searches */
|
||||
|
||||
int find_row(const Relay_log_info *const);
|
||||
int write_row(const Relay_log_info *const, const bool);
|
||||
int find_row(rpl_group_info *);
|
||||
int write_row(rpl_group_info *, const bool);
|
||||
|
||||
// Unpack the current row into m_table->record[0]
|
||||
int unpack_current_row(const Relay_log_info *const rli)
|
||||
int unpack_current_row(rpl_group_info *rgi)
|
||||
{
|
||||
DBUG_ASSERT(m_table);
|
||||
ASSERT_OR_RETURN_ERROR(m_curr_row < m_rows_end, HA_ERR_CORRUPT_EVENT);
|
||||
int const result= ::unpack_row(rli, m_table, m_width, m_curr_row,
|
||||
int const result= ::unpack_row(rgi, m_table, m_width, m_curr_row,
|
||||
m_rows_end, &m_cols,
|
||||
&m_curr_row_end, &m_master_reclength);
|
||||
ASSERT_OR_RETURN_ERROR(m_curr_row_end <= m_rows_end, HA_ERR_CORRUPT_EVENT);
|
||||
@ -214,9 +215,9 @@ protected:
|
||||
private:
|
||||
|
||||
#if !defined(MYSQL_CLIENT) && defined(HAVE_REPLICATION)
|
||||
virtual int do_apply_event(Relay_log_info const *rli);
|
||||
virtual int do_update_pos(Relay_log_info *rli);
|
||||
virtual enum_skip_reason do_shall_skip(Relay_log_info *rli);
|
||||
virtual int do_apply_event(rpl_group_info *rgi);
|
||||
virtual int do_update_pos(rpl_group_info *rgi);
|
||||
virtual enum_skip_reason do_shall_skip(rpl_group_info *rgi);
|
||||
|
||||
/*
|
||||
Primitive to prepare for a sequence of row executions.
|
||||
@ -267,7 +268,7 @@ private:
|
||||
0 if execution succeeded, 1 if execution failed.
|
||||
|
||||
*/
|
||||
virtual int do_exec_row(const Relay_log_info *const rli) = 0;
|
||||
virtual int do_exec_row(rpl_group_info *rgi) = 0;
|
||||
#endif /* !defined(MYSQL_CLIENT) && defined(HAVE_REPLICATION) */
|
||||
|
||||
/********** END OF CUT & PASTE FROM Rows_log_event **********/
|
||||
@ -275,7 +276,7 @@ private:
|
||||
|
||||
#if !defined(MYSQL_CLIENT) && defined(HAVE_REPLICATION)
|
||||
|
||||
int do_apply_event(Old_rows_log_event*,const Relay_log_info*);
|
||||
int do_apply_event(Old_rows_log_event*, rpl_group_info *rgi);
|
||||
|
||||
/*
|
||||
Primitive to prepare for a sequence of row executions.
|
||||
@ -324,7 +325,7 @@ private:
|
||||
RETURN VALUE
|
||||
Error code, if something went wrong, 0 otherwise.
|
||||
*/
|
||||
virtual int do_prepare_row(THD*, Relay_log_info const*, TABLE*,
|
||||
virtual int do_prepare_row(THD*, rpl_group_info*, TABLE*,
|
||||
uchar const *row_start,
|
||||
uchar const **row_end) = 0;
|
||||
|
||||
@ -387,7 +388,7 @@ private:
|
||||
#if !defined(MYSQL_CLIENT) && defined(HAVE_REPLICATION)
|
||||
virtual int do_before_row_operations(const Slave_reporting_capability *const);
|
||||
virtual int do_after_row_operations(const Slave_reporting_capability *const,int);
|
||||
virtual int do_exec_row(const Relay_log_info *const);
|
||||
virtual int do_exec_row(rpl_group_info *);
|
||||
#endif
|
||||
/********** END OF CUT & PASTE FROM Write_rows_log_event **********/
|
||||
|
||||
@ -403,13 +404,13 @@ private:
|
||||
|
||||
#if !defined(MYSQL_CLIENT) && defined(HAVE_REPLICATION)
|
||||
// use old definition of do_apply_event()
|
||||
virtual int do_apply_event(const Relay_log_info *rli)
|
||||
{ return Old_rows_log_event::do_apply_event(this,rli); }
|
||||
virtual int do_apply_event(rpl_group_info *rgi)
|
||||
{ return Old_rows_log_event::do_apply_event(this, rgi); }
|
||||
|
||||
// primitives for old version of do_apply_event()
|
||||
virtual int do_before_row_operations(TABLE *table);
|
||||
virtual int do_after_row_operations(TABLE *table, int error);
|
||||
virtual int do_prepare_row(THD*, Relay_log_info const*, TABLE*,
|
||||
virtual int do_prepare_row(THD*, rpl_group_info*, TABLE*,
|
||||
uchar const *row_start, uchar const **row_end);
|
||||
virtual int do_exec_row(TABLE *table);
|
||||
|
||||
@ -463,7 +464,7 @@ protected:
|
||||
#if !defined(MYSQL_CLIENT) && defined(HAVE_REPLICATION)
|
||||
virtual int do_before_row_operations(const Slave_reporting_capability *const);
|
||||
virtual int do_after_row_operations(const Slave_reporting_capability *const,int);
|
||||
virtual int do_exec_row(const Relay_log_info *const);
|
||||
virtual int do_exec_row(rpl_group_info *);
|
||||
#endif /* !defined(MYSQL_CLIENT) && defined(HAVE_REPLICATION) */
|
||||
/********** END OF CUT & PASTE FROM Update_rows_log_event **********/
|
||||
|
||||
@ -481,13 +482,13 @@ private:
|
||||
|
||||
#if !defined(MYSQL_CLIENT) && defined(HAVE_REPLICATION)
|
||||
// use old definition of do_apply_event()
|
||||
virtual int do_apply_event(const Relay_log_info *rli)
|
||||
{ return Old_rows_log_event::do_apply_event(this,rli); }
|
||||
virtual int do_apply_event(rpl_group_info *rgi)
|
||||
{ return Old_rows_log_event::do_apply_event(this, rgi); }
|
||||
|
||||
// primitives for old version of do_apply_event()
|
||||
virtual int do_before_row_operations(TABLE *table);
|
||||
virtual int do_after_row_operations(TABLE *table, int error);
|
||||
virtual int do_prepare_row(THD*, Relay_log_info const*, TABLE*,
|
||||
virtual int do_prepare_row(THD*, rpl_group_info*, TABLE*,
|
||||
uchar const *row_start, uchar const **row_end);
|
||||
virtual int do_exec_row(TABLE *table);
|
||||
#endif /* !defined(MYSQL_CLIENT) && defined(HAVE_REPLICATION) */
|
||||
@ -538,7 +539,7 @@ protected:
|
||||
#if !defined(MYSQL_CLIENT) && defined(HAVE_REPLICATION)
|
||||
virtual int do_before_row_operations(const Slave_reporting_capability *const);
|
||||
virtual int do_after_row_operations(const Slave_reporting_capability *const,int);
|
||||
virtual int do_exec_row(const Relay_log_info *const);
|
||||
virtual int do_exec_row(rpl_group_info *);
|
||||
#endif
|
||||
/********** END CUT & PASTE FROM Delete_rows_log_event **********/
|
||||
|
||||
@ -556,13 +557,13 @@ private:
|
||||
|
||||
#if !defined(MYSQL_CLIENT) && defined(HAVE_REPLICATION)
|
||||
// use old definition of do_apply_event()
|
||||
virtual int do_apply_event(const Relay_log_info *rli)
|
||||
{ return Old_rows_log_event::do_apply_event(this,rli); }
|
||||
virtual int do_apply_event(rpl_group_info *rgi)
|
||||
{ return Old_rows_log_event::do_apply_event(this, rgi); }
|
||||
|
||||
// primitives for old version of do_apply_event()
|
||||
virtual int do_before_row_operations(TABLE *table);
|
||||
virtual int do_after_row_operations(TABLE *table, int error);
|
||||
virtual int do_prepare_row(THD*, Relay_log_info const*, TABLE*,
|
||||
virtual int do_prepare_row(THD*, rpl_group_info*, TABLE*,
|
||||
uchar const *row_start, uchar const **row_end);
|
||||
virtual int do_exec_row(TABLE *table);
|
||||
#endif
|
||||
|
@ -467,11 +467,12 @@ uint lower_case_table_names;
|
||||
ulong tc_heuristic_recover= 0;
|
||||
int32 thread_count;
|
||||
int32 thread_running;
|
||||
int32 slave_open_temp_tables;
|
||||
ulong thread_created;
|
||||
ulong back_log, connect_timeout, concurrency, server_id;
|
||||
ulong table_cache_size, table_def_size;
|
||||
ulong what_to_log;
|
||||
ulong slow_launch_time, slave_open_temp_tables;
|
||||
ulong slow_launch_time;
|
||||
ulong open_files_limit, max_binlog_size;
|
||||
ulong slave_trans_retries;
|
||||
uint slave_net_timeout;
|
||||
@ -491,6 +492,7 @@ my_atomic_rwlock_t global_query_id_lock;
|
||||
my_atomic_rwlock_t thread_running_lock;
|
||||
my_atomic_rwlock_t thread_count_lock;
|
||||
my_atomic_rwlock_t statistics_lock;
|
||||
my_atomic_rwlock_t slave_executed_entries_lock;
|
||||
ulong aborted_threads, aborted_connects;
|
||||
ulong delayed_insert_timeout, delayed_insert_limit, delayed_queue_size;
|
||||
ulong delayed_insert_threads, delayed_insert_writes, delayed_rows_in_use;
|
||||
@ -543,6 +545,11 @@ ulong rpl_recovery_rank=0;
|
||||
*/
|
||||
ulong stored_program_cache_size= 0;
|
||||
|
||||
ulong opt_slave_parallel_threads= 0;
|
||||
ulong opt_binlog_commit_wait_count= 0;
|
||||
ulong opt_binlog_commit_wait_usec= 0;
|
||||
ulong opt_slave_parallel_max_queued= 131072;
|
||||
|
||||
const double log_10[] = {
|
||||
1e000, 1e001, 1e002, 1e003, 1e004, 1e005, 1e006, 1e007, 1e008, 1e009,
|
||||
1e010, 1e011, 1e012, 1e013, 1e014, 1e015, 1e016, 1e017, 1e018, 1e019,
|
||||
@ -762,18 +769,19 @@ PSI_mutex_key key_BINLOG_LOCK_index, key_BINLOG_LOCK_xid_list,
|
||||
key_master_info_sleep_lock,
|
||||
key_mutex_slave_reporting_capability_err_lock, key_relay_log_info_data_lock,
|
||||
key_relay_log_info_log_space_lock, key_relay_log_info_run_lock,
|
||||
key_relay_log_info_sleep_lock,
|
||||
key_rpl_group_info_sleep_lock,
|
||||
key_structure_guard_mutex, key_TABLE_SHARE_LOCK_ha_data,
|
||||
key_LOCK_error_messages, key_LOG_INFO_lock,
|
||||
key_LOCK_thread_count, key_LOCK_thread_cache,
|
||||
key_PARTITION_LOCK_auto_inc;
|
||||
PSI_mutex_key key_RELAYLOG_LOCK_index;
|
||||
PSI_mutex_key key_LOCK_slave_state, key_LOCK_binlog_state;
|
||||
PSI_mutex_key key_LOCK_slave_state, key_LOCK_binlog_state,
|
||||
key_LOCK_rpl_thread, key_LOCK_rpl_thread_pool, key_LOCK_parallel_entry;
|
||||
|
||||
PSI_mutex_key key_LOCK_stats,
|
||||
key_LOCK_global_user_client_stats, key_LOCK_global_table_stats,
|
||||
key_LOCK_global_index_stats,
|
||||
key_LOCK_wakeup_ready;
|
||||
key_LOCK_wakeup_ready, key_LOCK_wait_commit;
|
||||
|
||||
PSI_mutex_key key_LOCK_rpl_gtid_state;
|
||||
|
||||
@ -821,6 +829,7 @@ static PSI_mutex_info all_server_mutexes[]=
|
||||
{ &key_LOCK_global_index_stats, "LOCK_global_index_stats", PSI_FLAG_GLOBAL},
|
||||
{ &key_LOCK_wakeup_ready, "THD::LOCK_wakeup_ready", 0},
|
||||
{ &key_LOCK_rpl_gtid_state, "LOCK_rpl_gtid_state", PSI_FLAG_GLOBAL},
|
||||
{ &key_LOCK_wait_commit, "wait_for_commit::LOCK_wait_commit", 0},
|
||||
{ &key_LOCK_thd_data, "THD::LOCK_thd_data", 0},
|
||||
{ &key_LOCK_user_conn, "LOCK_user_conn", PSI_FLAG_GLOBAL},
|
||||
{ &key_LOCK_uuid_short_generator, "LOCK_uuid_short_generator", PSI_FLAG_GLOBAL},
|
||||
@ -832,7 +841,7 @@ static PSI_mutex_info all_server_mutexes[]=
|
||||
{ &key_relay_log_info_data_lock, "Relay_log_info::data_lock", 0},
|
||||
{ &key_relay_log_info_log_space_lock, "Relay_log_info::log_space_lock", 0},
|
||||
{ &key_relay_log_info_run_lock, "Relay_log_info::run_lock", 0},
|
||||
{ &key_relay_log_info_sleep_lock, "Relay_log_info::sleep_lock", 0},
|
||||
{ &key_rpl_group_info_sleep_lock, "Rpl_group_info::sleep_lock", 0},
|
||||
{ &key_structure_guard_mutex, "Query_cache::structure_guard_mutex", 0},
|
||||
{ &key_TABLE_SHARE_LOCK_ha_data, "TABLE_SHARE::LOCK_ha_data", 0},
|
||||
{ &key_LOCK_error_messages, "LOCK_error_messages", PSI_FLAG_GLOBAL},
|
||||
@ -843,7 +852,10 @@ static PSI_mutex_info all_server_mutexes[]=
|
||||
{ &key_LOCK_thread_cache, "LOCK_thread_cache", PSI_FLAG_GLOBAL},
|
||||
{ &key_PARTITION_LOCK_auto_inc, "HA_DATA_PARTITION::LOCK_auto_inc", 0},
|
||||
{ &key_LOCK_slave_state, "LOCK_slave_state", 0},
|
||||
{ &key_LOCK_binlog_state, "LOCK_binlog_state", 0}
|
||||
{ &key_LOCK_binlog_state, "LOCK_binlog_state", 0},
|
||||
{ &key_LOCK_rpl_thread, "LOCK_rpl_thread", 0},
|
||||
{ &key_LOCK_rpl_thread_pool, "LOCK_rpl_thread_pool", 0},
|
||||
{ &key_LOCK_parallel_entry, "LOCK_parallel_entry", 0}
|
||||
};
|
||||
|
||||
PSI_rwlock_key key_rwlock_LOCK_grant, key_rwlock_LOCK_logger,
|
||||
@ -878,13 +890,16 @@ PSI_cond_key key_BINLOG_COND_xid_list, key_BINLOG_update_cond,
|
||||
key_master_info_sleep_cond,
|
||||
key_relay_log_info_data_cond, key_relay_log_info_log_space_cond,
|
||||
key_relay_log_info_start_cond, key_relay_log_info_stop_cond,
|
||||
key_relay_log_info_sleep_cond,
|
||||
key_rpl_group_info_sleep_cond,
|
||||
key_TABLE_SHARE_cond, key_user_level_lock_cond,
|
||||
key_COND_thread_count, key_COND_thread_cache, key_COND_flush_thread_cache,
|
||||
key_BINLOG_COND_queue_busy;
|
||||
PSI_cond_key key_RELAYLOG_update_cond, key_COND_wakeup_ready;
|
||||
PSI_cond_key key_RELAYLOG_update_cond, key_COND_wakeup_ready,
|
||||
key_COND_wait_commit;
|
||||
PSI_cond_key key_RELAYLOG_COND_queue_busy;
|
||||
PSI_cond_key key_TC_LOG_MMAP_COND_queue_busy;
|
||||
PSI_cond_key key_COND_rpl_thread, key_COND_rpl_thread_pool,
|
||||
key_COND_parallel_entry, key_COND_prepare_ordered;
|
||||
|
||||
static PSI_cond_info all_server_conds[]=
|
||||
{
|
||||
@ -905,6 +920,7 @@ static PSI_cond_info all_server_conds[]=
|
||||
{ &key_RELAYLOG_update_cond, "MYSQL_RELAY_LOG::update_cond", 0},
|
||||
{ &key_RELAYLOG_COND_queue_busy, "MYSQL_RELAY_LOG::COND_queue_busy", 0},
|
||||
{ &key_COND_wakeup_ready, "THD::COND_wakeup_ready", 0},
|
||||
{ &key_COND_wait_commit, "wait_for_commit::COND_wait_commit", 0},
|
||||
{ &key_COND_cache_status_changed, "Query_cache::COND_cache_status_changed", 0},
|
||||
{ &key_COND_manager, "COND_manager", PSI_FLAG_GLOBAL},
|
||||
{ &key_COND_rpl_status, "COND_rpl_status", PSI_FLAG_GLOBAL},
|
||||
@ -920,18 +936,22 @@ static PSI_cond_info all_server_conds[]=
|
||||
{ &key_relay_log_info_log_space_cond, "Relay_log_info::log_space_cond", 0},
|
||||
{ &key_relay_log_info_start_cond, "Relay_log_info::start_cond", 0},
|
||||
{ &key_relay_log_info_stop_cond, "Relay_log_info::stop_cond", 0},
|
||||
{ &key_relay_log_info_sleep_cond, "Relay_log_info::sleep_cond", 0},
|
||||
{ &key_rpl_group_info_sleep_cond, "Rpl_group_info::sleep_cond", 0},
|
||||
{ &key_TABLE_SHARE_cond, "TABLE_SHARE::cond", 0},
|
||||
{ &key_user_level_lock_cond, "User_level_lock::cond", 0},
|
||||
{ &key_COND_thread_count, "COND_thread_count", PSI_FLAG_GLOBAL},
|
||||
{ &key_COND_thread_cache, "COND_thread_cache", PSI_FLAG_GLOBAL},
|
||||
{ &key_COND_flush_thread_cache, "COND_flush_thread_cache", PSI_FLAG_GLOBAL}
|
||||
{ &key_COND_flush_thread_cache, "COND_flush_thread_cache", PSI_FLAG_GLOBAL},
|
||||
{ &key_COND_rpl_thread, "COND_rpl_thread", 0},
|
||||
{ &key_COND_rpl_thread_pool, "COND_rpl_thread_pool", 0},
|
||||
{ &key_COND_parallel_entry, "COND_parallel_entry", 0},
|
||||
{ &key_COND_prepare_ordered, "COND_prepare_ordered", 0}
|
||||
};
|
||||
|
||||
PSI_thread_key key_thread_bootstrap, key_thread_delayed_insert,
|
||||
key_thread_handle_manager, key_thread_main,
|
||||
key_thread_one_connection, key_thread_signal_hand,
|
||||
key_thread_slave_init;
|
||||
key_thread_slave_init, key_rpl_parallel_thread;
|
||||
|
||||
static PSI_thread_info all_server_threads[]=
|
||||
{
|
||||
@ -957,7 +977,8 @@ static PSI_thread_info all_server_threads[]=
|
||||
{ &key_thread_main, "main", PSI_FLAG_GLOBAL},
|
||||
{ &key_thread_one_connection, "one_connection", 0},
|
||||
{ &key_thread_signal_hand, "signal_handler", PSI_FLAG_GLOBAL},
|
||||
{ &key_thread_slave_init, "slave_init", PSI_FLAG_GLOBAL}
|
||||
{ &key_thread_slave_init, "slave_init", PSI_FLAG_GLOBAL},
|
||||
{ &key_rpl_parallel_thread, "rpl_parallel_thread", 0}
|
||||
};
|
||||
|
||||
PSI_file_key key_file_binlog, key_file_binlog_index, key_file_casetest,
|
||||
@ -1920,6 +1941,7 @@ void clean_up(bool print_message)
|
||||
my_atomic_rwlock_destroy(&thread_running_lock);
|
||||
my_atomic_rwlock_destroy(&thread_count_lock);
|
||||
my_atomic_rwlock_destroy(&statistics_lock);
|
||||
my_atomic_rwlock_destroy(&slave_executed_entries_lock);
|
||||
free_charsets();
|
||||
mysql_mutex_lock(&LOCK_thread_count);
|
||||
DBUG_PRINT("quit", ("got thread count lock"));
|
||||
@ -2002,6 +2024,7 @@ static void clean_up_mutexes()
|
||||
mysql_mutex_destroy(&LOCK_server_started);
|
||||
mysql_cond_destroy(&COND_server_started);
|
||||
mysql_mutex_destroy(&LOCK_prepare_ordered);
|
||||
mysql_cond_destroy(&COND_prepare_ordered);
|
||||
mysql_mutex_destroy(&LOCK_commit_ordered);
|
||||
DBUG_VOID_RETURN;
|
||||
}
|
||||
@ -4136,6 +4159,7 @@ static int init_thread_environment()
|
||||
&LOCK_rpl_gtid_state, MY_MUTEX_INIT_SLOW);
|
||||
mysql_mutex_init(key_LOCK_prepare_ordered, &LOCK_prepare_ordered,
|
||||
MY_MUTEX_INIT_SLOW);
|
||||
mysql_cond_init(key_COND_prepare_ordered, &COND_prepare_ordered, NULL);
|
||||
mysql_mutex_init(key_LOCK_commit_ordered, &LOCK_commit_ordered,
|
||||
MY_MUTEX_INIT_SLOW);
|
||||
|
||||
@ -7332,7 +7356,7 @@ SHOW_VAR status_vars[]= {
|
||||
{"Select_range", (char*) offsetof(STATUS_VAR, select_range_count), SHOW_LONG_STATUS},
|
||||
{"Select_range_check", (char*) offsetof(STATUS_VAR, select_range_check_count), SHOW_LONG_STATUS},
|
||||
{"Select_scan", (char*) offsetof(STATUS_VAR, select_scan_count), SHOW_LONG_STATUS},
|
||||
{"Slave_open_temp_tables", (char*) &slave_open_temp_tables, SHOW_LONG},
|
||||
{"Slave_open_temp_tables", (char*) &slave_open_temp_tables, SHOW_INT},
|
||||
#ifdef HAVE_REPLICATION
|
||||
{"Slave_retried_transactions",(char*)&slave_retried_transactions, SHOW_LONG},
|
||||
{"Slave_heartbeat_period", (char*) &show_heartbeat_period, SHOW_SIMPLE_FUNC},
|
||||
@ -7596,6 +7620,7 @@ static int mysql_init_variables(void)
|
||||
my_atomic_rwlock_init(&thread_running_lock);
|
||||
my_atomic_rwlock_init(&thread_count_lock);
|
||||
my_atomic_rwlock_init(&statistics_lock);
|
||||
my_atomic_rwlock_init(slave_executed_entries_lock);
|
||||
strmov(server_version, MYSQL_SERVER_VERSION);
|
||||
threads.empty();
|
||||
thread_cache.empty();
|
||||
|
38
sql/mysqld.h
38
sql/mysqld.h
@ -153,7 +153,7 @@ extern ulong delayed_insert_timeout;
|
||||
extern ulong delayed_insert_limit, delayed_queue_size;
|
||||
extern ulong delayed_insert_threads, delayed_insert_writes;
|
||||
extern ulong delayed_rows_in_use,delayed_insert_errors;
|
||||
extern ulong slave_open_temp_tables;
|
||||
extern int32 slave_open_temp_tables;
|
||||
extern ulonglong query_cache_size;
|
||||
extern ulong query_cache_min_res_unit;
|
||||
extern ulong slow_launch_threads, slow_launch_time;
|
||||
@ -176,6 +176,10 @@ extern ulong slave_max_allowed_packet;
|
||||
extern ulong opt_binlog_rows_event_max_size;
|
||||
extern ulong rpl_recovery_rank, thread_cache_size;
|
||||
extern ulong stored_program_cache_size;
|
||||
extern ulong opt_slave_parallel_threads;
|
||||
extern ulong opt_slave_parallel_max_queued;
|
||||
extern ulong opt_binlog_commit_wait_count;
|
||||
extern ulong opt_binlog_commit_wait_usec;
|
||||
extern ulong back_log;
|
||||
extern ulong executed_events;
|
||||
extern char language[FN_REFLEN];
|
||||
@ -243,15 +247,16 @@ extern PSI_mutex_key key_BINLOG_LOCK_index, key_BINLOG_LOCK_xid_list,
|
||||
key_master_info_sleep_lock,
|
||||
key_mutex_slave_reporting_capability_err_lock, key_relay_log_info_data_lock,
|
||||
key_relay_log_info_log_space_lock, key_relay_log_info_run_lock,
|
||||
key_relay_log_info_sleep_lock,
|
||||
key_rpl_group_info_sleep_lock,
|
||||
key_structure_guard_mutex, key_TABLE_SHARE_LOCK_ha_data,
|
||||
key_LOCK_error_messages, key_LOCK_thread_count, key_PARTITION_LOCK_auto_inc;
|
||||
extern PSI_mutex_key key_RELAYLOG_LOCK_index;
|
||||
extern PSI_mutex_key key_LOCK_slave_state, key_LOCK_binlog_state;
|
||||
extern PSI_mutex_key key_LOCK_slave_state, key_LOCK_binlog_state,
|
||||
key_LOCK_rpl_thread, key_LOCK_rpl_thread_pool, key_LOCK_parallel_entry;
|
||||
|
||||
extern PSI_mutex_key key_LOCK_stats,
|
||||
key_LOCK_global_user_client_stats, key_LOCK_global_table_stats,
|
||||
key_LOCK_global_index_stats, key_LOCK_wakeup_ready;
|
||||
key_LOCK_global_index_stats, key_LOCK_wakeup_ready, key_LOCK_wait_commit;
|
||||
|
||||
extern PSI_mutex_key key_LOCK_rpl_gtid_state;
|
||||
|
||||
@ -274,16 +279,20 @@ extern PSI_cond_key key_BINLOG_COND_xid_list, key_BINLOG_update_cond,
|
||||
key_master_info_sleep_cond,
|
||||
key_relay_log_info_data_cond, key_relay_log_info_log_space_cond,
|
||||
key_relay_log_info_start_cond, key_relay_log_info_stop_cond,
|
||||
key_relay_log_info_sleep_cond,
|
||||
key_rpl_group_info_sleep_cond,
|
||||
key_TABLE_SHARE_cond, key_user_level_lock_cond,
|
||||
key_COND_thread_count, key_COND_thread_cache, key_COND_flush_thread_cache;
|
||||
extern PSI_cond_key key_RELAYLOG_update_cond, key_COND_wakeup_ready;
|
||||
extern PSI_cond_key key_RELAYLOG_update_cond, key_COND_wakeup_ready,
|
||||
key_COND_wait_commit;
|
||||
extern PSI_cond_key key_RELAYLOG_COND_queue_busy;
|
||||
extern PSI_cond_key key_TC_LOG_MMAP_COND_queue_busy;
|
||||
extern PSI_cond_key key_COND_rpl_thread, key_COND_rpl_thread_pool,
|
||||
key_COND_parallel_entry;
|
||||
|
||||
extern PSI_thread_key key_thread_bootstrap, key_thread_delayed_insert,
|
||||
key_thread_handle_manager, key_thread_kill_server, key_thread_main,
|
||||
key_thread_one_connection, key_thread_signal_hand, key_thread_slave_init;
|
||||
key_thread_one_connection, key_thread_signal_hand, key_thread_slave_init,
|
||||
key_rpl_parallel_thread;
|
||||
|
||||
extern PSI_file_key key_file_binlog, key_file_binlog_index, key_file_casetest,
|
||||
key_file_dbopt, key_file_des_key_file, key_file_ERRMSG, key_select_to_file,
|
||||
@ -358,6 +367,7 @@ extern mysql_cond_t COND_manager;
|
||||
extern int32 thread_running;
|
||||
extern int32 thread_count;
|
||||
extern my_atomic_rwlock_t thread_running_lock, thread_count_lock;
|
||||
extern my_atomic_rwlock_t slave_executed_entries_lock;
|
||||
|
||||
extern char *opt_ssl_ca, *opt_ssl_capath, *opt_ssl_cert, *opt_ssl_cipher,
|
||||
*opt_ssl_key;
|
||||
@ -498,6 +508,20 @@ inline void thread_safe_decrement32(int32 *value, my_atomic_rwlock_t *lock)
|
||||
my_atomic_rwlock_wrunlock(lock);
|
||||
}
|
||||
|
||||
inline void thread_safe_increment64(int64 *value, my_atomic_rwlock_t *lock)
|
||||
{
|
||||
my_atomic_rwlock_wrlock(lock);
|
||||
(void) my_atomic_add64(value, 1);
|
||||
my_atomic_rwlock_wrunlock(lock);
|
||||
}
|
||||
|
||||
inline void thread_safe_decrement64(int64 *value, my_atomic_rwlock_t *lock)
|
||||
{
|
||||
my_atomic_rwlock_wrlock(lock);
|
||||
(void) my_atomic_add64(value, -1);
|
||||
my_atomic_rwlock_wrunlock(lock);
|
||||
}
|
||||
|
||||
inline void
|
||||
inc_thread_running()
|
||||
{
|
||||
|
@ -62,27 +62,28 @@ rpl_slave_state::update_state_hash(uint64 sub_id, rpl_gtid *gtid)
|
||||
|
||||
|
||||
int
|
||||
rpl_slave_state::record_and_update_gtid(THD *thd, Relay_log_info *rli)
|
||||
rpl_slave_state::record_and_update_gtid(THD *thd, rpl_group_info *rgi)
|
||||
{
|
||||
uint64 sub_id;
|
||||
DBUG_ENTER("rpl_slave_state::record_and_update_gtid");
|
||||
|
||||
/*
|
||||
Update the GTID position, if we have it and did not already update
|
||||
it in a GTID transaction.
|
||||
*/
|
||||
if ((sub_id= rli->gtid_sub_id))
|
||||
if ((sub_id= rgi->gtid_sub_id))
|
||||
{
|
||||
rli->gtid_sub_id= 0;
|
||||
if (record_gtid(thd, &rli->current_gtid, sub_id, false, false))
|
||||
return 1;
|
||||
update_state_hash(sub_id, &rli->current_gtid);
|
||||
rgi->gtid_sub_id= 0;
|
||||
if (record_gtid(thd, &rgi->current_gtid, sub_id, false, false))
|
||||
DBUG_RETURN(1);
|
||||
update_state_hash(sub_id, &rgi->current_gtid);
|
||||
}
|
||||
return 0;
|
||||
DBUG_RETURN(0);
|
||||
}
|
||||
|
||||
|
||||
rpl_slave_state::rpl_slave_state()
|
||||
: inited(false), loaded(false)
|
||||
: last_sub_id(0), inited(false), loaded(false)
|
||||
{
|
||||
my_hash_init(&hash, &my_charset_bin, 32, offsetof(element, domain_id),
|
||||
sizeof(uint32), NULL, my_free, HASH_UNIQUE);
|
||||
@ -152,6 +153,9 @@ rpl_slave_state::update(uint32 domain_id, uint32 server_id, uint64 sub_id,
|
||||
list_elem->seq_no= seq_no;
|
||||
|
||||
elem->add(list_elem);
|
||||
if (last_sub_id < sub_id)
|
||||
last_sub_id= sub_id;
|
||||
|
||||
return 0;
|
||||
}
|
||||
|
||||
@ -168,7 +172,6 @@ rpl_slave_state::get_element(uint32 domain_id)
|
||||
if (!(elem= (element *)my_malloc(sizeof(*elem), MYF(MY_WME))))
|
||||
return NULL;
|
||||
elem->list= NULL;
|
||||
elem->last_sub_id= 0;
|
||||
elem->domain_id= domain_id;
|
||||
if (my_hash_insert(&hash, (uchar *)elem))
|
||||
{
|
||||
@ -310,6 +313,7 @@ rpl_slave_state::record_gtid(THD *thd, const rpl_gtid *gtid, uint64 sub_id,
|
||||
element *elem;
|
||||
ulonglong thd_saved_option= thd->variables.option_bits;
|
||||
Query_tables_list lex_backup;
|
||||
DBUG_ENTER("record_gtid");
|
||||
|
||||
if (unlikely(!loaded))
|
||||
{
|
||||
@ -320,7 +324,7 @@ rpl_slave_state::record_gtid(THD *thd, const rpl_gtid *gtid, uint64 sub_id,
|
||||
We already complained loudly about this, but we can try to continue
|
||||
until the DBA fixes it.
|
||||
*/
|
||||
return 0;
|
||||
DBUG_RETURN(0);
|
||||
}
|
||||
|
||||
if (!in_statement)
|
||||
@ -329,7 +333,7 @@ rpl_slave_state::record_gtid(THD *thd, const rpl_gtid *gtid, uint64 sub_id,
|
||||
DBUG_EXECUTE_IF("gtid_inject_record_gtid",
|
||||
{
|
||||
my_error(ER_CANNOT_UPDATE_GTID_STATE, MYF(0));
|
||||
return 1;
|
||||
DBUG_RETURN(1);
|
||||
} );
|
||||
|
||||
thd->lex->reset_n_backup_query_tables_list(&lex_backup);
|
||||
@ -347,8 +351,11 @@ rpl_slave_state::record_gtid(THD *thd, const rpl_gtid *gtid, uint64 sub_id,
|
||||
|
||||
table->no_replicate= 1;
|
||||
if (!in_transaction)
|
||||
{
|
||||
DBUG_PRINT("info", ("resetting OPTION_BEGIN"));
|
||||
thd->variables.option_bits&=
|
||||
~(ulonglong)(OPTION_NOT_AUTOCOMMIT|OPTION_BEGIN);
|
||||
}
|
||||
|
||||
bitmap_set_all(table->write_set);
|
||||
|
||||
@ -483,7 +490,7 @@ end:
|
||||
}
|
||||
thd->lex->restore_backup_query_tables_list(&lex_backup);
|
||||
thd->variables.option_bits= thd_saved_option;
|
||||
return err;
|
||||
DBUG_RETURN(err);
|
||||
}
|
||||
|
||||
|
||||
@ -491,12 +498,9 @@ uint64
|
||||
rpl_slave_state::next_sub_id(uint32 domain_id)
|
||||
{
|
||||
uint64 sub_id= 0;
|
||||
element *elem;
|
||||
|
||||
lock();
|
||||
elem= get_element(domain_id);
|
||||
if (elem)
|
||||
sub_id= ++elem->last_sub_id;
|
||||
sub_id= ++last_sub_id;
|
||||
unlock();
|
||||
|
||||
return sub_id;
|
||||
|
@ -60,7 +60,6 @@ struct rpl_slave_state
|
||||
struct element
|
||||
{
|
||||
struct list_element *list;
|
||||
uint64 last_sub_id;
|
||||
uint32 domain_id;
|
||||
|
||||
list_element *grab_list() { list_element *l= list; list= NULL; return l; }
|
||||
@ -68,8 +67,6 @@ struct rpl_slave_state
|
||||
{
|
||||
l->next= list;
|
||||
list= l;
|
||||
if (last_sub_id < l->sub_id)
|
||||
last_sub_id= l->sub_id;
|
||||
}
|
||||
};
|
||||
|
||||
@ -78,6 +75,7 @@ struct rpl_slave_state
|
||||
/* Mutex protecting access to the state. */
|
||||
mysql_mutex_t LOCK_slave_state;
|
||||
|
||||
uint64 last_sub_id;
|
||||
bool inited;
|
||||
bool loaded;
|
||||
|
||||
@ -108,7 +106,7 @@ struct rpl_slave_state
|
||||
int put_back_list(uint32 domain_id, list_element *list);
|
||||
|
||||
void update_state_hash(uint64 sub_id, rpl_gtid *gtid);
|
||||
int record_and_update_gtid(THD *thd, Relay_log_info *rli);
|
||||
int record_and_update_gtid(THD *thd, struct rpl_group_info *rgi);
|
||||
};
|
||||
|
||||
|
||||
|
956
sql/rpl_parallel.cc
Normal file
956
sql/rpl_parallel.cc
Normal file
@ -0,0 +1,956 @@
|
||||
#include "my_global.h"
|
||||
#include "rpl_parallel.h"
|
||||
#include "slave.h"
|
||||
#include "rpl_mi.h"
|
||||
|
||||
|
||||
/*
|
||||
Code for optional parallel execution of replicated events on the slave.
|
||||
|
||||
ToDo list:
|
||||
|
||||
- Retry of failed transactions is not yet implemented for the parallel case.
|
||||
|
||||
- All the waits (eg. in struct wait_for_commit and in
|
||||
rpl_parallel_thread_pool::get_thread()) need to be killable. And on kill,
|
||||
everything needs to be correctly rolled back and stopped in all threads,
|
||||
to ensure a consistent slave replication state.
|
||||
|
||||
- Handle the case of a partial event group. This occurs when the master
|
||||
crashes in the middle of writing the event group to the binlog. The
|
||||
slave rolls back the transaction; parallel execution needs to be able
|
||||
to deal with this wrt. commit_orderer and such.
|
||||
See Format_description_log_event::do_apply_event().
|
||||
*/
|
||||
|
||||
struct rpl_parallel_thread_pool global_rpl_thread_pool;
|
||||
|
||||
|
||||
static int
|
||||
rpt_handle_event(rpl_parallel_thread::queued_event *qev,
|
||||
struct rpl_parallel_thread *rpt)
|
||||
{
|
||||
int err __attribute__((unused));
|
||||
rpl_group_info *rgi= qev->rgi;
|
||||
Relay_log_info *rli= rgi->rli;
|
||||
THD *thd= rgi->thd;
|
||||
|
||||
thd->rgi_slave= rgi;
|
||||
thd->rpl_filter = rli->mi->rpl_filter;
|
||||
|
||||
/* ToDo: Access to thd, and what about rli, split out a parallel part? */
|
||||
mysql_mutex_lock(&rli->data_lock);
|
||||
qev->ev->thd= thd;
|
||||
strcpy(rgi->event_relay_log_name_buf, qev->event_relay_log_name);
|
||||
rgi->event_relay_log_name= rgi->event_relay_log_name_buf;
|
||||
rgi->event_relay_log_pos= qev->event_relay_log_pos;
|
||||
rgi->future_event_relay_log_pos= qev->future_event_relay_log_pos;
|
||||
strcpy(rgi->future_event_master_log_name, qev->future_event_master_log_name);
|
||||
err= apply_event_and_update_pos(qev->ev, thd, rgi, rpt);
|
||||
thd->rgi_slave= NULL;
|
||||
|
||||
thread_safe_increment64(&rli->executed_entries,
|
||||
&slave_executed_entries_lock);
|
||||
/* ToDo: error handling. */
|
||||
return err;
|
||||
}
|
||||
|
||||
|
||||
static void
|
||||
handle_queued_pos_update(THD *thd, rpl_parallel_thread::queued_event *qev)
|
||||
{
|
||||
int cmp;
|
||||
Relay_log_info *rli;
|
||||
/*
|
||||
Events that are not part of an event group, such as Format Description,
|
||||
Stop, GTID List and such, are executed directly in the driver SQL thread,
|
||||
to keep the relay log state up-to-date. But the associated position update
|
||||
is done here, in sync with other normal events as they are queued to
|
||||
worker threads.
|
||||
*/
|
||||
if ((thd->variables.option_bits & OPTION_BEGIN) &&
|
||||
opt_using_transactions)
|
||||
return;
|
||||
rli= qev->rgi->rli;
|
||||
mysql_mutex_lock(&rli->data_lock);
|
||||
cmp= strcmp(rli->group_relay_log_name, qev->event_relay_log_name);
|
||||
if (cmp < 0)
|
||||
{
|
||||
rli->group_relay_log_pos= qev->future_event_relay_log_pos;
|
||||
strmake_buf(rli->group_relay_log_name, qev->event_relay_log_name);
|
||||
rli->notify_group_relay_log_name_update();
|
||||
} else if (cmp == 0 &&
|
||||
rli->group_relay_log_pos < qev->future_event_relay_log_pos)
|
||||
rli->group_relay_log_pos= qev->future_event_relay_log_pos;
|
||||
|
||||
cmp= strcmp(rli->group_master_log_name, qev->future_event_master_log_name);
|
||||
if (cmp < 0)
|
||||
{
|
||||
strcpy(rli->group_master_log_name, qev->future_event_master_log_name);
|
||||
rli->notify_group_master_log_name_update();
|
||||
rli->group_master_log_pos= qev->future_event_master_log_pos;
|
||||
}
|
||||
else if (cmp == 0
|
||||
&& rli->group_master_log_pos < qev->future_event_master_log_pos)
|
||||
rli->group_master_log_pos= qev->future_event_master_log_pos;
|
||||
mysql_mutex_unlock(&rli->data_lock);
|
||||
mysql_cond_broadcast(&rli->data_cond);
|
||||
}
|
||||
|
||||
|
||||
static bool
|
||||
sql_worker_killed(THD *thd, rpl_group_info *rgi, bool in_event_group)
|
||||
{
|
||||
if (!rgi->rli->abort_slave && !abort_loop)
|
||||
return false;
|
||||
|
||||
/*
|
||||
Do not abort in the middle of an event group that cannot be rolled back.
|
||||
*/
|
||||
if ((thd->transaction.all.modified_non_trans_table ||
|
||||
(thd->variables.option_bits & OPTION_KEEP_LOG))
|
||||
&& in_event_group)
|
||||
return false;
|
||||
/* ToDo: should we add some timeout like in sql_slave_killed?
|
||||
if (rgi->last_event_start_time == 0)
|
||||
rgi->last_event_start_time= my_time(0);
|
||||
*/
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
pthread_handler_t
|
||||
handle_rpl_parallel_thread(void *arg)
|
||||
{
|
||||
THD *thd;
|
||||
const char* old_msg;
|
||||
struct rpl_parallel_thread::queued_event *events;
|
||||
bool group_standalone= true;
|
||||
bool in_event_group= false;
|
||||
uint64 event_gtid_sub_id= 0;
|
||||
int err;
|
||||
|
||||
struct rpl_parallel_thread *rpt= (struct rpl_parallel_thread *)arg;
|
||||
|
||||
my_thread_init();
|
||||
thd = new THD;
|
||||
thd->thread_stack = (char*)&thd;
|
||||
mysql_mutex_lock(&LOCK_thread_count);
|
||||
thd->thread_id= thd->variables.pseudo_thread_id= thread_id++;
|
||||
threads.append(thd);
|
||||
mysql_mutex_unlock(&LOCK_thread_count);
|
||||
set_current_thd(thd);
|
||||
pthread_detach_this_thread();
|
||||
thd->init_for_queries();
|
||||
thd->variables.binlog_annotate_row_events= 0;
|
||||
init_thr_lock();
|
||||
thd->store_globals();
|
||||
thd->system_thread= SYSTEM_THREAD_SLAVE_SQL;
|
||||
thd->security_ctx->skip_grants();
|
||||
thd->variables.max_allowed_packet= slave_max_allowed_packet;
|
||||
thd->slave_thread= 1;
|
||||
thd->enable_slow_log= opt_log_slow_slave_statements;
|
||||
thd->variables.log_slow_filter= global_system_variables.log_slow_filter;
|
||||
set_slave_thread_options(thd);
|
||||
thd->client_capabilities = CLIENT_LOCAL_FILES;
|
||||
thd_proc_info(thd, "Waiting for work from main SQL threads");
|
||||
thd->set_time();
|
||||
thd->variables.lock_wait_timeout= LONG_TIMEOUT;
|
||||
|
||||
mysql_mutex_lock(&rpt->LOCK_rpl_thread);
|
||||
rpt->thd= thd;
|
||||
|
||||
while (rpt->delay_start)
|
||||
mysql_cond_wait(&rpt->COND_rpl_thread, &rpt->LOCK_rpl_thread);
|
||||
|
||||
rpt->running= true;
|
||||
mysql_cond_signal(&rpt->COND_rpl_thread);
|
||||
|
||||
while (!rpt->stop && !thd->killed)
|
||||
{
|
||||
rpl_parallel_thread *list;
|
||||
|
||||
old_msg= thd->proc_info;
|
||||
thd->enter_cond(&rpt->COND_rpl_thread, &rpt->LOCK_rpl_thread,
|
||||
"Waiting for work from SQL thread");
|
||||
while (!(events= rpt->event_queue) && !rpt->stop && !thd->killed)
|
||||
mysql_cond_wait(&rpt->COND_rpl_thread, &rpt->LOCK_rpl_thread);
|
||||
rpt->dequeue(events);
|
||||
thd->exit_cond(old_msg);
|
||||
mysql_cond_signal(&rpt->COND_rpl_thread);
|
||||
|
||||
more_events:
|
||||
while (events)
|
||||
{
|
||||
struct rpl_parallel_thread::queued_event *next= events->next;
|
||||
Log_event_type event_type;
|
||||
rpl_group_info *rgi= events->rgi;
|
||||
rpl_parallel_entry *entry= rgi->parallel_entry;
|
||||
uint64 wait_for_sub_id;
|
||||
uint64 wait_start_sub_id;
|
||||
bool end_of_group;
|
||||
|
||||
if (!events->ev)
|
||||
{
|
||||
handle_queued_pos_update(thd, events);
|
||||
my_free(events);
|
||||
events= next;
|
||||
continue;
|
||||
}
|
||||
|
||||
err= 0;
|
||||
/* Handle a new event group, which will be initiated by a GTID event. */
|
||||
if ((event_type= events->ev->get_type_code()) == GTID_EVENT)
|
||||
{
|
||||
in_event_group= true;
|
||||
/*
|
||||
If the standalone flag is set, then this event group consists of a
|
||||
single statement (possibly preceeded by some Intvar_log_event and
|
||||
similar), without any terminating COMMIT/ROLLBACK/XID.
|
||||
*/
|
||||
group_standalone=
|
||||
(0 != (static_cast<Gtid_log_event *>(events->ev)->flags2 &
|
||||
Gtid_log_event::FL_STANDALONE));
|
||||
|
||||
/* Save this, as it gets cleared when the event group commits. */
|
||||
event_gtid_sub_id= rgi->gtid_sub_id;
|
||||
|
||||
rgi->thd= thd;
|
||||
|
||||
/*
|
||||
Register ourself to wait for the previous commit, if we need to do
|
||||
such registration _and_ that previous commit has not already
|
||||
occured.
|
||||
|
||||
Also do not start parallel execution of this event group until all
|
||||
prior groups have committed that are not safe to run in parallel with.
|
||||
*/
|
||||
wait_for_sub_id= rgi->wait_commit_sub_id;
|
||||
wait_start_sub_id= rgi->wait_start_sub_id;
|
||||
if (wait_for_sub_id || wait_start_sub_id)
|
||||
{
|
||||
mysql_mutex_lock(&entry->LOCK_parallel_entry);
|
||||
if (wait_start_sub_id)
|
||||
{
|
||||
while (wait_start_sub_id > entry->last_committed_sub_id)
|
||||
mysql_cond_wait(&entry->COND_parallel_entry,
|
||||
&entry->LOCK_parallel_entry);
|
||||
}
|
||||
rgi->wait_start_sub_id= 0; /* No need to check again. */
|
||||
if (wait_for_sub_id > entry->last_committed_sub_id)
|
||||
{
|
||||
wait_for_commit *waitee=
|
||||
&rgi->wait_commit_group_info->commit_orderer;
|
||||
rgi->commit_orderer.register_wait_for_prior_commit(waitee);
|
||||
}
|
||||
mysql_mutex_unlock(&entry->LOCK_parallel_entry);
|
||||
}
|
||||
|
||||
if(thd->wait_for_commit_ptr)
|
||||
{
|
||||
/*
|
||||
This indicates that we get a new GTID event in the middle of
|
||||
a not completed event group. This is corrupt binlog (the master
|
||||
will never write such binlog), so it does not happen unless
|
||||
someone tries to inject wrong crafted binlog, but let us still
|
||||
try to handle it somewhat nicely.
|
||||
*/
|
||||
rgi->cleanup_context(thd, true);
|
||||
thd->wait_for_commit_ptr->unregister_wait_for_prior_commit();
|
||||
thd->wait_for_commit_ptr->wakeup_subsequent_commits(err);
|
||||
}
|
||||
thd->wait_for_commit_ptr= &rgi->commit_orderer;
|
||||
}
|
||||
|
||||
/*
|
||||
If the SQL thread is stopping, we just skip execution of all the
|
||||
following event groups. We still do all the normal waiting and wakeup
|
||||
processing between the event groups as a simple way to ensure that
|
||||
everything is stopped and cleaned up correctly.
|
||||
*/
|
||||
if (!rgi->is_error && !sql_worker_killed(thd, rgi, in_event_group))
|
||||
err= rpt_handle_event(events, rpt);
|
||||
else
|
||||
err= thd->wait_for_prior_commit();
|
||||
|
||||
end_of_group=
|
||||
in_event_group &&
|
||||
((group_standalone && !Log_event::is_part_of_group(event_type)) ||
|
||||
event_type == XID_EVENT ||
|
||||
(event_type == QUERY_EVENT &&
|
||||
(((Query_log_event *)events->ev)->is_commit() ||
|
||||
((Query_log_event *)events->ev)->is_rollback())));
|
||||
|
||||
delete_or_keep_event_post_apply(rgi, event_type, events->ev);
|
||||
my_free(events);
|
||||
|
||||
if (err)
|
||||
{
|
||||
rgi->is_error= true;
|
||||
slave_output_error_info(rgi->rli, thd);
|
||||
rgi->cleanup_context(thd, true);
|
||||
rgi->rli->abort_slave= true;
|
||||
}
|
||||
if (end_of_group)
|
||||
{
|
||||
in_event_group= false;
|
||||
|
||||
/*
|
||||
Remove any left-over registration to wait for a prior commit to
|
||||
complete. Normally, such wait would already have been removed at
|
||||
this point by wait_for_prior_commit(), but eg. in error case we
|
||||
might have skipped waiting, so we would need to remove it explicitly.
|
||||
*/
|
||||
rgi->commit_orderer.unregister_wait_for_prior_commit();
|
||||
thd->wait_for_commit_ptr= NULL;
|
||||
|
||||
/*
|
||||
Record that this event group has finished (eg. transaction is
|
||||
committed, if transactional), so other event groups will no longer
|
||||
attempt to wait for us to commit. Once we have increased
|
||||
entry->last_committed_sub_id, no other threads will execute
|
||||
register_wait_for_prior_commit() against us. Thus, by doing one
|
||||
extra (usually redundant) wakeup_subsequent_commits() we can ensure
|
||||
that no register_wait_for_prior_commit() can ever happen without a
|
||||
subsequent wakeup_subsequent_commits() to wake it up.
|
||||
|
||||
We can race here with the next transactions, but that is fine, as
|
||||
long as we check that we do not decrease last_committed_sub_id. If
|
||||
this commit is done, then any prior commits will also have been
|
||||
done and also no longer need waiting for.
|
||||
*/
|
||||
mysql_mutex_lock(&entry->LOCK_parallel_entry);
|
||||
if (entry->last_committed_sub_id < event_gtid_sub_id)
|
||||
{
|
||||
entry->last_committed_sub_id= event_gtid_sub_id;
|
||||
mysql_cond_broadcast(&entry->COND_parallel_entry);
|
||||
}
|
||||
mysql_mutex_unlock(&entry->LOCK_parallel_entry);
|
||||
|
||||
rgi->commit_orderer.wakeup_subsequent_commits(err);
|
||||
delete rgi;
|
||||
}
|
||||
|
||||
events= next;
|
||||
}
|
||||
|
||||
mysql_mutex_lock(&rpt->LOCK_rpl_thread);
|
||||
if ((events= rpt->event_queue) != NULL)
|
||||
{
|
||||
/*
|
||||
Take next group of events from the replication pool.
|
||||
This is faster than having to wakeup the pool manager thread to give us
|
||||
a new event.
|
||||
*/
|
||||
rpt->dequeue(events);
|
||||
mysql_mutex_unlock(&rpt->LOCK_rpl_thread);
|
||||
mysql_cond_signal(&rpt->COND_rpl_thread);
|
||||
goto more_events;
|
||||
}
|
||||
|
||||
if (!in_event_group)
|
||||
{
|
||||
rpt->current_entry= NULL;
|
||||
if (!rpt->stop)
|
||||
{
|
||||
mysql_mutex_lock(&rpt->pool->LOCK_rpl_thread_pool);
|
||||
list= rpt->pool->free_list;
|
||||
rpt->next= list;
|
||||
rpt->pool->free_list= rpt;
|
||||
if (!list)
|
||||
mysql_cond_broadcast(&rpt->pool->COND_rpl_thread_pool);
|
||||
mysql_mutex_unlock(&rpt->pool->LOCK_rpl_thread_pool);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
rpt->thd= NULL;
|
||||
mysql_mutex_unlock(&rpt->LOCK_rpl_thread);
|
||||
|
||||
thd->clear_error();
|
||||
thd->catalog= 0;
|
||||
thd->reset_query();
|
||||
thd->reset_db(NULL, 0);
|
||||
thd_proc_info(thd, "Slave worker thread exiting");
|
||||
thd->temporary_tables= 0;
|
||||
mysql_mutex_lock(&LOCK_thread_count);
|
||||
THD_CHECK_SENTRY(thd);
|
||||
delete thd;
|
||||
mysql_mutex_unlock(&LOCK_thread_count);
|
||||
|
||||
mysql_mutex_lock(&rpt->LOCK_rpl_thread);
|
||||
rpt->running= false;
|
||||
mysql_cond_signal(&rpt->COND_rpl_thread);
|
||||
mysql_mutex_unlock(&rpt->LOCK_rpl_thread);
|
||||
|
||||
my_thread_end();
|
||||
|
||||
return NULL;
|
||||
}
|
||||
|
||||
|
||||
int
|
||||
rpl_parallel_change_thread_count(rpl_parallel_thread_pool *pool,
|
||||
uint32 new_count, bool skip_check)
|
||||
{
|
||||
uint32 i;
|
||||
rpl_parallel_thread **new_list= NULL;
|
||||
rpl_parallel_thread *new_free_list= NULL;
|
||||
rpl_parallel_thread *rpt_array= NULL;
|
||||
|
||||
/*
|
||||
Allocate the new list of threads up-front.
|
||||
That way, if we fail half-way, we only need to free whatever we managed
|
||||
to allocate, and will not be left with a half-functional thread pool.
|
||||
*/
|
||||
if (new_count &&
|
||||
!my_multi_malloc(MYF(MY_WME|MY_ZEROFILL),
|
||||
&new_list, new_count*sizeof(*new_list),
|
||||
&rpt_array, new_count*sizeof(*rpt_array),
|
||||
NULL))
|
||||
{
|
||||
my_error(ER_OUTOFMEMORY, MYF(0), (int(new_count*sizeof(*new_list) +
|
||||
new_count*sizeof(*rpt_array))));
|
||||
goto err;;
|
||||
}
|
||||
|
||||
for (i= 0; i < new_count; ++i)
|
||||
{
|
||||
pthread_t th;
|
||||
|
||||
new_list[i]= &rpt_array[i];
|
||||
new_list[i]->delay_start= true;
|
||||
mysql_mutex_init(key_LOCK_rpl_thread, &new_list[i]->LOCK_rpl_thread,
|
||||
MY_MUTEX_INIT_SLOW);
|
||||
mysql_cond_init(key_COND_rpl_thread, &new_list[i]->COND_rpl_thread, NULL);
|
||||
new_list[i]->pool= pool;
|
||||
if (mysql_thread_create(key_rpl_parallel_thread, &th, NULL,
|
||||
handle_rpl_parallel_thread, new_list[i]))
|
||||
{
|
||||
my_error(ER_OUT_OF_RESOURCES, MYF(0));
|
||||
goto err;
|
||||
}
|
||||
new_list[i]->next= new_free_list;
|
||||
new_free_list= new_list[i];
|
||||
}
|
||||
|
||||
if (!skip_check)
|
||||
{
|
||||
mysql_mutex_lock(&LOCK_active_mi);
|
||||
if (master_info_index->give_error_if_slave_running())
|
||||
{
|
||||
mysql_mutex_unlock(&LOCK_active_mi);
|
||||
goto err;
|
||||
}
|
||||
if (pool->changing)
|
||||
{
|
||||
mysql_mutex_unlock(&LOCK_active_mi);
|
||||
my_error(ER_CHANGE_SLAVE_PARALLEL_THREADS_ACTIVE, MYF(0));
|
||||
goto err;
|
||||
}
|
||||
pool->changing= true;
|
||||
mysql_mutex_unlock(&LOCK_active_mi);
|
||||
}
|
||||
|
||||
/*
|
||||
Grab each old thread in turn, and signal it to stop.
|
||||
|
||||
Note that since we require all replication threads to be stopped before
|
||||
changing the parallel replication worker thread pool, all the threads will
|
||||
be already idle and will terminate immediately.
|
||||
*/
|
||||
for (i= 0; i < pool->count; ++i)
|
||||
{
|
||||
rpl_parallel_thread *rpt= pool->get_thread(NULL);
|
||||
rpt->stop= true;
|
||||
mysql_cond_signal(&rpt->COND_rpl_thread);
|
||||
mysql_mutex_unlock(&rpt->LOCK_rpl_thread);
|
||||
}
|
||||
|
||||
for (i= 0; i < pool->count; ++i)
|
||||
{
|
||||
rpl_parallel_thread *rpt= pool->threads[i];
|
||||
mysql_mutex_lock(&rpt->LOCK_rpl_thread);
|
||||
while (rpt->running)
|
||||
mysql_cond_wait(&rpt->COND_rpl_thread, &rpt->LOCK_rpl_thread);
|
||||
mysql_mutex_unlock(&rpt->LOCK_rpl_thread);
|
||||
mysql_mutex_destroy(&rpt->LOCK_rpl_thread);
|
||||
mysql_cond_destroy(&rpt->COND_rpl_thread);
|
||||
}
|
||||
|
||||
my_free(pool->threads);
|
||||
pool->threads= new_list;
|
||||
pool->free_list= new_free_list;
|
||||
pool->count= new_count;
|
||||
for (i= 0; i < pool->count; ++i)
|
||||
{
|
||||
mysql_mutex_lock(&pool->threads[i]->LOCK_rpl_thread);
|
||||
pool->threads[i]->delay_start= false;
|
||||
mysql_cond_signal(&pool->threads[i]->COND_rpl_thread);
|
||||
while (!pool->threads[i]->running)
|
||||
mysql_cond_wait(&pool->threads[i]->COND_rpl_thread,
|
||||
&pool->threads[i]->LOCK_rpl_thread);
|
||||
mysql_mutex_unlock(&pool->threads[i]->LOCK_rpl_thread);
|
||||
}
|
||||
|
||||
if (!skip_check)
|
||||
{
|
||||
mysql_mutex_lock(&LOCK_active_mi);
|
||||
pool->changing= false;
|
||||
mysql_mutex_unlock(&LOCK_active_mi);
|
||||
}
|
||||
return 0;
|
||||
|
||||
err:
|
||||
if (new_list)
|
||||
{
|
||||
while (new_free_list)
|
||||
{
|
||||
mysql_mutex_lock(&new_free_list->LOCK_rpl_thread);
|
||||
new_free_list->delay_start= false;
|
||||
new_free_list->stop= true;
|
||||
mysql_cond_signal(&new_free_list->COND_rpl_thread);
|
||||
while (!new_free_list->running)
|
||||
mysql_cond_wait(&new_free_list->COND_rpl_thread,
|
||||
&new_free_list->LOCK_rpl_thread);
|
||||
while (new_free_list->running)
|
||||
mysql_cond_wait(&new_free_list->COND_rpl_thread,
|
||||
&new_free_list->LOCK_rpl_thread);
|
||||
mysql_mutex_unlock(&new_free_list->LOCK_rpl_thread);
|
||||
new_free_list= new_free_list->next;
|
||||
}
|
||||
my_free(new_list);
|
||||
}
|
||||
if (!skip_check)
|
||||
{
|
||||
mysql_mutex_lock(&LOCK_active_mi);
|
||||
pool->changing= false;
|
||||
mysql_mutex_unlock(&LOCK_active_mi);
|
||||
}
|
||||
return 1;
|
||||
}
|
||||
|
||||
|
||||
rpl_parallel_thread_pool::rpl_parallel_thread_pool()
|
||||
: count(0), threads(0), free_list(0), changing(false), inited(false)
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
int
|
||||
rpl_parallel_thread_pool::init(uint32 size)
|
||||
{
|
||||
count= 0;
|
||||
threads= NULL;
|
||||
free_list= NULL;
|
||||
|
||||
mysql_mutex_init(key_LOCK_rpl_thread_pool, &LOCK_rpl_thread_pool,
|
||||
MY_MUTEX_INIT_SLOW);
|
||||
mysql_cond_init(key_COND_rpl_thread_pool, &COND_rpl_thread_pool, NULL);
|
||||
changing= false;
|
||||
inited= true;
|
||||
|
||||
return rpl_parallel_change_thread_count(this, size, true);
|
||||
}
|
||||
|
||||
|
||||
void
|
||||
rpl_parallel_thread_pool::destroy()
|
||||
{
|
||||
if (!inited)
|
||||
return;
|
||||
rpl_parallel_change_thread_count(this, 0, true);
|
||||
mysql_mutex_destroy(&LOCK_rpl_thread_pool);
|
||||
mysql_cond_destroy(&COND_rpl_thread_pool);
|
||||
inited= false;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
Wait for a worker thread to become idle. When one does, grab the thread for
|
||||
our use and return it.
|
||||
|
||||
Note that we return with the worker threads's LOCK_rpl_thread mutex locked.
|
||||
*/
|
||||
struct rpl_parallel_thread *
|
||||
rpl_parallel_thread_pool::get_thread(rpl_parallel_entry *entry)
|
||||
{
|
||||
rpl_parallel_thread *rpt;
|
||||
|
||||
mysql_mutex_lock(&LOCK_rpl_thread_pool);
|
||||
while ((rpt= free_list) == NULL)
|
||||
mysql_cond_wait(&COND_rpl_thread_pool, &LOCK_rpl_thread_pool);
|
||||
free_list= rpt->next;
|
||||
mysql_mutex_unlock(&LOCK_rpl_thread_pool);
|
||||
mysql_mutex_lock(&rpt->LOCK_rpl_thread);
|
||||
rpt->current_entry= entry;
|
||||
|
||||
return rpt;
|
||||
}
|
||||
|
||||
|
||||
static void
|
||||
free_rpl_parallel_entry(void *element)
|
||||
{
|
||||
rpl_parallel_entry *e= (rpl_parallel_entry *)element;
|
||||
mysql_cond_destroy(&e->COND_parallel_entry);
|
||||
mysql_mutex_destroy(&e->LOCK_parallel_entry);
|
||||
my_free(e);
|
||||
}
|
||||
|
||||
|
||||
rpl_parallel::rpl_parallel() :
|
||||
current(NULL), sql_thread_stopping(false)
|
||||
{
|
||||
my_hash_init(&domain_hash, &my_charset_bin, 32,
|
||||
offsetof(rpl_parallel_entry, domain_id), sizeof(uint32),
|
||||
NULL, free_rpl_parallel_entry, HASH_UNIQUE);
|
||||
}
|
||||
|
||||
|
||||
void
|
||||
rpl_parallel::reset()
|
||||
{
|
||||
my_hash_reset(&domain_hash);
|
||||
current= NULL;
|
||||
sql_thread_stopping= false;
|
||||
}
|
||||
|
||||
|
||||
rpl_parallel::~rpl_parallel()
|
||||
{
|
||||
my_hash_free(&domain_hash);
|
||||
}
|
||||
|
||||
|
||||
rpl_parallel_entry *
|
||||
rpl_parallel::find(uint32 domain_id)
|
||||
{
|
||||
struct rpl_parallel_entry *e;
|
||||
|
||||
if (!(e= (rpl_parallel_entry *)my_hash_search(&domain_hash,
|
||||
(const uchar *)&domain_id, 0)))
|
||||
{
|
||||
/* Allocate a new, empty one. */
|
||||
if (!(e= (struct rpl_parallel_entry *)my_malloc(sizeof(*e),
|
||||
MYF(MY_ZEROFILL))))
|
||||
return NULL;
|
||||
e->domain_id= domain_id;
|
||||
if (my_hash_insert(&domain_hash, (uchar *)e))
|
||||
{
|
||||
my_free(e);
|
||||
return NULL;
|
||||
}
|
||||
mysql_mutex_init(key_LOCK_parallel_entry, &e->LOCK_parallel_entry,
|
||||
MY_MUTEX_INIT_FAST);
|
||||
mysql_cond_init(key_COND_parallel_entry, &e->COND_parallel_entry, NULL);
|
||||
}
|
||||
|
||||
return e;
|
||||
}
|
||||
|
||||
|
||||
void
|
||||
rpl_parallel::wait_for_done()
|
||||
{
|
||||
struct rpl_parallel_entry *e;
|
||||
uint32 i;
|
||||
|
||||
for (i= 0; i < domain_hash.records; ++i)
|
||||
{
|
||||
e= (struct rpl_parallel_entry *)my_hash_element(&domain_hash, i);
|
||||
mysql_mutex_lock(&e->LOCK_parallel_entry);
|
||||
while (e->current_sub_id > e->last_committed_sub_id)
|
||||
mysql_cond_wait(&e->COND_parallel_entry, &e->LOCK_parallel_entry);
|
||||
mysql_mutex_unlock(&e->LOCK_parallel_entry);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
do_event() is executed by the sql_driver_thd thread.
|
||||
It's main purpose is to find a thread that can execute the query.
|
||||
|
||||
@retval false ok, event was accepted
|
||||
@retval true error
|
||||
*/
|
||||
|
||||
bool
|
||||
rpl_parallel::do_event(rpl_group_info *serial_rgi, Log_event *ev,
|
||||
ulonglong event_size)
|
||||
{
|
||||
rpl_parallel_entry *e;
|
||||
rpl_parallel_thread *cur_thread;
|
||||
rpl_parallel_thread::queued_event *qev;
|
||||
rpl_group_info *rgi= NULL;
|
||||
Relay_log_info *rli= serial_rgi->rli;
|
||||
enum Log_event_type typ;
|
||||
bool is_group_event;
|
||||
|
||||
/* ToDo: what to do with this lock?!? */
|
||||
mysql_mutex_unlock(&rli->data_lock);
|
||||
|
||||
/*
|
||||
Stop queueing additional event groups once the SQL thread is requested to
|
||||
stop.
|
||||
*/
|
||||
if (((typ= ev->get_type_code()) == GTID_EVENT ||
|
||||
!(is_group_event= Log_event::is_group_event(typ))) &&
|
||||
rli->abort_slave)
|
||||
sql_thread_stopping= true;
|
||||
if (sql_thread_stopping)
|
||||
{
|
||||
/* QQ: Need a better comment why we return false here */
|
||||
return false;
|
||||
}
|
||||
|
||||
if (!(qev= (rpl_parallel_thread::queued_event *)my_malloc(sizeof(*qev),
|
||||
MYF(0))))
|
||||
{
|
||||
my_error(ER_OUT_OF_RESOURCES, MYF(0));
|
||||
return true;
|
||||
}
|
||||
qev->ev= ev;
|
||||
qev->event_size= event_size;
|
||||
qev->next= NULL;
|
||||
strcpy(qev->event_relay_log_name, rli->event_relay_log_name);
|
||||
qev->event_relay_log_pos= rli->event_relay_log_pos;
|
||||
qev->future_event_relay_log_pos= rli->future_event_relay_log_pos;
|
||||
strcpy(qev->future_event_master_log_name, rli->future_event_master_log_name);
|
||||
|
||||
if (typ == GTID_EVENT)
|
||||
{
|
||||
Gtid_log_event *gtid_ev= static_cast<Gtid_log_event *>(ev);
|
||||
uint32 domain_id= (rli->mi->using_gtid == Master_info::USE_GTID_NO ?
|
||||
0 : gtid_ev->domain_id);
|
||||
|
||||
if (!(e= find(domain_id)) ||
|
||||
!(rgi= new rpl_group_info(rli)) ||
|
||||
event_group_new_gtid(rgi, gtid_ev))
|
||||
{
|
||||
my_error(ER_OUT_OF_RESOURCES, MYF(MY_WME));
|
||||
delete rgi;
|
||||
return true;
|
||||
}
|
||||
rgi->is_parallel_exec = true;
|
||||
if ((rgi->deferred_events_collecting= rli->mi->rpl_filter->is_on()))
|
||||
rgi->deferred_events= new Deferred_log_events(rli);
|
||||
|
||||
if ((gtid_ev->flags2 & Gtid_log_event::FL_GROUP_COMMIT_ID) &&
|
||||
e->last_commit_id == gtid_ev->commit_id)
|
||||
{
|
||||
/*
|
||||
We are already executing something else in this domain. But the two
|
||||
event groups were committed together in the same group commit on the
|
||||
master, so we can still do them in parallel here on the slave.
|
||||
|
||||
However, the commit of this event must wait for the commit of the prior
|
||||
event, to preserve binlog commit order and visibility across all
|
||||
servers in the replication hierarchy.
|
||||
*/
|
||||
rpl_parallel_thread *rpt= global_rpl_thread_pool.get_thread(e);
|
||||
rgi->wait_commit_sub_id= e->current_sub_id;
|
||||
rgi->wait_commit_group_info= e->current_group_info;
|
||||
rgi->wait_start_sub_id= e->prev_groupcommit_sub_id;
|
||||
e->rpl_thread= cur_thread= rpt;
|
||||
/* get_thread() returns with the LOCK_rpl_thread locked. */
|
||||
}
|
||||
else
|
||||
{
|
||||
/*
|
||||
Check if we already have a worker thread for this entry.
|
||||
|
||||
We continue to queue more events up for the worker thread while it is
|
||||
still executing the first ones, to be able to start executing a large
|
||||
event group without having to wait for the end to be fetched from the
|
||||
master. And we continue to queue up more events after the first group,
|
||||
so that we can continue to process subsequent parts of the relay log in
|
||||
parallel without having to wait for previous long-running events to
|
||||
complete.
|
||||
|
||||
But if the worker thread is idle at any point, it may return to the
|
||||
idle list or start servicing a different request. So check this, and
|
||||
allocate a new thread if the old one is no longer processing for us.
|
||||
*/
|
||||
cur_thread= e->rpl_thread;
|
||||
if (cur_thread)
|
||||
{
|
||||
mysql_mutex_lock(&cur_thread->LOCK_rpl_thread);
|
||||
for (;;)
|
||||
{
|
||||
if (cur_thread->current_entry != e)
|
||||
{
|
||||
/*
|
||||
The worker thread became idle, and returned to the free list and
|
||||
possibly was allocated to a different request. This also means
|
||||
that everything previously queued has already been executed,
|
||||
else the worker thread would not have become idle. So we should
|
||||
allocate a new worker thread.
|
||||
*/
|
||||
mysql_mutex_unlock(&cur_thread->LOCK_rpl_thread);
|
||||
e->rpl_thread= cur_thread= NULL;
|
||||
break;
|
||||
}
|
||||
else if (cur_thread->queued_size <= opt_slave_parallel_max_queued)
|
||||
break; // The thread is ready to queue into
|
||||
else
|
||||
{
|
||||
/*
|
||||
We have reached the limit of how much memory we are allowed to
|
||||
use for queuing events, so wait for the thread to consume some
|
||||
of its queue.
|
||||
*/
|
||||
mysql_cond_wait(&cur_thread->COND_rpl_thread,
|
||||
&cur_thread->LOCK_rpl_thread);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (!cur_thread)
|
||||
{
|
||||
/*
|
||||
Nothing else is currently running in this domain. We can
|
||||
spawn a new thread to do this event group in parallel with
|
||||
anything else that might be running in other domains.
|
||||
*/
|
||||
cur_thread= e->rpl_thread= global_rpl_thread_pool.get_thread(e);
|
||||
/* get_thread() returns with the LOCK_rpl_thread locked. */
|
||||
}
|
||||
else
|
||||
{
|
||||
/*
|
||||
We are still executing the previous event group for this replication
|
||||
domain, and we have to wait for that to finish before we can start on
|
||||
the next one. So just re-use the thread.
|
||||
*/
|
||||
}
|
||||
|
||||
rgi->wait_commit_sub_id= 0;
|
||||
rgi->wait_start_sub_id= 0;
|
||||
e->prev_groupcommit_sub_id= e->current_sub_id;
|
||||
}
|
||||
|
||||
if (gtid_ev->flags2 & Gtid_log_event::FL_GROUP_COMMIT_ID)
|
||||
{
|
||||
e->last_server_id= gtid_ev->server_id;
|
||||
e->last_seq_no= gtid_ev->seq_no;
|
||||
e->last_commit_id= gtid_ev->commit_id;
|
||||
}
|
||||
else
|
||||
{
|
||||
e->last_server_id= 0;
|
||||
e->last_seq_no= 0;
|
||||
e->last_commit_id= 0;
|
||||
}
|
||||
|
||||
qev->rgi= e->current_group_info= rgi;
|
||||
e->current_sub_id= rgi->gtid_sub_id;
|
||||
current= rgi->parallel_entry= e;
|
||||
}
|
||||
else if (!is_group_event || !current)
|
||||
{
|
||||
my_off_t log_pos;
|
||||
int err;
|
||||
bool tmp;
|
||||
/*
|
||||
Events like ROTATE and FORMAT_DESCRIPTION. Do not run in worker thread.
|
||||
Same for events not preceeded by GTID (we should not see those normally,
|
||||
but they might be from an old master).
|
||||
|
||||
The varuable `current' is NULL for the case where the master did not
|
||||
have GTID, like a MariaDB 5.5 or MySQL master.
|
||||
*/
|
||||
qev->rgi= serial_rgi;
|
||||
/* Handle master log name change, seen in Rotate_log_event. */
|
||||
if (typ == ROTATE_EVENT)
|
||||
{
|
||||
Rotate_log_event *rev= static_cast<Rotate_log_event *>(qev->ev);
|
||||
if ((rev->server_id != global_system_variables.server_id ||
|
||||
rli->replicate_same_server_id) &&
|
||||
!rev->is_relay_log_event() &&
|
||||
!rli->is_in_group())
|
||||
{
|
||||
memcpy(rli->future_event_master_log_name,
|
||||
rev->new_log_ident, rev->ident_len+1);
|
||||
}
|
||||
}
|
||||
|
||||
tmp= serial_rgi->is_parallel_exec;
|
||||
serial_rgi->is_parallel_exec= true;
|
||||
err= rpt_handle_event(qev, NULL);
|
||||
serial_rgi->is_parallel_exec= tmp;
|
||||
log_pos= qev->ev->log_pos;
|
||||
delete_or_keep_event_post_apply(serial_rgi, typ, qev->ev);
|
||||
|
||||
if (err)
|
||||
{
|
||||
my_free(qev);
|
||||
return true;
|
||||
}
|
||||
qev->ev= NULL;
|
||||
qev->future_event_master_log_pos= log_pos;
|
||||
if (!current)
|
||||
{
|
||||
handle_queued_pos_update(rli->sql_driver_thd, qev);
|
||||
my_free(qev);
|
||||
return false;
|
||||
}
|
||||
/*
|
||||
Queue an empty event, so that the position will be updated in a
|
||||
reasonable way relative to other events:
|
||||
|
||||
- If the currently executing events are queued serially for a single
|
||||
thread, the position will only be updated when everything before has
|
||||
completed.
|
||||
|
||||
- If we are executing multiple independent events in parallel, then at
|
||||
least the position will not be updated until one of them has reached
|
||||
the current point.
|
||||
*/
|
||||
cur_thread= current->rpl_thread;
|
||||
if (cur_thread)
|
||||
{
|
||||
mysql_mutex_lock(&cur_thread->LOCK_rpl_thread);
|
||||
if (cur_thread->current_entry != current)
|
||||
{
|
||||
/* Not ours anymore, we need to grab a new one. */
|
||||
mysql_mutex_unlock(&cur_thread->LOCK_rpl_thread);
|
||||
cur_thread= NULL;
|
||||
}
|
||||
}
|
||||
if (!cur_thread)
|
||||
cur_thread= current->rpl_thread=
|
||||
global_rpl_thread_pool.get_thread(current);
|
||||
}
|
||||
else
|
||||
{
|
||||
cur_thread= current->rpl_thread;
|
||||
if (cur_thread)
|
||||
{
|
||||
mysql_mutex_lock(&cur_thread->LOCK_rpl_thread);
|
||||
if (cur_thread->current_entry != current)
|
||||
{
|
||||
/* Not ours anymore, we need to grab a new one. */
|
||||
mysql_mutex_unlock(&cur_thread->LOCK_rpl_thread);
|
||||
cur_thread= NULL;
|
||||
}
|
||||
}
|
||||
if (!cur_thread)
|
||||
{
|
||||
cur_thread= current->rpl_thread=
|
||||
global_rpl_thread_pool.get_thread(current);
|
||||
}
|
||||
qev->rgi= current->current_group_info;
|
||||
}
|
||||
|
||||
/*
|
||||
Queue the event for processing.
|
||||
*/
|
||||
rli->event_relay_log_pos= rli->future_event_relay_log_pos;
|
||||
cur_thread->enqueue(qev);
|
||||
mysql_mutex_unlock(&cur_thread->LOCK_rpl_thread);
|
||||
mysql_cond_signal(&cur_thread->COND_rpl_thread);
|
||||
|
||||
return false;
|
||||
}
|
125
sql/rpl_parallel.h
Normal file
125
sql/rpl_parallel.h
Normal file
@ -0,0 +1,125 @@
|
||||
#ifndef RPL_PARALLEL_H
|
||||
#define RPL_PARALLEL_H
|
||||
|
||||
#include "log_event.h"
|
||||
|
||||
|
||||
struct rpl_parallel;
|
||||
struct rpl_parallel_entry;
|
||||
struct rpl_parallel_thread_pool;
|
||||
|
||||
class Relay_log_info;
|
||||
struct rpl_parallel_thread {
|
||||
bool delay_start;
|
||||
bool running;
|
||||
bool stop;
|
||||
mysql_mutex_t LOCK_rpl_thread;
|
||||
mysql_cond_t COND_rpl_thread;
|
||||
struct rpl_parallel_thread *next; /* For free list. */
|
||||
struct rpl_parallel_thread_pool *pool;
|
||||
THD *thd;
|
||||
struct rpl_parallel_entry *current_entry;
|
||||
struct queued_event {
|
||||
queued_event *next;
|
||||
Log_event *ev;
|
||||
rpl_group_info *rgi;
|
||||
ulonglong future_event_relay_log_pos;
|
||||
char event_relay_log_name[FN_REFLEN];
|
||||
char future_event_master_log_name[FN_REFLEN];
|
||||
ulonglong event_relay_log_pos;
|
||||
my_off_t future_event_master_log_pos;
|
||||
size_t event_size;
|
||||
} *event_queue, *last_in_queue;
|
||||
uint64 queued_size;
|
||||
|
||||
void enqueue(queued_event *qev)
|
||||
{
|
||||
if (last_in_queue)
|
||||
last_in_queue->next= qev;
|
||||
else
|
||||
event_queue= qev;
|
||||
last_in_queue= qev;
|
||||
queued_size+= qev->event_size;
|
||||
}
|
||||
|
||||
void dequeue(queued_event *list)
|
||||
{
|
||||
queued_event *tmp;
|
||||
|
||||
DBUG_ASSERT(list == event_queue);
|
||||
event_queue= last_in_queue= NULL;
|
||||
for (tmp= list; tmp; tmp= tmp->next)
|
||||
queued_size-= tmp->event_size;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
struct rpl_parallel_thread_pool {
|
||||
uint32 count;
|
||||
struct rpl_parallel_thread **threads;
|
||||
struct rpl_parallel_thread *free_list;
|
||||
mysql_mutex_t LOCK_rpl_thread_pool;
|
||||
mysql_cond_t COND_rpl_thread_pool;
|
||||
bool changing;
|
||||
bool inited;
|
||||
|
||||
rpl_parallel_thread_pool();
|
||||
int init(uint32 size);
|
||||
void destroy();
|
||||
struct rpl_parallel_thread *get_thread(rpl_parallel_entry *entry);
|
||||
};
|
||||
|
||||
|
||||
struct rpl_parallel_entry {
|
||||
uint32 domain_id;
|
||||
uint32 last_server_id;
|
||||
uint64 last_seq_no;
|
||||
uint64 last_commit_id;
|
||||
bool active;
|
||||
rpl_parallel_thread *rpl_thread;
|
||||
/*
|
||||
The sub_id of the last transaction to commit within this domain_id.
|
||||
Must be accessed under LOCK_parallel_entry protection.
|
||||
*/
|
||||
uint64 last_committed_sub_id;
|
||||
mysql_mutex_t LOCK_parallel_entry;
|
||||
mysql_cond_t COND_parallel_entry;
|
||||
/*
|
||||
The sub_id of the last event group in this replication domain that was
|
||||
queued for execution by a worker thread.
|
||||
*/
|
||||
uint64 current_sub_id;
|
||||
rpl_group_info *current_group_info;
|
||||
/*
|
||||
The sub_id of the last event group in the previous batch of group-committed
|
||||
transactions.
|
||||
|
||||
When we spawn parallel worker threads for the next group-committed batch,
|
||||
they first need to wait for this sub_id to be committed before it is safe
|
||||
to start executing them.
|
||||
*/
|
||||
uint64 prev_groupcommit_sub_id;
|
||||
};
|
||||
struct rpl_parallel {
|
||||
HASH domain_hash;
|
||||
rpl_parallel_entry *current;
|
||||
bool sql_thread_stopping;
|
||||
|
||||
rpl_parallel();
|
||||
~rpl_parallel();
|
||||
void reset();
|
||||
rpl_parallel_entry *find(uint32 domain_id);
|
||||
void wait_for_done();
|
||||
bool do_event(rpl_group_info *serial_rgi, Log_event *ev,
|
||||
ulonglong event_size);
|
||||
};
|
||||
|
||||
|
||||
extern struct rpl_parallel_thread_pool global_rpl_thread_pool;
|
||||
|
||||
|
||||
extern int rpl_parallel_change_thread_count(rpl_parallel_thread_pool *pool,
|
||||
uint32 new_count,
|
||||
bool skip_check= false);
|
||||
|
||||
#endif /* RPL_PARALLEL_H */
|
@ -186,7 +186,7 @@ pack_row(TABLE *table, MY_BITMAP const* cols,
|
||||
*/
|
||||
#if !defined(MYSQL_CLIENT) && defined(HAVE_REPLICATION)
|
||||
int
|
||||
unpack_row(Relay_log_info const *rli,
|
||||
unpack_row(rpl_group_info *rgi,
|
||||
TABLE *table, uint const colcnt,
|
||||
uchar const *const row_data, uchar const *const row_buffer_end,
|
||||
MY_BITMAP const *cols,
|
||||
@ -214,18 +214,18 @@ unpack_row(Relay_log_info const *rli,
|
||||
uint i= 0;
|
||||
table_def *tabledef= NULL;
|
||||
TABLE *conv_table= NULL;
|
||||
bool table_found= rli && rli->get_table_data(table, &tabledef, &conv_table);
|
||||
bool table_found= rgi && rgi->get_table_data(table, &tabledef, &conv_table);
|
||||
DBUG_PRINT("debug", ("Table data: table_found: %d, tabldef: %p, conv_table: %p",
|
||||
table_found, tabledef, conv_table));
|
||||
DBUG_ASSERT(table_found);
|
||||
|
||||
/*
|
||||
If rli is NULL it means that there is no source table and that the
|
||||
If rgi is NULL it means that there is no source table and that the
|
||||
row shall just be unpacked without doing any checks. This feature
|
||||
is used by MySQL Backup, but can be used for other purposes as
|
||||
well.
|
||||
*/
|
||||
if (rli && !table_found)
|
||||
if (rgi && !table_found)
|
||||
DBUG_RETURN(HA_ERR_GENERIC);
|
||||
|
||||
for (field_ptr= begin_ptr ; field_ptr < end_ptr && *field_ptr ; ++field_ptr)
|
||||
@ -313,7 +313,7 @@ unpack_row(Relay_log_info const *rli,
|
||||
(int) (pack_ptr - old_pack_ptr)));
|
||||
if (!pack_ptr)
|
||||
{
|
||||
rli->report(ERROR_LEVEL, ER_SLAVE_CORRUPT_EVENT,
|
||||
rgi->rli->report(ERROR_LEVEL, ER_SLAVE_CORRUPT_EVENT,
|
||||
"Could not read field '%s' of table '%s.%s'",
|
||||
f->field_name, table->s->db.str,
|
||||
table->s->table_name.str);
|
||||
|
@ -21,7 +21,7 @@
|
||||
#include <rpl_reporting.h>
|
||||
#include "my_global.h" /* uchar */
|
||||
|
||||
class Relay_log_info;
|
||||
struct rpl_group_info;
|
||||
struct TABLE;
|
||||
typedef struct st_bitmap MY_BITMAP;
|
||||
|
||||
@ -31,7 +31,7 @@ size_t pack_row(TABLE* table, MY_BITMAP const* cols,
|
||||
#endif
|
||||
|
||||
#if !defined(MYSQL_CLIENT) && defined(HAVE_REPLICATION)
|
||||
int unpack_row(Relay_log_info const *rli,
|
||||
int unpack_row(rpl_group_info *rgi,
|
||||
TABLE *table, uint const colcnt,
|
||||
uchar const *const row_data, uchar const *row_buffer_end,
|
||||
MY_BITMAP const *cols,
|
||||
|
@ -88,7 +88,7 @@ pack_row_old(TABLE *table, MY_BITMAP const* cols,
|
||||
*/
|
||||
#if !defined(MYSQL_CLIENT) && defined(HAVE_REPLICATION)
|
||||
int
|
||||
unpack_row_old(Relay_log_info *rli,
|
||||
unpack_row_old(rpl_group_info *rgi,
|
||||
TABLE *table, uint const colcnt, uchar *record,
|
||||
uchar const *row, const uchar *row_buffer_end,
|
||||
MY_BITMAP const *cols,
|
||||
@ -141,7 +141,7 @@ unpack_row_old(Relay_log_info *rli,
|
||||
f->move_field_offset(-offset);
|
||||
if (!ptr)
|
||||
{
|
||||
rli->report(ERROR_LEVEL, ER_SLAVE_CORRUPT_EVENT,
|
||||
rgi->rli->report(ERROR_LEVEL, ER_SLAVE_CORRUPT_EVENT,
|
||||
"Could not read field `%s` of table `%s`.`%s`",
|
||||
f->field_name, table->s->db.str,
|
||||
table->s->table_name.str);
|
||||
@ -183,7 +183,7 @@ unpack_row_old(Relay_log_info *rli,
|
||||
if (event_type == WRITE_ROWS_EVENT &&
|
||||
((*field_ptr)->flags & mask) == mask)
|
||||
{
|
||||
rli->report(ERROR_LEVEL, ER_NO_DEFAULT_FOR_FIELD,
|
||||
rgi->rli->report(ERROR_LEVEL, ER_NO_DEFAULT_FOR_FIELD,
|
||||
"Field `%s` of table `%s`.`%s` "
|
||||
"has no default value and cannot be NULL",
|
||||
(*field_ptr)->field_name, table->s->db.str,
|
||||
|
@ -23,7 +23,7 @@ size_t pack_row_old(TABLE *table, MY_BITMAP const* cols,
|
||||
uchar *row_data, const uchar *record);
|
||||
|
||||
#ifdef HAVE_REPLICATION
|
||||
int unpack_row_old(Relay_log_info *rli,
|
||||
int unpack_row_old(rpl_group_info *rgi,
|
||||
TABLE *table, uint const colcnt, uchar *record,
|
||||
uchar const *row, uchar const *row_buffer_end,
|
||||
MY_BITMAP const *cols,
|
||||
|
438
sql/rpl_rli.cc
438
sql/rpl_rli.cc
@ -56,13 +56,10 @@ Relay_log_info::Relay_log_info(bool is_slave_recovery)
|
||||
#endif
|
||||
group_master_log_pos(0), log_space_total(0), ignore_log_space_limit(0),
|
||||
last_master_timestamp(0), slave_skip_counter(0),
|
||||
abort_pos_wait(0), slave_run_id(0), sql_thd(0),
|
||||
abort_pos_wait(0), slave_run_id(0), sql_driver_thd(),
|
||||
inited(0), abort_slave(0), slave_running(0), until_condition(UNTIL_NONE),
|
||||
until_log_pos(0), retried_trans(0), executed_entries(0),
|
||||
gtid_sub_id(0), tables_to_lock(0), tables_to_lock_count(0),
|
||||
last_event_start_time(0), deferred_events(NULL),m_flags(0),
|
||||
row_stmt_start_timestamp(0), long_find_row_note_printed(false),
|
||||
m_annotate_event(0)
|
||||
m_flags(0)
|
||||
{
|
||||
DBUG_ENTER("Relay_log_info::Relay_log_info");
|
||||
|
||||
@ -87,12 +84,10 @@ Relay_log_info::Relay_log_info(bool is_slave_recovery)
|
||||
&data_lock, MY_MUTEX_INIT_FAST);
|
||||
mysql_mutex_init(key_relay_log_info_log_space_lock,
|
||||
&log_space_lock, MY_MUTEX_INIT_FAST);
|
||||
mysql_mutex_init(key_relay_log_info_sleep_lock, &sleep_lock, MY_MUTEX_INIT_FAST);
|
||||
mysql_cond_init(key_relay_log_info_data_cond, &data_cond, NULL);
|
||||
mysql_cond_init(key_relay_log_info_start_cond, &start_cond, NULL);
|
||||
mysql_cond_init(key_relay_log_info_stop_cond, &stop_cond, NULL);
|
||||
mysql_cond_init(key_relay_log_info_log_space_cond, &log_space_cond, NULL);
|
||||
mysql_cond_init(key_relay_log_info_sleep_cond, &sleep_cond, NULL);
|
||||
relay_log.init_pthread_objects();
|
||||
DBUG_VOID_RETURN;
|
||||
}
|
||||
@ -105,14 +100,11 @@ Relay_log_info::~Relay_log_info()
|
||||
mysql_mutex_destroy(&run_lock);
|
||||
mysql_mutex_destroy(&data_lock);
|
||||
mysql_mutex_destroy(&log_space_lock);
|
||||
mysql_mutex_destroy(&sleep_lock);
|
||||
mysql_cond_destroy(&data_cond);
|
||||
mysql_cond_destroy(&start_cond);
|
||||
mysql_cond_destroy(&stop_cond);
|
||||
mysql_cond_destroy(&log_space_cond);
|
||||
mysql_cond_destroy(&sleep_cond);
|
||||
relay_log.cleanup();
|
||||
free_annotate_event();
|
||||
DBUG_VOID_RETURN;
|
||||
}
|
||||
|
||||
@ -137,8 +129,6 @@ int init_relay_log_info(Relay_log_info* rli,
|
||||
rli->abort_pos_wait=0;
|
||||
rli->log_space_limit= relay_log_space_limit;
|
||||
rli->log_space_total= 0;
|
||||
rli->tables_to_lock= 0;
|
||||
rli->tables_to_lock_count= 0;
|
||||
|
||||
char pattern[FN_REFLEN];
|
||||
(void) my_realpath(pattern, slave_load_tmpdir, 0);
|
||||
@ -528,6 +518,8 @@ int init_relay_log_pos(Relay_log_info* rli,const char* log,
|
||||
}
|
||||
|
||||
rli->group_relay_log_pos = rli->event_relay_log_pos = pos;
|
||||
rli->clear_flag(Relay_log_info::IN_STMT);
|
||||
rli->clear_flag(Relay_log_info::IN_TRANSACTION);
|
||||
|
||||
/*
|
||||
Test to see if the previous run was with the skip of purging
|
||||
@ -877,17 +869,54 @@ improper_arguments: %d timed_out: %d",
|
||||
|
||||
|
||||
void Relay_log_info::inc_group_relay_log_pos(ulonglong log_pos,
|
||||
bool skip_lock)
|
||||
rpl_group_info *rgi,
|
||||
bool skip_lock)
|
||||
{
|
||||
DBUG_ENTER("Relay_log_info::inc_group_relay_log_pos");
|
||||
|
||||
if (!skip_lock)
|
||||
mysql_mutex_lock(&data_lock);
|
||||
inc_event_relay_log_pos();
|
||||
group_relay_log_pos= event_relay_log_pos;
|
||||
strmake_buf(group_relay_log_name,event_relay_log_name);
|
||||
rgi->inc_event_relay_log_pos();
|
||||
DBUG_PRINT("info", ("log_pos: %lu group_master_log_pos: %lu",
|
||||
(long) log_pos, (long) group_master_log_pos));
|
||||
if (rgi->is_parallel_exec)
|
||||
{
|
||||
/* In case of parallel replication, do not update the position backwards. */
|
||||
int cmp= strcmp(group_relay_log_name, event_relay_log_name);
|
||||
if (cmp < 0)
|
||||
{
|
||||
group_relay_log_pos= event_relay_log_pos;
|
||||
strmake_buf(group_relay_log_name, event_relay_log_name);
|
||||
notify_group_relay_log_name_update();
|
||||
} else if (cmp == 0 && group_relay_log_pos < event_relay_log_pos)
|
||||
group_relay_log_pos= event_relay_log_pos;
|
||||
|
||||
notify_group_relay_log_name_update();
|
||||
/*
|
||||
In the parallel case we need to update the master_log_name here, rather
|
||||
than in Rotate_log_event::do_update_pos().
|
||||
*/
|
||||
cmp= strcmp(group_master_log_name, rgi->future_event_master_log_name);
|
||||
if (cmp <= 0)
|
||||
{
|
||||
if (cmp < 0)
|
||||
{
|
||||
strcpy(group_master_log_name, rgi->future_event_master_log_name);
|
||||
notify_group_master_log_name_update();
|
||||
group_master_log_pos= log_pos;
|
||||
}
|
||||
else if (group_master_log_pos < log_pos)
|
||||
group_master_log_pos= log_pos;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
/* Non-parallel case. */
|
||||
group_relay_log_pos= event_relay_log_pos;
|
||||
strmake_buf(group_relay_log_name, event_relay_log_name);
|
||||
notify_group_relay_log_name_update();
|
||||
if (log_pos) // 3.23 binlogs don't have log_posx
|
||||
group_master_log_pos= log_pos;
|
||||
}
|
||||
|
||||
/*
|
||||
If the slave does not support transactions and replicates a transaction,
|
||||
@ -919,12 +948,6 @@ void Relay_log_info::inc_group_relay_log_pos(ulonglong log_pos,
|
||||
the relay log is not "val".
|
||||
With the end_log_pos solution, we avoid computations involving lengthes.
|
||||
*/
|
||||
DBUG_PRINT("info", ("log_pos: %lu group_master_log_pos: %lu",
|
||||
(long) log_pos, (long) group_master_log_pos));
|
||||
if (log_pos) // 3.23 binlogs don't have log_posx
|
||||
{
|
||||
group_master_log_pos= log_pos;
|
||||
}
|
||||
mysql_cond_broadcast(&data_cond);
|
||||
if (!skip_lock)
|
||||
mysql_mutex_unlock(&data_lock);
|
||||
@ -940,6 +963,9 @@ void Relay_log_info::close_temporary_tables()
|
||||
for (table=save_temporary_tables ; table ; table=next)
|
||||
{
|
||||
next=table->next;
|
||||
|
||||
/* Reset in_use as the table may have been created by another thd */
|
||||
table->in_use=0;
|
||||
/*
|
||||
Don't ask for disk deletion. For now, anyway they will be deleted when
|
||||
slave restarts, but it is a better intention to not delete them.
|
||||
@ -1099,9 +1125,9 @@ bool Relay_log_info::is_until_satisfied(THD *thd, Log_event *ev)
|
||||
!replicate_same_server_id)
|
||||
DBUG_RETURN(FALSE);
|
||||
log_name= group_master_log_name;
|
||||
log_pos= (!ev)? group_master_log_pos :
|
||||
((thd->variables.option_bits & OPTION_BEGIN || !ev->log_pos) ?
|
||||
group_master_log_pos : ev->log_pos - ev->data_written);
|
||||
log_pos= ((!ev)? group_master_log_pos :
|
||||
(get_flag(IN_TRANSACTION) || !ev->log_pos) ?
|
||||
group_master_log_pos : ev->log_pos - ev->data_written);
|
||||
}
|
||||
else
|
||||
{ /* until_condition == UNTIL_RELAY_POS */
|
||||
@ -1194,19 +1220,24 @@ bool Relay_log_info::cached_charset_compare(char *charset) const
|
||||
|
||||
|
||||
void Relay_log_info::stmt_done(my_off_t event_master_log_pos,
|
||||
time_t event_creation_time, THD *thd)
|
||||
time_t event_creation_time, THD *thd,
|
||||
rpl_group_info *rgi)
|
||||
{
|
||||
#ifndef DBUG_OFF
|
||||
extern uint debug_not_change_ts_if_art_event;
|
||||
#endif
|
||||
clear_flag(IN_STMT);
|
||||
DBUG_ENTER("Relay_log_info::stmt_done");
|
||||
|
||||
DBUG_ASSERT(rgi->rli == this);
|
||||
/*
|
||||
If in a transaction, and if the slave supports transactions, just
|
||||
inc_event_relay_log_pos(). We only have to check for OPTION_BEGIN
|
||||
(not OPTION_NOT_AUTOCOMMIT) as transactions are logged with
|
||||
BEGIN/COMMIT, not with SET AUTOCOMMIT= .
|
||||
|
||||
We can't use rgi->rli->get_flag(IN_TRANSACTION) here as OPTION_BEGIN
|
||||
is also used for single row transactions.
|
||||
|
||||
CAUTION: opt_using_transactions means innodb || bdb ; suppose the
|
||||
master supports InnoDB and BDB, but the slave supports only BDB,
|
||||
problems will arise: - suppose an InnoDB table is created on the
|
||||
@ -1224,12 +1255,13 @@ void Relay_log_info::stmt_done(my_off_t event_master_log_pos,
|
||||
middle of the "transaction". START SLAVE will resume at BEGIN
|
||||
while the MyISAM table has already been updated.
|
||||
*/
|
||||
if ((sql_thd->variables.option_bits & OPTION_BEGIN) && opt_using_transactions)
|
||||
inc_event_relay_log_pos();
|
||||
if ((rgi->thd->variables.option_bits & OPTION_BEGIN) &&
|
||||
opt_using_transactions)
|
||||
rgi->inc_event_relay_log_pos();
|
||||
else
|
||||
{
|
||||
inc_group_relay_log_pos(event_master_log_pos);
|
||||
if (rpl_global_gtid_slave_state.record_and_update_gtid(thd, this))
|
||||
inc_group_relay_log_pos(event_master_log_pos, rgi);
|
||||
if (rpl_global_gtid_slave_state.record_and_update_gtid(thd, rgi))
|
||||
{
|
||||
report(WARNING_LEVEL, ER_CANNOT_UPDATE_GTID_STATE,
|
||||
"Failed to update GTID state in %s.%s, slave state may become "
|
||||
@ -1244,7 +1276,8 @@ void Relay_log_info::stmt_done(my_off_t event_master_log_pos,
|
||||
*/
|
||||
}
|
||||
DBUG_EXECUTE_IF("inject_crash_before_flush_rli", DBUG_SUICIDE(););
|
||||
flush_relay_log_info(this);
|
||||
if (mi->using_gtid == Master_info::USE_GTID_NO)
|
||||
flush_relay_log_info(this);
|
||||
DBUG_EXECUTE_IF("inject_crash_after_flush_rli", DBUG_SUICIDE(););
|
||||
/*
|
||||
Note that Rotate_log_event::do_apply_event() does not call this
|
||||
@ -1258,127 +1291,10 @@ void Relay_log_info::stmt_done(my_off_t event_master_log_pos,
|
||||
IF_DBUG(debug_not_change_ts_if_art_event > 0, 1)))
|
||||
last_master_timestamp= event_creation_time;
|
||||
}
|
||||
DBUG_VOID_RETURN;
|
||||
}
|
||||
|
||||
#if !defined(MYSQL_CLIENT) && defined(HAVE_REPLICATION)
|
||||
void Relay_log_info::cleanup_context(THD *thd, bool error)
|
||||
{
|
||||
DBUG_ENTER("Relay_log_info::cleanup_context");
|
||||
|
||||
DBUG_ASSERT(sql_thd == thd);
|
||||
/*
|
||||
1) Instances of Table_map_log_event, if ::do_apply_event() was called on them,
|
||||
may have opened tables, which we cannot be sure have been closed (because
|
||||
maybe the Rows_log_event have not been found or will not be, because slave
|
||||
SQL thread is stopping, or relay log has a missing tail etc). So we close
|
||||
all thread's tables. And so the table mappings have to be cancelled.
|
||||
2) Rows_log_event::do_apply_event() may even have started statements or
|
||||
transactions on them, which we need to rollback in case of error.
|
||||
3) If finding a Format_description_log_event after a BEGIN, we also need
|
||||
to rollback before continuing with the next events.
|
||||
4) so we need this "context cleanup" function.
|
||||
*/
|
||||
if (error)
|
||||
{
|
||||
trans_rollback_stmt(thd); // if a "statement transaction"
|
||||
trans_rollback(thd); // if a "real transaction"
|
||||
}
|
||||
m_table_map.clear_tables();
|
||||
slave_close_thread_tables(thd);
|
||||
if (error)
|
||||
thd->mdl_context.release_transactional_locks();
|
||||
clear_flag(IN_STMT);
|
||||
/*
|
||||
Cleanup for the flags that have been set at do_apply_event.
|
||||
*/
|
||||
thd->variables.option_bits&= ~OPTION_NO_FOREIGN_KEY_CHECKS;
|
||||
thd->variables.option_bits&= ~OPTION_RELAXED_UNIQUE_CHECKS;
|
||||
|
||||
/*
|
||||
Reset state related to long_find_row notes in the error log:
|
||||
- timestamp
|
||||
- flag that decides whether the slave prints or not
|
||||
*/
|
||||
reset_row_stmt_start_timestamp();
|
||||
unset_long_find_row_note_printed();
|
||||
|
||||
DBUG_VOID_RETURN;
|
||||
}
|
||||
|
||||
void Relay_log_info::clear_tables_to_lock()
|
||||
{
|
||||
DBUG_ENTER("Relay_log_info::clear_tables_to_lock()");
|
||||
#ifndef DBUG_OFF
|
||||
/**
|
||||
When replicating in RBR and MyISAM Merge tables are involved
|
||||
open_and_lock_tables (called in do_apply_event) appends the
|
||||
base tables to the list of tables_to_lock. Then these are
|
||||
removed from the list in close_thread_tables (which is called
|
||||
before we reach this point).
|
||||
|
||||
This assertion just confirms that we get no surprises at this
|
||||
point.
|
||||
*/
|
||||
uint i=0;
|
||||
for (TABLE_LIST *ptr= tables_to_lock ; ptr ; ptr= ptr->next_global, i++) ;
|
||||
DBUG_ASSERT(i == tables_to_lock_count);
|
||||
#endif
|
||||
|
||||
while (tables_to_lock)
|
||||
{
|
||||
uchar* to_free= reinterpret_cast<uchar*>(tables_to_lock);
|
||||
if (tables_to_lock->m_tabledef_valid)
|
||||
{
|
||||
tables_to_lock->m_tabledef.table_def::~table_def();
|
||||
tables_to_lock->m_tabledef_valid= FALSE;
|
||||
}
|
||||
|
||||
/*
|
||||
If blob fields were used during conversion of field values
|
||||
from the master table into the slave table, then we need to
|
||||
free the memory used temporarily to store their values before
|
||||
copying into the slave's table.
|
||||
*/
|
||||
if (tables_to_lock->m_conv_table)
|
||||
free_blobs(tables_to_lock->m_conv_table);
|
||||
|
||||
tables_to_lock=
|
||||
static_cast<RPL_TABLE_LIST*>(tables_to_lock->next_global);
|
||||
tables_to_lock_count--;
|
||||
my_free(to_free);
|
||||
}
|
||||
DBUG_ASSERT(tables_to_lock == NULL && tables_to_lock_count == 0);
|
||||
DBUG_VOID_RETURN;
|
||||
}
|
||||
|
||||
void Relay_log_info::slave_close_thread_tables(THD *thd)
|
||||
{
|
||||
DBUG_ENTER("Relay_log_info::slave_close_thread_tables(THD *thd)");
|
||||
thd->stmt_da->can_overwrite_status= TRUE;
|
||||
thd->is_error() ? trans_rollback_stmt(thd) : trans_commit_stmt(thd);
|
||||
thd->stmt_da->can_overwrite_status= FALSE;
|
||||
|
||||
close_thread_tables(thd);
|
||||
/*
|
||||
- If inside a multi-statement transaction,
|
||||
defer the release of metadata locks until the current
|
||||
transaction is either committed or rolled back. This prevents
|
||||
other statements from modifying the table for the entire
|
||||
duration of this transaction. This provides commit ordering
|
||||
and guarantees serializability across multiple transactions.
|
||||
- If in autocommit mode, or outside a transactional context,
|
||||
automatically release metadata locks of the current statement.
|
||||
*/
|
||||
if (! thd->in_multi_stmt_transaction_mode())
|
||||
thd->mdl_context.release_transactional_locks();
|
||||
else
|
||||
thd->mdl_context.release_statement_locks();
|
||||
|
||||
clear_tables_to_lock();
|
||||
DBUG_VOID_RETURN;
|
||||
}
|
||||
|
||||
|
||||
int
|
||||
rpl_load_gtid_slave_state(THD *thd)
|
||||
{
|
||||
@ -1554,4 +1470,224 @@ end:
|
||||
DBUG_RETURN(err);
|
||||
}
|
||||
|
||||
|
||||
rpl_group_info::rpl_group_info(Relay_log_info *rli_)
|
||||
: rli(rli_), thd(0), gtid_sub_id(0), wait_commit_sub_id(0),
|
||||
wait_commit_group_info(0), wait_start_sub_id(0), parallel_entry(0),
|
||||
deferred_events(NULL), m_annotate_event(0), tables_to_lock(0),
|
||||
tables_to_lock_count(0), trans_retries(0), last_event_start_time(0),
|
||||
is_parallel_exec(false), is_error(false),
|
||||
row_stmt_start_timestamp(0), long_find_row_note_printed(false)
|
||||
{
|
||||
bzero(¤t_gtid, sizeof(current_gtid));
|
||||
mysql_mutex_init(key_rpl_group_info_sleep_lock, &sleep_lock,
|
||||
MY_MUTEX_INIT_FAST);
|
||||
mysql_cond_init(key_rpl_group_info_sleep_cond, &sleep_cond, NULL);
|
||||
}
|
||||
|
||||
|
||||
rpl_group_info::~rpl_group_info()
|
||||
{
|
||||
free_annotate_event();
|
||||
mysql_mutex_destroy(&sleep_lock);
|
||||
mysql_cond_destroy(&sleep_cond);
|
||||
}
|
||||
|
||||
|
||||
int
|
||||
event_group_new_gtid(rpl_group_info *rgi, Gtid_log_event *gev)
|
||||
{
|
||||
uint64 sub_id= rpl_global_gtid_slave_state.next_sub_id(gev->domain_id);
|
||||
if (!sub_id)
|
||||
{
|
||||
/* Out of memory caused hash insertion to fail. */
|
||||
return 1;
|
||||
}
|
||||
rgi->gtid_sub_id= sub_id;
|
||||
rgi->current_gtid.server_id= gev->server_id;
|
||||
rgi->current_gtid.domain_id= gev->domain_id;
|
||||
rgi->current_gtid.seq_no= gev->seq_no;
|
||||
return 0;
|
||||
}
|
||||
|
||||
|
||||
void
|
||||
delete_or_keep_event_post_apply(rpl_group_info *rgi,
|
||||
Log_event_type typ, Log_event *ev)
|
||||
{
|
||||
/*
|
||||
ToDo: This needs to work on rpl_group_info, not Relay_log_info, to be
|
||||
thread-safe for parallel replication.
|
||||
*/
|
||||
|
||||
switch (typ) {
|
||||
case FORMAT_DESCRIPTION_EVENT:
|
||||
/*
|
||||
Format_description_log_event should not be deleted because it
|
||||
will be used to read info about the relay log's format;
|
||||
it will be deleted when the SQL thread does not need it,
|
||||
i.e. when this thread terminates.
|
||||
*/
|
||||
break;
|
||||
case ANNOTATE_ROWS_EVENT:
|
||||
/*
|
||||
Annotate_rows event should not be deleted because after it has
|
||||
been applied, thd->query points to the string inside this event.
|
||||
The thd->query will be used to generate new Annotate_rows event
|
||||
during applying the subsequent Rows events.
|
||||
*/
|
||||
rgi->set_annotate_event((Annotate_rows_log_event*) ev);
|
||||
break;
|
||||
case DELETE_ROWS_EVENT:
|
||||
case UPDATE_ROWS_EVENT:
|
||||
case WRITE_ROWS_EVENT:
|
||||
/*
|
||||
After the last Rows event has been applied, the saved Annotate_rows
|
||||
event (if any) is not needed anymore and can be deleted.
|
||||
*/
|
||||
if (((Rows_log_event*)ev)->get_flags(Rows_log_event::STMT_END_F))
|
||||
rgi->free_annotate_event();
|
||||
/* fall through */
|
||||
default:
|
||||
DBUG_PRINT("info", ("Deleting the event after it has been executed"));
|
||||
if (!rgi->is_deferred_event(ev))
|
||||
delete ev;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void rpl_group_info::cleanup_context(THD *thd, bool error)
|
||||
{
|
||||
DBUG_ENTER("Relay_log_info::cleanup_context");
|
||||
DBUG_PRINT("enter", ("error: %d", (int) error));
|
||||
|
||||
DBUG_ASSERT(this->thd == thd);
|
||||
/*
|
||||
1) Instances of Table_map_log_event, if ::do_apply_event() was called on them,
|
||||
may have opened tables, which we cannot be sure have been closed (because
|
||||
maybe the Rows_log_event have not been found or will not be, because slave
|
||||
SQL thread is stopping, or relay log has a missing tail etc). So we close
|
||||
all thread's tables. And so the table mappings have to be cancelled.
|
||||
2) Rows_log_event::do_apply_event() may even have started statements or
|
||||
transactions on them, which we need to rollback in case of error.
|
||||
3) If finding a Format_description_log_event after a BEGIN, we also need
|
||||
to rollback before continuing with the next events.
|
||||
4) so we need this "context cleanup" function.
|
||||
*/
|
||||
if (error)
|
||||
{
|
||||
trans_rollback_stmt(thd); // if a "statement transaction"
|
||||
trans_rollback(thd); // if a "real transaction"
|
||||
}
|
||||
m_table_map.clear_tables();
|
||||
slave_close_thread_tables(thd);
|
||||
if (error)
|
||||
{
|
||||
thd->mdl_context.release_transactional_locks();
|
||||
|
||||
if (thd == rli->sql_driver_thd)
|
||||
{
|
||||
/*
|
||||
Reset flags. This is needed to handle incident events and errors in
|
||||
the relay log noticed by the sql driver thread.
|
||||
*/
|
||||
rli->clear_flag(Relay_log_info::IN_STMT);
|
||||
rli->clear_flag(Relay_log_info::IN_TRANSACTION);
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
Cleanup for the flags that have been set at do_apply_event.
|
||||
*/
|
||||
thd->variables.option_bits&= ~OPTION_NO_FOREIGN_KEY_CHECKS;
|
||||
thd->variables.option_bits&= ~OPTION_RELAXED_UNIQUE_CHECKS;
|
||||
|
||||
/*
|
||||
Reset state related to long_find_row notes in the error log:
|
||||
- timestamp
|
||||
- flag that decides whether the slave prints or not
|
||||
*/
|
||||
reset_row_stmt_start_timestamp();
|
||||
unset_long_find_row_note_printed();
|
||||
|
||||
DBUG_VOID_RETURN;
|
||||
}
|
||||
|
||||
|
||||
void rpl_group_info::clear_tables_to_lock()
|
||||
{
|
||||
DBUG_ENTER("Relay_log_info::clear_tables_to_lock()");
|
||||
#ifndef DBUG_OFF
|
||||
/**
|
||||
When replicating in RBR and MyISAM Merge tables are involved
|
||||
open_and_lock_tables (called in do_apply_event) appends the
|
||||
base tables to the list of tables_to_lock. Then these are
|
||||
removed from the list in close_thread_tables (which is called
|
||||
before we reach this point).
|
||||
|
||||
This assertion just confirms that we get no surprises at this
|
||||
point.
|
||||
*/
|
||||
uint i=0;
|
||||
for (TABLE_LIST *ptr= tables_to_lock ; ptr ; ptr= ptr->next_global, i++) ;
|
||||
DBUG_ASSERT(i == tables_to_lock_count);
|
||||
#endif
|
||||
|
||||
while (tables_to_lock)
|
||||
{
|
||||
uchar* to_free= reinterpret_cast<uchar*>(tables_to_lock);
|
||||
if (tables_to_lock->m_tabledef_valid)
|
||||
{
|
||||
tables_to_lock->m_tabledef.table_def::~table_def();
|
||||
tables_to_lock->m_tabledef_valid= FALSE;
|
||||
}
|
||||
|
||||
/*
|
||||
If blob fields were used during conversion of field values
|
||||
from the master table into the slave table, then we need to
|
||||
free the memory used temporarily to store their values before
|
||||
copying into the slave's table.
|
||||
*/
|
||||
if (tables_to_lock->m_conv_table)
|
||||
free_blobs(tables_to_lock->m_conv_table);
|
||||
|
||||
tables_to_lock=
|
||||
static_cast<RPL_TABLE_LIST*>(tables_to_lock->next_global);
|
||||
tables_to_lock_count--;
|
||||
my_free(to_free);
|
||||
}
|
||||
DBUG_ASSERT(tables_to_lock == NULL && tables_to_lock_count == 0);
|
||||
DBUG_VOID_RETURN;
|
||||
}
|
||||
|
||||
|
||||
void rpl_group_info::slave_close_thread_tables(THD *thd)
|
||||
{
|
||||
DBUG_ENTER("Relay_log_info::slave_close_thread_tables(THD *thd)");
|
||||
thd->stmt_da->can_overwrite_status= TRUE;
|
||||
thd->is_error() ? trans_rollback_stmt(thd) : trans_commit_stmt(thd);
|
||||
thd->stmt_da->can_overwrite_status= FALSE;
|
||||
|
||||
close_thread_tables(thd);
|
||||
/*
|
||||
- If inside a multi-statement transaction,
|
||||
defer the release of metadata locks until the current
|
||||
transaction is either committed or rolled back. This prevents
|
||||
other statements from modifying the table for the entire
|
||||
duration of this transaction. This provides commit ordering
|
||||
and guarantees serializability across multiple transactions.
|
||||
- If in autocommit mode, or outside a transactional context,
|
||||
automatically release metadata locks of the current statement.
|
||||
*/
|
||||
if (! thd->in_multi_stmt_transaction_mode())
|
||||
thd->mdl_context.release_transactional_locks();
|
||||
else
|
||||
thd->mdl_context.release_statement_locks();
|
||||
|
||||
clear_tables_to_lock();
|
||||
DBUG_VOID_RETURN;
|
||||
}
|
||||
|
||||
|
||||
#endif
|
||||
|
416
sql/rpl_rli.h
416
sql/rpl_rli.h
@ -22,6 +22,7 @@
|
||||
#include "log.h" /* LOG_INFO, MYSQL_BIN_LOG */
|
||||
#include "sql_class.h" /* THD */
|
||||
#include "log_event.h"
|
||||
#include "rpl_parallel.h"
|
||||
|
||||
struct RPL_TABLE_LIST;
|
||||
class Master_info;
|
||||
@ -52,18 +53,20 @@ class Master_info;
|
||||
|
||||
*****************************************************************************/
|
||||
|
||||
struct rpl_group_info;
|
||||
|
||||
class Relay_log_info : public Slave_reporting_capability
|
||||
{
|
||||
public:
|
||||
/**
|
||||
Flags for the state of the replication.
|
||||
*/
|
||||
Flags for the state of reading the relay log. Note that these are
|
||||
bit masks.
|
||||
*/
|
||||
enum enum_state_flag {
|
||||
/** The replication thread is inside a statement */
|
||||
IN_STMT,
|
||||
|
||||
/** Flag counter. Should always be last */
|
||||
STATE_FLAGS_COUNT
|
||||
/** We are inside a group of events forming a statement */
|
||||
IN_STMT=1,
|
||||
/** We have inside a transaction */
|
||||
IN_TRANSACTION=2
|
||||
};
|
||||
|
||||
/*
|
||||
@ -128,9 +131,14 @@ public:
|
||||
IO_CACHE info_file;
|
||||
|
||||
/*
|
||||
When we restart slave thread we need to have access to the previously
|
||||
created temporary tables. Modified only on init/end and by the SQL
|
||||
thread, read only by SQL thread.
|
||||
List of temporary tables used by this connection.
|
||||
This is updated when a temporary table is created or dropped by
|
||||
a replication thread.
|
||||
|
||||
Not reset when replication ends, to allow one to access the tables
|
||||
when replication restarts.
|
||||
|
||||
Protected by data_lock.
|
||||
*/
|
||||
TABLE *save_temporary_tables;
|
||||
|
||||
@ -138,13 +146,13 @@ public:
|
||||
standard lock acquisition order to avoid deadlocks:
|
||||
run_lock, data_lock, relay_log.LOCK_log, relay_log.LOCK_index
|
||||
*/
|
||||
mysql_mutex_t data_lock, run_lock, sleep_lock;
|
||||
mysql_mutex_t data_lock, run_lock;
|
||||
/*
|
||||
start_cond is broadcast when SQL thread is started
|
||||
stop_cond - when stopped
|
||||
data_cond - when data protected by data_lock changes
|
||||
*/
|
||||
mysql_cond_t start_cond, stop_cond, data_cond, sleep_cond;
|
||||
mysql_cond_t start_cond, stop_cond, data_cond;
|
||||
/* parent Master_info structure */
|
||||
Master_info *mi;
|
||||
|
||||
@ -161,8 +169,8 @@ public:
|
||||
- an autocommiting query + its associated events (INSERT_ID,
|
||||
TIMESTAMP...)
|
||||
We need these rli coordinates :
|
||||
- relay log name and position of the beginning of the group we currently are
|
||||
executing. Needed to know where we have to restart when replication has
|
||||
- relay log name and position of the beginning of the group we currently
|
||||
are executing. Needed to know where we have to restart when replication has
|
||||
stopped in the middle of a group (which has been rolled back by the slave).
|
||||
- relay log name and position just after the event we have just
|
||||
executed. This event is part of the current group.
|
||||
@ -177,6 +185,10 @@ public:
|
||||
char event_relay_log_name[FN_REFLEN];
|
||||
ulonglong event_relay_log_pos;
|
||||
ulonglong future_event_relay_log_pos;
|
||||
/*
|
||||
The master log name for current event. Only used in parallel replication.
|
||||
*/
|
||||
char future_event_master_log_name[FN_REFLEN];
|
||||
|
||||
#ifdef HAVE_valgrind
|
||||
bool is_fake; /* Mark that this is a fake relay log info structure */
|
||||
@ -208,18 +220,6 @@ public:
|
||||
*/
|
||||
bool sql_force_rotate_relay;
|
||||
|
||||
/*
|
||||
When it commits, InnoDB internally stores the master log position it has
|
||||
processed so far; the position to store is the one of the end of the
|
||||
committing event (the COMMIT query event, or the event if in autocommit
|
||||
mode).
|
||||
*/
|
||||
#if MYSQL_VERSION_ID < 40100
|
||||
ulonglong future_master_log_pos;
|
||||
#else
|
||||
ulonglong future_group_master_log_pos;
|
||||
#endif
|
||||
|
||||
time_t last_master_timestamp;
|
||||
|
||||
void clear_until_condition();
|
||||
@ -236,7 +236,13 @@ public:
|
||||
ulong max_relay_log_size;
|
||||
mysql_mutex_t log_space_lock;
|
||||
mysql_cond_t log_space_cond;
|
||||
THD * sql_thd;
|
||||
/*
|
||||
THD for the main sql thread, the one that starts threads to process
|
||||
slave requests. If there is only one thread, then this THD is also
|
||||
used for SQL processing.
|
||||
A kill sent to this THD will kill the replication.
|
||||
*/
|
||||
THD *sql_driver_thd;
|
||||
#ifndef DBUG_OFF
|
||||
int events_till_abort;
|
||||
#endif
|
||||
@ -284,14 +290,16 @@ public:
|
||||
|
||||
char cached_charset[6];
|
||||
/*
|
||||
trans_retries varies between 0 to slave_transaction_retries and counts how
|
||||
many times the slave has retried the present transaction; gets reset to 0
|
||||
when the transaction finally succeeds. retried_trans is a cumulative
|
||||
counter: how many times the slave has retried a transaction (any) since
|
||||
slave started.
|
||||
retried_trans is a cumulative counter: how many times the slave
|
||||
has retried a transaction (any) since slave started.
|
||||
Protected by data_lock.
|
||||
*/
|
||||
ulong trans_retries, retried_trans;
|
||||
ulong executed_entries; /* For SLAVE STATUS */
|
||||
ulong retried_trans;
|
||||
/*
|
||||
Number of executed events for SLAVE STATUS.
|
||||
Protected by slave_executed_entries_lock
|
||||
*/
|
||||
int64 executed_entries;
|
||||
|
||||
/*
|
||||
If the end of the hot relay log is made of master's events ignored by the
|
||||
@ -313,13 +321,7 @@ public:
|
||||
char slave_patternload_file[FN_REFLEN];
|
||||
size_t slave_patternload_file_size;
|
||||
|
||||
/*
|
||||
Current GTID being processed.
|
||||
The sub_id gives the binlog order within one domain_id. A zero sub_id
|
||||
means that there is no active GTID.
|
||||
*/
|
||||
uint64 gtid_sub_id;
|
||||
rpl_gtid current_gtid;
|
||||
rpl_parallel parallel;
|
||||
|
||||
Relay_log_info(bool is_slave_recovery);
|
||||
~Relay_log_info();
|
||||
@ -343,13 +345,9 @@ public:
|
||||
if (until_condition==UNTIL_MASTER_POS)
|
||||
until_log_names_cmp_result= UNTIL_LOG_NAMES_CMP_UNKNOWN;
|
||||
}
|
||||
|
||||
inline void inc_event_relay_log_pos()
|
||||
{
|
||||
event_relay_log_pos= future_event_relay_log_pos;
|
||||
}
|
||||
|
||||
void inc_group_relay_log_pos(ulonglong log_pos,
|
||||
rpl_group_info *rgi,
|
||||
bool skip_lock=0);
|
||||
|
||||
int wait_for_pos(THD* thd, String* log_name, longlong log_pos,
|
||||
@ -366,27 +364,6 @@ public:
|
||||
group_relay_log_pos);
|
||||
}
|
||||
|
||||
RPL_TABLE_LIST *tables_to_lock; /* RBR: Tables to lock */
|
||||
uint tables_to_lock_count; /* RBR: Count of tables to lock */
|
||||
table_mapping m_table_map; /* RBR: Mapping table-id to table */
|
||||
|
||||
bool get_table_data(TABLE *table_arg, table_def **tabledef_var, TABLE **conv_table_var) const
|
||||
{
|
||||
DBUG_ASSERT(tabledef_var && conv_table_var);
|
||||
for (TABLE_LIST *ptr= tables_to_lock ; ptr != NULL ; ptr= ptr->next_global)
|
||||
if (ptr->table == table_arg)
|
||||
{
|
||||
*tabledef_var= &static_cast<RPL_TABLE_LIST*>(ptr)->m_tabledef;
|
||||
*conv_table_var= static_cast<RPL_TABLE_LIST*>(ptr)->m_conv_table;
|
||||
DBUG_PRINT("debug", ("Fetching table data for table %s.%s:"
|
||||
" tabledef: %p, conv_table: %p",
|
||||
table_arg->s->db.str, table_arg->s->table_name.str,
|
||||
*tabledef_var, *conv_table_var));
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/*
|
||||
Last charset (6 bytes) seen by slave SQL thread is cached here; it helps
|
||||
the thread save 3 get_charset() per Query_log_event if the charset is not
|
||||
@ -396,16 +373,147 @@ public:
|
||||
void cached_charset_invalidate();
|
||||
bool cached_charset_compare(char *charset) const;
|
||||
|
||||
void cleanup_context(THD *, bool);
|
||||
void slave_close_thread_tables(THD *);
|
||||
void clear_tables_to_lock();
|
||||
/**
|
||||
Helper function to do after statement completion.
|
||||
|
||||
This function is called from an event to complete the group by
|
||||
either stepping the group position, if the "statement" is not
|
||||
inside a transaction; or increase the event position, if the
|
||||
"statement" is inside a transaction.
|
||||
|
||||
@param event_log_pos
|
||||
Master log position of the event. The position is recorded in the
|
||||
relay log info and used to produce information for <code>SHOW
|
||||
SLAVE STATUS</code>.
|
||||
|
||||
@param event_creation_time
|
||||
Timestamp for the creation of the event on the master side. The
|
||||
time stamp is recorded in the relay log info and used to compute
|
||||
the <code>Seconds_behind_master</code> field.
|
||||
*/
|
||||
void stmt_done(my_off_t event_log_pos,
|
||||
time_t event_creation_time, THD *thd,
|
||||
rpl_group_info *rgi);
|
||||
|
||||
/**
|
||||
Is the replication inside a group?
|
||||
|
||||
The reader of the relay log is inside a group if either:
|
||||
- The IN_TRANSACTION flag is set, meaning we're inside a transaction
|
||||
- The IN_STMT flag is set, meaning we have read at least one row from
|
||||
a multi-event entry.
|
||||
|
||||
This flag reflects the state of the log 'just now', ie after the last
|
||||
read event would be executed.
|
||||
This allow us to test if we can stop replication before reading
|
||||
the next entry.
|
||||
|
||||
@retval true Replication thread is currently inside a group
|
||||
@retval false Replication thread is currently not inside a group
|
||||
*/
|
||||
bool is_in_group() const {
|
||||
return (m_flags & (IN_STMT | IN_TRANSACTION));
|
||||
}
|
||||
|
||||
/**
|
||||
Set the value of a replication state flag.
|
||||
|
||||
@param flag Flag to set
|
||||
*/
|
||||
void set_flag(enum_state_flag flag)
|
||||
{
|
||||
m_flags|= flag;
|
||||
}
|
||||
|
||||
/**
|
||||
Get the value of a replication state flag.
|
||||
|
||||
@param flag Flag to get value of
|
||||
|
||||
@return @c true if the flag was set, @c false otherwise.
|
||||
*/
|
||||
bool get_flag(enum_state_flag flag)
|
||||
{
|
||||
return m_flags & flag;
|
||||
}
|
||||
|
||||
/**
|
||||
Clear the value of a replication state flag.
|
||||
|
||||
@param flag Flag to clear
|
||||
*/
|
||||
void clear_flag(enum_state_flag flag)
|
||||
{
|
||||
m_flags&= ~flag;
|
||||
}
|
||||
|
||||
private:
|
||||
|
||||
/*
|
||||
Used to defer stopping the SQL thread to give it a chance
|
||||
to finish up the current group of events.
|
||||
The timestamp is set and reset in @c sql_slave_killed().
|
||||
Holds the state of the data in the relay log.
|
||||
We need this to ensure that we are not in the middle of a
|
||||
statement or inside BEGIN ... COMMIT when should rotate the
|
||||
relay log.
|
||||
*/
|
||||
time_t last_event_start_time;
|
||||
uint32 m_flags;
|
||||
};
|
||||
|
||||
|
||||
/*
|
||||
This is data for various state needed to be kept for the processing of
|
||||
one event group (transaction) during replication.
|
||||
|
||||
In single-threaded replication, there will be one global rpl_group_info and
|
||||
one global Relay_log_info per master connection. They will be linked
|
||||
together.
|
||||
|
||||
In parallel replication, there will be one rpl_group_info object for
|
||||
each running sql thread, each having their own thd.
|
||||
|
||||
All rpl_group_info will share the same Relay_log_info.
|
||||
*/
|
||||
|
||||
struct rpl_group_info
|
||||
{
|
||||
Relay_log_info *rli;
|
||||
THD *thd;
|
||||
/*
|
||||
Current GTID being processed.
|
||||
The sub_id gives the binlog order within one domain_id. A zero sub_id
|
||||
means that there is no active GTID.
|
||||
*/
|
||||
uint64 gtid_sub_id;
|
||||
rpl_gtid current_gtid;
|
||||
/*
|
||||
This is used to keep transaction commit order.
|
||||
We will signal this when we commit, and can register it to wait for the
|
||||
commit_orderer of the previous commit to signal us.
|
||||
*/
|
||||
wait_for_commit commit_orderer;
|
||||
/*
|
||||
If non-zero, the sub_id of a prior event group whose commit we have to wait
|
||||
for before committing ourselves. Then wait_commit_group_info points to the
|
||||
event group to wait for.
|
||||
|
||||
Before using this, rpl_parallel_entry::last_committed_sub_id should be
|
||||
compared against wait_commit_sub_id. Only if last_committed_sub_id is
|
||||
smaller than wait_commit_sub_id must the wait be done (otherwise the
|
||||
waited-for transaction is already committed, so we would otherwise wait
|
||||
for the wrong commit).
|
||||
*/
|
||||
uint64 wait_commit_sub_id;
|
||||
rpl_group_info *wait_commit_group_info;
|
||||
/*
|
||||
If non-zero, the event group must wait for this sub_id to be committed
|
||||
before the execution of the event group is allowed to start.
|
||||
|
||||
(When we execute in parallel the transactions that group committed
|
||||
together on the master, we still need to wait for any prior transactions
|
||||
to have commtted).
|
||||
*/
|
||||
uint64 wait_start_sub_id;
|
||||
|
||||
struct rpl_parallel_entry *parallel_entry;
|
||||
|
||||
/*
|
||||
A container to hold on Intvar-, Rand-, Uservar- log-events in case
|
||||
@ -421,6 +529,51 @@ public:
|
||||
*/
|
||||
bool deferred_events_collecting;
|
||||
|
||||
Annotate_rows_log_event *m_annotate_event;
|
||||
|
||||
RPL_TABLE_LIST *tables_to_lock; /* RBR: Tables to lock */
|
||||
uint tables_to_lock_count; /* RBR: Count of tables to lock */
|
||||
table_mapping m_table_map; /* RBR: Mapping table-id to table */
|
||||
mysql_mutex_t sleep_lock;
|
||||
mysql_cond_t sleep_cond;
|
||||
|
||||
/*
|
||||
trans_retries varies between 0 to slave_transaction_retries and counts how
|
||||
many times the slave has retried the present transaction; gets reset to 0
|
||||
when the transaction finally succeeds.
|
||||
*/
|
||||
ulong trans_retries;
|
||||
|
||||
/*
|
||||
Used to defer stopping the SQL thread to give it a chance
|
||||
to finish up the current group of events.
|
||||
The timestamp is set and reset in @c sql_slave_killed().
|
||||
*/
|
||||
time_t last_event_start_time;
|
||||
|
||||
char *event_relay_log_name;
|
||||
char event_relay_log_name_buf[FN_REFLEN];
|
||||
ulonglong event_relay_log_pos;
|
||||
ulonglong future_event_relay_log_pos;
|
||||
/*
|
||||
The master log name for current event. Only used in parallel replication.
|
||||
*/
|
||||
char future_event_master_log_name[FN_REFLEN];
|
||||
bool is_parallel_exec;
|
||||
bool is_error;
|
||||
|
||||
private:
|
||||
/*
|
||||
Runtime state for printing a note when slave is taking
|
||||
too long while processing a row event.
|
||||
*/
|
||||
time_t row_stmt_start_timestamp;
|
||||
bool long_find_row_note_printed;
|
||||
public:
|
||||
|
||||
rpl_group_info(Relay_log_info *rli_);
|
||||
~rpl_group_info();
|
||||
|
||||
/*
|
||||
Returns true if the argument event resides in the containter;
|
||||
more specifically, the checking is done against the last added event.
|
||||
@ -441,75 +594,6 @@ public:
|
||||
if (deferred_events)
|
||||
delete deferred_events;
|
||||
};
|
||||
|
||||
/**
|
||||
Helper function to do after statement completion.
|
||||
|
||||
This function is called from an event to complete the group by
|
||||
either stepping the group position, if the "statement" is not
|
||||
inside a transaction; or increase the event position, if the
|
||||
"statement" is inside a transaction.
|
||||
|
||||
@param event_log_pos
|
||||
Master log position of the event. The position is recorded in the
|
||||
relay log info and used to produce information for <code>SHOW
|
||||
SLAVE STATUS</code>.
|
||||
|
||||
@param event_creation_time
|
||||
Timestamp for the creation of the event on the master side. The
|
||||
time stamp is recorded in the relay log info and used to compute
|
||||
the <code>Seconds_behind_master</code> field.
|
||||
*/
|
||||
void stmt_done(my_off_t event_log_pos,
|
||||
time_t event_creation_time, THD *thd);
|
||||
|
||||
|
||||
/**
|
||||
Set the value of a replication state flag.
|
||||
|
||||
@param flag Flag to set
|
||||
*/
|
||||
void set_flag(enum_state_flag flag)
|
||||
{
|
||||
m_flags |= (1UL << flag);
|
||||
}
|
||||
|
||||
/**
|
||||
Get the value of a replication state flag.
|
||||
|
||||
@param flag Flag to get value of
|
||||
|
||||
@return @c true if the flag was set, @c false otherwise.
|
||||
*/
|
||||
bool get_flag(enum_state_flag flag)
|
||||
{
|
||||
return m_flags & (1UL << flag);
|
||||
}
|
||||
|
||||
/**
|
||||
Clear the value of a replication state flag.
|
||||
|
||||
@param flag Flag to clear
|
||||
*/
|
||||
void clear_flag(enum_state_flag flag)
|
||||
{
|
||||
m_flags &= ~(1UL << flag);
|
||||
}
|
||||
|
||||
/**
|
||||
Is the replication inside a group?
|
||||
|
||||
Replication is inside a group if either:
|
||||
- The OPTION_BEGIN flag is set, meaning we're inside a transaction
|
||||
- The RLI_IN_STMT flag is set, meaning we're inside a statement
|
||||
|
||||
@retval true Replication thread is currently inside a group
|
||||
@retval false Replication thread is currently not inside a group
|
||||
*/
|
||||
bool is_in_group() const {
|
||||
return (sql_thd->variables.option_bits & OPTION_BEGIN) ||
|
||||
(m_flags & (1UL << IN_STMT));
|
||||
}
|
||||
|
||||
/**
|
||||
Save pointer to Annotate_rows event and switch on the
|
||||
@ -520,7 +604,7 @@ public:
|
||||
{
|
||||
free_annotate_event();
|
||||
m_annotate_event= event;
|
||||
sql_thd->variables.binlog_annotate_row_events= 1;
|
||||
this->thd->variables.binlog_annotate_row_events= 1;
|
||||
}
|
||||
|
||||
/**
|
||||
@ -542,12 +626,33 @@ public:
|
||||
{
|
||||
if (m_annotate_event)
|
||||
{
|
||||
sql_thd->variables.binlog_annotate_row_events= 0;
|
||||
this->thd->variables.binlog_annotate_row_events= 0;
|
||||
delete m_annotate_event;
|
||||
m_annotate_event= 0;
|
||||
}
|
||||
}
|
||||
|
||||
bool get_table_data(TABLE *table_arg, table_def **tabledef_var, TABLE **conv_table_var) const
|
||||
{
|
||||
DBUG_ASSERT(tabledef_var && conv_table_var);
|
||||
for (TABLE_LIST *ptr= tables_to_lock ; ptr != NULL ; ptr= ptr->next_global)
|
||||
if (ptr->table == table_arg)
|
||||
{
|
||||
*tabledef_var= &static_cast<RPL_TABLE_LIST*>(ptr)->m_tabledef;
|
||||
*conv_table_var= static_cast<RPL_TABLE_LIST*>(ptr)->m_conv_table;
|
||||
DBUG_PRINT("debug", ("Fetching table data for table %s.%s:"
|
||||
" tabledef: %p, conv_table: %p",
|
||||
table_arg->s->db.str, table_arg->s->table_name.str,
|
||||
*tabledef_var, *conv_table_var));
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
void clear_tables_to_lock();
|
||||
void cleanup_context(THD *, bool);
|
||||
void slave_close_thread_tables(THD *);
|
||||
|
||||
time_t get_row_stmt_start_timestamp()
|
||||
{
|
||||
return row_stmt_start_timestamp;
|
||||
@ -581,18 +686,12 @@ public:
|
||||
return long_find_row_note_printed;
|
||||
}
|
||||
|
||||
private:
|
||||
|
||||
uint32 m_flags;
|
||||
|
||||
/*
|
||||
Runtime state for printing a note when slave is taking
|
||||
too long while processing a row event.
|
||||
*/
|
||||
time_t row_stmt_start_timestamp;
|
||||
bool long_find_row_note_printed;
|
||||
|
||||
Annotate_rows_log_event *m_annotate_event;
|
||||
inline void inc_event_relay_log_pos()
|
||||
{
|
||||
if (!is_parallel_exec ||
|
||||
rli->event_relay_log_pos < future_event_relay_log_pos)
|
||||
rli->event_relay_log_pos= future_event_relay_log_pos;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
@ -603,5 +702,8 @@ int init_relay_log_info(Relay_log_info* rli, const char* info_fname);
|
||||
extern struct rpl_slave_state rpl_global_gtid_slave_state;
|
||||
|
||||
int rpl_load_gtid_slave_state(THD *thd);
|
||||
int event_group_new_gtid(rpl_group_info *rgi, Gtid_log_event *gev);
|
||||
void delete_or_keep_event_post_apply(rpl_group_info *rgi,
|
||||
Log_event_type typ, Log_event *ev);
|
||||
|
||||
#endif /* RPL_RLI_H */
|
||||
|
@ -1143,20 +1143,20 @@ bool Deferred_log_events::is_empty()
|
||||
return array.elements == 0;
|
||||
}
|
||||
|
||||
bool Deferred_log_events::execute(Relay_log_info *rli)
|
||||
bool Deferred_log_events::execute(rpl_group_info *rgi)
|
||||
{
|
||||
bool res= false;
|
||||
DBUG_ENTER("Deferred_log_events::execute");
|
||||
DBUG_ASSERT(rli->deferred_events_collecting);
|
||||
DBUG_ASSERT(rgi->deferred_events_collecting);
|
||||
|
||||
rli->deferred_events_collecting= false;
|
||||
rgi->deferred_events_collecting= false;
|
||||
for (uint i= 0; !res && i < array.elements; i++)
|
||||
{
|
||||
Log_event *ev= (* (Log_event **)
|
||||
dynamic_array_ptr(&array, i));
|
||||
res= ev->apply_event(rli);
|
||||
res= ev->apply_event(rgi);
|
||||
}
|
||||
rli->deferred_events_collecting= true;
|
||||
rgi->deferred_events_collecting= true;
|
||||
DBUG_RETURN(res);
|
||||
}
|
||||
|
||||
|
@ -275,7 +275,7 @@ public:
|
||||
/* queue for exection at Query-log-event time prior the Query */
|
||||
int add(Log_event *ev);
|
||||
bool is_empty();
|
||||
bool execute(Relay_log_info *rli);
|
||||
bool execute(struct rpl_group_info *rgi);
|
||||
void rewind();
|
||||
bool is_last(Log_event *ev) { return ev == last_added; };
|
||||
};
|
||||
|
@ -6574,3 +6574,7 @@ ER_CANNOT_GRANT_ROLE
|
||||
ER_CANNOT_REVOKE_ROLE
|
||||
eng "Cannot revoke role '%s' from: %s."
|
||||
rum "Rolul '%s' nu poate fi revocat de la: %s."
|
||||
ER_CHANGE_SLAVE_PARALLEL_THREADS_ACTIVE
|
||||
eng "Cannot change @@slave_parallel_threads while another change is in progress"
|
||||
ER_PRIOR_COMMIT_FAILED
|
||||
eng "Commit failed due to failure of an earlier commit on which this one depends"
|
||||
|
687
sql/slave.cc
687
sql/slave.cc
File diff suppressed because it is too large
Load Diff
@ -51,6 +51,7 @@
|
||||
class Relay_log_info;
|
||||
class Master_info;
|
||||
class Master_info_index;
|
||||
struct rpl_parallel_thread;
|
||||
|
||||
int init_intvar_from_file(int* var, IO_CACHE* f, int default_val);
|
||||
int init_strvar_from_file(char *var, int max_size, IO_CACHE *f,
|
||||
@ -227,9 +228,12 @@ int purge_relay_logs(Relay_log_info* rli, THD *thd, bool just_reset,
|
||||
void set_slave_thread_options(THD* thd);
|
||||
void set_slave_thread_default_charset(THD *thd, Relay_log_info const *rli);
|
||||
int rotate_relay_log(Master_info* mi);
|
||||
int apply_event_and_update_pos(Log_event* ev, THD* thd, Relay_log_info* rli);
|
||||
int apply_event_and_update_pos(Log_event* ev, THD* thd,
|
||||
struct rpl_group_info *rgi,
|
||||
rpl_parallel_thread *rpt);
|
||||
|
||||
pthread_handler_t handle_slave_io(void *arg);
|
||||
void slave_output_error_info(Relay_log_info *rli, THD *thd);
|
||||
pthread_handler_t handle_slave_sql(void *arg);
|
||||
bool net_request_file(NET* net, const char* fname);
|
||||
|
||||
|
11
sql/sp.cc
11
sql/sp.cc
@ -1173,6 +1173,9 @@ sp_create_routine(THD *thd, stored_procedure_type type, sp_head *sp)
|
||||
ret= SP_OK;
|
||||
if (table->file->ha_write_row(table->record[0]))
|
||||
ret= SP_WRITE_ROW_FAILED;
|
||||
/* Make change permanent and avoid 'table is marked as crashed' errors */
|
||||
table->file->extra(HA_EXTRA_FLUSH);
|
||||
|
||||
if (ret == SP_OK)
|
||||
sp_cache_invalidate();
|
||||
|
||||
@ -1262,6 +1265,8 @@ sp_drop_routine(THD *thd, stored_procedure_type type, sp_name *name)
|
||||
{
|
||||
if (table->file->ha_delete_row(table->record[0]))
|
||||
ret= SP_DELETE_ROW_FAILED;
|
||||
/* Make change permanent and avoid 'table is marked as crashed' errors */
|
||||
table->file->extra(HA_EXTRA_FLUSH);
|
||||
}
|
||||
|
||||
if (ret == SP_OK)
|
||||
@ -1372,6 +1377,8 @@ sp_update_routine(THD *thd, stored_procedure_type type, sp_name *name,
|
||||
ret= SP_WRITE_ROW_FAILED;
|
||||
else
|
||||
ret= 0;
|
||||
/* Make change permanent and avoid 'table is marked as crashed' errors */
|
||||
table->file->extra(HA_EXTRA_FLUSH);
|
||||
}
|
||||
|
||||
if (ret == SP_OK)
|
||||
@ -1546,7 +1553,11 @@ sp_drop_db_routines(THD *thd, char *db)
|
||||
if (nxtres != HA_ERR_END_OF_FILE)
|
||||
ret= SP_KEY_NOT_FOUND;
|
||||
if (deleted)
|
||||
{
|
||||
sp_cache_invalidate();
|
||||
/* Make change permanent and avoid 'table is marked as crashed' errors */
|
||||
table->file->extra(HA_EXTRA_FLUSH);
|
||||
}
|
||||
}
|
||||
table->file->ha_index_end();
|
||||
|
||||
|
118
sql/sql_base.cc
118
sql/sql_base.cc
@ -57,6 +57,7 @@
|
||||
#include "sql_table.h" // build_table_filename
|
||||
#include "datadict.h" // dd_frm_is_view()
|
||||
#include "sql_hset.h" // Hash_set
|
||||
#include "rpl_rli.h" // rpl_group_info
|
||||
#ifdef __WIN__
|
||||
#include <io.h>
|
||||
#endif
|
||||
@ -1228,11 +1229,24 @@ bool close_cached_connection_tables(THD *thd, LEX_STRING *connection)
|
||||
|
||||
static void mark_temp_tables_as_free_for_reuse(THD *thd)
|
||||
{
|
||||
DBUG_ENTER("mark_temp_tables_as_free_for_reuse");
|
||||
|
||||
thd->lock_temporary_tables();
|
||||
for (TABLE *table= thd->temporary_tables ; table ; table= table->next)
|
||||
{
|
||||
if ((table->query_id == thd->query_id) && ! table->open_by_handler)
|
||||
mark_tmp_table_for_reuse(table);
|
||||
}
|
||||
thd->unlock_temporary_tables();
|
||||
if (thd->rgi_slave)
|
||||
{
|
||||
/*
|
||||
Temporary tables are shared with other by sql execution threads.
|
||||
As a safety messure, clear the pointer to the common area.
|
||||
*/
|
||||
thd->temporary_tables= 0;
|
||||
}
|
||||
DBUG_VOID_RETURN;
|
||||
}
|
||||
|
||||
|
||||
@ -1246,6 +1260,7 @@ static void mark_temp_tables_as_free_for_reuse(THD *thd)
|
||||
|
||||
void mark_tmp_table_for_reuse(TABLE *table)
|
||||
{
|
||||
DBUG_ENTER("mark_tmp_table_for_reuse");
|
||||
DBUG_ASSERT(table->s->tmp_table);
|
||||
|
||||
table->query_id= 0;
|
||||
@ -1276,6 +1291,7 @@ void mark_tmp_table_for_reuse(TABLE *table)
|
||||
LOCK TABLES is allowed (but ignored) for a temporary table.
|
||||
*/
|
||||
table->reginfo.lock_type= TL_WRITE;
|
||||
DBUG_VOID_RETURN;
|
||||
}
|
||||
|
||||
|
||||
@ -1626,6 +1642,10 @@ static inline uint tmpkeyval(THD *thd, TABLE *table)
|
||||
/*
|
||||
Close all temporary tables created by 'CREATE TEMPORARY TABLE' for thread
|
||||
creates one DROP TEMPORARY TABLE binlog event for each pseudo-thread
|
||||
|
||||
Temporary tables created in a sql slave is closed by
|
||||
Relay_log_info::close_temporary_tables()
|
||||
|
||||
*/
|
||||
|
||||
bool close_temporary_tables(THD *thd)
|
||||
@ -1640,6 +1660,7 @@ bool close_temporary_tables(THD *thd)
|
||||
|
||||
if (!thd->temporary_tables)
|
||||
DBUG_RETURN(FALSE);
|
||||
DBUG_ASSERT(!thd->rgi_slave);
|
||||
|
||||
if (!mysql_bin_log.is_open())
|
||||
{
|
||||
@ -2094,16 +2115,42 @@ TABLE *find_temporary_table(THD *thd,
|
||||
const char *table_key,
|
||||
uint table_key_length)
|
||||
{
|
||||
TABLE *result= 0;
|
||||
if (!thd->have_temporary_tables())
|
||||
return NULL;
|
||||
|
||||
thd->lock_temporary_tables();
|
||||
for (TABLE *table= thd->temporary_tables; table; table= table->next)
|
||||
{
|
||||
if (table->s->table_cache_key.length == table_key_length &&
|
||||
!memcmp(table->s->table_cache_key.str, table_key, table_key_length))
|
||||
{
|
||||
return table;
|
||||
/*
|
||||
We need to set the THD as it may be different in case of
|
||||
parallel replication
|
||||
*/
|
||||
if (table->in_use != thd)
|
||||
{
|
||||
table->in_use= thd;
|
||||
#ifdef REMOVE_AFTER_MERGE_WITH_10
|
||||
if (thd->rgi_slave)
|
||||
{
|
||||
/*
|
||||
We may be stealing an opened temporary tables from one slave
|
||||
thread to another, we need to let the performance schema know that,
|
||||
for aggregates per thread to work properly.
|
||||
*/
|
||||
table->file->unbind_psi();
|
||||
table->file->rebind_psi();
|
||||
}
|
||||
#endif
|
||||
}
|
||||
result= table;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
return NULL;
|
||||
thd->unlock_temporary_tables();
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
@ -2151,6 +2198,9 @@ int drop_temporary_table(THD *thd, TABLE_LIST *table_list, bool *is_trans)
|
||||
/* Table might be in use by some outer statement. */
|
||||
if (table->query_id && table->query_id != thd->query_id)
|
||||
{
|
||||
DBUG_PRINT("info", ("table->query_id: %lu thd->query_id: %lu",
|
||||
(ulong) table->query_id, (ulong) thd->query_id));
|
||||
|
||||
my_error(ER_CANT_REOPEN_TABLE, MYF(0), table->alias.c_ptr());
|
||||
DBUG_RETURN(-1);
|
||||
}
|
||||
@ -2179,6 +2229,7 @@ void close_temporary_table(THD *thd, TABLE *table,
|
||||
table->s->db.str, table->s->table_name.str,
|
||||
(long) table, table->alias.c_ptr()));
|
||||
|
||||
thd->lock_temporary_tables();
|
||||
if (table->prev)
|
||||
{
|
||||
table->prev->next= table->next;
|
||||
@ -2198,12 +2249,14 @@ void close_temporary_table(THD *thd, TABLE *table,
|
||||
if (thd->temporary_tables)
|
||||
table->next->prev= 0;
|
||||
}
|
||||
if (thd->slave_thread)
|
||||
if (thd->rgi_slave)
|
||||
{
|
||||
/* natural invariant of temporary_tables */
|
||||
DBUG_ASSERT(slave_open_temp_tables || !thd->temporary_tables);
|
||||
slave_open_temp_tables--;
|
||||
thread_safe_decrement32(&slave_open_temp_tables, &thread_running_lock);
|
||||
table->in_use= 0; // No statistics
|
||||
}
|
||||
thd->unlock_temporary_tables();
|
||||
close_temporary(table, free_share, delete_table);
|
||||
DBUG_VOID_RETURN;
|
||||
}
|
||||
@ -2649,35 +2702,30 @@ bool open_table(THD *thd, TABLE_LIST *table_list, MEM_ROOT *mem_root,
|
||||
TODO: move this block into a separate function.
|
||||
*/
|
||||
if (table_list->open_type != OT_BASE_ONLY &&
|
||||
! (flags & MYSQL_OPEN_SKIP_TEMPORARY))
|
||||
! (flags & MYSQL_OPEN_SKIP_TEMPORARY) && thd->have_temporary_tables())
|
||||
{
|
||||
for (table= thd->temporary_tables; table ; table=table->next)
|
||||
{
|
||||
if (table->s->table_cache_key.length == key_length +
|
||||
TMP_TABLE_KEY_EXTRA &&
|
||||
!memcmp(table->s->table_cache_key.str, key,
|
||||
key_length + TMP_TABLE_KEY_EXTRA))
|
||||
if ((table= find_temporary_table(thd, key,
|
||||
key_length + TMP_TABLE_KEY_EXTRA)))
|
||||
{
|
||||
/*
|
||||
Check if we're trying to use the same temporary table twice in a query.
|
||||
Right now we don't support this because a temporary table
|
||||
is always represented by only one TABLE object in THD, and
|
||||
it can not be cloned. Emit an error for an unsupported behaviour.
|
||||
*/
|
||||
if (table->query_id)
|
||||
{
|
||||
/*
|
||||
We're trying to use the same temporary table twice in a query.
|
||||
Right now we don't support this because a temporary table
|
||||
is always represented by only one TABLE object in THD, and
|
||||
it can not be cloned. Emit an error for an unsupported behaviour.
|
||||
*/
|
||||
if (table->query_id)
|
||||
{
|
||||
DBUG_PRINT("error",
|
||||
("query_id: %lu server_id: %u pseudo_thread_id: %lu",
|
||||
(ulong) table->query_id, (uint) thd->variables.server_id,
|
||||
(ulong) thd->variables.pseudo_thread_id));
|
||||
my_error(ER_CANT_REOPEN_TABLE, MYF(0), table->alias.c_ptr());
|
||||
DBUG_RETURN(TRUE);
|
||||
}
|
||||
table->query_id= thd->query_id;
|
||||
thd->thread_specific_used= TRUE;
|
||||
DBUG_PRINT("info",("Using temporary table"));
|
||||
goto reset;
|
||||
DBUG_PRINT("error",
|
||||
("query_id: %lu server_id: %u pseudo_thread_id: %lu",
|
||||
(ulong) table->query_id, (uint) thd->variables.server_id,
|
||||
(ulong) thd->variables.pseudo_thread_id));
|
||||
my_error(ER_CANT_REOPEN_TABLE, MYF(0), table->alias.c_ptr());
|
||||
DBUG_RETURN(TRUE);
|
||||
}
|
||||
table->query_id= thd->query_id;
|
||||
thd->thread_specific_used= TRUE;
|
||||
DBUG_PRINT("info",("Using temporary table"));
|
||||
goto reset;
|
||||
}
|
||||
}
|
||||
|
||||
@ -5985,14 +6033,18 @@ TABLE *open_table_uncached(THD *thd, handlerton *hton,
|
||||
|
||||
if (add_to_temporary_tables_list)
|
||||
{
|
||||
thd->lock_temporary_tables();
|
||||
/* growing temp list at the head */
|
||||
tmp_table->next= thd->temporary_tables;
|
||||
if (tmp_table->next)
|
||||
tmp_table->next->prev= tmp_table;
|
||||
thd->temporary_tables= tmp_table;
|
||||
thd->temporary_tables->prev= 0;
|
||||
if (thd->slave_thread)
|
||||
slave_open_temp_tables++;
|
||||
if (thd->rgi_slave)
|
||||
{
|
||||
thread_safe_increment32(&slave_open_temp_tables, &thread_running_lock);
|
||||
}
|
||||
thd->unlock_temporary_tables();
|
||||
}
|
||||
tmp_table->pos_in_table_list= 0;
|
||||
DBUG_PRINT("tmptable", ("opened table: '%s'.'%s' 0x%lx", tmp_table->s->db.str,
|
||||
|
@ -80,6 +80,8 @@ void mysql_client_binlog_statement(THD* thd)
|
||||
my_bool have_fd_event= TRUE;
|
||||
int err;
|
||||
Relay_log_info *rli;
|
||||
rpl_group_info *rgi;
|
||||
|
||||
rli= thd->rli_fake;
|
||||
if (!rli)
|
||||
{
|
||||
@ -95,6 +97,9 @@ void mysql_client_binlog_statement(THD* thd)
|
||||
new Format_description_log_event(4);
|
||||
have_fd_event= FALSE;
|
||||
}
|
||||
if (!(rgi= thd->rgi_fake))
|
||||
rgi= thd->rgi_fake= new rpl_group_info(rli);
|
||||
rgi->thd= thd;
|
||||
|
||||
const char *error= 0;
|
||||
char *buf= (char *) my_malloc(decoded_len, MYF(MY_WME));
|
||||
@ -111,7 +116,7 @@ void mysql_client_binlog_statement(THD* thd)
|
||||
goto end;
|
||||
}
|
||||
|
||||
rli->sql_thd= thd;
|
||||
rli->sql_driver_thd= thd;
|
||||
rli->no_storage= TRUE;
|
||||
|
||||
for (char const *strptr= thd->lex->comment.str ;
|
||||
@ -232,7 +237,7 @@ void mysql_client_binlog_statement(THD* thd)
|
||||
(ev->flags & LOG_EVENT_SKIP_REPLICATION_F ?
|
||||
OPTION_SKIP_REPLICATION : 0);
|
||||
|
||||
err= ev->apply_event(rli);
|
||||
err= ev->apply_event(rgi);
|
||||
|
||||
thd->variables.option_bits=
|
||||
(thd->variables.option_bits & ~OPTION_SKIP_REPLICATION) |
|
||||
@ -267,7 +272,7 @@ void mysql_client_binlog_statement(THD* thd)
|
||||
|
||||
end:
|
||||
thd->variables.option_bits= thd_options;
|
||||
rli->slave_close_thread_tables(thd);
|
||||
rgi->slave_close_thread_tables(thd);
|
||||
my_free(buf);
|
||||
DBUG_VOID_RETURN;
|
||||
}
|
||||
|
270
sql/sql_class.cc
270
sql/sql_class.cc
@ -605,6 +605,17 @@ void thd_set_ha_data(THD *thd, const struct handlerton *hton,
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
Allow storage engine to wakeup commits waiting in THD::wait_for_prior_commit.
|
||||
@see thd_wakeup_subsequent_commits() definition in plugin.h
|
||||
*/
|
||||
extern "C"
|
||||
void thd_wakeup_subsequent_commits(THD *thd, int wakeup_error)
|
||||
{
|
||||
thd->wakeup_subsequent_commits(wakeup_error);
|
||||
}
|
||||
|
||||
|
||||
extern "C"
|
||||
long long thd_test_options(const THD *thd, long long test_options)
|
||||
{
|
||||
@ -758,7 +769,7 @@ bool Drop_table_error_handler::handle_condition(THD *thd,
|
||||
THD::THD()
|
||||
:Statement(&main_lex, &main_mem_root, STMT_CONVENTIONAL_EXECUTION,
|
||||
/* statement id */ 0),
|
||||
rli_fake(0), rli_slave(NULL),
|
||||
rli_fake(0), rgi_fake(0), rgi_slave(NULL),
|
||||
in_sub_stmt(0), log_all_errors(0),
|
||||
binlog_unsafe_warning_flags(0),
|
||||
binlog_table_maps(0),
|
||||
@ -788,6 +799,7 @@ THD::THD()
|
||||
#if defined(ENABLED_DEBUG_SYNC)
|
||||
debug_sync_control(0),
|
||||
#endif /* defined(ENABLED_DEBUG_SYNC) */
|
||||
wait_for_commit_ptr(0),
|
||||
main_warning_info(0, false, false)
|
||||
{
|
||||
ulong tmp;
|
||||
@ -1478,6 +1490,11 @@ THD::~THD()
|
||||
dbug_sentry= THD_SENTRY_GONE;
|
||||
#endif
|
||||
#ifndef EMBEDDED_LIBRARY
|
||||
if (rgi_fake)
|
||||
{
|
||||
delete rgi_fake;
|
||||
rgi_fake= NULL;
|
||||
}
|
||||
if (rli_fake)
|
||||
{
|
||||
delete rli_fake;
|
||||
@ -1485,8 +1502,8 @@ THD::~THD()
|
||||
}
|
||||
|
||||
mysql_audit_free_thd(this);
|
||||
if (rli_slave)
|
||||
rli_slave->cleanup_after_session();
|
||||
if (rgi_slave)
|
||||
rgi_slave->cleanup_after_session();
|
||||
#endif
|
||||
|
||||
free_root(&main_mem_root, MYF(0));
|
||||
@ -1871,7 +1888,7 @@ void THD::cleanup_after_query()
|
||||
which is intended to consume its event (there can be other
|
||||
SET statements between them).
|
||||
*/
|
||||
if ((rli_slave || rli_fake) && is_update_query(lex->sql_command))
|
||||
if ((rgi_slave || rli_fake) && is_update_query(lex->sql_command))
|
||||
auto_inc_intervals_forced.empty();
|
||||
#endif
|
||||
}
|
||||
@ -1893,8 +1910,8 @@ void THD::cleanup_after_query()
|
||||
m_binlog_invoker= INVOKER_NONE;
|
||||
|
||||
#ifndef EMBEDDED_LIBRARY
|
||||
if (rli_slave)
|
||||
rli_slave->cleanup_after_query();
|
||||
if (rgi_slave)
|
||||
rgi_slave->cleanup_after_query();
|
||||
#endif
|
||||
|
||||
DBUG_VOID_RETURN;
|
||||
@ -5624,6 +5641,247 @@ THD::signal_wakeup_ready()
|
||||
}
|
||||
|
||||
|
||||
void THD::rgi_lock_temporary_tables()
|
||||
{
|
||||
mysql_mutex_lock(&rgi_slave->rli->data_lock);
|
||||
temporary_tables= rgi_slave->rli->save_temporary_tables;
|
||||
}
|
||||
|
||||
void THD::rgi_unlock_temporary_tables()
|
||||
{
|
||||
rgi_slave->rli->save_temporary_tables= temporary_tables;
|
||||
mysql_mutex_unlock(&rgi_slave->rli->data_lock);
|
||||
}
|
||||
|
||||
bool THD::rgi_have_temporary_tables()
|
||||
{
|
||||
return rgi_slave->rli->save_temporary_tables != 0;
|
||||
}
|
||||
|
||||
|
||||
wait_for_commit::wait_for_commit()
|
||||
: subsequent_commits_list(0), next_subsequent_commit(0), waitee(0),
|
||||
opaque_pointer(0),
|
||||
waiting_for_commit(false), wakeup_error(0),
|
||||
wakeup_subsequent_commits_running(false)
|
||||
{
|
||||
mysql_mutex_init(key_LOCK_wait_commit, &LOCK_wait_commit, MY_MUTEX_INIT_FAST);
|
||||
mysql_cond_init(key_COND_wait_commit, &COND_wait_commit, 0);
|
||||
}
|
||||
|
||||
|
||||
wait_for_commit::~wait_for_commit()
|
||||
{
|
||||
mysql_mutex_destroy(&LOCK_wait_commit);
|
||||
mysql_cond_destroy(&COND_wait_commit);
|
||||
}
|
||||
|
||||
|
||||
void
|
||||
wait_for_commit::wakeup(int wakeup_error)
|
||||
{
|
||||
/*
|
||||
We signal each waiter on their own condition and mutex (rather than using
|
||||
pthread_cond_broadcast() or something like that).
|
||||
|
||||
Otherwise we would need to somehow ensure that they were done
|
||||
waking up before we could allow this THD to be destroyed, which would
|
||||
be annoying and unnecessary.
|
||||
|
||||
Note that wakeup_subsequent_commits2() depends on this function being a
|
||||
full memory barrier (it is, because it takes a mutex lock).
|
||||
|
||||
*/
|
||||
mysql_mutex_lock(&LOCK_wait_commit);
|
||||
waiting_for_commit= false;
|
||||
this->wakeup_error= wakeup_error;
|
||||
mysql_mutex_unlock(&LOCK_wait_commit);
|
||||
mysql_cond_signal(&COND_wait_commit);
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
Register that the next commit of this THD should wait to complete until
|
||||
commit in another THD (the waitee) has completed.
|
||||
|
||||
The wait may occur explicitly, with the waiter sitting in
|
||||
wait_for_prior_commit() until the waitee calls wakeup_subsequent_commits().
|
||||
|
||||
Alternatively, the TC (eg. binlog) may do the commits of both waitee and
|
||||
waiter at once during group commit, resolving both of them in the right
|
||||
order.
|
||||
|
||||
Only one waitee can be registered for a waiter; it must be removed by
|
||||
wait_for_prior_commit() or unregister_wait_for_prior_commit() before a new
|
||||
one is registered. But it is ok for several waiters to register a wait for
|
||||
the same waitee. It is also permissible for one THD to be both a waiter and
|
||||
a waitee at the same time.
|
||||
*/
|
||||
void
|
||||
wait_for_commit::register_wait_for_prior_commit(wait_for_commit *waitee)
|
||||
{
|
||||
waiting_for_commit= true;
|
||||
wakeup_error= 0;
|
||||
DBUG_ASSERT(!this->waitee /* No prior registration allowed */);
|
||||
this->waitee= waitee;
|
||||
|
||||
mysql_mutex_lock(&waitee->LOCK_wait_commit);
|
||||
/*
|
||||
If waitee is in the middle of wakeup, then there is nothing to wait for,
|
||||
so we need not register. This is necessary to avoid a race in unregister,
|
||||
see comments on wakeup_subsequent_commits2() for details.
|
||||
*/
|
||||
if (waitee->wakeup_subsequent_commits_running)
|
||||
waiting_for_commit= false;
|
||||
else
|
||||
{
|
||||
/*
|
||||
Put ourself at the head of the waitee's list of transactions that must
|
||||
wait for it to commit first.
|
||||
*/
|
||||
this->next_subsequent_commit= waitee->subsequent_commits_list;
|
||||
waitee->subsequent_commits_list= this;
|
||||
}
|
||||
mysql_mutex_unlock(&waitee->LOCK_wait_commit);
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
Wait for commit of another transaction to complete, as already registered
|
||||
with register_wait_for_prior_commit(). If the commit already completed,
|
||||
returns immediately.
|
||||
*/
|
||||
int
|
||||
wait_for_commit::wait_for_prior_commit2()
|
||||
{
|
||||
mysql_mutex_lock(&LOCK_wait_commit);
|
||||
while (waiting_for_commit)
|
||||
mysql_cond_wait(&COND_wait_commit, &LOCK_wait_commit);
|
||||
mysql_mutex_unlock(&LOCK_wait_commit);
|
||||
waitee= NULL;
|
||||
return wakeup_error;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
Wakeup anyone waiting for us to have committed.
|
||||
|
||||
Note about locking:
|
||||
|
||||
We have a potential race or deadlock between wakeup_subsequent_commits() in
|
||||
the waitee and unregister_wait_for_prior_commit() in the waiter.
|
||||
|
||||
Both waiter and waitee needs to take their own lock before it is safe to take
|
||||
a lock on the other party - else the other party might disappear and invalid
|
||||
memory data could be accessed. But if we take the two locks in different
|
||||
order, we may end up in a deadlock.
|
||||
|
||||
The waiter needs to lock the waitee to delete itself from the list in
|
||||
unregister_wait_for_prior_commit(). Thus wakeup_subsequent_commits() can not
|
||||
hold its own lock while locking waiters, as this could lead to deadlock.
|
||||
|
||||
So we need to prevent unregister_wait_for_prior_commit() running while wakeup
|
||||
is in progress - otherwise the unregister could complete before the wakeup,
|
||||
leading to incorrect spurious wakeup or accessing invalid memory.
|
||||
|
||||
However, if we are in the middle of running wakeup_subsequent_commits(), then
|
||||
there is no need for unregister_wait_for_prior_commit() in the first place -
|
||||
the waiter can just do a normal wait_for_prior_commit(), as it will be
|
||||
immediately woken up.
|
||||
|
||||
So the solution to the potential race/deadlock is to set a flag in the waitee
|
||||
that wakeup_subsequent_commits() is in progress. When this flag is set,
|
||||
unregister_wait_for_prior_commit() becomes just wait_for_prior_commit().
|
||||
|
||||
Then also register_wait_for_prior_commit() needs to check if
|
||||
wakeup_subsequent_commits() is running, and skip the registration if
|
||||
so. This is needed in case a new waiter manages to register itself and
|
||||
immediately try to unregister while wakeup_subsequent_commits() is
|
||||
running. Else the new waiter would also wait rather than unregister, but it
|
||||
would not be woken up until next wakeup, which could be potentially much
|
||||
later than necessary.
|
||||
*/
|
||||
|
||||
void
|
||||
wait_for_commit::wakeup_subsequent_commits2(int wakeup_error)
|
||||
{
|
||||
wait_for_commit *waiter;
|
||||
|
||||
mysql_mutex_lock(&LOCK_wait_commit);
|
||||
wakeup_subsequent_commits_running= true;
|
||||
waiter= subsequent_commits_list;
|
||||
subsequent_commits_list= NULL;
|
||||
mysql_mutex_unlock(&LOCK_wait_commit);
|
||||
|
||||
while (waiter)
|
||||
{
|
||||
/*
|
||||
Important: we must grab the next pointer before waking up the waiter;
|
||||
once the wakeup is done, the field could be invalidated at any time.
|
||||
*/
|
||||
wait_for_commit *next= waiter->next_subsequent_commit;
|
||||
waiter->wakeup(wakeup_error);
|
||||
waiter= next;
|
||||
}
|
||||
|
||||
/*
|
||||
We need a full memory barrier between walking the list above, and clearing
|
||||
the flag wakeup_subsequent_commits_running below. This barrier is needed
|
||||
to ensure that no other thread will start to modify the list pointers
|
||||
before we are done traversing the list.
|
||||
|
||||
But wait_for_commit::wakeup() does a full memory barrier already (it locks
|
||||
a mutex), so no extra explicit barrier is needed here.
|
||||
*/
|
||||
wakeup_subsequent_commits_running= false;
|
||||
}
|
||||
|
||||
|
||||
/* Cancel a previously registered wait for another THD to commit before us. */
|
||||
void
|
||||
wait_for_commit::unregister_wait_for_prior_commit2()
|
||||
{
|
||||
mysql_mutex_lock(&LOCK_wait_commit);
|
||||
if (waiting_for_commit)
|
||||
{
|
||||
wait_for_commit *loc_waitee= this->waitee;
|
||||
wait_for_commit **next_ptr_ptr, *cur;
|
||||
mysql_mutex_lock(&loc_waitee->LOCK_wait_commit);
|
||||
if (loc_waitee->wakeup_subsequent_commits_running)
|
||||
{
|
||||
/*
|
||||
When a wakeup is running, we cannot safely remove ourselves from the
|
||||
list without corrupting it. Instead we can just wait, as wakeup is
|
||||
already in progress and will thus be immediate.
|
||||
|
||||
See comments on wakeup_subsequent_commits2() for more details.
|
||||
*/
|
||||
mysql_mutex_unlock(&loc_waitee->LOCK_wait_commit);
|
||||
while (waiting_for_commit)
|
||||
mysql_cond_wait(&COND_wait_commit, &LOCK_wait_commit);
|
||||
}
|
||||
else
|
||||
{
|
||||
/* Remove ourselves from the list in the waitee. */
|
||||
next_ptr_ptr= &loc_waitee->subsequent_commits_list;
|
||||
while ((cur= *next_ptr_ptr) != NULL)
|
||||
{
|
||||
if (cur == this)
|
||||
{
|
||||
*next_ptr_ptr= this->next_subsequent_commit;
|
||||
break;
|
||||
}
|
||||
next_ptr_ptr= &cur->next_subsequent_commit;
|
||||
}
|
||||
waiting_for_commit= false;
|
||||
mysql_mutex_unlock(&loc_waitee->LOCK_wait_commit);
|
||||
}
|
||||
}
|
||||
mysql_mutex_unlock(&LOCK_wait_commit);
|
||||
this->waitee= NULL;
|
||||
}
|
||||
|
||||
|
||||
bool Discrete_intervals_list::append(ulonglong start, ulonglong val,
|
||||
ulonglong incr)
|
||||
{
|
||||
|
158
sql/sql_class.h
158
sql/sql_class.h
@ -47,6 +47,7 @@
|
||||
|
||||
class Reprepare_observer;
|
||||
class Relay_log_info;
|
||||
struct rpl_group_info;
|
||||
class Rpl_filter;
|
||||
|
||||
class Query_log_event;
|
||||
@ -1556,6 +1557,120 @@ private:
|
||||
};
|
||||
|
||||
|
||||
/*
|
||||
Class to facilitate the commit of one transactions waiting for the commit of
|
||||
another transaction to complete first.
|
||||
|
||||
This is used during (parallel) replication, to allow different transactions
|
||||
to be applied in parallel, but still commit in order.
|
||||
|
||||
The transaction that wants to wait for a prior commit must first register
|
||||
to wait with register_wait_for_prior_commit(waitee). Such registration
|
||||
must be done holding the waitee->LOCK_wait_commit, to prevent the other
|
||||
THD from disappearing during the registration.
|
||||
|
||||
Then during commit, if a THD is registered to wait, it will call
|
||||
wait_for_prior_commit() as part of ha_commit_trans(). If no wait is
|
||||
registered, or if the waitee for has already completed commit, then
|
||||
wait_for_prior_commit() returns immediately.
|
||||
|
||||
And when a THD that may be waited for has completed commit (more precisely
|
||||
commit_ordered()), then it must call wakeup_subsequent_commits() to wake
|
||||
up any waiters. Note that this must be done at a point that is guaranteed
|
||||
to be later than any waiters registering themselves. It is safe to call
|
||||
wakeup_subsequent_commits() multiple times, as waiters are removed from
|
||||
registration as part of the wakeup.
|
||||
|
||||
The reason for separate register and wait calls is that this allows to
|
||||
register the wait early, at a point where the waited-for THD is known to
|
||||
exist. And then the actual wait can be done much later, where the
|
||||
waited-for THD may have been long gone. By registering early, the waitee
|
||||
can signal before disappearing.
|
||||
*/
|
||||
struct wait_for_commit
|
||||
{
|
||||
/*
|
||||
The LOCK_wait_commit protects the fields subsequent_commits_list and
|
||||
wakeup_subsequent_commits_running (for a waitee), and the flag
|
||||
waiting_for_commit and associated COND_wait_commit (for a waiter).
|
||||
*/
|
||||
mysql_mutex_t LOCK_wait_commit;
|
||||
mysql_cond_t COND_wait_commit;
|
||||
/* List of threads that did register_wait_for_prior_commit() on us. */
|
||||
wait_for_commit *subsequent_commits_list;
|
||||
/* Link field for entries in subsequent_commits_list. */
|
||||
wait_for_commit *next_subsequent_commit;
|
||||
/* Our waitee, if we did register_wait_for_prior_commit(), else NULL. */
|
||||
wait_for_commit *waitee;
|
||||
/*
|
||||
Generic pointer for use by the transaction coordinator to optimise the
|
||||
waiting for improved group commit.
|
||||
|
||||
Currently used by binlog TC to signal that a waiter is ready to commit, so
|
||||
that the waitee can grab it and group commit it directly. It is free to be
|
||||
used by another transaction coordinator for similar purposes.
|
||||
*/
|
||||
void *opaque_pointer;
|
||||
/*
|
||||
The waiting_for_commit flag is cleared when a waiter has been woken
|
||||
up. The COND_wait_commit condition is signalled when this has been
|
||||
cleared.
|
||||
*/
|
||||
bool waiting_for_commit;
|
||||
/* The wakeup error code from the waitee. 0 means no error. */
|
||||
int wakeup_error;
|
||||
/*
|
||||
Flag set when wakeup_subsequent_commits_running() is active, see comments
|
||||
on that function for details.
|
||||
*/
|
||||
bool wakeup_subsequent_commits_running;
|
||||
|
||||
void register_wait_for_prior_commit(wait_for_commit *waitee);
|
||||
int wait_for_prior_commit()
|
||||
{
|
||||
/*
|
||||
Quick inline check, to avoid function call and locking in the common case
|
||||
where no wakeup is registered, or a registered wait was already signalled.
|
||||
*/
|
||||
if (waiting_for_commit)
|
||||
return wait_for_prior_commit2();
|
||||
else
|
||||
return wakeup_error;
|
||||
}
|
||||
void wakeup_subsequent_commits(int wakeup_error)
|
||||
{
|
||||
/*
|
||||
Do the check inline, so only the wakeup case takes the cost of a function
|
||||
call for every commmit.
|
||||
|
||||
Note that the check is done without locking. It is the responsibility of
|
||||
the user of the wakeup facility to ensure that no waiters can register
|
||||
themselves after the last call to wakeup_subsequent_commits().
|
||||
|
||||
This avoids having to take another lock for every commit, which would be
|
||||
pointless anyway - even if we check under lock, there is nothing to
|
||||
prevent a waiter from arriving just after releasing the lock.
|
||||
*/
|
||||
if (subsequent_commits_list)
|
||||
wakeup_subsequent_commits2(wakeup_error);
|
||||
}
|
||||
void unregister_wait_for_prior_commit()
|
||||
{
|
||||
if (waiting_for_commit)
|
||||
unregister_wait_for_prior_commit2();
|
||||
}
|
||||
|
||||
void wakeup(int wakeup_error);
|
||||
|
||||
int wait_for_prior_commit2();
|
||||
void wakeup_subsequent_commits2(int wakeup_error);
|
||||
void unregister_wait_for_prior_commit2();
|
||||
|
||||
wait_for_commit();
|
||||
~wait_for_commit();
|
||||
};
|
||||
|
||||
|
||||
extern "C" void my_message_sql(uint error, const char *str, myf MyFlags);
|
||||
|
||||
class THD;
|
||||
@ -1590,8 +1705,9 @@ public:
|
||||
|
||||
/* Used to execute base64 coded binlog events in MySQL server */
|
||||
Relay_log_info* rli_fake;
|
||||
rpl_group_info* rgi_fake;
|
||||
/* Slave applier execution context */
|
||||
Relay_log_info* rli_slave;
|
||||
rpl_group_info* rgi_slave;
|
||||
|
||||
/* Used to SLAVE SQL thread */
|
||||
Rpl_filter* rpl_filter;
|
||||
@ -3207,6 +3323,25 @@ public:
|
||||
void wait_for_wakeup_ready();
|
||||
/* Wake this thread up from wait_for_wakeup_ready(). */
|
||||
void signal_wakeup_ready();
|
||||
|
||||
wait_for_commit *wait_for_commit_ptr;
|
||||
int wait_for_prior_commit()
|
||||
{
|
||||
if (wait_for_commit_ptr)
|
||||
{
|
||||
int err= wait_for_commit_ptr->wait_for_prior_commit();
|
||||
if (err)
|
||||
my_error(ER_PRIOR_COMMIT_FAILED, MYF(0));
|
||||
return err;
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
void wakeup_subsequent_commits(int wakeup_error)
|
||||
{
|
||||
if (wait_for_commit_ptr)
|
||||
wait_for_commit_ptr->wakeup_subsequent_commits(wakeup_error);
|
||||
}
|
||||
|
||||
private:
|
||||
|
||||
/** The current internal error handler for this thread, or NULL. */
|
||||
@ -3260,6 +3395,27 @@ private:
|
||||
bool wakeup_ready;
|
||||
mysql_mutex_t LOCK_wakeup_ready;
|
||||
mysql_cond_t COND_wakeup_ready;
|
||||
|
||||
/* Protect against add/delete of temporary tables in parallel replication */
|
||||
void rgi_lock_temporary_tables();
|
||||
void rgi_unlock_temporary_tables();
|
||||
bool rgi_have_temporary_tables();
|
||||
public:
|
||||
inline void lock_temporary_tables()
|
||||
{
|
||||
if (rgi_slave)
|
||||
rgi_lock_temporary_tables();
|
||||
}
|
||||
inline void unlock_temporary_tables()
|
||||
{
|
||||
if (rgi_slave)
|
||||
rgi_unlock_temporary_tables();
|
||||
}
|
||||
inline bool have_temporary_tables()
|
||||
{
|
||||
return (temporary_tables ||
|
||||
(rgi_slave && rgi_have_temporary_tables()));
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
|
@ -848,10 +848,10 @@ bool mysql_insert(THD *thd,TABLE_LIST *table_list,
|
||||
table->next_number_field=table->found_next_number_field;
|
||||
|
||||
#ifdef HAVE_REPLICATION
|
||||
if (thd->rli_slave &&
|
||||
if (thd->rgi_slave &&
|
||||
(info.handle_duplicates == DUP_UPDATE) &&
|
||||
(table->next_number_field != NULL) &&
|
||||
rpl_master_has_bug(thd->rli_slave, 24432, TRUE, NULL, NULL))
|
||||
rpl_master_has_bug(thd->rgi_slave->rli, 24432, TRUE, NULL, NULL))
|
||||
goto abort;
|
||||
#endif
|
||||
|
||||
@ -3510,10 +3510,10 @@ select_insert::prepare(List<Item> &values, SELECT_LEX_UNIT *u)
|
||||
table->next_number_field=table->found_next_number_field;
|
||||
|
||||
#ifdef HAVE_REPLICATION
|
||||
if (thd->rli_slave &&
|
||||
if (thd->rgi_slave &&
|
||||
(info.handle_duplicates == DUP_UPDATE) &&
|
||||
(table->next_number_field != NULL) &&
|
||||
rpl_master_has_bug(thd->rli_slave, 24432, TRUE, NULL, NULL))
|
||||
rpl_master_has_bug(thd->rgi_slave->rli, 24432, TRUE, NULL, NULL))
|
||||
DBUG_RETURN(1);
|
||||
#endif
|
||||
|
||||
|
@ -363,11 +363,11 @@ int mysql_load(THD *thd,sql_exchange *ex,TABLE_LIST *table_list,
|
||||
MY_RETURN_REAL_PATH);
|
||||
}
|
||||
|
||||
if (thd->rli_slave)
|
||||
if (thd->rgi_slave)
|
||||
{
|
||||
#if defined(HAVE_REPLICATION) && !defined(MYSQL_CLIENT)
|
||||
if (strncmp(thd->rli_slave->slave_patternload_file, name,
|
||||
thd->rli_slave->slave_patternload_file_size))
|
||||
if (strncmp(thd->rgi_slave->rli->slave_patternload_file, name,
|
||||
thd->rgi_slave->rli->slave_patternload_file_size))
|
||||
{
|
||||
/*
|
||||
LOAD DATA INFILE in the slave SQL Thread can only read from
|
||||
|
@ -4052,6 +4052,7 @@ end_with_restore_list:
|
||||
break;
|
||||
|
||||
case SQLCOM_BEGIN:
|
||||
DBUG_PRINT("info", ("Executing SQLCOM_BEGIN thd: %p", thd));
|
||||
if (trans_begin(thd, lex->start_transaction_opt))
|
||||
goto error;
|
||||
my_ok(thd);
|
||||
|
@ -57,6 +57,7 @@
|
||||
#include "threadpool.h"
|
||||
#include "sql_repl.h"
|
||||
#include "opt_range.h"
|
||||
#include "rpl_parallel.h"
|
||||
|
||||
/*
|
||||
The rule for this file: everything should be 'static'. When a sys_var
|
||||
@ -1531,9 +1532,83 @@ static Sys_var_gtid_binlog_state Sys_gtid_binlog_state(
|
||||
"The internal GTID state of the binlog, used to keep track of all "
|
||||
"GTIDs ever logged to the binlog.",
|
||||
GLOBAL_VAR(opt_gtid_binlog_state_dummy), NO_CMD_LINE);
|
||||
|
||||
|
||||
static bool
|
||||
check_slave_parallel_threads(sys_var *self, THD *thd, set_var *var)
|
||||
{
|
||||
bool running;
|
||||
|
||||
mysql_mutex_lock(&LOCK_active_mi);
|
||||
running= master_info_index->give_error_if_slave_running();
|
||||
mysql_mutex_unlock(&LOCK_active_mi);
|
||||
if (running)
|
||||
return true;
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
static bool
|
||||
fix_slave_parallel_threads(sys_var *self, THD *thd, enum_var_type type)
|
||||
{
|
||||
bool running;
|
||||
bool err= false;
|
||||
|
||||
mysql_mutex_unlock(&LOCK_global_system_variables);
|
||||
mysql_mutex_lock(&LOCK_active_mi);
|
||||
running= master_info_index->give_error_if_slave_running();
|
||||
mysql_mutex_unlock(&LOCK_active_mi);
|
||||
if (running || rpl_parallel_change_thread_count(&global_rpl_thread_pool,
|
||||
opt_slave_parallel_threads))
|
||||
err= true;
|
||||
mysql_mutex_lock(&LOCK_global_system_variables);
|
||||
|
||||
return err;
|
||||
}
|
||||
|
||||
|
||||
static Sys_var_ulong Sys_slave_parallel_threads(
|
||||
"slave_parallel_threads",
|
||||
"If non-zero, number of threads to spawn to apply in parallel events "
|
||||
"on the slave that were group-committed on the master or were logged "
|
||||
"with GTID in different replication domains.",
|
||||
GLOBAL_VAR(opt_slave_parallel_threads), CMD_LINE(REQUIRED_ARG),
|
||||
VALID_RANGE(0,16383), DEFAULT(0), BLOCK_SIZE(1), NO_MUTEX_GUARD,
|
||||
NOT_IN_BINLOG, ON_CHECK(check_slave_parallel_threads),
|
||||
ON_UPDATE(fix_slave_parallel_threads));
|
||||
|
||||
|
||||
static Sys_var_ulong Sys_slave_parallel_max_queued(
|
||||
"slave_parallel_max_queued",
|
||||
"Limit on how much memory SQL threads should use per parallel "
|
||||
"replication thread when reading ahead in the relay log looking for "
|
||||
"opportunities for parallel replication. Only used when "
|
||||
"--slave-parallel-threads > 0.",
|
||||
GLOBAL_VAR(opt_slave_parallel_max_queued), CMD_LINE(REQUIRED_ARG),
|
||||
VALID_RANGE(0,2147483647), DEFAULT(131072), BLOCK_SIZE(1));
|
||||
#endif
|
||||
|
||||
|
||||
static Sys_var_ulong Sys_binlog_commit_wait_count(
|
||||
"binlog_commit_wait_count",
|
||||
"If non-zero, binlog write will wait at most binlog_commit_wait_usec "
|
||||
"microseconds for at least this many commits to queue up for group "
|
||||
"commit to the binlog. This can reduce I/O on the binlog and provide "
|
||||
"increased opportunity for parallel apply on the slave, but too high "
|
||||
"a value will decrease commit throughput.",
|
||||
GLOBAL_VAR(opt_binlog_commit_wait_count), CMD_LINE(REQUIRED_ARG),
|
||||
VALID_RANGE(0, ULONG_MAX), DEFAULT(0), BLOCK_SIZE(1));
|
||||
|
||||
|
||||
static Sys_var_ulong Sys_binlog_commit_wait_usec(
|
||||
"binlog_commit_wait_usec",
|
||||
"Maximum time, in microseconds, to wait for more commits to queue up "
|
||||
"for binlog group commit. Only takes effect if the value of "
|
||||
"binlog_commit_wait_count is non-zero.",
|
||||
GLOBAL_VAR(opt_binlog_commit_wait_usec), CMD_LINE(REQUIRED_ARG),
|
||||
VALID_RANGE(0, ULONG_MAX), DEFAULT(100000), BLOCK_SIZE(1));
|
||||
|
||||
|
||||
static bool fix_max_join_size(sys_var *self, THD *thd, enum_var_type type)
|
||||
{
|
||||
SV *sv= type == OPT_GLOBAL ? &global_system_variables : &thd->variables;
|
||||
@ -4261,6 +4336,8 @@ static bool check_pseudo_slave_mode(sys_var *self, THD *thd, set_var *var)
|
||||
#ifndef EMBEDDED_LIBRARY
|
||||
delete thd->rli_fake;
|
||||
thd->rli_fake= NULL;
|
||||
delete thd->rgi_fake;
|
||||
thd->rgi_fake= NULL;
|
||||
#endif
|
||||
}
|
||||
else if (previous_val && val)
|
||||
|
@ -139,6 +139,11 @@ bool trans_begin(THD *thd, uint flags)
|
||||
}
|
||||
|
||||
thd->variables.option_bits&= ~(OPTION_BEGIN | OPTION_KEEP_LOG);
|
||||
|
||||
/*
|
||||
The following set should not be needed as the flag should always be 0
|
||||
when we come here. We should at some point change this to an assert.
|
||||
*/
|
||||
thd->transaction.all.modified_non_trans_table= FALSE;
|
||||
|
||||
if (res)
|
||||
|
@ -2929,6 +2929,11 @@ innobase_commit(
|
||||
/* We were instructed to commit the whole transaction, or
|
||||
this is an SQL statement end and autocommit is on */
|
||||
|
||||
/* At this point commit order is fixed and transaction is
|
||||
visible to others. So we can wakeup other commits waiting for
|
||||
this one, to allow then to group commit with us. */
|
||||
thd_wakeup_subsequent_commits(thd, 0);
|
||||
|
||||
/* We did the first part already in innobase_commit_ordered(),
|
||||
Now finish by doing a write + flush of logs. */
|
||||
trx_commit_complete_for_mysql(trx);
|
||||
|
@ -621,7 +621,7 @@ open_or_create_log_file(
|
||||
|| size_high != srv_calc_high32(srv_log_file_size)) {
|
||||
|
||||
fprintf(stderr,
|
||||
"InnoDB: Error: log file %s is"
|
||||
"InnoDB: Warning: log file %s is"
|
||||
" of different size %lu %lu bytes\n"
|
||||
"InnoDB: than specified in the .cnf"
|
||||
" file %lu %lu bytes!\n",
|
||||
@ -629,7 +629,8 @@ open_or_create_log_file(
|
||||
(ulong) srv_calc_high32(srv_log_file_size),
|
||||
(ulong) srv_calc_low32(srv_log_file_size));
|
||||
|
||||
return(DB_ERROR);
|
||||
srv_log_file_size= size +
|
||||
(((longlong) size_high) << 32);
|
||||
}
|
||||
} else {
|
||||
*log_file_created = TRUE;
|
||||
|
@ -311,12 +311,12 @@ bool CSphUrl::Parse ( const char * sUrl, int iLen )
|
||||
// unix-domain socket
|
||||
m_iPort = 0;
|
||||
if (!( m_sIndex = strrchr ( m_sHost, ':' ) ))
|
||||
m_sIndex = SPHINXSE_DEFAULT_INDEX;
|
||||
m_sIndex = const_cast<char *>(SPHINXSE_DEFAULT_INDEX);
|
||||
else
|
||||
{
|
||||
*m_sIndex++ = '\0';
|
||||
if ( !*m_sIndex )
|
||||
m_sIndex = SPHINXSE_DEFAULT_INDEX;
|
||||
m_sIndex = const_cast<char *>(SPHINXSE_DEFAULT_INDEX);
|
||||
}
|
||||
bOk = true;
|
||||
break;
|
||||
@ -336,7 +336,7 @@ bool CSphUrl::Parse ( const char * sUrl, int iLen )
|
||||
if ( m_sIndex )
|
||||
*m_sIndex++ = '\0';
|
||||
else
|
||||
m_sIndex = SPHINXSE_DEFAULT_INDEX;
|
||||
m_sIndex = const_cast<char *>(SPHINXSE_DEFAULT_INDEX);
|
||||
|
||||
m_iPort = atoi(sPort);
|
||||
if ( !m_iPort )
|
||||
@ -348,7 +348,7 @@ bool CSphUrl::Parse ( const char * sUrl, int iLen )
|
||||
if ( m_sIndex )
|
||||
*m_sIndex++ = '\0';
|
||||
else
|
||||
m_sIndex = SPHINXSE_DEFAULT_INDEX;
|
||||
m_sIndex = const_cast<char *>(SPHINXSE_DEFAULT_INDEX);
|
||||
}
|
||||
|
||||
bOk = true;
|
||||
|
@ -3659,6 +3659,11 @@ innobase_commit(
|
||||
/* We were instructed to commit the whole transaction, or
|
||||
this is an SQL statement end and autocommit is on */
|
||||
|
||||
/* At this point commit order is fixed and transaction is
|
||||
visible to others. So we can wakeup other commits waiting for
|
||||
this one, to allow then to group commit with us. */
|
||||
thd_wakeup_subsequent_commits(thd, 0);
|
||||
|
||||
/* We did the first part already in innobase_commit_ordered(),
|
||||
Now finish by doing a write + flush of logs. */
|
||||
trx_commit_complete_for_mysql(trx);
|
||||
|
@ -618,7 +618,7 @@ open_or_create_log_file(
|
||||
|| size_high != srv_calc_high32(srv_log_file_size)) {
|
||||
|
||||
fprintf(stderr,
|
||||
"InnoDB: Error: log file %s is"
|
||||
"InnoDB: Warning: log file %s is"
|
||||
" of different size %lu %lu bytes\n"
|
||||
"InnoDB: than specified in the .cnf"
|
||||
" file %lu %lu bytes!\n",
|
||||
@ -626,7 +626,9 @@ open_or_create_log_file(
|
||||
(ulong) srv_calc_high32(srv_log_file_size),
|
||||
(ulong) srv_calc_low32(srv_log_file_size));
|
||||
|
||||
return(DB_ERROR);
|
||||
srv_log_file_size= ((size +
|
||||
(((longlong) size_high) << 32)) /
|
||||
UNIV_PAGE_SIZE);
|
||||
}
|
||||
} else {
|
||||
*log_file_created = TRUE;
|
||||
|
Loading…
x
Reference in New Issue
Block a user