diff --git a/src/config.c b/src/config.c index ea733c099b..98414d6451 100644 --- a/src/config.c +++ b/src/config.c @@ -3074,6 +3074,7 @@ standardConfig static_configs[] = { createBoolConfig("lazyfree-lazy-user-flush", NULL, DEBUG_CONFIG | MODIFIABLE_CONFIG, server.lazyfree_lazy_user_flush, 0, NULL, NULL), createBoolConfig("repl-disable-tcp-nodelay", NULL, MODIFIABLE_CONFIG, server.repl_disable_tcp_nodelay, 0, NULL, NULL), createBoolConfig("repl-diskless-sync", NULL, DEBUG_CONFIG | MODIFIABLE_CONFIG, server.repl_diskless_sync, 1, NULL, NULL), + createBoolConfig("dual-channel-replication-enabled", NULL, DEBUG_CONFIG | MODIFIABLE_CONFIG | PROTECTED_CONFIG, server.dual_channel_replication, 0, NULL, NULL), createBoolConfig("aof-rewrite-incremental-fsync", NULL, MODIFIABLE_CONFIG, server.aof_rewrite_incremental_fsync, 1, NULL, NULL), createBoolConfig("no-appendfsync-on-rewrite", NULL, MODIFIABLE_CONFIG, server.aof_no_fsync_on_rewrite, 0, NULL, NULL), createBoolConfig("cluster-require-full-coverage", NULL, MODIFIABLE_CONFIG, server.cluster_require_full_coverage, 1, NULL, NULL), diff --git a/src/debug.c b/src/debug.c index 9501b8a658..09dbfb67eb 100644 --- a/src/debug.c +++ b/src/debug.c @@ -495,6 +495,10 @@ void debugCommand(client *c) { " In case RESET is provided the peak reset time will be restored to the default value", "REPLYBUFFER RESIZING <0|1>", " Enable or disable the reply buffer resize cron job", + "SLEEP-AFTER-FORK-SECONDS ", + " Stop the server's main process for after forking.", + "DELAY-RDB-CLIENT-FREE-SECOND ", + " Grace period in seconds for replica main channel to establish psync.", "DICT-RESIZING <0|1>", " Enable or disable the main dict and expire dict resizing.", NULL}; @@ -991,6 +995,17 @@ void debugCommand(client *c) { return; } addReply(c, shared.ok); + } else if (!strcasecmp(c->argv[1]->ptr, "sleep-after-fork-seconds") && c->argc == 3) { + double sleep_after_fork_seconds; + if (getDoubleFromObjectOrReply(c, c->argv[2], &sleep_after_fork_seconds, NULL) != C_OK) { + addReply(c, shared.err); + return; + } + server.debug_sleep_after_fork_us = (int)(sleep_after_fork_seconds * 1e6); + addReply(c, shared.ok); + } else if (!strcasecmp(c->argv[1]->ptr, "delay-rdb-client-free-seconds") && c->argc == 3) { + server.wait_before_rdb_client_free = atoi(c->argv[2]->ptr); + addReply(c, shared.ok); } else if (!strcasecmp(c->argv[1]->ptr, "dict-resizing") && c->argc == 3) { server.dict_resizing = atoi(c->argv[2]->ptr); addReply(c, shared.ok); diff --git a/src/networking.c b/src/networking.c index 93c9a653da..e27194b96e 100644 --- a/src/networking.c +++ b/src/networking.c @@ -119,6 +119,11 @@ int authRequired(client *c) { return auth_required; } +static inline int isReplicaReadyForReplData(client *replica) { + return (replica->repl_state == REPLICA_STATE_ONLINE || replica->repl_state == REPLICA_STATE_BG_RDB_LOAD) && + !(replica->flag.close_asap); +} + client *createClient(connection *conn) { client *c = zmalloc(sizeof(client)); @@ -189,6 +194,8 @@ client *createClient(connection *conn) { c->replica_version = 0; c->replica_capa = REPLICA_CAPA_NONE; c->replica_req = REPLICA_REQ_NONE; + c->associated_rdb_client_id = 0; + c->rdb_client_disconnect_time = 0; c->reply = listCreate(); c->deferred_reply_errors = NULL; c->reply_bytes = 0; @@ -255,8 +262,8 @@ void putClientInPendingWriteQueue(client *c) { /* Schedule the client to write the output buffers to the socket only * if not already done and, for replicas, if the replica can actually receive * writes at this stage. */ - if (!c->flag.pending_write && (c->repl_state == REPL_STATE_NONE || - (c->repl_state == REPLICA_STATE_ONLINE && !c->repl_start_cmd_stream_on_ack))) { + if (!c->flag.pending_write && + (c->repl_state == REPL_STATE_NONE || (isReplicaReadyForReplData(c) && !c->repl_start_cmd_stream_on_ack))) { /* Here instead of installing the write handler, we just flag the * client and put it into a list of clients that have something * to write to the socket. This way before re-entering the event @@ -1598,7 +1605,7 @@ void freeClient(client *c) { /* If a client is protected, yet we need to free it right now, make sure * to at least use asynchronous freeing. */ - if (c->flag.protected) { + if (c->flag.protected || c->flag.protected_rdb_channel) { freeClientAsync(c); return; } @@ -1644,7 +1651,10 @@ void freeClient(client *c) { /* Log link disconnection with replica */ if (getClientType(c) == CLIENT_TYPE_REPLICA) { - serverLog(LL_NOTICE, "Connection with replica %s lost.", replicationGetReplicaName(c)); + serverLog(LL_NOTICE, + c->flag.repl_rdb_channel ? "Replica %s rdb channel disconnected." + : "Connection with replica %s lost.", + replicationGetReplicaName(c)); } /* Free the query buffer */ @@ -1874,6 +1884,26 @@ int freeClientsInAsyncFreeQueue(void) { while ((ln = listNext(&li)) != NULL) { client *c = listNodeValue(ln); + if (c->flag.protected_rdb_channel) { + /* Check if it's safe to remove RDB connection protection during synchronization + * The primary gives a grace period before freeing this client because + * it serves as a reference to the first required replication data block for + * this replica */ + if (!c->rdb_client_disconnect_time) { + c->rdb_client_disconnect_time = server.unixtime; + serverLog(LL_VERBOSE, "Postpone RDB client id=%llu (%s) free for %d seconds", (unsigned long long)c->id, + replicationGetReplicaName(c), server.wait_before_rdb_client_free); + continue; + } + if (server.unixtime - c->rdb_client_disconnect_time > server.wait_before_rdb_client_free) { + serverLog(LL_NOTICE, + "Replica main channel failed to establish PSYNC within the grace period (%ld seconds). " + "Freeing RDB client %llu.", + (long int)(server.unixtime - c->rdb_client_disconnect_time), (unsigned long long)c->id); + c->flag.protected_rdb_channel = 0; + } + } + if (c->flag.protected) continue; c->flag.close_asap = 0; @@ -2984,6 +3014,10 @@ int processInputBuffer(client *c) { void readToQueryBuf(client *c) { int big_arg = 0; size_t qblen, readlen; + + /* If the replica RDB client is marked as closed ASAP, do not try to read from it */ + if (c->flag.close_asap) return; + int is_primary = c->read_flags & READ_FLAGS_PRIMARY; readlen = PROTO_IOBUF_LEN; @@ -4289,9 +4323,13 @@ int closeClientOnOutputBufferLimitReached(client *c, int async) { serverAssert(c->reply_bytes < SIZE_MAX - (1024 * 64)); /* Note that c->reply_bytes is irrelevant for replica clients * (they use the global repl buffers). */ - if ((c->reply_bytes == 0 && getClientType(c) != CLIENT_TYPE_REPLICA) || c->flag.close_asap) return 0; + if ((c->reply_bytes == 0 && getClientType(c) != CLIENT_TYPE_REPLICA) || + (c->flag.close_asap && !(c->flag.protected_rdb_channel))) + return 0; if (checkClientOutputBufferLimits(c)) { sds client = catClientInfoString(sdsempty(), c); + /* Remove RDB connection protection on COB overrun */ + c->flag.protected_rdb_channel = 0; if (async) { freeClientAsync(c); @@ -4335,7 +4373,7 @@ void flushReplicasOutputBuffers(void) { * * 3. Obviously if the replica is not ONLINE. */ - if (replica->repl_state == REPLICA_STATE_ONLINE && !(replica->flag.close_asap) && can_receive_writes && + if (isReplicaReadyForReplData(replica) && !(replica->flag.close_asap) && can_receive_writes && !replica->repl_start_cmd_stream_on_ack && clientHasPendingReplies(replica)) { writeToClient(replica); } diff --git a/src/rdb.c b/src/rdb.c index f9ccd676fd..38a848a26a 100644 --- a/src/rdb.c +++ b/src/rdb.c @@ -3451,12 +3451,15 @@ static void backgroundSaveDoneHandlerSocket(int exitcode, int bysignal) { serverLog(LL_NOTICE, "Background RDB transfer terminated with success"); } else if (!bysignal && exitcode != 0) { serverLog(LL_WARNING, "Background transfer error"); + server.lastbgsave_status = C_ERR; } else { serverLog(LL_WARNING, "Background transfer terminated by signal %d", bysignal); } if (server.rdb_child_exit_pipe != -1) close(server.rdb_child_exit_pipe); - aeDeleteFileEvent(server.el, server.rdb_pipe_read, AE_READABLE); - close(server.rdb_pipe_read); + if (server.rdb_pipe_read > 0) { + aeDeleteFileEvent(server.el, server.rdb_pipe_read, AE_READABLE); + close(server.rdb_pipe_read); + } server.rdb_child_exit_pipe = -1; server.rdb_pipe_read = -1; zfree(server.rdb_pipe_conns); @@ -3507,43 +3510,65 @@ int rdbSaveToReplicasSockets(int req, rdbSaveInfo *rsi) { listIter li; pid_t childpid; int pipefds[2], rdb_pipe_write, safe_to_exit_pipe; + int dual_channel = (req & REPLICA_REQ_RDB_CHANNEL); if (hasActiveChildProcess()) return C_ERR; + serverAssert(server.rdb_pipe_read == -1 && server.rdb_child_exit_pipe == -1); /* Even if the previous fork child exited, don't start a new one until we * drained the pipe. */ if (server.rdb_pipe_conns) return C_ERR; - /* Before to fork, create a pipe that is used to transfer the rdb bytes to - * the parent, we can't let it write directly to the sockets, since in case - * of TLS we must let the parent handle a continuous TLS state when the - * child terminates and parent takes over. */ - if (anetPipe(pipefds, O_NONBLOCK, 0) == -1) return C_ERR; - server.rdb_pipe_read = pipefds[0]; /* read end */ - rdb_pipe_write = pipefds[1]; /* write end */ - - /* create another pipe that is used by the parent to signal to the child - * that it can exit. */ - if (anetPipe(pipefds, 0, 0) == -1) { - close(rdb_pipe_write); - close(server.rdb_pipe_read); - return C_ERR; + if (!dual_channel) { + /* Before to fork, create a pipe that is used to transfer the rdb bytes to + * the parent, we can't let it write directly to the sockets, since in case + * of TLS we must let the parent handle a continuous TLS state when the + * child terminates and parent takes over. */ + if (anetPipe(pipefds, O_NONBLOCK, 0) == -1) return C_ERR; + server.rdb_pipe_read = pipefds[0]; /* read end */ + rdb_pipe_write = pipefds[1]; /* write end */ + + /* create another pipe that is used by the parent to signal to the child + * that it can exit. */ + if (anetPipe(pipefds, 0, 0) == -1) { + close(rdb_pipe_write); + close(server.rdb_pipe_read); + return C_ERR; + } + safe_to_exit_pipe = pipefds[0]; /* read end */ + server.rdb_child_exit_pipe = pipefds[1]; /* write end */ } - safe_to_exit_pipe = pipefds[0]; /* read end */ - server.rdb_child_exit_pipe = pipefds[1]; /* write end */ - /* Collect the connections of the replicas we want to transfer * the RDB to, which are i WAIT_BGSAVE_START state. */ - server.rdb_pipe_conns = zmalloc(sizeof(connection *) * listLength(server.replicas)); - server.rdb_pipe_numconns = 0; - server.rdb_pipe_numconns_writing = 0; + int connsnum = 0; + connection **conns = zmalloc(sizeof(connection *) * listLength(server.replicas)); + server.rdb_pipe_conns = NULL; + if (!dual_channel) { + server.rdb_pipe_conns = conns; + server.rdb_pipe_numconns = 0; + server.rdb_pipe_numconns_writing = 0; + } + /* Filter replica connections pending full sync (ie. in WAIT_BGSAVE_START state). */ listRewind(server.replicas, &li); while ((ln = listNext(&li))) { client *replica = ln->value; if (replica->repl_state == REPLICA_STATE_WAIT_BGSAVE_START) { /* Check replica has the exact requirements */ if (replica->replica_req != req) continue; - server.rdb_pipe_conns[server.rdb_pipe_numconns++] = replica->conn; + + conns[connsnum++] = replica->conn; + if (dual_channel) { + /* Put the socket in blocking mode to simplify RDB transfer. */ + connBlock(replica->conn); + connSendTimeout(replica->conn, server.repl_timeout * 1000); + /* This replica uses diskless dual channel sync, hence we need + * to inform it with the save end offset.*/ + sendCurrentOffsetToReplica(replica); + /* Make sure repl traffic is appended to the replication backlog */ + addRdbReplicaToPsyncWait(replica); + } else { + server.rdb_pipe_numconns++; + } replicationSetupReplicaForFullResync(replica, getPsyncInitialOffset()); } } @@ -3553,12 +3578,15 @@ int rdbSaveToReplicasSockets(int req, rdbSaveInfo *rsi) { /* Child */ int retval, dummy; rio rdb; - - rioInitWithFd(&rdb, rdb_pipe_write); + if (dual_channel) { + rioInitWithConnset(&rdb, conns, connsnum); + } else { + rioInitWithFd(&rdb, rdb_pipe_write); + } /* Close the reading part, so that if the parent crashes, the child will * get a write error and exit. */ - close(server.rdb_pipe_read); + if (!dual_channel) close(server.rdb_pipe_read); if (strstr(server.exec_argv[0], "redis-server") != NULL) { serverSetProcTitle("redis-rdb-to-slaves"); } else { @@ -3572,14 +3600,18 @@ int rdbSaveToReplicasSockets(int req, rdbSaveInfo *rsi) { if (retval == C_OK) { sendChildCowInfo(CHILD_INFO_TYPE_RDB_COW_SIZE, "RDB"); } - - rioFreeFd(&rdb); + if (dual_channel) { + rioFreeConnset(&rdb); + } else { + rioFreeFd(&rdb); + close(rdb_pipe_write); + } + zfree(conns); /* wake up the reader, tell it we're done. */ - close(rdb_pipe_write); close(server.rdb_child_exit_pipe); /* close write end so that we can detect the close on the parent. */ /* hold exit until the parent tells us it's safe. we're not expecting * to read anything, just get the error when the pipe is closed. */ - dummy = read(safe_to_exit_pipe, pipefds, 1); + if (!dual_channel) dummy = read(safe_to_exit_pipe, pipefds, 1); UNUSED(dummy); exitFromChild((retval == C_OK) ? 0 : 1); } else { @@ -3597,23 +3629,36 @@ int rdbSaveToReplicasSockets(int req, rdbSaveInfo *rsi) { replica->repl_state = REPLICA_STATE_WAIT_BGSAVE_START; } } - close(rdb_pipe_write); - close(server.rdb_pipe_read); + if (!dual_channel) { + close(rdb_pipe_write); + close(server.rdb_pipe_read); + } close(server.rdb_child_exit_pipe); - zfree(server.rdb_pipe_conns); - server.rdb_pipe_conns = NULL; - server.rdb_pipe_numconns = 0; - server.rdb_pipe_numconns_writing = 0; + zfree(conns); + if (dual_channel) { + closeChildInfoPipe(); + } else { + server.rdb_pipe_conns = NULL; + server.rdb_pipe_numconns = 0; + server.rdb_pipe_numconns_writing = 0; + } } else { - serverLog(LL_NOTICE, "Background RDB transfer started by pid %ld", (long)childpid); + serverLog(LL_NOTICE, "Background RDB transfer started by pid %ld to %s", (long)childpid, + dual_channel ? "direct socket to replica" : "pipe through parent process"); server.rdb_save_time_start = time(NULL); server.rdb_child_type = RDB_CHILD_TYPE_SOCKET; - close(rdb_pipe_write); /* close write in parent so that it can detect the close on the child. */ - if (aeCreateFileEvent(server.el, server.rdb_pipe_read, AE_READABLE, rdbPipeReadHandler, NULL) == AE_ERR) { - serverPanic("Unrecoverable error creating server.rdb_pipe_read file event."); + if (dual_channel) { + /* For dual channel sync, the main process no longer requires these RDB connections. */ + zfree(conns); + } else { + close(rdb_pipe_write); /* close write in parent so that it can detect the close on the child. */ + if (aeCreateFileEvent(server.el, server.rdb_pipe_read, AE_READABLE, rdbPipeReadHandler, NULL) == + AE_ERR) { + serverPanic("Unrecoverable error creating server.rdb_pipe_read file event."); + } } } - close(safe_to_exit_pipe); + if (!dual_channel) close(safe_to_exit_pipe); return (childpid == -1) ? C_ERR : C_OK; } return C_OK; /* Unreached. */ diff --git a/src/replication.c b/src/replication.c index 21ccb0e92d..e0beb1b1bd 100644 --- a/src/replication.c +++ b/src/replication.c @@ -42,13 +42,18 @@ #include #include #include +#include void replicationDiscardCachedPrimary(void); void replicationResurrectCachedPrimary(connection *conn); +void replicationResurrectProvisionalPrimary(void); void replicationSendAck(void); int replicaPutOnline(client *replica); void replicaStartCommandStream(client *replica); int cancelReplicationHandshake(int reconnect); +void replicationSteadyStateInit(void); +void setupMainConnForPsync(connection *conn); +void dualChannelSyncHandleRdbLoadCompletion(void); /* We take a global flag to remember if this instance generated an RDB * because of replication, so that we can remove the RDB file in case @@ -190,6 +195,88 @@ void rebaseReplicationBuffer(long long base_repl_offset) { } } +/* Return a client by ID, or NULL if the client ID is not in the set + * of replicas waiting psync clients. */ +static inline client *lookupRdbClientByID(uint64_t id) { + id = htonu64(id); + void *c = NULL; + raxFind(server.replicas_waiting_psync, (unsigned char *)&id, sizeof(id), &c); + return c; +} + +/* Replication: Primary side - connections association. + * During dual channel sync, association is used to keep replication data + * in the backlog until the replica requests PSYNC. + * Association occurs in two forms: + * 1. If there's an existing buffer block at fork time, the replica is attached to the tail. + * 2. If there's no tail, the replica is attached when a new buffer block is created + * (see the Retrospect function below). + * The replica RDB client ID is used as a unique key for this association. + * If a COB overrun occurs, the association is deleted and the RDB connection is dropped. */ +void addRdbReplicaToPsyncWait(client *replica_rdb_client) { + listNode *ln = NULL; + replBufBlock *tail = NULL; + if (server.repl_backlog == NULL) { + createReplicationBacklog(); + } else { + ln = listLast(server.repl_buffer_blocks); + tail = ln ? listNodeValue(ln) : NULL; + if (tail) { + tail->refcount++; + } + } + serverLog(LL_DEBUG, "Add rdb replica %s to waiting psync, with cid %llu, %s ", + replicationGetReplicaName(replica_rdb_client), (unsigned long long)replica_rdb_client->id, + tail ? "tracking repl-backlog tail" : "no repl-backlog to track"); + replica_rdb_client->ref_repl_buf_node = tail ? ln : NULL; + /* Prevent rdb client from being freed before psync is established. */ + replica_rdb_client->flag.protected_rdb_channel = 1; + uint64_t id = htonu64(replica_rdb_client->id); + raxInsert(server.replicas_waiting_psync, (unsigned char *)&id, sizeof(id), replica_rdb_client, NULL); +} + +/* Attach waiting psync replicas with new replication backlog head. */ +void backfillRdbReplicasToPsyncWait(void) { + listNode *ln = listFirst(server.repl_buffer_blocks); + replBufBlock *head = ln ? listNodeValue(ln) : NULL; + raxIterator iter; + + if (head == NULL) return; + /* Update waiting psync replicas to wait on new buffer block */ + raxStart(&iter, server.replicas_waiting_psync); + raxSeek(&iter, "^", NULL, 0); + while (raxNext(&iter)) { + client *replica_rdb_client = iter.data; + if (replica_rdb_client->ref_repl_buf_node) continue; + replica_rdb_client->ref_repl_buf_node = ln; + head->refcount++; + serverLog(LL_DEBUG, "Attach replica rdb client %llu to repl buf block", + (long long unsigned int)replica_rdb_client->id); + } + raxStop(&iter); +} + +void removeReplicaFromPsyncWait(client *replica_main_client) { + listNode *ln; + replBufBlock *o; + /* Get replBufBlock pointed by this replica */ + client *replica_rdb_client = lookupRdbClientByID(replica_main_client->associated_rdb_client_id); + ln = replica_rdb_client->ref_repl_buf_node; + o = ln ? listNodeValue(ln) : NULL; + if (o != NULL) { + serverAssert(o->refcount > 0); + o->refcount--; + } + replica_rdb_client->ref_repl_buf_node = NULL; + replica_rdb_client->flag.protected_rdb_channel = 0; + serverLog(LL_DEBUG, "Remove psync waiting replica %s with cid %llu, repl buffer block %s", + replicationGetReplicaName(replica_main_client), + (long long unsigned int)replica_main_client->associated_rdb_client_id, + o ? "ref count decreased" : "doesn't exist"); + uint64_t id = htonu64(replica_rdb_client->id); + raxRemove(server.replicas_waiting_psync, (unsigned char *)&id, sizeof(id), NULL); +} + void resetReplicationBuffer(void) { server.repl_buffer_mem = 0; server.repl_buffer_blocks = listCreate(); @@ -299,6 +386,13 @@ void incrementalTrimReplicationBacklog(size_t max_blocks) { /* Free replication buffer blocks that are referenced by this client. */ void freeReplicaReferencedReplBuffer(client *replica) { + if (replica->flag.repl_rdb_channel) { + uint64_t rdb_cid = htonu64(replica->id); + if (raxRemove(server.replicas_waiting_psync, (unsigned char *)&rdb_cid, sizeof(rdb_cid), NULL)) { + serverLog(LL_DEBUG, "Remove psync waiting replica %s with cid %llu from replicas rax.", + replicationGetReplicaName(replica), (long long unsigned int)replica->associated_rdb_client_id); + } + } if (replica->ref_repl_buf_node != NULL) { /* Decrease the start buffer node reference count. */ replBufBlock *o = listNodeValue(replica->ref_repl_buf_node); @@ -310,7 +404,8 @@ void freeReplicaReferencedReplBuffer(client *replica) { replica->ref_block_pos = 0; } -/* Append bytes into the global replication buffer list, replication backlog and +/* Replication: Primary side. + * Append bytes into the global replication buffer list, replication backlog and * all replica clients use replication buffers collectively, this function replace * 'addReply*', 'feedReplicationBacklog' for replicas and replication backlog, * First we add buffer into global replication buffer block list, and then @@ -326,6 +421,7 @@ void feedReplicationBuffer(char *s, size_t len) { int add_new_block = 0; /* Create new block if current block is total used. */ listNode *ln = listLast(server.repl_buffer_blocks); replBufBlock *tail = ln ? listNodeValue(ln) : NULL; + int empty_backlog = (tail == NULL); /* Append to tail string when possible. */ if (tail && tail->size > tail->used) { @@ -373,14 +469,17 @@ void feedReplicationBuffer(char *s, size_t len) { server.primary_repl_offset += copy; server.repl_backlog->histlen += copy; } + if (empty_backlog && raxSize(server.replicas_waiting_psync) > 0) { + /* Increase refcount for pending replicas. */ + backfillRdbReplicasToPsyncWait(); + } /* For output buffer of replicas. */ listIter li; listRewind(server.replicas, &li); while ((ln = listNext(&li))) { client *replica = ln->value; - if (!canFeedReplicaReplBuffer(replica)) continue; - + if (!canFeedReplicaReplBuffer(replica) && !(replica->flag.protected_rdb_channel)) continue; /* Update shared replication buffer start position. */ if (replica->ref_repl_buf_node == NULL) { replica->ref_repl_buf_node = start_node; @@ -405,7 +504,6 @@ void feedReplicationBuffer(char *s, size_t len) { } if (add_new_block) { createReplicationBacklogIndex(listLast(server.repl_buffer_blocks)); - /* It is important to trim after adding replication data to keep the backlog size close to * repl_backlog_size in the common case. We wait until we add a new block to avoid repeated * unnecessary trimming attempts when small amounts of data are added. See comments in @@ -764,14 +862,22 @@ int primaryTryPartialResynchronization(client *c, long long psync_offset) { goto need_full_resync; } - /* If we reached this point, we are able to perform a partial resync: + /* There are two scenarios that lead to this point. One is that we are able + * to perform a partial resync with the replica. The second is that the replica + * is using dual-channel-replication, while loading the snapshot in the background. + * in both cases: * 1) Make sure no IO operations are being performed before changing the client state. * 2) Set client state to make it a replica. * 3) Inform the client we can continue with +CONTINUE * 4) Send the backlog data (from the offset to the end) to the replica. */ waitForClientIO(c); c->flag.replica = 1; - c->repl_state = REPLICA_STATE_ONLINE; + if (c->associated_rdb_client_id && lookupRdbClientByID(c->associated_rdb_client_id)) { + c->repl_state = REPLICA_STATE_BG_RDB_LOAD; + removeReplicaFromPsyncWait(c); + } else { + c->repl_state = REPLICA_STATE_ONLINE; + } c->repl_ack_time = server.unixtime; c->repl_start_cmd_stream_on_ack = 0; listAddNodeTail(server.replicas, c); @@ -855,6 +961,7 @@ int startBgsaveForReplication(int mincapa, int req) { /* Keep the page cache since it'll get used soon */ retval = rdbSaveBackground(req, server.rdb_filename, rsiptr, RDBFLAGS_REPLICATION | RDBFLAGS_KEEP_CACHE); } + if (server.debug_sleep_after_fork_us) usleep(server.debug_sleep_after_fork_us); } else { serverLog(LL_WARNING, "BGSAVE for replication: replication information not available, can't generate the RDB " "file right now. Try later."); @@ -995,6 +1102,17 @@ void syncCommand(client *c) { * resync on purpose when they are not able to partially * resync. */ if (primary_replid[0] != '?') server.stat_sync_partial_err++; + if (c->replica_capa & REPLICA_CAPA_DUAL_CHANNEL) { + serverLog(LL_NOTICE, + "Replica %s is capable of dual channel synchronization, and partial sync isn't possible. " + "Full sync will continue with dedicated RDB channel.", + replicationGetReplicaName(c)); + const char *buf = "+DUALCHANNELSYNC\r\n"; + if (connWrite(c->conn, buf, strlen(buf)) != (int)strlen(buf)) { + freeClientAsync(c); + } + return; + } } } else { /* If a replica uses SYNC, we are dealing with an old implementation @@ -1107,10 +1225,11 @@ void syncCommand(client *c) { * the primary can accurately lists replicas and their listening ports in the * INFO output. * - * - capa + * - capa * What is the capabilities of this instance. * eof: supports EOF-style RDB transfer for diskless replication. * psync2: supports PSYNC v2, so understands +CONTINUE . + * dual-channel: supports full sync using rdb channel. * * - ack [fack ] * Replica informs the primary the amount of replication stream that it @@ -1130,7 +1249,12 @@ void syncCommand(client *c) { * result in an empty RDB. * * - version - * The replica reports its version. */ + * The replica reports its version. + * + * - rdb-channel <1|0> + * Used to identify the client as a replica's rdb connection in an dual channel + * sync session. + * */ void replconfCommand(client *c) { int j; @@ -1166,6 +1290,12 @@ void replconfCommand(client *c) { c->replica_capa |= REPLICA_CAPA_EOF; else if (!strcasecmp(c->argv[j + 1]->ptr, "psync2")) c->replica_capa |= REPLICA_CAPA_PSYNC2; + else if (!strcasecmp(c->argv[j + 1]->ptr, "dual-channel") && server.dual_channel_replication && + server.repl_diskless_sync) { + /* If dual-channel is disable on this primary, treat this command as unrecognized + * replconf option. */ + c->replica_capa |= REPLICA_CAPA_DUAL_CHANNEL; + } } else if (!strcasecmp(c->argv[j]->ptr, "ack")) { /* REPLCONF ACK is used by replica to inform the primary the amount * of replication stream that it processed so far. It is an @@ -1191,6 +1321,9 @@ void replconfCommand(client *c) { if (server.child_type == CHILD_TYPE_RDB && c->repl_state == REPLICA_STATE_WAIT_BGSAVE_END) checkChildrenDone(); if (c->repl_start_cmd_stream_on_ack && c->repl_state == REPLICA_STATE_ONLINE) replicaStartCommandStream(c); + if (c->repl_state == REPLICA_STATE_BG_RDB_LOAD) { + replicaPutOnline(c); + } /* Note: this command does not reply anything! */ return; } else if (!strcasecmp(c->argv[j]->ptr, "getack")) { @@ -1243,6 +1376,30 @@ void replconfCommand(client *c) { addReplyErrorFormat(c, "Unrecognized version format: %s", (char *)c->argv[j + 1]->ptr); return; } + } else if (!strcasecmp(c->argv[j]->ptr, "rdb-channel")) { + long start_with_offset = 0; + if (getRangeLongFromObjectOrReply(c, c->argv[j + 1], 0, 1, &start_with_offset, NULL) != C_OK) { + return; + } + if (start_with_offset == 1) { + c->flag.repl_rdb_channel = 1; + c->replica_req |= REPLICA_REQ_RDB_CHANNEL; + } else { + c->flag.repl_rdb_channel = 0; + c->replica_req &= ~REPLICA_REQ_RDB_CHANNEL; + } + } else if (!strcasecmp(c->argv[j]->ptr, "set-rdb-client-id")) { + /* REPLCONF identify is used to identify the current replica main channel with existing + * rdb-connection with the given id. */ + long long client_id = 0; + if (getLongLongFromObjectOrReply(c, c->argv[j + 1], &client_id, NULL) != C_OK) { + return; + } + if (!lookupRdbClientByID(client_id)) { + addReplyErrorFormat(c, "Unrecognized RDB client id %lld", client_id); + return; + } + c->associated_rdb_client_id = (uint64_t)client_id; } else { addReplyErrorFormat(c, "Unrecognized REPLCONF option: %s", (char *)c->argv[j]->ptr); return; @@ -1710,9 +1867,9 @@ void replicationEmptyDbCallback(dict *d) { /* Once we have a link with the primary and the synchronization was * performed, this function materializes the primary client we store * at server.primary, starting from the specified file descriptor. */ -void replicationCreatePrimaryClient(connection *conn, int dbid) { +void replicationCreatePrimaryClientWithHandler(connection *conn, int dbid, ConnectionCallbackFunc handler) { server.primary = createClient(conn); - if (conn) connSetReadHandler(server.primary->conn, readQueryFromClient); + if (conn) connSetReadHandler(server.primary->conn, handler); /** * Important note: @@ -1741,6 +1898,12 @@ void replicationCreatePrimaryClient(connection *conn, int dbid) { if (dbid != -1) selectDb(server.primary, dbid); } +/* Wrapper for replicationCreatePrimaryClientWithHandler, init primary connection handler + * with ordinary client connection handler */ +void replicationCreatePrimaryClient(connection *conn, int dbid) { + replicationCreatePrimaryClientWithHandler(conn, dbid, readQueryFromClient); +} + /* This function will try to re-enable the AOF file after the * primary-replica synchronization: if it fails after multiple attempts * the replica cannot be considered reliable and exists with an @@ -2186,18 +2349,25 @@ void readSyncBulkPayload(connection *conn) { } /* Final setup of the connected replica <- primary link */ - replicationCreatePrimaryClient(server.repl_transfer_s, rsi.repl_stream_db); - server.repl_state = REPL_STATE_CONNECTED; + if (conn == server.repl_rdb_transfer_s) { + dualChannelSyncHandleRdbLoadCompletion(); + } else { + replicationCreatePrimaryClient(server.repl_transfer_s, rsi.repl_stream_db); + server.repl_state = REPL_STATE_CONNECTED; + /* Send the initial ACK immediately to put this replica in online state. */ + replicationSendAck(); + } server.repl_down_since = 0; /* Fire the primary link modules event. */ moduleFireServerEvent(VALKEYMODULE_EVENT_PRIMARY_LINK_CHANGE, VALKEYMODULE_SUBEVENT_PRIMARY_LINK_UP, NULL); - - /* After a full resynchronization we use the replication ID and - * offset of the primary. The secondary ID / offset are cleared since - * we are starting a new history. */ - memcpy(server.replid, server.primary->replid, sizeof(server.replid)); - server.primary_repl_offset = server.primary->reploff; + if (server.repl_state == REPL_STATE_CONNECTED) { + /* After a full resynchronization we use the replication ID and + * offset of the primary. The secondary ID / offset are cleared since + * we are starting a new history. */ + memcpy(server.replid, server.primary->replid, sizeof(server.replid)); + server.primary_repl_offset = server.primary->reploff; + } clearReplicationId2(); /* Let's create the replication backlog if needed. Replicas need to @@ -2212,13 +2382,17 @@ void readSyncBulkPayload(connection *conn) { "in read-write mode.\n"); } - /* Send the initial ACK immediately to put this replica in online state. */ - if (usemark) replicationSendAck(); - /* 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(); + + /* In case of dual channel replication sync we want to close the RDB connection + * once the connection is established */ + if (conn == server.repl_rdb_transfer_s) { + connClose(conn); + server.repl_rdb_transfer_s = NULL; + } return; error: @@ -2313,6 +2487,431 @@ char *sendCommandArgv(connection *conn, int argc, char **argv, size_t *argv_lens return NULL; } +/* Replication: Replica side. + * Returns an sds represent this replica port to be used by the primary (mostly + * for logs) */ +sds getReplicaPortString(void) { + long long replica_port; + if (server.replica_announce_port) { + replica_port = server.replica_announce_port; + } else if (server.tls_replication && server.tls_port) { + replica_port = server.tls_port; + } else { + replica_port = server.port; + } + return sdsfromlonglong(replica_port); +} + +/* Replication: Replica side. + * Free replica's local replication buffer */ +void freePendingReplDataBuf(void) { + listRelease(server.pending_repl_data.blocks); + server.pending_repl_data.blocks = NULL; + server.pending_repl_data.len = 0; +} + +/* Replication: Replica side. + * Upon dual-channel sync failure, close rdb-connection, reset repl-state, reset + * provisional primary struct, and free local replication buffer. */ +void replicationAbortDualChannelSyncTransfer(void) { + serverAssert(server.repl_rdb_channel_state != REPL_DUAL_CHANNEL_STATE_NONE); + serverLog(LL_NOTICE, "Aborting dual channel sync"); + if (server.repl_rdb_transfer_s) { + connClose(server.repl_rdb_transfer_s); + server.repl_rdb_transfer_s = NULL; + } + zfree(server.repl_transfer_tmpfile); + server.repl_transfer_tmpfile = NULL; + if (server.repl_transfer_fd != -1) { + close(server.repl_transfer_fd); + server.repl_transfer_fd = -1; + } + server.repl_rdb_channel_state = REPL_DUAL_CHANNEL_STATE_NONE; + server.repl_provisional_primary.read_reploff = 0; + server.repl_provisional_primary.reploff = 0; + server.repl_provisional_primary.conn = NULL; + server.repl_provisional_primary.dbid = -1; + server.rdb_client_id = -1; + freePendingReplDataBuf(); + return; +} + +/* Replication: Primary side. + * Send current replication offset to replica. Use the following structure: + * $ENDOFF: */ +int sendCurrentOffsetToReplica(client *replica) { + char buf[128]; + int buflen; + buflen = snprintf(buf, sizeof(buf), "$ENDOFF:%lld %s %d %llu\r\n", server.primary_repl_offset, server.replid, + server.db->id, (long long unsigned int)replica->id); + serverLog(LL_NOTICE, "Sending to replica %s RDB end offset %lld and client-id %llu", + replicationGetReplicaName(replica), server.primary_repl_offset, (long long unsigned int)replica->id); + if (connSyncWrite(replica->conn, buf, buflen, server.repl_syncio_timeout * 1000) != buflen) { + freeClientAsync(replica); + return C_ERR; + } + return C_OK; +} + +/* Replication: Replica side. + * This connection handler is used to initialize the RDB connection (dual-channel-replication). + * Once a replica with dual-channel-replication enabled, denied from PSYNC with its primary, + * fullSyncWithPrimary begins its role. The connection handler prepares server.repl_rdb_transfer_s + * for a rdb stream, and server.repl_transfer_s for increamental replication data stream. */ +static void fullSyncWithPrimary(connection *conn) { + char *err = NULL; + serverAssert(conn == server.repl_rdb_transfer_s); + /* If this event fired after the user turned the instance into a primary + * with REPLICAOF NO ONE we must just return ASAP. */ + if (server.repl_state == REPL_STATE_NONE) { + goto error; + } + /* Check for errors in the socket: after a non blocking connect() we + * may find that the socket is in error state. */ + if (connGetState(conn) != CONN_STATE_CONNECTED) { + serverLog(LL_WARNING, "Error condition on socket for dual channel replication: %s", connGetLastError(conn)); + goto error; + } + /* Send replica capabilities */ + if (server.repl_rdb_channel_state == REPL_DUAL_CHANNEL_SEND_HANDSHAKE) { + serverLog(LL_DEBUG, "Received first reply from primary using rdb connection."); + /* AUTH with the primary if required. */ + if (server.primary_auth) { + char *args[] = {"AUTH", NULL, NULL}; + size_t lens[] = {4, 0, 0}; + int argc = 1; + if (server.primary_user) { + args[argc] = server.primary_user; + lens[argc] = strlen(server.primary_user); + argc++; + } + args[argc] = server.primary_auth; + lens[argc] = sdslen(server.primary_auth); + argc++; + err = sendCommandArgv(conn, argc, args, lens); + if (err) { + serverLog(LL_WARNING, "Sending command to primary in dual channel replication handshake: %s", err); + return; + } + } + /* Send replica lisening port to primary for clarification */ + sds portstr = getReplicaPortString(); + err = sendCommand(conn, "REPLCONF", "capa", "eof", "rdb-only", "1", "rdb-channel", "1", "listening-port", + portstr, NULL); + sdsfree(portstr); + if (err) { + serverLog(LL_WARNING, "Sending command to primary in dual channel replication handshake: %s", err); + return; + } + server.repl_rdb_channel_state = REPL_DUAL_CHANNEL_RECEIVE_AUTH_REPLY; + + if (connSetReadHandler(conn, fullSyncWithPrimary) == C_ERR) { + char conninfo[CONN_INFO_LEN]; + serverLog(LL_WARNING, "Can't create readable event for SYNC: %s (%s)", strerror(errno), + connGetInfo(conn, conninfo, sizeof(conninfo))); + goto error; + } + return; + } + if (server.repl_rdb_channel_state == REPL_DUAL_CHANNEL_RECEIVE_AUTH_REPLY && !server.primary_auth) { + server.repl_rdb_channel_state = REPL_DUAL_CHANNEL_RECEIVE_REPLCONF_REPLY; + } + /* Receive AUTH reply. */ + if (server.repl_rdb_channel_state == REPL_DUAL_CHANNEL_RECEIVE_AUTH_REPLY) { + err = receiveSynchronousResponse(conn); + if (err == NULL) { + serverLog(LL_WARNING, "Primary did not respond to auth command during SYNC handshake"); + goto error; + } + if (err[0] == '-') { + serverLog(LL_WARNING, "Unable to AUTH to Primary: %s", err); + goto error; + } + sdsfree(err); + server.repl_rdb_channel_state = REPL_DUAL_CHANNEL_RECEIVE_REPLCONF_REPLY; + return; + } + /* Receive replconf response */ + if (server.repl_rdb_channel_state == REPL_DUAL_CHANNEL_RECEIVE_REPLCONF_REPLY) { + err = receiveSynchronousResponse(conn); + if (err == NULL) { + serverLog(LL_WARNING, "Primary did not respond to replconf command during SYNC handshake"); + goto error; + } + + if (err[0] == '-') { + serverLog(LL_NOTICE, + "Server does not support sync with offset, dual channel sync approach cannot be used: %s", err); + goto error; + } + if (connSyncWrite(conn, "SYNC\r\n", 6, server.repl_syncio_timeout * 1000) == -1) { + serverLog(LL_WARNING, "I/O error writing to Primary: %s", connGetLastError(conn)); + goto error; + } + sdsfree(err); + server.repl_rdb_channel_state = REPL_DUAL_CHANNEL_RECEIVE_ENDOFF; + return; + } + /* Receive end offset response */ + if (server.repl_rdb_channel_state == REPL_DUAL_CHANNEL_RECEIVE_ENDOFF) { + int64_t rdb_client_id; + err = receiveSynchronousResponse(conn); + if (err == NULL) goto error; + if (err[0] == '\0') { + /* Retry again later */ + serverLog(LL_DEBUG, "Received empty $ENDOFF response"); + sdsfree(err); + return; + } + long long reploffset; + char primary_replid[CONFIG_RUN_ID_SIZE + 1]; + int dbid; + /* Parse end offset response */ + char *endoff_format = "$ENDOFF:%lld %40s %d %ld"; + if (sscanf(err, endoff_format, &reploffset, primary_replid, &dbid, &rdb_client_id) != 4) { + goto error; + } + sdsfree(err); + server.rdb_client_id = rdb_client_id; + server.primary_initial_offset = reploffset; + + /* Initiate repl_provisional_primary to act as this replica temp primary until RDB is loaded */ + server.repl_provisional_primary.conn = server.repl_transfer_s; + memcpy(server.repl_provisional_primary.replid, primary_replid, CONFIG_RUN_ID_SIZE); + server.repl_provisional_primary.reploff = reploffset; + server.repl_provisional_primary.read_reploff = reploffset; + server.repl_provisional_primary.dbid = dbid; + + /* Now that we have the snapshot end-offset, we can ask for psync from that offset. Prepare the + * main connection accordingly.*/ + server.repl_transfer_s->state = CONN_STATE_CONNECTED; + server.repl_state = REPL_STATE_SEND_HANDSHAKE; + serverAssert(connSetReadHandler(server.repl_transfer_s, setupMainConnForPsync) != C_ERR); + setupMainConnForPsync(server.repl_transfer_s); + + /* As the next block we will receive using this connection is the rdb, we need to prepare + * the connection accordingly */ + serverAssert(connSetReadHandler(server.repl_rdb_transfer_s, readSyncBulkPayload) != C_ERR); + server.repl_transfer_size = -1; + server.repl_transfer_read = 0; + server.repl_transfer_last_fsync_off = 0; + server.repl_transfer_lastio = server.unixtime; + + server.repl_rdb_channel_state = REPL_DUAL_CHANNEL_RDB_LOAD; + return; + } + +error: + sdsfree(err); + connClose(conn); + server.repl_transfer_s = NULL; + if (server.repl_rdb_transfer_s) { + connClose(server.repl_rdb_transfer_s); + server.repl_rdb_transfer_s = NULL; + } + if (server.repl_transfer_fd != -1) close(server.repl_transfer_fd); + server.repl_transfer_fd = -1; + server.repl_state = REPL_STATE_CONNECT; + replicationAbortDualChannelSyncTransfer(); + return; +} + +/* Replication: Replica side. + * Initialize server.pending_repl_data infrastructure, we will allocate the buffer + * itself once we need it */ +void replDataBufInit(void) { + serverAssert(server.pending_repl_data.blocks == NULL); + server.pending_repl_data.len = 0; + server.pending_repl_data.peak = 0; + server.pending_repl_data.blocks = listCreate(); + server.pending_repl_data.blocks->free = zfree; +} + +/* Replication: Replica side. + * Track the local repl-data buffer streaming progress and serve clients from time to time */ +void replStreamProgressCallback(size_t offset, int readlen, time_t *last_progress_callback) { + time_t now = mstime(); + if (server.loading_process_events_interval_bytes && + ((offset + readlen) / server.loading_process_events_interval_bytes > + offset / server.loading_process_events_interval_bytes) && + (now - *last_progress_callback > server.loading_process_events_interval_ms)) { + replicationSendNewlineToPrimary(); + processEventsWhileBlocked(); + *last_progress_callback = now; + } +} + +/* Link list block, used by replDataBuf during dual-channel-replication to store + * replication data */ +typedef struct replDataBufBlock { + size_t size, used; + char buf[]; +} replDataBufBlock; + +/* Replication: Replica side. + * Reads replication data from primary into specified repl buffer block */ +int readIntoReplDataBlock(connection *conn, replDataBufBlock *data_block, size_t read) { + int nread = connRead(conn, data_block->buf + data_block->used, read); + if (nread == -1) { + if (connGetState(conn) != CONN_STATE_CONNECTED) { + serverLog(LL_NOTICE, "Error reading from primary: %s", connGetLastError(conn)); + cancelReplicationHandshake(1); + } + return C_ERR; + } + if (nread == 0) { + serverLog(LL_VERBOSE, "Provisional primary closed connection"); + cancelReplicationHandshake(1); + return C_ERR; + } + data_block->used += nread; + server.stat_total_reads_processed++; + return read - nread; +} + +/* Replication: Replica side. + * Read handler for buffering incoming repl data during RDB download/loading. */ +void bufferReplData(connection *conn) { + size_t readlen = PROTO_IOBUF_LEN; + int remaining_bytes = 0; + + while (readlen > 0) { + listNode *ln = listLast(server.pending_repl_data.blocks); + replDataBufBlock *tail = ln ? listNodeValue(ln) : NULL; + + /* Append to tail string when possible */ + if (tail && tail->used < tail->size) { + size_t avail = tail->size - tail->used; + remaining_bytes = min(readlen, avail); + readlen -= remaining_bytes; + remaining_bytes = readIntoReplDataBlock(conn, tail, remaining_bytes); + } + if (readlen && remaining_bytes == 0) { + if (server.pending_repl_data.len > server.client_obuf_limits[CLIENT_TYPE_REPLICA].hard_limit_bytes) { + serverLog(LL_NOTICE, "Replication buffer limit reached, stopping buffering."); + /* Stop accumulating primary commands. */ + connSetReadHandler(conn, NULL); + break; + } + /* Create a new node, make sure it is allocated to at least PROTO_REPLY_CHUNK_BYTES. + * Use the same upper boundary as the shared replication buffer (feedReplicationBuffer), + * as they share the same purpose */ + size_t usable_size; + size_t limit = max((size_t)server.repl_backlog_size / 16, (size_t)PROTO_REPLY_CHUNK_BYTES); + size_t size = min(max(readlen, (size_t)PROTO_REPLY_CHUNK_BYTES), limit); + tail = zmalloc_usable(size + sizeof(replDataBufBlock), &usable_size); + tail->size = usable_size - sizeof(replDataBufBlock); + tail->used = 0; + listAddNodeTail(server.pending_repl_data.blocks, tail); + server.pending_repl_data.len += tail->size; + /* Update buffer's peak */ + if (server.pending_repl_data.peak < server.pending_repl_data.len) + server.pending_repl_data.peak = server.pending_repl_data.len; + + remaining_bytes = min(readlen, tail->size); + readlen -= remaining_bytes; + remaining_bytes = readIntoReplDataBlock(conn, tail, remaining_bytes); + } + if (remaining_bytes > 0) { + /* Stop reading in case we read less than we anticipated */ + break; + } + if (remaining_bytes == C_ERR) { + return; + } + } +} + +/* Replication: Replica side. + * Streams accumulated replication data into the database while freeing read nodes */ +int streamReplDataBufToDb(client *c) { + serverAssert(c->flag.primary); + blockingOperationStarts(); + size_t used, offset = 0; + listNode *cur = NULL; + time_t last_progress_callback = mstime(); + while (server.pending_repl_data.blocks && (cur = listFirst(server.pending_repl_data.blocks))) { + /* Read and process repl data block */ + replDataBufBlock *o = listNodeValue(cur); + used = o->used; + c->querybuf = sdscatlen(c->querybuf, o->buf, used); + c->read_reploff += used; + processInputBuffer(c); + server.pending_repl_data.len -= used; + offset += used; + listDelNode(server.pending_repl_data.blocks, cur); + replStreamProgressCallback(offset, used, &last_progress_callback); + } + blockingOperationEnds(); + if (!server.pending_repl_data.blocks) { + /* If we encounter a `replicaof` command during the replStreamProgressCallback, + * pending_repl_data.blocks will be NULL, and we should return an error and + * abort the current sync session. */ + return C_ERR; + } + return C_OK; +} + +/* Replication: Replica side. + * After done loading the snapshot using the rdb-channel prepare this replica for steady state by + * initializing the primary client, amd stream local increamental buffer into memory. */ +void dualChannelSyncSuccess(void) { + server.primary_initial_offset = server.repl_provisional_primary.reploff; + replicationResurrectProvisionalPrimary(); + /* Wait for the accumulated buffer to be processed before reading any more replication updates */ + if (streamReplDataBufToDb(server.primary) == C_ERR) { + /* Sync session aborted during repl data streaming. */ + serverLog(LL_WARNING, "Failed to stream local replication buffer into memory"); + return; + } + freePendingReplDataBuf(); + serverLog(LL_NOTICE, "Successfully streamed replication data into memory"); + /* We can resume reading from the primary connection once the local replication buffer has been loaded. */ + replicationSteadyStateInit(); + replicationSendAck(); /* Send ACK to notify primary that replica is synced */ + server.rdb_client_id = -1; + server.repl_rdb_channel_state = REPL_DUAL_CHANNEL_STATE_NONE; +} + +/* Replication: Replica side. + * Main channel successfully established psync with primary. Check whether the rdb channel + * has completed its part and act accordingly. */ +void dualChannelSyncHandlePsync(void) { + serverAssert(server.repl_state == REPL_STATE_RECEIVE_PSYNC_REPLY); + if (server.repl_rdb_channel_state < REPL_DUAL_CHANNEL_RDB_LOADED) { + /* RDB is still loading */ + if (connSetReadHandler(server.repl_provisional_primary.conn, bufferReplData) == C_ERR) { + serverLog(LL_WARNING, "Error while setting readable handler: %s", strerror(errno)); + cancelReplicationHandshake(1); + return; + } + replDataBufInit(); + server.repl_state = REPL_STATE_TRANSFER; + return; + } + serverAssert(server.repl_rdb_channel_state == REPL_DUAL_CHANNEL_RDB_LOADED); + /* RDB is loaded */ + serverLog(LL_DEBUG, "Dual channel sync - psync established after rdb load"); + dualChannelSyncSuccess(); + return; +} + +/* Replication: Replica side. + * RDB channel done loading the RDB. Check whether the main channel has completed its part + * and act accordingly. */ +void dualChannelSyncHandleRdbLoadCompletion(void) { + serverAssert(server.repl_rdb_channel_state == REPL_DUAL_CHANNEL_RDB_LOAD); + if (server.repl_state < REPL_STATE_TRANSFER) { + /* Main psync channel hasn't been established yet */ + server.repl_rdb_channel_state = REPL_DUAL_CHANNEL_RDB_LOADED; + return; + } + serverAssert(server.repl_state == REPL_STATE_TRANSFER); + connSetReadHandler(server.repl_transfer_s, NULL); + dualChannelSyncSuccess(); + return; +} + /* Try a partial resynchronization with the primary if we are about to reconnect. * If there is no cached primary structure, at least try to issue a * "PSYNC ? -1" command in order to trigger a full resync using the PSYNC @@ -2367,6 +2966,7 @@ char *sendCommandArgv(connection *conn, int argc, char **argv, size_t *argv_lens #define PSYNC_FULLRESYNC 3 #define PSYNC_NOT_SUPPORTED 4 #define PSYNC_TRY_LATER 5 +#define PSYNC_FULLRESYNC_DUAL_CHANNEL 6 int replicaTryPartialResynchronization(connection *conn, int read_reply) { char *psync_replid; char psync_offset[32]; @@ -2381,7 +2981,13 @@ int replicaTryPartialResynchronization(connection *conn, int read_reply) { * client structure representing the primary into server.primary. */ server.primary_initial_offset = -1; - if (server.cached_primary) { + if (server.repl_rdb_channel_state != REPL_DUAL_CHANNEL_STATE_NONE) { + /* While in dual channel replication, we should use our prepared repl id and offset. */ + psync_replid = server.repl_provisional_primary.replid; + snprintf(psync_offset, sizeof(psync_offset), "%lld", server.repl_provisional_primary.reploff + 1); + serverLog(LL_NOTICE, "Trying a partial resynchronization using main channel (request %s:%s).", psync_replid, + psync_offset); + } else if (server.cached_primary) { psync_replid = server.cached_primary->replid; snprintf(psync_offset, sizeof(psync_offset), "%lld", server.cached_primary->reploff + 1); serverLog(LL_NOTICE, "Trying a partial resynchronization (request %s:%s).", psync_replid, psync_offset); @@ -2457,6 +3063,11 @@ int replicaTryPartialResynchronization(connection *conn, int read_reply) { } if (!strncmp(reply, "+CONTINUE", 9)) { + if (server.repl_rdb_channel_state != REPL_DUAL_CHANNEL_STATE_NONE) { + /* During dual channel sync sesseion, primary struct is already initialized. */ + sdsfree(reply); + return PSYNC_CONTINUE; + } /* Partial resync was accepted. */ serverLog(LL_NOTICE, "Successful partial resynchronization with primary."); @@ -2518,6 +3129,15 @@ int replicaTryPartialResynchronization(connection *conn, int read_reply) { return PSYNC_TRY_LATER; } + if (!strncmp(reply, "+DUALCHANNELSYNC", strlen("+DUALCHANNELSYNC"))) { + /* A response of +DUALCHANNELSYNC from the primary implies that partial + * synchronization is not possible and that the primary supports full + * sync using dedicated RDB channel. Full sync will continue that way. */ + serverLog(LL_NOTICE, "PSYNC is not possible, initialize RDB channel."); + sdsfree(reply); + return PSYNC_FULLRESYNC_DUAL_CHANNEL; + } + if (strncmp(reply, "-ERR", 4)) { /* If it's not an error, log the unexpected event. */ serverLog(LL_WARNING, "Unexpected reply to PSYNC from primary: %s", reply); @@ -2531,6 +3151,142 @@ int replicaTryPartialResynchronization(connection *conn, int read_reply) { return PSYNC_NOT_SUPPORTED; } +/* Replication: Replica side. + * This connection handler fires after rdb-connection was initialized. We use it + * to adjust the replica main for loading incremental changes into the local buffer. */ +void setupMainConnForPsync(connection *conn) { + int psync_result = -1; + char llstr[LONG_STR_SIZE]; + char *err = NULL; + if (server.repl_state == REPL_STATE_SEND_HANDSHAKE) { + /* We already have an initialized connection at primary side, we only need to associate it with RDB connection */ + ll2string(llstr, sizeof(llstr), server.rdb_client_id); + err = sendCommand(conn, "REPLCONF", "set-rdb-client-id", llstr, NULL); + if (err) goto error; + server.repl_state = REPL_STATE_RECEIVE_CAPA_REPLY; + sdsfree(err); + return; + } + + if (server.repl_state == REPL_STATE_RECEIVE_CAPA_REPLY) { + err = receiveSynchronousResponse(conn); + if (err == NULL) goto error; + if (err[0] == '-') { + serverLog(LL_NOTICE, "Primary does not understand REPLCONF identify: %s", err); + goto error; + } + sdsfree(err); + err = NULL; + server.repl_state = REPL_STATE_SEND_PSYNC; + } + + if (server.repl_state == REPL_STATE_SEND_PSYNC) { + if (server.debug_sleep_after_fork_us) usleep(server.debug_sleep_after_fork_us); + if (replicaTryPartialResynchronization(conn, 0) == PSYNC_WRITE_ERROR) { + serverLog(LL_WARNING, "Aborting dual channel sync. Write error."); + cancelReplicationHandshake(1); + } + server.repl_state = REPL_STATE_RECEIVE_PSYNC_REPLY; + return; + } + psync_result = replicaTryPartialResynchronization(conn, 1); + if (psync_result == PSYNC_WAIT_REPLY) return; /* Try again later... */ + + if (psync_result == PSYNC_CONTINUE) { + serverLog(LL_NOTICE, "Primary <-> REPLICA sync: Primary accepted a Partial Resynchronization%s", + server.repl_rdb_transfer_s != NULL ? ", RDB load in background." : "."); + if (server.supervised_mode == SUPERVISED_SYSTEMD) { + serverCommunicateSystemd("STATUS=Primary <-> REPLICA sync: Partial Resynchronization accepted. Ready to " + "accept connections in read-write mode.\n"); + } + dualChannelSyncHandlePsync(); + return; + } + +error: + sdsfree(err); + /* The dual-channel sync session must be aborted for any psync_result other than PSYNC_CONTINUE or PSYNC_WAIT_REPLY. + */ + serverLog(LL_WARNING, "Aborting dual channel sync. Main channel psync result %d", psync_result); + cancelReplicationHandshake(1); +} + +/* + * Dual channel for full sync + * + * * Motivation * + * - Reduce primary memory load. We do that by moving the COB tracking to the replica side. This also decrease + * the chance for COB overruns. Note that primary's input buffer limits at the replica side are less restricted + * then primary's COB as the replica plays less critical part in the replication group. While increasing the + * primary’s COB may end up with primary reaching swap and clients suffering, at replica side we’re more at + * ease with it. Larger COB means better chance to sync successfully. + * - Reduce primary main process CPU load. By opening a new, dedicated channel for the RDB transfer, child + * processes can have direct access to the new channel. Due to TLS connection restrictions, this was not + * possible using one main channel. We eliminate the need for the child process to use the primary's + * child-proc -> main-proc pipeline, thus freeing up the main process to process clients queries. + * + * * High level interface design * + * - Dual channel sync begins when the replica sends a REPLCONF capa dual-channel to the primary during initial + * handshake. This allows the replica to verify whether the primary supports dual-channel-replication and, if + * so, state that this is the replica's main channel, which is not used for snapshot transfer. + * - When replica lacks sufficient data for PSYNC, the primary will send +DUALCHANNELSYNC response instead + * of RDB data. As a next step, the replica creates a new channel (rdb-channel) and configures it against + * the primary with the appropriate capabilities and requirements. The replica then requests a sync + * using the RDB channel. + * - Prior to forking, the primary sends the replica the snapshot's end repl-offset, and attaches the replica + * to the replication backlog to keep repl data until the replica requests psync. The replica uses the main + * channel to request a PSYNC starting at the snapshot end offset. + * - The primary main threads sends incremental changes via the main channel, while the bgsave process + * sends the RDB directly to the replica via the rdb-channel. As for the replica, the incremental + * changes are stored on a local buffer, while the RDB is loaded into memory. + * - Once the replica completes loading the rdb, it drops the rdb channel and streams the accumulated incremental + * changes into memory. Repl steady state continues normally. + * + * * Replica state machine * + * ┌───────────────────┐ Dual channel sync + * │RECEIVE_PING_REPLY │ ┌──────────────────────────────────────────────────────────────┐ + * └────────┬──────────┘ │ RDB channel states Main channel state │ + * │+PONG │ ┌────────────────────────────┐ ┌───────────────────┐ │ + * ┌────────▼──────────┐ ┌─┼─────►DUAL_CHANNEL_SEND_HANDSHAKE │ ┌─►SEND_HANDSHAKE │ │ + * │SEND_HANDSHAKE │ │ │ └────┬───────────────────────┘ │ └──┬────────────────┘ │ + * └────────┬──────────┘ │ │ │ │ │REPLCONF set-rdb-client-id + * │ │ │ ┌───────▼───────────────────────┐ │ ┌──▼────────────────┐ │ + * ┌────────▼──────────┐ │ │ │DUAL_CHANNEL_RECEIVE_AUTH_REPLY│ │ │RECEIVE_CAPA_REPLY │ │ + * │RECEIVE_AUTH_REPLY │ │ │ └───────┬───────────────────────┘ │ └──┬────────────────┘ │ + * └────────┬──────────┘ │ │ │+OK │ │+OK │ + * │+OK │ │ ┌───────▼───────────────────────┐ │ ┌──▼────────────────┐ │ + * ┌────────▼──────────┐ │ │ │DUAL_CHANNEL_RECEIVE_REPLCONF_.│ │ │SEND_PSYNC │ │ + * │RECEIVE_PORT_REPLY │ │ │ └───────┬───────────────────────┘ │ └──┬────────────────┘ │ + * └────────┬──────────┘ │ │ │+OK │ │PSYNC use snapshot │ + * │+OK │ │ ┌───────▼────────────────┐ │ │end-offset provided │ + * ┌────────▼──────────┐ │ │ │DUAL_CHANNEL_RECEIVE_EN.│ │ │by the primary │ + * │RECEIVE_IP_REPLY │ │ │ └───────┬────────────────┘ │ ┌──▼────────────────┐ │ + * └────────┬──────────┘ │ │ │$ENDOFF │ │RECEIVE_PSYNC_REPLY│ │ + * │+OK │ │ ├─────────────────────────┘ └──┬────────────────┘ │ + * ┌────────▼──────────┐ │ │ │ │+CONTINUE │ + * │RECEIVE_IP_REPLY │ │ │ ┌───────▼───────────────┐ ┌──▼────────────────┐ │ + * └────────┬──────────┘ │ │ │DUAL_CHANNEL_RDB_LOAD │ │TRANSFER │ │ + * │+OK │ │ └───────┬───────────────┘ └─────┬─────────────┘ │ + * ┌────────▼──────────┐ │ │ │Done loading │ │ + * │RECEIVE_CAPA_REPLY │ │ │ ┌───────▼───────────────┐ │ │ + * └────────┬──────────┘ │ │ │DUAL_CHANNEL_RDB_LOADE.│ │ │ + * │ │ │ └───────┬───────────────┘ │ │ + * ┌────────▼───┐ │ │ │ │ │ + * │SEND_PSYNC │ │ │ │Replica loads local replication │ │ + * └─┬──────────┘ │ │ │buffer into memory │ │ + * │PSYNC (use cached-primary)│ │ └─────────┬───────────────────────┘ │ + * ┌─▼─────────────────┐ │ │ │ │ + * │RECEIVE_PSYNC_REPLY│ │ └────────────────────┼─────────────────────────────────────────┘ + * └────────┬─┬────────┘ │ │ + * +CONTINUE│ │+DUALCHANNELSYNC │ │ + * │ │ └─────────────────┘ │ + * │ │+FULLRESYNC │ + * │ ┌─▼─────────────────┐ ┌────▼──────────────┐ + * │ │TRANSFER ├───────────────────►CONNECTED │ + * │ └───────────────────┘ └────▲──────────────┘ + * │ │ + * └─────────────────────────────────────────────────┘ + */ /* This handler fires when the non blocking connect was able to * establish a connection with the primary. */ void syncWithPrimary(connection *conn) { @@ -2613,14 +3369,7 @@ void syncWithPrimary(connection *conn) { /* Set the replica port, so that primary's INFO command can list the * replica listening port correctly. */ { - int port; - if (server.replica_announce_port) - port = server.replica_announce_port; - else if (server.tls_replication && server.tls_port) - port = server.tls_port; - else - port = server.port; - sds portstr = sdsfromlonglong(port); + sds portstr = getReplicaPortString(); err = sendCommand(conn, "REPLCONF", "listening-port", portstr, NULL); sdsfree(portstr); if (err) goto write_error; @@ -2640,7 +3389,9 @@ void syncWithPrimary(connection *conn) { * PSYNC2: supports PSYNC v2, so understands +CONTINUE . * * The primary will ignore capabilities it does not understand. */ - err = sendCommand(conn, "REPLCONF", "capa", "eof", "capa", "psync2", NULL); + err = sendCommand(conn, "REPLCONF", "capa", "eof", "capa", "psync2", + server.dual_channel_replication ? "capa" : NULL, + server.dual_channel_replication ? "dual-channel" : NULL, NULL); if (err) goto write_error; /* Inform the primary of our (replica) version. */ @@ -2824,6 +3575,27 @@ void syncWithPrimary(connection *conn) { server.repl_transfer_fd = dfd; } + /* Using dual-channel-replication, the primary responded +DUALCHANNELSYNC. We need to + * initialize the RDB channel. */ + if (psync_result == PSYNC_FULLRESYNC_DUAL_CHANNEL) { + /* Create RDB connection */ + server.repl_rdb_transfer_s = connCreate(connTypeOfReplication()); + if (connConnect(server.repl_rdb_transfer_s, server.primary_host, server.primary_port, server.bind_source_addr, + fullSyncWithPrimary) == C_ERR) { + serverLog(LL_WARNING, "Unable to connect to Primary: %s", connGetLastError(server.repl_transfer_s)); + connClose(server.repl_rdb_transfer_s); + server.repl_rdb_transfer_s = NULL; + goto error; + } + if (connSetReadHandler(conn, NULL) == C_ERR) { + char conninfo[CONN_INFO_LEN]; + serverLog(LL_WARNING, "Can't clear main connection handler: %s (%s)", strerror(errno), + connGetInfo(conn, conninfo, sizeof(conninfo))); + goto error; + } + server.repl_rdb_channel_state = REPL_DUAL_CHANNEL_SEND_HANDSHAKE; + return; + } /* Setup the non blocking download of the bulk file. */ if (connSetReadHandler(conn, readSyncBulkPayload) == C_ERR) { char conninfo[CONN_INFO_LEN]; @@ -2847,6 +3619,10 @@ void syncWithPrimary(connection *conn) { if (dfd != -1) close(dfd); connClose(conn); server.repl_transfer_s = NULL; + if (server.repl_rdb_transfer_s) { + connClose(server.repl_rdb_transfer_s); + server.repl_rdb_transfer_s = NULL; + } if (server.repl_transfer_fd != -1) close(server.repl_transfer_fd); if (server.repl_transfer_tmpfile) zfree(server.repl_transfer_tmpfile); server.repl_transfer_tmpfile = NULL; @@ -2910,6 +3686,9 @@ void replicationAbortSyncTransfer(void) { * * Otherwise zero is returned and no operation is performed at all. */ int cancelReplicationHandshake(int reconnect) { + if (server.repl_rdb_channel_state != REPL_DUAL_CHANNEL_STATE_NONE) { + replicationAbortDualChannelSyncTransfer(); + } if (server.repl_state == REPL_STATE_TRANSFER) { replicationAbortSyncTransfer(); server.repl_state = REPL_STATE_CONNECT; @@ -2973,6 +3752,8 @@ void replicationSetPrimary(char *ip, int port) { moduleFireServerEvent(VALKEYMODULE_EVENT_PRIMARY_LINK_CHANGE, VALKEYMODULE_SUBEVENT_PRIMARY_LINK_DOWN, NULL); server.repl_state = REPL_STATE_CONNECT; + /* Allow trying dual-channel-replication with the new primary. If new primary doesn't + * support dual-channel-replication, we will set to 0 afterwards. */ serverLog(LL_NOTICE, "Connecting to PRIMARY %s:%d", server.primary_host, server.primary_port); connectWithPrimary(); } @@ -3296,16 +4077,10 @@ void replicationDiscardCachedPrimary(void) { server.cached_primary = NULL; } -/* Turn the cached primary into the current primary, using the file descriptor - * passed as argument as the socket for the new primary. - * - * This function is called when successfully setup a partial resynchronization - * so the stream of data that we'll receive will start from where this - * primary left. */ -void replicationResurrectCachedPrimary(connection *conn) { - server.primary = server.cached_primary; - server.cached_primary = NULL; - server.primary->conn = conn; +/* Replication: Replica side. + * This method performs the necessary steps to establish a connection with the primary server. + * It sets private data, updates flags, and fires an event to notify modules about the primary link change. */ +void establishPrimaryConnection(void) { connSetPrivateData(server.primary->conn, server.primary); server.primary->flag.close_after_reply = 0; server.primary->flag.close_asap = 0; @@ -3316,9 +4091,30 @@ void replicationResurrectCachedPrimary(connection *conn) { /* Fire the primary link modules event. */ moduleFireServerEvent(VALKEYMODULE_EVENT_PRIMARY_LINK_CHANGE, VALKEYMODULE_SUBEVENT_PRIMARY_LINK_UP, NULL); +} + +/* Replication: Replica side. + * Turn the cached primary into the current primary, using the file descriptor + * passed as argument as the socket for the new primary. + * + * This function is called when successfully setup a partial resynchronization + * so the stream of data that we'll receive will start from where this + * primary left. */ +void replicationResurrectCachedPrimary(connection *conn) { + server.primary = server.cached_primary; + server.cached_primary = NULL; + server.primary->conn = conn; + establishPrimaryConnection(); /* Re-add to the list of clients. */ linkClient(server.primary); + replicationSteadyStateInit(); +} + +/* Replication: Replica side. + * Prepare replica to steady state. + * prerequisite: server.primary is already initialized and linked in client list. */ +void replicationSteadyStateInit(void) { if (connSetReadHandler(server.primary->conn, readQueryFromClient)) { serverLog(LL_WARNING, "Error resurrecting the cached primary, impossible to add the readable handler: %s", strerror(errno)); @@ -3336,6 +4132,19 @@ void replicationResurrectCachedPrimary(connection *conn) { } } +/* Replication: Replica side. + * Turn the provisional primary into the current primary. + * This function is called after dual channel sync is finished successfully. */ +void replicationResurrectProvisionalPrimary(void) { + /* Create a primary client, but do not initialize the read handler yet, as this replica still has a local buffer to + * drain. */ + replicationCreatePrimaryClientWithHandler(server.repl_transfer_s, server.repl_provisional_primary.dbid, NULL); + memcpy(server.primary->replid, server.repl_provisional_primary.replid, CONFIG_RUN_ID_SIZE); + server.primary->reploff = server.repl_provisional_primary.reploff; + server.primary->read_reploff = server.repl_provisional_primary.read_reploff; + establishPrimaryConnection(); +} + /* ------------------------- MIN-REPLICAS-TO-WRITE --------------------------- */ /* This function counts the number of replicas with lag <= min-replicas-max-lag. @@ -3789,7 +4598,8 @@ void replicationCron(void) { * replicas number + 1(replication backlog). */ if (listLength(server.repl_buffer_blocks) > 0) { replBufBlock *o = listNodeValue(listFirst(server.repl_buffer_blocks)); - serverAssert(o->refcount > 0 && o->refcount <= (int)listLength(server.replicas) + 1); + serverAssert(o->refcount > 0 && + o->refcount <= (int)listLength(server.replicas) + 1 + (int)raxSize(server.replicas_waiting_psync)); } /* Refresh the number of replicas with lag <= min-replicas-max-lag. */ diff --git a/src/rio.c b/src/rio.c index 408a931d17..93dedf94ff 100644 --- a/src/rio.c +++ b/src/rio.c @@ -55,6 +55,7 @@ #include "crc64.h" #include "config.h" #include "server.h" +#include "connhelpers.h" /* ------------------------- Buffer I/O implementation ----------------------- */ @@ -496,3 +497,128 @@ size_t rioWriteBulkDouble(rio *r, double d) { dbuf[dlen] = '\0'; return rioWriteBulkString(r, dbuf, dlen); } + +/* Returns 1 or 0 for success/failure. + * The function returns success as long as we are able to correctly write + * to at least one file descriptor. + * + * When buf is NULL and len is 0, the function performs a flush operation + * if there is some pending buffer, so this function is also used in order + * to implement rioConnsetFlush(). */ +static size_t rioConnsetWrite(rio *r, const void *buf, size_t len) { + ssize_t retval; + int j; + unsigned char *p = (unsigned char *)buf; + int doflush = (buf == NULL && len == 0); + + /* To start we always append to our buffer. If it gets larger than + * a given size, we actually write to the sockets. */ + if (len) { + r->io.connset.buf = sdscatlen(r->io.connset.buf, buf, len); + len = 0; /* Prevent entering the while below if we don't flush. */ + if (sdslen(r->io.connset.buf) > PROTO_IOBUF_LEN) doflush = 1; + } + + if (doflush) { + p = (unsigned char *)r->io.connset.buf; + len = sdslen(r->io.connset.buf); + } + + /* Write in little chunchs so that when there are big writes we + * parallelize while the kernel is sending data in background to + * the TCP socket. */ + while (len) { + size_t count = len < 1024 ? len : 1024; + int broken = 0; + for (j = 0; j < r->io.connset.numconns; j++) { + if (r->io.connset.state[j] != 0) { + /* Skip FDs already in error. */ + broken++; + continue; + } + + /* Make sure to write 'count' bytes to the socket regardless + * of short writes. */ + size_t nwritten = 0; + while (nwritten != count) { + retval = connWrite(r->io.connset.conns[j], p + nwritten, count - nwritten); + if (retval <= 0) { + /* With blocking sockets, which is the sole user of this + * rio target, EWOULDBLOCK is returned only because of + * the SO_SNDTIMEO socket option, so we translate the error + * into one more recognizable by the user. */ + if (retval == -1 && errno == EWOULDBLOCK) errno = ETIMEDOUT; + break; + } + nwritten += retval; + } + + if (nwritten != count) { + /* Mark this FD as broken. */ + r->io.connset.state[j] = errno; + if (r->io.connset.state[j] == 0) r->io.connset.state[j] = EIO; + } + } + if (broken == r->io.connset.numconns) return 0; /* All the FDs in error. */ + p += count; + len -= count; + r->io.connset.pos += count; + } + + if (doflush) sdsclear(r->io.connset.buf); + return 1; +} + +/* Returns 1 or 0 for success/failure. */ +static size_t rioConnsetRead(rio *r, void *buf, size_t len) { + UNUSED(r); + UNUSED(buf); + UNUSED(len); + return 0; /* Error, this target does not support reading. */ +} + +/* Returns read/write position in file. */ +static off_t rioConnsetTell(rio *r) { + return r->io.connset.pos; +} + +/* Flushes any buffer to target device if applicable. Returns 1 on success + * and 0 on failures. */ +static int rioConnsetFlush(rio *r) { + /* Our flush is implemented by the write method, that recognizes a + * buffer set to NULL with a count of zero as a flush request. */ + return rioConnsetWrite(r, NULL, 0); +} + +static const rio rioConnsetIO = { + rioConnsetRead, + rioConnsetWrite, + rioConnsetTell, + rioConnsetFlush, + NULL, /* update_checksum */ + 0, /* current checksum */ + 0, /* flags */ + 0, /* bytes read or written */ + 0, /* read/write chunk size */ + {{NULL, 0}} /* union for io-specific vars */ +}; + +void rioInitWithConnset(rio *r, connection **conns, int numconns) { + *r = rioConnsetIO; + r->io.connset.conns = zmalloc(sizeof(connection *) * numconns); + r->io.connset.state = zmalloc(sizeof(int) * numconns); + for (int i = 0; i < numconns; i++) { + r->io.connset.conns[i] = conns[i]; + r->io.connset.state[i] = 0; + } + r->io.connset.numconns = numconns; + r->io.connset.pos = 0; + r->io.connset.buf = sdsempty(); +} + +/* release the rio stream. */ +void rioFreeConnset(rio *r) { + zfree(r->io.connset.conns); + zfree(r->io.connset.state); + sdsfree(r->io.connset.buf); +} diff --git a/src/rio.h b/src/rio.h index 50aca7b047..698dcb66d1 100644 --- a/src/rio.h +++ b/src/rio.h @@ -97,6 +97,14 @@ struct _rio { off_t pos; sds buf; } fd; + /* Multiple connections target (used to write to N sockets). */ + struct { + connection **conns; /* Connections */ + int *state; /* Error state of each fd. 0 (if ok) or errno. */ + int numconns; + off_t pos; + sds buf; + } connset; } io; }; @@ -184,4 +192,6 @@ void rioGenericUpdateChecksum(rio *r, const void *buf, size_t len); void rioSetAutoSync(rio *r, off_t bytes); void rioSetReclaimCache(rio *r, int enabled); uint8_t rioCheckType(rio *r); +void rioInitWithConnset(rio *r, connection **conns, int numconns); +void rioFreeConnset(rio *r); #endif diff --git a/src/server.c b/src/server.c index a19d8af3fd..fe8b3dd6b0 100644 --- a/src/server.c +++ b/src/server.c @@ -2045,6 +2045,7 @@ void initServerConfig(void) { server.cached_primary = NULL; server.primary_initial_offset = -1; server.repl_state = REPL_STATE_NONE; + server.repl_rdb_channel_state = REPL_DUAL_CHANNEL_STATE_NONE; server.repl_transfer_tmpfile = NULL; server.repl_transfer_fd = -1; server.repl_transfer_s = NULL; @@ -2052,6 +2053,8 @@ void initServerConfig(void) { server.repl_down_since = 0; /* Never connected, repl is down since EVER. */ server.primary_repl_offset = 0; server.fsynced_reploff_pending = 0; + server.rdb_client_id = -1; + server.loading_process_events_interval_ms = LOADING_PROCESS_EVENTS_INTERVAL_DEFAULT; /* Replication partial resync backlog */ server.repl_backlog = NULL; @@ -2545,6 +2548,8 @@ void initServer(void) { server.hz = server.config_hz; server.pid = getpid(); server.in_fork_child = CHILD_TYPE_NONE; + server.rdb_pipe_read = -1; + server.rdb_child_exit_pipe = -1; server.main_thread_id = pthread_self(); server.current_client = NULL; server.errors = raxNew(); @@ -2554,6 +2559,8 @@ void initServer(void) { server.clients_to_close = listCreate(); server.replicas = listCreate(); server.monitors = listCreate(); + server.replicas_waiting_psync = raxNew(); + server.wait_before_rdb_client_free = DEFAULT_WAIT_BEFORE_RDB_CLIENT_FREE; server.clients_pending_write = listCreate(); server.clients_pending_io_write = listCreate(); server.clients_pending_io_read = listCreate(); @@ -5140,6 +5147,7 @@ const char *replstateToString(int replstate) { switch (replstate) { case REPLICA_STATE_WAIT_BGSAVE_START: case REPLICA_STATE_WAIT_BGSAVE_END: return "wait_bgsave"; + case REPLICA_STATE_BG_RDB_LOAD: return "bg_transfer"; case REPLICA_STATE_SEND_BULK: return "send_bulk"; case REPLICA_STATE_ONLINE: return "online"; default: return ""; @@ -5699,7 +5707,9 @@ sds genValkeyInfoString(dict *section_dict, int all_sections, int everything) { "master_last_io_seconds_ago:%d\r\n", server.primary ? ((int)(server.unixtime-server.primary->last_interaction)) : -1, "master_sync_in_progress:%d\r\n", server.repl_state == REPL_STATE_TRANSFER, "slave_read_repl_offset:%lld\r\n", replica_read_repl_offset, - "slave_repl_offset:%lld\r\n", replica_repl_offset)); + "slave_repl_offset:%lld\r\n", replica_repl_offset, + "replicas_repl_buffer_size:%zu\r\n", server.pending_repl_data.len, + "replicas_repl_buffer_peak:%zu\r\n", server.pending_repl_data.peak)); /* clang-format on */ if (server.repl_state == REPL_STATE_TRANSFER) { @@ -5759,14 +5769,18 @@ sds genValkeyInfoString(dict *section_dict, int all_sections, int everything) { info = sdscatprintf(info, "slave%d:ip=%s,port=%d,state=%s," - "offset=%lld,lag=%ld\r\n", + "offset=%lld,lag=%ld,type=%s\r\n", replica_id, replica_ip, replica->replica_listening_port, state, - replica->repl_ack_off, lag); + replica->repl_ack_off, lag, + replica->flag.repl_rdb_channel ? "rdb-channel" + : replica->repl_state == REPLICA_STATE_BG_RDB_LOAD ? "main-channel" + : "replica"); replica_id++; } } /* clang-format off */ info = sdscatprintf(info, FMTARGS( + "replicas_waiting_psync:%llu\r\n", (unsigned long long)raxSize(server.replicas_waiting_psync), "master_failover_state:%s\r\n", getFailoverStateString(), "master_replid:%s\r\n", server.replid, "master_replid2:%s\r\n", server.replid2, diff --git a/src/server.h b/src/server.h index f76824784e..9f755150d3 100644 --- a/src/server.h +++ b/src/server.h @@ -139,7 +139,11 @@ struct hdr_histogram; #define CONFIG_BINDADDR_MAX 16 #define CONFIG_MIN_RESERVED_FDS 32 #define CONFIG_DEFAULT_PROC_TITLE_TEMPLATE "{title} {listen-addr} {server-mode}" +#define DEFAULT_WAIT_BEFORE_RDB_CLIENT_FREE \ + 60 /* Grace period in seconds for replica main \ + channel to establish psync. */ #define INCREMENTAL_REHASHING_THRESHOLD_US 1000 +#define LOADING_PROCESS_EVENTS_INTERVAL_DEFAULT 100 /* Default: 0.1 seconds */ /* Bucket sizes for client eviction pools. Each bucket stores clients with * memory usage of up to twice the size of the bucket below it. */ @@ -398,6 +402,18 @@ typedef enum { REPL_STATE_CONNECTED, /* Connected to primary */ } repl_state; +/* Replica rdb-channel replication state. Used in server.repl_rdb_channel_state for + * replicas to remember what to do next. */ +typedef enum { + REPL_DUAL_CHANNEL_STATE_NONE = 0, /* No active rdb channel sync */ + REPL_DUAL_CHANNEL_SEND_HANDSHAKE, /* Send handshake sequence to primary */ + REPL_DUAL_CHANNEL_RECEIVE_AUTH_REPLY, /* Wait for AUTH reply */ + REPL_DUAL_CHANNEL_RECEIVE_REPLCONF_REPLY, /* Wait for REPLCONF reply */ + REPL_DUAL_CHANNEL_RECEIVE_ENDOFF, /* Wait for $ENDOFF reply */ + REPL_DUAL_CHANNEL_RDB_LOAD, /* Loading rdb using rdb channel */ + REPL_DUAL_CHANNEL_RDB_LOADED, +} repl_rdb_channel_state; + /* The state of an in progress coordinated failover */ typedef enum { NO_FAILOVER = 0, /* No failover in progress */ @@ -415,18 +431,21 @@ typedef enum { #define REPLICA_STATE_SEND_BULK 8 /* Sending RDB file to replica. */ #define REPLICA_STATE_ONLINE 9 /* RDB file transmitted, sending just updates. */ #define REPLICA_STATE_RDB_TRANSMITTED \ - 10 /* RDB file transmitted - This state is used only for \ - * a replica that only wants RDB without replication buffer */ + 10 /* RDB file transmitted - This state is used only for \ + * a replica that only wants RDB without replication buffer */ +#define REPLICA_STATE_BG_RDB_LOAD 11 /* Main channel of a replica which uses dual channel replication. */ /* Replica capabilities. */ #define REPLICA_CAPA_NONE 0 -#define REPLICA_CAPA_EOF (1 << 0) /* Can parse the RDB EOF streaming format. */ -#define REPLICA_CAPA_PSYNC2 (1 << 1) /* Supports PSYNC2 protocol. */ +#define REPLICA_CAPA_EOF (1 << 0) /* Can parse the RDB EOF streaming format. */ +#define REPLICA_CAPA_PSYNC2 (1 << 1) /* Supports PSYNC2 protocol. */ +#define REPLICA_CAPA_DUAL_CHANNEL (1 << 2) /* Supports dual channel replication sync */ /* Replica requirements */ #define REPLICA_REQ_NONE 0 #define REPLICA_REQ_RDB_EXCLUDE_DATA (1 << 0) /* Exclude data from RDB */ #define REPLICA_REQ_RDB_EXCLUDE_FUNCTIONS (1 << 1) /* Exclude functions from RDB */ +#define REPLICA_REQ_RDB_CHANNEL (1 << 2) /* Use dual-channel-replication */ /* Mask of all bits in the replica requirements bitfield that represent non-standard (filtered) RDB requirements */ #define REPLICA_REQ_RDB_MASK (REPLICA_REQ_RDB_EXCLUDE_DATA | REPLICA_REQ_RDB_EXCLUDE_FUNCTIONS) @@ -1087,6 +1106,12 @@ typedef struct replBacklog { * byte in the replication backlog buffer.*/ } replBacklog; +typedef struct replDataBuf { + list *blocks; /* List of replDataBufBlock */ + size_t len; /* Number of bytes stored in all blocks */ + size_t peak; +} replDataBuf; + typedef struct { list *clients; size_t mem_usage_sum; @@ -1180,7 +1205,23 @@ typedef struct ClientFlags { uint64_t reprocessing_command : 1; /* The client is re-processing the command. */ uint64_t replication_done : 1; /* Indicate that replication has been done on the client */ uint64_t authenticated : 1; /* Indicate a client has successfully authenticated */ - uint64_t reserved : 9; /* Reserved for future use */ + uint64_t + protected_rdb_channel : 1; /* Dual channel replication sync: Protects the RDB client from premature \ + * release during full sync. This flag is used to ensure that the RDB client, which \ + * references the first replication data block required by the replica, is not \ + * released prematurely. Protecting the client is crucial for prevention of \ + * synchronization failures: \ + * If the RDB client is released before the replica initiates PSYNC, the primary \ + * will reduce the reference count (o->refcount) of the block needed by the replica. + * \ + * This could potentially lead to the removal of the required data block, resulting \ + * in synchronization failures. Such failures could occur even in scenarios where \ + * the replica only needs an additional 4KB beyond the minimum size of the + * repl_backlog. + * By using this flag, we ensure that the RDB client remains intact until the replica + * \ has successfully initiated PSYNC. */ + uint64_t repl_rdb_channel : 1; /* Dual channel replication sync: track a connection which is used for rdb snapshot */ + uint64_t reserved : 7; /* Reserved for future use */ } ClientFlags; typedef struct client { @@ -1257,6 +1298,8 @@ typedef struct client { int replica_version; /* Version on the form 0xMMmmpp. */ short replica_capa; /* Replica capabilities: REPLICA_CAPA_* bitwise OR. */ short replica_req; /* Replica requirements: REPLICA_REQ_* */ + uint64_t associated_rdb_client_id; /* The client id of this replica's rdb connection */ + time_t rdb_client_disconnect_time; /* Time of the first freeClient call on this client. Used for delaying free. */ multiState mstate; /* MULTI/EXEC state */ blockingState bstate; /* blocking state */ long long woff; /* Last write global replication offset. */ @@ -1656,6 +1699,11 @@ struct valkeyServer { list *clients_pending_io_read; /* List of clients with pending read to be process by I/O threads. */ list *clients_pending_io_write; /* List of clients with pending write to be process by I/O threads. */ list *replicas, *monitors; /* List of replicas and MONITORs */ + rax *replicas_waiting_psync; /* Radix tree for tracking replicas awaiting partial synchronization. + * Key: RDB client ID + * Value: RDB client object + * This structure holds dual-channel sync replicas from the start of their + * RDB transfer until their main channel establishes partial synchronization. */ client *current_client; /* The client that triggered the command execution (External or AOF). */ client *executing_client; /* The client executing the current command (possibly script or module). */ @@ -1698,6 +1746,7 @@ struct valkeyServer { off_t loading_loaded_bytes; time_t loading_start_time; off_t loading_process_events_interval_bytes; + time_t loading_process_events_interval_ms; /* Fields used only for stats */ time_t stat_starttime; /* Server start time */ long long stat_numcommands; /* Number of processed commands */ @@ -1930,6 +1979,7 @@ struct valkeyServer { int repl_ping_replica_period; /* Primary pings the replica every N seconds */ replBacklog *repl_backlog; /* Replication backlog for partial syncs */ long long repl_backlog_size; /* Backlog circular buffer size */ + replDataBuf pending_repl_data; /* Replication data buffer for dual-channel-replication */ time_t repl_backlog_time_limit; /* Time without replicas after the backlog gets released. */ time_t repl_no_replicas_since; /* We have no replicas since that time. @@ -1943,23 +1993,39 @@ struct valkeyServer { int repl_diskless_sync_delay; /* Delay to start a diskless repl BGSAVE. */ int repl_diskless_sync_max_replicas; /* Max replicas for diskless repl BGSAVE * delay (start sooner if they all connect). */ + int dual_channel_replication; /* Config used to determine if the replica should + * use dual channel replication for full syncs. */ + int wait_before_rdb_client_free; /* Grace period in seconds for replica main channel + * to establish psync. */ + int debug_sleep_after_fork_us; /* Debug param that force the main process to + * sleep for N microseconds after fork() in repl. */ size_t repl_buffer_mem; /* The memory of replication buffer. */ list *repl_buffer_blocks; /* Replication buffers blocks list * (serving replica clients and repl backlog) */ /* Replication (replica) */ - char *primary_user; /* AUTH with this user and primary_auth with primary */ - sds primary_auth; /* AUTH with this password with primary */ - char *primary_host; /* Hostname of primary */ - int primary_port; /* Port of primary */ - int repl_timeout; /* Timeout after N seconds of primary idle */ - client *primary; /* Client that is primary for this replica */ + char *primary_user; /* AUTH with this user and primary_auth with primary */ + sds primary_auth; /* AUTH with this password with primary */ + char *primary_host; /* Hostname of primary */ + int primary_port; /* Port of primary */ + int repl_timeout; /* Timeout after N seconds of primary idle */ + client *primary; /* Client that is primary for this replica */ + uint64_t rdb_client_id; /* Rdb client id as it defined at primary side */ + struct { + connection *conn; + char replid[CONFIG_RUN_ID_SIZE + 1]; + long long reploff; + long long read_reploff; + int dbid; + } repl_provisional_primary; client *cached_primary; /* Cached primary to be reused for PSYNC. */ int repl_syncio_timeout; /* Timeout for synchronous I/O calls */ int repl_state; /* Replication status if the instance is a replica */ + int repl_rdb_channel_state; /* State of the replica's rdb channel during dual-channel-replication */ off_t repl_transfer_size; /* Size of RDB to read from primary during sync. */ off_t repl_transfer_read; /* Amount of RDB read from primary during sync. */ off_t repl_transfer_last_fsync_off; /* Offset when we fsync-ed last time. */ connection *repl_transfer_s; /* Replica -> Primary SYNC connection */ + connection *repl_rdb_transfer_s; /* Primary FULL SYNC connection (RDB download) */ int repl_transfer_fd; /* Replica -> Primary SYNC temp file descriptor */ char *repl_transfer_tmpfile; /* Replica-> Primary SYNC temp file name */ time_t repl_transfer_lastio; /* Unix time of the latest read, for timeout */ @@ -2764,6 +2830,7 @@ int clientHasPendingReplies(client *c); int updateClientMemUsageAndBucket(client *c); void removeClientFromMemUsageBucket(client *c, int allow_eviction); void unlinkClient(client *c); +void removeFromServerClientList(client *c); int writeToClient(client *c); void linkClient(client *c); void protectClient(client *c); @@ -2965,6 +3032,8 @@ void clearFailoverState(void); void updateFailoverStatus(void); void abortFailover(const char *err); const char *getFailoverStateString(void); +int sendCurrentOffsetToReplica(client *replica); +void addRdbReplicaToPsyncWait(client *replica); /* Generic persistence functions */ void startLoadingFile(size_t size, char *filename, int rdbflags); diff --git a/tests/helpers/bg_server_sleep.tcl b/tests/helpers/bg_server_sleep.tcl new file mode 100644 index 0000000000..7e46fe9c90 --- /dev/null +++ b/tests/helpers/bg_server_sleep.tcl @@ -0,0 +1,10 @@ +source tests/support/valkey.tcl +source tests/support/util.tcl + +proc bg_server_sleep {host port sec} { + set r [valkey $host $port 0] + $r client setname SLEEP_HANDLER + $r debug sleep $sec +} + +bg_server_sleep [lindex $argv 0] [lindex $argv 1] [lindex $argv 2] diff --git a/tests/helpers/gen_write_load.tcl b/tests/helpers/gen_write_load.tcl index 9a148b73cc..44d5813acc 100644 --- a/tests/helpers/gen_write_load.tcl +++ b/tests/helpers/gen_write_load.tcl @@ -2,17 +2,23 @@ source tests/support/valkey.tcl set ::tlsdir "tests/tls" -proc gen_write_load {host port seconds tls} { +# Continuously sends SET commands to the node. If key is omitted, a random key is +# used for every SET command. The value is always random. +proc gen_write_load {host port seconds tls {key ""}} { set start_time [clock seconds] set r [valkey $host $port 1 $tls] $r client setname LOAD_HANDLER $r select 9 while 1 { - $r set [expr rand()] [expr rand()] + if {$key == ""} { + $r set [expr rand()] [expr rand()] + } else { + $r set $key [expr rand()] + } if {[clock seconds]-$start_time > $seconds} { exit 0 } } } -gen_write_load [lindex $argv 0] [lindex $argv 1] [lindex $argv 2] [lindex $argv 3] +gen_write_load [lindex $argv 0] [lindex $argv 1] [lindex $argv 2] [lindex $argv 3] [lindex $argv 4] diff --git a/tests/integration/dual-channel-replication.tcl b/tests/integration/dual-channel-replication.tcl new file mode 100644 index 0000000000..7344d0d3b1 --- /dev/null +++ b/tests/integration/dual-channel-replication.tcl @@ -0,0 +1,1116 @@ +proc log_file_matches {log pattern} { + set fp [open $log r] + set content [read $fp] + close $fp + string match $pattern $content +} + +proc start_bg_server_sleep {host port sec} { + set tclsh [info nameofexecutable] + exec $tclsh tests/helpers/bg_server_sleep.tcl $host $port $sec & +} + +proc stop_bg_server_sleep {handle} { + catch {exec /bin/kill -9 $handle} +} + +proc get_client_id_by_last_cmd {r cmd} { + set client_list [$r client list] + set client_id "" + set lines [split $client_list "\n"] + foreach line $lines { + if {[string match *cmd=$cmd* $line]} { + set parts [split $line " "] + foreach part $parts { + if {[string match id=* $part]} { + set client_id [lindex [split $part "="] 1] + return $client_id + } + } + } + } + return $client_id +} + +start_server {tags {"dual-channel-replication external:skip"}} { + set replica [srv 0 client] + set replica_host [srv 0 host] + set replica_port [srv 0 port] + set replica_log [srv 0 stdout] + start_server {} { + set primary [srv 0 client] + set primary_host [srv 0 host] + set primary_port [srv 0 port] + + # Configure the primary in order to hang waiting for the BGSAVE + # operation, so that the replica remains in the handshake state. + $primary config set repl-diskless-sync yes + $primary config set repl-diskless-sync-delay 1000 + $primary config set dual-channel-replication-enabled yes + + # Start the replication process... + $replica config set dual-channel-replication-enabled yes + $replica replicaof $primary_host $primary_port + + test "Test dual-channel-replication-enabled replica enters handshake" { + wait_for_condition 50 1000 { + [string match *handshake* [$replica role]] + } else { + fail "Replica does not enter handshake state" + } + } + + test "Test dual-channel-replication-enabled enters wait_bgsave" { + wait_for_condition 50 1000 { + [string match *state=wait_bgsave* [$primary info replication]] + } else { + fail "Replica does not enter wait_bgsave state" + } + } + + $primary config set repl-diskless-sync-delay 0 + + test "Test dual-channel-replication-enabled replica is able to sync" { + verify_replica_online $primary 0 500 + wait_for_condition 50 1000 { + [string match *connected_slaves:1* [$primary info]] + } else { + fail "Replica rdb connection is still open" + } + set offset [status $primary master_repl_offset] + wait_for_condition 500 100 { + [string match "*slave0:*,offset=$offset,*" [$primary info replication]] && + $offset == [status $replica master_repl_offset] + } else { + fail "Replicas and primary offsets were unable to match." + } + } + } +} + +start_server {tags {"dual-channel-replication external:skip"}} { + set replica [srv 0 client] + set replica_host [srv 0 host] + set replica_port [srv 0 port] + set replica_log [srv 0 stdout] + start_server {} { + set primary [srv 0 client] + set primary_host [srv 0 host] + set primary_port [srv 0 port] + + $primary config set rdb-key-save-delay 200 + $primary config set dual-channel-replication-enabled yes + $replica config set dual-channel-replication-enabled yes + $replica config set repl-diskless-sync no + + populate 1000 primary 10000 + set load_handle1 [start_one_key_write_load $primary_host $primary_port 100 "mykey1"] + set load_handle2 [start_one_key_write_load $primary_host $primary_port 100 "mykey2"] + set load_handle3 [start_one_key_write_load $primary_host $primary_port 100 "mykey3"] + + # wait for load handlers to start + wait_for_condition 50 1000 { + ([$primary get "mykey1"] != "") && + ([$primary get "mykey2"] != "") && + ([$primary get "mykey3"] != "") + } else { + fail "Can't set new keys" + } + + set before_used [s 0 used_memory] + + test "Primary memory usage does not increase during dual-channel-replication sync" { + $replica replicaof $primary_host $primary_port + + # Verify used_memory stays low through all the sync + set max_retry 500 + while {$max_retry} { + # Verify memory + set used_memory [s 0 used_memory] + assert {$used_memory-$before_used <= 1.5*10^6}; # ~1/3 of the space + # Check replica state + set primary_info [$primary info] + set replica_info [$replica info] + if {[string match *slave0:*state=online* $primary_info] && + [string match *master_link_status:up* $replica_info]} { + break + } else { + incr max_retry -1 + after 10 + } + } + if {$max_retry == 0} { + error "assertion:Replica not in sync after 5 seconds" + } + } + stop_write_load $load_handle1 + stop_write_load $load_handle2 + stop_write_load $load_handle3 + + test "Steady state after dual channel sync" { + wait_for_condition 50 1000 { + ([$replica get "mykey1"] eq [$primary get mykey1]) && + ([$replica get "mykey2"] eq [$primary get mykey2]) && + ([$replica get "mykey3"] eq [$primary get mykey3]) + } else { + fail "Can't set new keys" + } + } + + test "Dual channel replication sync doesn't impair subsequent normal syncs" { + $replica replicaof no one + $replica config set dual-channel-replication-enabled no + $primary set newkey newval + + set sync_full [s 0 sync_full] + set sync_partial [s 0 sync_partial_ok] + + $replica replicaof $primary_host $primary_port + verify_replica_online $primary 0 500 + # Verify replica used normal sync this time + assert_equal [expr $sync_full + 1] [s 0 sync_full] + assert_equal [expr $sync_partial] [s 0 sync_partial_ok] + assert [string match *connected_slaves:1* [$primary info]] + } + } +} + +start_server {tags {"dual-channel-replication external:skip"}} { + set replica [srv 0 client] + set replica_host [srv 0 host] + set replica_port [srv 0 port] + set replica_log [srv 0 stdout] + start_server {} { + foreach enable {yes no} { + set primary [srv 0 client] + set primary_host [srv 0 host] + set primary_port [srv 0 port] + + $primary config set repl-diskless-sync yes + # Set primary shared replication buffer size to a bit more then the size of + # a replication buffer block. + $primary config set client-output-buffer-limit "replica 1100k 0 0" + $primary config set dual-channel-replication-enabled $enable + $replica config set dual-channel-replication-enabled $enable + + test "Toggle dual-channel-replication-enabled: $enable start" { + populate 1000 primary 10000 + set prev_sync_full [s 0 sync_full] + set prev_sync_partial [s 0 sync_partial_ok] + + $replica replicaof $primary_host $primary_port + verify_replica_online $primary 0 500 + wait_for_sync $replica + + + set cur_sync_full [s 0 sync_full] + set cur_sync_partial [s 0 sync_partial_ok] + if {$enable == "yes"} { + # Verify that dual channel replication sync was used + assert {$cur_sync_full == [expr $prev_sync_full + 1]} + assert {$cur_sync_partial == [expr $prev_sync_partial + 1]} + } else { + # Verify that normal sync was used + assert {[s 0 sync_full] == [expr $prev_sync_full + 1]} + assert {[s 0 sync_partial_ok] == $prev_sync_partial} + } + + $replica replicaof no one + if {$enable == "yes"} { + # disable dual channel sync + $replica config set dual-channel-replication-enabled no + $primary config set dual-channel-replication-enabled no + } else { + $replica config set dual-channel-replication-enabled yes + $primary config set dual-channel-replication-enabled yes + } + + # Force replica to full sync next time + populate 1000 primary 10000 + set prev_sync_full [s 0 sync_full] + set prev_sync_partial [s 0 sync_partial_ok] + + $replica replicaof $primary_host $primary_port + verify_replica_online $primary 0 500 + wait_for_sync $replica + + set cur_sync_full [s 0 sync_full] + set cur_sync_partial [s 0 sync_partial_ok] + if {$enable == "yes"} { + # Verify that normal sync was used + assert {$cur_sync_full == [expr $prev_sync_full + 1]} + assert {$cur_sync_partial == $prev_sync_partial} + } else { + # Verify that dual channel replication sync was used + assert {$cur_sync_full == [expr $prev_sync_full + 1]} + assert {$cur_sync_partial == [expr $prev_sync_partial + 1]} + } + $replica replicaof no one + } + } + } +} + +start_server {tags {"dual-channel-replication external:skip"}} { + set replica1 [srv 0 client] + set replica1_host [srv 0 host] + set replica1_port [srv 0 port] + set replica1_log [srv 0 stdout] + start_server {} { + set replica2 [srv 0 client] + set replica2_host [srv 0 host] + set replica2_port [srv 0 port] + set replica2_log [srv 0 stdout] + start_server {} { + set primary [srv 0 client] + set primary_host [srv 0 host] + set primary_port [srv 0 port] + set loglines [count_log_lines -1] + + populate 10000 primary 10000 + $primary set key1 val1 + + $primary config set repl-diskless-sync yes + $primary config set repl-diskless-sync-delay 5; # allow both replicas to ask for sync + $primary config set dual-channel-replication-enabled yes + + $replica1 config set dual-channel-replication-enabled yes + $replica2 config set dual-channel-replication-enabled yes + $replica1 config set repl-diskless-sync no + $replica2 config set repl-diskless-sync no + $replica1 config set loglevel debug + $replica2 config set loglevel debug + + test "dual-channel-replication with multiple replicas" { + $replica1 replicaof $primary_host $primary_port + $replica2 replicaof $primary_host $primary_port + + wait_for_value_to_propegate_to_replica $primary $replica1 "key1" + wait_for_value_to_propegate_to_replica $primary $replica2 "key1" + + wait_for_condition 100 100 { + [s 0 total_forks] eq "1" + } else { + fail "Primary <-> Replica didn't start the full sync" + } + + verify_replica_online $primary 0 500 + verify_replica_online $primary 1 500 + wait_for_condition 50 1000 { + [status $replica1 master_link_status] == "up" + } else { + fail "Replica is not synced" + } + } + + $replica1 replicaof no one + $replica2 replicaof no one + + $replica1 config set dual-channel-replication-enabled yes + $replica2 config set dual-channel-replication-enabled no + + $primary set key2 val2 + + test "Test diverse replica sync: dual-channel on/off" { + $replica1 replicaof $primary_host $primary_port + $replica2 replicaof $primary_host $primary_port + + wait_for_value_to_propegate_to_replica $primary $replica1 "key2" + wait_for_value_to_propegate_to_replica $primary $replica2 "key2" + + verify_replica_online $primary 0 500 + verify_replica_online $primary 1 500 + wait_for_condition 50 1000 { + [status $replica1 master_link_status] == "up" + } else { + fail "Replica is not synced" + } + } + + $replica1 replicaof no one + $primary set key4 val4 + + test "Test replica's buffer limit reached" { + $primary config set repl-diskless-sync-delay 0 + $primary config set rdb-key-save-delay 500 + # At this point we have about 10k keys in the db, + # We expect that the next full sync will take 5 seconds (10k*500)ms + # It will give us enough time to fill the replica buffer. + $replica1 config set dual-channel-replication-enabled yes + $replica1 config set client-output-buffer-limit "replica 16383 16383 0" + $replica1 config set loglevel debug + + $replica1 replicaof $primary_host $primary_port + # Wait for replica to establish psync using main channel + wait_for_condition 500 1000 { + [string match "*slave*,state=bg_transfer*,type=main-channel*" [$primary info replication]] + } else { + fail "replica didn't start sync session in time" + } + + populate 10000 primary 10000 + # Wait for replica's buffer limit reached + wait_for_condition 50 1000 { + [log_file_matches $replica1_log "*Replication buffer limit reached, stopping buffering*"] + } else { + fail "Replica buffer should fill" + } + assert {[s -2 replicas_replication_buffer_size] <= 16385*2} + + # Wait for sync to succeed + wait_for_value_to_propegate_to_replica $primary $replica1 "key4" + wait_for_condition 50 1000 { + [status $replica1 master_link_status] == "up" + } else { + fail "Replica is not synced" + } + } + + $replica1 replicaof no one + $replica1 config set client-output-buffer-limit "replica 256mb 256mb 0"; # remove repl buffer limitation + + $primary set key5 val5 + + test "dual-channel-replication fails when primary diskless disabled" { + set cur_psync [status $primary sync_partial_ok] + $primary config set repl-diskless-sync no + + $replica1 config set dual-channel-replication-enabled yes + $replica1 replicaof $primary_host $primary_port + + # Wait for mitigation and resync + wait_for_value_to_propegate_to_replica $primary $replica1 "key5" + + wait_for_condition 50 1000 { + [status $replica1 master_link_status] == "up" + } else { + fail "Replica is not synced" + } + + # Verify that we did not use dual-channel-replication sync + assert {[status $primary sync_partial_ok] == $cur_psync} + } + } + } +} + +start_server {tags {"dual-channel-replication external:skip"}} { + set replica [srv 0 client] + set replica_host [srv 0 host] + set replica_port [srv 0 port] + set replica_log [srv 0 stdout] + start_server {} { + set primary [srv 0 client] + set primary_host [srv 0 host] + set primary_port [srv 0 port] + set loglines [count_log_lines -1] + # Create small enough db to be loaded before replica establish psync connection + $primary set key1 val1 + + $primary config set repl-diskless-sync yes + $primary debug sleep-after-fork-seconds 5;# Stop primary after fork for 5 seconds + $primary config set dual-channel-replication-enabled yes + + $replica config set dual-channel-replication-enabled yes + $replica config set loglevel debug + + test "Test dual-channel-replication sync- psync established after rdb load" { + $replica replicaof $primary_host $primary_port + + verify_replica_online $primary 0 500 + wait_for_condition 50 1000 { + [status $replica master_link_status] == "up" + } else { + fail "Replica is not synced" + } + wait_for_value_to_propegate_to_replica $primary $replica "key1" + # Confirm the occurrence of a race condition. + set res [wait_for_log_messages -1 {"*Dual channel sync - psync established after rdb load*"} $loglines 2000 1] + set loglines [lindex $res 1] + incr $loglines + } + } +} + +start_server {tags {"dual-channel-replication external:skip"}} { + set replica [srv 0 client] + set replica_host [srv 0 host] + set replica_port [srv 0 port] + set replica_log [srv 0 stdout] + start_server {} { + set primary [srv 0 client] + set primary_host [srv 0 host] + set primary_port [srv 0 port] + set backlog_size [expr {10 ** 5}] + set loglines [count_log_lines -1] + + $primary config set repl-diskless-sync yes + $primary config set dual-channel-replication-enabled yes + $primary config set repl-backlog-size $backlog_size + $primary config set loglevel debug + $primary config set repl-timeout 10 + $primary config set rdb-key-save-delay 200 + populate 10000 primary 10000 + + set load_handle1 [start_one_key_write_load $primary_host $primary_port 100 "mykey1"] + set load_handle2 [start_one_key_write_load $primary_host $primary_port 100 "mykey2"] + set load_handle3 [start_one_key_write_load $primary_host $primary_port 100 "mykey3"] + + $replica config set dual-channel-replication-enabled yes + $replica config set loglevel debug + $replica config set repl-timeout 10 + # Stop replica after primary fork for 5 seconds + $replica debug sleep-after-fork-seconds 5 + + test "dual-channel-replication: Primary COB growth with inactive replica" { + $replica replicaof $primary_host $primary_port + # Verify repl backlog can grow + wait_for_condition 1000 10 { + [s 0 mem_total_replication_buffers] > [expr {2 * $backlog_size}] + } else { + fail "Primary should allow backlog to grow beyond its limits during dual-channel-replication sync handshake" + } + + verify_replica_online $primary 0 500 + wait_for_condition 50 1000 { + [status $replica master_link_status] == "up" + } else { + fail "Replica is not synced" + } + } + + stop_write_load $load_handle1 + stop_write_load $load_handle2 + stop_write_load $load_handle3 + + } +} + +start_server {tags {"dual-channel-replication external:skip"}} { + set replica1 [srv 0 client] + set replica1_host [srv 0 host] + set replica1_port [srv 0 port] + set replica1_log [srv 0 stdout] + start_server {} { + set replica2 [srv 0 client] + set replica2_host [srv 0 host] + set replica2_port [srv 0 port] + set replica2_log [srv 0 stdout] + start_server {} { + set primary [srv 0 client] + set primary_host [srv 0 host] + set primary_port [srv 0 port] + set backlog_size [expr {10 ** 6}] + set loglines [count_log_lines -1] + + $primary config set repl-diskless-sync yes + $primary config set dual-channel-replication-enabled yes + $primary config set repl-backlog-size $backlog_size + $primary config set loglevel debug + $primary config set repl-timeout 10 + $primary config set rdb-key-save-delay 10 + populate 1024 primary 16 + + set load_handle0 [start_write_load $primary_host $primary_port 20] + + $replica1 config set dual-channel-replication-enabled yes + $replica2 config set dual-channel-replication-enabled yes + $replica1 config set loglevel debug + $replica2 config set loglevel debug + $replica1 config set repl-timeout 10 + $replica2 config set repl-timeout 10 + + # Stop replica after primary fork for 2 seconds + $replica1 debug sleep-after-fork-seconds 2 + $replica2 debug sleep-after-fork-seconds 2 + test "Test dual-channel: primary tracking replica backlog refcount - start with empty backlog" { + $replica1 replicaof $primary_host $primary_port + set res [wait_for_log_messages 0 {"*Add rdb replica * no repl-backlog to track*"} $loglines 2000 1] + set res [wait_for_log_messages 0 {"*Attach replica rdb client*"} $loglines 2000 1] + set loglines [lindex $res 1] + incr $loglines + verify_replica_online $primary 0 700 + wait_for_condition 50 1000 { + [status $replica1 master_link_status] == "up" + } else { + fail "Replica is not synced" + } + $replica1 replicaof no one + assert [string match *replicas_waiting_psync:0* [$primary info replication]] + } + + test "Test dual-channel: primary tracking replica backlog refcount - start with backlog" { + $replica2 replicaof $primary_host $primary_port + set res [wait_for_log_messages 0 {"*Add rdb replica * tracking repl-backlog tail*"} $loglines 2000 1] + set loglines [lindex $res 1] + incr $loglines + verify_replica_online $primary 0 700 + wait_for_condition 50 1000 { + [status $replica2 master_link_status] == "up" + } else { + fail "Replica is not synced" + } + assert [string match *replicas_waiting_psync:0* [$primary info replication]] + } + + stop_write_load $load_handle0 + } + } +} + +start_server {tags {"dual-channel-replication external:skip"}} { + set primary [srv 0 client] + set primary_host [srv 0 host] + set primary_port [srv 0 port] + + $primary config set repl-diskless-sync yes + $primary config set dual-channel-replication-enabled yes + $primary config set repl-backlog-size [expr {10 ** 6}] + $primary config set loglevel debug + $primary config set repl-timeout 10 + # generate small db + populate 10 primary 10 + # Stop primary main process after fork for 2 seconds + $primary debug sleep-after-fork-seconds 2 + $primary debug delay-rdb-client-free-seconds 5 + + start_server {} { + set replica [srv 0 client] + set replica_host [srv 0 host] + set replica_port [srv 0 port] + set replica_log [srv 0 stdout] + set loglines [count_log_lines 0] + + set load_handle0 [start_write_load $primary_host $primary_port 20] + + $replica config set dual-channel-replication-enabled yes + $replica config set loglevel debug + $replica config set repl-timeout 10 + + test "Psync established after rdb load - within grace period" { + $replica replicaof $primary_host $primary_port + set res [wait_for_log_messages 0 {"*Done loading RDB*"} $loglines 2000 1] + set loglines [lindex $res 1] + incr $loglines + # At this point rdb is loaded but psync hasn't been established yet. + # Force the replica to sleep for 3 seconds so the primary main process will wake up, while the replica is unresponsive. + set sleep_handle [start_bg_server_sleep $replica_host $replica_port 3] + wait_for_condition 50 100 { + [string match {*replicas_waiting_psync:1*} [$primary info replication]] + } else { + fail "Primary freed RDB client before psync was established" + } + + verify_replica_online $primary 0 500 + wait_for_condition 50 100 { + [string match {*replicas_waiting_psync:0*} [$primary info replication]] + } else { + fail "Primary did not free repl buf block after psync establishment" + } + $replica replicaof no one + } + stop_write_load $load_handle0 + } + + start_server {} { + set replica [srv 0 client] + set replica_host [srv 0 host] + set replica_port [srv 0 port] + set replica_log [srv 0 stdout] + set loglines [count_log_lines 0] + + set load_handle0 [start_write_load $primary_host $primary_port 20] + + $replica config set dual-channel-replication-enabled yes + $replica config set loglevel debug + $replica config set repl-timeout 10 + + test "Psync established after RDB load - beyond grace period" { + $replica replicaof $primary_host $primary_port + set res [wait_for_log_messages 0 {"*Done loading RDB*"} $loglines 2000 1] + set loglines [lindex $res 1] + incr $loglines + # At this point rdb is loaded but psync hasn't been established yet. + # Force the replica to sleep for 8 seconds so the primary main process will wake up, while the replica is unresponsive. + # We expect the grace time to be over before the replica wake up, so sync will fail. + set sleep_handle [start_bg_server_sleep $replica_host $replica_port 8] + wait_for_condition 50 100 { + [string match {*replicas_waiting_psync:1*} [$primary info replication]] + } else { + fail "Primary should wait before freeing repl block" + } + + # Sync should fail once the replica ask for PSYNC using main channel + set res [wait_for_log_messages -1 {"*Replica main channel failed to establish PSYNC within the grace period*"} 0 4000 1] + + # Should succeed on retry + verify_replica_online $primary 0 500 + wait_for_condition 50 100 { + [string match {*replicas_waiting_psync:0*} [$primary info replication]] + } else { + fail "Primary did not free repl buf block after psync establishment" + } + $replica replicaof no one + } + stop_write_load $load_handle0 + } +} + +start_server {tags {"dual-channel-replication external:skip"}} { + set primary [srv 0 client] + set primary_host [srv 0 host] + set primary_port [srv 0 port] + set loglines [count_log_lines 0] + + $primary config set repl-diskless-sync yes + $primary config set dual-channel-replication-enabled yes + $primary config set client-output-buffer-limit "replica 1100k 0 0" + $primary config set loglevel debug + # generate small db + populate 10 primary 10 + # Stop primary main process after fork for 1 seconds + $primary debug sleep-after-fork-seconds 2 + start_server {} { + set replica [srv 0 client] + set replica_host [srv 0 host] + set replica_port [srv 0 port] + set replica_log [srv 0 stdout] + + set load_handle0 [start_write_load $primary_host $primary_port 20] + set load_handle1 [start_write_load $primary_host $primary_port 20] + set load_handle2 [start_write_load $primary_host $primary_port 20] + + $replica config set dual-channel-replication-enabled yes + $replica config set loglevel debug + $replica config set repl-timeout 10 + + test "Test dual-channel-replication primary gets cob overrun before established psync" { + $replica replicaof $primary_host $primary_port + wait_for_log_messages 0 {"*Done loading RDB*"} 0 2000 1 + + # At this point rdb is loaded but psync hasn't been established yet. + # Force the replica to sleep for 5 seconds so the primary main process will wake up while the + # replica is unresponsive. We expect the main process to fill the COB before the replica wakes. + set sleep_handle [start_bg_server_sleep $replica_host $replica_port 5] + wait_for_log_messages -1 {"*Client * closed * for overcoming of output buffer limits.*"} $loglines 2000 1 + wait_for_condition 50 100 { + [string match {*replicas_waiting_psync:0*} [$primary info replication]] + } else { + fail "Primary did not free repl buf block after sync failure" + } + set res [wait_for_log_messages -1 {"*Unable to partial resync with replica * for lack of backlog*"} $loglines 20000 1] + set loglines [lindex $res 1] + } + + $replica replicaof no one + $replica debug sleep-after-fork-seconds 2 + + $primary debug populate 1000 primary 100000 + # Set primary with a slow rdb generation, so that we can easily intercept loading + # 10ms per key, with 1000 keys is 10 seconds + $primary config set rdb-key-save-delay 10000 + $primary debug sleep-after-fork-seconds 0 + + test "Test dual-channel-replication primary gets cob overrun during replica rdb load" { + set cur_client_closed_count [s -1 client_output_buffer_limit_disconnections] + $replica replicaof $primary_host $primary_port + wait_for_condition 500 1000 { + [s -1 client_output_buffer_limit_disconnections] > $cur_client_closed_count + } else { + fail "Primary should disconnect replica due to COB overrun" + } + + wait_for_condition 50 100 { + [string match {*replicas_waiting_psync:0*} [$primary info replication]] + } else { + fail "Primary did not free repl buf block after sync failure" + } + set res [wait_for_log_messages -1 {"*Unable to partial resync with replica * for lack of backlog*"} $loglines 20000 1] + set loglines [lindex $res 0] + } + stop_write_load $load_handle0 + stop_write_load $load_handle1 + stop_write_load $load_handle2 + } +} + +foreach dualchannel {yes no} { +start_server {tags {"dual-channel-replication external:skip"}} { + set primary [srv 0 client] + set primary_host [srv 0 host] + set primary_port [srv 0 port] + set loglines [count_log_lines 0] + + $primary config set repl-diskless-sync yes + $primary config set dual-channel-replication-enabled yes + $primary config set loglevel debug + $primary config set repl-diskless-sync-delay 5 + + # Generating RDB will cost 5s(10000 * 0.0005s) + $primary debug populate 10000 primary 1 + $primary config set rdb-key-save-delay 500 + + $primary config set dual-channel-replication-enabled $dualchannel + + start_server {} { + set replica1 [srv 0 client] + $replica1 config set dual-channel-replication-enabled $dualchannel + $replica1 config set loglevel debug + start_server {} { + set replica2 [srv 0 client] + $replica2 config set dual-channel-replication-enabled $dualchannel + $replica2 config set loglevel debug + $replica2 config set repl-timeout 60 + + set load_handle [start_one_key_write_load $primary_host $primary_port 100 "mykey1"] + test "Sync should continue if not all slaves dropped dual-channel-replication $dualchannel" { + $replica1 replicaof $primary_host $primary_port + $replica2 replicaof $primary_host $primary_port + + wait_for_condition 50 1000 { + [status $primary rdb_bgsave_in_progress] == 1 + } else { + fail "Sync did not start" + } + if {$dualchannel == "yes"} { + # Wait for both replicas main conns to establish psync + wait_for_condition 50 1000 { + [status $primary sync_partial_ok] == 2 + } else { + fail "Replicas main conns didn't establish psync [status $primary sync_partial_ok]" + } + } + + catch {$replica1 shutdown nosave} + wait_for_condition 50 2000 { + [status $replica2 master_link_status] == "up" && + [status $primary sync_full] == 2 && + (($dualchannel == "yes" && [status $primary sync_partial_ok] == 2) || $dualchannel == "no") + } else { + fail "Sync session interapted\n + sync_full:[status $primary sync_full]\n + sync_partial_ok:[status $primary sync_partial_ok]" + } + } + + $replica2 replicaof no one + + # Generating RDB will cost 500s(1000000 * 0.0001s) + $primary debug populate 1000000 primary 1 + $primary config set rdb-key-save-delay 100 + + test "Primary abort sync if all slaves dropped dual-channel-replication $dualchannel" { + set cur_psync [status $primary sync_partial_ok] + $replica2 replicaof $primary_host $primary_port + + wait_for_condition 50 1000 { + [status $primary rdb_bgsave_in_progress] == 1 + } else { + fail "Sync did not start" + } + if {$dualchannel == "yes"} { + # Wait for both replicas main conns to establish psync + wait_for_condition 50 1000 { + [status $primary sync_partial_ok] == $cur_psync + 1 + } else { + fail "Replicas main conns didn't establish psync [status $primary sync_partial_ok]" + } + } + + catch {$replica2 shutdown nosave} + wait_for_condition 50 1000 { + [status $primary rdb_bgsave_in_progress] == 0 + } else { + fail "Primary should abort the sync" + } + } + stop_write_load $load_handle + } + } +} +} + +start_server {tags {"dual-channel-replication external:skip"}} { + set primary [srv 0 client] + set primary_host [srv 0 host] + set primary_port [srv 0 port] + set loglines [count_log_lines 0] + + $primary config set repl-diskless-sync yes + $primary config set dual-channel-replication-enabled yes + $primary config set loglevel debug + $primary config set repl-diskless-sync-delay 5; # allow catch failed sync before retry + + # Generating RDB will cost 500s(1000000 * 0.0001s) + $primary debug populate 1000000 primary 1 + $primary config set rdb-key-save-delay 100 + + start_server {} { + set replica [srv 0 client] + set replica_host [srv 0 host] + set replica_port [srv 0 port] + set replica_log [srv 0 stdout] + + set load_handle [start_write_load $primary_host $primary_port 20] + + $replica config set dual-channel-replication-enabled yes + $replica config set loglevel debug + $replica config set repl-timeout 10 + test "Test dual-channel-replication replica main channel disconnected" { + $replica replicaof $primary_host $primary_port + # Wait for sync session to start + wait_for_condition 500 1000 { + [string match "*slave*,state=wait_bgsave*,type=rdb-channel*" [$primary info replication]] && + [string match "*slave*,state=bg_transfer*,type=main-channel*" [$primary info replication]] && + [s -1 rdb_bgsave_in_progress] eq 1 + } else { + fail "replica didn't start sync session in time" + } + + $primary debug log "killing replica main connection" + set replica_main_conn_id [get_client_id_by_last_cmd $primary "psync"] + assert {$replica_main_conn_id != ""} + $primary client kill id $replica_main_conn_id + # Wait for primary to abort the sync + wait_for_condition 50 1000 { + [string match {*replicas_waiting_psync:0*} [$primary info replication]] + } else { + fail "Primary did not free repl buf block after sync failure" + } + wait_for_condition 1000 10 { + [s -1 rdb_last_bgsave_status] eq "err" + } else { + fail "bgsave did not stop in time" + } + } + + test "Test dual channel replication slave of no one after main conn kill" { + $replica replicaof no one + wait_for_condition 500 1000 { + [s -1 rdb_bgsave_in_progress] eq 0 + } else { + fail "Primary should abort sync" + } + } + + test "Test dual-channel-replication replica rdb connection disconnected" { + $replica replicaof $primary_host $primary_port + # Wait for sync session to start + wait_for_condition 500 1000 { + [string match "*slave*,state=wait_bgsave*,type=rdb-channel*" [$primary info replication]] && + [string match "*slave*,state=bg_transfer*,type=main-channel*" [$primary info replication]] && + [s -1 rdb_bgsave_in_progress] eq 1 + } else { + fail "replica didn't start sync session in time" + } + + $primary debug log "killing replica rdb connection" + set replica_rdb_channel_id [get_client_id_by_last_cmd $primary "sync"] + assert {$replica_rdb_channel_id != ""} + $primary client kill id $replica_rdb_channel_id + # Wait for primary to abort the sync + wait_for_condition 1000 10 { + [s -1 rdb_bgsave_in_progress] eq 0 && + [s -1 rdb_last_bgsave_status] eq "err" + } else { + fail "Primary should abort sync" + } + } + + test "Test dual channel replication slave of no one after rdb conn kill" { + $replica replicaof no one + wait_for_condition 500 1000 { + [s -1 rdb_bgsave_in_progress] eq 0 + } else { + fail "Primary should abort sync" + } + } + + test "Test dual-channel-replication primary reject set-rdb-client after client killed" { + # Ensure replica main channel will not handshake before rdb client is killed + $replica debug sleep-after-fork-seconds 10 + $replica replicaof $primary_host $primary_port + # Wait for sync session to start + wait_for_condition 500 1000 { + [string match "*slave*,state=wait_bgsave*,type=rdb-channel*" [$primary info replication]] && + [s -1 rdb_bgsave_in_progress] eq 1 + } else { + fail "replica didn't start sync session in time" + } + + set replica_rdb_channel_id [get_client_id_by_last_cmd $primary "sync"] + assert {$replica_rdb_channel_id != ""} + $primary debug log "killing replica rdb connection $replica_rdb_channel_id" + $primary client kill id $replica_rdb_channel_id + # Wait for primary to abort the sync + wait_for_condition 10000000 10 { + [s -1 rdb_bgsave_in_progress] eq 0 + } else { + fail "Primary should abort sync" + } + # Verify primary reject replconf set-rdb-client-id + set res [catch {$primary replconf set-rdb-client-id $replica_rdb_channel_id} err] + assert [string match *ERR* $err] + } + stop_write_load $load_handle + } +} + +start_server {tags {"dual-channel-replication external:skip"}} { + set primary [srv 0 client] + set primary_host [srv 0 host] + set primary_port [srv 0 port] + set loglines [count_log_lines 0] + + $primary config set repl-diskless-sync yes + $primary config set dual-channel-replication-enabled yes + $primary config set loglevel debug + $primary config set repl-diskless-sync-delay 0; # don't wait for other replicas + + # Generating RDB will cost 5s(10000 * 0.0001s) + $primary debug populate 10000 primary 1 + $primary config set rdb-key-save-delay 100 + + start_server {} { + set replica_1 [srv 0 client] + set replica_host_1 [srv 0 host] + set replica_port_1 [srv 0 port] + set replica_log_1 [srv 0 stdout] + + $replica_1 config set dual-channel-replication-enabled yes + $replica_1 config set loglevel debug + $replica_1 config set repl-timeout 10 + start_server {} { + set replica_2 [srv 0 client] + set replica_host_2 [srv 0 host] + set replica_port_2 [srv 0 port] + set replica_log_2 [srv 0 stdout] + + set load_handle [start_write_load $primary_host $primary_port 20] + + $replica_2 config set dual-channel-replication-enabled yes + $replica_2 config set loglevel debug + $replica_2 config set repl-timeout 10 + test "Test replica unable to join dual channel replication sync after started" { + $replica_1 replicaof $primary_host $primary_port + # Wait for sync session to start + wait_for_condition 50 100 { + [s -2 rdb_bgsave_in_progress] eq 1 + } else { + fail "replica didn't start sync session in time1" + } + $replica_2 replicaof $primary_host $primary_port + wait_for_log_messages -2 {"*Current BGSAVE has socket target. Waiting for next BGSAVE for SYNC*"} $loglines 100 1000 + $primary config set rdb-key-save-delay 0 + # Verify second replica needed new session + wait_for_sync $replica_2 + assert {[s -2 sync_partial_ok] eq 2} + assert {[s -2 sync_full] eq 2} + } + stop_write_load $load_handle + } + } +} + +start_server {tags {"dual-channel-replication external:skip"}} { + set primary [srv 0 client] + set primary_host [srv 0 host] + set primary_port [srv 0 port] + set loglines [count_log_lines 0] + + $primary config set repl-diskless-sync yes + $primary config set dual-channel-replication-enabled yes + $primary config set loglevel debug + $primary config set repl-diskless-sync-delay 5; # allow catch failed sync before retry + + # Generating RDB will cost 5s(10000 * 0.0001s) + $primary debug populate 10000 primary 1 + $primary config set rdb-key-save-delay 100 + + start_server {} { + set replica [srv 0 client] + set replica_host [srv 0 host] + set replica_port [srv 0 port] + set replica_log [srv 0 stdout] + + $replica config set dual-channel-replication-enabled yes + $replica config set loglevel debug + $replica config set repl-timeout 10 + test "Replica recover rdb-connection killed" { + set load_handle [start_one_key_write_load $primary_host $primary_port 100 "mykey"] + $replica replicaof $primary_host $primary_port + # Wait for sync session to start + wait_for_condition 500 1000 { + [string match "*slave*,state=wait_bgsave*,type=rdb-channel*" [$primary info replication]] && + [string match "*slave*,state=bg_transfer*,type=main-channel*" [$primary info replication]] && + [s -1 rdb_bgsave_in_progress] eq 1 + } else { + fail "replica didn't start sync session in time" + } + + $primary debug log "killing replica rdb connection" + set replica_rdb_channel_id [get_client_id_by_last_cmd $primary "sync"] + assert {$replica_rdb_channel_id != ""} + $primary client kill id $replica_rdb_channel_id + # Wait for primary to abort the sync + wait_for_condition 50 1000 { + [string match {*replicas_waiting_psync:0*} [$primary info replication]] + } else { + fail "Primary did not free repl buf block after sync failure" + } + wait_for_condition 1000 10 { + [s -1 rdb_last_bgsave_status] eq "err" + } else { + fail "bgsave did not stop in time" + } + # Replica should retry + verify_replica_online $primary 0 500 + stop_write_load $load_handle + wait_for_condition 1000 100 { + [s -1 master_repl_offset] eq [s master_repl_offset] + } else { + fail "Replica offset didn't catch up with the primary after too long time" + } + } + $replica replicaof no one + + test "Replica recover main-connection killed" { + set load_handle [start_one_key_write_load $primary_host $primary_port 100 "mykey"] + $replica replicaof $primary_host $primary_port + # Wait for sync session to start + wait_for_condition 500 1000 { + [string match "*slave*,state=wait_bgsave*,type=rdb-channel*" [$primary info replication]] && + [string match "*slave*,state=bg_transfer*,type=main-channel*" [$primary info replication]] && + [s -1 rdb_bgsave_in_progress] eq 1 + } else { + fail "replica didn't start sync session in time" + } + + $primary debug log "killing replica main connection" + set replica_main_conn_id [get_client_id_by_last_cmd $primary "sync"] + assert {$replica_main_conn_id != ""} + $primary client kill id $replica_main_conn_id + # Wait for primary to abort the sync + wait_for_condition 50 1000 { + [string match {*replicas_waiting_psync:0*} [$primary info replication]] + } else { + fail "Primary did not free repl buf block after sync failure" + } + wait_for_condition 1000 10 { + [s -1 rdb_last_bgsave_status] eq "err" + } else { + fail "bgsave did not stop in time" + } + # Replica should retry + verify_replica_online $primary 0 500 + stop_write_load $load_handle + wait_for_condition 1000 100 { + [s -1 master_repl_offset] eq [s master_repl_offset] + } else { + fail "Replica offset didn't catch up with the primary after too long time" + } + } + + + } +} \ No newline at end of file diff --git a/tests/integration/psync2-master-restart.tcl b/tests/integration/psync2-master-restart.tcl index 6f7a31d218..34319e66a3 100644 --- a/tests/integration/psync2-master-restart.tcl +++ b/tests/integration/psync2-master-restart.tcl @@ -203,10 +203,16 @@ start_server {} { } else { fail "Replicas didn't sync after master restart" } + set dualchannel [lindex [r config get dual-channel-replication-enabled] 1] + set psync_count 0 + if {$dualchannel == "yes"} { + # Expect one fake psync + set psync_count 1 + } # Replication backlog is full assert {[status $master repl_backlog_first_byte_offset] > [status $master second_repl_offset]} - assert {[status $master sync_partial_ok] == 0} + assert {[status $master sync_partial_ok] == $psync_count} assert {[status $master sync_full] == 1} assert {[status $master rdb_last_load_keys_expired] == 2048} assert {[status $replica sync_full] == 1} diff --git a/tests/integration/replication-buffer.tcl b/tests/integration/replication-buffer.tcl index c57b86897b..2303d01273 100644 --- a/tests/integration/replication-buffer.tcl +++ b/tests/integration/replication-buffer.tcl @@ -1,6 +1,7 @@ # This test group aims to test that all replicas share one global replication buffer, # two replicas don't make replication buffer size double, and when there is no replica, # replica buffer will shrink. +foreach dualchannel {"yes" "no"} { start_server {tags {"repl external:skip"}} { start_server {} { start_server {} { @@ -8,6 +9,9 @@ start_server {} { set replica1 [srv -3 client] set replica2 [srv -2 client] set replica3 [srv -1 client] + $replica1 config set dual-channel-replication-enabled $dualchannel + $replica2 config set dual-channel-replication-enabled $dualchannel + $replica3 config set dual-channel-replication-enabled $dualchannel set master [srv 0 client] set master_host [srv 0 host] @@ -18,6 +22,7 @@ start_server {} { $master config set repl-diskless-sync-delay 5 $master config set repl-diskless-sync-max-replicas 1 $master config set client-output-buffer-limit "replica 0 0 0" + $master config set dual-channel-replication-enabled $dualchannel # Make sure replica3 is synchronized with master $replica3 replicaof $master_host $master_port @@ -39,7 +44,7 @@ start_server {} { fail "fail to sync with replicas" } - test {All replicas share one global replication buffer} { + test "All replicas share one global replication buffer dualchannel $dualchannel" { set before_used [s used_memory] populate 1024 "" 1024 ; # Write extra 1M data # New data uses 1M memory, but all replicas use only one @@ -47,19 +52,29 @@ start_server {} { # more than double of replication buffer. set repl_buf_mem [s mem_total_replication_buffers] set extra_mem [expr {[s used_memory]-$before_used-1024*1024}] - assert {$extra_mem < 2*$repl_buf_mem} - + if {$dualchannel == "yes"} { + # master's replication buffers should not grow during dual channel replication + assert {$extra_mem < 1024*1024} + assert {$repl_buf_mem < 1024*1024} + } else { + assert {$extra_mem < 2*$repl_buf_mem} + } # Kill replica1, replication_buffer will not become smaller catch {$replica1 shutdown nosave} - wait_for_condition 50 100 { - [s connected_slaves] eq {2} + set cur_slave_count 2 + if {$dualchannel == "yes"} { + # slave3 is connected, slave2 is syncing (has two connection) + set cur_slave_count 3 + } + wait_for_condition 500 100 { + [s connected_slaves] eq $cur_slave_count } else { fail "replica doesn't disconnect with master" } assert_equal $repl_buf_mem [s mem_total_replication_buffers] } - test {Replication buffer will become smaller when no replica uses} { + test "Replication buffer will become smaller when no replica uses dualchannel $dualchannel" { # Make sure replica3 catch up with the master wait_for_ofs_sync $master $replica3 @@ -71,12 +86,18 @@ start_server {} { } else { fail "replica2 doesn't disconnect with master" } - assert {[expr $repl_buf_mem - 1024*1024] > [s mem_total_replication_buffers]} + if {$dualchannel == "yes"} { + # master's replication buffers should not grow during dual channel replication + assert {1024*512 > [s mem_total_replication_buffers]} + } else { + assert {[expr $repl_buf_mem - 1024*1024] > [s mem_total_replication_buffers]} + } } } } } } +} # This test group aims to test replication backlog size can outgrow the backlog # limit config if there is a slow replica which keep massive replication buffers, @@ -84,6 +105,7 @@ start_server {} { # partial re-synchronization. Of course, replication backlog memory also can # become smaller when master disconnects with slow replicas since output buffer # limit is reached. +foreach dualchannel {yes no} { start_server {tags {"repl external:skip"}} { start_server {} { start_server {} { @@ -91,6 +113,7 @@ start_server {} { set replica1_pid [s -2 process_id] set replica2 [srv -1 client] set replica2_pid [s -1 process_id] + $replica1 config set dual-channel-replication-enabled $dualchannel set master [srv 0 client] set master_host [srv 0 host] @@ -99,6 +122,7 @@ start_server {} { $master config set save "" $master config set repl-backlog-size 16384 $master config set client-output-buffer-limit "replica 0 0 0" + $master config set dual-channel-replication-enabled $dualchannel # Executing 'debug digest' on master which has many keys costs much time # (especially in valgrind), this causes that replica1 and replica2 disconnect @@ -106,11 +130,13 @@ start_server {} { $master config set repl-timeout 1000 $replica1 config set repl-timeout 1000 $replica2 config set repl-timeout 1000 + $replica2 config set client-output-buffer-limit "replica 0 0 0" + $replica2 config set dual-channel-replication-enabled $dualchannel $replica1 replicaof $master_host $master_port wait_for_sync $replica1 - test {Replication backlog size can outgrow the backlog limit config} { + test "Replication backlog size can outgrow the backlog limit config dualchannel $dualchannel" { # Generating RDB will take 1000 seconds $master config set rdb-key-save-delay 1000000 populate 1000 master 10000 @@ -124,7 +150,7 @@ start_server {} { } # Replication actual backlog grow more than backlog setting since # the slow replica2 kept replication buffer. - populate 10000 master 10000 + populate 20000 master 10000 assert {[s repl_backlog_histlen] > [expr 10000*10000]} } @@ -135,7 +161,7 @@ start_server {} { fail "Replica offset didn't catch up with the master after too long time" } - test {Replica could use replication buffer (beyond backlog config) for partial resynchronization} { + test "Replica could use replication buffer (beyond backlog config) for partial resynchronization dualchannel $dualchannel" { # replica1 disconnects with master $replica1 replicaof [srv -1 host] [srv -1 port] # Write a mass of data that exceeds repl-backlog-size @@ -151,22 +177,36 @@ start_server {} { # replica2 still waits for bgsave ending assert {[s rdb_bgsave_in_progress] eq {1} && [lindex [$replica2 role] 3] eq {sync}} # master accepted replica1 partial resync - assert_equal [s sync_partial_ok] {1} + if { $dualchannel == "yes" } { + # 2 psync using main channel + # +1 "real" psync + assert_equal [s sync_partial_ok] {3} + } else { + assert_equal [s sync_partial_ok] {1} + } assert_equal [$master debug digest] [$replica1 debug digest] } - test {Replication backlog memory will become smaller if disconnecting with replica} { + test "Replication backlog memory will become smaller if disconnecting with replica dualchannel $dualchannel" { assert {[s repl_backlog_histlen] > [expr 2*10000*10000]} - assert_equal [s connected_slaves] {2} + if {$dualchannel == "yes"} { + # 1 connection of replica1 + # +2 connections during sync of replica2 + assert_equal [s connected_slaves] {3} + } else { + assert_equal [s connected_slaves] {2} + } pause_process $replica2_pid r config set client-output-buffer-limit "replica 128k 0 0" # trigger output buffer limit check - r set key [string repeat A [expr 64*1024]] + r set key [string repeat A [expr 64*2048]] # master will close replica2's connection since replica2's output # buffer limit is reached, so there only is replica1. wait_for_condition 100 100 { - [s connected_slaves] eq {1} + [s connected_slaves] eq {1} || + ([s connected_slaves] eq {2} && + [string match {*slave*state=wait_bgsave*type=rdb-channel*} [$master info]]) } else { fail "master didn't disconnect with replica2" } @@ -185,15 +225,19 @@ start_server {} { } } } +} -test {Partial resynchronization is successful even client-output-buffer-limit is less than repl-backlog-size} { +foreach dualchannel {"yes" "no"} { +test "Partial resynchronization is successful even client-output-buffer-limit is less than repl-backlog-size. dualchannel $dualchannel" { start_server {tags {"repl external:skip"}} { start_server {} { r config set save "" r config set repl-backlog-size 100mb r config set client-output-buffer-limit "replica 512k 0 0" + r config set dual-channel-replication-enabled $dualchannel set replica [srv -1 client] + $replica config set dual-channel-replication-enabled $dualchannel $replica replicaof [srv 0 host] [srv 0 port] wait_for_sync $replica @@ -210,8 +254,13 @@ test {Partial resynchronization is successful even client-output-buffer-limit is r set key $big_str ;# trigger output buffer limit check wait_for_ofs_sync r $replica # master accepted replica partial resync + set psync_count 1 + if {$dualchannel == "yes"} { + # One fake and one real psync + set psync_count 2 + } assert_equal [s sync_full] {1} - assert_equal [s sync_partial_ok] {1} + assert_equal [s sync_partial_ok] $psync_count r multi r set key $big_str @@ -225,13 +274,13 @@ test {Partial resynchronization is successful even client-output-buffer-limit is fail "Replica offset didn't catch up with the master after too long time" } assert_equal [s sync_full] {1} - assert_equal [s sync_partial_ok] {1} + assert_equal [s sync_partial_ok] $psync_count } } } # This test was added to make sure big keys added to the backlog do not trigger psync loop. -test {Replica client-output-buffer size is limited to backlog_limit/16 when no replication data is pending} { +test "Replica client-output-buffer size is limited to backlog_limit/16 when no replication data is pending. dualchannel $dualchannel" { proc client_field {r type f} { set client [$r client list type $type] if {![regexp $f=(\[a-zA-Z0-9-\]+) $client - res]} { @@ -252,6 +301,8 @@ test {Replica client-output-buffer size is limited to backlog_limit/16 when no r $master config set repl-backlog-size 16384 $master config set client-output-buffer-limit "replica 32768 32768 60" + $master config set dual-channel-replication-enabled $dualchannel + $replica config set dual-channel-replication-enabled $dualchannel # Key has has to be larger than replica client-output-buffer limit. set keysize [expr 256*1024] @@ -290,7 +341,11 @@ test {Replica client-output-buffer size is limited to backlog_limit/16 when no r # now we expect the replica to re-connect but fail partial sync (it doesn't have large # enough COB limit and must result in a full-sync) - assert {[status $master sync_partial_ok] == 0} + if {$dualchannel == "yes"} { + assert {[status $master sync_partial_ok] == [status $master sync_full]} + } else { + assert {[status $master sync_partial_ok] == 0} + } # Before this fix (#11905), the test would trigger an assertion in 'o->used >= c->ref_block_pos' test {The update of replBufBlock's repl_offset is ok - Regression test for #11666} { @@ -304,4 +359,5 @@ test {Replica client-output-buffer size is limited to backlog_limit/16 when no r } } } +} diff --git a/tests/integration/replication-psync.tcl b/tests/integration/replication-psync.tcl index dc1df0fa62..4c305ebff4 100644 --- a/tests/integration/replication-psync.tcl +++ b/tests/integration/replication-psync.tcl @@ -8,7 +8,7 @@ # If reconnect is > 0, the test actually try to break the connection and # reconnect with the master, otherwise just the initial synchronization is # checked for consistency. -proc test_psync {descr duration backlog_size backlog_ttl delay cond mdl sdl reconnect} { +proc test_psync {descr duration backlog_size backlog_ttl delay cond mdl sdl dualchannel reconnect} { start_server {tags {"repl"} overrides {save {}}} { start_server {overrides {save {}}} { @@ -21,7 +21,9 @@ proc test_psync {descr duration backlog_size backlog_ttl delay cond mdl sdl reco $master config set repl-backlog-ttl $backlog_ttl $master config set repl-diskless-sync $mdl $master config set repl-diskless-sync-delay 1 + $master config set dual-channel-replication-enabled $dualchannel $slave config set repl-diskless-load $sdl + $slave config set dual-channel-replication-enabled $dualchannel set load_handle0 [start_bg_complex_data $master_host $master_port 9 100000] set load_handle1 [start_bg_complex_data $master_host $master_port 11 100000] @@ -46,8 +48,8 @@ proc test_psync {descr duration backlog_size backlog_ttl delay cond mdl sdl reco } } - test "Test replication partial resync: $descr (diskless: $mdl, $sdl, reconnect: $reconnect)" { - # Now while the clients are writing data, break the maste-slave + test "Test replication partial resync: $descr (diskless: $mdl, $sdl, dual-channel: $dualchannel, reconnect: $reconnect)" { + # Now while the clients are writing data, break the master-slave # link multiple times. if ($reconnect) { for {set j 0} {$j < $duration*10} {incr j} { @@ -74,19 +76,7 @@ proc test_psync {descr duration backlog_size backlog_ttl delay cond mdl sdl reco # Wait for the slave to reach the "online" # state from the POV of the master. - set retry 5000 - while {$retry} { - set info [$master info] - if {[string match {*slave0:*state=online*} $info]} { - break - } else { - incr retry -1 - after 100 - } - } - if {$retry == 0} { - error "assertion:Slave not correctly synchronized" - } + verify_replica_online $master 0 5000 # Wait that slave acknowledge it is online so # we are sure that DBSIZE and DEBUG DIGEST will not @@ -111,6 +101,10 @@ proc test_psync {descr duration backlog_size backlog_ttl delay cond mdl sdl reco fail "Master - Replica inconsistency, Run diff -u against /tmp/repldump*.txt for more info" } assert {[$master dbsize] > 0} + # if {$descr == "no backlog" && $mdl == "yes" && $sdl == "disabled"} { + # puts "Master port: $master_port" + # after 100000000 + # } eval $cond } } @@ -120,24 +114,26 @@ proc test_psync {descr duration backlog_size backlog_ttl delay cond mdl sdl reco tags {"external:skip"} { foreach mdl {no yes} { foreach sdl {disabled swapdb} { - test_psync {no reconnection, just sync} 6 1000000 3600 0 { - } $mdl $sdl 0 + foreach dualchannel {yes no} { + test_psync {no reconnection, just sync} 6 1000000 3600 0 { + } $mdl $sdl $dualchannel 0 - test_psync {ok psync} 6 100000000 3600 0 { - assert {[s -1 sync_partial_ok] > 0} - } $mdl $sdl 1 + test_psync {ok psync} 6 100000000 3600 0 { + assert {[s -1 sync_partial_ok] > 0} + } $mdl $sdl $dualchannel 1 - test_psync {no backlog} 6 100 3600 0.5 { - assert {[s -1 sync_partial_err] > 0} - } $mdl $sdl 1 + test_psync {no backlog} 6 100 3600 0.5 { + assert {[s -1 sync_partial_err] > 0} + } $mdl $sdl $dualchannel 1 - test_psync {ok after delay} 3 100000000 3600 3 { - assert {[s -1 sync_partial_ok] > 0} - } $mdl $sdl 1 + test_psync {ok after delay} 3 100000000 3600 3 { + assert {[s -1 sync_partial_ok] > 0} + } $mdl $sdl $dualchannel 1 - test_psync {backlog expired} 3 100000000 1 3 { - assert {[s -1 sync_partial_err] > 0} - } $mdl $sdl 1 + test_psync {backlog expired} 3 100000000 1 3 { + assert {[s -1 sync_partial_err] > 0} + } $mdl $sdl $dualchannel 1 + } } } } diff --git a/tests/integration/replication.tcl b/tests/integration/replication.tcl index 9634f78252..b175f4ff34 100644 --- a/tests/integration/replication.tcl +++ b/tests/integration/replication.tcl @@ -301,7 +301,7 @@ start_server {tags {"repl external:skip"}} { } } -foreach mdl {no yes} { +foreach mdl {no yes} dualchannel {no yes} { foreach sdl {disabled swapdb} { start_server {tags {"repl external:skip"} overrides {save {}}} { set master [srv 0 client] @@ -317,7 +317,7 @@ foreach mdl {no yes} { lappend slaves [srv 0 client] start_server {overrides {save {}}} { lappend slaves [srv 0 client] - test "Connect multiple replicas at the same time (issue #141), master diskless=$mdl, replica diskless=$sdl" { + test "Connect multiple replicas at the same time (issue #141), master diskless=$mdl, replica diskless=$sdl dual-channel-replication-enabled=$dualchannel" { # start load handles only inside the test, so that the test can be skipped set load_handle0 [start_bg_complex_data $master_host $master_port 9 100000000] set load_handle1 [start_bg_complex_data $master_host $master_port 11 100000000] @@ -326,7 +326,11 @@ foreach mdl {no yes} { set load_handle4 [start_write_load $master_host $master_port 4] after 5000 ;# wait for some data to accumulate so that we have RDB part for the fork + $master config set dual-channel-replication-enabled $dualchannel # Send SLAVEOF commands to slaves + [lindex $slaves 0] config set dual-channel-replication-enabled $dualchannel + [lindex $slaves 1] config set dual-channel-replication-enabled $dualchannel + [lindex $slaves 2] config set dual-channel-replication-enabled $dualchannel [lindex $slaves 0] config set repl-diskless-load $sdl [lindex $slaves 1] config set repl-diskless-load $sdl [lindex $slaves 2] config set repl-diskless-load $sdl @@ -336,7 +340,7 @@ foreach mdl {no yes} { # Wait for all the three slaves to reach the "online" # state from the POV of the master. - set retry 500 + set retry 1000 while {$retry} { set info [r -3 info] if {[string match {*slave0:*state=online*slave1:*state=online*slave2:*state=online*} $info]} { @@ -375,6 +379,8 @@ foreach mdl {no yes} { wait_for_ofs_sync $master [lindex $slaves 1] wait_for_ofs_sync $master [lindex $slaves 2] + assert [string match *replicas_waiting_psync:0* [$master info replication]] + # Check digests set digest [$master debug digest] set digest0 [[lindex $slaves 0] debug digest] @@ -436,7 +442,7 @@ start_server {tags {"repl external:skip"} overrides {save {}}} { } # Diskless load swapdb when NOT async_loading (different master replid) -foreach testType {Successful Aborted} { +foreach testType {Successful Aborted} dualchannel {yes no} { start_server {tags {"repl external:skip"}} { set replica [srv 0 client] set replica_host [srv 0 host] @@ -451,8 +457,10 @@ foreach testType {Successful Aborted} { $master config set repl-diskless-sync yes $master config set repl-diskless-sync-delay 0 $master config set save "" + $master config set dual-channel-replication-enabled $dualchannel $replica config set repl-diskless-load swapdb $replica config set save "" + $replica config set dual-channel-replication-enabled $dualchannel # Put different data sets on the master and replica # We need to put large keys on the master since the replica replies to info only once in 2mb @@ -472,7 +480,7 @@ foreach testType {Successful Aborted} { # Start the replication process $replica replicaof $master_host $master_port - test {Diskless load swapdb (different replid): replica enter loading} { + test "Diskless load swapdb (different replid): replica enter loading dual-channel-replication-enabled=$dualchannel" { # Wait for the replica to start reading the rdb wait_for_condition 100 100 { [s -1 loading] eq 1 @@ -496,7 +504,7 @@ foreach testType {Successful Aborted} { fail "Replica didn't disconnect" } - test {Diskless load swapdb (different replid): old database is exposed after replication fails} { + test "Diskless load swapdb (different replid): old database is exposed after replication fails dual-channel=$dualchannel" { # Ensure we see old values from replica assert_equal [$replica get mykey] "myvalue" @@ -518,7 +526,7 @@ foreach testType {Successful Aborted} { fail "Master <-> Replica didn't finish sync" } - test {Diskless load swapdb (different replid): new database is exposed after swapping} { + test "Diskless load swapdb (different replid): new database is exposed after swapping dual-channel=$dualchannel" { # Ensure we don't see anymore the key that was stored only to replica and also that we don't get LOADING status assert_equal [$replica GET mykey] "" @@ -549,6 +557,7 @@ foreach testType {Successful Aborted} { $master config set save "" $replica config set repl-diskless-load swapdb $replica config set save "" + $replica config set dual-channel-replication-enabled no; # Doesn't work with swapdb # Set replica writable so we can check that a key we manually added is served # during replication and after failure, but disappears on success @@ -853,6 +862,7 @@ start_server {tags {"repl external:skip"} overrides {save ""}} { $master config set repl-diskless-sync yes $master config set repl-diskless-sync-delay 5 $master config set repl-diskless-sync-max-replicas 2 + $master config set dual-channel-replication-enabled "no"; # dual-channel-replication doesn't use pipe set master_host [srv 0 host] set master_port [srv 0 port] set master_pid [srv 0 pid] @@ -1042,8 +1052,8 @@ test "diskless replication child being killed is collected" { } } {} {external:skip} -foreach mdl {yes no} { - test "replication child dies when parent is killed - diskless: $mdl" { +foreach mdl {yes no} dualchannel {yes no} { + test "replication child dies when parent is killed - diskless: $mdl dual-channel-replication-enabled: $dualchannel" { # when master is killed, make sure the fork child can detect that and exit start_server {tags {"repl"} overrides {save ""}} { set master [srv 0 client] @@ -1057,6 +1067,7 @@ foreach mdl {yes no} { $master debug populate 10000 start_server {overrides {save ""}} { set replica [srv 0 client] + $replica config set dual-channel-replication-enabled $dualchannel $replica replicaof $master_host $master_port # wait for rdb child to start @@ -1236,69 +1247,80 @@ test {Kill rdb child process if its dumping RDB is not useful} { } } } {} {external:skip} - -start_server {tags {"repl external:skip"}} { - set master1_host [srv 0 host] - set master1_port [srv 0 port] - r set a b - - start_server {} { - set master2 [srv 0 client] - set master2_host [srv 0 host] - set master2_port [srv 0 port] - # Take 10s for dumping RDB - $master2 debug populate 10 master2 10 - $master2 config set rdb-key-save-delay 1000000 +foreach dualchannel {yes no} { + start_server {tags {"repl external:skip"}} { + set master1 [srv 0 client] + set master1_host [srv 0 host] + set master1_port [srv 0 port] + $master1 config set dual-channel-replication-enabled $dualchannel + r set a b start_server {} { - set sub_replica [srv 0 client] + set master2 [srv 0 client] + set master2_host [srv 0 host] + set master2_port [srv 0 port] + # Take 10s for dumping RDB + $master2 debug populate 10 master2 10 + $master2 config set rdb-key-save-delay 1000000 + $master2 config set dual-channel-replication-enabled $dualchannel start_server {} { - # Full sync with master1 - r slaveof $master1_host $master1_port - wait_for_sync r - assert_equal "b" [r get a] - - # Let sub replicas sync with me - $sub_replica slaveof [srv 0 host] [srv 0 port] - wait_for_sync $sub_replica - assert_equal "b" [$sub_replica get a] - - # Full sync with master2, and then kill master2 before finishing dumping RDB - r slaveof $master2_host $master2_port - wait_for_condition 50 100 { - ([s -2 rdb_bgsave_in_progress] == 1) && - ([string match "*wait_bgsave*" [s -2 slave0]]) - } else { - fail "full sync didn't start" - } - catch {$master2 shutdown nosave} - - test {Don't disconnect with replicas before loading transferred RDB when full sync} { - assert ![log_file_matches [srv -1 stdout] "*Connection with master lost*"] - # The replication id is not changed in entire replication chain - assert_equal [s master_replid] [s -3 master_replid] - assert_equal [s master_replid] [s -1 master_replid] - } + set sub_replica [srv 0 client] + $sub_replica config set dual-channel-replication-enabled $dualchannel - test {Discard cache master before loading transferred RDB when full sync} { - set full_sync [s -3 sync_full] - set partial_sync [s -3 sync_partial_ok] - # Partial sync with master1 + start_server {} { + # Full sync with master1 + set replica [srv 0 client] + $replica config set dual-channel-replication-enabled $dualchannel r slaveof $master1_host $master1_port wait_for_sync r - # master1 accepts partial sync instead of full sync - assert_equal $full_sync [s -3 sync_full] - assert_equal [expr $partial_sync+1] [s -3 sync_partial_ok] - - # Since master only partially sync replica, and repl id is not changed, - # the replica doesn't disconnect with its sub-replicas - assert_equal [s master_replid] [s -3 master_replid] - assert_equal [s master_replid] [s -1 master_replid] - assert ![log_file_matches [srv -1 stdout] "*Connection with master lost*"] - # Sub replica just has one full sync, no partial resync. - assert_equal 1 [s sync_full] - assert_equal 0 [s sync_partial_ok] + assert_equal "b" [r get a] + + # Let sub replicas sync with me + $sub_replica slaveof [srv 0 host] [srv 0 port] + wait_for_sync $sub_replica + assert_equal "b" [$sub_replica get a] + + # Full sync with master2, and then kill master2 before finishing dumping RDB + r slaveof $master2_host $master2_port + wait_for_condition 50 100 { + ([s -2 rdb_bgsave_in_progress] == 1) && + ([string match "*wait_bgsave*" [s -2 slave0]]) + } else { + fail "full sync didn't start" + } + catch {$master2 shutdown nosave} + + test "Don't disconnect with replicas before loading transferred RDB when full sync with dual-channel-replication $dualchannel" { + assert ![log_file_matches [srv -1 stdout] "*Connection with master lost*"] + # The replication id is not changed in entire replication chain + assert_equal [s master_replid] [s -3 master_replid] + assert_equal [s master_replid] [s -1 master_replid] + } + + test "Discard cache master before loading transferred RDB when full sync with dual-channel-replication $dualchannel" { + set full_sync [s -3 sync_full] + set partial_sync [s -3 sync_partial_ok] + # Partial sync with master1 + r slaveof $master1_host $master1_port + wait_for_sync r + # master1 accepts partial sync instead of full sync + assert_equal $full_sync [s -3 sync_full] + assert_equal [expr $partial_sync+1] [s -3 sync_partial_ok] + + # Since master only partially sync replica, and repl id is not changed, + # the replica doesn't disconnect with its sub-replicas + assert_equal [s master_replid] [s -3 master_replid] + assert_equal [s master_replid] [s -1 master_replid] + assert ![log_file_matches [srv -1 stdout] "*Connection with master lost*"] + # Sub replica just has one full sync, no partial resync. + assert_equal 1 [s sync_full] + if {$dualchannel == "yes"} { + assert_equal 1 [s sync_partial_ok] + } else { + assert_equal 0 [s sync_partial_ok] + } + } } } } diff --git a/tests/support/test.tcl b/tests/support/test.tcl index bb59ee7972..262dc66041 100644 --- a/tests/support/test.tcl +++ b/tests/support/test.tcl @@ -141,6 +141,34 @@ proc wait_for_condition {maxtries delay e _else_ elsescript} { } } +proc verify_replica_online {master replica_idx max_retry} { + set pause 100 + set count_down $max_retry + while {$count_down} { + set info [$master info] + set pattern *slave$replica_idx:*state=online* + if {[string match $pattern $info]} { + break + } else { + incr count_down -1 + after $pause + } + } + if {$count_down == 0} { + set threshold [expr {$max_retry*$pause/1000}] + error "assertion:Replica is not in sync after $threshold seconds" + } +} + +proc wait_for_value_to_propegate_to_replica {master replica key} { + set val [$master get $key] + wait_for_condition 50 500 { + ([$replica get $key] eq $val) + } else { + error "Key $key did not propegate. Expected $val but got [$replica get $key]" + } +} + # try to match a value to a list of patterns that are either regex (starts with "/") or plain string. # The caller can specify to use only glob-pattern match proc search_pattern_list {value pattern_list {glob_pattern false}} { diff --git a/tests/support/util.tcl b/tests/support/util.tcl index c6c405b191..a8948338ea 100644 --- a/tests/support/util.tcl +++ b/tests/support/util.tcl @@ -555,7 +555,15 @@ proc find_valgrind_errors {stderr on_termination} { # of seconds to the specified the server instance. proc start_write_load {host port seconds} { set tclsh [info nameofexecutable] - exec $tclsh tests/helpers/gen_write_load.tcl $host $port $seconds $::tls & + exec $tclsh tests/helpers/gen_write_load.tcl $host $port $seconds $::tls "" & +} + +# Execute a background process writing only one key for the specified number +# of seconds to the specified Redis instance. This load handler is useful for +# tests which requires heavy replication stream but no memory load. +proc start_one_key_write_load {host port seconds key} { + set tclsh [info nameofexecutable] + exec $tclsh tests/helpers/gen_write_load.tcl $host $port $seconds $::tls $key & } # Stop a process generating write load executed with start_write_load. diff --git a/tests/unit/auth.tcl b/tests/unit/auth.tcl index 5c2071c176..5654bf7763 100644 --- a/tests/unit/auth.tcl +++ b/tests/unit/auth.tcl @@ -47,6 +47,7 @@ start_server {tags {"auth external:skip"} overrides {requirepass foobar}} { } } +foreach dualchannel {yes no} { start_server {tags {"auth_binary_password external:skip"}} { test {AUTH fails when binary password is wrong} { r config set requirepass "abc\x00def" @@ -65,12 +66,13 @@ start_server {tags {"auth_binary_password external:skip"}} { set master_port [srv -1 port] set slave [srv 0 client] - test {primaryauth test with binary password} { + test "primaryauth test with binary password dualchannel = $dualchannel" { $master config set requirepass "abc\x00def" - + $master config set dual-channel-replication-enabled $dualchannel # Configure the replica with primaryauth set loglines [count_log_lines 0] $slave config set primaryauth "abc" + $slave config set dual-channel-replication-enabled $dualchannel $slave slaveof $master_host $master_port # Verify replica is not able to sync with master @@ -87,3 +89,4 @@ start_server {tags {"auth_binary_password external:skip"}} { } } } +} diff --git a/tests/unit/introspection.tcl b/tests/unit/introspection.tcl index a12a3ba23d..5396cd2e56 100644 --- a/tests/unit/introspection.tcl +++ b/tests/unit/introspection.tcl @@ -558,6 +558,7 @@ start_server {tags {"introspection"}} { socket-mark-id req-res-logfile client-default-resp + dual-channel-replication-enabled } if {!$::tls} { diff --git a/valkey.conf b/valkey.conf index f26a19b3da..bb007fe3e4 100644 --- a/valkey.conf +++ b/valkey.conf @@ -668,6 +668,24 @@ repl-diskless-sync-max-replicas 0 # during replication. repl-diskless-load disabled +# This dual channel replication sync feature optimizes the full synchronization process +# between a primary and its replicas. When enabled, it reduces both memory and CPU load +# on the primary server. +# +# How it works: +# 1. During full sync, instead of accumulating replication data on the primary server, +# the data is sent directly to the syncing replica. +# 2. The primary's background save (bgsave) process streams the RDB snapshot directly +# to the replica over a separate connection. +# +# Tradeoff: +# While this approach reduces load on the primary, it shifts the burden of storing +# the replication buffer to the replica. This means the replica must have sufficient +# memory to accommodate the buffer during synchronization. However, this tradeoff is +# generally beneficial as it prevents potential performance degradation on the primary +# server, which is typically handling more critical operations. +dual-channel-replication-enabled no + # Master send PINGs to its replicas in a predefined interval. It's possible to # change this interval with the repl_ping_replica_period option. The default # value is 10 seconds.