Unverified Commit a7afd1d2 authored by Ozan Tezcan's avatar Ozan Tezcan Committed by GitHub
Browse files

Reply LOADING on replica while flushing the db (#13495)

On a full sync, replica starts discarding existing db. If the existing 
db is huge and flush is happening synchronously, replica may become 
unresponsive. 

Adding a change to yield back to event loop while flushing db on 
a replica. Replica will reply -LOADING in this case. Note that while 
replica is loading the new rdb, it may get an error and start flushing
the partial db. This step may take a long time as well. Similarly, 
replica will reply -LOADING in this case. 

To call processEventsWhileBlocked() and reply -LOADING, we need to do:
- Set connSetReadHandler() null not to process further data from the master
- Set server.loading flag
- Call blockingOperationStarts()

rdbload() already does these steps and calls processEventsWhileBlocked()
while loading the rdb. Added a new call rdbLoadWithEmptyFunc() which 
accepts callback to flush db before loading rdb or when an error 
happens while loading. 

For diskless replication, d...
parent 3fcddfb6
......@@ -997,6 +997,29 @@ int startAppendOnly(void) {
return C_OK;
}
void startAppendOnlyWithRetry(void) {
unsigned int tries, max_tries = 10;
for (tries = 0; tries < max_tries; ++tries) {
if (startAppendOnly() == C_OK)
break;
serverLog(LL_WARNING, "Failed to enable AOF! Trying it again in one second.");
sleep(1);
}
if (tries == max_tries) {
serverLog(LL_WARNING, "FATAL: AOF can't be turned on. Exiting now.");
exit(1);
}
}
/* Called after "appendonly" config is changed. */
void applyAppendOnlyConfig(void) {
if (!server.aof_enabled && server.aof_state != AOF_OFF) {
stopAppendOnly();
} else if (server.aof_enabled && server.aof_state == AOF_OFF) {
startAppendOnlyWithRetry();
}
}
/* This is a wrapper to the write syscall in order to retry on short writes
* or if the syscall gets interrupted. It could look strange that we retry
* on short writes given that we are writing to a block device: normally if
......
......@@ -2502,6 +2502,13 @@ static int updateWatchdogPeriod(const char **err) {
}
static int updateAppendonly(const char **err) {
/* If loading flag is set, AOF might have been stopped temporarily, and it
* will be restarted depending on server.aof_enabled flag after loading is
* completed. So, we just need to update 'server.aof_enabled' which has been
* updated already before calling this function. */
if (server.loading)
return 1;
if (!server.aof_enabled && server.aof_state != AOF_OFF) {
stopAppendOnly();
} else if (server.aof_enabled && server.aof_state == AOF_OFF) {
......@@ -3080,7 +3087,7 @@ standardConfig static_configs[] = {
createBoolConfig("activedefrag", NULL, DEBUG_CONFIG | MODIFIABLE_CONFIG, server.active_defrag_enabled, 0, isValidActiveDefrag, NULL),
createBoolConfig("syslog-enabled", NULL, IMMUTABLE_CONFIG, server.syslog_enabled, 0, NULL, NULL),
createBoolConfig("cluster-enabled", NULL, IMMUTABLE_CONFIG, server.cluster_enabled, 0, NULL, NULL),
createBoolConfig("appendonly", NULL, MODIFIABLE_CONFIG | DENY_LOADING_CONFIG, server.aof_enabled, 0, NULL, updateAppendonly),
createBoolConfig("appendonly", NULL, MODIFIABLE_CONFIG, server.aof_enabled, 0, NULL, updateAppendonly),
createBoolConfig("cluster-allow-reads-when-down", NULL, MODIFIABLE_CONFIG, server.cluster_allow_reads_when_down, 0, NULL, NULL),
createBoolConfig("cluster-allow-pubsubshard-when-down", NULL, MODIFIABLE_CONFIG, server.cluster_allow_pubsubshard_when_down, 1, NULL, NULL),
createBoolConfig("crash-log-enabled", NULL, MODIFIABLE_CONFIG, server.crashlog_enabled, 1, NULL, updateSighandlerEnabled),
......
......@@ -575,11 +575,11 @@ redisDb *initTempDb(void) {
}
/* Discard tempDb, this can be slow (similar to FLUSHALL), but it's always async. */
void discardTempDb(redisDb *tempDb, void(callback)(dict*)) {
void discardTempDb(redisDb *tempDb) {
int async = 1;
/* Release temp DBs. */
emptyDbStructure(tempDb, -1, async, callback);
emptyDbStructure(tempDb, -1, async, NULL);
for (int i=0; i<server.dbnum; i++) {
/* Destroy global HFE DS before deleting the hashes since ebuckets DS is
* embedded in the stored objects. */
......
......@@ -567,6 +567,7 @@ NULL
addReplyError(c,"Error trying to load the RDB dump, check server logs.");
return;
}
applyAppendOnlyConfig(); /* Check if AOF config was changed while loading */
serverLog(LL_NOTICE,"DB reloaded by DEBUG RELOAD");
addReply(c,shared.ok);
} else if (!strcasecmp(c->argv[1]->ptr,"loadaof")) {
......@@ -582,6 +583,7 @@ NULL
addReplyError(c, "Error trying to load the AOF files, check server logs.");
return;
}
applyAppendOnlyConfig(); /* Check if AOF config was changed while loading */
server.dirty = 0; /* Prevent AOF / replication */
serverLog(LL_NOTICE,"Append Only File loaded by DEBUG LOADAOF");
addReply(c,shared.ok);
......
......@@ -4274,7 +4274,7 @@ int RM_SetAbsExpire(RedisModuleKey *key, mstime_t expire) {
void RM_ResetDataset(int restart_aof, int async) {
if (restart_aof && server.aof_state != AOF_OFF) stopAppendOnly();
flushAllDataAndResetRDB((async? EMPTYDB_ASYNC: EMPTYDB_NO_FLAGS) | EMPTYDB_NOFUNCTIONS);
if (server.aof_enabled && restart_aof) restartAOFAfterSYNC();
if (server.aof_enabled && restart_aof) startAppendOnlyWithRetry();
}
 
/* Returns the number of keys in the current db. */
......
......@@ -3161,7 +3161,13 @@ emptykey:
/* Mark that we are loading in the global state and setup the fields
* needed to provide loading stats. */
void startLoading(size_t size, int rdbflags, int async) {
/* Load the DB */
loadingSetFlags(NULL, size, async);
loadingFireEvent(rdbflags);
}
/* Initialize stats, set loading flags and filename if provided. */
void loadingSetFlags(char *filename, size_t size, int async) {
rdbFileBeingLoaded = filename;
server.loading = 1;
if (async == 1) server.async_loading = 1;
server.loading_start_time = time(NULL);
......@@ -3171,7 +3177,9 @@ void startLoading(size_t size, int rdbflags, int async) {
server.rdb_last_load_keys_expired = 0;
server.rdb_last_load_keys_loaded = 0;
blockingOperationStarts();
}
void loadingFireEvent(int rdbflags) {
/* Fire the loading modules start event. */
int subevent;
if (rdbflags & RDBFLAGS_AOF_PREAMBLE)
......@@ -3187,8 +3195,8 @@ void startLoading(size_t size, int rdbflags, int async) {
* needed to provide loading stats.
* 'filename' is optional and used for rdb-check on error */
void startLoadingFile(size_t size, char* filename, int rdbflags) {
rdbFileBeingLoaded = filename;
startLoading(size, rdbflags, 0);
loadingSetFlags(filename, size, 0);
loadingFireEvent(rdbflags);
}
/* Refresh the absolute loading progress info */
......@@ -3702,14 +3710,21 @@ eoferr:
return C_ERR;
}
int rdbLoad(char *filename, rdbSaveInfo *rsi, int rdbflags) {
return rdbLoadWithEmptyFunc(filename, rsi, rdbflags, NULL);
}
/* Like rdbLoadRio() but takes a filename instead of a rio stream. The
* filename is open for reading and a rio stream object created in order
* to do the actual loading. Moreover the ETA displayed in the INFO
* output is initialized and finalized.
*
* If you pass an 'rsi' structure initialized with RDB_SAVE_INFO_INIT, the
* loading code will fill the information fields in the structure. */
int rdbLoad(char *filename, rdbSaveInfo *rsi, int rdbflags) {
* loading code will fill the information fields in the structure.
*
* If emptyDbFunc is not NULL, it will be called to flush old db or to
* discard partial db on error. */
int rdbLoadWithEmptyFunc(char *filename, rdbSaveInfo *rsi, int rdbflags, void (*emptyDbFunc)(void)) {
FILE *fp;
rio rdb;
int retval;
......@@ -3727,12 +3742,20 @@ int rdbLoad(char *filename, rdbSaveInfo *rsi, int rdbflags) {
if (fstat(fileno(fp), &sb) == -1)
sb.st_size = 0;
startLoadingFile(sb.st_size, filename, rdbflags);
loadingSetFlags(filename, sb.st_size, 0);
/* Note that inside loadingSetFlags(), server.loading is set.
* emptyDbCallback() may yield back to event-loop to reply -LOADING. */
if (emptyDbFunc)
emptyDbFunc(); /* Flush existing db. */
loadingFireEvent(rdbflags);
rioInitWithFile(&rdb,fp);
retval = rdbLoadRio(&rdb,rdbflags,rsi);
fclose(fp);
if (retval != C_OK && emptyDbFunc)
emptyDbFunc(); /* Clean up partial db. */
stopLoading(retval==C_OK);
/* Reclaim the cache backed by rdb */
if (retval == C_OK && !(rdbflags & RDBFLAGS_KEEP_CACHE)) {
......
......@@ -136,6 +136,7 @@ uint64_t rdbLoadLen(rio *rdb, int *isencoded);
int rdbLoadLenByRef(rio *rdb, int *isencoded, uint64_t *lenptr);
int rdbSaveObjectType(rio *rdb, robj *o);
int rdbLoadObjectType(rio *rdb);
int rdbLoadWithEmptyFunc(char *filename, rdbSaveInfo *rsi, int rdbflags, void (*emptyDbFunc)(void));
int rdbLoad(char *filename, rdbSaveInfo *rsi, int rdbflags);
int rdbSaveBackground(int req, char *filename, rdbSaveInfo *rsi, int rdbflags);
int rdbSaveToSlavesSockets(int req, rdbSaveInfo *rsi);
......
......@@ -1736,12 +1736,24 @@ void replicationSendNewlineToMaster(void) {
}
/* Callback used by emptyData() while flushing away old data to load
* the new dataset received by the master and by discardTempDb()
* after loading succeeded or failed. */
* the new dataset received by the master or to clear partial db if loading
* fails. */
void replicationEmptyDbCallback(dict *d) {
UNUSED(d);
if (server.repl_state == REPL_STATE_TRANSFER)
replicationSendNewlineToMaster();
processEventsWhileBlocked();
}
/* Function to flush old db or the partial db on error. */
static void rdbLoadEmptyDbFunc(void) {
serverAssert(server.loading);
serverLog(LL_NOTICE, "MASTER <-> REPLICA sync: Flushing old data");
int empty_db_flags = server.repl_slave_lazy_flush ? EMPTYDB_ASYNC :
EMPTYDB_NO_FLAGS;
emptyData(-1, empty_db_flags, replicationEmptyDbCallback);
}
/* Once we have a link with the master and the synchronization was
......@@ -1778,27 +1790,6 @@ void replicationCreateMasterClient(connection *conn, int dbid) {
if (dbid != -1) selectDb(server.master,dbid);
}
/* This function will try to re-enable the AOF file after the
* master-replica synchronization: if it fails after multiple attempts
* the replica cannot be considered reliable and exists with an
* error. */
void restartAOFAfterSYNC(void) {
unsigned int tries, max_tries = 10;
for (tries = 0; tries < max_tries; ++tries) {
if (startAppendOnly() == C_OK) break;
serverLog(LL_WARNING,
"Failed enabling the AOF after successful master synchronization! "
"Trying it again in one second.");
sleep(1);
}
if (tries == max_tries) {
serverLog(LL_WARNING,
"FATAL: this replica instance finished the synchronization with "
"its master, but the AOF can't be turned on. Exiting now.");
exit(1);
}
}
static int useDisklessLoad(void) {
/* compute boolean decision to use diskless load */
int enabled = server.repl_diskless_load == REPL_DISKLESS_LOAD_SWAPDB ||
......@@ -1831,7 +1822,7 @@ redisDb *disklessLoadInitTempDb(void) {
/* Helper function for readSyncBulkPayload() to discard our tempDb
* when the loading succeeded or failed. */
void disklessLoadDiscardTempDb(redisDb *tempDb) {
discardTempDb(tempDb, replicationEmptyDbCallback);
discardTempDb(tempDb);
}
/* If we know we got an entirely different data set from our master
......@@ -2057,9 +2048,6 @@ void readSyncBulkPayload(connection *conn) {
NULL);
} else {
replicationAttachToNewMaster();
serverLog(LL_NOTICE, "MASTER <-> REPLICA sync: Flushing old data");
emptyData(-1,empty_db_flags,replicationEmptyDbCallback);
}
/* Before loading the DB into memory we need to delete the readable
......@@ -2093,13 +2081,22 @@ void readSyncBulkPayload(connection *conn) {
functionsLibCtxClear(functions_lib_ctx);
}
loadingSetFlags(NULL, server.repl_transfer_size, asyncLoading);
if (server.repl_diskless_load != REPL_DISKLESS_LOAD_SWAPDB) {
serverLog(LL_NOTICE, "MASTER <-> REPLICA sync: Flushing old data");
/* Note that inside loadingSetFlags(), server.loading is set.
* replicationEmptyDbCallback() may yield back to event-loop to
* reply -LOADING. */
emptyData(-1, empty_db_flags, replicationEmptyDbCallback);
}
loadingFireEvent(RDBFLAGS_REPLICATION);
rioInitWithConn(&rdb,conn,server.repl_transfer_size);
/* Put the socket in blocking mode to simplify RDB transfer.
* We'll restore it when the RDB is received. */
connBlock(conn);
connRecvTimeout(conn, server.repl_timeout*1000);
startLoading(server.repl_transfer_size, RDBFLAGS_REPLICATION, asyncLoading);
int loadingFailed = 0;
rdbLoadingCtx loadingCtx = { .dbarray = dbarray, .functions_lib_ctx = functions_lib_ctx };
......@@ -2120,7 +2117,6 @@ void readSyncBulkPayload(connection *conn) {
}
if (loadingFailed) {
stopLoading(0);
cancelReplicationHandshake(1);
rioFreeConn(&rdb, NULL);
......@@ -2138,6 +2134,11 @@ void readSyncBulkPayload(connection *conn) {
emptyData(-1,empty_db_flags,replicationEmptyDbCallback);
}
/* Note that replicationEmptyDbCallback() may yield back to event
* loop to reply -LOADING if flushing the db takes a long time. So,
* stopLoading() must be called after emptyData() above. */
stopLoading(0);
/* Note that there's no point in restarting the AOF on SYNC
* failure, it'll be restarted when sync succeeds or the replica
* gets promoted. */
......@@ -2213,7 +2214,7 @@ void readSyncBulkPayload(connection *conn) {
return;
}
if (rdbLoad(server.rdb_filename,&rsi,RDBFLAGS_REPLICATION) != RDB_OK) {
if (rdbLoadWithEmptyFunc(server.rdb_filename,&rsi,RDBFLAGS_REPLICATION,rdbLoadEmptyDbFunc) != RDB_OK) {
serverLog(LL_WARNING,
"Failed trying to load the MASTER synchronization "
"DB from disk, check server logs.");
......@@ -2225,9 +2226,6 @@ void readSyncBulkPayload(connection *conn) {
bg_unlink(server.rdb_filename);
}
/* If disk-based RDB loading fails, remove the half-loaded dataset. */
emptyData(-1, empty_db_flags, replicationEmptyDbCallback);
/* Note that there's no point in restarting the AOF on sync failure,
it'll be restarted when sync succeeds or replica promoted. */
return;
......@@ -2281,7 +2279,10 @@ void readSyncBulkPayload(connection *conn) {
/* Restart the AOF subsystem now that we finished the sync. This
* will trigger an AOF rewrite, and when done will start appending
* to the new file. */
if (server.aof_enabled) restartAOFAfterSYNC();
if (server.aof_enabled) {
serverLog(LL_NOTICE, "MASTER <-> REPLICA sync: Starting AOF after a successful sync");
startAppendOnlyWithRetry();
}
return;
error:
......@@ -3098,7 +3099,10 @@ void replicationUnsetMaster(void) {
/* Restart the AOF subsystem in case we shut it down during a sync when
* we were still a slave. */
if (server.aof_enabled && server.aof_state == AOF_OFF) restartAOFAfterSYNC();
if (server.aof_enabled && server.aof_state == AOF_OFF) {
serverLog(LL_NOTICE, "Restarting AOF after becoming master");
startAppendOnlyWithRetry();
}
}
/* This function is called when the slave lose the connection with the
......
......@@ -7212,6 +7212,11 @@ int main(int argc, char **argv) {
loadDataFromDisk();
aofOpenIfNeededOnServerStart();
aofDelHistoryFiles();
/* While loading data, we delay applying "appendonly" config change.
* If there was a config change while we were inside loadDataFromDisk()
* above, we'll apply it here. */
applyAppendOnlyConfig();
if (server.cluster_enabled) {
serverAssert(verifyClusterConfigWithData() == C_OK);
}
......
......@@ -2877,6 +2877,8 @@ const char *getFailoverStateString(void);
/* Generic persistence functions */
void startLoadingFile(size_t size, char* filename, int rdbflags);
void startLoading(size_t size, int rdbflags, int async);
void loadingSetFlags(char *filename, size_t size, int async);
void loadingFireEvent(int rdbflags);
void loadingAbsProgress(off_t pos);
void loadingIncrProgress(off_t size);
void stopLoading(int success);
......@@ -2904,9 +2906,10 @@ int rewriteAppendOnlyFileBackground(void);
int loadAppendOnlyFiles(aofManifest *am);
void stopAppendOnly(void);
int startAppendOnly(void);
void startAppendOnlyWithRetry(void);
void applyAppendOnlyConfig(void);
void backgroundRewriteDoneHandler(int exitcode, int bysignal);
void killAppendOnlyChild(void);
void restartAOFAfterSYNC(void);
void aofLoadManifestFromDisk(void);
void aofOpenIfNeededOnServerStart(void);
void aofManifestFree(aofManifest *am);
......@@ -3399,7 +3402,7 @@ long long emptyDbStructure(redisDb *dbarray, int dbnum, int async, void(callback
void flushAllDataAndResetRDB(int flags);
long long dbTotalServerKeyCount(void);
redisDb *initTempDb(void);
void discardTempDb(redisDb *tempDb, void(callback)(dict*));
void discardTempDb(redisDb *tempDb);
int selectDb(client *c, int id);
......
......@@ -70,6 +70,7 @@ proc test_slave_load_expired_keys {aof} {
# wait for replica to be in sync with master
wait_for_condition 500 10 {
[RI $replica_id master_link_status] eq {up} &&
[R $replica_id dbsize] eq [R $master_id dbsize]
} else {
fail "replica didn't sync"
......@@ -113,6 +114,7 @@ proc test_slave_load_expired_keys {aof} {
# wait for the master to expire all keys and replica to get the DELs
wait_for_condition 500 10 {
[RI $replica_id master_link_status] eq {up} &&
[R $replica_id dbsize] eq $master_dbsize_0
} else {
fail "keys didn't expire"
......
......@@ -137,7 +137,8 @@ test "Restarting primary node" {
test "Instance #0 gets converted into a replica" {
wait_for_condition 1000 50 {
[RI $replica_id role] eq {slave}
[RI $replica_id role] eq {slave} &&
[RI $replica_id master_link_status] eq {up}
} else {
fail "Old primary was not converted into replica"
}
......
......@@ -32,6 +32,17 @@ test "Cluster nodes hard reset" {
} else {
set node_timeout 3000
}
# Wait until slave is synced. Otherwise, it may reply -LOADING
# for any commands below.
if {[RI $id role] eq {slave}} {
wait_for_condition 50 1000 {
[RI $id master_link_status] eq {up}
} else {
fail "Slave were not able to sync."
}
}
catch {R $id flushall} ; # May fail for readonly slaves.
R $id MULTI
R $id cluster reset hard
......
......@@ -655,4 +655,50 @@ tags {"aof external:skip"} {
}
}
}
start_server {overrides {loading-process-events-interval-bytes 1024}} {
test "Allow changing appendonly config while loading from AOF on startup" {
# Set AOF enabled, populate db and restart.
r config set appendonly yes
r config set key-load-delay 100
r config rewrite
populate 10000
restart_server 0 false false
# Disable AOF while loading from the disk.
assert_equal 1 [s loading]
r config set appendonly no
assert_equal 1 [s loading]
# Speed up loading, verify AOF disabled.
r config set key-load-delay 0
wait_done_loading r
assert_equal {10000} [r dbsize]
assert_equal 0 [s aof_enabled]
}
test "Allow changing appendonly config while loading from RDB on startup" {
# Set AOF disabled, populate db and restart.
r flushall
r config set appendonly no
r config set key-load-delay 100
r config rewrite
populate 10000
r save
restart_server 0 false false
# Enable AOF while loading from the disk.
assert_equal 1 [s loading]
r config set appendonly yes
assert_equal 1 [s loading]
# Speed up loading, verify AOF enabled, do a quick sanity.
r config set key-load-delay 0
wait_done_loading r
assert_equal {10000} [r dbsize]
assert_equal 1 [s aof_enabled]
r set t 1
assert_equal {1} [r get t]
}
}
}
......@@ -652,7 +652,6 @@ foreach testType {Successful Aborted} {
}
test {Blocked commands and configs during async-loading} {
assert_error {LOADING*} {$replica config set appendonly no}
assert_error {LOADING*} {$replica REPLICAOF no one}
}
......@@ -1457,3 +1456,143 @@ start_server {tags {"repl external:skip"}} {
}
}
}
foreach disklessload {disabled on-empty-db} {
test "Replica should reply LOADING while flushing a large db (disklessload: $disklessload)" {
start_server {} {
set replica [srv 0 client]
start_server {} {
set master [srv 0 client]
set master_host [srv 0 host]
set master_port [srv 0 port]
$replica config set repl-diskless-load $disklessload
# Populate replica with many keys, master with a few keys.
$replica debug populate 2000000
populate 3 master 10
# Start the replication process...
$replica replicaof $master_host $master_port
wait_for_condition 100 100 {
[s -1 loading] eq 1
} else {
fail "Replica didn't get into loading mode"
}
# If replica has a large db, it may take some time to discard it
# after receiving new db from the master. In this case, replica
# should reply -LOADING. Replica may reply -LOADING while
# loading the new db as well. To test the first case, populated
# replica with large amount of keys and master with a few keys.
# Discarding old db will take a long time and loading new one
# will be quick. So, if we receive -LOADING, most probably it is
# when flushing the db.
wait_for_condition 1 10000 {
[catch {$replica ping} err] &&
[string match *LOADING* $err]
} else {
# There is a chance that we may not catch LOADING response
# if flushing db happens too fast compared to test execution
# Then, we may consider increasing key count or introducing
# artificial delay to db flush.
fail "Replica did not reply LOADING."
}
catch {$replica shutdown nosave}
}
}
} {} {repl external:skip}
}
start_server {tags {"repl external:skip"} overrides {save {}}} {
set master [srv 0 client]
set master_host [srv 0 host]
set master_port [srv 0 port]
populate 10000 master 10
start_server {overrides {save {} rdb-del-sync-files yes loading-process-events-interval-bytes 1024}} {
test "Allow appendonly config change while loading rdb on slave" {
set replica [srv 0 client]
# While loading rdb on slave, verify appendonly config changes are allowed
# 1- Change appendonly config from no to yes
$replica config set appendonly no
$replica config set key-load-delay 100
$replica debug populate 1000
# Start the replication process...
$replica replicaof $master_host $master_port
wait_for_condition 10 1000 {
[s loading] eq 1
} else {
fail "Replica didn't get into loading mode"
}
# Change config while replica is loading data
$replica config set appendonly yes
assert_equal 1 [s loading]
# Speed up loading and verify aof is enabled
$replica config set key-load-delay 0
wait_done_loading $replica
assert_equal 1 [s aof_enabled]
# Quick sanity for AOF
$replica replicaof no one
set prev [s aof_current_size]
$replica set x 100
assert_morethan [s aof_current_size] $prev
# 2- While loading rdb, change appendonly from yes to no
$replica config set appendonly yes
$replica config set key-load-delay 100
$replica flushall
# Start the replication process...
$replica replicaof $master_host $master_port
wait_for_condition 10 1000 {
[s loading] eq 1
} else {
fail "Replica didn't get into loading mode"
}
# Change config while replica is loading data
$replica config set appendonly no
assert_equal 1 [s loading]
# Speed up loading and verify aof is disabled
$replica config set key-load-delay 0
wait_done_loading $replica
assert_equal 0 [s 0 aof_enabled]
}
}
}
start_server {tags {"repl external:skip"}} {
set replica [srv 0 client]
start_server {} {
set master [srv 0 client]
set master_host [srv 0 host]
set master_port [srv 0 port]
test "Replica flushes db lazily when replica-lazy-flush enabled" {
$replica config set replica-lazy-flush yes
$replica debug populate 1000
populate 1 master 10
# Start the replication process...
$replica replicaof $master_host $master_port
wait_for_condition 100 100 {
[s -1 lazyfreed_objects] >= 1000 &&
[s -1 master_link_status] eq {up}
} else {
fail "Replica did not free db lazily"
}
}
}
}
Markdown is supported
0% or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment