Prefer storing iterators on stack instead of the heap

This commit is contained in:
Slavomir Kaslev 2025-07-14 12:00:35 +03:00
parent 75cdc51f86
commit 4de37f09d5
24 changed files with 620 additions and 548 deletions

View File

@ -633,12 +633,13 @@ void ACLChangeSelectorPerm(aclSelector *selector, struct redisCommand *cmd, int
ACLResetFirstArgsForCommand(selector,id); ACLResetFirstArgsForCommand(selector,id);
if (cmd->subcommands_dict) { if (cmd->subcommands_dict) {
dictEntry *de; dictEntry *de;
dictIterator *di = dictGetSafeIterator(cmd->subcommands_dict); dictIterator di;
while((de = dictNext(di)) != NULL) { dictInitSafeIterator(&di, cmd->subcommands_dict);
while((de = dictNext(&di)) != NULL) {
struct redisCommand *sub = (struct redisCommand *)dictGetVal(de); struct redisCommand *sub = (struct redisCommand *)dictGetVal(de);
ACLSetSelectorCommandBit(selector,sub->id,allow); ACLSetSelectorCommandBit(selector,sub->id,allow);
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
} }
@ -649,9 +650,10 @@ void ACLChangeSelectorPerm(aclSelector *selector, struct redisCommand *cmd, int
* function returns C_ERR if the category was not found, or C_OK if it was * function returns C_ERR if the category was not found, or C_OK if it was
* found and the operation was performed. */ * found and the operation was performed. */
void ACLSetSelectorCommandBitsForCategory(dict *commands, aclSelector *selector, uint64_t cflag, int value) { void ACLSetSelectorCommandBitsForCategory(dict *commands, aclSelector *selector, uint64_t cflag, int value) {
dictIterator *di = dictGetIterator(commands); dictIterator di;
dictEntry *de; dictEntry *de;
while ((de = dictNext(di)) != NULL) { dictInitIterator(&di, commands);
while ((de = dictNext(&di)) != NULL) {
struct redisCommand *cmd = dictGetVal(de); struct redisCommand *cmd = dictGetVal(de);
if (cmd->acl_categories & cflag) { if (cmd->acl_categories & cflag) {
ACLChangeSelectorPerm(selector,cmd,value); ACLChangeSelectorPerm(selector,cmd,value);
@ -660,7 +662,7 @@ void ACLSetSelectorCommandBitsForCategory(dict *commands, aclSelector *selector,
ACLSetSelectorCommandBitsForCategory(cmd->subcommands_dict, selector, cflag, value); ACLSetSelectorCommandBitsForCategory(cmd->subcommands_dict, selector, cflag, value);
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* This function is responsible for recomputing the command bits for all selectors of the existing users. /* This function is responsible for recomputing the command bits for all selectors of the existing users.
@ -713,9 +715,10 @@ int ACLSetSelectorCategory(aclSelector *selector, const char *category, int allo
} }
void ACLCountCategoryBitsForCommands(dict *commands, aclSelector *selector, unsigned long *on, unsigned long *off, uint64_t cflag) { void ACLCountCategoryBitsForCommands(dict *commands, aclSelector *selector, unsigned long *on, unsigned long *off, uint64_t cflag) {
dictIterator *di = dictGetIterator(commands); dictIterator di;
dictEntry *de; dictEntry *de;
while ((de = dictNext(di)) != NULL) { dictInitIterator(&di, commands);
while ((de = dictNext(&di)) != NULL) {
struct redisCommand *cmd = dictGetVal(de); struct redisCommand *cmd = dictGetVal(de);
if (cmd->acl_categories & cflag) { if (cmd->acl_categories & cflag) {
if (ACLGetSelectorCommandBit(selector,cmd->id)) if (ACLGetSelectorCommandBit(selector,cmd->id))
@ -727,7 +730,7 @@ void ACLCountCategoryBitsForCommands(dict *commands, aclSelector *selector, unsi
ACLCountCategoryBitsForCommands(cmd->subcommands_dict, selector, on, off, cflag); ACLCountCategoryBitsForCommands(cmd->subcommands_dict, selector, on, off, cflag);
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* Return the number of commands allowed (on) and denied (off) for the user 'u' /* Return the number of commands allowed (on) and denied (off) for the user 'u'
@ -1959,36 +1962,37 @@ int ACLShouldKillPubsubClient(client *c, list *upcoming) {
if (getClientType(c) == CLIENT_TYPE_PUBSUB) { if (getClientType(c) == CLIENT_TYPE_PUBSUB) {
/* Check for pattern violations. */ /* Check for pattern violations. */
dictIterator *di = dictGetIterator(c->pubsub_patterns); dictIterator di;
dictEntry *de; dictEntry *de;
while (!kill && ((de = dictNext(di)) != NULL)) { dictInitIterator(&di, c->pubsub_patterns);
while (!kill && ((de = dictNext(&di)) != NULL)) {
o = dictGetKey(de); o = dictGetKey(de);
int res = ACLCheckChannelAgainstList(upcoming, o->ptr, sdslen(o->ptr), 1); int res = ACLCheckChannelAgainstList(upcoming, o->ptr, sdslen(o->ptr), 1);
kill = (res == ACL_DENIED_CHANNEL); kill = (res == ACL_DENIED_CHANNEL);
} }
dictReleaseIterator(di); dictResetIterator(&di);
/* Check for channel violations. */ /* Check for channel violations. */
if (!kill) { if (!kill) {
/* Check for global channels violation. */ /* Check for global channels violation. */
di = dictGetIterator(c->pubsub_channels); dictInitIterator(&di, c->pubsub_channels);
while (!kill && ((de = dictNext(di)) != NULL)) { while (!kill && ((de = dictNext(&di)) != NULL)) {
o = dictGetKey(de); o = dictGetKey(de);
int res = ACLCheckChannelAgainstList(upcoming, o->ptr, sdslen(o->ptr), 0); int res = ACLCheckChannelAgainstList(upcoming, o->ptr, sdslen(o->ptr), 0);
kill = (res == ACL_DENIED_CHANNEL); kill = (res == ACL_DENIED_CHANNEL);
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
if (!kill) { if (!kill) {
/* Check for shard channels violation. */ /* Check for shard channels violation. */
di = dictGetIterator(c->pubsubshard_channels); dictInitIterator(&di, c->pubsubshard_channels);
while (!kill && ((de = dictNext(di)) != NULL)) { while (!kill && ((de = dictNext(&di)) != NULL)) {
o = dictGetKey(de); o = dictGetKey(de);
int res = ACLCheckChannelAgainstList(upcoming, o->ptr, sdslen(o->ptr), 0); int res = ACLCheckChannelAgainstList(upcoming, o->ptr, sdslen(o->ptr), 0);
kill = (res == ACL_DENIED_CHANNEL); kill = (res == ACL_DENIED_CHANNEL);
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
if (kill) { if (kill) {
@ -2778,9 +2782,9 @@ sds getAclErrorMessage(int acl_res, user *user, struct redisCommand *cmd, sds er
/* ACL CAT category */ /* ACL CAT category */
void aclCatWithFlags(client *c, dict *commands, uint64_t cflag, int *arraylen) { void aclCatWithFlags(client *c, dict *commands, uint64_t cflag, int *arraylen) {
dictEntry *de; dictEntry *de;
dictIterator *di = dictGetIterator(commands); dictIterator di;
dictInitIterator(&di, commands);
while ((de = dictNext(di)) != NULL) { while ((de = dictNext(&di)) != NULL) {
struct redisCommand *cmd = dictGetVal(de); struct redisCommand *cmd = dictGetVal(de);
if (cmd->acl_categories & cflag) { if (cmd->acl_categories & cflag) {
addReplyBulkCBuffer(c, cmd->fullname, sdslen(cmd->fullname)); addReplyBulkCBuffer(c, cmd->fullname, sdslen(cmd->fullname));
@ -2791,7 +2795,7 @@ void aclCatWithFlags(client *c, dict *commands, uint64_t cflag, int *arraylen) {
aclCatWithFlags(c, cmd->subcommands_dict, cflag, arraylen); aclCatWithFlags(c, cmd->subcommands_dict, cflag, arraylen);
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* Add the formatted response from a single selector to the ACL GETUSER /* Add the formatted response from a single selector to the ACL GETUSER

View File

@ -2000,10 +2000,11 @@ int rewriteSortedSetObject(rio *r, robj *key, robj *o) {
} }
} else if (o->encoding == OBJ_ENCODING_SKIPLIST) { } else if (o->encoding == OBJ_ENCODING_SKIPLIST) {
zset *zs = o->ptr; zset *zs = o->ptr;
dictIterator *di = dictGetIterator(zs->dict); dictIterator di;
dictEntry *de; dictEntry *de;
while((de = dictNext(di)) != NULL) { dictInitIterator(&di, zs->dict);
while((de = dictNext(&di)) != NULL) {
sds ele = dictGetKey(de); sds ele = dictGetKey(de);
double *score = dictGetVal(de); double *score = dictGetVal(de);
@ -2015,20 +2016,20 @@ int rewriteSortedSetObject(rio *r, robj *key, robj *o) {
!rioWriteBulkString(r,"ZADD",4) || !rioWriteBulkString(r,"ZADD",4) ||
!rioWriteBulkObject(r,key)) !rioWriteBulkObject(r,key))
{ {
dictReleaseIterator(di); dictResetIterator(&di);
return 0; return 0;
} }
} }
if (!rioWriteBulkDouble(r,*score) || if (!rioWriteBulkDouble(r,*score) ||
!rioWriteBulkString(r,ele,sdslen(ele))) !rioWriteBulkString(r,ele,sdslen(ele)))
{ {
dictReleaseIterator(di); dictResetIterator(&di);
return 0; return 0;
} }
if (++count == AOF_REWRITE_ITEMS_PER_CMD) count = 0; if (++count == AOF_REWRITE_ITEMS_PER_CMD) count = 0;
items--; items--;
} }
dictReleaseIterator(di); dictResetIterator(&di);
} else { } else {
serverPanic("Unknown sorted zset encoding"); serverPanic("Unknown sorted zset encoding");
} }
@ -2332,20 +2333,21 @@ int rewriteModuleObject(rio *r, robj *key, robj *o, int dbid) {
static int rewriteFunctions(rio *aof) { static int rewriteFunctions(rio *aof) {
dict *functions = functionsLibGet(); dict *functions = functionsLibGet();
dictIterator *iter = dictGetIterator(functions); dictIterator iter;
dictEntry *entry = NULL; dictEntry *entry = NULL;
while ((entry = dictNext(iter))) { dictInitIterator(&iter, functions);
while ((entry = dictNext(&iter))) {
functionLibInfo *li = dictGetVal(entry); functionLibInfo *li = dictGetVal(entry);
if (rioWrite(aof, "*3\r\n", 4) == 0) goto werr; if (rioWrite(aof, "*3\r\n", 4) == 0) goto werr;
char function_load[] = "$8\r\nFUNCTION\r\n$4\r\nLOAD\r\n"; char function_load[] = "$8\r\nFUNCTION\r\n$4\r\nLOAD\r\n";
if (rioWrite(aof, function_load, sizeof(function_load) - 1) == 0) goto werr; if (rioWrite(aof, function_load, sizeof(function_load) - 1) == 0) goto werr;
if (rioWriteBulkString(aof, li->code, sdslen(li->code)) == 0) goto werr; if (rioWriteBulkString(aof, li->code, sdslen(li->code)) == 0) goto werr;
} }
dictReleaseIterator(iter); dictResetIterator(&iter);
return 1; return 1;
werr: werr:
dictReleaseIterator(iter); dictResetIterator(&iter);
return 0; return 0;
} }

View File

@ -423,17 +423,17 @@ void blockForKeys(client *c, int btype, robj **keys, int numkeys, mstime_t timeo
* Internal function for unblockClient() */ * Internal function for unblockClient() */
static void unblockClientWaitingData(client *c) { static void unblockClientWaitingData(client *c) {
dictEntry *de; dictEntry *de;
dictIterator *di; dictIterator di;
if (dictSize(c->bstate.keys) == 0) if (dictSize(c->bstate.keys) == 0)
return; return;
di = dictGetIterator(c->bstate.keys); dictInitIterator(&di, c->bstate.keys);
/* The client may wait for multiple keys, so unblock it for every key. */ /* The client may wait for multiple keys, so unblock it for every key. */
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
releaseBlockedEntry(c, de, 0); releaseBlockedEntry(c, de, 0);
} }
dictReleaseIterator(di); dictResetIterator(&di);
dictEmpty(c->bstate.keys, NULL); dictEmpty(c->bstate.keys, NULL);
} }

View File

@ -373,10 +373,11 @@ void migrateCloseSocket(robj *host, robj *port) {
} }
void migrateCloseTimedoutSockets(void) { void migrateCloseTimedoutSockets(void) {
dictIterator *di = dictGetSafeIterator(server.migrate_cached_sockets); dictIterator di;
dictEntry *de; dictEntry *de;
while((de = dictNext(di)) != NULL) { dictInitSafeIterator(&di, server.migrate_cached_sockets);
while((de = dictNext(&di)) != NULL) {
migrateCachedSocket *cs = dictGetVal(de); migrateCachedSocket *cs = dictGetVal(de);
if ((server.unixtime - cs->last_use_time) > MIGRATE_SOCKET_CACHE_TTL) { if ((server.unixtime - cs->last_use_time) > MIGRATE_SOCKET_CACHE_TTL) {
@ -385,7 +386,7 @@ void migrateCloseTimedoutSockets(void) {
dictDelete(server.migrate_cached_sockets,dictGetKey(de)); dictDelete(server.migrate_cached_sockets,dictGetKey(de));
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* MIGRATE host port key dbid timeout [COPY | REPLACE | AUTH password | /* MIGRATE host port key dbid timeout [COPY | REPLACE | AUTH password |
@ -1365,7 +1366,7 @@ int clusterRedirectBlockedClientIfNeeded(client *c) {
c->bstate.btype == BLOCKED_MODULE)) c->bstate.btype == BLOCKED_MODULE))
{ {
dictEntry *de; dictEntry *de;
dictIterator *di; dictIterator di;
/* If the cluster is down, unblock the client with the right error. /* If the cluster is down, unblock the client with the right error.
* If the cluster is configured to allow reads on cluster down, we * If the cluster is configured to allow reads on cluster down, we
@ -1382,8 +1383,8 @@ int clusterRedirectBlockedClientIfNeeded(client *c) {
return 0; return 0;
/* All keys must belong to the same slot, so check first key only. */ /* All keys must belong to the same slot, so check first key only. */
di = dictGetIterator(c->bstate.keys); dictInitIterator(&di, c->bstate.keys);
if ((de = dictNext(di)) != NULL) { if ((de = dictNext(&di)) != NULL) {
robj *key = dictGetKey(de); robj *key = dictGetKey(de);
int slot = keyHashSlot((char*)key->ptr, sdslen(key->ptr)); int slot = keyHashSlot((char*)key->ptr, sdslen(key->ptr));
clusterNode *node = getNodeBySlot(slot); clusterNode *node = getNodeBySlot(slot);
@ -1409,11 +1410,11 @@ int clusterRedirectBlockedClientIfNeeded(client *c) {
clusterRedirectClient(c,node,slot, clusterRedirectClient(c,node,slot,
CLUSTER_REDIR_MOVED); CLUSTER_REDIR_MOVED);
} }
dictReleaseIterator(di); dictResetIterator(&di);
return 1; return 1;
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
return 0; return 0;
} }

View File

@ -1070,7 +1070,7 @@ void clusterInitLast(void) {
* 6) The new configuration is saved and the cluster state updated. * 6) The new configuration is saved and the cluster state updated.
* 7) If the node was a slave, the whole data set is flushed away. */ * 7) If the node was a slave, the whole data set is flushed away. */
void clusterReset(int hard) { void clusterReset(int hard) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
int j; int j;
@ -1092,14 +1092,14 @@ void clusterReset(int hard) {
dictEmpty(server.cluster->shards, NULL); dictEmpty(server.cluster->shards, NULL);
/* Forget all the nodes, but myself. */ /* Forget all the nodes, but myself. */
di = dictGetSafeIterator(server.cluster->nodes); dictInitSafeIterator(&di, server.cluster->nodes);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
clusterNode *node = dictGetVal(de); clusterNode *node = dictGetVal(de);
if (node == myself) continue; if (node == myself) continue;
clusterDelNode(node); clusterDelNode(node);
} }
dictReleaseIterator(di); dictResetIterator(&di);
/* Empty the nodes blacklist. */ /* Empty the nodes blacklist. */
dictEmpty(server.cluster->nodes_black_list, NULL); dictEmpty(server.cluster->nodes_black_list, NULL);
@ -1543,7 +1543,7 @@ void clusterAddNode(clusterNode *node) {
*/ */
void clusterDelNode(clusterNode *delnode) { void clusterDelNode(clusterNode *delnode) {
int j; int j;
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
/* 1) Mark slots as unassigned. */ /* 1) Mark slots as unassigned. */
@ -1557,14 +1557,14 @@ void clusterDelNode(clusterNode *delnode) {
} }
/* 2) Remove failure reports. */ /* 2) Remove failure reports. */
di = dictGetSafeIterator(server.cluster->nodes); dictInitSafeIterator(&di, server.cluster->nodes);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
clusterNode *node = dictGetVal(de); clusterNode *node = dictGetVal(de);
if (node == delnode) continue; if (node == delnode) continue;
clusterNodeDelFailureReport(node,delnode); clusterNodeDelFailureReport(node,delnode);
} }
dictReleaseIterator(di); dictResetIterator(&di);
/* 3) Remove the node from the owning shard */ /* 3) Remove the node from the owning shard */
clusterRemoveNodeFromShard(delnode); clusterRemoveNodeFromShard(delnode);
@ -1662,15 +1662,15 @@ void clusterRemoveNodeFromShard(clusterNode *node) {
* epoch if greater than any node configEpoch. */ * epoch if greater than any node configEpoch. */
uint64_t clusterGetMaxEpoch(void) { uint64_t clusterGetMaxEpoch(void) {
uint64_t max = 0; uint64_t max = 0;
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
di = dictGetSafeIterator(server.cluster->nodes); dictInitSafeIterator(&di, server.cluster->nodes);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
clusterNode *node = dictGetVal(de); clusterNode *node = dictGetVal(de);
if (node->configEpoch > max) max = node->configEpoch; if (node->configEpoch > max) max = node->configEpoch;
} }
dictReleaseIterator(di); dictResetIterator(&di);
if (max < server.cluster->currentEpoch) max = server.cluster->currentEpoch; if (max < server.cluster->currentEpoch) max = server.cluster->currentEpoch;
return max; return max;
} }
@ -1818,17 +1818,17 @@ void clusterHandleConfigEpochCollision(clusterNode *sender) {
* However without the cleanup during long uptime and with some automated * However without the cleanup during long uptime and with some automated
* node add/removal procedures, entries could accumulate. */ * node add/removal procedures, entries could accumulate. */
void clusterBlacklistCleanup(void) { void clusterBlacklistCleanup(void) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
di = dictGetSafeIterator(server.cluster->nodes_black_list); dictInitSafeIterator(&di, server.cluster->nodes_black_list);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
int64_t expire = dictGetUnsignedIntegerVal(de); int64_t expire = dictGetUnsignedIntegerVal(de);
if (expire < server.unixtime) if (expire < server.unixtime)
dictDelete(server.cluster->nodes_black_list,dictGetKey(de)); dictDelete(server.cluster->nodes_black_list,dictGetKey(de));
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* Cleanup the blacklist and add a new node ID to the black list. */ /* Cleanup the blacklist and add a new node ID to the black list. */
@ -1953,11 +1953,11 @@ void clearNodeFailureIfNeeded(clusterNode *node) {
* specified ip address and port number. This function is used in order to * specified ip address and port number. This function is used in order to
* avoid adding a new handshake node for the same address multiple times. */ * avoid adding a new handshake node for the same address multiple times. */
int clusterHandshakeInProgress(char *ip, int port, int cport) { int clusterHandshakeInProgress(char *ip, int port, int cport) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
di = dictGetSafeIterator(server.cluster->nodes); dictInitSafeIterator(&di, server.cluster->nodes);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
clusterNode *node = dictGetVal(de); clusterNode *node = dictGetVal(de);
if (!nodeInHandshake(node)) continue; if (!nodeInHandshake(node)) continue;
@ -1965,7 +1965,7 @@ int clusterHandshakeInProgress(char *ip, int port, int cport) {
getNodeDefaultClientPort(node) == port && getNodeDefaultClientPort(node) == port &&
node->cport == cport) break; node->cport == cport) break;
} }
dictReleaseIterator(di); dictResetIterator(&di);
return de != NULL; return de != NULL;
} }
@ -2579,9 +2579,11 @@ uint32_t writePingExt(clusterMsg *hdr, int gossipcount) {
/* Gossip forgotten nodes */ /* Gossip forgotten nodes */
if (dictSize(server.cluster->nodes_black_list) > 0) { if (dictSize(server.cluster->nodes_black_list) > 0) {
dictIterator *di = dictGetIterator(server.cluster->nodes_black_list); dictIterator di;
dictEntry *de; dictEntry *de;
while ((de = dictNext(di)) != NULL) {
dictInitIterator(&di, server.cluster->nodes_black_list);
while ((de = dictNext(&di)) != NULL) {
if (cursor != NULL) { if (cursor != NULL) {
uint64_t expire = dictGetUnsignedIntegerVal(de); uint64_t expire = dictGetUnsignedIntegerVal(de);
if ((time_t)expire < server.unixtime) continue; /* already expired */ if ((time_t)expire < server.unixtime) continue; /* already expired */
@ -2596,7 +2598,7 @@ uint32_t writePingExt(clusterMsg *hdr, int gossipcount) {
totlen += getForgottenNodeExtSize(); totlen += getForgottenNodeExtSize();
extensions++; extensions++;
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* Populate shard_id */ /* Populate shard_id */
@ -3518,18 +3520,18 @@ void clusterSendMessage(clusterLink *link, clusterMsgSendBlock *msgblock) {
* some node->link to be invalidated, so it is safe to call this function * some node->link to be invalidated, so it is safe to call this function
* from event handlers that will do stuff with node links later. */ * from event handlers that will do stuff with node links later. */
void clusterBroadcastMessage(clusterMsgSendBlock *msgblock) { void clusterBroadcastMessage(clusterMsgSendBlock *msgblock) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
di = dictGetSafeIterator(server.cluster->nodes); dictInitSafeIterator(&di, server.cluster->nodes);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
clusterNode *node = dictGetVal(de); clusterNode *node = dictGetVal(de);
if (node->flags & (CLUSTER_NODE_MYSELF|CLUSTER_NODE_HANDSHAKE)) if (node->flags & (CLUSTER_NODE_MYSELF|CLUSTER_NODE_HANDSHAKE))
continue; continue;
clusterSendMessage(node->link,msgblock); clusterSendMessage(node->link,msgblock);
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* Build the message header. hdr must point to a buffer at least /* Build the message header. hdr must point to a buffer at least
@ -3725,11 +3727,11 @@ void clusterSendPing(clusterLink *link, int type) {
/* If there are PFAIL nodes, add them at the end. */ /* If there are PFAIL nodes, add them at the end. */
if (pfail_wanted) { if (pfail_wanted) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
di = dictGetSafeIterator(server.cluster->nodes); dictInitSafeIterator(&di, server.cluster->nodes);
while((de = dictNext(di)) != NULL && pfail_wanted > 0) { while((de = dictNext(&di)) != NULL && pfail_wanted > 0) {
clusterNode *node = dictGetVal(de); clusterNode *node = dictGetVal(de);
if (node->flags & CLUSTER_NODE_HANDSHAKE) continue; if (node->flags & CLUSTER_NODE_HANDSHAKE) continue;
if (node->flags & CLUSTER_NODE_NOADDR) continue; if (node->flags & CLUSTER_NODE_NOADDR) continue;
@ -3741,7 +3743,7 @@ void clusterSendPing(clusterLink *link, int type) {
* of PFAIL nodes. */ * of PFAIL nodes. */
pfail_wanted--; pfail_wanted--;
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* Compute the actual total length and send! */ /* Compute the actual total length and send! */
@ -3776,11 +3778,11 @@ void clusterSendPing(clusterLink *link, int type) {
#define CLUSTER_BROADCAST_ALL 0 #define CLUSTER_BROADCAST_ALL 0
#define CLUSTER_BROADCAST_LOCAL_SLAVES 1 #define CLUSTER_BROADCAST_LOCAL_SLAVES 1
void clusterBroadcastPong(int target) { void clusterBroadcastPong(int target) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
di = dictGetSafeIterator(server.cluster->nodes); dictInitSafeIterator(&di, server.cluster->nodes);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
clusterNode *node = dictGetVal(de); clusterNode *node = dictGetVal(de);
if (!node->link) continue; if (!node->link) continue;
@ -3793,7 +3795,7 @@ void clusterBroadcastPong(int target) {
} }
clusterSendPing(node->link,CLUSTERMSG_TYPE_PONG); clusterSendPing(node->link,CLUSTERMSG_TYPE_PONG);
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* Create a PUBLISH message block. /* Create a PUBLISH message block.
@ -4439,7 +4441,7 @@ void clusterHandleSlaveFailover(void) {
void clusterHandleSlaveMigration(int max_slaves) { void clusterHandleSlaveMigration(int max_slaves) {
int j, okslaves = 0; int j, okslaves = 0;
clusterNode *mymaster = myself->slaveof, *target = NULL, *candidate = NULL; clusterNode *mymaster = myself->slaveof, *target = NULL, *candidate = NULL;
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
/* Step 1: Don't migrate if the cluster state is not ok. */ /* Step 1: Don't migrate if the cluster state is not ok. */
@ -4464,8 +4466,8 @@ void clusterHandleSlaveMigration(int max_slaves) {
* slaves to migrate at the same time, but this is unlikely to * slaves to migrate at the same time, but this is unlikely to
* happen and relatively harmless when it does. */ * happen and relatively harmless when it does. */
candidate = myself; candidate = myself;
di = dictGetSafeIterator(server.cluster->nodes); dictInitSafeIterator(&di, server.cluster->nodes);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
clusterNode *node = dictGetVal(de); clusterNode *node = dictGetVal(de);
int okslaves = 0, is_orphaned = 1; int okslaves = 0, is_orphaned = 1;
@ -4504,7 +4506,7 @@ void clusterHandleSlaveMigration(int max_slaves) {
} }
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
/* Step 4: perform the migration if there is a target, and if I'm the /* Step 4: perform the migration if there is a target, and if I'm the
* candidate, but only if the master is continuously orphaned for a * candidate, but only if the master is continuously orphaned for a
@ -4668,7 +4670,7 @@ static void clusterNodeCronFreeLinkOnBufferLimitReached(clusterNode *node) {
/* This is executed 10 times every second */ /* This is executed 10 times every second */
void clusterCron(void) { void clusterCron(void) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
int update_state = 0; int update_state = 0;
int orphaned_masters; /* How many masters there are without ok slaves. */ int orphaned_masters; /* How many masters there are without ok slaves. */
@ -4693,8 +4695,8 @@ void clusterCron(void) {
/* Clear so clusterNodeCronHandleReconnect can count the number of nodes in PFAIL. */ /* Clear so clusterNodeCronHandleReconnect can count the number of nodes in PFAIL. */
server.cluster->stats_pfail_nodes = 0; server.cluster->stats_pfail_nodes = 0;
/* Run through some of the operations we want to do on each cluster node. */ /* Run through some of the operations we want to do on each cluster node. */
di = dictGetSafeIterator(server.cluster->nodes); dictInitSafeIterator(&di, server.cluster->nodes);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
clusterNode *node = dictGetVal(de); clusterNode *node = dictGetVal(de);
/* We free the inbound or outboud link to the node if the link has an /* We free the inbound or outboud link to the node if the link has an
* oversized message send queue and immediately try reconnecting. */ * oversized message send queue and immediately try reconnecting. */
@ -4704,7 +4706,7 @@ void clusterCron(void) {
*/ */
if(clusterNodeCronHandleReconnect(node, handshake_timeout, now)) continue; if(clusterNodeCronHandleReconnect(node, handshake_timeout, now)) continue;
} }
dictReleaseIterator(di); dictResetIterator(&di);
/* Ping some random node 1 time every 10 iterations, so that we usually ping /* Ping some random node 1 time every 10 iterations, so that we usually ping
* one random node every second. */ * one random node every second. */
@ -4741,8 +4743,8 @@ void clusterCron(void) {
orphaned_masters = 0; orphaned_masters = 0;
max_slaves = 0; max_slaves = 0;
this_slaves = 0; this_slaves = 0;
di = dictGetSafeIterator(server.cluster->nodes); dictInitSafeIterator(&di, server.cluster->nodes);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
clusterNode *node = dictGetVal(de); clusterNode *node = dictGetVal(de);
now = mstime(); /* Use an updated time at every iteration. */ now = mstime(); /* Use an updated time at every iteration. */
@ -4839,7 +4841,7 @@ void clusterCron(void) {
} }
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
/* If we are a slave node but the replication is still turned off, /* If we are a slave node but the replication is still turned off,
* enable it if we know the address of our master and it appears to * enable it if we know the address of our master and it appears to
@ -4944,16 +4946,18 @@ void bitmapClearBit(unsigned char *bitmap, int pos) {
* Otherwise zero is returned. Used by clusterNodeSetSlotBit() to set the * Otherwise zero is returned. Used by clusterNodeSetSlotBit() to set the
* MIGRATE_TO flag the when a master gets the first slot. */ * MIGRATE_TO flag the when a master gets the first slot. */
int clusterMastersHaveSlaves(void) { int clusterMastersHaveSlaves(void) {
dictIterator *di = dictGetSafeIterator(server.cluster->nodes); dictIterator di;
dictEntry *de; dictEntry *de;
int slaves = 0; int slaves = 0;
while((de = dictNext(di)) != NULL) {
dictInitSafeIterator(&di, server.cluster->nodes);
while((de = dictNext(&di)) != NULL) {
clusterNode *node = dictGetVal(de); clusterNode *node = dictGetVal(de);
if (nodeIsSlave(node)) continue; if (nodeIsSlave(node)) continue;
slaves += node->numslaves; slaves += node->numslaves;
} }
dictReleaseIterator(di); dictResetIterator(&di);
return slaves != 0; return slaves != 0;
} }
@ -5122,12 +5126,12 @@ void clusterUpdateState(void) {
* At the same time count the number of reachable masters having * At the same time count the number of reachable masters having
* at least one slot. */ * at least one slot. */
{ {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
server.cluster->size = 0; server.cluster->size = 0;
di = dictGetSafeIterator(server.cluster->nodes); dictInitSafeIterator(&di, server.cluster->nodes);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
clusterNode *node = dictGetVal(de); clusterNode *node = dictGetVal(de);
if (clusterNodeIsMaster(node) && node->numslots) { if (clusterNodeIsMaster(node) && node->numslots) {
@ -5136,7 +5140,7 @@ void clusterUpdateState(void) {
reachable_masters++; reachable_masters++;
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* If we are in a minority partition, change the cluster state /* If we are in a minority partition, change the cluster state
@ -5495,14 +5499,14 @@ void clusterFreeNodesSlotsInfo(clusterNode *n) {
* configuration file (nodes.conf) for a given node. */ * configuration file (nodes.conf) for a given node. */
sds clusterGenNodesDescription(client *c, int filter, int tls_primary) { sds clusterGenNodesDescription(client *c, int filter, int tls_primary) {
sds ci = sdsempty(), ni; sds ci = sdsempty(), ni;
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
/* Generate all nodes slots info firstly. */ /* Generate all nodes slots info firstly. */
clusterGenNodesSlotsInfo(filter); clusterGenNodesSlotsInfo(filter);
di = dictGetSafeIterator(server.cluster->nodes); dictInitSafeIterator(&di, server.cluster->nodes);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
clusterNode *node = dictGetVal(de); clusterNode *node = dictGetVal(de);
if (node->flags & filter) continue; if (node->flags & filter) continue;
@ -5514,7 +5518,7 @@ sds clusterGenNodesDescription(client *c, int filter, int tls_primary) {
/* Release slots info. */ /* Release slots info. */
clusterFreeNodesSlotsInfo(node); clusterFreeNodesSlotsInfo(node);
} }
dictReleaseIterator(di); dictResetIterator(&di);
return ci; return ci;
} }
@ -5558,15 +5562,15 @@ void addReplyClusterLinkDescription(client *c, clusterLink *link) {
/* Add to the output buffer of the given client an array of cluster link descriptions, /* Add to the output buffer of the given client an array of cluster link descriptions,
* with array entry being a description of a single current cluster link. */ * with array entry being a description of a single current cluster link. */
void addReplyClusterLinksDescription(client *c) { void addReplyClusterLinksDescription(client *c) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
void *arraylen_ptr = NULL; void *arraylen_ptr = NULL;
int num_links = 0; int num_links = 0;
arraylen_ptr = addReplyDeferredLen(c); arraylen_ptr = addReplyDeferredLen(c);
di = dictGetSafeIterator(server.cluster->nodes); dictInitSafeIterator(&di, server.cluster->nodes);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
clusterNode *node = dictGetVal(de); clusterNode *node = dictGetVal(de);
if (node->link) { if (node->link) {
num_links++; num_links++;
@ -5577,7 +5581,7 @@ void addReplyClusterLinksDescription(client *c) {
addReplyClusterLinkDescription(c, node->inbound_link); addReplyClusterLinkDescription(c, node->inbound_link);
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
setDeferredArrayLen(c, arraylen_ptr, num_links); setDeferredArrayLen(c, arraylen_ptr, num_links);
} }
@ -5867,10 +5871,12 @@ int getMyShardSlotCount(void) {
char **getClusterNodesList(size_t *numnodes) { char **getClusterNodesList(size_t *numnodes) {
size_t count = dictSize(server.cluster->nodes); size_t count = dictSize(server.cluster->nodes);
char **ids = zmalloc((count+1)*CLUSTER_NAMELEN); char **ids = zmalloc((count+1)*CLUSTER_NAMELEN);
dictIterator *di = dictGetIterator(server.cluster->nodes); dictIterator di;
dictEntry *de; dictEntry *de;
int j = 0; int j = 0;
while((de = dictNext(di)) != NULL) {
dictInitIterator(&di, server.cluster->nodes);
while((de = dictNext(&di)) != NULL) {
clusterNode *node = dictGetVal(de); clusterNode *node = dictGetVal(de);
if (node->flags & (CLUSTER_NODE_NOADDR|CLUSTER_NODE_HANDSHAKE)) continue; if (node->flags & (CLUSTER_NODE_NOADDR|CLUSTER_NODE_HANDSHAKE)) continue;
ids[j] = zmalloc(CLUSTER_NAMELEN); ids[j] = zmalloc(CLUSTER_NAMELEN);
@ -5880,7 +5886,7 @@ char **getClusterNodesList(size_t *numnodes) {
*numnodes = j; *numnodes = j;
ids[j] = NULL; /* Null term so that FreeClusterNodesList does not need ids[j] = NULL; /* Null term so that FreeClusterNodesList does not need
* to also get the count argument. */ * to also get the count argument. */
dictReleaseIterator(di); dictResetIterator(&di);
return ids; return ids;
} }

View File

@ -949,7 +949,7 @@ end:
void configGetCommand(client *c) { void configGetCommand(client *c) {
int i; int i;
dictEntry *de; dictEntry *de;
dictIterator *di; dictIterator di;
/* Create a dictionary to store the matched configs */ /* Create a dictionary to store the matched configs */
dict *matches = dictCreate(&externalStringType); dict *matches = dictCreate(&externalStringType);
for (i = 0; i < c->argc - 2; i++) { for (i = 0; i < c->argc - 2; i++) {
@ -969,9 +969,8 @@ void configGetCommand(client *c) {
} }
/* Otherwise, do a match against all items in the dictionary. */ /* Otherwise, do a match against all items in the dictionary. */
di = dictGetIterator(configs); dictInitIterator(&di, configs);
while ((de = dictNext(&di)) != NULL) {
while ((de = dictNext(di)) != NULL) {
standardConfig *config = dictGetVal(de); standardConfig *config = dictGetVal(de);
/* Note that hidden configs require an exact match (not a pattern) */ /* Note that hidden configs require an exact match (not a pattern) */
if (config->flags & HIDDEN_CONFIG) continue; if (config->flags & HIDDEN_CONFIG) continue;
@ -980,17 +979,17 @@ void configGetCommand(client *c) {
dictAdd(matches, dictGetKey(de), config); dictAdd(matches, dictGetKey(de), config);
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
di = dictGetIterator(matches); dictInitIterator(&di, matches);
addReplyMapLen(c, dictSize(matches)); addReplyMapLen(c, dictSize(matches));
while ((de = dictNext(di)) != NULL) { while ((de = dictNext(&di)) != NULL) {
standardConfig *config = (standardConfig *) dictGetVal(de); standardConfig *config = (standardConfig *) dictGetVal(de);
addReplyBulkCString(c, dictGetKey(de)); addReplyBulkCString(c, dictGetKey(de));
addReplyBulkSds(c, config->interface.get(config)); addReplyBulkSds(c, config->interface.get(config));
} }
dictReleaseIterator(di); dictResetIterator(&di);
dictRelease(matches); dictRelease(matches);
} }
@ -1564,10 +1563,10 @@ void rewriteConfigBindOption(standardConfig *config, const char *name, struct re
/* Rewrite the loadmodule option. */ /* Rewrite the loadmodule option. */
void rewriteConfigLoadmoduleOption(struct rewriteConfigState *state) { void rewriteConfigLoadmoduleOption(struct rewriteConfigState *state) {
sds line; sds line;
dictIterator di;
dictIterator *di = dictGetIterator(modules);
dictEntry *de; dictEntry *de;
while ((de = dictNext(di)) != NULL) { dictInitIterator(&di, modules);
while ((de = dictNext(&di)) != NULL) {
struct RedisModule *module = dictGetVal(de); struct RedisModule *module = dictGetVal(de);
/* Internal modules doesn't have path and are not part of the configuration file */ /* Internal modules doesn't have path and are not part of the configuration file */
if (sdslen(module->loadmod->path) == 0) continue; if (sdslen(module->loadmod->path) == 0) continue;
@ -1580,7 +1579,7 @@ void rewriteConfigLoadmoduleOption(struct rewriteConfigState *state) {
} }
rewriteConfigRewriteLine(state,"loadmodule",line,1); rewriteConfigRewriteLine(state,"loadmodule",line,1);
} }
dictReleaseIterator(di); dictResetIterator(&di);
/* Mark "loadmodule" as processed in case modules is empty. */ /* Mark "loadmodule" as processed in case modules is empty. */
rewriteConfigMarkAsProcessed(state,"loadmodule"); rewriteConfigMarkAsProcessed(state,"loadmodule");
} }
@ -1614,10 +1613,11 @@ sds rewriteConfigGetContentFromState(struct rewriteConfigState *state) {
* This function does just this, iterating all the option names and * This function does just this, iterating all the option names and
* blanking all the lines still associated. */ * blanking all the lines still associated. */
void rewriteConfigRemoveOrphaned(struct rewriteConfigState *state) { void rewriteConfigRemoveOrphaned(struct rewriteConfigState *state) {
dictIterator *di = dictGetIterator(state->option_to_line); dictIterator di;
dictEntry *de; dictEntry *de;
while((de = dictNext(di)) != NULL) { dictInitIterator(&di, state->option_to_line);
while((de = dictNext(&di)) != NULL) {
list *l = dictGetVal(de); list *l = dictGetVal(de);
sds option = dictGetKey(de); sds option = dictGetKey(de);
@ -1637,7 +1637,7 @@ void rewriteConfigRemoveOrphaned(struct rewriteConfigState *state) {
listDelNode(l,ln); listDelNode(l,ln);
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* This function returns a string representation of all the config options /* This function returns a string representation of all the config options
@ -1649,14 +1649,15 @@ sds getConfigDebugInfo(void) {
/* Iterate the configs and "rewrite" the ones that have /* Iterate the configs and "rewrite" the ones that have
* the debug flag. */ * the debug flag. */
dictIterator *di = dictGetIterator(configs); dictIterator di;
dictEntry *de; dictEntry *de;
while ((de = dictNext(di)) != NULL) { dictInitIterator(&di, configs);
while ((de = dictNext(&di)) != NULL) {
standardConfig *config = dictGetVal(de); standardConfig *config = dictGetVal(de);
if (!(config->flags & DEBUG_CONFIG)) continue; if (!(config->flags & DEBUG_CONFIG)) continue;
config->interface.rewrite(config, config->name, state); config->interface.rewrite(config, config->name, state);
} }
dictReleaseIterator(di); dictResetIterator(&di);
sds info = rewriteConfigGetContentFromState(state); sds info = rewriteConfigGetContentFromState(state);
rewriteConfigReleaseState(state); rewriteConfigReleaseState(state);
return info; return info;
@ -1749,15 +1750,16 @@ int rewriteConfig(char *path, int force_write) {
* the rewrite state. */ * the rewrite state. */
/* Iterate the configs that are standard */ /* Iterate the configs that are standard */
dictIterator *di = dictGetIterator(configs); dictIterator di;
dictEntry *de; dictEntry *de;
while ((de = dictNext(di)) != NULL) { dictInitIterator(&di, configs);
while ((de = dictNext(&di)) != NULL) {
standardConfig *config = dictGetVal(de); standardConfig *config = dictGetVal(de);
/* Only rewrite the primary names */ /* Only rewrite the primary names */
if (config->flags & ALIAS_CONFIG) continue; if (config->flags & ALIAS_CONFIG) continue;
if (config->interface.rewrite) config->interface.rewrite(config, dictGetKey(de), state); if (config->interface.rewrite) config->interface.rewrite(config, dictGetKey(de), state);
} }
dictReleaseIterator(di); dictResetIterator(&di);
rewriteConfigUserOption(state); rewriteConfigUserOption(state);
rewriteConfigLoadmoduleOption(state); rewriteConfigLoadmoduleOption(state);

View File

@ -2116,14 +2116,15 @@ void copyCommand(client *c) {
* where the function is used for more info. */ * where the function is used for more info. */
void scanDatabaseForReadyKeys(redisDb *db) { void scanDatabaseForReadyKeys(redisDb *db) {
dictEntry *de; dictEntry *de;
dictIterator *di = dictGetSafeIterator(db->blocking_keys); dictIterator di;
while((de = dictNext(di)) != NULL) { dictInitSafeIterator(&di, db->blocking_keys);
while((de = dictNext(&di)) != NULL) {
robj *key = dictGetKey(de); robj *key = dictGetKey(de);
kvobj *kv = dbFind(db, key->ptr); kvobj *kv = dbFind(db, key->ptr);
if (kv) if (kv)
signalKeyAsReady(db, key, kv->type); signalKeyAsReady(db, key, kv->type);
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* Since we are unblocking XREADGROUP clients in the event the /* Since we are unblocking XREADGROUP clients in the event the
@ -2131,8 +2132,10 @@ void scanDatabaseForReadyKeys(redisDb *db) {
* database was flushed/swapped. */ * database was flushed/swapped. */
void scanDatabaseForDeletedKeys(redisDb *emptied, redisDb *replaced_with) { void scanDatabaseForDeletedKeys(redisDb *emptied, redisDb *replaced_with) {
dictEntry *de; dictEntry *de;
dictIterator *di = dictGetSafeIterator(emptied->blocking_keys); dictIterator di;
while((de = dictNext(di)) != NULL) {
dictInitSafeIterator(&di, emptied->blocking_keys);
while((de = dictNext(&di)) != NULL) {
robj *key = dictGetKey(de); robj *key = dictGetKey(de);
int existed = 0, exists = 0; int existed = 0, exists = 0;
int original_type = -1, curr_type = -1; int original_type = -1, curr_type = -1;
@ -2154,7 +2157,7 @@ void scanDatabaseForDeletedKeys(redisDb *emptied, redisDb *replaced_with) {
if ((existed && !exists) || original_type != curr_type) if ((existed && !exists) || original_type != curr_type)
signalDeletedKeyAsReady(emptied, key, original_type); signalDeletedKeyAsReady(emptied, key, original_type);
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* Swap two databases at runtime so that all clients will magically see /* Swap two databases at runtime so that all clients will magically see

View File

@ -189,10 +189,11 @@ void xorObjectDigest(redisDb *db, robj *keyobj, unsigned char *digest, robj *o)
} }
} else if (o->encoding == OBJ_ENCODING_SKIPLIST) { } else if (o->encoding == OBJ_ENCODING_SKIPLIST) {
zset *zs = o->ptr; zset *zs = o->ptr;
dictIterator *di = dictGetIterator(zs->dict); dictIterator di;
dictEntry *de; dictEntry *de;
while((de = dictNext(di)) != NULL) { dictInitIterator(&di, zs->dict);
while((de = dictNext(&di)) != NULL) {
sds sdsele = dictGetKey(de); sds sdsele = dictGetKey(de);
double *score = dictGetVal(de); double *score = dictGetVal(de);
const int len = fpconv_dtoa(*score, buf); const int len = fpconv_dtoa(*score, buf);
@ -202,7 +203,7 @@ void xorObjectDigest(redisDb *db, robj *keyobj, unsigned char *digest, robj *o)
mixDigest(eledigest,buf,strlen(buf)); mixDigest(eledigest,buf,strlen(buf));
xorDigest(digest,eledigest,20); xorDigest(digest,eledigest,20);
} }
dictReleaseIterator(di); dictResetIterator(&di);
} else { } else {
serverPanic("Unknown sorted set encoding"); serverPanic("Unknown sorted set encoding");
} }
@ -1065,14 +1066,15 @@ NULL
addReply(c, shared.ok); addReply(c, shared.ok);
} else if (!strcasecmp(c->argv[1]->ptr,"script") && c->argc == 3) { } else if (!strcasecmp(c->argv[1]->ptr,"script") && c->argc == 3) {
if (!strcasecmp(c->argv[2]->ptr,"list")) { if (!strcasecmp(c->argv[2]->ptr,"list")) {
dictIterator *di = dictGetIterator(evalScriptsDict()); dictIterator di;
dictEntry *de; dictEntry *de;
while ((de = dictNext(di)) != NULL) { dictInitIterator(&di, evalScriptsDict());
while ((de = dictNext(&di)) != NULL) {
luaScript *script = dictGetVal(de); luaScript *script = dictGetVal(de);
sds *sha = dictGetKey(de); sds *sha = dictGetKey(de);
serverLog(LL_WARNING, "SCRIPT SHA: %s\n%s", (char*)sha, (char*)script->body->ptr); serverLog(LL_WARNING, "SCRIPT SHA: %s\n%s", (char*)sha, (char*)script->body->ptr);
} }
dictReleaseIterator(di); dictResetIterator(&di);
} else if (sdslen(c->argv[2]->ptr) == 40) { } else if (sdslen(c->argv[2]->ptr) == 40) {
dictEntry *de; dictEntry *de;
if ((de = dictFind(evalScriptsDict(), c->argv[2]->ptr)) == NULL) { if ((de = dictFind(evalScriptsDict(), c->argv[2]->ptr)) == NULL) {

View File

@ -664,11 +664,12 @@ void defragZsetSkiplist(defragKeysCtx *ctx, kvobj *ob) {
if (dictSize(zs->dict) > server.active_defrag_max_scan_fields) if (dictSize(zs->dict) > server.active_defrag_max_scan_fields)
defragLater(ctx, ob); defragLater(ctx, ob);
else { else {
dictIterator *di = dictGetIterator(zs->dict); dictIterator di;
while((de = dictNext(di)) != NULL) { dictInitIterator(&di, zs->dict);
while((de = dictNext(&di)) != NULL) {
activeDefragZsetEntry(zs, de); activeDefragZsetEntry(zs, de);
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* defrag the dict struct and tables */ /* defrag the dict struct and tables */
if ((newdict = dictDefragTables(zs->dict))) if ((newdict = dictDefragTables(zs->dict)))
@ -1029,16 +1030,17 @@ void defragPubsubScanCallback(void *privdata, const dictEntry *de, dictEntryLink
/* The channel name is shared by the client's pubsub(shard) and server's /* The channel name is shared by the client's pubsub(shard) and server's
* pubsub(shard), after defraging the channel name, we need to update * pubsub(shard), after defraging the channel name, we need to update
* the reference in the clients' dictionary. */ * the reference in the clients' dictionary. */
dictIterator *di = dictGetIterator(clients); dictIterator di;
dictEntry *clientde; dictEntry *clientde;
while((clientde = dictNext(di)) != NULL) { dictInitIterator(&di, clients);
while((clientde = dictNext(&di)) != NULL) {
client *c = dictGetKey(clientde); client *c = dictGetKey(clientde);
dict *client_channels = ctx->getPubSubChannels(c); dict *client_channels = ctx->getPubSubChannels(c);
dictEntry *pubsub_channel = dictFind(client_channels, newchannel); dictEntry *pubsub_channel = dictFind(client_channels, newchannel);
serverAssert(pubsub_channel); serverAssert(pubsub_channel);
dictSetKey(ctx->getPubSubChannels(c), pubsub_channel, newchannel); dictSetKey(ctx->getPubSubChannels(c), pubsub_channel, newchannel);
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* Try to defrag the dictionary of clients that is stored as the value part. */ /* Try to defrag the dictionary of clients that is stored as the value part. */
@ -1654,9 +1656,10 @@ static void beginDefragCycle(void) {
addDefragStage(defragLuaScripts, NULL, NULL); addDefragStage(defragLuaScripts, NULL, NULL);
/* Add stages for modules. */ /* Add stages for modules. */
dictIterator *di = dictGetIterator(modules); dictIterator di;
dictEntry *de; dictEntry *de;
while ((de = dictNext(di)) != NULL) { dictInitIterator(&di, modules);
while ((de = dictNext(&di)) != NULL) {
struct RedisModule *module = dictGetVal(de); struct RedisModule *module = dictGetVal(de);
if (module->defrag_cb || module->defrag_cb_2) { if (module->defrag_cb || module->defrag_cb_2) {
defragModuleCtx *ctx = zmalloc(sizeof(defragModuleCtx)); defragModuleCtx *ctx = zmalloc(sizeof(defragModuleCtx));
@ -1665,7 +1668,7 @@ static void beginDefragCycle(void) {
addDefragStage(defragModuleGlobals, freeDefragModelContext, ctx); addDefragStage(defragModuleGlobals, freeDefragModelContext, ctx);
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
defrag.current_stage = NULL; defrag.current_stage = NULL;
defrag.start_cycle = getMonotonicUs(); defrag.start_cycle = getMonotonicUs();

View File

@ -278,15 +278,16 @@ uint64_t dictIncrUnsignedIntegerVal(dictEntry *de, uint64_t val);
uint64_t dictGetUnsignedIntegerVal(const dictEntry *de); uint64_t dictGetUnsignedIntegerVal(const dictEntry *de);
#define dictForEach(d, ty, m, ...) do { \ #define dictForEach(d, ty, m, ...) do { \
dictIterator *di = dictGetIterator(d); \ dictIterator di; \
dictEntry *de; \ dictEntry *de; \
while ((de = dictNext(di)) != NULL) { \ dictInitIterator(&di, d); \
while ((de = dictNext(&di)) != NULL) { \
ty *m = dictGetVal(de); \ ty *m = dictGetVal(de); \
do { \ do { \
__VA_ARGS__ \ __VA_ARGS__ \
} while(0); \ } while(0); \
} \ } \
dictReleaseIterator(di); \ dictResetIterator(&di); \
} while(0); } while(0);
#ifdef REDIS_TEST #ifdef REDIS_TEST

View File

@ -168,14 +168,15 @@ static void engineDispose(dict *d, void *obj) {
void functionsLibCtxClear(functionsLibCtx *lib_ctx) { void functionsLibCtxClear(functionsLibCtx *lib_ctx) {
dictEmpty(lib_ctx->functions, NULL); dictEmpty(lib_ctx->functions, NULL);
dictEmpty(lib_ctx->libraries, NULL); dictEmpty(lib_ctx->libraries, NULL);
dictIterator *iter = dictGetIterator(lib_ctx->engines_stats); dictIterator iter;
dictEntry *entry = NULL; dictEntry *entry = NULL;
while ((entry = dictNext(iter))) { dictInitIterator(&iter, lib_ctx->engines_stats);
while ((entry = dictNext(&iter))) {
functionsLibEngineStats *stats = dictGetVal(entry); functionsLibEngineStats *stats = dictGetVal(entry);
stats->n_functions = 0; stats->n_functions = 0;
stats->n_lib = 0; stats->n_lib = 0;
} }
dictReleaseIterator(iter); dictResetIterator(&iter);
lib_ctx->cache_memory = 0; lib_ctx->cache_memory = 0;
} }
@ -218,14 +219,15 @@ functionsLibCtx* functionsLibCtxCreate(void) {
ret->libraries = dictCreate(&librariesDictType); ret->libraries = dictCreate(&librariesDictType);
ret->functions = dictCreate(&functionDictType); ret->functions = dictCreate(&functionDictType);
ret->engines_stats = dictCreate(&engineStatsDictType); ret->engines_stats = dictCreate(&engineStatsDictType);
dictIterator *iter = dictGetIterator(engines); dictIterator iter;
dictEntry *entry = NULL; dictEntry *entry = NULL;
while ((entry = dictNext(iter))) { dictInitIterator(&iter, engines);
while ((entry = dictNext(&iter))) {
engineInfo *ei = dictGetVal(entry); engineInfo *ei = dictGetVal(entry);
functionsLibEngineStats *stats = zcalloc(sizeof(*stats)); functionsLibEngineStats *stats = zcalloc(sizeof(*stats));
dictAdd(ret->engines_stats, ei->name, stats); dictAdd(ret->engines_stats, ei->name, stats);
} }
dictReleaseIterator(iter); dictResetIterator(&iter);
ret->cache_memory = 0; ret->cache_memory = 0;
return ret; return ret;
} }
@ -277,15 +279,16 @@ static functionLibInfo* engineLibraryCreate(sds name, engineInfo *ei, sds code)
} }
static void libraryUnlink(functionsLibCtx *lib_ctx, functionLibInfo* li) { static void libraryUnlink(functionsLibCtx *lib_ctx, functionLibInfo* li) {
dictIterator *iter = dictGetIterator(li->functions); dictIterator iter;
dictEntry *entry = NULL; dictEntry *entry = NULL;
while ((entry = dictNext(iter))) { dictInitIterator(&iter, li->functions);
while ((entry = dictNext(&iter))) {
functionInfo *fi = dictGetVal(entry); functionInfo *fi = dictGetVal(entry);
int ret = dictDelete(lib_ctx->functions, fi->name); int ret = dictDelete(lib_ctx->functions, fi->name);
serverAssert(ret == DICT_OK); serverAssert(ret == DICT_OK);
lib_ctx->cache_memory -= functionMallocSize(fi); lib_ctx->cache_memory -= functionMallocSize(fi);
} }
dictReleaseIterator(iter); dictResetIterator(&iter);
entry = dictUnlink(lib_ctx->libraries, li->name); entry = dictUnlink(lib_ctx->libraries, li->name);
dictSetVal(lib_ctx->libraries, entry, NULL); dictSetVal(lib_ctx->libraries, entry, NULL);
dictFreeUnlinkedEntry(lib_ctx->libraries, entry); dictFreeUnlinkedEntry(lib_ctx->libraries, entry);
@ -299,14 +302,15 @@ static void libraryUnlink(functionsLibCtx *lib_ctx, functionLibInfo* li) {
} }
static void libraryLink(functionsLibCtx *lib_ctx, functionLibInfo* li) { static void libraryLink(functionsLibCtx *lib_ctx, functionLibInfo* li) {
dictIterator *iter = dictGetIterator(li->functions); dictIterator iter;
dictEntry *entry = NULL; dictEntry *entry = NULL;
while ((entry = dictNext(iter))) { dictInitIterator(&iter, li->functions);
while ((entry = dictNext(&iter))) {
functionInfo *fi = dictGetVal(entry); functionInfo *fi = dictGetVal(entry);
dictAdd(lib_ctx->functions, fi->name, fi); dictAdd(lib_ctx->functions, fi->name, fi);
lib_ctx->cache_memory += functionMallocSize(fi); lib_ctx->cache_memory += functionMallocSize(fi);
} }
dictReleaseIterator(iter); dictResetIterator(&iter);
dictAdd(lib_ctx->libraries, li->name, li); dictAdd(lib_ctx->libraries, li->name, li);
lib_ctx->cache_memory += libraryMallocSize(li); lib_ctx->cache_memory += libraryMallocSize(li);
@ -326,13 +330,13 @@ static void libraryLink(functionsLibCtx *lib_ctx, functionLibInfo* li) {
* */ * */
static int libraryJoin(functionsLibCtx *functions_lib_ctx_dst, functionsLibCtx *functions_lib_ctx_src, int replace, sds *err) { static int libraryJoin(functionsLibCtx *functions_lib_ctx_dst, functionsLibCtx *functions_lib_ctx_src, int replace, sds *err) {
int ret = C_ERR; int ret = C_ERR;
dictIterator *iter = NULL; dictIterator iter;
/* Stores the libraries we need to replace in case a revert is required. /* Stores the libraries we need to replace in case a revert is required.
* Only initialized when needed */ * Only initialized when needed */
list *old_libraries_list = NULL; list *old_libraries_list = NULL;
dictEntry *entry = NULL; dictEntry *entry = NULL;
iter = dictGetIterator(functions_lib_ctx_src->libraries); dictInitIterator(&iter, functions_lib_ctx_src->libraries);
while ((entry = dictNext(iter))) { while ((entry = dictNext(&iter))) {
functionLibInfo *li = dictGetVal(entry); functionLibInfo *li = dictGetVal(entry);
functionLibInfo *old_li = dictFetchValue(functions_lib_ctx_dst->libraries, li->name); functionLibInfo *old_li = dictFetchValue(functions_lib_ctx_dst->libraries, li->name);
if (old_li) { if (old_li) {
@ -350,30 +354,28 @@ static int libraryJoin(functionsLibCtx *functions_lib_ctx_dst, functionsLibCtx *
} }
} }
} }
dictReleaseIterator(iter); dictResetIterator(&iter);
iter = NULL;
/* Make sure no functions collision */ /* Make sure no functions collision */
iter = dictGetIterator(functions_lib_ctx_src->functions); dictInitIterator(&iter, functions_lib_ctx_src->functions);
while ((entry = dictNext(iter))) { while ((entry = dictNext(&iter))) {
functionInfo *fi = dictGetVal(entry); functionInfo *fi = dictGetVal(entry);
if (dictFetchValue(functions_lib_ctx_dst->functions, fi->name)) { if (dictFetchValue(functions_lib_ctx_dst->functions, fi->name)) {
*err = sdscatfmt(sdsempty(), "Function %s already exists", fi->name); *err = sdscatfmt(sdsempty(), "Function %s already exists", fi->name);
dictResetIterator(&iter);
goto done; goto done;
} }
} }
dictReleaseIterator(iter); dictResetIterator(&iter);
iter = NULL;
/* No collision, it is safe to link all the new libraries. */ /* No collision, it is safe to link all the new libraries. */
iter = dictGetIterator(functions_lib_ctx_src->libraries); dictInitIterator(&iter, functions_lib_ctx_src->libraries);
while ((entry = dictNext(iter))) { while ((entry = dictNext(&iter))) {
functionLibInfo *li = dictGetVal(entry); functionLibInfo *li = dictGetVal(entry);
libraryLink(functions_lib_ctx_dst, li); libraryLink(functions_lib_ctx_dst, li);
dictSetVal(functions_lib_ctx_src->libraries, entry, NULL); dictSetVal(functions_lib_ctx_src->libraries, entry, NULL);
} }
dictReleaseIterator(iter); dictResetIterator(&iter);
iter = NULL;
functionsLibCtxClear(functions_lib_ctx_src); functionsLibCtxClear(functions_lib_ctx_src);
if (old_libraries_list) { if (old_libraries_list) {
@ -383,7 +385,6 @@ static int libraryJoin(functionsLibCtx *functions_lib_ctx_dst, functionsLibCtx *
ret = C_OK; ret = C_OK;
done: done:
if (iter) dictReleaseIterator(iter);
if (old_libraries_list) { if (old_libraries_list) {
/* Link back all libraries on tmp_l_ctx */ /* Link back all libraries on tmp_l_ctx */
while (listLength(old_libraries_list) > 0) { while (listLength(old_libraries_list) > 0) {
@ -454,9 +455,10 @@ void functionStatsCommand(client *c) {
addReplyBulkCString(c, "engines"); addReplyBulkCString(c, "engines");
addReplyMapLen(c, dictSize(engines)); addReplyMapLen(c, dictSize(engines));
dictIterator *iter = dictGetIterator(engines); dictIterator iter;
dictEntry *entry = NULL; dictEntry *entry = NULL;
while ((entry = dictNext(iter))) { dictInitIterator(&iter, engines);
while ((entry = dictNext(&iter))) {
engineInfo *ei = dictGetVal(entry); engineInfo *ei = dictGetVal(entry);
addReplyBulkCString(c, ei->name); addReplyBulkCString(c, ei->name);
addReplyMapLen(c, 2); addReplyMapLen(c, 2);
@ -466,7 +468,7 @@ void functionStatsCommand(client *c) {
addReplyBulkCString(c, "functions_count"); addReplyBulkCString(c, "functions_count");
addReplyLongLong(c, e_stats->n_functions); addReplyLongLong(c, e_stats->n_functions);
} }
dictReleaseIterator(iter); dictResetIterator(&iter);
} }
static void functionListReplyFlags(client *c, functionInfo *fi) { static void functionListReplyFlags(client *c, functionInfo *fi) {
@ -528,9 +530,10 @@ void functionListCommand(client *c) {
/* If no pattern is asked we know the reply len and we can just set it */ /* If no pattern is asked we know the reply len and we can just set it */
addReplyArrayLen(c, dictSize(curr_functions_lib_ctx->libraries)); addReplyArrayLen(c, dictSize(curr_functions_lib_ctx->libraries));
} }
dictIterator *iter = dictGetIterator(curr_functions_lib_ctx->libraries); dictIterator iter;
dictEntry *entry = NULL; dictEntry *entry = NULL;
while ((entry = dictNext(iter))) { dictInitIterator(&iter, curr_functions_lib_ctx->libraries);
while ((entry = dictNext(&iter))) {
functionLibInfo *li = dictGetVal(entry); functionLibInfo *li = dictGetVal(entry);
if (library_name) { if (library_name) {
if (!stringmatchlen(library_name, sdslen(library_name), li->name, sdslen(li->name), 1)) { if (!stringmatchlen(library_name, sdslen(library_name), li->name, sdslen(li->name), 1)) {
@ -546,9 +549,10 @@ void functionListCommand(client *c) {
addReplyBulkCString(c, "functions"); addReplyBulkCString(c, "functions");
addReplyArrayLen(c, dictSize(li->functions)); addReplyArrayLen(c, dictSize(li->functions));
dictIterator *functions_iter = dictGetIterator(li->functions); dictIterator functions_iter;
dictEntry *function_entry = NULL; dictEntry *function_entry = NULL;
while ((function_entry = dictNext(functions_iter))) { dictInitIterator(&functions_iter, li->functions);
while ((function_entry = dictNext(&functions_iter))) {
functionInfo *fi = dictGetVal(function_entry); functionInfo *fi = dictGetVal(function_entry);
addReplyMapLen(c, 3); addReplyMapLen(c, 3);
addReplyBulkCString(c, "name"); addReplyBulkCString(c, "name");
@ -562,14 +566,14 @@ void functionListCommand(client *c) {
addReplyBulkCString(c, "flags"); addReplyBulkCString(c, "flags");
functionListReplyFlags(c, fi); functionListReplyFlags(c, fi);
} }
dictReleaseIterator(functions_iter); dictResetIterator(&functions_iter);
if (with_code) { if (with_code) {
addReplyBulkCString(c, "library_code"); addReplyBulkCString(c, "library_code");
addReplyBulkCBuffer(c, li->code, sdslen(li->code)); addReplyBulkCBuffer(c, li->code, sdslen(li->code));
} }
} }
dictReleaseIterator(iter); dictResetIterator(&iter);
if (len_ptr) { if (len_ptr) {
setDeferredArrayLen(c, len_ptr, reply_len); setDeferredArrayLen(c, len_ptr, reply_len);
} }
@ -951,7 +955,7 @@ void functionFreeLibMetaData(functionsLibMataData *md) {
/* Compile and save the given library, return the loaded library name on success /* Compile and save the given library, return the loaded library name on success
* and NULL on failure. In case on failure the err out param is set with relevant error message */ * and NULL on failure. In case on failure the err out param is set with relevant error message */
sds functionsCreateWithLibraryCtx(sds code, int replace, sds* err, functionsLibCtx *lib_ctx, size_t timeout) { sds functionsCreateWithLibraryCtx(sds code, int replace, sds* err, functionsLibCtx *lib_ctx, size_t timeout) {
dictIterator *iter = NULL; dictIterator iter;
dictEntry *entry = NULL; dictEntry *entry = NULL;
functionLibInfo *new_li = NULL; functionLibInfo *new_li = NULL;
functionLibInfo *old_li = NULL; functionLibInfo *old_li = NULL;
@ -994,17 +998,17 @@ sds functionsCreateWithLibraryCtx(sds code, int replace, sds* err, functionsLibC
} }
/* Verify no duplicate functions */ /* Verify no duplicate functions */
iter = dictGetIterator(new_li->functions); dictInitIterator(&iter, new_li->functions);
while ((entry = dictNext(iter))) { while ((entry = dictNext(&iter))) {
functionInfo *fi = dictGetVal(entry); functionInfo *fi = dictGetVal(entry);
if (dictFetchValue(lib_ctx->functions, fi->name)) { if (dictFetchValue(lib_ctx->functions, fi->name)) {
/* functions name collision, abort. */ /* functions name collision, abort. */
*err = sdscatfmt(sdsempty(), "Function %s already exists", fi->name); *err = sdscatfmt(sdsempty(), "Function %s already exists", fi->name);
dictResetIterator(&iter);
goto error; goto error;
} }
} }
dictReleaseIterator(iter); dictResetIterator(&iter);
iter = NULL;
libraryLink(lib_ctx, new_li); libraryLink(lib_ctx, new_li);
@ -1019,7 +1023,6 @@ sds functionsCreateWithLibraryCtx(sds code, int replace, sds* err, functionsLibC
return loaded_lib_name; return loaded_lib_name;
error: error:
if (iter) dictReleaseIterator(iter);
if (new_li) engineLibraryFree(new_li); if (new_li) engineLibraryFree(new_li);
if (old_li) libraryLink(lib_ctx, old_li); if (old_li) libraryLink(lib_ctx, old_li);
functionFreeLibMetaData(&md); functionFreeLibMetaData(&md);
@ -1069,15 +1072,17 @@ void functionLoadCommand(client *c) {
/* Return memory usage of all the engines combine */ /* Return memory usage of all the engines combine */
unsigned long functionsMemoryVM(void) { unsigned long functionsMemoryVM(void) {
dictIterator *iter = dictGetIterator(engines); dictIterator iter;
dictEntry *entry = NULL; dictEntry *entry = NULL;
size_t engines_memory = 0; size_t engines_memory = 0;
while ((entry = dictNext(iter))) {
dictInitIterator(&iter, engines);
while ((entry = dictNext(&iter))) {
engineInfo *ei = dictGetVal(entry); engineInfo *ei = dictGetVal(entry);
engine *engine = ei->engine; engine *engine = ei->engine;
engines_memory += engine->get_used_memory(engine->engine_ctx); engines_memory += engine->get_used_memory(engine->engine_ctx);
} }
dictReleaseIterator(iter); dictResetIterator(&iter);
return engines_memory; return engines_memory;
} }

View File

@ -98,12 +98,12 @@ void latencyAddSample(const char *event, mstime_t latency) {
* Note: this is O(N) even when event_to_reset is not NULL because makes * Note: this is O(N) even when event_to_reset is not NULL because makes
* the code simpler and we have a small fixed max number of events. */ * the code simpler and we have a small fixed max number of events. */
int latencyResetEvent(char *event_to_reset) { int latencyResetEvent(char *event_to_reset) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
int resets = 0; int resets = 0;
di = dictGetSafeIterator(server.latency_events); dictInitSafeIterator(&di, server.latency_events);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
char *event = dictGetKey(de); char *event = dictGetKey(de);
if (event_to_reset == NULL || strcasecmp(event,event_to_reset) == 0) { if (event_to_reset == NULL || strcasecmp(event,event_to_reset) == 0) {
@ -111,7 +111,7 @@ int latencyResetEvent(char *event_to_reset) {
resets++; resets++;
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
return resets; return resets;
} }
@ -210,12 +210,12 @@ sds createLatencyReport(void) {
/* Show all the events stats and add for each event some event-related /* Show all the events stats and add for each event some event-related
* comment depending on the values. */ * comment depending on the values. */
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
int eventnum = 0; int eventnum = 0;
di = dictGetSafeIterator(server.latency_events); dictInitSafeIterator(&di, server.latency_events);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
char *event = dictGetKey(de); char *event = dictGetKey(de);
struct latencyTimeSeries *ts = dictGetVal(de); struct latencyTimeSeries *ts = dictGetVal(de);
struct latencyStats ls; struct latencyStats ls;
@ -343,7 +343,7 @@ sds createLatencyReport(void) {
report = sdscatlen(report,"\n",1); report = sdscatlen(report,"\n",1);
} }
dictReleaseIterator(di); dictResetIterator(&di);
/* Add non event based advices. */ /* Add non event based advices. */
if (THPGetAnonHugePagesSize() > 0) { if (THPGetAnonHugePagesSize() > 0) {
@ -471,11 +471,12 @@ void fillCommandCDF(client *c, struct hdr_histogram* histogram) {
/* latencyCommand() helper to produce for all commands, /* latencyCommand() helper to produce for all commands,
* a per command cumulative distribution of latencies. */ * a per command cumulative distribution of latencies. */
void latencyAllCommandsFillCDF(client *c, dict *commands, int *command_with_data) { void latencyAllCommandsFillCDF(client *c, dict *commands, int *command_with_data) {
dictIterator *di = dictGetSafeIterator(commands); dictIterator di;
dictEntry *de; dictEntry *de;
struct redisCommand *cmd; struct redisCommand *cmd;
while((de = dictNext(di)) != NULL) { dictInitSafeIterator(&di, commands);
while((de = dictNext(&di)) != NULL) {
cmd = (struct redisCommand *) dictGetVal(de); cmd = (struct redisCommand *) dictGetVal(de);
if (cmd->latency_histogram) { if (cmd->latency_histogram) {
addReplyBulkCBuffer(c, cmd->fullname, sdslen(cmd->fullname)); addReplyBulkCBuffer(c, cmd->fullname, sdslen(cmd->fullname));
@ -487,7 +488,7 @@ void latencyAllCommandsFillCDF(client *c, dict *commands, int *command_with_data
latencyAllCommandsFillCDF(c, cmd->subcommands_dict, command_with_data); latencyAllCommandsFillCDF(c, cmd->subcommands_dict, command_with_data);
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* latencyCommand() helper to produce for a specific command set, /* latencyCommand() helper to produce for a specific command set,
@ -510,9 +511,10 @@ void latencySpecificCommandsFillCDF(client *c) {
if (cmd->subcommands_dict) { if (cmd->subcommands_dict) {
dictEntry *de; dictEntry *de;
dictIterator *di = dictGetSafeIterator(cmd->subcommands_dict); dictIterator di;
while ((de = dictNext(di)) != NULL) { dictInitSafeIterator(&di, cmd->subcommands_dict);
while ((de = dictNext(&di)) != NULL) {
struct redisCommand *sub = dictGetVal(de); struct redisCommand *sub = dictGetVal(de);
if (sub->latency_histogram) { if (sub->latency_histogram) {
addReplyBulkCBuffer(c, sub->fullname, sdslen(sub->fullname)); addReplyBulkCBuffer(c, sub->fullname, sdslen(sub->fullname));
@ -520,7 +522,7 @@ void latencySpecificCommandsFillCDF(client *c) {
command_with_data++; command_with_data++;
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
} }
setDeferredMapLen(c,replylen,command_with_data); setDeferredMapLen(c,replylen,command_with_data);
@ -547,12 +549,12 @@ void latencyCommandReplyWithSamples(client *c, struct latencyTimeSeries *ts) {
/* latencyCommand() helper to produce the reply for the LATEST subcommand, /* latencyCommand() helper to produce the reply for the LATEST subcommand,
* listing the last latency sample for every event type registered so far. */ * listing the last latency sample for every event type registered so far. */
void latencyCommandReplyWithLatestEvents(client *c) { void latencyCommandReplyWithLatestEvents(client *c) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
addReplyArrayLen(c,dictSize(server.latency_events)); addReplyArrayLen(c,dictSize(server.latency_events));
di = dictGetIterator(server.latency_events); dictInitIterator(&di, server.latency_events);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
char *event = dictGetKey(de); char *event = dictGetKey(de);
struct latencyTimeSeries *ts = dictGetVal(de); struct latencyTimeSeries *ts = dictGetVal(de);
int last = (ts->idx + LATENCY_TS_LEN - 1) % LATENCY_TS_LEN; int last = (ts->idx + LATENCY_TS_LEN - 1) % LATENCY_TS_LEN;
@ -563,7 +565,7 @@ void latencyCommandReplyWithLatestEvents(client *c) {
addReplyLongLong(c,ts->samples[last].latency); addReplyLongLong(c,ts->samples[last].latency);
addReplyLongLong(c,ts->max); addReplyLongLong(c,ts->max);
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
#define LATENCY_GRAPH_COLS 80 #define LATENCY_GRAPH_COLS 80

View File

@ -6806,10 +6806,11 @@ uint64_t moduleTypeEncodeId(const char *name, int encver) {
* a type with the same name as the one given. Returns the moduleType * a type with the same name as the one given. Returns the moduleType
* structure pointer if such a module is found, or NULL otherwise. */ * structure pointer if such a module is found, or NULL otherwise. */
moduleType *moduleTypeLookupModuleByNameInternal(const char *name, int ignore_case) { moduleType *moduleTypeLookupModuleByNameInternal(const char *name, int ignore_case) {
dictIterator *di = dictGetIterator(modules); dictIterator di;
dictEntry *de; dictEntry *de;
while ((de = dictNext(di)) != NULL) { dictInitIterator(&di, modules);
while ((de = dictNext(&di)) != NULL) {
struct RedisModule *module = dictGetVal(de); struct RedisModule *module = dictGetVal(de);
listIter li; listIter li;
listNode *ln; listNode *ln;
@ -6820,12 +6821,12 @@ moduleType *moduleTypeLookupModuleByNameInternal(const char *name, int ignore_ca
if ((!ignore_case && memcmp(name,mt->name,sizeof(mt->name)) == 0) if ((!ignore_case && memcmp(name,mt->name,sizeof(mt->name)) == 0)
|| (ignore_case && !strcasecmp(name, mt->name))) || (ignore_case && !strcasecmp(name, mt->name)))
{ {
dictReleaseIterator(di); dictResetIterator(&di);
return mt; return mt;
} }
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
return NULL; return NULL;
} }
/* Search all registered modules by name, and name is case sensitive */ /* Search all registered modules by name, and name is case sensitive */
@ -6856,10 +6857,11 @@ moduleType *moduleTypeLookupModuleByID(uint64_t id) {
/* Slow module by module lookup. */ /* Slow module by module lookup. */
moduleType *mt = NULL; moduleType *mt = NULL;
dictIterator *di = dictGetIterator(modules); dictIterator di;
dictEntry *de; dictEntry *de;
while ((de = dictNext(di)) != NULL && mt == NULL) { dictInitIterator(&di, modules);
while ((de = dictNext(&di)) != NULL && mt == NULL) {
struct RedisModule *module = dictGetVal(de); struct RedisModule *module = dictGetVal(de);
listIter li; listIter li;
listNode *ln; listNode *ln;
@ -6875,7 +6877,7 @@ moduleType *moduleTypeLookupModuleByID(uint64_t id) {
} }
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
/* Add to cache if possible. */ /* Add to cache if possible. */
if (mt && j < MODULE_LOOKUP_CACHE_SIZE) { if (mt && j < MODULE_LOOKUP_CACHE_SIZE) {
@ -7202,19 +7204,20 @@ void moduleRDBLoadError(RedisModuleIO *io) {
* REDISMODULE_OPTIONS_HANDLE_IO_ERRORS, in which case diskless loading should * REDISMODULE_OPTIONS_HANDLE_IO_ERRORS, in which case diskless loading should
* be avoided since it could cause data loss. */ * be avoided since it could cause data loss. */
int moduleAllDatatypesHandleErrors(void) { int moduleAllDatatypesHandleErrors(void) {
dictIterator *di = dictGetIterator(modules); dictIterator di;
dictEntry *de; dictEntry *de;
while ((de = dictNext(di)) != NULL) { dictInitIterator(&di, modules);
while ((de = dictNext(&di)) != NULL) {
struct RedisModule *module = dictGetVal(de); struct RedisModule *module = dictGetVal(de);
if (listLength(module->types) && if (listLength(module->types) &&
!(module->options & REDISMODULE_OPTIONS_HANDLE_IO_ERRORS)) !(module->options & REDISMODULE_OPTIONS_HANDLE_IO_ERRORS))
{ {
dictReleaseIterator(di); dictResetIterator(&di);
return 0; return 0;
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
return 1; return 1;
} }
@ -7222,17 +7225,18 @@ int moduleAllDatatypesHandleErrors(void) {
* diskless async loading should be avoided because module doesn't know there can be traffic during * diskless async loading should be avoided because module doesn't know there can be traffic during
* database full resynchronization. */ * database full resynchronization. */
int moduleAllModulesHandleReplAsyncLoad(void) { int moduleAllModulesHandleReplAsyncLoad(void) {
dictIterator *di = dictGetIterator(modules); dictIterator di;
dictEntry *de; dictEntry *de;
while ((de = dictNext(di)) != NULL) { dictInitIterator(&di, modules);
while ((de = dictNext(&di)) != NULL) {
struct RedisModule *module = dictGetVal(de); struct RedisModule *module = dictGetVal(de);
if (!(module->options & REDISMODULE_OPTIONS_HANDLE_REPL_ASYNC_LOAD)) { if (!(module->options & REDISMODULE_OPTIONS_HANDLE_REPL_ASYNC_LOAD)) {
dictReleaseIterator(di); dictResetIterator(&di);
return 0; return 0;
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
return 1; return 1;
} }
@ -7489,10 +7493,11 @@ long double RM_LoadLongDouble(RedisModuleIO *io) {
* who asked for it. */ * who asked for it. */
ssize_t rdbSaveModulesAux(rio *rdb, int when) { ssize_t rdbSaveModulesAux(rio *rdb, int when) {
size_t total_written = 0; size_t total_written = 0;
dictIterator *di = dictGetIterator(modules); dictIterator di;
dictEntry *de; dictEntry *de;
while ((de = dictNext(di)) != NULL) { dictInitIterator(&di, modules);
while ((de = dictNext(&di)) != NULL) {
struct RedisModule *module = dictGetVal(de); struct RedisModule *module = dictGetVal(de);
listIter li; listIter li;
listNode *ln; listNode *ln;
@ -7504,14 +7509,14 @@ ssize_t rdbSaveModulesAux(rio *rdb, int when) {
continue; continue;
ssize_t ret = rdbSaveSingleModuleAux(rdb, when, mt); ssize_t ret = rdbSaveSingleModuleAux(rdb, when, mt);
if (ret==-1) { if (ret==-1) {
dictReleaseIterator(di); dictResetIterator(&di);
return -1; return -1;
} }
total_written += ret; total_written += ret;
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
return total_written; return total_written;
} }
@ -10582,10 +10587,11 @@ int RM_RegisterInfoFunc(RedisModuleCtx *ctx, RedisModuleInfoFunc cb) {
} }
sds modulesCollectInfo(sds info, dict *sections_dict, int for_crash_report, int sections) { sds modulesCollectInfo(sds info, dict *sections_dict, int for_crash_report, int sections) {
dictIterator *di = dictGetIterator(modules); dictIterator di;
dictEntry *de; dictEntry *de;
while ((de = dictNext(di)) != NULL) { dictInitIterator(&di, modules);
while ((de = dictNext(&di)) != NULL) {
struct RedisModule *module = dictGetVal(de); struct RedisModule *module = dictGetVal(de);
if (!module->info_cb) if (!module->info_cb)
continue; continue;
@ -10597,7 +10603,7 @@ sds modulesCollectInfo(sds info, dict *sections_dict, int for_crash_report, int
info = info_ctx.info; info = info_ctx.info;
sections = info_ctx.sections; sections = info_ctx.sections;
} }
dictReleaseIterator(di); dictResetIterator(&di);
return info; return info;
} }
@ -10821,9 +10827,10 @@ void *RM_GetSharedAPI(RedisModuleCtx *ctx, const char *apiname) {
* The number of unregistered APIs is returned. */ * The number of unregistered APIs is returned. */
int moduleUnregisterSharedAPI(RedisModule *module) { int moduleUnregisterSharedAPI(RedisModule *module) {
int count = 0; int count = 0;
dictIterator *di = dictGetSafeIterator(server.sharedapi); dictIterator di;
dictEntry *de; dictEntry *de;
while ((de = dictNext(di)) != NULL) { dictInitSafeIterator(&di, server.sharedapi);
while ((de = dictNext(&di)) != NULL) {
const char *apiname = dictGetKey(de); const char *apiname = dictGetKey(de);
RedisModuleSharedAPI *sapi = dictGetVal(de); RedisModuleSharedAPI *sapi = dictGetVal(de);
if (sapi->module == module) { if (sapi->module == module) {
@ -10832,7 +10839,7 @@ int moduleUnregisterSharedAPI(RedisModule *module) {
count++; count++;
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
return count; return count;
} }
@ -12325,11 +12332,13 @@ void moduleLoadFromQueue(void) {
} }
if (dictSize(server.module_configs_queue)) { if (dictSize(server.module_configs_queue)) {
serverLog(LL_WARNING, "Unresolved Configuration(s) Detected:"); serverLog(LL_WARNING, "Unresolved Configuration(s) Detected:");
dictIterator *di = dictGetIterator(server.module_configs_queue); dictIterator di;
dictEntry *de; dictEntry *de;
while ((de = dictNext(di)) != NULL) { dictInitIterator(&di, server.module_configs_queue);
while ((de = dictNext(&di)) != NULL) {
serverLog(LL_WARNING, ">>> '%s %s'", (char *)dictGetKey(de), (char *)dictGetVal(de)); serverLog(LL_WARNING, ">>> '%s %s'", (char *)dictGetKey(de), (char *)dictGetVal(de));
} }
dictResetIterator(&di);
serverLog(LL_WARNING, "Module Configuration detected without loadmodule directive or no ApplyConfig call: aborting"); serverLog(LL_WARNING, "Module Configuration detected without loadmodule directive or no ApplyConfig call: aborting");
exit(1); exit(1);
} }
@ -12405,8 +12414,9 @@ int moduleFreeCommand(struct RedisModule *module, struct redisCommand *cmd) {
if (cmd->subcommands_dict) { if (cmd->subcommands_dict) {
dictEntry *de; dictEntry *de;
dictIterator *di = dictGetSafeIterator(cmd->subcommands_dict); dictIterator di;
while ((de = dictNext(di)) != NULL) { dictInitSafeIterator(&di, cmd->subcommands_dict);
while ((de = dictNext(&di)) != NULL) {
struct redisCommand *sub = dictGetVal(de); struct redisCommand *sub = dictGetVal(de);
if (moduleFreeCommand(module, sub) != C_OK) continue; if (moduleFreeCommand(module, sub) != C_OK) continue;
@ -12415,7 +12425,7 @@ int moduleFreeCommand(struct RedisModule *module, struct redisCommand *cmd) {
sdsfree(sub->fullname); sdsfree(sub->fullname);
zfree(sub); zfree(sub);
} }
dictReleaseIterator(di); dictResetIterator(&di);
dictRelease(cmd->subcommands_dict); dictRelease(cmd->subcommands_dict);
} }
@ -12425,9 +12435,10 @@ int moduleFreeCommand(struct RedisModule *module, struct redisCommand *cmd) {
void moduleUnregisterCommands(struct RedisModule *module) { void moduleUnregisterCommands(struct RedisModule *module) {
pauseAllIOThreads(); pauseAllIOThreads();
/* Unregister all the commands registered by this module. */ /* Unregister all the commands registered by this module. */
dictIterator *di = dictGetSafeIterator(server.commands); dictIterator di;
dictEntry *de; dictEntry *de;
while ((de = dictNext(di)) != NULL) { dictInitSafeIterator(&di, server.commands);
while ((de = dictNext(&di)) != NULL) {
struct redisCommand *cmd = dictGetVal(de); struct redisCommand *cmd = dictGetVal(de);
if (moduleFreeCommand(module, cmd) != C_OK) continue; if (moduleFreeCommand(module, cmd) != C_OK) continue;
@ -12437,7 +12448,7 @@ void moduleUnregisterCommands(struct RedisModule *module) {
sdsfree(cmd->fullname); sdsfree(cmd->fullname);
zfree(cmd); zfree(cmd);
} }
dictReleaseIterator(di); dictResetIterator(&di);
resumeAllIOThreads(); resumeAllIOThreads();
} }
@ -12689,10 +12700,11 @@ void addReplyLoadedModules(client *c) {
if (ln == 0) { if (ln == 0) {
return; return;
} }
dictIterator *di = dictGetIterator(modules); dictIterator di;
dictEntry *de; dictEntry *de;
while ((de = dictNext(di)) != NULL) { dictInitIterator(&di, modules);
while ((de = dictNext(&di)) != NULL) {
sds name = dictGetKey(de); sds name = dictGetKey(de);
struct RedisModule *module = dictGetVal(de); struct RedisModule *module = dictGetVal(de);
sds path = module->loadmod->path; sds path = module->loadmod->path;
@ -12709,7 +12721,7 @@ void addReplyLoadedModules(client *c) {
addReplyBulk(c,module->loadmod->argv[i]); addReplyBulk(c,module->loadmod->argv[i]);
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* Helper for genModulesInfoString(): given a list of modules, return /* Helper for genModulesInfoString(): given a list of modules, return
@ -12750,10 +12762,11 @@ sds genModulesInfoStringRenderModuleOptions(struct RedisModule *module) {
* After the call, the passed sds info string is no longer valid and all the * After the call, the passed sds info string is no longer valid and all the
* references must be substituted with the new pointer returned by the call. */ * references must be substituted with the new pointer returned by the call. */
sds genModulesInfoString(sds info) { sds genModulesInfoString(sds info) {
dictIterator *di = dictGetIterator(modules); dictIterator di;
dictEntry *de; dictEntry *de;
while ((de = dictNext(di)) != NULL) { dictInitIterator(&di, modules);
while ((de = dictNext(&di)) != NULL) {
sds name = dictGetKey(de); sds name = dictGetKey(de);
struct RedisModule *module = dictGetVal(de); struct RedisModule *module = dictGetVal(de);
@ -12769,7 +12782,7 @@ sds genModulesInfoString(sds info) {
sdsfree(using); sdsfree(using);
sdsfree(options); sdsfree(options);
} }
dictReleaseIterator(di); dictResetIterator(&di);
return info; return info;
} }

View File

@ -418,8 +418,9 @@ void touchAllWatchedKeysInDb(redisDb *emptied, redisDb *replaced_with) {
if (dictSize(emptied->watched_keys) == 0) return; if (dictSize(emptied->watched_keys) == 0) return;
dictIterator *di = dictGetSafeIterator(emptied->watched_keys); dictIterator di;
while((de = dictNext(di)) != NULL) { dictInitSafeIterator(&di, emptied->watched_keys);
while((de = dictNext(&di)) != NULL) {
robj *key = dictGetKey(de); robj *key = dictGetKey(de);
int exists_in_emptied = dbFind(emptied, key->ptr) != NULL; int exists_in_emptied = dbFind(emptied, key->ptr) != NULL;
if (exists_in_emptied || if (exists_in_emptied ||
@ -453,7 +454,7 @@ void touchAllWatchedKeysInDb(redisDb *emptied, redisDb *replaced_with) {
} }
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
void watchCommand(client *c) { void watchCommand(client *c) {

View File

@ -658,11 +658,12 @@ void dismissSetObject(robj *o, size_t size_hint) {
* page size, and there's a high chance we'll actually dismiss something. */ * page size, and there's a high chance we'll actually dismiss something. */
if (size_hint / dictSize(set) >= server.page_size) { if (size_hint / dictSize(set) >= server.page_size) {
dictEntry *de; dictEntry *de;
dictIterator *di = dictGetIterator(set); dictIterator di;
while ((de = dictNext(di)) != NULL) { dictInitIterator(&di, set);
while ((de = dictNext(&di)) != NULL) {
dismissSds(dictGetKey(de)); dismissSds(dictGetKey(de));
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* Dismiss hash table memory. */ /* Dismiss hash table memory. */
@ -713,13 +714,14 @@ void dismissHashObject(robj *o, size_t size_hint) {
* a page size, and there's a high chance we'll actually dismiss something. */ * a page size, and there's a high chance we'll actually dismiss something. */
if (size_hint / dictSize(d) >= server.page_size) { if (size_hint / dictSize(d) >= server.page_size) {
dictEntry *de; dictEntry *de;
dictIterator *di = dictGetIterator(d); dictIterator di;
while ((de = dictNext(di)) != NULL) { dictInitIterator(&di, d);
while ((de = dictNext(&di)) != NULL) {
/* Only dismiss values memory since the field size /* Only dismiss values memory since the field size
* usually is small. */ * usually is small. */
dismissSds(dictGetVal(de)); dismissSds(dictGetVal(de));
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* Dismiss hash table memory. */ /* Dismiss hash table memory. */
@ -1207,7 +1209,7 @@ size_t streamRadixTreeMemoryUsage(rax *rax) {
#define OBJ_COMPUTE_SIZE_DEF_SAMPLES 5 /* Default sample size. */ #define OBJ_COMPUTE_SIZE_DEF_SAMPLES 5 /* Default sample size. */
size_t objectComputeSize(robj *key, robj *o, size_t sample_size, int dbid) { size_t objectComputeSize(robj *key, robj *o, size_t sample_size, int dbid) {
dict *d; dict *d;
dictIterator *di; dictIterator di;
struct dictEntry *de; struct dictEntry *de;
size_t asize = 0, elesize = 0, elecount = 0, samples = 0; size_t asize = 0, elesize = 0, elecount = 0, samples = 0;
@ -1240,14 +1242,14 @@ size_t objectComputeSize(robj *key, robj *o, size_t sample_size, int dbid) {
} else if (o->type == OBJ_SET) { } else if (o->type == OBJ_SET) {
if (o->encoding == OBJ_ENCODING_HT) { if (o->encoding == OBJ_ENCODING_HT) {
d = o->ptr; d = o->ptr;
di = dictGetIterator(d); dictInitIterator(&di, d);
asize = sizeof(*o)+sizeof(dict)+(sizeof(struct dictEntry*)*dictBuckets(d)); asize = sizeof(*o)+sizeof(dict)+(sizeof(struct dictEntry*)*dictBuckets(d));
while((de = dictNext(di)) != NULL && samples < sample_size) { while((de = dictNext(&di)) != NULL && samples < sample_size) {
sds ele = dictGetKey(de); sds ele = dictGetKey(de);
elesize += dictEntryMemUsage(0) + sdsZmallocSize(ele); elesize += dictEntryMemUsage(0) + sdsZmallocSize(ele);
samples++; samples++;
} }
dictReleaseIterator(di); dictResetIterator(&di);
if (samples) asize += (double)elesize/samples*dictSize(d); if (samples) asize += (double)elesize/samples*dictSize(d);
} else if (o->encoding == OBJ_ENCODING_INTSET) { } else if (o->encoding == OBJ_ENCODING_INTSET) {
asize = sizeof(*o)+zmalloc_size(o->ptr); asize = sizeof(*o)+zmalloc_size(o->ptr);
@ -1284,16 +1286,16 @@ size_t objectComputeSize(robj *key, robj *o, size_t sample_size, int dbid) {
asize = sizeof(*o) + zmalloc_size(lpt) + zmalloc_size(lpt->lp); asize = sizeof(*o) + zmalloc_size(lpt) + zmalloc_size(lpt->lp);
} else if (o->encoding == OBJ_ENCODING_HT) { } else if (o->encoding == OBJ_ENCODING_HT) {
d = o->ptr; d = o->ptr;
di = dictGetIterator(d); dictInitIterator(&di, d);
asize = sizeof(*o)+sizeof(dict)+(sizeof(struct dictEntry*)*dictBuckets(d)); asize = sizeof(*o)+sizeof(dict)+(sizeof(struct dictEntry*)*dictBuckets(d));
while((de = dictNext(di)) != NULL && samples < sample_size) { while((de = dictNext(&di)) != NULL && samples < sample_size) {
hfield ele = dictGetKey(de); hfield ele = dictGetKey(de);
sds ele2 = dictGetVal(de); sds ele2 = dictGetVal(de);
elesize += hfieldZmallocSize(ele) + sdsZmallocSize(ele2); elesize += hfieldZmallocSize(ele) + sdsZmallocSize(ele2);
elesize += dictEntryMemUsage(0); elesize += dictEntryMemUsage(0);
samples++; samples++;
} }
dictReleaseIterator(di); dictResetIterator(&di);
if (samples) asize += (double)elesize/samples*dictSize(d); if (samples) asize += (double)elesize/samples*dictSize(d);
} else { } else {
serverPanic("Unknown hash encoding"); serverPanic("Unknown hash encoding");

View File

@ -325,9 +325,11 @@ void pubsubShardUnsubscribeAllChannelsInSlot(unsigned int slot) {
robj *channel = dictGetKey(de); robj *channel = dictGetKey(de);
dict *clients = dictGetVal(de); dict *clients = dictGetVal(de);
/* For each client subscribed to the channel, unsubscribe it. */ /* For each client subscribed to the channel, unsubscribe it. */
dictIterator *iter = dictGetIterator(clients); dictIterator iter;
dictEntry *entry; dictEntry *entry;
while ((entry = dictNext(iter)) != NULL) {
dictInitIterator(&iter, clients);
while ((entry = dictNext(&iter)) != NULL) {
client *c = dictGetKey(entry); client *c = dictGetKey(entry);
int retval = dictDelete(c->pubsubshard_channels, channel); int retval = dictDelete(c->pubsubshard_channels, channel);
serverAssertWithInfo(c,channel,retval == DICT_OK); serverAssertWithInfo(c,channel,retval == DICT_OK);
@ -338,7 +340,7 @@ void pubsubShardUnsubscribeAllChannelsInSlot(unsigned int slot) {
unmarkClientAsPubSub(c); unmarkClientAsPubSub(c);
} }
} }
dictReleaseIterator(iter); dictResetIterator(&iter);
kvstoreDictDelete(server.pubsubshard_channels, slot, channel); kvstoreDictDelete(server.pubsubshard_channels, slot, channel);
} }
kvstoreReleaseDictIterator(kvs_di); kvstoreReleaseDictIterator(kvs_di);
@ -401,15 +403,16 @@ int pubsubUnsubscribePattern(client *c, robj *pattern, int notify) {
int pubsubUnsubscribeAllChannelsInternal(client *c, int notify, pubsubtype type) { int pubsubUnsubscribeAllChannelsInternal(client *c, int notify, pubsubtype type) {
int count = 0; int count = 0;
if (dictSize(type.clientPubSubChannels(c)) > 0) { if (dictSize(type.clientPubSubChannels(c)) > 0) {
dictIterator *di = dictGetSafeIterator(type.clientPubSubChannels(c)); dictIterator di;
dictEntry *de; dictEntry *de;
while((de = dictNext(di)) != NULL) { dictInitSafeIterator(&di, type.clientPubSubChannels(c));
while((de = dictNext(&di)) != NULL) {
robj *channel = dictGetKey(de); robj *channel = dictGetKey(de);
count += pubsubUnsubscribeChannel(c,channel,notify,type); count += pubsubUnsubscribeChannel(c,channel,notify,type);
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* We were subscribed to nothing? Still reply to the client. */ /* We were subscribed to nothing? Still reply to the client. */
if (notify && count == 0) { if (notify && count == 0) {
@ -440,14 +443,15 @@ int pubsubUnsubscribeAllPatterns(client *c, int notify) {
int count = 0; int count = 0;
if (dictSize(c->pubsub_patterns) > 0) { if (dictSize(c->pubsub_patterns) > 0) {
dictIterator *di = dictGetSafeIterator(c->pubsub_patterns); dictIterator di;
dictEntry *de; dictEntry *de;
while ((de = dictNext(di)) != NULL) { dictInitSafeIterator(&di, c->pubsub_patterns);
while ((de = dictNext(&di)) != NULL) {
robj *pattern = dictGetKey(de); robj *pattern = dictGetKey(de);
count += pubsubUnsubscribePattern(c, pattern, notify); count += pubsubUnsubscribePattern(c, pattern, notify);
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* We were subscribed to nothing? Still reply to the client. */ /* We were subscribed to nothing? Still reply to the client. */
@ -461,7 +465,7 @@ int pubsubUnsubscribeAllPatterns(client *c, int notify) {
int pubsubPublishMessageInternal(robj *channel, robj *message, pubsubtype type) { int pubsubPublishMessageInternal(robj *channel, robj *message, pubsubtype type) {
int receivers = 0; int receivers = 0;
dictEntry *de; dictEntry *de;
dictIterator *di; dictIterator di;
unsigned int slot = 0; unsigned int slot = 0;
/* Send to clients listening for that channel */ /* Send to clients listening for that channel */
@ -472,8 +476,10 @@ int pubsubPublishMessageInternal(robj *channel, robj *message, pubsubtype type)
if (de) { if (de) {
dict *clients = dictGetVal(de); dict *clients = dictGetVal(de);
dictEntry *entry; dictEntry *entry;
dictIterator *iter = dictGetIterator(clients); dictIterator iter;
while ((entry = dictNext(iter)) != NULL) {
dictInitIterator(&iter, clients);
while ((entry = dictNext(&iter)) != NULL) {
client *c = dictGetKey(entry); client *c = dictGetKey(entry);
addReplyPubsubMessage(c,channel,message,*type.messageBulk); addReplyPubsubMessage(c,channel,message,*type.messageBulk);
if (server.cluster_enabled && server.cluster_slot_stats_enabled) if (server.cluster_enabled && server.cluster_slot_stats_enabled)
@ -481,7 +487,7 @@ int pubsubPublishMessageInternal(robj *channel, robj *message, pubsubtype type)
updateClientMemUsageAndBucket(c); updateClientMemUsageAndBucket(c);
receivers++; receivers++;
} }
dictReleaseIterator(iter); dictResetIterator(&iter);
} }
if (type.shard) { if (type.shard) {
@ -490,10 +496,10 @@ int pubsubPublishMessageInternal(robj *channel, robj *message, pubsubtype type)
} }
/* Send to clients listening to matching channels */ /* Send to clients listening to matching channels */
di = dictGetIterator(server.pubsub_patterns); if (dictSize(server.pubsub_patterns) > 0) {
if (di) {
channel = getDecodedObject(channel); channel = getDecodedObject(channel);
while((de = dictNext(di)) != NULL) { dictInitIterator(&di, server.pubsub_patterns);
while((de = dictNext(&di)) != NULL) {
robj *pattern = dictGetKey(de); robj *pattern = dictGetKey(de);
dict *clients = dictGetVal(de); dict *clients = dictGetVal(de);
if (!stringmatchlen((char*)pattern->ptr, if (!stringmatchlen((char*)pattern->ptr,
@ -502,17 +508,19 @@ int pubsubPublishMessageInternal(robj *channel, robj *message, pubsubtype type)
sdslen(channel->ptr),0)) continue; sdslen(channel->ptr),0)) continue;
dictEntry *entry; dictEntry *entry;
dictIterator *iter = dictGetIterator(clients); dictIterator iter;
while ((entry = dictNext(iter)) != NULL) {
dictInitIterator(&iter, clients);
while ((entry = dictNext(&iter)) != NULL) {
client *c = dictGetKey(entry); client *c = dictGetKey(entry);
addReplyPubsubPatMessage(c,pattern,channel,message); addReplyPubsubPatMessage(c,pattern,channel,message);
updateClientMemUsageAndBucket(c); updateClientMemUsageAndBucket(c);
receivers++; receivers++;
} }
dictReleaseIterator(iter); dictResetIterator(&iter);
} }
decrRefCount(channel); decrRefCount(channel);
dictReleaseIterator(di); dictResetIterator(&di);
} }
return receivers; return receivers;
} }

View File

@ -880,26 +880,26 @@ ssize_t rdbSaveObject(rio *rdb, robj *o, robj *key, int dbid) {
/* Save a set value */ /* Save a set value */
if (o->encoding == OBJ_ENCODING_HT) { if (o->encoding == OBJ_ENCODING_HT) {
dict *set = o->ptr; dict *set = o->ptr;
dictIterator *di = dictGetIterator(set); dictIterator di;
dictEntry *de; dictEntry *de;
if ((n = rdbSaveLen(rdb,dictSize(set))) == -1) { if ((n = rdbSaveLen(rdb,dictSize(set))) == -1) {
dictReleaseIterator(di);
return -1; return -1;
} }
nwritten += n; nwritten += n;
while((de = dictNext(di)) != NULL) { dictInitIterator(&di, set);
while((de = dictNext(&di)) != NULL) {
sds ele = dictGetKey(de); sds ele = dictGetKey(de);
if ((n = rdbSaveRawString(rdb,(unsigned char*)ele,sdslen(ele))) if ((n = rdbSaveRawString(rdb,(unsigned char*)ele,sdslen(ele)))
== -1) == -1)
{ {
dictReleaseIterator(di); dictResetIterator(&di);
return -1; return -1;
} }
nwritten += n; nwritten += n;
} }
dictReleaseIterator(di); dictResetIterator(&di);
} else if (o->encoding == OBJ_ENCODING_INTSET) { } else if (o->encoding == OBJ_ENCODING_INTSET) {
size_t l = intsetBlobLen((intset*)o->ptr); size_t l = intsetBlobLen((intset*)o->ptr);
@ -969,7 +969,7 @@ ssize_t rdbSaveObject(rio *rdb, robj *o, robj *key, int dbid) {
nwritten += n; nwritten += n;
} else if (o->encoding == OBJ_ENCODING_HT) { } else if (o->encoding == OBJ_ENCODING_HT) {
int hashWithMeta = 0; /* RDB_TYPE_HASH_METADATA */ int hashWithMeta = 0; /* RDB_TYPE_HASH_METADATA */
dictIterator *di = dictGetIterator(o->ptr); dictIterator di;
dictEntry *de; dictEntry *de;
/* Determine the hash layout to use based on the presence of at least /* Determine the hash layout to use based on the presence of at least
* one field with a valid TTL. If such a field exists, employ the * one field with a valid TTL. If such a field exists, employ the
@ -983,20 +983,19 @@ ssize_t rdbSaveObject(rio *rdb, robj *o, robj *key, int dbid) {
hashWithMeta = 1; hashWithMeta = 1;
/* Save next field expire time of hash */ /* Save next field expire time of hash */
if (rdbSaveMillisecondTime(rdb, minExpire) == -1) { if (rdbSaveMillisecondTime(rdb, minExpire) == -1) {
dictReleaseIterator(di);
return -1; return -1;
} }
} }
/* save number of fields in hash */ /* save number of fields in hash */
if ((n = rdbSaveLen(rdb,dictSize((dict*)o->ptr))) == -1) { if ((n = rdbSaveLen(rdb,dictSize((dict*)o->ptr))) == -1) {
dictReleaseIterator(di);
return -1; return -1;
} }
nwritten += n; nwritten += n;
/* save all hash fields */ /* save all hash fields */
while((de = dictNext(di)) != NULL) { dictInitIterator(&di, o->ptr);
while((de = dictNext(&di)) != NULL) {
hfield field = dictGetKey(de); hfield field = dictGetKey(de);
sds value = dictGetVal(de); sds value = dictGetVal(de);
@ -1010,7 +1009,7 @@ ssize_t rdbSaveObject(rio *rdb, robj *o, robj *key, int dbid) {
*/ */
ttl = (expiryTime == EB_EXPIRE_TIME_INVALID) ? 0 : expiryTime - minExpire + 1; ttl = (expiryTime == EB_EXPIRE_TIME_INVALID) ? 0 : expiryTime - minExpire + 1;
if ((n = rdbSaveLen(rdb, ttl)) == -1) { if ((n = rdbSaveLen(rdb, ttl)) == -1) {
dictReleaseIterator(di); dictResetIterator(&di);
return -1; return -1;
} }
nwritten += n; nwritten += n;
@ -1020,7 +1019,7 @@ ssize_t rdbSaveObject(rio *rdb, robj *o, robj *key, int dbid) {
if ((n = rdbSaveRawString(rdb,(unsigned char*)field, if ((n = rdbSaveRawString(rdb,(unsigned char*)field,
hfieldlen(field))) == -1) hfieldlen(field))) == -1)
{ {
dictReleaseIterator(di); dictResetIterator(&di);
return -1; return -1;
} }
nwritten += n; nwritten += n;
@ -1029,12 +1028,12 @@ ssize_t rdbSaveObject(rio *rdb, robj *o, robj *key, int dbid) {
if ((n = rdbSaveRawString(rdb,(unsigned char*)value, if ((n = rdbSaveRawString(rdb,(unsigned char*)value,
sdslen(value))) == -1) sdslen(value))) == -1)
{ {
dictReleaseIterator(di); dictResetIterator(&di);
return -1; return -1;
} }
nwritten += n; nwritten += n;
} }
dictReleaseIterator(di); dictResetIterator(&di);
} else { } else {
serverPanic("Unknown hash encoding"); serverPanic("Unknown hash encoding");
} }
@ -1349,22 +1348,24 @@ error:
ssize_t rdbSaveFunctions(rio *rdb) { ssize_t rdbSaveFunctions(rio *rdb) {
dict *functions = functionsLibGet(); dict *functions = functionsLibGet();
dictIterator *iter = dictGetIterator(functions); dictIterator iter;
dictEntry *entry = NULL; dictEntry *entry = NULL;
ssize_t written = 0; ssize_t written = 0;
ssize_t ret; ssize_t ret;
while ((entry = dictNext(iter))) {
dictInitIterator(&iter, functions);
while ((entry = dictNext(&iter))) {
if ((ret = rdbSaveType(rdb, RDB_OPCODE_FUNCTION2)) < 0) goto werr; if ((ret = rdbSaveType(rdb, RDB_OPCODE_FUNCTION2)) < 0) goto werr;
written += ret; written += ret;
functionLibInfo *li = dictGetVal(entry); functionLibInfo *li = dictGetVal(entry);
if ((ret = rdbSaveRawString(rdb, (unsigned char *) li->code, sdslen(li->code))) < 0) goto werr; if ((ret = rdbSaveRawString(rdb, (unsigned char *) li->code, sdslen(li->code))) < 0) goto werr;
written += ret; written += ret;
} }
dictReleaseIterator(iter); dictResetIterator(&iter);
return written; return written;
werr: werr:
dictReleaseIterator(iter); dictResetIterator(&iter);
return -1; return -1;
} }

View File

@ -708,7 +708,7 @@ int helpEntryCompare(const void *entry1, const void *entry2) {
* Extends the help table with new entries for the command groups. * Extends the help table with new entries for the command groups.
*/ */
void cliInitGroupHelpEntries(dict *groups) { void cliInitGroupHelpEntries(dict *groups) {
dictIterator *iter = dictGetIterator(groups); dictIterator iter;
dictEntry *entry; dictEntry *entry;
helpEntry tmp; helpEntry tmp;
@ -717,7 +717,8 @@ void cliInitGroupHelpEntries(dict *groups) {
helpEntriesLen += numGroups; helpEntriesLen += numGroups;
helpEntries = zrealloc(helpEntries, sizeof(helpEntry)*helpEntriesLen); helpEntries = zrealloc(helpEntries, sizeof(helpEntry)*helpEntriesLen);
for (entry = dictNext(iter); entry != NULL; entry = dictNext(iter)) { dictInitIterator(&iter, groups);
for (entry = dictNext(&iter); entry != NULL; entry = dictNext(&iter)) {
tmp.argc = 1; tmp.argc = 1;
tmp.argv = zmalloc(sizeof(sds)); tmp.argv = zmalloc(sizeof(sds));
tmp.argv[0] = sdscatprintf(sdsempty(),"@%s",(char *)dictGetKey(entry)); tmp.argv[0] = sdscatprintf(sdsempty(),"@%s",(char *)dictGetKey(entry));
@ -732,7 +733,7 @@ void cliInitGroupHelpEntries(dict *groups) {
tmp.docs.group = NULL; tmp.docs.group = NULL;
helpEntries[pos++] = tmp; helpEntries[pos++] = tmp;
} }
dictReleaseIterator(iter); dictResetIterator(&iter);
} }
/* Initializes help entries for all commands in the COMMAND DOCS reply. */ /* Initializes help entries for all commands in the COMMAND DOCS reply. */
@ -4403,9 +4404,11 @@ static int clusterManagerGetAntiAffinityScore(clusterManagerNodeArray *ipnodes,
} }
/* Now it's trivial to check, for each related group having the /* Now it's trivial to check, for each related group having the
* same host, what is their local score. */ * same host, what is their local score. */
dictIterator *iter = dictGetIterator(related); dictIterator iter;
dictEntry *entry; dictEntry *entry;
while ((entry = dictNext(iter)) != NULL) {
dictInitIterator(&iter, related);
while ((entry = dictNext(&iter)) != NULL) {
sds types = (sds) dictGetVal(entry); sds types = (sds) dictGetVal(entry);
sds name = (sds) dictGetKey(entry); sds name = (sds) dictGetKey(entry);
int typeslen = sdslen(types); int typeslen = sdslen(types);
@ -4428,7 +4431,7 @@ static int clusterManagerGetAntiAffinityScore(clusterManagerNodeArray *ipnodes,
} }
} }
//if (offending_len != NULL) *offending_len = offending_p - *offending; //if (offending_len != NULL) *offending_len = offending_p - *offending;
dictReleaseIterator(iter); dictResetIterator(&iter);
dictRelease(related); dictRelease(related);
} }
return score; return score;
@ -5405,14 +5408,14 @@ static void clusterManagerWaitForClusterJoin(void) {
sleep(1); sleep(1);
if (++counter > check_after) { if (++counter > check_after) {
dict *status = clusterManagerGetLinkStatus(); dict *status = clusterManagerGetLinkStatus();
dictIterator *iter = NULL;
if (status != NULL && dictSize(status) > 0) { if (status != NULL && dictSize(status) > 0) {
printf("\n"); printf("\n");
clusterManagerLogErr("Warning: %d node(s) may " clusterManagerLogErr("Warning: %d node(s) may "
"be unreachable\n", dictSize(status)); "be unreachable\n", dictSize(status));
iter = dictGetIterator(status); dictIterator iter;
dictEntry *entry; dictEntry *entry;
while ((entry = dictNext(iter)) != NULL) { dictInitIterator(&iter, status);
while ((entry = dictNext(&iter)) != NULL) {
sds nodeaddr = (sds) dictGetKey(entry); sds nodeaddr = (sds) dictGetKey(entry);
char *node_ip = NULL; char *node_ip = NULL;
int node_port = 0, node_bus_port = 0; int node_port = 0, node_bus_port = 0;
@ -5440,8 +5443,8 @@ static void clusterManagerWaitForClusterJoin(void) {
"from standard instance ports.\n"); "from standard instance ports.\n");
listEmpty(from); listEmpty(from);
} }
dictResetIterator(&iter);
} }
if (iter != NULL) dictReleaseIterator(iter);
if (status != NULL) dictRelease(status); if (status != NULL) dictRelease(status);
counter = 0; counter = 0;
} }
@ -6116,9 +6119,11 @@ static int clusterManagerFixSlotsCoverage(char *all_slots) {
none = listCreate(); none = listCreate();
single = listCreate(); single = listCreate();
multi = listCreate(); multi = listCreate();
dictIterator *iter = dictGetIterator(clusterManagerUncoveredSlots); dictIterator iter;
dictEntry *entry; dictEntry *entry;
while ((entry = dictNext(iter)) != NULL) {
dictInitIterator(&iter, clusterManagerUncoveredSlots);
while ((entry = dictNext(&iter)) != NULL) {
sds slot = (sds) dictGetKey(entry); sds slot = (sds) dictGetKey(entry);
list *nodes = (list *) dictGetVal(entry); list *nodes = (list *) dictGetVal(entry);
switch (listLength(nodes)){ switch (listLength(nodes)){
@ -6127,7 +6132,7 @@ static int clusterManagerFixSlotsCoverage(char *all_slots) {
default: listAddNodeTail(multi, slot); break; default: listAddNodeTail(multi, slot); break;
} }
} }
dictReleaseIterator(iter); dictResetIterator(&iter);
/* we want explicit manual confirmation from users for all the fix cases */ /* we want explicit manual confirmation from users for all the fix cases */
int ignore_force = 1; int ignore_force = 1;
@ -6699,28 +6704,30 @@ static int clusterManagerCheckCluster(int quiet) {
} }
if (open_slots != NULL) { if (open_slots != NULL) {
result = 0; result = 0;
dictIterator *iter = dictGetIterator(open_slots); dictIterator iter;
dictEntry *entry; dictEntry *entry;
sds errstr = sdsnew("[WARNING] The following slots are open: "); sds errstr = sdsnew("[WARNING] The following slots are open: ");
i = 0; i = 0;
while ((entry = dictNext(iter)) != NULL) {
dictInitIterator(&iter, open_slots);
while ((entry = dictNext(&iter)) != NULL) {
sds slot = (sds) dictGetKey(entry); sds slot = (sds) dictGetKey(entry);
char *fmt = (i++ > 0 ? ",%S" : "%S"); char *fmt = (i++ > 0 ? ",%S" : "%S");
errstr = sdscatfmt(errstr, fmt, slot); errstr = sdscatfmt(errstr, fmt, slot);
} }
dictResetIterator(&iter);
clusterManagerLogErr("%s.\n", (char *) errstr); clusterManagerLogErr("%s.\n", (char *) errstr);
sdsfree(errstr); sdsfree(errstr);
if (do_fix) { if (do_fix) {
/* Fix open slots. */ /* Fix open slots. */
dictReleaseIterator(iter); dictInitIterator(&iter, open_slots);
iter = dictGetIterator(open_slots); while ((entry = dictNext(&iter)) != NULL) {
while ((entry = dictNext(iter)) != NULL) {
sds slot = (sds) dictGetKey(entry); sds slot = (sds) dictGetKey(entry);
result = clusterManagerFixOpenSlot(atoi(slot)); result = clusterManagerFixOpenSlot(atoi(slot));
if (!result) break; if (!result) break;
} }
dictResetIterator(&iter);
} }
dictReleaseIterator(iter);
dictRelease(open_slots); dictRelease(open_slots);
} }
clusterManagerLogInfo(">>> Check slots coverage...\n"); clusterManagerLogInfo(">>> Check slots coverage...\n");
@ -9229,7 +9236,7 @@ static void findBigKeys(int memkeys, long long memkeys_samples) {
unsigned long long sampled = 0, total_keys, totlen=0, *sizes=NULL, it=0, scan_loops = 0; unsigned long long sampled = 0, total_keys, totlen=0, *sizes=NULL, it=0, scan_loops = 0;
redisReply *reply, *keys; redisReply *reply, *keys;
unsigned int arrsize=0, i; unsigned int arrsize=0, i;
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
typeinfo **types = NULL; typeinfo **types = NULL;
double pct; double pct;
@ -9329,8 +9336,8 @@ static void findBigKeys(int memkeys, long long memkeys_samples) {
line_count = displayKeyStatsProgressbar(sampled, total_keys); line_count = displayKeyStatsProgressbar(sampled, total_keys);
line_count += cleanPrintfln(""); line_count += cleanPrintfln("");
di = dictGetIterator(types_dict); dictInitIterator(&di, types_dict);
while ((de = dictNext(di))) { while ((de = dictNext(&di))) {
typeinfo *current_type = dictGetVal(de); typeinfo *current_type = dictGetVal(de);
if (current_type->biggest > 0) { if (current_type->biggest > 0) {
line_count += cleanPrintfln("Biggest %-9s found so far %s with %llu %s", line_count += cleanPrintfln("Biggest %-9s found so far %s with %llu %s",
@ -9338,7 +9345,7 @@ static void findBigKeys(int memkeys, long long memkeys_samples) {
!memkeys? current_type->sizeunit: "bytes"); !memkeys? current_type->sizeunit: "bytes");
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
printf("\033[%dA\r", line_count); printf("\033[%dA\r", line_count);
} }
@ -9358,10 +9365,10 @@ static void findBigKeys(int memkeys, long long memkeys_samples) {
/* Clean the types info shown during the progress bar */ /* Clean the types info shown during the progress bar */
int line_count = 0; int line_count = 0;
di = dictGetIterator(types_dict); dictInitIterator(&di, types_dict);
while ((de = dictNext(di))) while ((de = dictNext(&di)))
line_count += cleanPrintfln(""); line_count += cleanPrintfln("");
dictReleaseIterator(di); dictResetIterator(&di);
printf("\033[%dA\r", line_count); printf("\033[%dA\r", line_count);
} }
@ -9381,27 +9388,27 @@ static void findBigKeys(int memkeys, long long memkeys_samples) {
totlen, totlen ? (double)totlen/sampled : 0); totlen, totlen ? (double)totlen/sampled : 0);
/* Output the biggest keys we found, for types we did find */ /* Output the biggest keys we found, for types we did find */
di = dictGetIterator(types_dict); dictInitIterator(&di, types_dict);
while ((de = dictNext(di))) { while ((de = dictNext(&di))) {
typeinfo *type = dictGetVal(de); typeinfo *type = dictGetVal(de);
if(type->biggest_key) { if(type->biggest_key) {
printf("Biggest %6s found %s has %llu %s\n", type->name, type->biggest_key, printf("Biggest %6s found %s has %llu %s\n", type->name, type->biggest_key,
type->biggest, !memkeys? type->sizeunit: "bytes"); type->biggest, !memkeys? type->sizeunit: "bytes");
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
printf("\n"); printf("\n");
di = dictGetIterator(types_dict); dictInitIterator(&di, types_dict);
while ((de = dictNext(di))) { while ((de = dictNext(&di))) {
typeinfo *type = dictGetVal(de); typeinfo *type = dictGetVal(de);
printf("%llu %ss with %llu %s (%05.2f%% of keys, avg size %.2f)\n", printf("%llu %ss with %llu %s (%05.2f%% of keys, avg size %.2f)\n",
type->count, type->name, type->totalsize, !memkeys? type->sizeunit: "bytes", type->count, type->name, type->totalsize, !memkeys? type->sizeunit: "bytes",
sampled ? 100 * (double)type->count/sampled : 0, sampled ? 100 * (double)type->count/sampled : 0,
type->count ? (double)type->totalsize/type->count : 0); type->count ? (double)type->totalsize/type->count : 0);
} }
dictReleaseIterator(di); dictResetIterator(&di);
dictRelease(types_dict); dictRelease(types_dict);
@ -10139,14 +10146,14 @@ static int displayKeyStatsProgressbar(unsigned long long sampled,
} }
static int displayKeyStatsSizeType(dict *memkeys_types_dict) { static int displayKeyStatsSizeType(dict *memkeys_types_dict) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
int line_count = 0; int line_count = 0;
char buf[256]; char buf[256];
line_count += cleanPrintfln("--- Top size per type ---"); line_count += cleanPrintfln("--- Top size per type ---");
di = dictGetIterator(memkeys_types_dict); dictInitIterator(&di, memkeys_types_dict);
while ((de = dictNext(di))) { while ((de = dictNext(&di))) {
typeinfo *type = dictGetVal(de); typeinfo *type = dictGetVal(de);
if (type->biggest_key) { if (type->biggest_key) {
line_count += cleanPrintfln("%-10s %s is %s", line_count += cleanPrintfln("%-10s %s is %s",
@ -10154,20 +10161,20 @@ static int displayKeyStatsSizeType(dict *memkeys_types_dict) {
bytesToHuman(buf, sizeof(buf),type->biggest)); bytesToHuman(buf, sizeof(buf),type->biggest));
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
return line_count; return line_count;
} }
static int displayKeyStatsLengthType(dict *bigkeys_types_dict) { static int displayKeyStatsLengthType(dict *bigkeys_types_dict) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
int line_count = 0; int line_count = 0;
char buf[256]; char buf[256];
line_count += cleanPrintfln("--- Top length and cardinality per type ---"); line_count += cleanPrintfln("--- Top length and cardinality per type ---");
di = dictGetIterator(bigkeys_types_dict); dictInitIterator(&di, bigkeys_types_dict);
while ((de = dictNext(di))) { while ((de = dictNext(&di))) {
typeinfo *type = dictGetVal(de); typeinfo *type = dictGetVal(de);
if (type->biggest_key) { if (type->biggest_key) {
if (!strcmp(type->sizeunit, "bytes")) { if (!strcmp(type->sizeunit, "bytes")) {
@ -10178,7 +10185,7 @@ static int displayKeyStatsLengthType(dict *bigkeys_types_dict) {
line_count += cleanPrintfln("%-10s %s has %s", type->name, type->biggest_key, buf); line_count += cleanPrintfln("%-10s %s has %s", type->name, type->biggest_key, buf);
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
return line_count; return line_count;
} }
@ -10233,7 +10240,7 @@ static int displayKeyStatsType(unsigned long long sampled,
dict *memkeys_types_dict, dict *memkeys_types_dict,
dict *bigkeys_types_dict) dict *bigkeys_types_dict)
{ {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
int line_count = 0; int line_count = 0;
char total_size[64], size_avg[64], total_length[64], length_avg[64]; char total_size[64], size_avg[64], total_length[64], length_avg[64];
@ -10241,8 +10248,8 @@ static int displayKeyStatsType(unsigned long long sampled,
line_count += cleanPrintfln("Type Total keys Keys %% Tot size Avg size Total length/card Avg ln/card"); line_count += cleanPrintfln("Type Total keys Keys %% Tot size Avg size Total length/card Avg ln/card");
line_count += cleanPrintfln("--------- ------------ ------- -------- -------- ------------------ -----------"); line_count += cleanPrintfln("--------- ------------ ------- -------- -------- ------------------ -----------");
di = dictGetIterator(memkeys_types_dict); dictInitIterator(&di, memkeys_types_dict);
while ((de = dictNext(di))) { while ((de = dictNext(&di))) {
typeinfo *memkey_type = dictGetVal(de); typeinfo *memkey_type = dictGetVal(de);
if (memkey_type->count) { if (memkey_type->count) {
/* Key count, percentage, memkeys info */ /* Key count, percentage, memkeys info */
@ -10274,7 +10281,7 @@ static int displayKeyStatsType(unsigned long long sampled,
total_size, size_avg, total_length, length_avg); total_size, size_avg, total_length, length_avg);
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
return line_count; return line_count;
} }
@ -10290,14 +10297,14 @@ static int displayKeyStatsTopSizes(list *top_key_sizes, unsigned long top_sizes_
line_count += cleanPrintfln("--- Top %llu key sizes ---", top_sizes_limit); line_count += cleanPrintfln("--- Top %llu key sizes ---", top_sizes_limit);
char buffer[32]; char buffer[32];
listIter *iter = listGetIterator(top_key_sizes, AL_START_HEAD); listIter iter;
listNode *node; listNode *node;
while ((node = listNext(iter)) != NULL) { listRewind(top_key_sizes, &iter);
while ((node = listNext(&iter)) != NULL) {
key_info *key = (key_info*) listNodeValue(node); key_info *key = (key_info*) listNodeValue(node);
line_count += cleanPrintfln("%3d %8s %-10s %s", ++i, bytesToHuman(buffer, sizeof(buffer), key->size), line_count += cleanPrintfln("%3d %8s %-10s %s", ++i, bytesToHuman(buffer, sizeof(buffer), key->size),
key->type_name, key->key_name); key->type_name, key->key_name);
} }
listReleaseIterator(iter);
return line_count; return line_count;
} }
@ -10322,7 +10329,7 @@ static int updateTopSizes(char *key_name, size_t key_name_len, unsigned long lon
char *type_name, list *topkeys, unsigned long top_sizes_limit) char *type_name, list *topkeys, unsigned long top_sizes_limit)
{ {
listNode *node; listNode *node;
listIter *iter; listIter iter;
key_info *new_node; key_info *new_node;
/* Check if we do not need to add to the list */ /* Check if we do not need to add to the list */
@ -10333,11 +10340,10 @@ static int updateTopSizes(char *key_name, size_t key_name_len, unsigned long lon
} }
/* Find where to insert the new key size */ /* Find where to insert the new key size */
iter = listGetIterator(topkeys, AL_START_HEAD); listRewind(topkeys, &iter);
do { do {
node = listNext(iter); node = listNext(&iter);
} while (node != NULL && key_size <= ((key_info*)node->value)->size); } while (node != NULL && key_size <= ((key_info*)node->value)->size);
listReleaseIterator(iter);
new_node = createKeySizeInfo(key_name, key_name_len, type_name, key_size); new_node = createKeySizeInfo(key_name, key_name_len, type_name, key_size);
if (node) { if (node) {
@ -10569,13 +10575,13 @@ static void keyStats(long long memkeys_samples, unsigned long long cursor, unsig
hdr_close(keysize_histogram); hdr_close(keysize_histogram);
/* sdsfree before listRelease */ /* sdsfree before listRelease */
listIter *iter = listGetIterator(top_sizes, AL_START_HEAD); listIter iter;
listNode *node; listNode *node;
while ((node = listNext(iter)) != NULL) { listRewind(top_sizes, &iter);
while ((node = listNext(&iter)) != NULL) {
key_info *key = (key_info*) listNodeValue(node); key_info *key = (key_info*) listNodeValue(node);
sdsfree(key->key_name); sdsfree(key->key_name);
} }
listReleaseIterator(iter);
listRelease(top_sizes); /* list->free is set */ listRelease(top_sizes); /* list->free is set */
exit(0); exit(0);

View File

@ -710,15 +710,15 @@ void sentinelEvent(int level, char *type, sentinelRedisInstance *ri,
* generated when a master to monitor is added at runtime via the * generated when a master to monitor is added at runtime via the
* SENTINEL MONITOR command. */ * SENTINEL MONITOR command. */
void sentinelGenerateInitialMonitorEvents(void) { void sentinelGenerateInitialMonitorEvents(void) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
di = dictGetIterator(sentinel.masters); dictInitIterator(&di, sentinel.masters);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
sentinelRedisInstance *ri = dictGetVal(de); sentinelRedisInstance *ri = dictGetVal(de);
sentinelEvent(LL_WARNING,"+monitor",ri,"%@ quorum %d",ri->quorum); sentinelEvent(LL_WARNING,"+monitor",ri,"%@ quorum %d",ri->quorum);
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* ============================ script execution ============================ */ /* ============================ script execution ============================ */
@ -1088,14 +1088,14 @@ instanceLink *releaseInstanceLink(instanceLink *link, sentinelRedisInstance *ri)
* is returned. */ * is returned. */
int sentinelTryConnectionSharing(sentinelRedisInstance *ri) { int sentinelTryConnectionSharing(sentinelRedisInstance *ri) {
serverAssert(ri->flags & SRI_SENTINEL); serverAssert(ri->flags & SRI_SENTINEL);
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
if (ri->runid == NULL) return C_ERR; /* No way to identify it. */ if (ri->runid == NULL) return C_ERR; /* No way to identify it. */
if (ri->link->refcount > 1) return C_ERR; /* Already shared. */ if (ri->link->refcount > 1) return C_ERR; /* Already shared. */
di = dictGetIterator(sentinel.masters); dictInitIterator(&di, sentinel.masters);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
sentinelRedisInstance *master = dictGetVal(de), *match; sentinelRedisInstance *master = dictGetVal(de), *match;
/* We want to share with the same physical Sentinel referenced /* We want to share with the same physical Sentinel referenced
* in other masters, so skip our master. */ * in other masters, so skip our master. */
@ -1110,10 +1110,10 @@ int sentinelTryConnectionSharing(sentinelRedisInstance *ri) {
releaseInstanceLink(ri->link,NULL); releaseInstanceLink(ri->link,NULL);
ri->link = match->link; ri->link = match->link;
match->link->refcount++; match->link->refcount++;
dictReleaseIterator(di); dictResetIterator(&di);
return C_OK; return C_OK;
} }
dictReleaseIterator(di); dictResetIterator(&di);
return C_ERR; return C_ERR;
} }
@ -1126,33 +1126,34 @@ void dropInstanceConnections(sentinelRedisInstance *ri) {
instanceLinkCloseConnection(ri->link, ri->link->pc); instanceLinkCloseConnection(ri->link, ri->link->pc);
/* Disconnect with all replicas. */ /* Disconnect with all replicas. */
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
sentinelRedisInstance *repl_ri; sentinelRedisInstance *repl_ri;
di = dictGetIterator(ri->slaves);
while ((de = dictNext(di)) != NULL) { dictInitIterator(&di, ri->slaves);
while ((de = dictNext(&di)) != NULL) {
repl_ri = dictGetVal(de); repl_ri = dictGetVal(de);
instanceLinkCloseConnection(repl_ri->link, repl_ri->link->cc); instanceLinkCloseConnection(repl_ri->link, repl_ri->link->cc);
instanceLinkCloseConnection(repl_ri->link, repl_ri->link->pc); instanceLinkCloseConnection(repl_ri->link, repl_ri->link->pc);
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* Drop all connections to other sentinels. Returns the number of connections /* Drop all connections to other sentinels. Returns the number of connections
* dropped.*/ * dropped.*/
int sentinelDropConnections(void) { int sentinelDropConnections(void) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
int dropped = 0; int dropped = 0;
di = dictGetIterator(sentinel.masters); dictInitIterator(&di, sentinel.masters);
while ((de = dictNext(di)) != NULL) { while ((de = dictNext(&di)) != NULL) {
dictIterator *sdi; dictIterator sdi;
dictEntry *sde; dictEntry *sde;
sentinelRedisInstance *ri = dictGetVal(de); sentinelRedisInstance *ri = dictGetVal(de);
sdi = dictGetIterator(ri->sentinels); dictInitIterator(&sdi, ri->sentinels);
while ((sde = dictNext(sdi)) != NULL) { while ((sde = dictNext(&sdi)) != NULL) {
sentinelRedisInstance *si = dictGetVal(sde); sentinelRedisInstance *si = dictGetVal(sde);
if (!si->link->disconnected) { if (!si->link->disconnected) {
instanceLinkCloseConnection(si->link, si->link->pc); instanceLinkCloseConnection(si->link, si->link->pc);
@ -1160,9 +1161,9 @@ int sentinelDropConnections(void) {
dropped++; dropped++;
} }
} }
dictReleaseIterator(sdi); dictResetIterator(&sdi);
} }
dictReleaseIterator(di); dictResetIterator(&di);
return dropped; return dropped;
} }
@ -1175,12 +1176,12 @@ int sentinelDropConnections(void) {
* Return the number of updated Sentinel addresses. */ * Return the number of updated Sentinel addresses. */
int sentinelUpdateSentinelAddressInAllMasters(sentinelRedisInstance *ri) { int sentinelUpdateSentinelAddressInAllMasters(sentinelRedisInstance *ri) {
serverAssert(ri->flags & SRI_SENTINEL); serverAssert(ri->flags & SRI_SENTINEL);
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
int reconfigured = 0; int reconfigured = 0;
di = dictGetIterator(sentinel.masters); dictInitIterator(&di, sentinel.masters);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
sentinelRedisInstance *master = dictGetVal(de), *match; sentinelRedisInstance *master = dictGetVal(de), *match;
match = getSentinelRedisInstanceByAddrAndRunID(master->sentinels, match = getSentinelRedisInstanceByAddrAndRunID(master->sentinels,
NULL,0,ri->runid); NULL,0,ri->runid);
@ -1202,7 +1203,7 @@ int sentinelUpdateSentinelAddressInAllMasters(sentinelRedisInstance *ri) {
match->addr = dupSentinelAddr(ri->addr); match->addr = dupSentinelAddr(ri->addr);
reconfigured++; reconfigured++;
} }
dictReleaseIterator(di); dictResetIterator(&di);
if (reconfigured) if (reconfigured)
sentinelEvent(LL_NOTICE,"+sentinel-address-update", ri, sentinelEvent(LL_NOTICE,"+sentinel-address-update", ri,
"%@ %d additional matching instances", reconfigured); "%@ %d additional matching instances", reconfigured);
@ -1432,14 +1433,14 @@ const char *sentinelRedisInstanceTypeStr(sentinelRedisInstance *ri) {
* The function returns 1 if the matching Sentinel was removed, otherwise * The function returns 1 if the matching Sentinel was removed, otherwise
* 0 if there was no Sentinel with this ID. */ * 0 if there was no Sentinel with this ID. */
int removeMatchingSentinelFromMaster(sentinelRedisInstance *master, char *runid) { int removeMatchingSentinelFromMaster(sentinelRedisInstance *master, char *runid) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
int removed = 0; int removed = 0;
if (runid == NULL) return 0; if (runid == NULL) return 0;
di = dictGetSafeIterator(master->sentinels); dictInitSafeIterator(&di, master->sentinels);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
sentinelRedisInstance *ri = dictGetVal(de); sentinelRedisInstance *ri = dictGetVal(de);
if (ri->runid && strcmp(ri->runid,runid) == 0) { if (ri->runid && strcmp(ri->runid,runid) == 0) {
@ -1447,7 +1448,7 @@ int removeMatchingSentinelFromMaster(sentinelRedisInstance *master, char *runid)
removed++; removed++;
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
return removed; return removed;
} }
@ -1458,7 +1459,7 @@ int removeMatchingSentinelFromMaster(sentinelRedisInstance *master, char *runid)
* runid or addr can be NULL. In such a case the search is performed only * runid or addr can be NULL. In such a case the search is performed only
* by the non-NULL field. */ * by the non-NULL field. */
sentinelRedisInstance *getSentinelRedisInstanceByAddrAndRunID(dict *instances, char *addr, int port, char *runid) { sentinelRedisInstance *getSentinelRedisInstanceByAddrAndRunID(dict *instances, char *addr, int port, char *runid) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
sentinelRedisInstance *instance = NULL; sentinelRedisInstance *instance = NULL;
sentinelAddr *ri_addr = NULL; sentinelAddr *ri_addr = NULL;
@ -1471,8 +1472,8 @@ sentinelRedisInstance *getSentinelRedisInstanceByAddrAndRunID(dict *instances, c
ri_addr = createSentinelAddr(addr,port,1); ri_addr = createSentinelAddr(addr,port,1);
if (!ri_addr) return NULL; if (!ri_addr) return NULL;
} }
di = dictGetIterator(instances); dictInitIterator(&di, instances);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
sentinelRedisInstance *ri = dictGetVal(de); sentinelRedisInstance *ri = dictGetVal(de);
if (runid && !ri->runid) continue; if (runid && !ri->runid) continue;
@ -1483,7 +1484,7 @@ sentinelRedisInstance *getSentinelRedisInstanceByAddrAndRunID(dict *instances, c
break; break;
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
if (ri_addr != NULL) if (ri_addr != NULL)
releaseSentinelAddr(ri_addr); releaseSentinelAddr(ri_addr);
@ -1548,12 +1549,12 @@ void sentinelResetMaster(sentinelRedisInstance *ri, int flags) {
/* Call sentinelResetMaster() on every master with a name matching the specified /* Call sentinelResetMaster() on every master with a name matching the specified
* pattern. */ * pattern. */
int sentinelResetMastersByPattern(char *pattern, int flags) { int sentinelResetMastersByPattern(char *pattern, int flags) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
int reset = 0; int reset = 0;
di = dictGetIterator(sentinel.masters); dictInitIterator(&di, sentinel.masters);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
sentinelRedisInstance *ri = dictGetVal(de); sentinelRedisInstance *ri = dictGetVal(de);
if (ri->name) { if (ri->name) {
@ -1563,7 +1564,7 @@ int sentinelResetMastersByPattern(char *pattern, int flags) {
} }
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
return reset; return reset;
} }
@ -1578,7 +1579,7 @@ int sentinelResetMasterAndChangeAddress(sentinelRedisInstance *master, char *hos
sentinelAddr *oldaddr, *newaddr; sentinelAddr *oldaddr, *newaddr;
sentinelAddr **slaves = NULL; sentinelAddr **slaves = NULL;
int numslaves = 0, j; int numslaves = 0, j;
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
newaddr = createSentinelAddr(hostname,port,0); newaddr = createSentinelAddr(hostname,port,0);
@ -1590,14 +1591,14 @@ int sentinelResetMasterAndChangeAddress(sentinelRedisInstance *master, char *hos
slaves = zmalloc(sizeof(sentinelAddr*)*(dictSize(master->slaves) + 1)); slaves = zmalloc(sizeof(sentinelAddr*)*(dictSize(master->slaves) + 1));
/* Don't include the one having the address we are switching to. */ /* Don't include the one having the address we are switching to. */
di = dictGetIterator(master->slaves); dictInitIterator(&di, master->slaves);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
sentinelRedisInstance *slave = dictGetVal(de); sentinelRedisInstance *slave = dictGetVal(de);
if (sentinelAddrOrHostnameEqual(slave->addr,newaddr)) continue; if (sentinelAddrOrHostnameEqual(slave->addr,newaddr)) continue;
slaves[numslaves++] = dupSentinelAddr(slave->addr); slaves[numslaves++] = dupSentinelAddr(slave->addr);
} }
dictReleaseIterator(di); dictResetIterator(&di);
/* If we are switching to a different address, include the old address /* If we are switching to a different address, include the old address
* as a slave as well, so that we'll be able to sense / reconfigure * as a slave as well, so that we'll be able to sense / reconfigure
@ -1663,18 +1664,18 @@ sentinelAddr *sentinelGetCurrentMasterAddress(sentinelRedisInstance *master) {
/* This function sets the down_after_period field value in 'master' to all /* This function sets the down_after_period field value in 'master' to all
* the slaves and sentinel instances connected to this master. */ * the slaves and sentinel instances connected to this master. */
void sentinelPropagateDownAfterPeriod(sentinelRedisInstance *master) { void sentinelPropagateDownAfterPeriod(sentinelRedisInstance *master) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
int j; int j;
dict *d[] = {master->slaves, master->sentinels, NULL}; dict *d[] = {master->slaves, master->sentinels, NULL};
for (j = 0; d[j]; j++) { for (j = 0; d[j]; j++) {
di = dictGetIterator(d[j]); dictInitIterator(&di, d[j]);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
sentinelRedisInstance *ri = dictGetVal(de); sentinelRedisInstance *ri = dictGetVal(de);
ri->down_after_period = master->down_after_period; ri->down_after_period = master->down_after_period;
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
} }
@ -2009,7 +2010,7 @@ const char *sentinelHandleConfiguration(char **argv, int argc) {
* Sentinel across restarts: config epoch of masters, associated slaves * Sentinel across restarts: config epoch of masters, associated slaves
* and sentinel instances, and so forth. */ * and sentinel instances, and so forth. */
void rewriteConfigSentinelOption(struct rewriteConfigState *state) { void rewriteConfigSentinelOption(struct rewriteConfigState *state) {
dictIterator *di, *di2; dictIterator di, di2;
dictEntry *de; dictEntry *de;
sds line; sds line;
@ -2039,8 +2040,8 @@ void rewriteConfigSentinelOption(struct rewriteConfigState *state) {
sentinel.announce_hostnames != SENTINEL_DEFAULT_ANNOUNCE_HOSTNAMES); sentinel.announce_hostnames != SENTINEL_DEFAULT_ANNOUNCE_HOSTNAMES);
/* For every master emit a "sentinel monitor" config entry. */ /* For every master emit a "sentinel monitor" config entry. */
di = dictGetIterator(sentinel.masters); dictInitIterator(&di, sentinel.masters);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
sentinelRedisInstance *master, *ri; sentinelRedisInstance *master, *ri;
sentinelAddr *master_addr; sentinelAddr *master_addr;
@ -2141,8 +2142,8 @@ void rewriteConfigSentinelOption(struct rewriteConfigState *state) {
/* rewriteConfigMarkAsProcessed is handled after the loop */ /* rewriteConfigMarkAsProcessed is handled after the loop */
/* sentinel known-slave */ /* sentinel known-slave */
di2 = dictGetIterator(master->slaves); dictInitIterator(&di2, master->slaves);
while((de = dictNext(di2)) != NULL) { while((de = dictNext(&di2)) != NULL) {
sentinelAddr *slave_addr; sentinelAddr *slave_addr;
ri = dictGetVal(de); ri = dictGetVal(de);
@ -2166,11 +2167,11 @@ void rewriteConfigSentinelOption(struct rewriteConfigState *state) {
} }
/* rewriteConfigMarkAsProcessed is handled after the loop */ /* rewriteConfigMarkAsProcessed is handled after the loop */
} }
dictReleaseIterator(di2); dictResetIterator(&di2);
/* sentinel known-sentinel */ /* sentinel known-sentinel */
di2 = dictGetIterator(master->sentinels); dictInitIterator(&di2, master->sentinels);
while((de = dictNext(di2)) != NULL) { while((de = dictNext(&di2)) != NULL) {
ri = dictGetVal(de); ri = dictGetVal(de);
if (ri->runid == NULL) continue; if (ri->runid == NULL) continue;
line = sdscatprintf(sdsempty(), line = sdscatprintf(sdsempty(),
@ -2179,11 +2180,11 @@ void rewriteConfigSentinelOption(struct rewriteConfigState *state) {
rewriteConfigRewriteLine(state,"sentinel known-sentinel",line,1); rewriteConfigRewriteLine(state,"sentinel known-sentinel",line,1);
/* rewriteConfigMarkAsProcessed is handled after the loop */ /* rewriteConfigMarkAsProcessed is handled after the loop */
} }
dictReleaseIterator(di2); dictResetIterator(&di2);
/* sentinel rename-command */ /* sentinel rename-command */
di2 = dictGetIterator(master->renamed_commands); dictInitIterator(&di2, master->renamed_commands);
while((de = dictNext(di2)) != NULL) { while((de = dictNext(&di2)) != NULL) {
sds oldname = dictGetKey(de); sds oldname = dictGetKey(de);
sds newname = dictGetVal(de); sds newname = dictGetVal(de);
line = sdscatprintf(sdsempty(), line = sdscatprintf(sdsempty(),
@ -2192,7 +2193,7 @@ void rewriteConfigSentinelOption(struct rewriteConfigState *state) {
rewriteConfigRewriteLine(state,"sentinel rename-command",line,1); rewriteConfigRewriteLine(state,"sentinel rename-command",line,1);
/* rewriteConfigMarkAsProcessed is handled after the loop */ /* rewriteConfigMarkAsProcessed is handled after the loop */
} }
dictReleaseIterator(di2); dictResetIterator(&di2);
} }
/* sentinel current-epoch is a global state valid for all the masters. */ /* sentinel current-epoch is a global state valid for all the masters. */
@ -2234,7 +2235,7 @@ void rewriteConfigSentinelOption(struct rewriteConfigState *state) {
rewriteConfigMarkAsProcessed(state,"sentinel sentinel-pass"); rewriteConfigMarkAsProcessed(state,"sentinel sentinel-pass");
} }
dictReleaseIterator(di); dictResetIterator(&di);
/* NOTE: the purpose here is in case due to the state change, the config rewrite /* NOTE: the purpose here is in case due to the state change, the config rewrite
does not handle the configs, however, previously the config was set in the config file, does not handle the configs, however, previously the config was set in the config file,
@ -2880,18 +2881,18 @@ void sentinelProcessHelloMessage(char *hello, int hello_len) {
/* If there is already other sentinel with same address (but /* If there is already other sentinel with same address (but
* different runid) then remove the old one across all masters */ * different runid) then remove the old one across all masters */
sentinelEvent(LL_NOTICE,"+sentinel-invalid-addr",other,"%@"); sentinelEvent(LL_NOTICE,"+sentinel-invalid-addr",other,"%@");
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
/* Keep a copy of runid. 'other' about to be deleted in loop. */ /* Keep a copy of runid. 'other' about to be deleted in loop. */
sds runid_obsolete = sdsnew(other->runid); sds runid_obsolete = sdsnew(other->runid);
di = dictGetIterator(sentinel.masters); dictInitIterator(&di, sentinel.masters);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
sentinelRedisInstance *master = dictGetVal(de); sentinelRedisInstance *master = dictGetVal(de);
removeMatchingSentinelFromMaster(master, runid_obsolete); removeMatchingSentinelFromMaster(master, runid_obsolete);
} }
dictReleaseIterator(di); dictResetIterator(&di);
sdsfree(runid_obsolete); sdsfree(runid_obsolete);
} }
} }
@ -3041,16 +3042,16 @@ int sentinelSendHello(sentinelRedisInstance *ri) {
/* Reset last_pub_time in all the instances in the specified dictionary /* Reset last_pub_time in all the instances in the specified dictionary
* in order to force the delivery of a Hello update ASAP. */ * in order to force the delivery of a Hello update ASAP. */
void sentinelForceHelloUpdateDictOfRedisInstances(dict *instances) { void sentinelForceHelloUpdateDictOfRedisInstances(dict *instances) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
di = dictGetSafeIterator(instances); dictInitSafeIterator(&di, instances);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
sentinelRedisInstance *ri = dictGetVal(de); sentinelRedisInstance *ri = dictGetVal(de);
if (ri->last_pub_time >= (sentinel_publish_period+1)) if (ri->last_pub_time >= (sentinel_publish_period+1))
ri->last_pub_time -= (sentinel_publish_period+1); ri->last_pub_time -= (sentinel_publish_period+1);
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* This function forces the delivery of a "Hello" message (see /* This function forces the delivery of a "Hello" message (see
@ -3785,13 +3786,13 @@ void addReplySentinelDebugInfo(client *c) {
/* Output a number of instances contained inside a dictionary as /* Output a number of instances contained inside a dictionary as
* Redis protocol. */ * Redis protocol. */
void addReplyDictOfRedisInstances(client *c, dict *instances) { void addReplyDictOfRedisInstances(client *c, dict *instances) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
long slaves = 0; long slaves = 0;
void *replylen = addReplyDeferredLen(c); void *replylen = addReplyDeferredLen(c);
di = dictGetIterator(instances); dictInitIterator(&di, instances);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
sentinelRedisInstance *ri = dictGetVal(de); sentinelRedisInstance *ri = dictGetVal(de);
/* don't announce unannounced replicas */ /* don't announce unannounced replicas */
@ -3799,7 +3800,7 @@ void addReplyDictOfRedisInstances(client *c, dict *instances) {
addReplySentinelRedisInstance(c,ri); addReplySentinelRedisInstance(c,ri);
slaves++; slaves++;
} }
dictReleaseIterator(di); dictResetIterator(&di);
setDeferredArrayLen(c, replylen, slaves); setDeferredArrayLen(c, replylen, slaves);
} }
@ -3823,20 +3824,20 @@ sentinelRedisInstance *sentinelGetMasterByNameOrReplyError(client *c,
#define SENTINEL_ISQR_NOQUORUM (1<<0) #define SENTINEL_ISQR_NOQUORUM (1<<0)
#define SENTINEL_ISQR_NOAUTH (1<<1) #define SENTINEL_ISQR_NOAUTH (1<<1)
int sentinelIsQuorumReachable(sentinelRedisInstance *master, int *usableptr) { int sentinelIsQuorumReachable(sentinelRedisInstance *master, int *usableptr) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
int usable = 1; /* Number of usable Sentinels. Init to 1 to count myself. */ int usable = 1; /* Number of usable Sentinels. Init to 1 to count myself. */
int result = SENTINEL_ISQR_OK; int result = SENTINEL_ISQR_OK;
int voters = dictSize(master->sentinels)+1; /* Known Sentinels + myself. */ int voters = dictSize(master->sentinels)+1; /* Known Sentinels + myself. */
di = dictGetIterator(master->sentinels); dictInitIterator(&di, master->sentinels);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
sentinelRedisInstance *ri = dictGetVal(de); sentinelRedisInstance *ri = dictGetVal(de);
if (ri->flags & (SRI_S_DOWN|SRI_O_DOWN)) continue; if (ri->flags & (SRI_S_DOWN|SRI_O_DOWN)) continue;
usable++; usable++;
} }
dictReleaseIterator(di); dictResetIterator(&di);
if (usable < (int)master->quorum) result |= SENTINEL_ISQR_NOQUORUM; if (usable < (int)master->quorum) result |= SENTINEL_ISQR_NOQUORUM;
if (usable < voters/2+1) result |= SENTINEL_ISQR_NOAUTH; if (usable < voters/2+1) result |= SENTINEL_ISQR_NOAUTH;
@ -4147,10 +4148,11 @@ NULL
*/ */
addReplyArrayLen(c,dictSize(masters_local) * 2); addReplyArrayLen(c,dictSize(masters_local) * 2);
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
di = dictGetIterator(masters_local);
while ((de = dictNext(di)) != NULL) { dictInitIterator(&di, masters_local);
while ((de = dictNext(&di)) != NULL) {
sentinelRedisInstance *ri = dictGetVal(de); sentinelRedisInstance *ri = dictGetVal(de);
addReplyBulkCBuffer(c,ri->name,strlen(ri->name)); addReplyBulkCBuffer(c,ri->name,strlen(ri->name));
addReplyArrayLen(c,dictSize(ri->slaves) + 1); /* +1 for self */ addReplyArrayLen(c,dictSize(ri->slaves) + 1); /* +1 for self */
@ -4162,10 +4164,11 @@ NULL
else else
addReplyNull(c); addReplyNull(c);
dictIterator *sdi; dictIterator sdi;
dictEntry *sde; dictEntry *sde;
sdi = dictGetIterator(ri->slaves);
while ((sde = dictNext(sdi)) != NULL) { dictInitIterator(&sdi, ri->slaves);
while ((sde = dictNext(&sdi)) != NULL) {
sentinelRedisInstance *sri = dictGetVal(sde); sentinelRedisInstance *sri = dictGetVal(sde);
addReplyArrayLen(c,2); addReplyArrayLen(c,2);
addReplyLongLong(c, addReplyLongLong(c,
@ -4175,9 +4178,9 @@ NULL
else else
addReplyNull(c); addReplyNull(c);
} }
dictReleaseIterator(sdi); dictResetIterator(&sdi);
} }
dictReleaseIterator(di); dictResetIterator(&di);
if (masters_local != sentinel.masters) dictRelease(masters_local); if (masters_local != sentinel.masters) dictRelease(masters_local);
} else if (!strcasecmp(c->argv[1]->ptr,"simulate-failure")) { } else if (!strcasecmp(c->argv[1]->ptr,"simulate-failure")) {
/* SENTINEL SIMULATE-FAILURE [CRASH-AFTER-ELECTION] [CRASH-AFTER-PROMOTION] [HELP] */ /* SENTINEL SIMULATE-FAILURE [CRASH-AFTER-ELECTION] [CRASH-AFTER-PROMOTION] [HELP] */
@ -4235,8 +4238,10 @@ void sentinelInfoCommand(client *c) {
/* Purge unsupported sections from the requested ones. */ /* Purge unsupported sections from the requested ones. */
dictEntry *de; dictEntry *de;
dictIterator *di = dictGetSafeIterator(sections_dict); dictIterator di;
while((de = dictNext(di)) != NULL) {
dictInitSafeIterator(&di, sections_dict);
while((de = dictNext(&di)) != NULL) {
int i; int i;
sds sec = dictGetKey(de); sds sec = dictGetKey(de);
for (i=0; sentinel_sections[i]; i++) for (i=0; sentinel_sections[i]; i++)
@ -4246,7 +4251,7 @@ void sentinelInfoCommand(client *c) {
if (!sentinel_sections[i]) if (!sentinel_sections[i])
dictDelete(sections_dict, sec); dictDelete(sections_dict, sec);
} }
dictReleaseIterator(di); dictResetIterator(&di);
/* Insert explicit all sections (don't pass these vars to genRedisInfoString) */ /* Insert explicit all sections (don't pass these vars to genRedisInfoString) */
if (sec_all || sec_everything) { if (sec_all || sec_everything) {
@ -4261,7 +4266,7 @@ void sentinelInfoCommand(client *c) {
sds info = genRedisInfoString(sections_dict, 0, 0); sds info = genRedisInfoString(sections_dict, 0, 0);
if (sec_all || (dictFind(sections_dict, "sentinel") != NULL)) { if (sec_all || (dictFind(sections_dict, "sentinel") != NULL)) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
int master_id = 0; int master_id = 0;
@ -4284,8 +4289,8 @@ void sentinelInfoCommand(client *c) {
listLength(sentinel.scripts_queue), listLength(sentinel.scripts_queue),
sentinel.simfailure_flags); sentinel.simfailure_flags);
di = dictGetIterator(sentinel.masters); dictInitIterator(&di, sentinel.masters);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
sentinelRedisInstance *ri = dictGetVal(de); sentinelRedisInstance *ri = dictGetVal(de);
char *status = "ok"; char *status = "ok";
@ -4299,7 +4304,7 @@ void sentinelInfoCommand(client *c) {
dictSize(ri->slaves), dictSize(ri->slaves),
dictSize(ri->sentinels)+1); dictSize(ri->sentinels)+1);
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
if (sections_dict != cached_all_info_sections) if (sections_dict != cached_all_info_sections)
releaseInfoSectionDict(sections_dict); releaseInfoSectionDict(sections_dict);
@ -4309,20 +4314,20 @@ void sentinelInfoCommand(client *c) {
/* Implements Sentinel version of the ROLE command. The output is /* Implements Sentinel version of the ROLE command. The output is
* "sentinel" and the list of currently monitored master names. */ * "sentinel" and the list of currently monitored master names. */
void sentinelRoleCommand(client *c) { void sentinelRoleCommand(client *c) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
addReplyArrayLen(c,2); addReplyArrayLen(c,2);
addReplyBulkCBuffer(c,"sentinel",8); addReplyBulkCBuffer(c,"sentinel",8);
addReplyArrayLen(c,dictSize(sentinel.masters)); addReplyArrayLen(c,dictSize(sentinel.masters));
di = dictGetIterator(sentinel.masters); dictInitIterator(&di, sentinel.masters);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
sentinelRedisInstance *ri = dictGetVal(de); sentinelRedisInstance *ri = dictGetVal(de);
addReplyBulkCString(c,ri->name); addReplyBulkCString(c,ri->name);
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* SENTINEL SET <mastername> [<option> <value> ...] */ /* SENTINEL SET <mastername> [<option> <value> ...] */
@ -4593,7 +4598,7 @@ void sentinelCheckSubjectivelyDown(sentinelRedisInstance *ri) {
* However messages can be delayed so there are no strong guarantees about * However messages can be delayed so there are no strong guarantees about
* N instances agreeing at the same time about the down state. */ * N instances agreeing at the same time about the down state. */
void sentinelCheckObjectivelyDown(sentinelRedisInstance *master) { void sentinelCheckObjectivelyDown(sentinelRedisInstance *master) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
unsigned int quorum = 0, odown = 0; unsigned int quorum = 0, odown = 0;
@ -4601,13 +4606,13 @@ void sentinelCheckObjectivelyDown(sentinelRedisInstance *master) {
/* Is down for enough sentinels? */ /* Is down for enough sentinels? */
quorum = 1; /* the current sentinel. */ quorum = 1; /* the current sentinel. */
/* Count all the other sentinels. */ /* Count all the other sentinels. */
di = dictGetIterator(master->sentinels); dictInitIterator(&di, master->sentinels);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
sentinelRedisInstance *ri = dictGetVal(de); sentinelRedisInstance *ri = dictGetVal(de);
if (ri->flags & SRI_MASTER_DOWN) quorum++; if (ri->flags & SRI_MASTER_DOWN) quorum++;
} }
dictReleaseIterator(di); dictResetIterator(&di);
if (quorum >= master->quorum) odown = 1; if (quorum >= master->quorum) odown = 1;
} }
@ -4673,11 +4678,11 @@ void sentinelReceiveIsMasterDownReply(redisAsyncContext *c, void *reply, void *p
* needed to mark the master in ODOWN state and trigger a failover. */ * needed to mark the master in ODOWN state and trigger a failover. */
#define SENTINEL_ASK_FORCED (1<<0) #define SENTINEL_ASK_FORCED (1<<0)
void sentinelAskMasterStateToOtherSentinels(sentinelRedisInstance *master, int flags) { void sentinelAskMasterStateToOtherSentinels(sentinelRedisInstance *master, int flags) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
di = dictGetIterator(master->sentinels); dictInitIterator(&di, master->sentinels);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
sentinelRedisInstance *ri = dictGetVal(de); sentinelRedisInstance *ri = dictGetVal(de);
mstime_t elapsed = mstime() - ri->last_master_down_reply_time; mstime_t elapsed = mstime() - ri->last_master_down_reply_time;
char port[32]; char port[32];
@ -4713,7 +4718,7 @@ void sentinelAskMasterStateToOtherSentinels(sentinelRedisInstance *master, int f
sentinel.myid : "*"); sentinel.myid : "*");
if (retval == C_OK) ri->link->pending_commands++; if (retval == C_OK) ri->link->pending_commands++;
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* =============================== FAILOVER ================================= */ /* =============================== FAILOVER ================================= */
@ -4788,7 +4793,7 @@ int sentinelLeaderIncr(dict *counters, char *runid) {
* reported the same instance as leader for the same epoch. */ * reported the same instance as leader for the same epoch. */
char *sentinelGetLeader(sentinelRedisInstance *master, uint64_t epoch) { char *sentinelGetLeader(sentinelRedisInstance *master, uint64_t epoch) {
dict *counters; dict *counters;
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
unsigned int voters = 0, voters_quorum; unsigned int voters = 0, voters_quorum;
char *myvote; char *myvote;
@ -4802,19 +4807,19 @@ char *sentinelGetLeader(sentinelRedisInstance *master, uint64_t epoch) {
voters = dictSize(master->sentinels)+1; /* All the other sentinels and me.*/ voters = dictSize(master->sentinels)+1; /* All the other sentinels and me.*/
/* Count other sentinels votes */ /* Count other sentinels votes */
di = dictGetIterator(master->sentinels); dictInitIterator(&di, master->sentinels);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
sentinelRedisInstance *ri = dictGetVal(de); sentinelRedisInstance *ri = dictGetVal(de);
if (ri->leader != NULL && ri->leader_epoch == sentinel.current_epoch) if (ri->leader != NULL && ri->leader_epoch == sentinel.current_epoch)
sentinelLeaderIncr(counters,ri->leader); sentinelLeaderIncr(counters,ri->leader);
} }
dictReleaseIterator(di); dictResetIterator(&di);
/* Check what's the winner. For the winner to win, it needs two conditions: /* Check what's the winner. For the winner to win, it needs two conditions:
* 1) Absolute majority between voters (50% + 1). * 1) Absolute majority between voters (50% + 1).
* 2) And anyway at least master->quorum votes. */ * 2) And anyway at least master->quorum votes. */
di = dictGetIterator(counters); dictInitIterator(&di, counters);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
uint64_t votes = dictGetUnsignedIntegerVal(de); uint64_t votes = dictGetUnsignedIntegerVal(de);
if (votes > max_votes) { if (votes > max_votes) {
@ -4822,7 +4827,7 @@ char *sentinelGetLeader(sentinelRedisInstance *master, uint64_t epoch) {
winner = dictGetKey(de); winner = dictGetKey(de);
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
/* Count this Sentinel vote: /* Count this Sentinel vote:
* if this Sentinel did not voted yet, either vote for the most * if this Sentinel did not voted yet, either vote for the most
@ -5048,7 +5053,7 @@ sentinelRedisInstance *sentinelSelectSlave(sentinelRedisInstance *master) {
zmalloc(sizeof(instance[0])*dictSize(master->slaves)); zmalloc(sizeof(instance[0])*dictSize(master->slaves));
sentinelRedisInstance *selected = NULL; sentinelRedisInstance *selected = NULL;
int instances = 0; int instances = 0;
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
mstime_t max_master_down_time = 0; mstime_t max_master_down_time = 0;
@ -5056,9 +5061,9 @@ sentinelRedisInstance *sentinelSelectSlave(sentinelRedisInstance *master) {
max_master_down_time += mstime() - master->s_down_since_time; max_master_down_time += mstime() - master->s_down_since_time;
max_master_down_time += master->down_after_period * 10; max_master_down_time += master->down_after_period * 10;
di = dictGetIterator(master->slaves); dictInitIterator(&di, master->slaves);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
sentinelRedisInstance *slave = dictGetVal(de); sentinelRedisInstance *slave = dictGetVal(de);
mstime_t info_validity_time; mstime_t info_validity_time;
@ -5078,7 +5083,7 @@ sentinelRedisInstance *sentinelSelectSlave(sentinelRedisInstance *master) {
if (slave->master_link_down_time > max_master_down_time) continue; if (slave->master_link_down_time > max_master_down_time) continue;
instance[instances++] = slave; instance[instances++] = slave;
} }
dictReleaseIterator(di); dictResetIterator(&di);
if (instances) { if (instances) {
qsort(instance,instances,sizeof(sentinelRedisInstance*), qsort(instance,instances,sizeof(sentinelRedisInstance*),
compareSlavesForPromotion); compareSlavesForPromotion);
@ -5180,7 +5185,7 @@ void sentinelFailoverWaitPromotion(sentinelRedisInstance *ri) {
void sentinelFailoverDetectEnd(sentinelRedisInstance *master) { void sentinelFailoverDetectEnd(sentinelRedisInstance *master) {
int not_reconfigured = 0, timeout = 0; int not_reconfigured = 0, timeout = 0;
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
mstime_t elapsed = mstime() - master->failover_state_change_time; mstime_t elapsed = mstime() - master->failover_state_change_time;
@ -5191,15 +5196,15 @@ void sentinelFailoverDetectEnd(sentinelRedisInstance *master) {
/* The failover terminates once all the reachable slaves are properly /* The failover terminates once all the reachable slaves are properly
* configured. */ * configured. */
di = dictGetIterator(master->slaves); dictInitIterator(&di, master->slaves);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
sentinelRedisInstance *slave = dictGetVal(de); sentinelRedisInstance *slave = dictGetVal(de);
if (slave->flags & (SRI_PROMOTED|SRI_RECONF_DONE)) continue; if (slave->flags & (SRI_PROMOTED|SRI_RECONF_DONE)) continue;
if (slave->flags & SRI_S_DOWN) continue; if (slave->flags & SRI_S_DOWN) continue;
not_reconfigured++; not_reconfigured++;
} }
dictReleaseIterator(di); dictResetIterator(&di);
/* Force end of failover on timeout. */ /* Force end of failover on timeout. */
if (elapsed > master->failover_timeout) { if (elapsed > master->failover_timeout) {
@ -5218,11 +5223,11 @@ void sentinelFailoverDetectEnd(sentinelRedisInstance *master) {
* command to all the slaves still not reconfigured to replicate with * command to all the slaves still not reconfigured to replicate with
* the new master. */ * the new master. */
if (timeout) { if (timeout) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
di = dictGetIterator(master->slaves); dictInitIterator(&di, master->slaves);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
sentinelRedisInstance *slave = dictGetVal(de); sentinelRedisInstance *slave = dictGetVal(de);
int retval; int retval;
@ -5235,29 +5240,29 @@ void sentinelFailoverDetectEnd(sentinelRedisInstance *master) {
slave->flags |= SRI_RECONF_SENT; slave->flags |= SRI_RECONF_SENT;
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
} }
/* Send SLAVE OF <new master address> to all the remaining slaves that /* Send SLAVE OF <new master address> to all the remaining slaves that
* still don't appear to have the configuration updated. */ * still don't appear to have the configuration updated. */
void sentinelFailoverReconfNextSlave(sentinelRedisInstance *master) { void sentinelFailoverReconfNextSlave(sentinelRedisInstance *master) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
int in_progress = 0; int in_progress = 0;
di = dictGetIterator(master->slaves); dictInitIterator(&di, master->slaves);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
sentinelRedisInstance *slave = dictGetVal(de); sentinelRedisInstance *slave = dictGetVal(de);
if (slave->flags & (SRI_RECONF_SENT|SRI_RECONF_INPROG)) if (slave->flags & (SRI_RECONF_SENT|SRI_RECONF_INPROG))
in_progress++; in_progress++;
} }
dictReleaseIterator(di); dictResetIterator(&di);
di = dictGetIterator(master->slaves); dictInitIterator(&di, master->slaves);
while(in_progress < master->parallel_syncs && while(in_progress < master->parallel_syncs &&
(de = dictNext(di)) != NULL) (de = dictNext(&di)) != NULL)
{ {
sentinelRedisInstance *slave = dictGetVal(de); sentinelRedisInstance *slave = dictGetVal(de);
int retval; int retval;
@ -5292,7 +5297,7 @@ void sentinelFailoverReconfNextSlave(sentinelRedisInstance *master) {
in_progress++; in_progress++;
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
/* Check if all the slaves are reconfigured and handle timeout. */ /* Check if all the slaves are reconfigured and handle timeout. */
sentinelFailoverDetectEnd(master); sentinelFailoverDetectEnd(master);
@ -5397,13 +5402,13 @@ void sentinelHandleRedisInstance(sentinelRedisInstance *ri) {
/* Perform scheduled operations for all the instances in the dictionary. /* Perform scheduled operations for all the instances in the dictionary.
* Recursively call the function against dictionaries of slaves. */ * Recursively call the function against dictionaries of slaves. */
void sentinelHandleDictOfRedisInstances(dict *instances) { void sentinelHandleDictOfRedisInstances(dict *instances) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
sentinelRedisInstance *switch_to_promoted = NULL; sentinelRedisInstance *switch_to_promoted = NULL;
/* There are a number of things we need to perform against every master. */ /* There are a number of things we need to perform against every master. */
di = dictGetIterator(instances); dictInitIterator(&di, instances);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
sentinelRedisInstance *ri = dictGetVal(de); sentinelRedisInstance *ri = dictGetVal(de);
sentinelHandleRedisInstance(ri); sentinelHandleRedisInstance(ri);
@ -5417,7 +5422,7 @@ void sentinelHandleDictOfRedisInstances(dict *instances) {
} }
if (switch_to_promoted) if (switch_to_promoted)
sentinelFailoverSwitchToPromotedSlave(switch_to_promoted); sentinelFailoverSwitchToPromotedSlave(switch_to_promoted);
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* This function checks if we need to enter the TILT mode. /* This function checks if we need to enter the TILT mode.

View File

@ -3275,10 +3275,10 @@ void populateCommandTable(void) {
void resetCommandTableStats(dict* commands) { void resetCommandTableStats(dict* commands) {
struct redisCommand *c; struct redisCommand *c;
dictEntry *de; dictEntry *de;
dictIterator *di; dictIterator di;
di = dictGetSafeIterator(commands); dictInitSafeIterator(&di, commands);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
c = (struct redisCommand *) dictGetVal(de); c = (struct redisCommand *) dictGetVal(de);
c->microseconds = 0; c->microseconds = 0;
c->calls = 0; c->calls = 0;
@ -3291,7 +3291,7 @@ void resetCommandTableStats(dict* commands) {
if (c->subcommands_dict) if (c->subcommands_dict)
resetCommandTableStats(c->subcommands_dict); resetCommandTableStats(c->subcommands_dict);
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
void resetErrorTableStats(void) { void resetErrorTableStats(void) {
@ -5214,14 +5214,15 @@ void addReplyCommandSubCommands(client *c, struct redisCommand *cmd, void (*repl
else else
addReplyArrayLen(c, dictSize(cmd->subcommands_dict)); addReplyArrayLen(c, dictSize(cmd->subcommands_dict));
dictEntry *de; dictEntry *de;
dictIterator *di = dictGetSafeIterator(cmd->subcommands_dict); dictIterator di;
while((de = dictNext(di)) != NULL) { dictInitSafeIterator(&di, cmd->subcommands_dict);
while((de = dictNext(&di)) != NULL) {
struct redisCommand *sub = (struct redisCommand *)dictGetVal(de); struct redisCommand *sub = (struct redisCommand *)dictGetVal(de);
if (use_map) if (use_map)
addReplyBulkCBuffer(c, sub->fullname, sdslen(sub->fullname)); addReplyBulkCBuffer(c, sub->fullname, sdslen(sub->fullname));
reply_function(c, sub); reply_function(c, sub);
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* Output the representation of a Redis command. Used by the COMMAND command and COMMAND INFO. */ /* Output the representation of a Redis command. Used by the COMMAND command and COMMAND INFO. */
@ -5375,7 +5376,7 @@ void getKeysSubcommand(client *c) {
} }
void genericCommandCommand(client *c, int count_only) { void genericCommandCommand(client *c, int count_only) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
void *len = NULL; void *len = NULL;
int count = 0; int count = 0;
@ -5383,8 +5384,8 @@ void genericCommandCommand(client *c, int count_only) {
if (!count_only) if (!count_only)
len = addReplyDeferredLen(c); len = addReplyDeferredLen(c);
di = dictGetIterator(server.commands); dictInitIterator(&di, server.commands);
while ((de = dictNext(di)) != NULL) { while ((de = dictNext(&di)) != NULL) {
struct redisCommand *cmd = dictGetVal(de); struct redisCommand *cmd = dictGetVal(de);
if (!commandVisibleForClient(c, cmd)) if (!commandVisibleForClient(c, cmd))
continue; continue;
@ -5392,7 +5393,7 @@ void genericCommandCommand(client *c, int count_only) {
addReplyCommandInfo(c, dictGetVal(de)); addReplyCommandInfo(c, dictGetVal(de));
count++; count++;
} }
dictReleaseIterator(di); dictResetIterator(&di);
if (count_only) if (count_only)
addReplyLongLong(c, count); addReplyLongLong(c, count);
else else
@ -5456,9 +5457,10 @@ int shouldFilterFromCommandList(struct redisCommand *cmd, commandListFilter *fil
/* COMMAND LIST FILTERBY (MODULE <module-name>|ACLCAT <cat>|PATTERN <pattern>) */ /* COMMAND LIST FILTERBY (MODULE <module-name>|ACLCAT <cat>|PATTERN <pattern>) */
void commandListWithFilter(client *c, dict *commands, commandListFilter filter, int *numcmds) { void commandListWithFilter(client *c, dict *commands, commandListFilter filter, int *numcmds) {
dictEntry *de; dictEntry *de;
dictIterator *di = dictGetIterator(commands); dictIterator di;
while ((de = dictNext(di)) != NULL) { dictInitIterator(&di, commands);
while ((de = dictNext(&di)) != NULL) {
struct redisCommand *cmd = dictGetVal(de); struct redisCommand *cmd = dictGetVal(de);
if (commandVisibleForClient(c, cmd) && !shouldFilterFromCommandList(cmd,&filter)) { if (commandVisibleForClient(c, cmd) && !shouldFilterFromCommandList(cmd,&filter)) {
addReplyBulkCBuffer(c, cmd->fullname, sdslen(cmd->fullname)); addReplyBulkCBuffer(c, cmd->fullname, sdslen(cmd->fullname));
@ -5469,15 +5471,16 @@ void commandListWithFilter(client *c, dict *commands, commandListFilter filter,
commandListWithFilter(c, cmd->subcommands_dict, filter, numcmds); commandListWithFilter(c, cmd->subcommands_dict, filter, numcmds);
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* COMMAND LIST */ /* COMMAND LIST */
void commandListWithoutFilter(client *c, dict *commands, int *numcmds) { void commandListWithoutFilter(client *c, dict *commands, int *numcmds) {
dictEntry *de; dictEntry *de;
dictIterator *di = dictGetIterator(commands); dictIterator di;
while ((de = dictNext(di)) != NULL) { dictInitIterator(&di, commands);
while ((de = dictNext(&di)) != NULL) {
struct redisCommand *cmd = dictGetVal(de); struct redisCommand *cmd = dictGetVal(de);
if (commandVisibleForClient(c, cmd)) { if (commandVisibleForClient(c, cmd)) {
addReplyBulkCBuffer(c, cmd->fullname, sdslen(cmd->fullname)); addReplyBulkCBuffer(c, cmd->fullname, sdslen(cmd->fullname));
@ -5488,7 +5491,7 @@ void commandListWithoutFilter(client *c, dict *commands, int *numcmds) {
commandListWithoutFilter(c, cmd->subcommands_dict, numcmds); commandListWithoutFilter(c, cmd->subcommands_dict, numcmds);
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
} }
/* COMMAND LIST [FILTERBY (MODULE <module-name>|ACLCAT <cat>|PATTERN <pattern>)] */ /* COMMAND LIST [FILTERBY (MODULE <module-name>|ACLCAT <cat>|PATTERN <pattern>)] */
@ -5553,11 +5556,11 @@ void commandDocsCommand(client *c) {
int numcmds = 0; int numcmds = 0;
if (c->argc == 2) { if (c->argc == 2) {
/* Reply with an array of all commands */ /* Reply with an array of all commands */
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
void *replylen = addReplyDeferredLen(c); void *replylen = addReplyDeferredLen(c);
di = dictGetIterator(server.commands); dictInitIterator(&di, server.commands);
while ((de = dictNext(di)) != NULL) { while ((de = dictNext(&di)) != NULL) {
struct redisCommand *cmd = dictGetVal(de); struct redisCommand *cmd = dictGetVal(de);
if (commandVisibleForClient(c, cmd)) { if (commandVisibleForClient(c, cmd)) {
addReplyBulkCBuffer(c, cmd->fullname, sdslen(cmd->fullname)); addReplyBulkCBuffer(c, cmd->fullname, sdslen(cmd->fullname));
@ -5565,7 +5568,7 @@ void commandDocsCommand(client *c) {
numcmds++; numcmds++;
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
setDeferredMapLen(c,replylen,numcmds); setDeferredMapLen(c,replylen,numcmds);
} else { } else {
/* Reply with an array of the requested commands (if we find them) */ /* Reply with an array of the requested commands (if we find them) */
@ -5698,9 +5701,9 @@ const char *getSafeInfoString(const char *s, size_t len, char **tmp) {
sds genRedisInfoStringCommandStats(sds info, dict *commands) { sds genRedisInfoStringCommandStats(sds info, dict *commands) {
struct redisCommand *c; struct redisCommand *c;
dictEntry *de; dictEntry *de;
dictIterator *di; dictIterator di;
di = dictGetSafeIterator(commands); dictInitSafeIterator(&di, commands);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
char *tmpsafe; char *tmpsafe;
c = (struct redisCommand *) dictGetVal(de); c = (struct redisCommand *) dictGetVal(de);
if (c->calls || c->failed_calls || c->rejected_calls) { if (c->calls || c->failed_calls || c->rejected_calls) {
@ -5716,7 +5719,7 @@ sds genRedisInfoStringCommandStats(sds info, dict *commands) {
info = genRedisInfoStringCommandStats(info, c->subcommands_dict); info = genRedisInfoStringCommandStats(info, c->subcommands_dict);
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
return info; return info;
} }
@ -5738,9 +5741,9 @@ sds genRedisInfoStringACLStats(sds info) {
sds genRedisInfoStringLatencyStats(sds info, dict *commands) { sds genRedisInfoStringLatencyStats(sds info, dict *commands) {
struct redisCommand *c; struct redisCommand *c;
dictEntry *de; dictEntry *de;
dictIterator *di; dictIterator di;
di = dictGetSafeIterator(commands); dictInitSafeIterator(&di, commands);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
char *tmpsafe; char *tmpsafe;
c = (struct redisCommand *) dictGetVal(de); c = (struct redisCommand *) dictGetVal(de);
if (c->latency_histogram) { if (c->latency_histogram) {
@ -5753,7 +5756,7 @@ sds genRedisInfoStringLatencyStats(sds info, dict *commands) {
info = genRedisInfoStringLatencyStats(info, c->subcommands_dict); info = genRedisInfoStringLatencyStats(info, c->subcommands_dict);
} }
} }
dictReleaseIterator(di); dictResetIterator(&di);
return info; return info;
} }

View File

@ -467,18 +467,19 @@ void sortCommandGeneric(client *c, int readonly) {
start = 0; start = 0;
} else if (sortval->type == OBJ_ZSET) { } else if (sortval->type == OBJ_ZSET) {
dict *set = ((zset*)sortval->ptr)->dict; dict *set = ((zset*)sortval->ptr)->dict;
dictIterator *di; dictIterator di;
dictEntry *setele; dictEntry *setele;
sds sdsele; sds sdsele;
di = dictGetIterator(set);
while((setele = dictNext(di)) != NULL) { dictInitIterator(&di, set);
while((setele = dictNext(&di)) != NULL) {
sdsele = dictGetKey(setele); sdsele = dictGetKey(setele);
vector[j].obj = createStringObject(sdsele,sdslen(sdsele)); vector[j].obj = createStringObject(sdsele,sdslen(sdsele));
vector[j].u.score = 0; vector[j].u.score = 0;
vector[j].u.cmpobj = NULL; vector[j].u.cmpobj = NULL;
j++; j++;
} }
dictReleaseIterator(di); dictResetIterator(&di);
} else { } else {
serverPanic("Unknown type"); serverPanic("Unknown type");
} }

View File

@ -3208,7 +3208,7 @@ void hrandfieldWithCountCommand(client *c, long l, int withvalues) {
if (count*HRANDFIELD_SUB_STRATEGY_MUL > size) { if (count*HRANDFIELD_SUB_STRATEGY_MUL > size) {
/* Hashtable encoding (generic implementation) */ /* Hashtable encoding (generic implementation) */
dict *ht = hash->ptr; dict *ht = hash->ptr;
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
unsigned long idx = 0; unsigned long idx = 0;
@ -3220,10 +3220,10 @@ void hrandfieldWithCountCommand(client *c, long l, int withvalues) {
} *pairs = zmalloc(sizeof(struct FieldValPair) * size); } *pairs = zmalloc(sizeof(struct FieldValPair) * size);
/* Add all the elements into the temporary array. */ /* Add all the elements into the temporary array. */
di = dictGetIterator(ht); dictInitIterator(&di, ht);
while((de = dictNext(di)) != NULL) while((de = dictNext(&di)) != NULL)
pairs[idx++] = (struct FieldValPair) {dictGetKey(de), dictGetVal(de)}; pairs[idx++] = (struct FieldValPair) {dictGetKey(de), dictGetVal(de)};
dictReleaseIterator(di); dictResetIterator(&di);
/* Remove random elements to reach the right count. */ /* Remove random elements to reach the right count. */
while (size > count) { while (size > count) {

View File

@ -1218,14 +1218,14 @@ void srandmemberWithCountCommand(client *c) {
/* CASE 3 & 4: send the result to the user. */ /* CASE 3 & 4: send the result to the user. */
{ {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
addReplyArrayLen(c,count); addReplyArrayLen(c,count);
di = dictGetIterator(d); dictInitIterator(&di, d);
while((de = dictNext(di)) != NULL) while((de = dictNext(&di)) != NULL)
addReplyBulkSds(c,dictGetKey(de)); addReplyBulkSds(c,dictGetKey(de));
dictReleaseIterator(di); dictResetIterator(&di);
dictRelease(d); dictRelease(d);
} }
} }

View File

@ -2433,20 +2433,20 @@ inline static void zunionInterAggregate(double *target, double val, int aggregat
} }
static size_t zsetDictGetMaxElementLength(dict *d, size_t *totallen) { static size_t zsetDictGetMaxElementLength(dict *d, size_t *totallen) {
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
size_t maxelelen = 0; size_t maxelelen = 0;
di = dictGetIterator(d); dictInitIterator(&di, d);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
sds ele = dictGetKey(de); sds ele = dictGetKey(de);
if (sdslen(ele) > maxelelen) maxelelen = sdslen(ele); if (sdslen(ele) > maxelelen) maxelelen = sdslen(ele);
if (totallen) if (totallen)
(*totallen) += sdslen(ele); (*totallen) += sdslen(ele);
} }
dictReleaseIterator(di); dictResetIterator(&di);
return maxelelen; return maxelelen;
} }
@ -2808,7 +2808,7 @@ void zunionInterDiffGenericCommand(client *c, robj *dstkey, int numkeysIndex, in
zuiClearIterator(&src[0]); zuiClearIterator(&src[0]);
} }
} else if (op == SET_OP_UNION) { } else if (op == SET_OP_UNION) {
dictIterator *di; dictIterator di;
dictEntry *de, *existing; dictEntry *de, *existing;
double score; double score;
@ -2857,15 +2857,15 @@ void zunionInterDiffGenericCommand(client *c, robj *dstkey, int numkeysIndex, in
} }
/* Step 2: convert the dictionary into the final sorted set. */ /* Step 2: convert the dictionary into the final sorted set. */
di = dictGetIterator(dstzset->dict); dictInitIterator(&di, dstzset->dict);
while((de = dictNext(di)) != NULL) { while((de = dictNext(&di)) != NULL) {
sds ele = dictGetKey(de); sds ele = dictGetKey(de);
score = dictGetDoubleVal(de); score = dictGetDoubleVal(de);
znode = zslInsert(dstzset->zsl,score,ele); znode = zslInsert(dstzset->zsl,score,ele);
dictSetVal(dstzset->dict,de,&znode->score); dictSetVal(dstzset->dict,de,&znode->score);
} }
dictReleaseIterator(di); dictResetIterator(&di);
} else if (op == SET_OP_DIFF) { } else if (op == SET_OP_DIFF) {
zdiff(src, setnum, dstzset, &maxelelen, &totelelen); zdiff(src, setnum, dstzset, &maxelelen, &totelelen);
} else { } else {
@ -4353,10 +4353,11 @@ void zrandmemberWithCountCommand(client *c, long l, int withscores) {
} }
/* Reply with what's in the dict and release memory */ /* Reply with what's in the dict and release memory */
dictIterator *di; dictIterator di;
dictEntry *de; dictEntry *de;
di = dictGetIterator(d);
while ((de = dictNext(di)) != NULL) { dictInitIterator(&di, d);
while ((de = dictNext(&di)) != NULL) {
if (withscores && c->resp > 2) if (withscores && c->resp > 2)
addReplyArrayLen(c,2); addReplyArrayLen(c,2);
addReplyBulkSds(c, dictGetKey(de)); addReplyBulkSds(c, dictGetKey(de));
@ -4364,7 +4365,7 @@ void zrandmemberWithCountCommand(client *c, long l, int withscores) {
addReplyDouble(c, dictGetDoubleVal(de)); addReplyDouble(c, dictGetDoubleVal(de));
} }
dictReleaseIterator(di); dictResetIterator(&di);
dictRelease(d); dictRelease(d);
} }