Unverified Commit 7ac21307 authored by guybe7's avatar guybe7 Committed by GitHub
Browse files

Sort out mess around propagation and MULTI/EXEC (#9890)

The mess:
Some parts use alsoPropagate for late propagation, others using an immediate one (propagate()),
causing edge cases, ugly/hacky code, and the tendency for bugs

The basic idea is that all commands are propagated via alsoPropagate (i.e. added to a list) and the
top-most call() is responsible for going over that list and actually propagating them (and wrapping
them in MULTI/EXEC if there's more than one command). This is done in the new function,
propagatePendingCommands.

Callers to propagatePendingCommands:
1. top-most call() (we want all nested call()s to add to the also_propagate array and just the top-most
   one to propagate them) - via `afterCommand`
2. handleClientsBlockedOnKeys: it is out of call() context and it may propagate stuff - via `afterCommand`. 
3. handleClientsBlockedOnKeys edge case: if the looked-up key is already expired, we will propagate the
   expire but will not unblock any client so `afterCommand` isn't called. in that case, we have to propagate
   the deletion explicitly.
4. cron stuff: active-expire and eviction may also propagate stuff
5. modules: the module API allows to propagate stuff from just about anywhere (timers, keyspace notifications,
   threads). I could have tried to catch all the out-of-call-context places but it seemed easier to handle it in one
   place: when we free the context. in the spirit of what was done in call(), only the top-most freeing of a module
   context may cause propagation.
6. modules: when using a thread-safe ctx it's not clear when/if the ctx will be freed. we do know that the module
   must lock the GIL before calling RM_Replicate/RM_Call so we propagate the pending commands when
   releasing the GIL.

A "known limitation", which were actually a bug, was fixed because of this commit (see propagate.tcl):
   When using a mix of RM_Call with `!` and RM_Replicate, the command would propagate out-of-order:
   first all the commands from RM_Call, and then the ones from RM_Replicate

Another thing worth mentioning is that if, in the past, a client would issue a MULTI/EXEC with just one
write command the server would blindly propagate the MULTI/EXEC too, even though it's redundant.
not anymore.

This commit renames propagate() to propagateNow() in order to cause conflicts in pending PRs.
propagatePendingCommands is the only caller of propagateNow, which is now a static, internal helper function.

Optimizations:
1. alsoPropagate will not add stuff to also_propagate if there's no AOF and replicas
2. alsoPropagate reallocs also_propagagte exponentially, to save calls to memmove

Bugfixes:
1. CONFIG SET can create evictions, sending notifications which can cause to dirty++ with modules.
   we need to prevent it from propagating to AOF/replicas
2. We need to set current_client in RM_Call. buggy scenario:
   - CONFIG SET maxmemory, eviction notifications, module hook calls RM_Call
   - assertion in lookupKey crashes, because current_client has CONFIG SET, which isn't CMD_WRITE
3. minor: in eviction, call propagateDeletion after notification, like active-expire and all commands
   (we always send a notification before propagating the command)
parent b7567394
...@@ -628,6 +628,8 @@ sds genAofTimestampAnnotationIfNeeded(int force) { ...@@ -628,6 +628,8 @@ sds genAofTimestampAnnotationIfNeeded(int force) {
void feedAppendOnlyFile(int dictid, robj **argv, int argc) { void feedAppendOnlyFile(int dictid, robj **argv, int argc) {
sds buf = sdsempty(); sds buf = sdsempty();
serverAssert(dictid >= 0 && dictid < server.dbnum);
/* Feed timestamp if needed */ /* Feed timestamp if needed */
if (server.aof_timestamp_enabled) { if (server.aof_timestamp_enabled) {
sds ts = genAofTimestampAnnotationIfNeeded(0); sds ts = genAofTimestampAnnotationIfNeeded(0);
......
...@@ -352,10 +352,6 @@ void serveClientsBlockedOnSortedSetKey(robj *o, readyList *rl) { ...@@ -352,10 +352,6 @@ void serveClientsBlockedOnSortedSetKey(robj *o, readyList *rl) {
monotime replyTimer; monotime replyTimer;
elapsedStart(&replyTimer); elapsedStart(&replyTimer);
genericZpopCommand(receiver, &rl->key, 1, where, 1, count, use_nested_array, reply_nil_when_empty, &deleted); genericZpopCommand(receiver, &rl->key, 1, where, 1, count, use_nested_array, reply_nil_when_empty, &deleted);
updateStatsOnUnblock(receiver, 0, elapsedUs(replyTimer));
unblockClient(receiver);
afterCommand(receiver);
server.current_client = old_client;
/* Replicate the command. */ /* Replicate the command. */
int argc = 2; int argc = 2;
...@@ -369,10 +365,15 @@ void serveClientsBlockedOnSortedSetKey(robj *o, readyList *rl) { ...@@ -369,10 +365,15 @@ void serveClientsBlockedOnSortedSetKey(robj *o, readyList *rl) {
argv[2] = count_obj; argv[2] = count_obj;
argc++; argc++;
} }
propagate(receiver->db->id, argv, argc, PROPAGATE_AOF|PROPAGATE_REPL); alsoPropagate(receiver->db->id, argv, argc, PROPAGATE_AOF|PROPAGATE_REPL);
decrRefCount(argv[1]); decrRefCount(argv[1]);
if (count != -1) decrRefCount(argv[2]); if (count != -1) decrRefCount(argv[2]);
updateStatsOnUnblock(receiver, 0, elapsedUs(replyTimer));
unblockClient(receiver);
afterCommand(receiver);
server.current_client = old_client;
/* The zset is empty and has been deleted. */ /* The zset is empty and has been deleted. */
if (deleted) break; if (deleted) break;
} }
...@@ -474,7 +475,6 @@ void serveClientsBlockedOnStreamKey(robj *o, readyList *rl) { ...@@ -474,7 +475,6 @@ void serveClientsBlockedOnStreamKey(robj *o, readyList *rl) {
receiver->bpop.xread_count, receiver->bpop.xread_count,
0, group, consumer, noack, &pi); 0, group, consumer, noack, &pi);
updateStatsOnUnblock(receiver, 0, elapsedUs(replyTimer)); updateStatsOnUnblock(receiver, 0, elapsedUs(replyTimer));
/* Note that after we unblock the client, 'gt' /* Note that after we unblock the client, 'gt'
* and other receiver->bpop stuff are no longer * and other receiver->bpop stuff are no longer
* valid, so we must do the setup above before * valid, so we must do the setup above before
...@@ -532,7 +532,6 @@ void serveClientsBlockedOnKeyByModule(readyList *rl) { ...@@ -532,7 +532,6 @@ void serveClientsBlockedOnKeyByModule(readyList *rl) {
elapsedStart(&replyTimer); elapsedStart(&replyTimer);
if (!moduleTryServeClientBlockedOnKey(receiver, rl->key)) continue; if (!moduleTryServeClientBlockedOnKey(receiver, rl->key)) continue;
updateStatsOnUnblock(receiver, 0, elapsedUs(replyTimer)); updateStatsOnUnblock(receiver, 0, elapsedUs(replyTimer));
moduleUnblockClient(receiver); moduleUnblockClient(receiver);
afterCommand(receiver); afterCommand(receiver);
server.current_client = old_client; server.current_client = old_client;
...@@ -562,6 +561,11 @@ void serveClientsBlockedOnKeyByModule(readyList *rl) { ...@@ -562,6 +561,11 @@ void serveClientsBlockedOnKeyByModule(readyList *rl) {
* be used only for a single type, like virtually any Redis application will * be used only for a single type, like virtually any Redis application will
* do, the function is already fair. */ * do, the function is already fair. */
void handleClientsBlockedOnKeys(void) { void handleClientsBlockedOnKeys(void) {
/* This function is called only when also_propagate is in its basic state
* (i.e. not from call(), module context, etc.) */
serverAssert(server.also_propagate.numops == 0);
server.core_propagates = 1;
while(listLength(server.ready_keys) != 0) { while(listLength(server.ready_keys) != 0) {
list *l; list *l;
...@@ -603,6 +607,11 @@ void handleClientsBlockedOnKeys(void) { ...@@ -603,6 +607,11 @@ void handleClientsBlockedOnKeys(void) {
* regardless of the object type: we don't know what the * regardless of the object type: we don't know what the
* module is trying to accomplish right now. */ * module is trying to accomplish right now. */
serveClientsBlockedOnKeyByModule(rl); serveClientsBlockedOnKeyByModule(rl);
} else {
/* Edge case: If lookupKeyReadWithFlags decides to expire the key we have to
* take care of the propagation here, because afterCommand wasn't called */
if (server.also_propagate.numops > 0)
propagatePendingCommands();
} }
server.fixed_time_expire--; server.fixed_time_expire--;
...@@ -613,6 +622,10 @@ void handleClientsBlockedOnKeys(void) { ...@@ -613,6 +622,10 @@ void handleClientsBlockedOnKeys(void) {
} }
listRelease(l); /* We have the new list on place at this point. */ listRelease(l); /* We have the new list on place at this point. */
} }
serverAssert(server.core_propagates); /* This function should not be re-entrant */
server.core_propagates = 0;
} }
/* This is how the current blocking lists/sorted sets/streams work, we use /* This is how the current blocking lists/sorted sets/streams work, we use
......
...@@ -2207,6 +2207,9 @@ static int updateMaxmemory(const char **err) { ...@@ -2207,6 +2207,9 @@ static int updateMaxmemory(const char **err) {
} }
performEvictions(); performEvictions();
} }
/* The function is called via 'CONFIG SET maxmemory', we don't want to propagate it
* because server.dirty might have been incremented by performEvictions() */
preventCommandPropagation(server.current_client);
return 1; return 1;
} }
......
...@@ -1466,7 +1466,7 @@ void deleteExpiredKeyAndPropagate(redisDb *db, robj *keyobj) { ...@@ -1466,7 +1466,7 @@ void deleteExpiredKeyAndPropagate(redisDb *db, robj *keyobj) {
latencyAddSampleIfNeeded("expire-del",expire_latency); latencyAddSampleIfNeeded("expire-del",expire_latency);
notifyKeyspaceEvent(NOTIFY_EXPIRED,"expired",keyobj,db->id); notifyKeyspaceEvent(NOTIFY_EXPIRED,"expired",keyobj,db->id);
signalModifiedKey(NULL, db, keyobj); signalModifiedKey(NULL, db, keyobj);
propagateExpire(db,keyobj,server.lazyfree_lazy_expire); propagateDeletion(db,keyobj,server.lazyfree_lazy_expire);
server.stat_expiredkeys++; server.stat_expiredkeys++;
} }
...@@ -1477,8 +1477,18 @@ void deleteExpiredKeyAndPropagate(redisDb *db, robj *keyobj) { ...@@ -1477,8 +1477,18 @@ void deleteExpiredKeyAndPropagate(redisDb *db, robj *keyobj) {
* This way the key expiry is centralized in one place, and since both * This way the key expiry is centralized in one place, and since both
* AOF and the master->slave link guarantee operation ordering, everything * AOF and the master->slave link guarantee operation ordering, everything
* will be consistent even if we allow write operations against expiring * will be consistent even if we allow write operations against expiring
* keys. */ * keys.
void propagateExpire(redisDb *db, robj *key, int lazy) { *
* This function may be called from:
* 1. Within call(): Example: Lazy-expire on key access.
* In this case the caller doesn't have to do anything
* because call() handles server.also_propagate(); or
* 2. Outside of call(): Example: Active-expire, eviction.
* In this the caller must remember to call
* propagatePendingCommands, preferably at the end of
* the deletion batch, so that DELs will be wrapped
* in MULTI/EXEC */
void propagateDeletion(redisDb *db, robj *key, int lazy) {
robj *argv[2]; robj *argv[2];
argv[0] = lazy ? shared.unlink : shared.del; argv[0] = lazy ? shared.unlink : shared.del;
...@@ -1490,7 +1500,7 @@ void propagateExpire(redisDb *db, robj *key, int lazy) { ...@@ -1490,7 +1500,7 @@ void propagateExpire(redisDb *db, robj *key, int lazy) {
* Even if module executed a command without asking for propagation. */ * Even if module executed a command without asking for propagation. */
int prev_replication_allowed = server.replication_allowed; int prev_replication_allowed = server.replication_allowed;
server.replication_allowed = 1; server.replication_allowed = 1;
propagate(db->id,argv,2,PROPAGATE_AOF|PROPAGATE_REPL); alsoPropagate(db->id,argv,2,PROPAGATE_AOF|PROPAGATE_REPL);
server.replication_allowed = prev_replication_allowed; server.replication_allowed = prev_replication_allowed;
decrRefCount(argv[0]); decrRefCount(argv[0]);
......
...@@ -484,6 +484,10 @@ static int isSafeToPerformEvictions(void) { ...@@ -484,6 +484,10 @@ static int isSafeToPerformEvictions(void) {
* expires and evictions of keys not being performed. */ * expires and evictions of keys not being performed. */
if (checkClientPauseTimeoutAndReturnIfPaused()) return 0; if (checkClientPauseTimeoutAndReturnIfPaused()) return 0;
/* We cannot evict if we already have stuff to propagate (for example,
* CONFIG SET maxmemory inside a MULTI/EXEC) */
if (server.also_propagate.numops != 0) return 0;
return 1; return 1;
} }
...@@ -561,6 +565,13 @@ int performEvictions(void) { ...@@ -561,6 +565,13 @@ int performEvictions(void) {
monotime evictionTimer; monotime evictionTimer;
elapsedStart(&evictionTimer); elapsedStart(&evictionTimer);
/* Unlike active-expire and blocked client, we can reach here from 'CONFIG SET maxmemory'
* so we have to back-up and restore server.core_propagates. */
int prev_core_propagates = server.core_propagates;
serverAssert(server.also_propagate.numops == 0);
server.core_propagates = 1;
server.propagate_no_multi = 1;
while (mem_freed < (long long)mem_tofree) { while (mem_freed < (long long)mem_tofree) {
int j, k, i; int j, k, i;
static unsigned int next_db = 0; static unsigned int next_db = 0;
...@@ -648,7 +659,6 @@ int performEvictions(void) { ...@@ -648,7 +659,6 @@ int performEvictions(void) {
if (bestkey) { if (bestkey) {
db = server.db+bestdbid; db = server.db+bestdbid;
robj *keyobj = createStringObject(bestkey,sdslen(bestkey)); robj *keyobj = createStringObject(bestkey,sdslen(bestkey));
propagateExpire(db,keyobj,server.lazyfree_lazy_eviction);
/* We compute the amount of memory freed by db*Delete() alone. /* We compute the amount of memory freed by db*Delete() alone.
* It is possible that actually the memory needed to propagate * It is possible that actually the memory needed to propagate
* the DEL in AOF and replication link is greater than the one * the DEL in AOF and replication link is greater than the one
...@@ -673,6 +683,7 @@ int performEvictions(void) { ...@@ -673,6 +683,7 @@ int performEvictions(void) {
signalModifiedKey(NULL,db,keyobj); signalModifiedKey(NULL,db,keyobj);
notifyKeyspaceEvent(NOTIFY_EVICTED, "evicted", notifyKeyspaceEvent(NOTIFY_EVICTED, "evicted",
keyobj, db->id); keyobj, db->id);
propagateDeletion(db,keyobj,server.lazyfree_lazy_eviction);
decrRefCount(keyobj); decrRefCount(keyobj);
keys_freed++; keys_freed++;
...@@ -729,6 +740,14 @@ cant_free: ...@@ -729,6 +740,14 @@ cant_free:
} }
} }
serverAssert(server.core_propagates); /* This function should not be re-entrant */
/* Propagate all DELs */
propagatePendingCommands();
server.core_propagates = prev_core_propagates;
server.propagate_no_multi = 0;
latencyEndMonitor(latency); latencyEndMonitor(latency);
latencyAddSampleIfNeeded("eviction-cycle",latency); latencyAddSampleIfNeeded("eviction-cycle",latency);
......
...@@ -182,6 +182,12 @@ void activeExpireCycle(int type) { ...@@ -182,6 +182,12 @@ void activeExpireCycle(int type) {
long total_sampled = 0; long total_sampled = 0;
long total_expired = 0; long total_expired = 0;
/* Sanity: There can't be any pending commands to propagate when
* we're in cron */
serverAssert(server.also_propagate.numops == 0);
server.core_propagates = 1;
server.propagate_no_multi = 1;
for (j = 0; j < dbs_per_call && timelimit_exit == 0; j++) { for (j = 0; j < dbs_per_call && timelimit_exit == 0; j++) {
/* Expired and checked in a single loop. */ /* Expired and checked in a single loop. */
unsigned long expired, sampled; unsigned long expired, sampled;
...@@ -302,6 +308,14 @@ void activeExpireCycle(int type) { ...@@ -302,6 +308,14 @@ void activeExpireCycle(int type) {
(expired*100/sampled) > config_cycle_acceptable_stale); (expired*100/sampled) > config_cycle_acceptable_stale);
} }
serverAssert(server.core_propagates); /* This function should not be re-entrant */
/* Propagate all DELs */
propagatePendingCommands();
server.core_propagates = 0;
server.propagate_no_multi = 0;
elapsed = ustime()-start; elapsed = ustime()-start;
server.stat_expire_cycle_time_used += elapsed; server.stat_expire_cycle_time_used += elapsed;
latencyAddSampleIfNeeded("expire-cycle",elapsed/1000); latencyAddSampleIfNeeded("expire-cycle",elapsed/1000);
......
This diff is collapsed.
...@@ -120,30 +120,6 @@ void discardCommand(client *c) { ...@@ -120,30 +120,6 @@ void discardCommand(client *c) {
addReply(c,shared.ok); addReply(c,shared.ok);
} }
void beforePropagateMulti() {
/* Propagating MULTI */
serverAssert(!server.propagate_in_transaction);
server.propagate_in_transaction = 1;
}
void afterPropagateExec() {
/* Propagating EXEC */
serverAssert(server.propagate_in_transaction == 1);
server.propagate_in_transaction = 0;
}
/* Send a MULTI command to all the slaves and AOF file. Check the execCommand
* implementation for more information. */
void execCommandPropagateMulti(int dbid) {
beforePropagateMulti();
propagate(dbid,&shared.multi,1,PROPAGATE_AOF|PROPAGATE_REPL);
}
void execCommandPropagateExec(int dbid) {
propagate(dbid,&shared.exec,1,PROPAGATE_AOF|PROPAGATE_REPL);
afterPropagateExec();
}
/* Aborts a transaction, with a specific error message. /* Aborts a transaction, with a specific error message.
* The transaction is always aborted with -EXECABORT so that the client knows * The transaction is always aborted with -EXECABORT so that the client knows
* the server exited the multi state, but the actual reason for the abort is * the server exited the multi state, but the actual reason for the abort is
...@@ -166,7 +142,6 @@ void execCommand(client *c) { ...@@ -166,7 +142,6 @@ void execCommand(client *c) {
robj **orig_argv; robj **orig_argv;
int orig_argc, orig_argv_len; int orig_argc, orig_argv_len;
struct redisCommand *orig_cmd; struct redisCommand *orig_cmd;
int was_master = server.masterhost == NULL;
if (!(c->flags & CLIENT_MULTI)) { if (!(c->flags & CLIENT_MULTI)) {
addReplyError(c,"EXEC without MULTI"); addReplyError(c,"EXEC without MULTI");
...@@ -268,23 +243,6 @@ void execCommand(client *c) { ...@@ -268,23 +243,6 @@ void execCommand(client *c) {
c->cmd = orig_cmd; c->cmd = orig_cmd;
discardTransaction(c); discardTransaction(c);
/* Make sure the EXEC command will be propagated as well if MULTI
* was already propagated. */
if (server.propagate_in_transaction) {
int is_master = server.masterhost == NULL;
server.dirty++;
/* If inside the MULTI/EXEC block this instance was suddenly
* switched from master to slave (using the SLAVEOF command), the
* initial MULTI was propagated into the replication backlog, but the
* rest was not. We need to make sure to at least terminate the
* backlog with the final EXEC. */
if (server.repl_backlog && was_master && !is_master) {
char *execcmd = "*1\r\n$4\r\nEXEC\r\n";
feedReplicationBuffer(execcmd,strlen(execcmd));
}
afterPropagateExec();
}
server.in_exec = 0; server.in_exec = 0;
} }
......
...@@ -3563,7 +3563,7 @@ void pauseClients(mstime_t end, pause_type type) { ...@@ -3563,7 +3563,7 @@ void pauseClients(mstime_t end, pause_type type) {
/* We allow write commands that were queued /* We allow write commands that were queued
* up before and after to execute. We need * up before and after to execute. We need
* to track this state so that we don't assert * to track this state so that we don't assert
* in propagate(). */ * in propagateNow(). */
if (server.in_exec) { if (server.in_exec) {
server.client_pause_in_transaction = 1; server.client_pause_in_transaction = 1;
} }
......
...@@ -421,6 +421,10 @@ void replicationFeedSlaves(list *slaves, int dictid, robj **argv, int argc) { ...@@ -421,6 +421,10 @@ void replicationFeedSlaves(list *slaves, int dictid, robj **argv, int argc) {
int j, len; int j, len;
char llstr[LONG_STR_SIZE]; char llstr[LONG_STR_SIZE];
/* In case we propagate a command that doesn't touch keys (PING, REPLCONF) we
* pass dbid=server.slaveseldb which may be -1. */
serverAssert(dictid == -1 || (dictid >= 0 && dictid < server.dbnum));
/* If the instance is not a top level master, return ASAP: we'll just proxy /* If the instance is not a top level master, return ASAP: we'll just proxy
* the stream of data we receive from our master instead, in order to * the stream of data we receive from our master instead, in order to
* propagate *identical* replication stream. In this way this slave can * propagate *identical* replication stream. In this way this slave can
...@@ -456,8 +460,9 @@ void replicationFeedSlaves(list *slaves, int dictid, robj **argv, int argc) { ...@@ -456,8 +460,9 @@ void replicationFeedSlaves(list *slaves, int dictid, robj **argv, int argc) {
if (dictid < 0 || dictid >= PROTO_SHARED_SELECT_CMDS) if (dictid < 0 || dictid >= PROTO_SHARED_SELECT_CMDS)
decrRefCount(selectcmd); decrRefCount(selectcmd);
server.slaveseldb = dictid;
} }
server.slaveseldb = dictid;
/* Write the command to the replication buffer if any. */ /* Write the command to the replication buffer if any. */
char aux[LONG_STR_SIZE+3]; char aux[LONG_STR_SIZE+3];
......
...@@ -148,11 +148,7 @@ void scriptResetRun(scriptRunCtx *run_ctx) { ...@@ -148,11 +148,7 @@ void scriptResetRun(scriptRunCtx *run_ctx) {
unprotectClient(run_ctx->original_client); unprotectClient(run_ctx->original_client);
} }
/* emit EXEC if MULTI has been propagated. */
preventCommandPropagation(run_ctx->original_client); preventCommandPropagation(run_ctx->original_client);
if (run_ctx->flags & SCRIPT_MULTI_EMMITED) {
execCommandPropagateExec(run_ctx->original_client->db->id);
}
/* unset curr_run_ctx so we will know there is no running script */ /* unset curr_run_ctx so we will know there is no running script */
curr_run_ctx = NULL; curr_run_ctx = NULL;
...@@ -332,25 +328,6 @@ static int scriptVerifyClusterState(client *c, client *original_c, sds *err) { ...@@ -332,25 +328,6 @@ static int scriptVerifyClusterState(client *c, client *original_c, sds *err) {
return C_OK; return C_OK;
} }
static void scriptEmitMultiIfNeeded(scriptRunCtx *run_ctx) {
/* If we are using single commands replication, we need to wrap what
* we propagate into a MULTI/EXEC block, so that it will be atomic like
* a Lua script in the context of AOF and slaves. */
client *c = run_ctx->c;
if (!(run_ctx->flags & SCRIPT_MULTI_EMMITED)
&& !(run_ctx->original_client->flags & CLIENT_MULTI)
&& (run_ctx->flags & SCRIPT_WRITE_DIRTY)
&& ((run_ctx->repl_flags & PROPAGATE_AOF)
|| (run_ctx->repl_flags & PROPAGATE_REPL)))
{
execCommandPropagateMulti(run_ctx->original_client->db->id);
run_ctx->flags |= SCRIPT_MULTI_EMMITED;
/* Now we are in the MULTI context, the lua_client should be
* flag as CLIENT_MULTI. */
c->flags |= CLIENT_MULTI;
}
}
/* set RESP for a given run_ctx */ /* set RESP for a given run_ctx */
int scriptSetResp(scriptRunCtx *run_ctx, int resp) { int scriptSetResp(scriptRunCtx *run_ctx, int resp) {
if (resp != 2 && resp != 3) { if (resp != 2 && resp != 3) {
...@@ -423,8 +400,6 @@ void scriptCall(scriptRunCtx *run_ctx, robj* *argv, int argc, sds *err) { ...@@ -423,8 +400,6 @@ void scriptCall(scriptRunCtx *run_ctx, robj* *argv, int argc, sds *err) {
return; return;
} }
scriptEmitMultiIfNeeded(run_ctx);
int call_flags = CMD_CALL_SLOWLOG | CMD_CALL_STATS; int call_flags = CMD_CALL_SLOWLOG | CMD_CALL_STATS;
if (run_ctx->repl_flags & PROPAGATE_AOF) { if (run_ctx->repl_flags & PROPAGATE_AOF) {
call_flags |= CMD_CALL_PROPAGATE_AOF; call_flags |= CMD_CALL_PROPAGATE_AOF;
......
...@@ -59,12 +59,9 @@ ...@@ -59,12 +59,9 @@
/* runCtx flags */ /* runCtx flags */
#define SCRIPT_WRITE_DIRTY (1ULL<<0) /* indicate that the current script already performed a write command */ #define SCRIPT_WRITE_DIRTY (1ULL<<0) /* indicate that the current script already performed a write command */
#define SCRIPT_MULTI_EMMITED (1ULL<<2) /* indicate that we already wrote a multi command to replication/aof */
#define SCRIPT_TIMEDOUT (1ULL<<3) /* indicate that the current script timedout */ #define SCRIPT_TIMEDOUT (1ULL<<3) /* indicate that the current script timedout */
#define SCRIPT_KILLED (1ULL<<4) /* indicate that the current script was marked to be killed */ #define SCRIPT_KILLED (1ULL<<4) /* indicate that the current script was marked to be killed */
#define SCRIPT_READ_ONLY (1ULL<<5) /* indicate that the current script should only perform read commands */ #define SCRIPT_READ_ONLY (1ULL<<5) /* indicate that the current script should only perform read commands */
#define SCRIPT_EVAL_MODE (1ULL<<7) /* Indicate that the current script called from legacy Lua */ #define SCRIPT_EVAL_MODE (1ULL<<7) /* Indicate that the current script called from legacy Lua */
typedef struct scriptRunCtx scriptRunCtx; typedef struct scriptRunCtx scriptRunCtx;
......
...@@ -2314,7 +2314,9 @@ void initServer(void) { ...@@ -2314,7 +2314,9 @@ void initServer(void) {
server.cronloops = 0; server.cronloops = 0;
server.in_script = 0; server.in_script = 0;
server.in_exec = 0; server.in_exec = 0;
server.propagate_in_transaction = 0; server.core_propagates = 0;
server.propagate_no_multi = 0;
server.module_ctx_nesting = 0;
server.client_pause_in_transaction = 0; server.client_pause_in_transaction = 0;
server.child_pid = -1; server.child_pid = -1;
server.child_type = CHILD_TYPE_NONE; server.child_type = CHILD_TYPE_NONE;
...@@ -2633,12 +2635,21 @@ void resetErrorTableStats(void) { ...@@ -2633,12 +2635,21 @@ void resetErrorTableStats(void) {
void redisOpArrayInit(redisOpArray *oa) { void redisOpArrayInit(redisOpArray *oa) {
oa->ops = NULL; oa->ops = NULL;
oa->numops = 0; oa->numops = 0;
oa->capacity = 0;
} }
int redisOpArrayAppend(redisOpArray *oa, int dbid, robj **argv, int argc, int target) { int redisOpArrayAppend(redisOpArray *oa, int dbid, robj **argv, int argc, int target) {
redisOp *op; redisOp *op;
int prev_capacity = oa->capacity;
oa->ops = zrealloc(oa->ops,sizeof(redisOp)*(oa->numops+1)); if (oa->numops == 0) {
oa->capacity = 16;
} else if (oa->numops >= oa->capacity) {
oa->capacity *= 2;
}
if (prev_capacity != oa->capacity)
oa->ops = zrealloc(oa->ops,sizeof(redisOp)*oa->capacity);
op = oa->ops+oa->numops; op = oa->ops+oa->numops;
op->dbid = dbid; op->dbid = dbid;
op->argv = argv; op->argv = argv;
...@@ -2660,7 +2671,7 @@ void redisOpArrayFree(redisOpArray *oa) { ...@@ -2660,7 +2671,7 @@ void redisOpArrayFree(redisOpArray *oa) {
zfree(op->argv); zfree(op->argv);
} }
zfree(oa->ops); zfree(oa->ops);
oa->ops = NULL; redisOpArrayInit(oa);
} }
/* ====================== Commands lookup and execution ===================== */ /* ====================== Commands lookup and execution ===================== */
...@@ -2735,6 +2746,22 @@ struct redisCommand *lookupCommandOrOriginal(robj **argv ,int argc) { ...@@ -2735,6 +2746,22 @@ struct redisCommand *lookupCommandOrOriginal(robj **argv ,int argc) {
return cmd; return cmd;
} }
static int shouldPropagate(int target) {
if (!server.replication_allowed || target == PROPAGATE_NONE || server.loading)
return 0;
if (target & PROPAGATE_AOF) {
if (server.aof_state != AOF_OFF)
return 1;
}
if (target & PROPAGATE_REPL) {
if (server.masterhost == NULL && (server.repl_backlog || listLength(server.slaves) != 0))
return 1;
}
return 0;
}
/* Propagate the specified command (in the context of the specified database id) /* Propagate the specified command (in the context of the specified database id)
* to AOF and Slaves. * to AOF and Slaves.
* *
...@@ -2743,33 +2770,21 @@ struct redisCommand *lookupCommandOrOriginal(robj **argv ,int argc) { ...@@ -2743,33 +2770,21 @@ struct redisCommand *lookupCommandOrOriginal(robj **argv ,int argc) {
* + PROPAGATE_AOF (propagate into the AOF file if is enabled) * + PROPAGATE_AOF (propagate into the AOF file if is enabled)
* + PROPAGATE_REPL (propagate into the replication link) * + PROPAGATE_REPL (propagate into the replication link)
* *
* This should not be used inside commands implementation since it will not * This is an internal low-level function and should not be called!
* wrap the resulting commands in MULTI/EXEC. Use instead alsoPropagate(),
* preventCommandPropagation(), forceCommandPropagation().
* *
* However for functions that need to (also) propagate out of the context of a * The API for propagating commands is alsoPropagate().
* command execution, for example when serving a blocked client, you
* want to use propagate().
*/ */
void propagate(int dbid, robj **argv, int argc, int flags) { static void propagateNow(int dbid, robj **argv, int argc, int target) {
if (!server.replication_allowed) if (!shouldPropagate(target))
return; return;
/* Propagate a MULTI request once we encounter the first command which
* is a write command.
* This way we'll deliver the MULTI/..../EXEC block as a whole and
* both the AOF and the replication link will have the same consistency
* and atomicity guarantees. */
if (server.in_exec && !server.propagate_in_transaction)
execCommandPropagateMulti(dbid);
/* This needs to be unreachable since the dataset should be fixed during /* This needs to be unreachable since the dataset should be fixed during
* client pause, otherwise data may be lost during a failover. */ * client pause, otherwise data may be lost during a failover. */
serverAssert(!(areClientsPaused() && !server.client_pause_in_transaction)); serverAssert(!(areClientsPaused() && !server.client_pause_in_transaction));
if (server.aof_state != AOF_OFF && flags & PROPAGATE_AOF) if (server.aof_state != AOF_OFF && target & PROPAGATE_AOF)
feedAppendOnlyFile(dbid,argv,argc); feedAppendOnlyFile(dbid,argv,argc);
if (flags & PROPAGATE_REPL) if (target & PROPAGATE_REPL)
replicationFeedSlaves(server.slaves,dbid,argv,argc); replicationFeedSlaves(server.slaves,dbid,argv,argc);
} }
...@@ -2788,7 +2803,8 @@ void alsoPropagate(int dbid, robj **argv, int argc, int target) { ...@@ -2788,7 +2803,8 @@ void alsoPropagate(int dbid, robj **argv, int argc, int target) {
robj **argvcopy; robj **argvcopy;
int j; int j;
if (server.loading) return; /* No propagation during loading. */ if (!shouldPropagate(target))
return;
argvcopy = zmalloc(sizeof(robj*)*argc); argvcopy = zmalloc(sizeof(robj*)*argc);
for (j = 0; j < argc; j++) { for (j = 0; j < argc; j++) {
...@@ -2837,6 +2853,46 @@ void slowlogPushCurrentCommand(client *c, struct redisCommand *cmd, ustime_t dur ...@@ -2837,6 +2853,46 @@ void slowlogPushCurrentCommand(client *c, struct redisCommand *cmd, ustime_t dur
slowlogPushEntryIfNeeded(c,argv,argc,duration); slowlogPushEntryIfNeeded(c,argv,argc,duration);
} }
/* Handle the alsoPropagate() API to handle commands that want to propagate
* multiple separated commands. Note that alsoPropagate() is not affected
* by CLIENT_PREVENT_PROP flag. */
void propagatePendingCommands() {
if (server.also_propagate.numops == 0)
return;
int j;
redisOp *rop;
int multi_emitted = 0;
/* Wrap the commands in server.also_propagate array,
* but don't wrap it if we are already in MULTI context,
* in case the nested MULTI/EXEC.
*
* And if the array contains only one command, no need to
* wrap it, since the single command is atomic. */
if (server.also_propagate.numops > 1 && !server.propagate_no_multi) {
/* We use the first command-to-propagate to set the dbid for MULTI,
* so that the SELECT will be propagated beforehand */
int multi_dbid = server.also_propagate.ops[0].dbid;
propagateNow(multi_dbid,&shared.multi,1,PROPAGATE_AOF|PROPAGATE_REPL);
multi_emitted = 1;
}
for (j = 0; j < server.also_propagate.numops; j++) {
rop = &server.also_propagate.ops[j];
serverAssert(rop->target);
propagateNow(rop->dbid,rop->argv,rop->argc,rop->target);
}
if (multi_emitted) {
/* We take the dbid from last command so that propagateNow() won't inject another SELECT */
int exec_dbid = server.also_propagate.ops[server.also_propagate.numops-1].dbid;
propagateNow(exec_dbid,&shared.exec,1,PROPAGATE_AOF|PROPAGATE_REPL);
}
redisOpArrayFree(&server.also_propagate);
}
/* Call() is the core of Redis execution of a command. /* Call() is the core of Redis execution of a command.
* *
* The following flags can be passed: * The following flags can be passed:
...@@ -2884,8 +2940,19 @@ void call(client *c, int flags) { ...@@ -2884,8 +2940,19 @@ void call(client *c, int flags) {
/* Initialization: clear the flags that must be set by the command on /* Initialization: clear the flags that must be set by the command on
* demand, and initialize the array for additional commands propagation. */ * demand, and initialize the array for additional commands propagation. */
c->flags &= ~(CLIENT_FORCE_AOF|CLIENT_FORCE_REPL|CLIENT_PREVENT_PROP); c->flags &= ~(CLIENT_FORCE_AOF|CLIENT_FORCE_REPL|CLIENT_PREVENT_PROP);
redisOpArray prev_also_propagate = server.also_propagate;
redisOpArrayInit(&server.also_propagate); /* Redis core is in charge of propagation when the first entry point
* of call() is processCommand().
* The only other option to get to call() without having processCommand
* as an entry point is if a module triggers RM_Call outside of call()
* context (for example, in a timer).
* In that case, the module is in charge of propagation.
*
* Because call() is re-entrant we have to cache and restore
* server.core_propagates. */
int prev_core_propagates = server.core_propagates;
if (!server.core_propagates && !(flags & CMD_CALL_FROM_MODULE))
server.core_propagates = 1;
/* Call the command. */ /* Call the command. */
dirty = server.dirty; dirty = server.dirty;
...@@ -2974,9 +3041,14 @@ void call(client *c, int flags) { ...@@ -2974,9 +3041,14 @@ void call(client *c, int flags) {
real_cmd->calls++; real_cmd->calls++;
} }
/* Propagate the command into the AOF and replication link */ /* Propagate the command into the AOF and replication link.
* We never propagate EXEC explicitly, it will be implicitly
* propagated if needed (see propagatePendingCommands).
* Also, module commands take care of themselves */
if (flags & CMD_CALL_PROPAGATE && if (flags & CMD_CALL_PROPAGATE &&
(c->flags & CLIENT_PREVENT_PROP) != CLIENT_PREVENT_PROP) (c->flags & CLIENT_PREVENT_PROP) != CLIENT_PREVENT_PROP &&
c->cmd->proc != execCommand &&
!(c->cmd->flags & CMD_MODULE))
{ {
int propagate_flags = PROPAGATE_NONE; int propagate_flags = PROPAGATE_NONE;
...@@ -2999,11 +3071,10 @@ void call(client *c, int flags) { ...@@ -2999,11 +3071,10 @@ void call(client *c, int flags) {
!(flags & CMD_CALL_PROPAGATE_AOF)) !(flags & CMD_CALL_PROPAGATE_AOF))
propagate_flags &= ~PROPAGATE_AOF; propagate_flags &= ~PROPAGATE_AOF;
/* Call propagate() only if at least one of AOF / replication /* Call alsoPropagate() only if at least one of AOF / replication
* propagation is needed. Note that modules commands handle replication * propagation is needed. */
* in an explicit way, so we never replicate them automatically. */ if (propagate_flags != PROPAGATE_NONE)
if (propagate_flags != PROPAGATE_NONE && !(c->cmd->flags & CMD_MODULE)) alsoPropagate(c->db->id,c->argv,c->argc,propagate_flags);
propagate(c->db->id,c->argv,c->argc,propagate_flags);
} }
/* Restore the old replication flags, since call() can be executed /* Restore the old replication flags, since call() can be executed
...@@ -3012,54 +3083,6 @@ void call(client *c, int flags) { ...@@ -3012,54 +3083,6 @@ void call(client *c, int flags) {
c->flags |= client_old_flags & c->flags |= client_old_flags &
(CLIENT_FORCE_AOF|CLIENT_FORCE_REPL|CLIENT_PREVENT_PROP); (CLIENT_FORCE_AOF|CLIENT_FORCE_REPL|CLIENT_PREVENT_PROP);
/* Handle the alsoPropagate() API to handle commands that want to propagate
* multiple separated commands. Note that alsoPropagate() is not affected
* by CLIENT_PREVENT_PROP flag. */
if (server.also_propagate.numops) {
int j;
redisOp *rop;
if (flags & CMD_CALL_PROPAGATE) {
int multi_emitted = 0;
/* Wrap the commands in server.also_propagate array,
* but don't wrap it if we are already in MULTI context,
* in case the nested MULTI/EXEC.
*
* And if the array contains only one command, no need to
* wrap it, since the single command is atomic. */
if (server.also_propagate.numops > 1 &&
!(c->cmd->flags & CMD_MODULE) &&
!(c->flags & CLIENT_MULTI) &&
!(flags & CMD_CALL_NOWRAP))
{
execCommandPropagateMulti(c->db->id);
multi_emitted = 1;
}
for (j = 0; j < server.also_propagate.numops; j++) {
rop = &server.also_propagate.ops[j];
int target = rop->target;
/* Whatever the command wish is, we honor the call() flags. */
if (!(flags&CMD_CALL_PROPAGATE_AOF)) target &= ~PROPAGATE_AOF;
if (!(flags&CMD_CALL_PROPAGATE_REPL)) target &= ~PROPAGATE_REPL;
if (target)
propagate(rop->dbid,rop->argv,rop->argc,target);
}
if (multi_emitted) {
execCommandPropagateExec(c->db->id);
}
}
redisOpArrayFree(&server.also_propagate);
}
server.also_propagate = prev_also_propagate;
/* Client pause takes effect after a transaction has finished. This needs
* to be located after everything is propagated. */
if (!server.in_exec && server.client_pause_in_transaction) {
server.client_pause_in_transaction = 0;
}
/* If the client has keys tracking enabled for client side caching, /* If the client has keys tracking enabled for client side caching,
* make sure to remember the keys it fetched via this command. */ * make sure to remember the keys it fetched via this command. */
if (c->cmd->flags & CMD_READONLY) { if (c->cmd->flags & CMD_READONLY) {
...@@ -3084,6 +3107,14 @@ void call(client *c, int flags) { ...@@ -3084,6 +3107,14 @@ void call(client *c, int flags) {
/* Do some maintenance job and cleanup */ /* Do some maintenance job and cleanup */
afterCommand(c); afterCommand(c);
/* Client pause takes effect after a transaction has finished. This needs
* to be located after everything is propagated. */
if (!server.in_exec && server.client_pause_in_transaction) {
server.client_pause_in_transaction = 0;
}
server.core_propagates = prev_core_propagates;
} }
/* Used when a command that is ready for execution needs to be rejected, due to /* Used when a command that is ready for execution needs to be rejected, due to
...@@ -3124,9 +3155,16 @@ void rejectCommandFormat(client *c, const char *fmt, ...) { ...@@ -3124,9 +3155,16 @@ void rejectCommandFormat(client *c, const char *fmt, ...) {
/* This is called after a command in call, we can do some maintenance job in it. */ /* This is called after a command in call, we can do some maintenance job in it. */
void afterCommand(client *c) { void afterCommand(client *c) {
UNUSED(c); UNUSED(c);
/* Flush pending invalidation messages only when we are not in nested call. if (!server.in_nested_call) {
* So the messages are not interleaved with transaction response. */ /* If we are at the top-most call() we can propagate what we accumulated.
if (!server.in_nested_call) trackingHandlePendingKeyInvalidations(); * Should be done before trackingHandlePendingKeyInvalidations so that we
* reply to client before invalidating cache (makes more sense) */
if (server.core_propagates)
propagatePendingCommands();
/* Flush pending invalidation messages only when we are not in nested call.
* So the messages are not interleaved with transaction response. */
trackingHandlePendingKeyInvalidations();
}
} }
/* Returns 1 for commands that may have key names in their arguments, but the legacy range /* Returns 1 for commands that may have key names in their arguments, but the legacy range
...@@ -3167,10 +3205,9 @@ void populateCommandMovableKeys(struct redisCommand *cmd) { ...@@ -3167,10 +3205,9 @@ void populateCommandMovableKeys(struct redisCommand *cmd) {
int processCommand(client *c) { int processCommand(client *c) {
if (!scriptIsTimedout()) { if (!scriptIsTimedout()) {
/* Both EXEC and EVAL call call() directly so there should be /* Both EXEC and EVAL call call() directly so there should be
* no way in_exec or in_eval or propagate_in_transaction is 1. * no way in_exec or in_eval is 1.
* That is unless lua_timedout, in which case client may run * That is unless lua_timedout, in which case client may run
* some commands. */ * some commands. */
serverAssert(!server.propagate_in_transaction);
serverAssert(!server.in_exec); serverAssert(!server.in_exec);
serverAssert(!server.in_script); serverAssert(!server.in_script);
} }
......
...@@ -489,10 +489,9 @@ typedef enum { ...@@ -489,10 +489,9 @@ typedef enum {
#define CMD_CALL_PROPAGATE_REPL (1<<3) #define CMD_CALL_PROPAGATE_REPL (1<<3)
#define CMD_CALL_PROPAGATE (CMD_CALL_PROPAGATE_AOF|CMD_CALL_PROPAGATE_REPL) #define CMD_CALL_PROPAGATE (CMD_CALL_PROPAGATE_AOF|CMD_CALL_PROPAGATE_REPL)
#define CMD_CALL_FULL (CMD_CALL_SLOWLOG | CMD_CALL_STATS | CMD_CALL_PROPAGATE) #define CMD_CALL_FULL (CMD_CALL_SLOWLOG | CMD_CALL_STATS | CMD_CALL_PROPAGATE)
#define CMD_CALL_NOWRAP (1<<4) /* Don't wrap also propagate array into #define CMD_CALL_FROM_MODULE (1<<4) /* From RM_Call */
MULTI/EXEC: the caller will handle it. */
/* Command propagation flags, see propagate() function */ /* Command propagation flags, see propagateNow() function */
#define PROPAGATE_NONE 0 #define PROPAGATE_NONE 0
#define PROPAGATE_AOF 1 #define PROPAGATE_AOF 1
#define PROPAGATE_REPL 2 #define PROPAGATE_REPL 2
...@@ -1212,6 +1211,7 @@ typedef struct redisOp { ...@@ -1212,6 +1211,7 @@ typedef struct redisOp {
typedef struct redisOpArray { typedef struct redisOpArray {
redisOp *ops; redisOp *ops;
int numops; int numops;
int capacity;
} redisOpArray; } redisOpArray;
/* This structure is returned by the getMemoryOverheadData() function in /* This structure is returned by the getMemoryOverheadData() function in
...@@ -1358,9 +1358,11 @@ struct redisServer { ...@@ -1358,9 +1358,11 @@ struct redisServer {
int sentinel_mode; /* True if this instance is a Sentinel. */ int sentinel_mode; /* True if this instance is a Sentinel. */
size_t initial_memory_usage; /* Bytes used after initialization. */ size_t initial_memory_usage; /* Bytes used after initialization. */
int always_show_logo; /* Show logo even for non-stdout logging. */ int always_show_logo; /* Show logo even for non-stdout logging. */
int in_script; /* Are we inside EVAL? */ int in_script; /* Are we inside EVAL? */
int in_exec; /* Are we inside EXEC? */ int in_exec; /* Are we inside EXEC? */
int propagate_in_transaction; /* Make sure we don't propagate nested MULTI/EXEC */ int core_propagates; /* Is the core (in oppose to the module subsystem) is in charge of calling propagatePendingCommands? */
int propagate_no_multi; /* True if propagatePendingCommands should avoid wrapping command in MULTI/EXEC */
int module_ctx_nesting; /* moduleCreateContext() nesting level */
char *ignore_warnings; /* Config: warnings that should be ignored. */ char *ignore_warnings; /* Config: warnings that should be ignored. */
int client_pause_in_transaction; /* Was a client pause executed during this Exec? */ int client_pause_in_transaction; /* Was a client pause executed during this Exec? */
int thp_enabled; /* If true, THP is enabled. */ int thp_enabled; /* If true, THP is enabled. */
...@@ -2420,10 +2422,6 @@ void touchAllWatchedKeysInDb(redisDb *emptied, redisDb *replaced_with); ...@@ -2420,10 +2422,6 @@ void touchAllWatchedKeysInDb(redisDb *emptied, redisDb *replaced_with);
void discardTransaction(client *c); void discardTransaction(client *c);
void flagTransaction(client *c); void flagTransaction(client *c);
void execCommandAbort(client *c, sds error); void execCommandAbort(client *c, sds error);
void execCommandPropagateMulti(int dbid);
void execCommandPropagateExec(int dbid);
void beforePropagateMulti();
void afterPropagateExec();
/* Redis object implementation */ /* Redis object implementation */
void decrRefCount(robj *o); void decrRefCount(robj *o);
...@@ -2695,8 +2693,8 @@ struct redisCommand *lookupCommandByCStringLogic(dict *commands, const char *s); ...@@ -2695,8 +2693,8 @@ struct redisCommand *lookupCommandByCStringLogic(dict *commands, const char *s);
struct redisCommand *lookupCommandByCString(const char *s); struct redisCommand *lookupCommandByCString(const char *s);
struct redisCommand *lookupCommandOrOriginal(robj **argv ,int argc); struct redisCommand *lookupCommandOrOriginal(robj **argv ,int argc);
void call(client *c, int flags); void call(client *c, int flags);
void propagate(int dbid, robj **argv, int argc, int flags);
void alsoPropagate(int dbid, robj **argv, int argc, int target); void alsoPropagate(int dbid, robj **argv, int argc, int target);
void propagatePendingCommands();
void redisOpArrayInit(redisOpArray *oa); void redisOpArrayInit(redisOpArray *oa);
void redisOpArrayFree(redisOpArray *oa); void redisOpArrayFree(redisOpArray *oa);
void forceCommandPropagation(client *c, int flags); void forceCommandPropagation(client *c, int flags);
...@@ -2812,7 +2810,7 @@ int allowProtectedAction(int config, client *c); ...@@ -2812,7 +2810,7 @@ int allowProtectedAction(int config, client *c);
/* db.c -- Keyspace access API */ /* db.c -- Keyspace access API */
int removeExpire(redisDb *db, robj *key); int removeExpire(redisDb *db, robj *key);
void deleteExpiredKeyAndPropagate(redisDb *db, robj *keyobj); void deleteExpiredKeyAndPropagate(redisDb *db, robj *keyobj);
void propagateExpire(redisDb *db, robj *key, int lazy); void propagateDeletion(redisDb *db, robj *key, int lazy);
int keyIsExpired(redisDb *db, robj *key); int keyIsExpired(redisDb *db, robj *key);
long long getExpire(redisDb *db, robj *key); long long getExpire(redisDb *db, robj *key);
void setExpire(client *c, redisDb *db, robj *key, long long when); void setExpire(client *c, redisDb *db, robj *key, long long when);
......
...@@ -960,7 +960,7 @@ void serveClientBlockedOnList(client *receiver, robj *o, robj *key, robj *dstkey ...@@ -960,7 +960,7 @@ void serveClientBlockedOnList(client *receiver, robj *o, robj *key, robj *dstkey
serverAssert(llen > 0); serverAssert(llen > 0);
argv[2] = createStringObjectFromLongLong((count > llen) ? llen : count); argv[2] = createStringObjectFromLongLong((count > llen) ? llen : count);
propagate(db->id, argv, 3, PROPAGATE_AOF|PROPAGATE_REPL); alsoPropagate(db->id, argv, 3, PROPAGATE_AOF|PROPAGATE_REPL);
decrRefCount(argv[2]); decrRefCount(argv[2]);
/* Pop a range of elements in a nested arrays way. */ /* Pop a range of elements in a nested arrays way. */
...@@ -968,7 +968,7 @@ void serveClientBlockedOnList(client *receiver, robj *o, robj *key, robj *dstkey ...@@ -968,7 +968,7 @@ void serveClientBlockedOnList(client *receiver, robj *o, robj *key, robj *dstkey
return; return;
} }
propagate(db->id, argv, 2, PROPAGATE_AOF|PROPAGATE_REPL); alsoPropagate(db->id, argv, 2, PROPAGATE_AOF|PROPAGATE_REPL);
/* BRPOP/BLPOP */ /* BRPOP/BLPOP */
value = listTypePop(o, wherefrom); value = listTypePop(o, wherefrom);
...@@ -999,7 +999,7 @@ void serveClientBlockedOnList(client *receiver, robj *o, robj *key, robj *dstkey ...@@ -999,7 +999,7 @@ void serveClientBlockedOnList(client *receiver, robj *o, robj *key, robj *dstkey
argv[2] = dstkey; argv[2] = dstkey;
argv[3] = getStringObjectFromListPosition(wherefrom); argv[3] = getStringObjectFromListPosition(wherefrom);
argv[4] = getStringObjectFromListPosition(whereto); argv[4] = getStringObjectFromListPosition(whereto);
propagate(db->id,argv,(isbrpoplpush ? 3 : 5),PROPAGATE_AOF|PROPAGATE_REPL); alsoPropagate(db->id,argv,(isbrpoplpush ? 3 : 5),PROPAGATE_AOF|PROPAGATE_REPL);
/* Notify event ("lpush" or "rpush" was notified by lmoveHandlePush). */ /* Notify event ("lpush" or "rpush" was notified by lmoveHandlePush). */
notifyKeyspaceEvent(NOTIFY_LIST,wherefrom == LIST_TAIL ? "rpop" : "lpop", notifyKeyspaceEvent(NOTIFY_LIST,wherefrom == LIST_TAIL ? "rpop" : "lpop",
......
...@@ -1399,11 +1399,8 @@ void streamPropagateXCLAIM(client *c, robj *key, streamCG *group, robj *groupnam ...@@ -1399,11 +1399,8 @@ void streamPropagateXCLAIM(client *c, robj *key, streamCG *group, robj *groupnam
argv[12] = shared.lastid; argv[12] = shared.lastid;
argv[13] = createObjectFromStreamID(&group->last_id); argv[13] = createObjectFromStreamID(&group->last_id);
/* We use propagate() because this code path is not always called from alsoPropagate(c->db->id,argv,14,PROPAGATE_AOF|PROPAGATE_REPL);
* the command execution context. Moreover this will just alter the
* consumer group state, and we don't need MULTI/EXEC wrapping because
* there is no message state cross-message atomicity required. */
propagate(c->db->id,argv,14,PROPAGATE_AOF|PROPAGATE_REPL);
decrRefCount(argv[3]); decrRefCount(argv[3]);
decrRefCount(argv[7]); decrRefCount(argv[7]);
decrRefCount(argv[9]); decrRefCount(argv[9]);
...@@ -1424,11 +1421,8 @@ void streamPropagateGroupID(client *c, robj *key, streamCG *group, robj *groupna ...@@ -1424,11 +1421,8 @@ void streamPropagateGroupID(client *c, robj *key, streamCG *group, robj *groupna
argv[3] = groupname; argv[3] = groupname;
argv[4] = createObjectFromStreamID(&group->last_id); argv[4] = createObjectFromStreamID(&group->last_id);
/* We use propagate() because this code path is not always called from alsoPropagate(c->db->id,argv,5,PROPAGATE_AOF|PROPAGATE_REPL);
* the command execution context. Moreover this will just alter the
* consumer group state, and we don't need MULTI/EXEC wrapping because
* there is no message state cross-message atomicity required. */
propagate(c->db->id,argv,5,PROPAGATE_AOF|PROPAGATE_REPL);
decrRefCount(argv[4]); decrRefCount(argv[4]);
} }
...@@ -1446,11 +1440,8 @@ void streamPropagateConsumerCreation(client *c, robj *key, robj *groupname, sds ...@@ -1446,11 +1440,8 @@ void streamPropagateConsumerCreation(client *c, robj *key, robj *groupname, sds
argv[3] = groupname; argv[3] = groupname;
argv[4] = createObject(OBJ_STRING,sdsdup(consumername)); argv[4] = createObject(OBJ_STRING,sdsdup(consumername));
/* We use propagate() because this code path is not always called from alsoPropagate(c->db->id,argv,5,PROPAGATE_AOF|PROPAGATE_REPL);
* the command execution context. Moreover this will just alter the
* consumer group state, and we don't need MULTI/EXEC wrapping because
* there is no message state cross-message atomicity required. */
propagate(c->db->id,argv,5,PROPAGATE_AOF|PROPAGATE_REPL);
decrRefCount(argv[4]); decrRefCount(argv[4]);
} }
......
...@@ -40,6 +40,20 @@ ...@@ -40,6 +40,20 @@
#define REDISMODULE_EXPERIMENTAL_API #define REDISMODULE_EXPERIMENTAL_API
#include "redismodule.h" #include "redismodule.h"
#include <pthread.h> #include <pthread.h>
#include <errno.h>
RedisModuleCtx *detached_ctx = NULL;
static int KeySpace_NotificationGeneric(RedisModuleCtx *ctx, int type, const char *event, RedisModuleString *key) {
REDISMODULE_NOT_USED(type);
REDISMODULE_NOT_USED(event);
REDISMODULE_NOT_USED(key);
RedisModuleCallReply* rep = RedisModule_Call(ctx, "INCR", "c!", "notifications");
RedisModule_FreeCallReply(rep);
return REDISMODULE_OK;
}
/* Timer callback. */ /* Timer callback. */
void timerHandler(RedisModuleCtx *ctx, void *data) { void timerHandler(RedisModuleCtx *ctx, void *data) {
...@@ -79,6 +93,8 @@ void timerNestedHandler(RedisModuleCtx *ctx, void *data) { ...@@ -79,6 +93,8 @@ void timerNestedHandler(RedisModuleCtx *ctx, void *data) {
RedisModule_Replicate(ctx,"INCRBY","cc","timer-nested-start","1"); RedisModule_Replicate(ctx,"INCRBY","cc","timer-nested-start","1");
RedisModuleCallReply *reply = RedisModule_Call(ctx,"propagate-test.nested", repl? "!" : ""); RedisModuleCallReply *reply = RedisModule_Call(ctx,"propagate-test.nested", repl? "!" : "");
RedisModule_FreeCallReply(reply); RedisModule_FreeCallReply(reply);
reply = RedisModule_Call(ctx, "INCR", repl? "c!" : "c", "timer-nested-middle");
RedisModule_FreeCallReply(reply);
RedisModule_Replicate(ctx,"INCRBY","cc","timer-nested-end","1"); RedisModule_Replicate(ctx,"INCRBY","cc","timer-nested-end","1");
} }
...@@ -108,6 +124,62 @@ int propagateTestTimerNestedReplCommand(RedisModuleCtx *ctx, RedisModuleString * ...@@ -108,6 +124,62 @@ int propagateTestTimerNestedReplCommand(RedisModuleCtx *ctx, RedisModuleString *
return REDISMODULE_OK; return REDISMODULE_OK;
} }
void timerHandlerMaxmemory(RedisModuleCtx *ctx, void *data) {
REDISMODULE_NOT_USED(ctx);
REDISMODULE_NOT_USED(data);
RedisModuleCallReply *reply = RedisModule_Call(ctx,"SETEX","ccc!","timer-maxmemory-volatile-start","100","1");
RedisModule_FreeCallReply(reply);
reply = RedisModule_Call(ctx, "CONFIG", "ccc!", "SET", "maxmemory", "1");
RedisModule_FreeCallReply(reply);
RedisModule_Replicate(ctx, "INCR", "c", "timer-maxmemory-middle");
reply = RedisModule_Call(ctx,"SETEX","ccc!","timer-maxmemory-volatile-end","100","1");
RedisModule_FreeCallReply(reply);
}
int propagateTestTimerMaxmemoryCommand(RedisModuleCtx *ctx, RedisModuleString **argv, int argc)
{
REDISMODULE_NOT_USED(argv);
REDISMODULE_NOT_USED(argc);
RedisModuleTimerID timer_id =
RedisModule_CreateTimer(ctx,100,timerHandlerMaxmemory,(void*)1);
REDISMODULE_NOT_USED(timer_id);
RedisModule_ReplyWithSimpleString(ctx,"OK");
return REDISMODULE_OK;
}
void timerHandlerEval(RedisModuleCtx *ctx, void *data) {
REDISMODULE_NOT_USED(ctx);
REDISMODULE_NOT_USED(data);
RedisModuleCallReply *reply = RedisModule_Call(ctx,"INCRBY","cc!","timer-eval-start","1");
RedisModule_FreeCallReply(reply);
reply = RedisModule_Call(ctx, "EVAL", "cccc!", "redis.call('set',KEYS[1],ARGV[1])", "1", "foo", "bar");
RedisModule_FreeCallReply(reply);
RedisModule_Replicate(ctx, "INCR", "c", "timer-eval-middle");
reply = RedisModule_Call(ctx,"INCRBY","cc!","timer-eval-end","1");
RedisModule_FreeCallReply(reply);
}
int propagateTestTimerEvalCommand(RedisModuleCtx *ctx, RedisModuleString **argv, int argc)
{
REDISMODULE_NOT_USED(argv);
REDISMODULE_NOT_USED(argc);
RedisModuleTimerID timer_id =
RedisModule_CreateTimer(ctx,100,timerHandlerEval,(void*)1);
REDISMODULE_NOT_USED(timer_id);
RedisModule_ReplyWithSimpleString(ctx,"OK");
return REDISMODULE_OK;
}
/* The thread entry point. */ /* The thread entry point. */
void *threadMain(void *arg) { void *threadMain(void *arg) {
REDISMODULE_NOT_USED(arg); REDISMODULE_NOT_USED(arg);
...@@ -116,6 +188,8 @@ void *threadMain(void *arg) { ...@@ -116,6 +188,8 @@ void *threadMain(void *arg) {
for (int i = 0; i < 3; i++) { for (int i = 0; i < 3; i++) {
RedisModule_ThreadSafeContextLock(ctx); RedisModule_ThreadSafeContextLock(ctx);
RedisModule_Replicate(ctx,"INCR","c","a-from-thread"); RedisModule_Replicate(ctx,"INCR","c","a-from-thread");
RedisModuleCallReply *reply = RedisModule_Call(ctx,"INCR","c!","thread-call");
RedisModule_FreeCallReply(reply);
RedisModule_Replicate(ctx,"INCR","c","b-from-thread"); RedisModule_Replicate(ctx,"INCR","c","b-from-thread");
RedisModule_ThreadSafeContextUnlock(ctx); RedisModule_ThreadSafeContextUnlock(ctx);
} }
...@@ -137,6 +211,37 @@ int propagateTestThreadCommand(RedisModuleCtx *ctx, RedisModuleString **argv, in ...@@ -137,6 +211,37 @@ int propagateTestThreadCommand(RedisModuleCtx *ctx, RedisModuleString **argv, in
return REDISMODULE_OK; return REDISMODULE_OK;
} }
/* The thread entry point. */
void *threadDetachedMain(void *arg) {
REDISMODULE_NOT_USED(arg);
RedisModule_SelectDb(detached_ctx,9); /* Tests ran in database number 9. */
RedisModule_ThreadSafeContextLock(detached_ctx);
RedisModule_Replicate(detached_ctx,"INCR","c","thread-detached-before");
RedisModuleCallReply *reply = RedisModule_Call(detached_ctx,"INCR","c!","thread-detached-1");
RedisModule_FreeCallReply(reply);
reply = RedisModule_Call(detached_ctx,"INCR","c!","thread-detached-2");
RedisModule_FreeCallReply(reply);
RedisModule_Replicate(detached_ctx,"INCR","c","thread-detached-after");
RedisModule_ThreadSafeContextUnlock(detached_ctx);
return NULL;
}
int propagateTestDetachedThreadCommand(RedisModuleCtx *ctx, RedisModuleString **argv, int argc)
{
REDISMODULE_NOT_USED(argv);
REDISMODULE_NOT_USED(argc);
pthread_t tid;
if (pthread_create(&tid,NULL,threadDetachedMain,NULL) != 0)
return RedisModule_ReplyWithError(ctx,"-ERR Can't start thread");
REDISMODULE_NOT_USED(tid);
RedisModule_ReplyWithSimpleString(ctx,"OK");
return REDISMODULE_OK;
}
int propagateTestSimpleCommand(RedisModuleCtx *ctx, RedisModuleString **argv, int argc) int propagateTestSimpleCommand(RedisModuleCtx *ctx, RedisModuleString **argv, int argc)
{ {
REDISMODULE_NOT_USED(argv); REDISMODULE_NOT_USED(argv);
...@@ -188,6 +293,18 @@ int propagateTestNestedCommand(RedisModuleCtx *ctx, RedisModuleString **argv, in ...@@ -188,6 +293,18 @@ int propagateTestNestedCommand(RedisModuleCtx *ctx, RedisModuleString **argv, in
reply = RedisModule_Call(ctx, "INCR", "c!", "after-call"); reply = RedisModule_Call(ctx, "INCR", "c!", "after-call");
RedisModule_FreeCallReply(reply); RedisModule_FreeCallReply(reply);
reply = RedisModule_Call(ctx, "INCR", "c!", "before-call-2");
RedisModule_FreeCallReply(reply);
reply = RedisModule_Call(ctx, "keyspace.incr_case1", "c!", "asdf"); /* Propagates INCR */
RedisModule_FreeCallReply(reply);
reply = RedisModule_Call(ctx, "keyspace.del_key_copy", "c!", "asdf"); /* Propagates DEL */
RedisModule_FreeCallReply(reply);
reply = RedisModule_Call(ctx, "INCR", "c!", "after-call-2");
RedisModule_FreeCallReply(reply);
RedisModule_ReplyWithSimpleString(ctx,"OK"); RedisModule_ReplyWithSimpleString(ctx,"OK");
return REDISMODULE_OK; return REDISMODULE_OK;
} }
...@@ -212,6 +329,11 @@ int RedisModule_OnLoad(RedisModuleCtx *ctx, RedisModuleString **argv, int argc) ...@@ -212,6 +329,11 @@ int RedisModule_OnLoad(RedisModuleCtx *ctx, RedisModuleString **argv, int argc)
if (RedisModule_Init(ctx,"propagate-test",1,REDISMODULE_APIVER_1) if (RedisModule_Init(ctx,"propagate-test",1,REDISMODULE_APIVER_1)
== REDISMODULE_ERR) return REDISMODULE_ERR; == REDISMODULE_ERR) return REDISMODULE_ERR;
detached_ctx = RedisModule_GetDetachedThreadSafeContext(ctx);
if (RedisModule_SubscribeToKeyspaceEvents(ctx, REDISMODULE_NOTIFY_ALL, KeySpace_NotificationGeneric) == REDISMODULE_ERR)
return REDISMODULE_ERR;
if (RedisModule_CreateCommand(ctx,"propagate-test.timer", if (RedisModule_CreateCommand(ctx,"propagate-test.timer",
propagateTestTimerCommand, propagateTestTimerCommand,
"",1,1,1) == REDISMODULE_ERR) "",1,1,1) == REDISMODULE_ERR)
...@@ -227,11 +349,26 @@ int RedisModule_OnLoad(RedisModuleCtx *ctx, RedisModuleString **argv, int argc) ...@@ -227,11 +349,26 @@ int RedisModule_OnLoad(RedisModuleCtx *ctx, RedisModuleString **argv, int argc)
"",1,1,1) == REDISMODULE_ERR) "",1,1,1) == REDISMODULE_ERR)
return REDISMODULE_ERR; return REDISMODULE_ERR;
if (RedisModule_CreateCommand(ctx,"propagate-test.timer-maxmemory",
propagateTestTimerMaxmemoryCommand,
"",1,1,1) == REDISMODULE_ERR)
return REDISMODULE_ERR;
if (RedisModule_CreateCommand(ctx,"propagate-test.timer-eval",
propagateTestTimerEvalCommand,
"",1,1,1) == REDISMODULE_ERR)
return REDISMODULE_ERR;
if (RedisModule_CreateCommand(ctx,"propagate-test.thread", if (RedisModule_CreateCommand(ctx,"propagate-test.thread",
propagateTestThreadCommand, propagateTestThreadCommand,
"",1,1,1) == REDISMODULE_ERR) "",1,1,1) == REDISMODULE_ERR)
return REDISMODULE_ERR; return REDISMODULE_ERR;
if (RedisModule_CreateCommand(ctx,"propagate-test.detached-thread",
propagateTestDetachedThreadCommand,
"",1,1,1) == REDISMODULE_ERR)
return REDISMODULE_ERR;
if (RedisModule_CreateCommand(ctx,"propagate-test.simple", if (RedisModule_CreateCommand(ctx,"propagate-test.simple",
propagateTestSimpleCommand, propagateTestSimpleCommand,
"",1,1,1) == REDISMODULE_ERR) "",1,1,1) == REDISMODULE_ERR)
......
...@@ -493,3 +493,75 @@ start_server {tags {"maxmemory external:skip"}} { ...@@ -493,3 +493,75 @@ start_server {tags {"maxmemory external:skip"}} {
if {$::verbose} { puts "evicted: $evicted" } if {$::verbose} { puts "evicted: $evicted" }
} }
} }
start_server {tags {"maxmemory" "external:skip"}} {
test {propagation with eviction} {
set repl [attach_to_replication_stream]
r set asdf1 1
r set asdf2 2
r set asdf3 3
r config set maxmemory-policy allkeys-lru
r config set maxmemory 1
wait_for_condition 5000 10 {
[r dbsize] eq 0
} else {
fail "Not all keys have been evicted"
}
r config set maxmemory 0
r config set maxmemory-policy noeviction
r set asdf4 4
assert_replication_stream $repl {
{select *}
{set asdf1 1}
{set asdf2 2}
{set asdf3 3}
{del asdf*}
{del asdf*}
{del asdf*}
{set asdf4 4}
}
close_replication_stream $repl
r config set maxmemory 0
r config set maxmemory-policy noeviction
}
}
start_server {tags {"maxmemory" "external:skip"}} {
test {propagation with eviction in MULTI} {
set repl [attach_to_replication_stream]
r config set maxmemory-policy allkeys-lru
r multi
r incr x
r config set maxmemory 1
r incr x
assert_equal [r exec] {1 OK 2}
wait_for_condition 5000 10 {
[r dbsize] eq 0
} else {
fail "Not all keys have been evicted"
}
assert_replication_stream $repl {
{select *}
{multi}
{incr x}
{incr x}
{exec}
{del x}
}
close_replication_stream $repl
r config set maxmemory 0
r config set maxmemory-policy noeviction
}
}
This diff is collapsed.
...@@ -29,4 +29,4 @@ start_server {tags {"modules"}} { ...@@ -29,4 +29,4 @@ start_server {tags {"modules"}} {
fail "Module memory is not reclaimed by UNLINK" fail "Module memory is not reclaimed by UNLINK"
} }
} }
} }
\ No newline at end of file
Markdown is supported
0% or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment