Skip to content

Commit

Permalink
RAFT SQUASH part 4
Browse files Browse the repository at this point in the history
recover raft logs by removing partial trxs

Summary:
Port D24628821

mysqld removes partial trxs in the tail of trx log (named binary-logs on
primaries and apply-logs on secondaries) during startup. However, relay logs
were not of much importance since it was anyways discarded and a new one would
be created.
However, with raft, this is not ideal. Relay logs are raft logs on secondaries
and have to be kept around (and kept sane and consistent). This diff adds the
ability to remove partial trxs from raft/relay logs.
Much of the code to open the last relay log (based on relay log index) and
identify partial trxs is borrowed from existing logic in
MYSQL_BIN_LOG::open_binlog() and binlog_recover()

Reviewed By: Pushapgl

Differential Revision: D26447448

---------------------------------------------------------------------

Checking for inited bool to make sure global_init_info was successful

Summary:
Port D25584004

A master.info and relay.info file can be present
but needs to be properly inited for use. We were bypassing the inited
check which could lead to issues in Raft.
In case there is an error in global_init_info, Raft will do a
raft_reset_slave and make another attempt at it. If both recourses
fail, the init of the plugin would fail.

Reviewed By: Pushapgl

Differential Revision: D26447457

---------------------------------------------------------------------

Support for dumping raft logs to vanilla async replicas

Summary:
[Porting Notes]
We want to dump raft logs to vanilla async replicas regardless
of whether it's the relay log or binlog. Effectively after this change
we'll dump relay logs on the followers and binlogs on the leader. When
the raft role changes, the logs to the dumped are also changed.
Dump_log class is introduced as a thin wrapper/continer around
mysql_bin_log or rli->relay_log and is inited with mysql_bin_log
to emulate vanilla mysql behavior. Dump threads use the global
dump_log object instead of mysql_bin_log directly. We switch the log
in dump log only when raft role changes (in binlog_change_to_binlog()
and binlog_change_to_apply_log()).
During raft role change we take all log releated locks (LOCK_log,
LOCK_index, LOCK_binlog_end_pos, and dump log lock) to serialize it with
other log operations like dumping logs.

Related doc - https://fb.quip.com/oTVAAdgEi4zY

This diff contains below 7 patches:
D23013977
D24766787
D24716539
D24900223
D24955284
D25174166
D25775525

Reviewed By: luqun

Differential Revision: D26141496

---------------------------------------------------------------------

Fixed the flaky raft test suite

Summary:
First clean run of entire raft test suite :)

**Changes**
* Reset apply_logs on raft secondaries before the start of every test
* Increased the lock timeouts and changed isolation level in rpl_raft_slave_out_of_order_commit.

Reviewed By: luqun

Differential Revision: D26651257

---------------------------------------------------------------------

return error from Raft_replication_delegate when plugin is not available

Summary:
Port D23065441 (facebook@b9067f7)

The new macro is used to call into raft plugin. If plugin gets unloaded
accidentally when enable_raft_plugin is ON, then this STRICT version returns
failure. This is to be called only by raft plugin currently

Reviewed By: Pushapgl

Differential Revision: D26447523

---------------------------------------------------------------------

Adding timestamps for raft rotates which happen in the context of listener thread

Summary:
Port D25572614

The timestamp of a binlog event is picked up from the when field in
the event. In most cases of rotation, the when is left unpopulated
during rotation for the top 3 events (fd, pgtid, metadata). However
in such a situation, a normal rotate (flush binary logs) still manages
to get a valid timestamp, since the thread in which the flush binary
logs happens has a valid start time.
Now enter Raft relay log rotations. In those cases and in the case
of config change rotate, the rotations are happening in the context
of a raft listener queue thread. In that context, the when and start
time of the thread are both 0. The diff handles this case by populating
the when field appropriately.

Reviewed By: bhatvinay

Differential Revision: D26194612

---------------------------------------------------------------------

Raft abrupt stepdown and trim binlog file / gtid test

Summary: binlog file should get trimmed for abrupt stepdown

Reviewed By: Pushapgl, bhatvinay

Differential Revision: D26169975

---------------------------------------------------------------------

Port: Fixes around raft log truncation.

Summary:
**Notes**
* New functions to block and unblock dump threads for plugin to use
during raft log truncation.

Below check is already done in raft plugin as part of raft plugin in D26866429.
* Re-init rli->cur_log in Relay_log_info::rli_init_info() instead of
just seeking to the beginning to handle the case when raft log is
truncated before starting the applier

Reviewed By: luqun

Differential Revision: D26759813

---------------------------------------------------------------------

rebase due to relay log Format_description_event event size difference

Summary:
WL#3549: Binlog Compression add extra 1 bytes data into Format_description_event

  3298 @@ -134,6 +137,7 @@ Format_description_event::Format_description_event(uint8_t binlog_ver,
  3299            VIEW_CHANGE_HEADER_LEN,
  3300            XA_PREPARE_HEADER_LEN,
  3301            ROWS_HEADER_LEN_V2,
  3302 +          TRANSACTION_PAYLOAD_EVENT,
  3303        };

Reviewed By: Pushapgl

Differential Revision: D27145095

---------------------------------------------------------------------

fix raft change string metadata event

Summary:
Saw a bunch stage-1 replicaset instance failed due to config change string failure during add/remove instance
```
I0401 00:16:10.894434 1823842 IoCacheUtils.cpp:333] getConfigChangeString eventType = ^G
E0401 00:16:10.894451 1823842 IoCacheUtils.cpp:363] Failed to read metadata event body from cache
E0401 00:16:10.894456 1823842 MysqlRaft.cpp:659] [replicate] Failed to get config change string from iocache
2021-04-01T00:16:10.894464-07:00 8307 [ERROR] [MY-010207] [Repl] Run function 'before_flush' in plugin 'RPL_RAFT' failed
2021-04-01T00:16:10.894478-07:00 8307 [ERROR] [MY-000000] [Server] Failed to rotate binary log
```

After some investigation, the issue is caused is that calculate metadata event length with config change string but forgot write config change string into event body.

Reviewed By: Pushapgl

Differential Revision: D27504157

---------------------------------------------------------------------

Tells raft mode in binlog

Summary:
WIn-Win: Tell whether this binlog is generated while this host is in raft mode.

We already has the bit in FD event, and this diff just speaks it out.

Reviewed By: mpercy

Differential Revision: D28664300

---------------------------------------------------------------------

fix flaky raft testcase

Summary:
There are multiple issues for MTR:
1. in sql/rpl_binlog_sender.cc, if secondaries IO thread receives fatal_error,
   it will quit IO thread instead of reconnect. use unknown error so that
   secondary IO thread can try to reconnect
2. mtr.add_suppression() call: mtr.add_suppression() will execute an insert
   mysql statement into mtr.test_suppressions table and mtr.test_suppressions
   table doesn't contain primary key, thus during idempotent recovery, secondary
   will fail to execute mtr.add_suppression() due to missing PK. Try to move all
   mtr.add_suppression() at the end of testcase to workaround  idempotent recovery failure.
3. When promotion, use raft_promote_to_leader.inc instead of `set rpl_raft_new_leader_uuid`,
   since raft_promote_to_leader will wait the new primary state becomes writeable
4. pass specific warning instead of '.*' to mtr.add_supression
5. etc

Reviewed By: Pushapgl, bhatvinay

Differential Revision: D28774820

---------------------------------------------------------------------

Using locks in Dump_log methods only when raft is enabled

Summary:
We don't need to take locks in non-raft mode since the
underlying MYSQL_BIN_LOG obj will never change. To handle race between
updation of enable_raft_plugin var and using its values in Dump_log we
kill and block all dump threads while updating the var and unblock them
once the var is updated.

Reviewed By: bhatvinay

Differential Revision: D28905522

---------------------------------------------------------------------

leader election to be a sync point in the new leader

Summary:
Port of D27582002 (facebook@39c70ca) from 5.6.35 to 8.0

Newly elected raft leader makes sure that all trxs from the previous
leader is committed by sql appliers. It then switches the server's trx
logs from apply-log-* to binary-log-*. To other part of the system this
looks like a rotation, but the necessary sync calls are not made here.
So, if the server (or os) restarts, then the storage engine could lose
the commit markers of the last batch of trxs. This will result in silent
data drift.
This diff fixes the problem by making an explicit call to
ha_flush_logs() before switching the server's trx logs

Reviewed By: luqun

Differential Revision: D28880407

---------------------------------------------------------------------

Error out SQL thread on out of order opids in raft logs

Summary:
OpIds should always be in order in the raft logs. Added a check
in SQL threads that thows an error and stops the applier when out of
order opids are detected.

Reviewed By: li-chi

Differential Revision: D28810840

---------------------------------------------------------------------

add GET_COMMITTED_GTIDS for raft

Summary:
During recovery Raft Log::Init needs to check with server
what gtids have been committed. Before doing that it finds the entire
set of trxs in the last raft log. The difference between logged -
committed are the pending opids.

Reviewed By: Pushapgl

Differential Revision: D29622786

---------------------------------------------------------------------

raft: skip mts_recovery_groups during start slave

Summary:
During MySQL8+Raft DMP, some instance fail to switch to Leader or start slave

```
2021-06-24T17:56:38.627423-07:00 431 [Note] [MY-010574] [Repl] Slave: MTS group recovery relay log info group_master_log_name /data/mysql/3127/bls-unittestdb658.frc2-3305-mysql.replicaset.180021/binary-logs-3727.000033, event_master_log_pos 1129.
2021-06-24T17:56:38.627473-07:00 431 [ERROR] [MY-010575] [Repl] Error looking for file after /binlogs/binary-logs-3307.000120.
2021-06-24T17:56:38.627516-07:00 431 [ERROR] [MY-000000] [Repl] load_mi_and_rli_from_repositories: rli_init_info returned error
```

similar to 5.6, we don't need to run mts_recovery_groups due to GTID_MODE is always enabled.

Reviewed By: Pushapgl

Differential Revision: D29520066

---------------------------------------------------------------------

update RaftListenerCallbackArg struct

Summary: Due to contract between raft plugin and mysql change, update RaftListenerCallbackArg struct to add  master_uuid field

Reviewed By: Pushapgl, bhatvinay

Differential Revision: D29981349

---------------------------------------------------------------------

always check mi during raft_reset_slave

Summary: add similar nullptr check for raft_reset_slave as raft_stop_sql_thread

Reviewed By: bhatvinay

Differential Revision: D29967915

---------------------------------------------------------------------

raft mtr: update rpl_end_raft.inc for disable-raft

Summary:
rpl_end_raft.inc will unregister raft plugin which will call reset slaves when online disable-raft is enabled. move rpl_end_raft.inc after rpl_stop_slaves.inc to stop slave correctly first.

after stop slaves, call mtr.add_suppression("") won't replicate to slaves. just call mtr.add_suppression("") for all instances(replicas).

Reviewed By: yizhang82

Differential Revision: D30062236

---------------------------------------------------------------------

fix master_uuid

Summary: fix master_uuid in raft mode.

Reviewed By: luqun

Differential Revision: D30261465

---------------------------------------------------------------------

incorrect File_size value in show raft logs result

Summary:
in show raft logs result, the File_size for latest logs file isn't updated correctly.

such as show raft logs;

```
+-------------------------+------------
| Log_name                | File_size  |
| binary-logs-3304.000670 |       1529 |
```

in fact
```
-rw-r----- 1 mysql backup 1669180487 Aug  4 14:49 binary-logs-3304.000670
-rw-r----- 1 mysql backup 1723994154 Aug  4 14:49 binary-logs-3304.000670
```
the  file_size from show raft logs is always 1529 but the real file_size is 1669180487.

The issue is related when writing IO_CACHE directly, its wrapper ostream's position isn't updated.

Reviewed By: yizhang82

Differential Revision: D30116345

---------------------------------------------------------------------

Add gtid_purged_for_tailing

Summary:
Add a new global variable gtid_purged_for_tailing.

It shows the purged GTID for binlog in non-raft mode, and purged GTID for raft log in raft mode.

Reviewed By: abhinav04sharma

Differential Revision: D29372776

---------------------------------------------------------------------

disable skip_setup_replica_if_unnecessary

Summary: After sync with latest official raft plugin, most of MTR failed due to skip_setup_replica_if_unnecessary optimize.

Reviewed By: yizhang82

Differential Revision: D30821648

---------------------------------------------------------------------

latency histograms for raft trx wait

Summary: Port of support added for the same in mysql 5.6. Should help monitor latencies in 8.0 + raft stage -1 replicasets.

Reviewed By: luqun

Differential Revision: D31064764

---------------------------------------------------------------------

handle cases where clean shutdown in raft aborts trxs

Summary: This is a port of the feature in 5.6.

Reviewed By: anirbanr-fb

Differential Revision: D31070593

---------------------------------------------------------------------

fix crash during binlog purging

Summary:
Any error returned by the plugin during binlog purging results in a
crash in mysql8 as the server tries to execute
binlog_error_action_abort. We need to differentiate explicitly between a
plugin error and other error (such as error related to doing disk IO
etc). In thsi particular case, the crash is because of trying to purge a
file that does not exist (i.e which is already purged previosuly) and
raft cannot find it in its index chunk (so it returns a error).

Reviewed By: anirbanr-fb

Differential Revision: D31149997

---------------------------------------------------------------------

update flaky rpl_raft_purged_gtids_dump_threads

Summary:
rpl_raft_purged_gtids_dump_threads MTR failed due to "Cannot replicate because the master purged required binary logs" after server4 will tail server2.

Try to sync server4 before switch to tail server2.

Reviewed By: bhatvinay

Differential Revision: D30818078

---------------------------------------------------------------------

fix various porting issues in mysql8 (raft) crash recovery

Summary:
1. Trimming of the binlog is left to raft plugin (based on the current
    leader's log). Server should skip this step as part of recovery. This
    essentially means setting 'valid_pos' to the last successfully parsed
    trx in the trx log (instead of the engine's view of the trx log) in
    MYSQL_BIN_LOG::recover()
  2. executed_gtid_set should be initialized based on the engine's view of
the trx log file cordinates. So, during startup appropriate flags need
to be passed into MYSQL_BIN_LOG::init_gtid_sets(). init_gtid_sets() is
already changed to handle this, but the flag was not set correctly
during server startup
3. Another fix is in MYSQL_BIN_LOG::init_gtid_sets()to corretly set the position
to read and calculate executed-gtid-set (based on the file name read from the
engine)

Reviewed By: anirbanr-fb

Differential Revision: D31284902

---------------------------------------------------------------------

show_raft_status should take LOCK_status

Summary:
This is a straight port of a 5.6 patch to 8.0

SHOW RAFT STATUS and SHOW GLOBAL STATUS go to the
same functions in the plugin and access shared data structures.
These functions return internal char * pointers to plugin global
variables. They need to be serialized with LOCK_status otherwise
it leads to race conditions.

Reviewed By: li-chi

Differential Revision: D31318340

---------------------------------------------------------------------

Disallowing reset master on raft replicasets

Summary:
This is a straight port of similar patch on 5.6 raft

reset master is inherently not raft compliant.
Its get rid of all binlogs and make an instance appear like
a fresh single instance database without consulting the ring.
We need to block it.

However under certain circumstances (in the future ) e.g. during
first time replicaset build, or when we can guarantee that instance
is single node raft ring, we can potentially do a reset master
followed by rebootstrap of raft. This can also be achieved by
disabling raft, reset master and then re-enabling raft, however
to keep open the door, I have left a force option and will thread
a mechanism from the plugin to call reset_master(force=true)

Reviewed By: li-chi

Differential Revision: D31299214

---------------------------------------------------------------------

Setting topology config in MTR

Summary:
Setting topology config in MTR so that feature that use
topology config can be tested easily. Setting rpl_raft_skip_smc_updates
to avoid unnecessary calls to SMC (even though we supply a dummy
replicaset name).

Reviewed By: li-chi

Differential Revision: D31543877

---------------------------------------------------------------------

Do not allow sql thread start when raft is doing a stop->{}->start transition

Summary:
This is a port of an existing patch made to 5.6 for Raft.

Raft will do a stop of the SQL thread during StopAllWrites.
Then it will repoint the binlog files and during that action, the
SQL threads have to remain stopped. We block it out in this diff by
keeping an atomic bool which can be checked from other functions.

This only applies to raft mode i.e. enable_raft_plugin = true.

Reviewed By: li-chi

Differential Revision: D31319499

---------------------------------------------------------------------

Handle printing of FD event generated by slave SQL thread

Summary:
Early returning in the FD:print() function makes mysqlbinlog not be able to parse Raft logs on secondaries.

The original commit which added this is d048c0f (P173872135)

To comply with the intent of the original bug fix, we avoid printing the FD event of a relay log as a 'BINLOG'.

Reviewed By: anirbanr-fb

Differential Revision: D26359417

---------------------------------------------------------------------

Add exponential backoff for smart restart

Summary:
RAFT Instance crash and Tx Logs filling up.

rocksdb can fill up txlogs.
we should stop restarting mysqld if we have restarted many times in a day

Reviewed By: anirbanr-fb

Differential Revision: D27372750

---------------------------------------------------------------------

always release data_lock mutex to avoid deadlock

Summary: in stage-1 replicaset, when kill a secondary instance, sometime the instance will run into deadlock due to process_raft_queue thread forgot to release its acquired mutex in raft_change_master

Reviewed By: Pushapgl, bhatvinay

Differential Revision: D27602667

---------------------------------------------------------------------

Gracefully exit mysqld_safe loop during backoff

Summary:
Currentt systemctl stop [email protected] can take 10 mins when mysqld_safe is in backoff period.

D28517599 adds a interrupt to sleep in mysql_stop, and mysqld_safe immediately break the retry loop if sleep is interruptted.

Reviewed By: anirbanr-fb

Differential Revision: D28606439

---------------------------------------------------------------------

Fix heap overflow in group_relay_log_name handling

Summary:
We were accessing group_relay_log_name in
Query_log_event::do_apply_event_worker() but it's assigned only after
the coordinator thread encounters an end event (i.e. xid event or a
query event with "COMMIT" or "ROLLBACK" query). This was causing a race
between accessing group_relay_log_name in the worker thread and writing
it on the coordinator thread. We don't need to set transaction position
in events other than end event, so now we set transaction position in
query event only if it's an end event. The race is eliminated because
group_relay_log_name is set before enqueuing the event to the worker
thread (in both dep repl and vanilla mts).

Reviewed By: lth

Differential Revision: D28767430

---------------------------------------------------------------------

fix memory during MYSQL_BIN_LOG::open_existing_binlog

Summary:
asandebug complain there are memory leaks during MYSQL_BIN_LOG open

Direct leak of 50 byte(s) in 1 object(s) allocated from:
    #0 0x67460ef in malloc
    #1 0x93f0777 in my_raw_malloc(unsigned long, int)
    #2 0x93f064a in my_malloc(unsigned int, unsigned long, int)
    #3 0x93f0eb0 in my_strdup(unsigned int, char const*, int)
    #4 0x8af01a6 in MYSQL_BIN_LOG::open(unsigned int, char const*, char const*, unsigned int)
    #5 0x8af8064 in MYSQL_BIN_LOG::open_binlog(char const*, char const*, unsigned long, bool, bool, bool, Format_description_log_event*, unsigned int, RaftRotateInfo*, bool)
    #6 0x8b00c00 in MYSQL_BIN_LOG::new_file_impl(bool, Format_description_log_event*, RaftRotateInfo*)
    #7 0x8d65e47 in rotate_relay_log(Master_info*, bool, bool, bool, RaftRotateInfo*)
    #8 0x8d661c0 in rotate_relay_log_for_raft(RaftRotateInfo*)
    #9 0x8c7696a in process_raft_queue
    #10 0xa0fa1fd in pfs_spawn_thread(void*)
    #11 0x7f8c9a12b20b in start_thread

release these memory before assign them

Reviewed By: Pushapgl

Differential Revision: D28819752
  • Loading branch information
inikep committed Apr 4, 2022
1 parent 6c65475 commit 7329da0
Show file tree
Hide file tree
Showing 94 changed files with 3,796 additions and 659 deletions.
5 changes: 4 additions & 1 deletion include/mysql/raft_listener_queue_if.h
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ enum class RaftListenerCallbackType {
GET_EXECUTED_GTIDS = 15,
SET_BINLOG_DURABILITY = 16,
RAFT_CONFIG_CHANGE = 17,

HANDLE_DUMP_THREADS = 18,
// Note: Please update CallbackTypeToString() below when adding/removing elems
// here
};
Expand All @@ -41,6 +41,7 @@ class RaftListenerCallbackArg {
bool val_bool;
uint32_t val_uint;
std::pair<std::string, unsigned int> master_instance;
std::string master_uuid;
std::string val_str;
std::map<std::string, unsigned int> val_sys_var_uint;
std::pair<int64_t, int64_t> val_opid;
Expand Down Expand Up @@ -152,6 +153,8 @@ class RaftListenerQueueIf {
return "SET_BINLOG_DURABILITY";
case RaftListenerCallbackType::RAFT_CONFIG_CHANGE:
return "RAFT_CONFIG_CHANGE";
case RaftListenerCallbackType::HANDLE_DUMP_THREADS:
return "HANDLE_DUMP_THREADS";
default:
return {};
}
Expand Down
2 changes: 1 addition & 1 deletion mysql-test/include/check-testcase.test
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ if ($tmp) {
let $rpl_enable_raft= `SELECT COUNT(*) FROM INFORMATION_SCHEMA.PLUGINS WHERE PLUGIN_NAME = 'RPL_RAFT'`;
if ($rpl_enable_raft)
{
--echo Source_Host ::1
--echo Source_Host 0000:0000:0000:0000:0000:0000:0000:0001
}
if (!$rpl_enable_raft)
{
Expand Down
2 changes: 1 addition & 1 deletion mysql-test/include/mtr_check.sql
Original file line number Diff line number Diff line change
Expand Up @@ -108,6 +108,7 @@ BEGIN
SELECT * FROM performance_schema.global_variables
WHERE variable_name NOT IN ('timestamp', 'server_uuid',
'gtid_executed', 'gtid_purged',
'gtid_purged_for_tailing',
'gtid_committed',
'group_replication_group_name',
'keyring_file_data',
Expand Down Expand Up @@ -247,4 +248,3 @@ BEGIN
END$$

DELIMITER ;

73 changes: 73 additions & 0 deletions mysql-test/include/raft_config.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,73 @@
#!/usr/local/bin/python3

import os
import sys
import json
import socket
import re

def execute_sql(server_id, query):
socket = f'{socket_path}/mysqld.{server_id}.sock'
stream = os.popen(
f'mysql --no-defaults -S {socket} -u root -D test -sNe "{query}"'
)
return stream.read().strip()

def ip():
stream = os.popen(
f'hostname -i'
)
return stream.read().strip()

num_servers = int(sys.argv[1])
socket_path = sys.argv[2]

hostname = socket.gethostname()
hostname = hostname.replace('.facebook.com', '')
dc = hostname.split(".")[1]
region = re.sub('[0-9]+$', '', dc)
ip = ip()

config={}

for i in range(1, num_servers + 1):
uuid = execute_sql(i, "select @@global.server_uuid")
server_id = int(execute_sql(i, "select @@global.server_id"))
port = execute_sql(i, "select @@global.port")

config["bootstrap"] = True
config["commit_rule"] = {"mode": 2}
config["replicaset_name"] = "mysql.replicaset.0"
server_config = {}
server_config["region"] = region
server_config["hostname"] = hostname
server_config["ip_port"] = f"[{ip}]:{port}"
server_config["uuid"] = uuid
server_config["backed_by_database"] = True
server_config["voter_type"] = 0
server_config["server_id"] = server_id
config["server_config"] = server_config

server_props = []

for j in range(1, num_servers + 1):
uuid = execute_sql(j, "select @@global.server_uuid")
server_id = int(execute_sql(j, "select @@global.server_id"))
port = execute_sql(j, "select @@global.port")
server_prop = {}
server_prop["region"] = region
server_prop["hostname"] = hostname
server_prop["ip_port"] = f"[{ip}]:{port}"
server_prop["uuid"] = uuid
server_prop["server_id"] = server_id
server_prop["backed_by_database"] = True
server_prop["voter_type"] = 0
server_props.append(server_prop)

config["raft_topology"] = {"raft_server_properties": server_props}
config["voter_distribution"] = {region: num_servers}
config["enable_flexiraft"] = False

config_str = json.dumps(config)
config_str = config_str.replace('"', r'\"')
execute_sql(i, f"set @@global.rpl_raft_topology_config_json='{config_str}'")
10 changes: 5 additions & 5 deletions mysql-test/include/rpl_end.inc
Original file line number Diff line number Diff line change
Expand Up @@ -150,11 +150,6 @@ if ($rpl_group_replication)
}
}

if ($rpl_enable_raft)
{
--source include/rpl_end_raft.inc
}

if (!$rpl_group_replication)
{
if (!$rpl_skip_stop_slave)
Expand Down Expand Up @@ -219,6 +214,11 @@ if (!$rpl_group_replication)
}
}

if ($rpl_enable_raft)
{
--source include/rpl_end_raft.inc
}

# Restore the server state by deleting all channels
if ($rpl_multi_source)
{
Expand Down
17 changes: 17 additions & 0 deletions mysql-test/include/rpl_end_raft.inc
Original file line number Diff line number Diff line change
@@ -1,3 +1,17 @@
# Suppress known warnings and reset binlogs on all raft followers.
--disable_query_log
connection server_1;
call mtr.add_suppression(".*using --replicate-same-server-id in conjunction with --log-slave-updates.*");
connection server_2;
call mtr.add_suppression(".*using --replicate-same-server-id in conjunction with --log-slave-updates.*");
connection server_3;
call mtr.add_suppression(".*using --replicate-same-server-id in conjunction with --log-slave-updates.*");
connection server_4;
call mtr.add_suppression(".*using --replicate-same-server-id in conjunction with --log-slave-updates.*");
connection server_5;
call mtr.add_suppression(".*using --replicate-same-server-id in conjunction with --log-slave-updates.*");
--enable_query_log

let $_rpl_server= $rpl_server_count;
while ($_rpl_server)
{
Expand All @@ -10,7 +24,10 @@ while ($_rpl_server)
disable_query_log;
SET GLOBAL RPL_RAFT_ON= 0;
SET GLOBAL RPL_RAFT_CONFIG_JSON= default;
SET GLOBAL RPL_RAFT_TOPOLOGY_CONFIG_JSON= default;
SET GLOBAL RPL_RAFT_SKIP_SMC_UPDATES= default;
SET GLOBAL RPL_RAFT_NEW_LEADER_UUID= default;
SET GLOBAL RPL_RAFT_START_ELECTION= default;
SET GLOBAL READ_ONLY_ERROR_MSG_EXTRA= default;
exec rm -fr $fs_wal_path;
if (!$plugin_was_early_installed)
Expand Down
13 changes: 13 additions & 0 deletions mysql-test/include/rpl_init_raft.inc
Original file line number Diff line number Diff line change
Expand Up @@ -32,11 +32,24 @@ while ($_rpl_server)
--let $ipv6_host_port= `SELECT CONCAT('[::1]', ':', (SELECT @@GLOBAL.PORT), ',')`
--let $instances= `SELECT CONCAT('$instances', '$ipv6_host_port')`
let $date= `SELECT DATE_FORMAT(NOW(), '%m-%d-%Y')`;
eval SET @@GLOBAL.RPL_RAFT_SKIP_SMC_UPDATES= 'ON,$date';
--dec $_rpl_server
}
--let $instances= `SELECT TRIM(TRAILING ',' FROM '$instances')`
--let $instances= `SELECT CONCAT('$instances', '"')`
let $socket_dir= `SELECT SUBSTRING(VARIABLE_VALUE, 1,
LOCATE(SUBSTRING_INDEX(
VARIABLE_VALUE, '/', -1),
VARIABLE_VALUE) - 2)
FROM performance_schema.global_variables
WHERE VARIABLE_NAME = 'socket'`;
exec ./include/raft_config.py $rpl_server_count $socket_dir;
--let $_rpl_server= $rpl_server_count
while ($_rpl_server)
{
Expand Down
8 changes: 8 additions & 0 deletions mysql-test/r/mysqld--help-notwin.result
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,9 @@ The following options may be given as the first argument:
Also read groups with concat(group, suffix)
--login-path=# Read this path from the login file.

--abort-on-raft-purge-error
Any error in raft plugin to purge files will abort the
server
--abort-slave-event-count=#
Option used by mysql-test for debugging and testing of
replication.
Expand Down Expand Up @@ -1386,6 +1389,9 @@ The following options may be given as the first argument:
--read-rnd-buffer-size=#
When reading rows in sorted order after a sort, the rows
are read through this buffer to avoid a disk seeks
--recover-raft-log Temprary variable to control recovery of raft log by
removing partial trxs. This should be removed later.
(Defaults to on; use --skip-recover-raft-log to disable.)
--regexp-stack-limit=#
Stack size limit for regular expressions matches
--regexp-time-limit=#
Expand Down Expand Up @@ -2702,6 +2708,7 @@ The following options may be given as the first argument:
library is selected.

Variables (--variable-name=value)
abort-on-raft-purge-error FALSE
abort-slave-event-count 0
activate-all-roles-on-login FALSE
admin-address (No default value)
Expand Down Expand Up @@ -3088,6 +3095,7 @@ read-only FALSE
read-only-error-msg-extra
read-only-slave TRUE
read-rnd-buffer-size 262144
recover-raft-log TRUE
regexp-stack-limit 8000000
regexp-time-limit 32
relay-log relaylog
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ VARIABLE_NAME LIKE '%source%') AND
'binlog_file_basedir', 'binlog_index_basedir',
'skip_flush_master_info', 'skip_flush_relay_worker_info',
'read_only_slave', 'rpl_semi_sync_source_crash_if_active_trxs',
'rpl_skip_tx_api',
'rpl_skip_tx_api', 'gtid_purged_for_tailing',
'reset_seconds_behind_master', 'group_replication_plugin_hooks',
'innodb_master_thread_disabled_debug', 'innodb_replication_delay'))
AND (VARIABLE_NAME NOT LIKE 'rocksdb%')
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ VARIABLE_NAME LIKE '%slave%' OR
VARIABLE_NAME LIKE '%source%') AND
(VARIABLE_NAME NOT IN ('innodb_api_enable_binlog',
'binlog_file_basedir', 'binlog_index_basedir',
'gtid_purged_for_tailing',
'skip_flush_master_info', 'skip_flush_relay_worker_info',
'read_only_slave', 'rpl_semi_sync_source_crash_if_active_trxs',
'reset_seconds_behind_master', 'group_replication_plugin_hooks',
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ INSERT INTO rplvars (varname, varvalue)
'binlog_file_basedir', 'binlog_index_basedir',
'skip_flush_master_info', 'skip_flush_relay_worker_info',
'read_only_slave', 'rpl_semi_sync_source_crash_if_active_trxs',
'rpl_skip_tx_api',
'rpl_skip_tx_api', 'gtid_purged_for_tailing',
'reset_seconds_behind_master', 'group_replication_plugin_hooks',
'innodb_master_thread_disabled_debug', 'innodb_replication_delay'))
AND (VARIABLE_NAME NOT LIKE 'rocksdb%')
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@ INSERT INTO rplvars (varname, varvalue)
VARIABLE_NAME LIKE '%source%') AND
(VARIABLE_NAME NOT IN ('innodb_api_enable_binlog',
'binlog_file_basedir', 'binlog_index_basedir',
'gtid_purged_for_tailing',
'skip_flush_master_info', 'skip_flush_relay_worker_info',
'read_only_slave', 'rpl_semi_sync_source_crash_if_active_trxs',
'reset_seconds_behind_master', 'group_replication_plugin_hooks',
Expand Down
4 changes: 4 additions & 0 deletions mysql-test/suite/perfschema/r/relaylog.result
Original file line number Diff line number Diff line change
Expand Up @@ -72,6 +72,7 @@ wait/synch/mutex/sql/MYSQL_BIN_LOG::LOCK_done MANY
wait/synch/mutex/sql/MYSQL_BIN_LOG::LOCK_flush_queue MANY
wait/synch/mutex/sql/MYSQL_BIN_LOG::LOCK_index MANY
wait/synch/mutex/sql/MYSQL_BIN_LOG::LOCK_log MANY
wait/synch/mutex/sql/MYSQL_BIN_LOG::LOCK_lost_gtids_for_tailing MANY
wait/synch/mutex/sql/MYSQL_BIN_LOG::LOCK_non_xid_trxs MANY
wait/synch/mutex/sql/MYSQL_BIN_LOG::LOCK_sync MANY
wait/synch/mutex/sql/MYSQL_BIN_LOG::LOCK_sync_queue MANY
Expand Down Expand Up @@ -103,6 +104,7 @@ wait/synch/mutex/sql/MYSQL_RELAY_LOG::LOCK_commit NONE NONE NONE NONE NONE
wait/synch/mutex/sql/MYSQL_RELAY_LOG::LOCK_index MANY MANY MANY MANY MANY
wait/synch/mutex/sql/MYSQL_RELAY_LOG::LOCK_log MANY MANY MANY MANY MANY
wait/synch/mutex/sql/MYSQL_RELAY_LOG::LOCK_log_end_pos NONE NONE NONE NONE NONE
wait/synch/mutex/sql/MYSQL_RELAY_LOG::LOCK_lost_gtids_for_tailing NONE NONE NONE NONE NONE
wait/synch/mutex/sql/MYSQL_RELAY_LOG::LOCK_sync NONE NONE NONE NONE NONE
wait/synch/mutex/sql/MYSQL_RELAY_LOG::LOCK_xids NONE NONE NONE NONE NONE
"============ Performance schema on slave ============"
Expand Down Expand Up @@ -174,6 +176,7 @@ wait/synch/mutex/sql/MYSQL_BIN_LOG::LOCK_done MANY
wait/synch/mutex/sql/MYSQL_BIN_LOG::LOCK_flush_queue MANY
wait/synch/mutex/sql/MYSQL_BIN_LOG::LOCK_index MANY
wait/synch/mutex/sql/MYSQL_BIN_LOG::LOCK_log MANY
wait/synch/mutex/sql/MYSQL_BIN_LOG::LOCK_lost_gtids_for_tailing MANY
wait/synch/mutex/sql/MYSQL_BIN_LOG::LOCK_non_xid_trxs MANY
wait/synch/mutex/sql/MYSQL_BIN_LOG::LOCK_sync MANY
wait/synch/mutex/sql/MYSQL_BIN_LOG::LOCK_sync_queue MANY
Expand Down Expand Up @@ -221,6 +224,7 @@ wait/synch/mutex/sql/MYSQL_RELAY_LOG::LOCK_commit NONE
wait/synch/mutex/sql/MYSQL_RELAY_LOG::LOCK_index MANY
wait/synch/mutex/sql/MYSQL_RELAY_LOG::LOCK_log MANY
wait/synch/mutex/sql/MYSQL_RELAY_LOG::LOCK_log_end_pos MANY
wait/synch/mutex/sql/MYSQL_RELAY_LOG::LOCK_lost_gtids_for_tailing MANY
wait/synch/mutex/sql/MYSQL_RELAY_LOG::LOCK_sync NONE
wait/synch/mutex/sql/MYSQL_RELAY_LOG::LOCK_xids MANY
include/rpl_end.inc
47 changes: 45 additions & 2 deletions mysql-test/suite/rpl_raft/include/raft_3_node.inc
Original file line number Diff line number Diff line change
Expand Up @@ -73,12 +73,55 @@ let $show_metadata_event= 1;
# See: show_events.inc
let $dont_decrement_filename= 1;


--let $include_filename= raft_3_node.inc
--source include/end_include_file.inc


# Set the default connection to 'master'. Do this after
# end_include_file.inc, so that it gets printed to the query log.
--let $rpl_connection_name= master
--source include/rpl_connection.inc

# Create connections to server 4 and 5 (these are not in the ring)
let $rpl_server_number= 4;
let $rpl_connection_name= server_4;
source include/rpl_connect.inc;

let $rpl_server_number= 5;
let $rpl_connection_name= server_5;
source include/rpl_connect.inc;

connection server_1;
show status like 'rpl_raft_role';

connection server_2;
show status like 'rpl_raft_role';
let $exec_gtid_set= query_get_value(SHOW MASTER STATUS, Executed_Gtid_Set, 1);
reset master;
--disable_query_log
eval set global gtid_purged='$exec_gtid_set';
--enable_query_log

connection server_3;
show status like 'rpl_raft_role';
let $exec_gtid_set= query_get_value(SHOW MASTER STATUS, Executed_Gtid_Set, 1);
reset master;
--disable_query_log
eval set global gtid_purged='$exec_gtid_set';
--enable_query_log

connection server_4;
let $exec_gtid_set= query_get_value(SHOW MASTER STATUS, Executed_Gtid_Set, 1);
--disable_query_log
reset master;
eval set global gtid_purged='$exec_gtid_set';
--enable_query_log

connection server_5;
let $exec_gtid_set= query_get_value(SHOW MASTER STATUS, Executed_Gtid_Set, 1);
--disable_query_log
reset master;
eval set global gtid_purged='$exec_gtid_set';
--enable_query_log

# switch back to server_1
connection server_1;
Loading

0 comments on commit 7329da0

Please sign in to comment.