Commit 852ec0ab authored by Vitaly Arbuzov's avatar Vitaly Arbuzov
Browse files

Replace cluster metadata with slot specific dictionaries

parent 2bb29e4a
...@@ -2229,7 +2229,10 @@ int rewriteAppendOnlyFileRio(rio *aof) { ...@@ -2229,7 +2229,10 @@ int rewriteAppendOnlyFileRio(rio *aof) {
/* Record timestamp at the beginning of rewriting AOF. */ /* Record timestamp at the beginning of rewriting AOF. */
if (server.aof_timestamp_enabled) { if (server.aof_timestamp_enabled) {
sds ts = genAofTimestampAnnotationIfNeeded(1); sds ts = genAofTimestampAnnotationIfNeeded(1);
if (rioWrite(aof,ts,sdslen(ts)) == 0) { sdsfree(ts); goto werr; } if (rioWrite(aof, ts, sdslen(ts)) == 0) {
sdsfree(ts);
goto werr;
}
sdsfree(ts); sdsfree(ts);
} }
...@@ -2237,83 +2240,85 @@ int rewriteAppendOnlyFileRio(rio *aof) { ...@@ -2237,83 +2240,85 @@ int rewriteAppendOnlyFileRio(rio *aof) {
for (j = 0; j < server.dbnum; j++) { for (j = 0; j < server.dbnum; j++) {
char selectcmd[] = "*2\r\n$6\r\nSELECT\r\n"; char selectcmd[] = "*2\r\n$6\r\nSELECT\r\n";
redisDb *db = server.db+j; redisDb *db = server.db + j;
dict *d = db->dict; for (int k = 0; k < db->dict_count; k++) {
if (dictSize(d) == 0) continue; dict *d = db->dict[k];
di = dictGetSafeIterator(d); if (dictSize(d) == 0) continue;
di = dictGetSafeIterator(d);
/* SELECT the new DB */
if (rioWrite(aof,selectcmd,sizeof(selectcmd)-1) == 0) goto werr; /* SELECT the new DB */
if (rioWriteBulkLongLong(aof,j) == 0) goto werr; if (rioWrite(aof, selectcmd, sizeof(selectcmd) - 1) == 0) goto werr;
if (rioWriteBulkLongLong(aof, j) == 0) goto werr;
/* Iterate this DB writing every entry */
while((de = dictNext(di)) != NULL) { /* Iterate this DB writing every entry */
sds keystr; while ((de = dictNext(di)) != NULL) {
robj key, *o; sds keystr;
long long expiretime; robj key, *o;
size_t aof_bytes_before_key = aof->processed_bytes; long long expiretime;
size_t aof_bytes_before_key = aof->processed_bytes;
keystr = dictGetKey(de);
o = dictGetVal(de); keystr = dictGetKey(de);
initStaticStringObject(key,keystr); o = dictGetVal(de);
initStaticStringObject(key, keystr);
expiretime = getExpire(db,&key);
expiretime = getExpire(db, &key);
/* Save the key and associated value */
if (o->type == OBJ_STRING) { /* Save the key and associated value */
/* Emit a SET command */ if (o->type == OBJ_STRING) {
char cmd[]="*3\r\n$3\r\nSET\r\n"; /* Emit a SET command */
if (rioWrite(aof,cmd,sizeof(cmd)-1) == 0) goto werr; char cmd[] = "*3\r\n$3\r\nSET\r\n";
/* Key and value */ if (rioWrite(aof, cmd, sizeof(cmd) - 1) == 0) goto werr;
if (rioWriteBulkObject(aof,&key) == 0) goto werr; /* Key and value */
if (rioWriteBulkObject(aof,o) == 0) goto werr; if (rioWriteBulkObject(aof, &key) == 0) goto werr;
} else if (o->type == OBJ_LIST) { if (rioWriteBulkObject(aof, o) == 0) goto werr;
if (rewriteListObject(aof,&key,o) == 0) goto werr; } else if (o->type == OBJ_LIST) {
} else if (o->type == OBJ_SET) { if (rewriteListObject(aof, &key, o) == 0) goto werr;
if (rewriteSetObject(aof,&key,o) == 0) goto werr; } else if (o->type == OBJ_SET) {
} else if (o->type == OBJ_ZSET) { if (rewriteSetObject(aof, &key, o) == 0) goto werr;
if (rewriteSortedSetObject(aof,&key,o) == 0) goto werr; } else if (o->type == OBJ_ZSET) {
} else if (o->type == OBJ_HASH) { if (rewriteSortedSetObject(aof, &key, o) == 0) goto werr;
if (rewriteHashObject(aof,&key,o) == 0) goto werr; } else if (o->type == OBJ_HASH) {
} else if (o->type == OBJ_STREAM) { if (rewriteHashObject(aof, &key, o) == 0) goto werr;
if (rewriteStreamObject(aof,&key,o) == 0) goto werr; } else if (o->type == OBJ_STREAM) {
} else if (o->type == OBJ_MODULE) { if (rewriteStreamObject(aof, &key, o) == 0) goto werr;
if (rewriteModuleObject(aof,&key,o,j) == 0) goto werr; } else if (o->type == OBJ_MODULE) {
} else { if (rewriteModuleObject(aof, &key, o, j) == 0) goto werr;
serverPanic("Unknown object type"); } else {
} serverPanic("Unknown object type");
}
/* In fork child process, we can try to release memory back to the /* In fork child process, we can try to release memory back to the
* OS and possibly avoid or decrease COW. We give the dismiss * OS and possibly avoid or decrease COW. We give the dismiss
* mechanism a hint about an estimated size of the object we stored. */ * mechanism a hint about an estimated size of the object we stored. */
size_t dump_size = aof->processed_bytes - aof_bytes_before_key; size_t dump_size = aof->processed_bytes - aof_bytes_before_key;
if (server.in_fork_child) dismissObject(o, dump_size); if (server.in_fork_child) dismissObject(o, dump_size);
/* Save the expire time */ /* Save the expire time */
if (expiretime != -1) { if (expiretime != -1) {
char cmd[]="*3\r\n$9\r\nPEXPIREAT\r\n"; char cmd[] = "*3\r\n$9\r\nPEXPIREAT\r\n";
if (rioWrite(aof,cmd,sizeof(cmd)-1) == 0) goto werr; if (rioWrite(aof, cmd, sizeof(cmd) - 1) == 0) goto werr;
if (rioWriteBulkObject(aof,&key) == 0) goto werr; if (rioWriteBulkObject(aof, &key) == 0) goto werr;
if (rioWriteBulkLongLong(aof,expiretime) == 0) goto werr; if (rioWriteBulkLongLong(aof, expiretime) == 0) goto werr;
} }
/* Update info every 1 second (approximately). /* Update info every 1 second (approximately).
* in order to avoid calling mstime() on each iteration, we will * in order to avoid calling mstime() on each iteration, we will
* check the diff every 1024 keys */ * check the diff every 1024 keys */
if ((key_count++ & 1023) == 0) { if ((key_count++ & 1023) == 0) {
long long now = mstime(); long long now = mstime();
if (now - updated_time >= 1000) { if (now - updated_time >= 1000) {
sendChildInfo(CHILD_INFO_TYPE_CURRENT_INFO, key_count, "AOF rewrite"); sendChildInfo(CHILD_INFO_TYPE_CURRENT_INFO, key_count, "AOF rewrite");
updated_time = now; updated_time = now;
}
} }
}
/* Delay before next key if required (for testing) */ /* Delay before next key if required (for testing) */
if (server.rdb_key_save_delay) if (server.rdb_key_save_delay)
debugDelay(server.rdb_key_save_delay); debugDelay(server.rdb_key_save_delay);
}
dictReleaseIterator(di);
di = NULL;
} }
dictReleaseIterator(di);
di = NULL;
} }
return C_OK; return C_OK;
......
...@@ -90,6 +90,7 @@ int auxShardIdSetter(clusterNode *n, void *value, int length); ...@@ -90,6 +90,7 @@ int auxShardIdSetter(clusterNode *n, void *value, int length);
sds auxShardIdGetter(clusterNode *n, sds s); sds auxShardIdGetter(clusterNode *n, sds s);
int auxShardIdPresent(clusterNode *n); int auxShardIdPresent(clusterNode *n);
static void clusterBuildMessageHdr(clusterMsg *hdr, int type, size_t msglen); static void clusterBuildMessageHdr(clusterMsg *hdr, int type, size_t msglen);
unsigned long long int dbSize(const redisDb *db);
/* Links to the next and previous entries for keys in the same slot are stored /* Links to the next and previous entries for keys in the same slot are stored
* in the dict entry metadata. See Slot to Key API below. */ * in the dict entry metadata. See Slot to Key API below. */
...@@ -903,9 +904,6 @@ void clusterInit(void) { ...@@ -903,9 +904,6 @@ void clusterInit(void) {
exit(1); exit(1);
} }
/* Initialize data for the Slot to key API. */
slotToKeyInit(server.db);
/* The slots -> channels map is a radix tree. Initialize it here. */ /* The slots -> channels map is a radix tree. Initialize it here. */
server.cluster->slots_to_channels = raxNew(); server.cluster->slots_to_channels = raxNew();
...@@ -3877,13 +3875,13 @@ void clusterLogCantFailover(int reason) { ...@@ -3877,13 +3875,13 @@ void clusterLogCantFailover(int reason) {
} }
lastlog_time = time(NULL); lastlog_time = time(NULL);
serverLog(LL_NOTICE,"Currently unable to failover: %s", msg); serverLog(LL_NOTICE,"Currently unable to failover: %s", msg);
int cur_vote = server.cluster->failover_auth_count; int cur_vote = server.cluster->failover_auth_count;
int cur_quorum = (server.cluster->size / 2) + 1; int cur_quorum = (server.cluster->size / 2) + 1;
/* Emits a log when an election is in progress and waiting for votes or when the failover attempt expired. */ /* Emits a log when an election is in progress and waiting for votes or when the failover attempt expired. */
if (reason == CLUSTER_CANT_FAILOVER_WAITING_VOTES || reason == CLUSTER_CANT_FAILOVER_EXPIRED) { if (reason == CLUSTER_CANT_FAILOVER_WAITING_VOTES || reason == CLUSTER_CANT_FAILOVER_EXPIRED) {
serverLog(LL_NOTICE, "Needed quorum: %d. Number of votes received so far: %d", cur_quorum, cur_vote); serverLog(LL_NOTICE, "Needed quorum: %d. Number of votes received so far: %d", cur_quorum, cur_vote);
} }
} }
/* This function implements the final part of automatic and manual failovers, /* This function implements the final part of automatic and manual failovers,
...@@ -4886,7 +4884,7 @@ int verifyClusterConfigWithData(void) { ...@@ -4886,7 +4884,7 @@ int verifyClusterConfigWithData(void) {
/* Make sure we only have keys in DB0. */ /* Make sure we only have keys in DB0. */
for (j = 1; j < server.dbnum; j++) { for (j = 1; j < server.dbnum; j++) {
if (dictSize(server.db[j].dict)) return C_ERR; if (dbSize(&server.db[j])) return C_ERR;
} }
/* Check that all the slots we see populated memory have a corresponding /* Check that all the slots we see populated memory have a corresponding
...@@ -5754,7 +5752,7 @@ NULL ...@@ -5754,7 +5752,7 @@ NULL
clusterReplyShards(c); clusterReplyShards(c);
} else if (!strcasecmp(c->argv[1]->ptr,"flushslots") && c->argc == 2) { } else if (!strcasecmp(c->argv[1]->ptr,"flushslots") && c->argc == 2) {
/* CLUSTER FLUSHSLOTS */ /* CLUSTER FLUSHSLOTS */
if (dictSize(server.db[0].dict) != 0) { if (dbSize(&server.db[0]) != 0) {
addReplyError(c,"DB must be empty to perform CLUSTER FLUSHSLOTS."); addReplyError(c,"DB must be empty to perform CLUSTER FLUSHSLOTS.");
return; return;
} }
...@@ -6016,13 +6014,16 @@ NULL ...@@ -6016,13 +6014,16 @@ NULL
unsigned int keys_in_slot = countKeysInSlot(slot); unsigned int keys_in_slot = countKeysInSlot(slot);
unsigned int numkeys = maxkeys > keys_in_slot ? keys_in_slot : maxkeys; unsigned int numkeys = maxkeys > keys_in_slot ? keys_in_slot : maxkeys;
addReplyArrayLen(c,numkeys); addReplyArrayLen(c,numkeys);
dictEntry *de = (*server.db->slots_to_keys).by_slot[slot].head; dictIterator *iter = NULL;
for (unsigned int j = 0; j < numkeys; j++) { dictEntry *de = NULL;
iter = dictGetIterator(server.db->dict[slot]);
for (unsigned int i = 0; i < numkeys; i++) {
de = dictNext(iter);
serverAssert(de != NULL); serverAssert(de != NULL);
sds sdskey = dictGetKey(de); sds sdskey = dictGetKey(de);
addReplyBulkCBuffer(c, sdskey, sdslen(sdskey)); addReplyBulkCBuffer(c, sdskey, sdslen(sdskey));
de = dictEntryNextInSlot(de);
} }
dictReleaseIterator(iter);
} else if (!strcasecmp(c->argv[1]->ptr,"forget") && c->argc == 3) { } else if (!strcasecmp(c->argv[1]->ptr,"forget") && c->argc == 3) {
/* CLUSTER FORGET <NODE ID> */ /* CLUSTER FORGET <NODE ID> */
clusterNode *n = clusterLookupNode(c->argv[2]->ptr, sdslen(c->argv[2]->ptr)); clusterNode *n = clusterLookupNode(c->argv[2]->ptr, sdslen(c->argv[2]->ptr));
...@@ -6071,7 +6072,7 @@ NULL ...@@ -6071,7 +6072,7 @@ NULL
* slots nor keys to accept to replicate some other node. * slots nor keys to accept to replicate some other node.
* Slaves can switch to another master without issues. */ * Slaves can switch to another master without issues. */
if (nodeIsMaster(myself) && if (nodeIsMaster(myself) &&
(myself->numslots != 0 || dictSize(server.db[0].dict) != 0)) { (myself->numslots != 0 || dbSize(&server.db[0]) != 0)) {
addReplyError(c, addReplyError(c,
"To set a master the node must be empty and " "To set a master the node must be empty and "
"without assigned slots."); "without assigned slots.");
...@@ -6231,7 +6232,7 @@ NULL ...@@ -6231,7 +6232,7 @@ NULL
/* Slaves can be reset while containing data, but not master nodes /* Slaves can be reset while containing data, but not master nodes
* that must be empty. */ * that must be empty. */
if (nodeIsMaster(myself) && dictSize(c->db->dict) != 0) { if (nodeIsMaster(myself) && dbSize(c->db) != 0) {
addReplyError(c,"CLUSTER RESET can't be called with " addReplyError(c,"CLUSTER RESET can't be called with "
"master nodes containing keys"); "master nodes containing keys");
return; return;
...@@ -7315,98 +7316,19 @@ int clusterRedirectBlockedClientIfNeeded(client *c) { ...@@ -7315,98 +7316,19 @@ int clusterRedirectBlockedClientIfNeeded(client *c) {
return 0; return 0;
} }
/* Slot to Key API. This is used by Redis Cluster in order to obtain in
* a fast way a key that belongs to a specified hash slot. This is useful
* while rehashing the cluster and in other conditions when we need to
* understand if we have keys for a given hash slot. */
void slotToKeyAddEntry(dictEntry *entry, redisDb *db) {
sds key = dictGetKey(entry);
unsigned int hashslot = keyHashSlot(key, sdslen(key));
slotToKeys *slot_to_keys = &(*db->slots_to_keys).by_slot[hashslot];
slot_to_keys->count++;
/* Insert entry before the first element in the list. */
dictEntry *first = slot_to_keys->head;
dictEntryNextInSlot(entry) = first;
if (first != NULL) {
serverAssert(dictEntryPrevInSlot(first) == NULL);
dictEntryPrevInSlot(first) = entry;
}
serverAssert(dictEntryPrevInSlot(entry) == NULL);
slot_to_keys->head = entry;
}
void slotToKeyDelEntry(dictEntry *entry, redisDb *db) {
sds key = dictGetKey(entry);
unsigned int hashslot = keyHashSlot(key, sdslen(key));
slotToKeys *slot_to_keys = &(*db->slots_to_keys).by_slot[hashslot];
slot_to_keys->count--;
/* Connect previous and next entries to each other. */
dictEntry *next = dictEntryNextInSlot(entry);
dictEntry *prev = dictEntryPrevInSlot(entry);
if (next != NULL) {
dictEntryPrevInSlot(next) = prev;
}
if (prev != NULL) {
dictEntryNextInSlot(prev) = next;
} else {
/* The removed entry was the first in the list. */
serverAssert(slot_to_keys->head == entry);
slot_to_keys->head = next;
}
}
/* Updates neighbour entries when an entry has been replaced (e.g. reallocated
* during active defrag). */
void slotToKeyReplaceEntry(dict *d, dictEntry *entry) {
dictEntry *next = dictEntryNextInSlot(entry);
dictEntry *prev = dictEntryPrevInSlot(entry);
if (next != NULL) {
dictEntryPrevInSlot(next) = entry;
}
if (prev != NULL) {
dictEntryNextInSlot(prev) = entry;
} else {
/* The replaced entry was the first in the list. */
sds key = dictGetKey(entry);
unsigned int hashslot = keyHashSlot(key, sdslen(key));
clusterDictMetadata *dictmeta = dictMetadata(d);
redisDb *db = dictmeta->db;
slotToKeys *slot_to_keys = &(*db->slots_to_keys).by_slot[hashslot];
slot_to_keys->head = entry;
}
}
/* Initialize slots-keys map of given db. */
void slotToKeyInit(redisDb *db) {
db->slots_to_keys = zcalloc(sizeof(clusterSlotToKeyMapping));
clusterDictMetadata *dictmeta = dictMetadata(db->dict);
dictmeta->db = db;
}
/* Empty slots-keys map of given db. */
void slotToKeyFlush(redisDb *db) {
memset(db->slots_to_keys, 0,
sizeof(clusterSlotToKeyMapping));
}
/* Free slots-keys map of given db. */
void slotToKeyDestroy(redisDb *db) {
zfree(db->slots_to_keys);
db->slots_to_keys = NULL;
}
/* Remove all the keys in the specified hash slot. /* Remove all the keys in the specified hash slot.
* The number of removed items is returned. */ * The number of removed items is returned. */
unsigned int delKeysInSlot(unsigned int hashslot) { unsigned int delKeysInSlot(unsigned int slot) {
unsigned int j = 0; unsigned int j = 0;
dictEntry *de = (*server.db->slots_to_keys).by_slot[hashslot].head; server.core_propagates = 1;
while (de != NULL) { server.in_nested_call++;
dictIterator *iter = NULL;
dictEntry *de = NULL;
iter = dictGetSafeIterator(server.db->dict[slot]);
while((de = dictNext(iter)) != NULL) {
sds sdskey = dictGetKey(de); sds sdskey = dictGetKey(de);
de = dictEntryNextInSlot(de);
robj *key = createStringObject(sdskey, sdslen(sdskey)); robj *key = createStringObject(sdskey, sdslen(sdskey));
dbDelete(&server.db[0], key); dbDelete(&server.db[0], key);
propagateDeletion(&server.db[0], key, server.lazyfree_lazy_server_del); propagateDeletion(&server.db[0], key, server.lazyfree_lazy_server_del);
...@@ -7417,12 +7339,14 @@ unsigned int delKeysInSlot(unsigned int hashslot) { ...@@ -7417,12 +7339,14 @@ unsigned int delKeysInSlot(unsigned int hashslot) {
j++; j++;
server.dirty++; server.dirty++;
} }
dictReleaseIterator(iter);
serverAssert(server.core_propagates); /* This function should not be re-entrant */
return j; return j;
} }
unsigned int countKeysInSlot(unsigned int hashslot) { unsigned int countKeysInSlot(unsigned int slot) {
return (*server.db->slots_to_keys).by_slot[hashslot].count; return dictSize(server.db->dict[slot]);
} }
/* ----------------------------------------------------------------------------- /* -----------------------------------------------------------------------------
......
...@@ -150,29 +150,6 @@ typedef struct clusterNode { ...@@ -150,29 +150,6 @@ typedef struct clusterNode {
list *fail_reports; /* List of nodes signaling this as failing */ list *fail_reports; /* List of nodes signaling this as failing */
} clusterNode; } clusterNode;
/* Slot to keys for a single slot. The keys in the same slot are linked together
* using dictEntry metadata. */
typedef struct slotToKeys {
uint64_t count; /* Number of keys in the slot. */
dictEntry *head; /* The first key-value entry in the slot. */
} slotToKeys;
/* Slot to keys mapping for all slots, opaque outside this file. */
struct clusterSlotToKeyMapping {
slotToKeys by_slot[CLUSTER_SLOTS];
};
/* Dict entry metadata for cluster mode, used for the Slot to Key API to form a
* linked list of the entries belonging to the same slot. */
typedef struct clusterDictEntryMetadata {
dictEntry *prev; /* Prev entry with key in the same slot */
dictEntry *next; /* Next entry with key in the same slot */
} clusterDictEntryMetadata;
typedef struct {
redisDb *db; /* A link back to the db this dict belongs to */
} clusterDictMetadata;
typedef struct clusterState { typedef struct clusterState {
clusterNode *myself; /* This node */ clusterNode *myself; /* This node */
uint64_t currentEpoch; uint64_t currentEpoch;
...@@ -410,12 +387,6 @@ unsigned long getClusterConnectionsCount(void); ...@@ -410,12 +387,6 @@ unsigned long getClusterConnectionsCount(void);
int clusterSendModuleMessageToTarget(const char *target, uint64_t module_id, uint8_t type, const char *payload, uint32_t len); int clusterSendModuleMessageToTarget(const char *target, uint64_t module_id, uint8_t type, const char *payload, uint32_t len);
void clusterPropagatePublish(robj *channel, robj *message, int sharded); void clusterPropagatePublish(robj *channel, robj *message, int sharded);
unsigned int keyHashSlot(char *key, int keylen); unsigned int keyHashSlot(char *key, int keylen);
void slotToKeyAddEntry(dictEntry *entry, redisDb *db);
void slotToKeyDelEntry(dictEntry *entry, redisDb *db);
void slotToKeyReplaceEntry(dict *d, dictEntry *entry);
void slotToKeyInit(redisDb *db);
void slotToKeyFlush(redisDb *db);
void slotToKeyDestroy(redisDb *db);
void clusterUpdateMyselfFlags(void); void clusterUpdateMyselfFlags(void);
void clusterUpdateMyselfIp(void); void clusterUpdateMyselfIp(void);
void slotToChannelAdd(sds channel); void slotToChannelAdd(sds channel);
......
...@@ -85,7 +85,7 @@ void updateLFU(robj *val) { ...@@ -85,7 +85,7 @@ void updateLFU(robj *val) {
* expired on replicas even if the master is lagging expiring our key via DELs * expired on replicas even if the master is lagging expiring our key via DELs
* in the replication link. */ * in the replication link. */
robj *lookupKey(redisDb *db, robj *key, int flags) { robj *lookupKey(redisDb *db, robj *key, int flags) {
dictEntry *de = dictFind(db->dict,key->ptr); dictEntry *de = dictFind(getDict(db, key->ptr),key->ptr);
robj *val = NULL; robj *val = NULL;
if (de) { if (de) {
val = dictGetVal(de); val = dictGetVal(de);
...@@ -190,14 +190,19 @@ robj *lookupKeyWriteOrReply(client *c, robj *key, robj *reply) { ...@@ -190,14 +190,19 @@ robj *lookupKeyWriteOrReply(client *c, robj *key, robj *reply) {
* The program is aborted if the key already exists. */ * The program is aborted if the key already exists. */
void dbAdd(redisDb *db, robj *key, robj *val) { void dbAdd(redisDb *db, robj *key, robj *val) {
sds copy = sdsdup(key->ptr); sds copy = sdsdup(key->ptr);
dictEntry *de = dictAddRaw(db->dict, copy, NULL); dict *d = getDict(db, key->ptr);
dictEntry *de = dictAddRaw(d, copy, NULL);
serverAssertWithInfo(NULL, key, de != NULL); serverAssertWithInfo(NULL, key, de != NULL);
dictSetVal(db->dict, de, val); dictSetVal(d, de, val);
signalKeyAsReady(db, key, val->type); signalKeyAsReady(db, key, val->type);
if (server.cluster_enabled) slotToKeyAddEntry(de, db);
notifyKeyspaceEvent(NOTIFY_NEW,"new",key,db->id); notifyKeyspaceEvent(NOTIFY_NEW,"new",key,db->id);
} }
/* Return slot-specific dictionary for key based on key's hash slot in CME, or 0 in CMD.*/
dict *getDict(redisDb *db, sds key) {
return db->dict[(server.cluster_enabled ? keyHashSlot(key, (int) sdslen(key)) : 0)];
}
/* This is a special version of dbAdd() that is used only when loading /* This is a special version of dbAdd() that is used only when loading
* keys from the RDB file: the key is passed as an SDS string that is * keys from the RDB file: the key is passed as an SDS string that is
* retained by the function (and not freed by the caller). * retained by the function (and not freed by the caller).
...@@ -210,10 +215,10 @@ void dbAdd(redisDb *db, robj *key, robj *val) { ...@@ -210,10 +215,10 @@ void dbAdd(redisDb *db, robj *key, robj *val) {
* ownership of the SDS string, otherwise 0 is returned, and is up to the * ownership of the SDS string, otherwise 0 is returned, and is up to the
* caller to free the SDS string. */ * caller to free the SDS string. */
int dbAddRDBLoad(redisDb *db, sds key, robj *val) { int dbAddRDBLoad(redisDb *db, sds key, robj *val) {
dictEntry *de = dictAddRaw(db->dict, key, NULL); dict *d = getDict(db, key);
dictEntry *de = dictAddRaw(d, key, NULL);
if (de == NULL) return 0; if (de == NULL) return 0;
dictSetVal(db->dict, de, val); dictSetVal(d, de, val);
if (server.cluster_enabled) slotToKeyAddEntry(de, db);
return 1; return 1;
} }
...@@ -228,7 +233,8 @@ int dbAddRDBLoad(redisDb *db, sds key, robj *val) { ...@@ -228,7 +233,8 @@ int dbAddRDBLoad(redisDb *db, sds key, robj *val) {
* *
* The program is aborted if the key was not already present. */ * The program is aborted if the key was not already present. */
static void dbSetValue(redisDb *db, robj *key, robj *val, int overwrite) { static void dbSetValue(redisDb *db, robj *key, robj *val, int overwrite) {
dictEntry *de = dictFind(db->dict,key->ptr); dict *d = getDict(db, key->ptr);
dictEntry *de = dictFind(d, key->ptr);
serverAssertWithInfo(NULL,key,de != NULL); serverAssertWithInfo(NULL,key,de != NULL);
robj *old = dictGetVal(de); robj *old = dictGetVal(de);
...@@ -249,13 +255,13 @@ static void dbSetValue(redisDb *db, robj *key, robj *val, int overwrite) { ...@@ -249,13 +255,13 @@ static void dbSetValue(redisDb *db, robj *key, robj *val, int overwrite) {
/* Because of RM_StringDMA, old may be changed, so we need get old again */ /* Because of RM_StringDMA, old may be changed, so we need get old again */
old = dictGetVal(de); old = dictGetVal(de);
} }
dictSetVal(db->dict, de, val); dictSetVal(d, de, val);
if (server.lazyfree_lazy_server_del) { if (server.lazyfree_lazy_server_del) {
freeObjAsync(key,old,db->id); freeObjAsync(key,old,db->id);
} else { } else {
/* This is just decrRefCount(old); */ /* This is just decrRefCount(old); */
db->dict->type->valDestructor(db->dict, old); db->dict->type->valDestructor(d, old);
} }
} }
...@@ -303,19 +309,19 @@ void setKey(client *c, redisDb *db, robj *key, robj *val, int flags) { ...@@ -303,19 +309,19 @@ void setKey(client *c, redisDb *db, robj *key, robj *val, int flags) {
robj *dbRandomKey(redisDb *db) { robj *dbRandomKey(redisDb *db) {
dictEntry *de; dictEntry *de;
int maxtries = 100; int maxtries = 100;
int allvolatile = dictSize(db->dict) == dictSize(db->expires); dict *randomDict = getRandomDict(db, 0);
while(1) { while(1) {
sds key; sds key;
robj *keyobj; robj *keyobj;
de = dictGetFairRandomKey(db->dict); de = dictGetFairRandomKey(randomDict);
if (de == NULL) return NULL; if (de == NULL) return NULL; // TODO consider other non-empty slot buckets.
key = dictGetKey(de); key = dictGetKey(de);
keyobj = createStringObject(key,sdslen(key)); keyobj = createStringObject(key,sdslen(key));
if (dictFind(db->expires,key)) { if (dictFind(db->expires,key)) {
if (allvolatile && server.masterhost && --maxtries == 0) { if (server.masterhost && --maxtries == 0) {
/* If the DB is composed only of keys with an expire set, /* If the DB is composed only of keys with an expire set,
* it could happen that all the keys are already logically * it could happen that all the keys are already logically
* expired in the slave, so the function cannot stop because * expired in the slave, so the function cannot stop because
...@@ -335,11 +341,28 @@ robj *dbRandomKey(redisDb *db) { ...@@ -335,11 +341,28 @@ robj *dbRandomKey(redisDb *db) {
} }
} }
/* Return random non-empty dictionary from this DB, if shouldBeRehashing is set, then it ignores dicts that aren't rehashing. */
dict *getRandomDict(redisDb *db, int shouldBeRehashing) {
if (db->dict_count == 1) return db->dict[0];
int i = 0, r = 0;
for (int j = 0; j < CLUSTER_SLOTS; j++) {
// Skip empty dicts or if we want only rehashing dicts and the dict isn't rehashing.
if (dictSize(db->dict[j]) == 0 || (shouldBeRehashing && !dictIsRehashing(db->dict[j]))) continue;
if (i == 0 || (rand() % (i + 1)) == 0) {
r = j; // Select K-th non-empty bucket with 1/K probability, this keeps balanced probabilities for all non-empty buckets.
}
i++;
}
return db->dict[r];
}
/* Helper for sync and async delete. */ /* Helper for sync and async delete. */
int dbGenericDelete(redisDb *db, robj *key, int async, int flags) { int dbGenericDelete(redisDb *db, robj *key, int async, int flags) {
dictEntry **plink; dictEntry **plink;
int table; int table;
dictEntry *de = dictTwoPhaseUnlinkFind(db->dict,key->ptr,&plink,&table); dict *d = getDict(db, key->ptr);
dictEntry *de = dictTwoPhaseUnlinkFind(d,key->ptr,&plink,&table);
if (de) { if (de) {
robj *val = dictGetVal(de); robj *val = dictGetVal(de);
/* RM_StringDMA may call dbUnshareStringValue which may free val, so we /* RM_StringDMA may call dbUnshareStringValue which may free val, so we
...@@ -355,14 +378,12 @@ int dbGenericDelete(redisDb *db, robj *key, int async, int flags) { ...@@ -355,14 +378,12 @@ int dbGenericDelete(redisDb *db, robj *key, int async, int flags) {
if (async) { if (async) {
/* Because of dbUnshareStringValue, the val in de may change. */ /* Because of dbUnshareStringValue, the val in de may change. */
freeObjAsync(key, dictGetVal(de), db->id); freeObjAsync(key, dictGetVal(de), db->id);
dictSetVal(db->dict, de, NULL); dictSetVal(d, de, NULL);
} }
if (server.cluster_enabled) slotToKeyDelEntry(de, db);
/* Deleting an entry from the expires dict will not free the sds of /* Deleting an entry from the expires dict will not free the sds of
* the key, because it is shared with the main dictionary. */ * the key, because it is shared with the main dictionary. */
if (dictSize(db->expires) > 0) dictDelete(db->expires,key->ptr); if (dictSize(db->expires) > 0) dictDelete(db->expires,key->ptr);
dictTwoPhaseUnlinkFree(db->dict,de,plink,table); dictTwoPhaseUnlinkFree(d,de,plink,table);
return 1; return 1;
} else { } else {
return 0; return 0;
...@@ -444,11 +465,13 @@ long long emptyDbStructure(redisDb *dbarray, int dbnum, int async, ...@@ -444,11 +465,13 @@ long long emptyDbStructure(redisDb *dbarray, int dbnum, int async,
} }
for (int j = startdb; j <= enddb; j++) { for (int j = startdb; j <= enddb; j++) {
removed += dictSize(dbarray[j].dict); removed += dbSize(&dbarray[j]);
if (async) { if (async) {
emptyDbAsync(&dbarray[j]); emptyDbAsync(&dbarray[j]);
} else { } else {
dictEmpty(dbarray[j].dict,callback); for (int k=0; k<dbarray[j].dict_count;k++) {
dictEmpty(dbarray[j].dict[k],callback);
}
dictEmpty(dbarray[j].expires,callback); dictEmpty(dbarray[j].expires,callback);
} }
/* Because all keys of database are removed, reset average ttl. */ /* Because all keys of database are removed, reset average ttl. */
...@@ -498,11 +521,6 @@ long long emptyData(int dbnum, int flags, void(callback)(dict*)) { ...@@ -498,11 +521,6 @@ long long emptyData(int dbnum, int flags, void(callback)(dict*)) {
/* Empty redis database structure. */ /* Empty redis database structure. */
removed = emptyDbStructure(server.db, dbnum, async, callback); removed = emptyDbStructure(server.db, dbnum, async, callback);
/* Flush slots to keys map if enable cluster, we can flush entire
* slots to keys map whatever dbnum because only support one DB
* in cluster mode. */
if (server.cluster_enabled) slotToKeyFlush(server.db);
if (dbnum == -1) flushSlaveKeysWithExpireList(); if (dbnum == -1) flushSlaveKeysWithExpireList();
if (with_functions) { if (with_functions) {
...@@ -523,14 +541,9 @@ long long emptyData(int dbnum, int flags, void(callback)(dict*)) { ...@@ -523,14 +541,9 @@ long long emptyData(int dbnum, int flags, void(callback)(dict*)) {
redisDb *initTempDb(void) { redisDb *initTempDb(void) {
redisDb *tempDb = zcalloc(sizeof(redisDb)*server.dbnum); redisDb *tempDb = zcalloc(sizeof(redisDb)*server.dbnum);
for (int i=0; i<server.dbnum; i++) { for (int i=0; i<server.dbnum; i++) {
tempDb[i].dict = dictCreate(&dbDictType); tempDb[i].dict_count = (server.cluster_enabled) ? CLUSTER_SLOTS : 1;
tempDb[i].dict = dictCreateMultiple(&dbDictType, tempDb[i].dict_count);
tempDb[i].expires = dictCreate(&dbExpiresDictType); tempDb[i].expires = dictCreate(&dbExpiresDictType);
tempDb[i].slots_to_keys = NULL;
}
if (server.cluster_enabled) {
/* Prepare temp slot to key map to be written during async diskless replication. */
slotToKeyInit(tempDb);
} }
return tempDb; return tempDb;
...@@ -543,15 +556,12 @@ void discardTempDb(redisDb *tempDb, void(callback)(dict*)) { ...@@ -543,15 +556,12 @@ void discardTempDb(redisDb *tempDb, void(callback)(dict*)) {
/* Release temp DBs. */ /* Release temp DBs. */
emptyDbStructure(tempDb, -1, async, callback); emptyDbStructure(tempDb, -1, async, callback);
for (int i=0; i<server.dbnum; i++) { for (int i=0; i<server.dbnum; i++) {
dictRelease(tempDb[i].dict); for (int j=0; j<tempDb[i].dict_count; j++) {
dictRelease(tempDb[i].dict[j]);
}
dictRelease(tempDb[i].expires); dictRelease(tempDb[i].expires);
} }
if (server.cluster_enabled) {
/* Release temp slot to key map. */
slotToKeyDestroy(tempDb);
}
zfree(tempDb); zfree(tempDb);
} }
...@@ -566,7 +576,7 @@ long long dbTotalServerKeyCount() { ...@@ -566,7 +576,7 @@ long long dbTotalServerKeyCount() {
long long total = 0; long long total = 0;
int j; int j;
for (j = 0; j < server.dbnum; j++) { for (j = 0; j < server.dbnum; j++) {
total += dictSize(server.db[j].dict); total += dbSize(&server.db[j]);
} }
return total; return total;
} }
...@@ -769,27 +779,29 @@ void keysCommand(client *c) { ...@@ -769,27 +779,29 @@ void keysCommand(client *c) {
dictEntry *de; dictEntry *de;
sds pattern = c->argv[1]->ptr; sds pattern = c->argv[1]->ptr;
int plen = sdslen(pattern), allkeys; int plen = sdslen(pattern), allkeys;
unsigned long numkeys = 0; long numkeys = 0;
void *replylen = addReplyDeferredLen(c); void *replylen = addReplyDeferredLen(c);
for (int i = 0; i < c->db->dict_count; i++) {
di = dictGetSafeIterator(c->db->dict); dict *d = c->db->dict[i];
allkeys = (pattern[0] == '*' && plen == 1); if (dictSize(d) == 0) continue;
while((de = dictNext(di)) != NULL) { di = dictGetSafeIterator(d);
sds key = dictGetKey(de); allkeys = (pattern[0] == '*' && plen == 1);
robj *keyobj; while((de = dictNext(di)) != NULL) {
sds key = dictGetKey(de);
if (allkeys || stringmatchlen(pattern,plen,key,sdslen(key),0)) { robj *keyobj;
keyobj = createStringObject(key,sdslen(key));
if (!keyIsExpired(c->db,keyobj)) { if (allkeys || stringmatchlen(pattern,plen,key,sdslen(key),0)) {
addReplyBulk(c,keyobj); keyobj = createStringObject(key,sdslen(key));
numkeys++; if (!keyIsExpired(c->db,keyobj)) {
addReplyBulk(c,keyobj);
numkeys++;
}
decrRefCount(keyobj);
} }
decrRefCount(keyobj);
}
if (c->flags & CLIENT_CLOSE_ASAP) if (c->flags & CLIENT_CLOSE_ASAP)
break; break;}
dictReleaseIterator(di);
} }
dictReleaseIterator(di);
setDeferredArrayLen(c,replylen,numkeys); setDeferredArrayLen(c,replylen,numkeys);
} }
...@@ -828,7 +840,7 @@ void scanCallback(void *privdata, const dictEntry *de) { ...@@ -828,7 +840,7 @@ void scanCallback(void *privdata, const dictEntry *de) {
* if the cursor is valid, store it as unsigned integer into *cursor and * if the cursor is valid, store it as unsigned integer into *cursor and
* returns C_OK. Otherwise return C_ERR and send an error to the * returns C_OK. Otherwise return C_ERR and send an error to the
* client. */ * client. */
int parseScanCursorOrReply(client *c, robj *o, unsigned long *cursor) { int parseScanCursorOrReply(client *c, robj *o, unsigned long long *cursor) {
char *eptr; char *eptr;
/* Use strtoul() because we need an *unsigned* long, so /* Use strtoul() because we need an *unsigned* long, so
...@@ -854,7 +866,7 @@ int parseScanCursorOrReply(client *c, robj *o, unsigned long *cursor) { ...@@ -854,7 +866,7 @@ int parseScanCursorOrReply(client *c, robj *o, unsigned long *cursor) {
* *
* In the case of a Hash object the function returns both the field and value * In the case of a Hash object the function returns both the field and value
* of every element on the Hash. */ * of every element on the Hash. */
void scanGenericCommand(client *c, robj *o, unsigned long cursor) { void scanGenericCommand(client *c, robj *o, unsigned long long cursor) {
int i, j; int i, j;
list *keys = listCreate(); list *keys = listCreate();
listNode *node, *nextnode; listNode *node, *nextnode;
...@@ -917,8 +929,14 @@ void scanGenericCommand(client *c, robj *o, unsigned long cursor) { ...@@ -917,8 +929,14 @@ void scanGenericCommand(client *c, robj *o, unsigned long cursor) {
/* Handle the case of a hash table. */ /* Handle the case of a hash table. */
ht = NULL; ht = NULL;
unsigned long long slot_mask = (unsigned long long)(CLUSTER_SLOTS - 1) << SLOT_MASK_SHIFT;
/* During main dictionary traversal in cluster mode, 48 lower bits in the cursor are used for positioning in the HT.
* Following 14 bits are used for the slot number, ranging from 0 to 2^14-1.
* Slot is always 0 at the start of iteration and can be incremented only in cluster mode. */
int slot = (int) ((cursor & slot_mask) >> SLOT_MASK_SHIFT);
cursor = cursor & ~slot_mask; // clear slot mask from cursor for the time of iteration.
if (o == NULL) { if (o == NULL) {
ht = c->db->dict; ht = c->db->dict[slot];
} else if (o->type == OBJ_SET && o->encoding == OBJ_ENCODING_HT) { } else if (o->type == OBJ_SET && o->encoding == OBJ_ENCODING_HT) {
ht = o->ptr; ht = o->ptr;
} else if (o->type == OBJ_HASH && o->encoding == OBJ_ENCODING_HT) { } else if (o->type == OBJ_HASH && o->encoding == OBJ_ENCODING_HT) {
...@@ -943,11 +961,23 @@ void scanGenericCommand(client *c, robj *o, unsigned long cursor) { ...@@ -943,11 +961,23 @@ void scanGenericCommand(client *c, robj *o, unsigned long cursor) {
* it is possible to fetch more data in a type-dependent way. */ * it is possible to fetch more data in a type-dependent way. */
privdata[0] = keys; privdata[0] = keys;
privdata[1] = o; privdata[1] = o;
int hasUnvisitedSlot;
do { do {
hasUnvisitedSlot = 0;
cursor = dictScan(ht, cursor, scanCallback, privdata); cursor = dictScan(ht, cursor, scanCallback, privdata);
} while (cursor && /* In cluster mode there is a separate dictionary for each slot.
maxiterations-- && * If cursor is empty, we should try exploring next non-empty slot. */
while (o == NULL && !cursor && slot < c->db->dict_count - 1) {
slot++; ht = c->db->dict[slot];
if (dictSize(ht) > 0) {
hasUnvisitedSlot = 1;
break; // Found next non-empty slot.
}
}
} while ((cursor || hasUnvisitedSlot) &&
maxiterations-- &&
listLength(keys) < (unsigned long)count); listLength(keys) < (unsigned long)count);
if (cursor) cursor |= ((unsigned long long) slot) << SLOT_MASK_SHIFT;
} else if (o->type == OBJ_SET) { } else if (o->type == OBJ_SET) {
char *str; char *str;
size_t len; size_t len;
...@@ -1053,13 +1083,31 @@ cleanup: ...@@ -1053,13 +1083,31 @@ cleanup:
/* The SCAN command completely relies on scanGenericCommand. */ /* The SCAN command completely relies on scanGenericCommand. */
void scanCommand(client *c) { void scanCommand(client *c) {
unsigned long cursor; unsigned long long cursor;
if (parseScanCursorOrReply(c,c->argv[1],&cursor) == C_ERR) return; if (parseScanCursorOrReply(c,c->argv[1],&cursor) == C_ERR) return;
scanGenericCommand(c,NULL,cursor); scanGenericCommand(c,NULL,cursor);
} }
void dbsizeCommand(client *c) { void dbsizeCommand(client *c) {
addReplyLongLong(c,dictSize(c->db->dict)); redisDb *db = c->db;
unsigned long long int size = dbSize(db);
addReplyLongLong(c, size);
}
unsigned long long int dbSize(const redisDb *db) {
unsigned long long size = 0;
for (int i = 0; i < db->dict_count; i++) {
size += dictSize(db->dict[i]);
}
return size;
}
unsigned long dbSlots(const redisDb *db) {
unsigned long slots = 0;
for (int i = 0; i < db->dict_count; i++) {
slots += dictSlots(db->dict[i]);
}
return slots;
} }
void lastsaveCommand(client *c) { void lastsaveCommand(client *c) {
...@@ -1378,7 +1426,7 @@ void scanDatabaseForReadyKeys(redisDb *db) { ...@@ -1378,7 +1426,7 @@ void scanDatabaseForReadyKeys(redisDb *db) {
dictIterator *di = dictGetSafeIterator(db->blocking_keys); dictIterator *di = dictGetSafeIterator(db->blocking_keys);
while((de = dictNext(di)) != NULL) { while((de = dictNext(di)) != NULL) {
robj *key = dictGetKey(de); robj *key = dictGetKey(de);
dictEntry *kde = dictFind(db->dict,key->ptr); dictEntry *kde = dictFind(getDict(db, key->ptr), key->ptr);
if (kde) { if (kde) {
robj *value = dictGetVal(kde); robj *value = dictGetVal(kde);
signalKeyAsReady(db, key, value->type); signalKeyAsReady(db, key, value->type);
...@@ -1398,7 +1446,7 @@ void scanDatabaseForDeletedKeys(redisDb *emptied, redisDb *replaced_with) { ...@@ -1398,7 +1446,7 @@ void scanDatabaseForDeletedKeys(redisDb *emptied, redisDb *replaced_with) {
int existed = 0, exists = 0; int existed = 0, exists = 0;
int original_type = -1, curr_type = -1; int original_type = -1, curr_type = -1;
dictEntry *kde = dictFind(emptied->dict, key->ptr); dictEntry *kde = dictFind(getDict(emptied, key->ptr), key->ptr);
if (kde) { if (kde) {
robj *value = dictGetVal(kde); robj *value = dictGetVal(kde);
original_type = value->type; original_type = value->type;
...@@ -1406,7 +1454,7 @@ void scanDatabaseForDeletedKeys(redisDb *emptied, redisDb *replaced_with) { ...@@ -1406,7 +1454,7 @@ void scanDatabaseForDeletedKeys(redisDb *emptied, redisDb *replaced_with) {
} }
if (replaced_with) { if (replaced_with) {
dictEntry *kde = dictFind(replaced_with->dict, key->ptr); kde = dictFind(getDict(replaced_with, key->ptr), key->ptr);
if (kde) { if (kde) {
robj *value = dictGetVal(kde); robj *value = dictGetVal(kde);
curr_type = value->type; curr_type = value->type;
...@@ -1451,11 +1499,13 @@ int dbSwapDatabases(int id1, int id2) { ...@@ -1451,11 +1499,13 @@ int dbSwapDatabases(int id1, int id2) {
db1->expires = db2->expires; db1->expires = db2->expires;
db1->avg_ttl = db2->avg_ttl; db1->avg_ttl = db2->avg_ttl;
db1->expires_cursor = db2->expires_cursor; db1->expires_cursor = db2->expires_cursor;
db1->dict_count = db2->dict_count;
db2->dict = aux.dict; db2->dict = aux.dict;
db2->expires = aux.expires; db2->expires = aux.expires;
db2->avg_ttl = aux.avg_ttl; db2->avg_ttl = aux.avg_ttl;
db2->expires_cursor = aux.expires_cursor; db2->expires_cursor = aux.expires_cursor;
db2->dict_count = aux.dict_count;
/* Now we need to handle clients blocked on lists: as an effect /* Now we need to handle clients blocked on lists: as an effect
* of swapping the two DBs, a client that was waiting for list * of swapping the two DBs, a client that was waiting for list
...@@ -1475,13 +1525,6 @@ int dbSwapDatabases(int id1, int id2) { ...@@ -1475,13 +1525,6 @@ int dbSwapDatabases(int id1, int id2) {
* database (temp) as the main (active) database, the actual freeing of old database * database (temp) as the main (active) database, the actual freeing of old database
* (which will now be placed in the temp one) is done later. */ * (which will now be placed in the temp one) is done later. */
void swapMainDbWithTempDb(redisDb *tempDb) { void swapMainDbWithTempDb(redisDb *tempDb) {
if (server.cluster_enabled) {
/* Swap slots_to_keys from tempdb just loaded with main db slots_to_keys. */
clusterSlotToKeyMapping *aux = server.db->slots_to_keys;
server.db->slots_to_keys = tempDb->slots_to_keys;
tempDb->slots_to_keys = aux;
}
for (int i=0; i<server.dbnum; i++) { for (int i=0; i<server.dbnum; i++) {
redisDb aux = server.db[i]; redisDb aux = server.db[i];
redisDb *activedb = &server.db[i], *newdb = &tempDb[i]; redisDb *activedb = &server.db[i], *newdb = &tempDb[i];
...@@ -1500,11 +1543,13 @@ void swapMainDbWithTempDb(redisDb *tempDb) { ...@@ -1500,11 +1543,13 @@ void swapMainDbWithTempDb(redisDb *tempDb) {
activedb->expires = newdb->expires; activedb->expires = newdb->expires;
activedb->avg_ttl = newdb->avg_ttl; activedb->avg_ttl = newdb->avg_ttl;
activedb->expires_cursor = newdb->expires_cursor; activedb->expires_cursor = newdb->expires_cursor;
activedb->dict_count = newdb->dict_count;
newdb->dict = aux.dict; newdb->dict = aux.dict;
newdb->expires = aux.expires; newdb->expires = aux.expires;
newdb->avg_ttl = aux.avg_ttl; newdb->avg_ttl = aux.avg_ttl;
newdb->expires_cursor = aux.expires_cursor; newdb->expires_cursor = aux.expires_cursor;
newdb->dict_count = aux.dict_count;
/* Now we need to handle clients blocked on lists: as an effect /* Now we need to handle clients blocked on lists: as an effect
* of swapping the two DBs, a client that was waiting for list * of swapping the two DBs, a client that was waiting for list
...@@ -1560,7 +1605,7 @@ void swapdbCommand(client *c) { ...@@ -1560,7 +1605,7 @@ void swapdbCommand(client *c) {
int removeExpire(redisDb *db, robj *key) { int removeExpire(redisDb *db, robj *key) {
/* An expire may only be removed if there is a corresponding entry in the /* An expire may only be removed if there is a corresponding entry in the
* main dict. Otherwise, the key will never be freed. */ * main dict. Otherwise, the key will never be freed. */
serverAssertWithInfo(NULL,key,dictFind(db->dict,key->ptr) != NULL); serverAssertWithInfo(NULL,key,dictFind(getDict(db, key->ptr),key->ptr) != NULL);
return dictDelete(db->expires,key->ptr) == DICT_OK; return dictDelete(db->expires,key->ptr) == DICT_OK;
} }
...@@ -1572,7 +1617,7 @@ void setExpire(client *c, redisDb *db, robj *key, long long when) { ...@@ -1572,7 +1617,7 @@ void setExpire(client *c, redisDb *db, robj *key, long long when) {
dictEntry *kde, *de; dictEntry *kde, *de;
/* Reuse the sds from the main dict in the expire dict */ /* Reuse the sds from the main dict in the expire dict */
kde = dictFind(db->dict,key->ptr); kde = dictFind(getDict(db, key->ptr),key->ptr);
serverAssertWithInfo(NULL,key,kde != NULL); serverAssertWithInfo(NULL,key,kde != NULL);
de = dictAddOrFind(db->expires,dictGetKey(kde)); de = dictAddOrFind(db->expires,dictGetKey(kde));
dictSetSignedIntegerVal(de,when); dictSetSignedIntegerVal(de,when);
...@@ -1593,7 +1638,7 @@ long long getExpire(redisDb *db, robj *key) { ...@@ -1593,7 +1638,7 @@ long long getExpire(redisDb *db, robj *key) {
/* The entry was found in the expire dict, this means it should also /* The entry was found in the expire dict, this means it should also
* be present in the main dict (safety check). */ * be present in the main dict (safety check). */
serverAssertWithInfo(NULL,key,dictFind(db->dict,key->ptr) != NULL); serverAssertWithInfo(NULL,key,dictFind(getDict(db, key->ptr),key->ptr) != NULL);
return dictGetSignedIntegerVal(de); return dictGetSignedIntegerVal(de);
} }
......
...@@ -285,34 +285,37 @@ void computeDatasetDigest(unsigned char *final) { ...@@ -285,34 +285,37 @@ void computeDatasetDigest(unsigned char *final) {
for (j = 0; j < server.dbnum; j++) { for (j = 0; j < server.dbnum; j++) {
redisDb *db = server.db+j; redisDb *db = server.db+j;
int hasEntries = 0;
for (int k = 0; k < db->dict_count; k++) {
if (dictSize(db->dict[k]) == 0) continue;
hasEntries = 1;
di = dictGetSafeIterator(db->dict[k]);
if (dictSize(db->dict) == 0) continue; /* Iterate this DB writing every entry */
di = dictGetSafeIterator(db->dict); while ((de = dictNext(di)) != NULL) {
sds key;
robj *keyobj, *o;
/* hash the DB id, so the same dataset moved in a different memset(digest, 0, 20); /* This key-val digest */
* DB will lead to a different digest */ key = dictGetKey(de);
aux = htonl(j); keyobj = createStringObject(key, sdslen(key));
mixDigest(final,&aux,sizeof(aux));
/* Iterate this DB writing every entry */ mixDigest(digest, key, sdslen(key));
while((de = dictNext(di)) != NULL) {
sds key;
robj *keyobj, *o;
memset(digest,0,20); /* This key-val digest */ o = dictGetVal(de);
key = dictGetKey(de); xorObjectDigest(db, keyobj, digest, o);
keyobj = createStringObject(key,sdslen(key));
mixDigest(digest,key,sdslen(key)); /* We can finally xor the key-val digest to the final digest */
xorDigest(final, digest, 20);
o = dictGetVal(de); decrRefCount(keyobj);
xorObjectDigest(db,keyobj,digest,o); }
dictReleaseIterator(di);
/* We can finally xor the key-val digest to the final digest */ }
xorDigest(final,digest,20); if (hasEntries) {
decrRefCount(keyobj); /* hash the DB id, so the same dataset moved in a different DB will lead to a different digest */
aux = htonl(j);
mixDigest(final, &aux, sizeof(aux));
} }
dictReleaseIterator(di);
} }
} }
...@@ -604,7 +607,7 @@ NULL ...@@ -604,7 +607,7 @@ NULL
robj *val; robj *val;
char *strenc; char *strenc;
if ((de = dictFind(c->db->dict,c->argv[2]->ptr)) == NULL) { if ((de = dictFind(getDict(c->db, c->argv[2]->ptr),c->argv[2]->ptr)) == NULL) {
addReplyErrorObject(c,shared.nokeyerr); addReplyErrorObject(c,shared.nokeyerr);
return; return;
} }
...@@ -656,7 +659,7 @@ NULL ...@@ -656,7 +659,7 @@ NULL
robj *val; robj *val;
sds key; sds key;
if ((de = dictFind(c->db->dict,c->argv[2]->ptr)) == NULL) { if ((de = dictFind(getDict(c->db, c->argv[2]->ptr), c->argv[2]->ptr)) == NULL) {
addReplyErrorObject(c,shared.nokeyerr); addReplyErrorObject(c,shared.nokeyerr);
return; return;
} }
...@@ -712,7 +715,7 @@ NULL ...@@ -712,7 +715,7 @@ NULL
if (getPositiveLongFromObjectOrReply(c, c->argv[2], &keys, NULL) != C_OK) if (getPositiveLongFromObjectOrReply(c, c->argv[2], &keys, NULL) != C_OK)
return; return;
dictExpand(c->db->dict,keys); // dictExpand(c->db->dict,keys); FIXME (vitarb) find right dicts to expand https://sim.amazon.com/issues/ELMO-63796
long valsize = 0; long valsize = 0;
if ( c->argc == 5 && getPositiveLongFromObjectOrReply(c, c->argv[4], &valsize, NULL) != C_OK ) if ( c->argc == 5 && getPositiveLongFromObjectOrReply(c, c->argv[4], &valsize, NULL) != C_OK )
return; return;
...@@ -757,7 +760,7 @@ NULL ...@@ -757,7 +760,7 @@ NULL
/* We don't use lookupKey because a debug command should /* We don't use lookupKey because a debug command should
* work on logically expired keys */ * work on logically expired keys */
dictEntry *de; dictEntry *de;
robj *o = ((de = dictFind(c->db->dict,c->argv[j]->ptr)) == NULL) ? NULL : dictGetVal(de); robj *o = ((de = dictFind(getDict(c->db, c->argv[j]->ptr),c->argv[j]->ptr)) == NULL) ? NULL : dictGetVal(de);
if (o) xorObjectDigest(c->db,c->argv[j],digest,o); if (o) xorObjectDigest(c->db,c->argv[j],digest,o);
sds d = sdsempty(); sds d = sdsempty();
...@@ -895,7 +898,7 @@ NULL ...@@ -895,7 +898,7 @@ NULL
} }
stats = sdscatprintf(stats,"[Dictionary HT]\n"); stats = sdscatprintf(stats,"[Dictionary HT]\n");
dictGetStats(buf,sizeof(buf),server.db[dbid].dict); // dictGetStats(buf,sizeof(buf),server.db[dbid].dict); FIXME (vitarb) get aggregated stats https://sim.amazon.com/issues/ELMO-63798
stats = sdscat(stats,buf); stats = sdscat(stats,buf);
stats = sdscatprintf(stats,"[Expires HT]\n"); stats = sdscatprintf(stats,"[Expires HT]\n");
...@@ -1871,7 +1874,7 @@ void logCurrentClient(client *cc, const char *title) { ...@@ -1871,7 +1874,7 @@ void logCurrentClient(client *cc, const char *title) {
dictEntry *de; dictEntry *de;
key = getDecodedObject(cc->argv[1]); key = getDecodedObject(cc->argv[1]);
de = dictFind(cc->db->dict, key->ptr); de = dictFind(getDict(cc->db, key->ptr), key->ptr);
if (de) { if (de) {
val = dictGetVal(de); val = dictGetVal(de);
serverLog(LL_WARNING,"key '%s' found in DB containing the following object:", (char*)key->ptr); serverLog(LL_WARNING,"key '%s' found in DB containing the following object:", (char*)key->ptr);
......
...@@ -194,6 +194,16 @@ dict *dictCreate(dictType *type) ...@@ -194,6 +194,16 @@ dict *dictCreate(dictType *type)
return d; return d;
} }
/* Create an array of dictionaries */
dict **dictCreateMultiple(dictType *type, int count)
{
dict **d = zmalloc(sizeof(dict*) * count);
for (int i = 0; i < count; i++) {
d[i] = dictCreate(type);
}
return d;
}
/* Initialize the hash table */ /* Initialize the hash table */
int _dictInit(dict *d, dictType *type) int _dictInit(dict *d, dictType *type)
{ {
...@@ -295,7 +305,7 @@ int dictTryExpand(dict *d, unsigned long size) { ...@@ -295,7 +305,7 @@ int dictTryExpand(dict *d, unsigned long size) {
int dictRehash(dict *d, int n) { int dictRehash(dict *d, int n) {
int empty_visits = n*10; /* Max number of empty buckets to visit. */ int empty_visits = n*10; /* Max number of empty buckets to visit. */
if (dict_can_resize == DICT_RESIZE_FORBID || !dictIsRehashing(d)) return 0; if (dict_can_resize == DICT_RESIZE_FORBID || !dictIsRehashing(d)) return 0;
if (dict_can_resize == DICT_RESIZE_AVOID && if (dict_can_resize == DICT_RESIZE_AVOID &&
(DICTHT_SIZE(d->ht_size_exp[1]) / DICTHT_SIZE(d->ht_size_exp[0]) < dict_force_resize_ratio)) (DICTHT_SIZE(d->ht_size_exp[1]) / DICTHT_SIZE(d->ht_size_exp[0]) < dict_force_resize_ratio))
{ {
return 0; return 0;
...@@ -469,9 +479,7 @@ dictEntry *dictInsertAtPosition(dict *d, void *key, void *position) { ...@@ -469,9 +479,7 @@ dictEntry *dictInsertAtPosition(dict *d, void *key, void *position) {
int htidx = dictIsRehashing(d) ? 1 : 0; int htidx = dictIsRehashing(d) ? 1 : 0;
assert(bucket >= &d->ht_table[htidx][0] && assert(bucket >= &d->ht_table[htidx][0] &&
bucket <= &d->ht_table[htidx][DICTHT_SIZE_MASK(d->ht_size_exp[htidx])]); bucket <= &d->ht_table[htidx][DICTHT_SIZE_MASK(d->ht_size_exp[htidx])]);
size_t metasize = dictEntryMetadataSize(d);
if (d->type->no_value) { if (d->type->no_value) {
assert(!metasize); /* Entry metadata + no value not supported. */
if (d->type->keys_are_odd && !*bucket) { if (d->type->keys_are_odd && !*bucket) {
/* We can store the key directly in the destination bucket without the /* We can store the key directly in the destination bucket without the
* allocated entry. * allocated entry.
...@@ -491,11 +499,8 @@ dictEntry *dictInsertAtPosition(dict *d, void *key, void *position) { ...@@ -491,11 +499,8 @@ dictEntry *dictInsertAtPosition(dict *d, void *key, void *position) {
* Insert the element in top, with the assumption that in a database * Insert the element in top, with the assumption that in a database
* system it is more likely that recently added entries are accessed * system it is more likely that recently added entries are accessed
* more frequently. */ * more frequently. */
entry = zmalloc(sizeof(*entry) + metasize); entry = zmalloc(sizeof(*entry));
assert(entryIsNormal(entry)); /* Check alignment of allocation */ assert(entryIsNormal(entry)); /* Check alignment of allocation */
if (metasize > 0) {
memset(dictEntryMetadata(entry), 0, metasize);
}
entry->key = key; entry->key = key;
entry->next = *bucket; entry->next = *bucket;
} }
...@@ -1173,7 +1178,7 @@ dictEntry *dictGetFairRandomKey(dict *d) { ...@@ -1173,7 +1178,7 @@ dictEntry *dictGetFairRandomKey(dict *d) {
/* Function to reverse bits. Algorithm from: /* Function to reverse bits. Algorithm from:
* http://graphics.stanford.edu/~seander/bithacks.html#ReverseParallel */ * http://graphics.stanford.edu/~seander/bithacks.html#ReverseParallel */
static unsigned long rev(unsigned long v) { unsigned long rev(unsigned long v) {
unsigned long s = CHAR_BIT * sizeof(v); // bit size; must be power of 2 unsigned long s = CHAR_BIT * sizeof(v); // bit size; must be power of 2
unsigned long mask = ~0UL; unsigned long mask = ~0UL;
while ((s >>= 1) > 0) { while ((s >>= 1) > 0) {
......
...@@ -44,7 +44,16 @@ ...@@ -44,7 +44,16 @@
#define DICT_OK 0 #define DICT_OK 0
#define DICT_ERR 1 #define DICT_ERR 1
typedef struct dictEntry dictEntry; /* opaque */ typedef struct dictEntry {
void *key;
union {
void *val;
uint64_t u64;
int64_t s64;
double d;
} v;
struct dictEntry *next; /* Next entry in the same hash bucket. */
} dictEntry;
typedef struct dict dict; typedef struct dict dict;
...@@ -68,14 +77,6 @@ typedef struct dictType { ...@@ -68,14 +77,6 @@ typedef struct dictType {
unsigned int keys_are_odd:1; unsigned int keys_are_odd:1;
/* TODO: Add a 'keys_are_even' flag and use a similar optimization if that /* TODO: Add a 'keys_are_even' flag and use a similar optimization if that
* flag is set. */ * flag is set. */
/* Allow each dict and dictEntry to carry extra caller-defined metadata. The
* extra memory is initialized to 0 when allocated. */
size_t (*dictEntryMetadataBytes)(dict *d);
size_t (*dictMetadataBytes)(void);
/* Optional callback called after an entry has been reallocated (due to
* active defrag). Only called if the entry has metadata. */
void (*afterReplaceEntry)(dict *d, dictEntry *entry);
} dictType; } dictType;
#define DICTHT_SIZE(exp) ((exp) == -1 ? 0 : (unsigned long)1<<(exp)) #define DICTHT_SIZE(exp) ((exp) == -1 ? 0 : (unsigned long)1<<(exp))
...@@ -138,11 +139,6 @@ typedef struct { ...@@ -138,11 +139,6 @@ typedef struct {
(d)->type->keyCompare((d), key1, key2) : \ (d)->type->keyCompare((d), key1, key2) : \
(key1) == (key2)) (key1) == (key2))
#define dictEntryMetadataSize(d) ((d)->type->dictEntryMetadataBytes \
? (d)->type->dictEntryMetadataBytes(d) : 0)
#define dictMetadataSize(d) ((d)->type->dictMetadataBytes \
? (d)->type->dictMetadataBytes() : 0)
#define dictHashKey(d, key) ((d)->type->hashFunction(key)) #define dictHashKey(d, key) ((d)->type->hashFunction(key))
#define dictSlots(d) (DICTHT_SIZE((d)->ht_size_exp[0])+DICTHT_SIZE((d)->ht_size_exp[1])) #define dictSlots(d) (DICTHT_SIZE((d)->ht_size_exp[0])+DICTHT_SIZE((d)->ht_size_exp[1]))
#define dictSize(d) ((d)->ht_used[0]+(d)->ht_used[1]) #define dictSize(d) ((d)->ht_used[0]+(d)->ht_used[1])
...@@ -165,6 +161,7 @@ typedef enum { ...@@ -165,6 +161,7 @@ typedef enum {
/* API */ /* API */
dict *dictCreate(dictType *type); dict *dictCreate(dictType *type);
dict **dictCreateMultiple(dictType *type, int count);
int dictExpand(dict *d, unsigned long size); int dictExpand(dict *d, unsigned long size);
int dictTryExpand(dict *d, unsigned long size); int dictTryExpand(dict *d, unsigned long size);
void *dictMetadata(dict *d); void *dictMetadata(dict *d);
...@@ -223,6 +220,7 @@ unsigned long dictScan(dict *d, unsigned long v, dictScanFunction *fn, void *pri ...@@ -223,6 +220,7 @@ unsigned long dictScan(dict *d, unsigned long v, dictScanFunction *fn, void *pri
unsigned long dictScanDefrag(dict *d, unsigned long v, dictScanFunction *fn, dictDefragFunctions *defragfns, void *privdata); unsigned long dictScanDefrag(dict *d, unsigned long v, dictScanFunction *fn, dictDefragFunctions *defragfns, void *privdata);
uint64_t dictGetHash(dict *d, const void *key); uint64_t dictGetHash(dict *d, const void *key);
dictEntry *dictFindEntryByPtrAndHash(dict *d, const void *oldptr, uint64_t hash); dictEntry *dictFindEntryByPtrAndHash(dict *d, const void *oldptr, uint64_t hash);
unsigned long rev(unsigned long v);
#ifdef REDIS_TEST #ifdef REDIS_TEST
int dictTest(int argc, char *argv[], int flags); int dictTest(int argc, char *argv[], int flags);
......
...@@ -143,7 +143,7 @@ void evictionPoolAlloc(void) { ...@@ -143,7 +143,7 @@ void evictionPoolAlloc(void) {
* idle time are on the left, and keys with the higher idle time on the * idle time are on the left, and keys with the higher idle time on the
* right. */ * right. */
void evictionPoolPopulate(int dbid, dict *sampledict, dict *keydict, struct evictionPoolEntry *pool) { void evictionPoolPopulate(int dbid, dict *sampledict, redisDb *db, struct evictionPoolEntry *pool) {
int j, k, count; int j, k, count;
dictEntry *samples[server.maxmemory_samples]; dictEntry *samples[server.maxmemory_samples];
...@@ -161,7 +161,7 @@ void evictionPoolPopulate(int dbid, dict *sampledict, dict *keydict, struct evic ...@@ -161,7 +161,7 @@ void evictionPoolPopulate(int dbid, dict *sampledict, dict *keydict, struct evic
* dictionary (but the expires one) we need to lookup the key * dictionary (but the expires one) we need to lookup the key
* again in the key dictionary to obtain the value object. */ * again in the key dictionary to obtain the value object. */
if (server.maxmemory_policy != MAXMEMORY_VOLATILE_TTL) { if (server.maxmemory_policy != MAXMEMORY_VOLATILE_TTL) {
if (sampledict != keydict) de = dictFind(keydict, key); if (!(server.maxmemory_policy & MAXMEMORY_FLAG_ALLKEYS)) de = dictFind(getDict(db, key), key);
o = dictGetVal(de); o = dictGetVal(de);
} }
...@@ -593,9 +593,9 @@ int performEvictions(void) { ...@@ -593,9 +593,9 @@ int performEvictions(void) {
for (i = 0; i < server.dbnum; i++) { for (i = 0; i < server.dbnum; i++) {
db = server.db+i; db = server.db+i;
dict = (server.maxmemory_policy & MAXMEMORY_FLAG_ALLKEYS) ? dict = (server.maxmemory_policy & MAXMEMORY_FLAG_ALLKEYS) ?
db->dict : db->expires; getRandomDict(db, 0) : db->expires;
if ((keys = dictSize(dict)) != 0) { if ((keys = dictSize(dict)) != 0) {
evictionPoolPopulate(i, dict, db->dict, pool); evictionPoolPopulate(i, dict, db, pool);
total_keys += keys; total_keys += keys;
} }
} }
...@@ -607,7 +607,7 @@ int performEvictions(void) { ...@@ -607,7 +607,7 @@ int performEvictions(void) {
bestdbid = pool[k].dbid; bestdbid = pool[k].dbid;
if (server.maxmemory_policy & MAXMEMORY_FLAG_ALLKEYS) { if (server.maxmemory_policy & MAXMEMORY_FLAG_ALLKEYS) {
de = dictFind(server.db[bestdbid].dict, de = dictFind(getDict(&server.db[bestdbid], pool[k].key),
pool[k].key); pool[k].key);
} else { } else {
de = dictFind(server.db[bestdbid].expires, de = dictFind(server.db[bestdbid].expires,
...@@ -643,7 +643,7 @@ int performEvictions(void) { ...@@ -643,7 +643,7 @@ int performEvictions(void) {
j = (++next_db) % server.dbnum; j = (++next_db) % server.dbnum;
db = server.db+j; db = server.db+j;
dict = (server.maxmemory_policy == MAXMEMORY_ALLKEYS_RANDOM) ? dict = (server.maxmemory_policy == MAXMEMORY_ALLKEYS_RANDOM) ?
db->dict : db->expires; getRandomDict(db, 0) : db->expires;
if (dictSize(dict) != 0) { if (dictSize(dict) != 0) {
de = dictGetRandomKey(dict); de = dictGetRandomKey(dict);
bestkey = dictGetKey(de); bestkey = dictGetKey(de);
......
...@@ -2,6 +2,7 @@ ...@@ -2,6 +2,7 @@
#include "bio.h" #include "bio.h"
#include "atomicvar.h" #include "atomicvar.h"
#include "functions.h" #include "functions.h"
#include "cluster.h"
static redisAtomic size_t lazyfree_objects = 0; static redisAtomic size_t lazyfree_objects = 0;
static redisAtomic size_t lazyfreed_objects = 0; static redisAtomic size_t lazyfreed_objects = 0;
...@@ -19,14 +20,18 @@ void lazyfreeFreeObject(void *args[]) { ...@@ -19,14 +20,18 @@ void lazyfreeFreeObject(void *args[]) {
* database which was substituted with a fresh one in the main thread * database which was substituted with a fresh one in the main thread
* when the database was logically deleted. */ * when the database was logically deleted. */
void lazyfreeFreeDatabase(void *args[]) { void lazyfreeFreeDatabase(void *args[]) {
dict *ht1 = (dict *) args[0]; dict **ht1 = (dict **) args[0];
dict *ht2 = (dict *) args[1]; dict *ht2 = (dict *) args[1];
int *dictCount = (int *) args[2];
size_t numkeys = dictSize(ht1); for (int i=0; i<*dictCount; i++) {
dictRelease(ht1); size_t numkeys = dictSize(ht1[i]);
dictRelease(ht1[i]);
atomicDecr(lazyfree_objects,numkeys);
atomicIncr(lazyfreed_objects,numkeys);
}
zfree(ht1);
zfree(dictCount);
dictRelease(ht2); dictRelease(ht2);
atomicDecr(lazyfree_objects,numkeys);
atomicIncr(lazyfreed_objects,numkeys);
} }
/* Release the key tracking table. */ /* Release the key tracking table. */
...@@ -170,15 +175,20 @@ void freeObjAsync(robj *key, robj *obj, int dbid) { ...@@ -170,15 +175,20 @@ void freeObjAsync(robj *key, robj *obj, int dbid) {
} }
} }
unsigned long long int dbSize(const redisDb *db);
/* Empty a Redis DB asynchronously. What the function does actually is to /* Empty a Redis DB asynchronously. What the function does actually is to
* create a new empty set of hash tables and scheduling the old ones for * create a new empty set of hash tables and scheduling the old ones for
* lazy freeing. */ * lazy freeing. */
void emptyDbAsync(redisDb *db) { void emptyDbAsync(redisDb *db) {
dict *oldht1 = db->dict, *oldht2 = db->expires; dict **oldDict = db->dict;
db->dict = dictCreate(&dbDictType); dict *oldExpires = db->expires;
atomicIncr(lazyfree_objects,dbSize(db));
db->dict = dictCreateMultiple(&dbDictType, db->dict_count);
db->expires = dictCreate(&dbExpiresDictType); db->expires = dictCreate(&dbExpiresDictType);
atomicIncr(lazyfree_objects,dictSize(oldht1)); int *count = zmalloc(sizeof(int));
bioCreateLazyFreeJob(lazyfreeFreeDatabase,2,oldht1,oldht2); *count = db->dict_count;
bioCreateLazyFreeJob(lazyfreeFreeDatabase, 3, oldDict, oldExpires, count);
} }
/* Free the key tracking table. /* Free the key tracking table.
......
...@@ -4007,7 +4007,7 @@ void RM_ResetDataset(int restart_aof, int async) { ...@@ -4007,7 +4007,7 @@ void RM_ResetDataset(int restart_aof, int async) {
   
/* Returns the number of keys in the current db. */ /* Returns the number of keys in the current db. */
unsigned long long RM_DbSize(RedisModuleCtx *ctx) { unsigned long long RM_DbSize(RedisModuleCtx *ctx) {
return dictSize(ctx->client->db->dict); return dbSize(ctx->client->db);
} }
   
/* Returns a name of a random key, or NULL if current db is empty. */ /* Returns a name of a random key, or NULL if current db is empty. */
...@@ -10365,7 +10365,7 @@ int RM_Scan(RedisModuleCtx *ctx, RedisModuleScanCursor *cursor, RedisModuleScanC ...@@ -10365,7 +10365,7 @@ int RM_Scan(RedisModuleCtx *ctx, RedisModuleScanCursor *cursor, RedisModuleScanC
} }
int ret = 1; int ret = 1;
ScanCBData data = { ctx, privdata, fn }; ScanCBData data = { ctx, privdata, fn };
cursor->cursor = dictScan(ctx->client->db->dict, cursor->cursor, moduleScanCallback, &data); // cursor->cursor = dictScan(ctx->client->db->dict, cursor->cursor, moduleScanCallback, &data); FIXME (vitarb) https://sim.amazon.com/issues/ELMO-63799
if (cursor->cursor == 0) { if (cursor->cursor == 0) {
cursor->done = 1; cursor->done = 1;
ret = 0; ret = 0;
......
...@@ -394,7 +394,7 @@ void touchWatchedKey(redisDb *db, robj *key) { ...@@ -394,7 +394,7 @@ void touchWatchedKey(redisDb *db, robj *key) {
/* The key was already expired when WATCH was called. */ /* The key was already expired when WATCH was called. */
if (db == wk->db && if (db == wk->db &&
equalStringObjects(key, wk->key) && equalStringObjects(key, wk->key) &&
dictFind(db->dict, key->ptr) == NULL) dictFind(getDict(db, key->ptr), key->ptr) == NULL)
{ {
/* Already expired key is deleted, so logically no change. Clear /* Already expired key is deleted, so logically no change. Clear
* the flag. Deleted keys are not flagged as expired. */ * the flag. Deleted keys are not flagged as expired. */
...@@ -432,9 +432,9 @@ void touchAllWatchedKeysInDb(redisDb *emptied, redisDb *replaced_with) { ...@@ -432,9 +432,9 @@ void touchAllWatchedKeysInDb(redisDb *emptied, redisDb *replaced_with) {
dictIterator *di = dictGetSafeIterator(emptied->watched_keys); dictIterator *di = dictGetSafeIterator(emptied->watched_keys);
while((de = dictNext(di)) != NULL) { while((de = dictNext(di)) != NULL) {
robj *key = dictGetKey(de); robj *key = dictGetKey(de);
int exists_in_emptied = dictFind(emptied->dict, key->ptr) != NULL; int exists_in_emptied = dictFind(getDict(emptied, key->ptr), key->ptr) != NULL;
if (exists_in_emptied || if (exists_in_emptied ||
(replaced_with && dictFind(replaced_with->dict, key->ptr))) (replaced_with && dictFind(getDict(replaced_with, key->ptr), key->ptr)))
{ {
list *clients = dictGetVal(de); list *clients = dictGetVal(de);
if (!clients) continue; if (!clients) continue;
...@@ -442,7 +442,7 @@ void touchAllWatchedKeysInDb(redisDb *emptied, redisDb *replaced_with) { ...@@ -442,7 +442,7 @@ void touchAllWatchedKeysInDb(redisDb *emptied, redisDb *replaced_with) {
while((ln = listNext(&li))) { while((ln = listNext(&li))) {
watchedKey *wk = redis_member2struct(watchedKey, node, ln); watchedKey *wk = redis_member2struct(watchedKey, node, ln);
if (wk->expired) { if (wk->expired) {
if (!replaced_with || !dictFind(replaced_with->dict, key->ptr)) { if (!replaced_with || !dictFind(getDict(replaced_with, key->ptr), key->ptr)) {
/* Expired key now deleted. No logical change. Clear the /* Expired key now deleted. No logical change. Clear the
* flag. Deleted keys are not flagged as expired. */ * flag. Deleted keys are not flagged as expired. */
wk->expired = 0; wk->expired = 0;
......
...@@ -1236,15 +1236,16 @@ struct redisMemOverhead *getMemoryOverheadData(void) { ...@@ -1236,15 +1236,16 @@ struct redisMemOverhead *getMemoryOverheadData(void) {
for (j = 0; j < server.dbnum; j++) { for (j = 0; j < server.dbnum; j++) {
redisDb *db = server.db+j; redisDb *db = server.db+j;
long long keyscount = dictSize(db->dict); long long keyscount = dbSize(db);
if (keyscount==0) continue; if (keyscount==0) continue;
mh->total_keys += keyscount; mh->total_keys += keyscount;
mh->db = zrealloc(mh->db,sizeof(mh->db[0])*(mh->num_dbs+1)); mh->db = zrealloc(mh->db,sizeof(mh->db[0])*(mh->num_dbs+1));
mh->db[mh->num_dbs].dbid = j; mh->db[mh->num_dbs].dbid = j;
mem = dictMemUsage(db->dict) + mem = keyscount * sizeof(dictEntry) +
dictSize(db->dict) * sizeof(robj); dbSlots(db) * sizeof(dictEntry*) +
keyscount * sizeof(robj);
mh->db[mh->num_dbs].overhead_ht_main = mem; mh->db[mh->num_dbs].overhead_ht_main = mem;
mem_total+=mem; mem_total+=mem;
...@@ -1252,12 +1253,6 @@ struct redisMemOverhead *getMemoryOverheadData(void) { ...@@ -1252,12 +1253,6 @@ struct redisMemOverhead *getMemoryOverheadData(void) {
mh->db[mh->num_dbs].overhead_ht_expires = mem; mh->db[mh->num_dbs].overhead_ht_expires = mem;
mem_total+=mem; mem_total+=mem;
/* Account for the slot to keys map in cluster mode */
mem = dictSize(db->dict) * dictEntryMetadataSize(db->dict) +
dictMetadataSize(db->dict);
mh->db[mh->num_dbs].overhead_ht_slot_to_keys = mem;
mem_total+=mem;
mh->num_dbs++; mh->num_dbs++;
} }
...@@ -1541,14 +1536,13 @@ NULL ...@@ -1541,14 +1536,13 @@ NULL
return; return;
} }
} }
if ((de = dictFind(c->db->dict,c->argv[2]->ptr)) == NULL) { if ((de = dictFind(getDict(c->db, c->argv[2]->ptr),c->argv[2]->ptr)) == NULL) {
addReplyNull(c); addReplyNull(c);
return; return;
} }
size_t usage = objectComputeSize(c->argv[2],dictGetVal(de),samples,c->db->id); size_t usage = objectComputeSize(c->argv[2],dictGetVal(de),samples,c->db->id);
usage += sdsZmallocSize(dictGetKey(de)); usage += sdsZmallocSize(dictGetKey(de));
usage += dictEntryMemUsage(); usage += dictEntryMemUsage();
usage += dictMetadataSize(c->db->dict);
addReplyLongLong(c,usage); addReplyLongLong(c,usage);
} else if (!strcasecmp(c->argv[1]->ptr,"stats") && c->argc == 2) { } else if (!strcasecmp(c->argv[1]->ptr,"stats") && c->argc == 2) {
struct redisMemOverhead *mh = getMemoryOverheadData(); struct redisMemOverhead *mh = getMemoryOverheadData();
......
...@@ -1301,9 +1301,8 @@ ssize_t rdbSaveDb(rio *rdb, int dbid, int rdbflags, long *key_counter) { ...@@ -1301,9 +1301,8 @@ ssize_t rdbSaveDb(rio *rdb, int dbid, int rdbflags, long *key_counter) {
char *pname = (rdbflags & RDBFLAGS_AOF_PREAMBLE) ? "AOF rewrite" : "RDB"; char *pname = (rdbflags & RDBFLAGS_AOF_PREAMBLE) ? "AOF rewrite" : "RDB";
redisDb *db = server.db + dbid; redisDb *db = server.db + dbid;
dict *d = db->dict; unsigned long long int db_size = dbSize(db);
if (dictSize(d) == 0) return 0; if (db_size == 0) return 0;
di = dictGetSafeIterator(d);
/* Write the SELECT DB opcode */ /* Write the SELECT DB opcode */
if ((res = rdbSaveType(rdb,RDB_OPCODE_SELECTDB)) < 0) goto werr; if ((res = rdbSaveType(rdb,RDB_OPCODE_SELECTDB)) < 0) goto werr;
...@@ -1312,8 +1311,7 @@ ssize_t rdbSaveDb(rio *rdb, int dbid, int rdbflags, long *key_counter) { ...@@ -1312,8 +1311,7 @@ ssize_t rdbSaveDb(rio *rdb, int dbid, int rdbflags, long *key_counter) {
written += res; written += res;
/* Write the RESIZE DB opcode. */ /* Write the RESIZE DB opcode. */
uint64_t db_size, expires_size; uint64_t expires_size;
db_size = dictSize(db->dict);
expires_size = dictSize(db->expires); expires_size = dictSize(db->expires);
if ((res = rdbSaveType(rdb,RDB_OPCODE_RESIZEDB)) < 0) goto werr; if ((res = rdbSaveType(rdb,RDB_OPCODE_RESIZEDB)) < 0) goto werr;
written += res; written += res;
...@@ -1322,37 +1320,42 @@ ssize_t rdbSaveDb(rio *rdb, int dbid, int rdbflags, long *key_counter) { ...@@ -1322,37 +1320,42 @@ ssize_t rdbSaveDb(rio *rdb, int dbid, int rdbflags, long *key_counter) {
if ((res = rdbSaveLen(rdb,expires_size)) < 0) goto werr; if ((res = rdbSaveLen(rdb,expires_size)) < 0) goto werr;
written += res; written += res;
/* Iterate this DB writing every entry */ for (int i = 0; i < db->dict_count; i++) {
while((de = dictNext(di)) != NULL) { dict *d = db->dict[i];
sds keystr = dictGetKey(de); if (!dictSize(d)) continue;
robj key, *o = dictGetVal(de); di = dictGetSafeIterator(d);
long long expire; /* Iterate this DB writing every entry */
size_t rdb_bytes_before_key = rdb->processed_bytes; while ((de = dictNext(di)) != NULL) {
sds keystr = dictGetKey(de);
initStaticStringObject(key,keystr); robj key, *o = dictGetVal(de);
expire = getExpire(db,&key); long long expire;
if ((res = rdbSaveKeyValuePair(rdb, &key, o, expire, dbid)) < 0) goto werr; size_t rdb_bytes_before_key = rdb->processed_bytes;
written += res;
initStaticStringObject(key, keystr);
/* In fork child process, we can try to release memory back to the expire = getExpire(db, &key);
* OS and possibly avoid or decrease COW. We give the dismiss if ((res = rdbSaveKeyValuePair(rdb, &key, o, expire, dbid)) < 0) goto werr;
* mechanism a hint about an estimated size of the object we stored. */ written += res;
size_t dump_size = rdb->processed_bytes - rdb_bytes_before_key;
if (server.in_fork_child) dismissObject(o, dump_size); /* In fork child process, we can try to release memory back to the
* OS and possibly avoid or decrease COW. We give the dismiss
/* Update child info every 1 second (approximately). * mechanism a hint about an estimated size of the object we stored. */
* in order to avoid calling mstime() on each iteration, we will size_t dump_size = rdb->processed_bytes - rdb_bytes_before_key;
* check the diff every 1024 keys */ if (server.in_fork_child) dismissObject(o, dump_size);
if (((*key_counter)++ & 1023) == 0) {
long long now = mstime(); /* Update child info every 1 second (approximately).
if (now - info_updated_time >= 1000) { * in order to avoid calling mstime() on each iteration, we will
sendChildInfo(CHILD_INFO_TYPE_CURRENT_INFO, *key_counter, pname); * check the diff every 1024 keys */
info_updated_time = now; if (((*key_counter)++ & 1023) == 0) {
long long now = mstime();
if (now - info_updated_time >= 1000) {
sendChildInfo(CHILD_INFO_TYPE_CURRENT_INFO, *key_counter, pname);
info_updated_time = now;
}
} }
} }
}
dictReleaseIterator(di); dictReleaseIterator(di);
}
return written; return written;
werr: werr:
...@@ -3063,7 +3066,7 @@ int rdbLoadRioWithLoadingCtx(rio *rdb, int rdbflags, rdbSaveInfo *rsi, rdbLoadin ...@@ -3063,7 +3066,7 @@ int rdbLoadRioWithLoadingCtx(rio *rdb, int rdbflags, rdbSaveInfo *rsi, rdbLoadin
goto eoferr; goto eoferr;
if ((expires_size = rdbLoadLen(rdb,NULL)) == RDB_LENERR) if ((expires_size = rdbLoadLen(rdb,NULL)) == RDB_LENERR)
goto eoferr; goto eoferr;
dictExpand(db->dict,db_size); // dictExpand(db->dict,db_size); // FIXME (vitarb) potentially approximate number of keys per slot in cluster mode https://sim.amazon.com/issues/ELMO-63800
dictExpand(db->expires,expires_size); dictExpand(db->expires,expires_size);
continue; /* Read next opcode. */ continue; /* Read next opcode. */
} else if (type == RDB_OPCODE_AUX) { } else if (type == RDB_OPCODE_AUX) {
......
...@@ -392,27 +392,6 @@ int dictExpandAllowed(size_t moreMem, double usedRatio) { ...@@ -392,27 +392,6 @@ int dictExpandAllowed(size_t moreMem, double usedRatio) {
} }
} }
/* Returns the size of the DB dict entry metadata in bytes. In cluster mode, the
* metadata is used for constructing a doubly linked list of the dict entries
* belonging to the same cluster slot. See the Slot to Key API in cluster.c. */
size_t dbDictEntryMetadataSize(dict *d) {
UNUSED(d);
/* NOTICE: this also affects overhead_ht_slot_to_keys in getMemoryOverheadData.
* If we ever add non-cluster related data here, that code must be modified too. */
return server.cluster_enabled ? sizeof(clusterDictEntryMetadata) : 0;
}
/* Returns the size of the DB dict metadata in bytes. In cluster mode, we store
* a pointer to the db in the main db dict, used for updating the slot-to-key
* mapping when a dictEntry is reallocated. */
size_t dbDictMetadataSize(void) {
return server.cluster_enabled ? sizeof(clusterDictMetadata) : 0;
}
void dbDictAfterReplaceEntry(dict *d, dictEntry *de) {
if (server.cluster_enabled) slotToKeyReplaceEntry(d, de);
}
/* Generic hash table type where keys are Redis Objects, Values /* Generic hash table type where keys are Redis Objects, Values
* dummy pointers. */ * dummy pointers. */
dictType objectKeyPointerValueDictType = { dictType objectKeyPointerValueDictType = {
...@@ -468,9 +447,6 @@ dictType dbDictType = { ...@@ -468,9 +447,6 @@ dictType dbDictType = {
dictSdsDestructor, /* key destructor */ dictSdsDestructor, /* key destructor */
dictObjectDestructor, /* val destructor */ dictObjectDestructor, /* val destructor */
dictExpandAllowed, /* allow to expand */ dictExpandAllowed, /* allow to expand */
.dictEntryMetadataBytes = dbDictEntryMetadataSize,
.dictMetadataBytes = dbDictMetadataSize,
.afterReplaceEntry = dbDictAfterReplaceEntry
}; };
/* Db->expires */ /* Db->expires */
...@@ -601,8 +577,11 @@ int htNeedsResize(dict *dict) { ...@@ -601,8 +577,11 @@ int htNeedsResize(dict *dict) {
/* If the percentage of used slots in the HT reaches HASHTABLE_MIN_FILL /* If the percentage of used slots in the HT reaches HASHTABLE_MIN_FILL
* we resize the hash table to save memory */ * we resize the hash table to save memory */
void tryResizeHashTables(int dbid) { void tryResizeHashTables(int dbid) {
if (htNeedsResize(server.db[dbid].dict)) for (int i = 0; i < server.db[dbid].dict_count; i++) {
dictResize(server.db[dbid].dict); dict *d = server.db[dbid].dict[i];
if (htNeedsResize(d))
dictResize(d);
}
if (htNeedsResize(server.db[dbid].expires)) if (htNeedsResize(server.db[dbid].expires))
dictResize(server.db[dbid].expires); dictResize(server.db[dbid].expires);
} }
...@@ -616,8 +595,9 @@ void tryResizeHashTables(int dbid) { ...@@ -616,8 +595,9 @@ void tryResizeHashTables(int dbid) {
* is returned. */ * is returned. */
int incrementallyRehash(int dbid) { int incrementallyRehash(int dbid) {
/* Keys dictionary */ /* Keys dictionary */
if (dictIsRehashing(server.db[dbid].dict)) { dict *d = getRandomDict(&server.db[dbid], 1);
dictRehashMilliseconds(server.db[dbid].dict,1); if (dictIsRehashing(d)) {
dictRehashMilliseconds(d, 1);
return 1; /* already used our millisecond for this loop... */ return 1; /* already used our millisecond for this loop... */
} }
/* Expires */ /* Expires */
...@@ -1324,8 +1304,8 @@ int serverCron(struct aeEventLoop *eventLoop, long long id, void *clientData) { ...@@ -1324,8 +1304,8 @@ int serverCron(struct aeEventLoop *eventLoop, long long id, void *clientData) {
for (j = 0; j < server.dbnum; j++) { for (j = 0; j < server.dbnum; j++) {
long long size, used, vkeys; long long size, used, vkeys;
size = dictSlots(server.db[j].dict); size = dbSlots(&server.db[j]);
used = dictSize(server.db[j].dict); used = dbSize(&server.db[j]);
vkeys = dictSize(server.db[j].expires); vkeys = dictSize(server.db[j].expires);
if (used || vkeys) { if (used || vkeys) {
serverLog(LL_VERBOSE,"DB %d: %lld keys (%lld volatile) in %lld slots HT.",j,used,vkeys,size); serverLog(LL_VERBOSE,"DB %d: %lld keys (%lld volatile) in %lld slots HT.",j,used,vkeys,size);
...@@ -2566,7 +2546,8 @@ void initServer(void) { ...@@ -2566,7 +2546,8 @@ void initServer(void) {
/* Create the Redis databases, and initialize other internal state. */ /* Create the Redis databases, and initialize other internal state. */
for (j = 0; j < server.dbnum; j++) { for (j = 0; j < server.dbnum; j++) {
server.db[j].dict = dictCreate(&dbDictType); int slotCount = (server.cluster_enabled) ? CLUSTER_SLOTS : 1;
server.db[j].dict = dictCreateMultiple(&dbDictType, slotCount);
server.db[j].expires = dictCreate(&dbExpiresDictType); server.db[j].expires = dictCreate(&dbExpiresDictType);
server.db[j].expires_cursor = 0; server.db[j].expires_cursor = 0;
server.db[j].blocking_keys = dictCreate(&keylistDictType); server.db[j].blocking_keys = dictCreate(&keylistDictType);
...@@ -2576,7 +2557,7 @@ void initServer(void) { ...@@ -2576,7 +2557,7 @@ void initServer(void) {
server.db[j].id = j; server.db[j].id = j;
server.db[j].avg_ttl = 0; server.db[j].avg_ttl = 0;
server.db[j].defrag_later = listCreate(); server.db[j].defrag_later = listCreate();
server.db[j].slots_to_keys = NULL; /* Set by clusterInit later on if necessary. */ server.db[j].dict_count = slotCount;
listSetFreeMethod(server.db[j].defrag_later,(void (*)(void*))sdsfree); listSetFreeMethod(server.db[j].defrag_later,(void (*)(void*))sdsfree);
} }
evictionPoolAlloc(); /* Initialize the LRU keys pool. */ evictionPoolAlloc(); /* Initialize the LRU keys pool. */
...@@ -6147,7 +6128,7 @@ sds genRedisInfoString(dict *section_dict, int all_sections, int everything) { ...@@ -6147,7 +6128,7 @@ sds genRedisInfoString(dict *section_dict, int all_sections, int everything) {
for (j = 0; j < server.dbnum; j++) { for (j = 0; j < server.dbnum; j++) {
long long keys, vkeys; long long keys, vkeys;
keys = dictSize(server.db[j].dict); keys = dbSize(&server.db[j]);
vkeys = dictSize(server.db[j].expires); vkeys = dictSize(server.db[j].expires);
if (keys || vkeys) { if (keys || vkeys) {
info = sdscatprintf(info, info = sdscatprintf(info,
......
...@@ -939,14 +939,11 @@ typedef struct replBufBlock { ...@@ -939,14 +939,11 @@ typedef struct replBufBlock {
char buf[]; char buf[];
} replBufBlock; } replBufBlock;
/* Opaque type for the Slot to Key API. */
typedef struct clusterSlotToKeyMapping clusterSlotToKeyMapping;
/* Redis database representation. There are multiple databases identified /* Redis database representation. There are multiple databases identified
* by integers from 0 (the default database) up to the max configured * by integers from 0 (the default database) up to the max configured
* database. The database number is the 'id' field in the structure. */ * database. The database number is the 'id' field in the structure. */
typedef struct redisDb { typedef struct redisDb {
dict *dict; /* The keyspace for this DB */ dict **dict; /* The keyspace for this DB */
dict *expires; /* Timeout of keys with a timeout set */ dict *expires; /* Timeout of keys with a timeout set */
dict *blocking_keys; /* Keys with clients waiting for data (BLPOP)*/ dict *blocking_keys; /* Keys with clients waiting for data (BLPOP)*/
dict *blocking_keys_unblock_on_nokey; /* Keys with clients waiting for dict *blocking_keys_unblock_on_nokey; /* Keys with clients waiting for
...@@ -958,7 +955,7 @@ typedef struct redisDb { ...@@ -958,7 +955,7 @@ typedef struct redisDb {
long long avg_ttl; /* Average TTL, just for stats */ long long avg_ttl; /* Average TTL, just for stats */
unsigned long expires_cursor; /* Cursor of the active expire cycle. */ unsigned long expires_cursor; /* Cursor of the active expire cycle. */
list *defrag_later; /* List of key names to attempt to defrag one by one, gradually. */ list *defrag_later; /* List of key names to attempt to defrag one by one, gradually. */
clusterSlotToKeyMapping *slots_to_keys; /* Array of slots to keys. Only used in cluster mode (db 0). */ int dict_count; /* Indicates total number of dictionaires owned by this DB, 1 dict per slot in cluster mode. */
} redisDb; } redisDb;
/* forward declaration for functions ctx */ /* forward declaration for functions ctx */
...@@ -3006,6 +3003,10 @@ void dismissMemoryInChild(void); ...@@ -3006,6 +3003,10 @@ void dismissMemoryInChild(void);
#define RESTART_SERVER_GRACEFULLY (1<<0) /* Do proper shutdown. */ #define RESTART_SERVER_GRACEFULLY (1<<0) /* Do proper shutdown. */
#define RESTART_SERVER_CONFIG_REWRITE (1<<1) /* CONFIG REWRITE before restart.*/ #define RESTART_SERVER_CONFIG_REWRITE (1<<1) /* CONFIG REWRITE before restart.*/
int restartServer(int flags, mstime_t delay); int restartServer(int flags, mstime_t delay);
unsigned long long int dbSize(const redisDb *db);
dict *getDict(redisDb *db, sds key);
dict *getRandomDict(redisDb *db, int shouldBeRehashing);
unsigned long dbSlots(const redisDb *db);
/* Set data type */ /* Set data type */
robj *setTypeCreate(sds value); robj *setTypeCreate(sds value);
...@@ -3090,6 +3091,7 @@ sds keyspaceEventsFlagsToString(int flags); ...@@ -3090,6 +3091,7 @@ sds keyspaceEventsFlagsToString(int flags);
#define MEMORY_CONFIG (1<<0) /* Indicates if this value can be loaded as a memory value */ #define MEMORY_CONFIG (1<<0) /* Indicates if this value can be loaded as a memory value */
#define PERCENT_CONFIG (1<<1) /* Indicates if this value can be loaded as a percent (and stored as a negative int) */ #define PERCENT_CONFIG (1<<1) /* Indicates if this value can be loaded as a percent (and stored as a negative int) */
#define OCTAL_CONFIG (1<<2) /* This value uses octal representation */ #define OCTAL_CONFIG (1<<2) /* This value uses octal representation */
#define SLOT_MASK_SHIFT 48
/* Enum Configs contain an array of configEnum objects that match a string with an integer. */ /* Enum Configs contain an array of configEnum objects that match a string with an integer. */
typedef struct configEnum { typedef struct configEnum {
...@@ -3195,8 +3197,8 @@ void discardTempDb(redisDb *tempDb, void(callback)(dict*)); ...@@ -3195,8 +3197,8 @@ void discardTempDb(redisDb *tempDb, void(callback)(dict*));
int selectDb(client *c, int id); int selectDb(client *c, int id);
void signalModifiedKey(client *c, redisDb *db, robj *key); void signalModifiedKey(client *c, redisDb *db, robj *key);
void signalFlushedDb(int dbid, int async); void signalFlushedDb(int dbid, int async);
void scanGenericCommand(client *c, robj *o, unsigned long cursor); void scanGenericCommand(client *c, robj *o, unsigned long long cursor);
int parseScanCursorOrReply(client *c, robj *o, unsigned long *cursor); int parseScanCursorOrReply(client *c, robj *o, unsigned long long *cursor);
int dbAsyncDelete(redisDb *db, robj *key); int dbAsyncDelete(redisDb *db, robj *key);
void emptyDbAsync(redisDb *db); void emptyDbAsync(redisDb *db);
size_t lazyfreeGetPendingObjectsCount(void); size_t lazyfreeGetPendingObjectsCount(void);
......
...@@ -878,7 +878,7 @@ void hexistsCommand(client *c) { ...@@ -878,7 +878,7 @@ void hexistsCommand(client *c) {
void hscanCommand(client *c) { void hscanCommand(client *c) {
robj *o; robj *o;
unsigned long cursor; unsigned long long cursor;
if (parseScanCursorOrReply(c,c->argv[2],&cursor) == C_ERR) return; if (parseScanCursorOrReply(c,c->argv[2],&cursor) == C_ERR) return;
if ((o = lookupKeyReadOrReply(c,c->argv[1],shared.emptyscan)) == NULL || if ((o = lookupKeyReadOrReply(c,c->argv[1],shared.emptyscan)) == NULL ||
......
...@@ -1624,7 +1624,7 @@ void sdiffstoreCommand(client *c) { ...@@ -1624,7 +1624,7 @@ void sdiffstoreCommand(client *c) {
void sscanCommand(client *c) { void sscanCommand(client *c) {
robj *set; robj *set;
unsigned long cursor; unsigned long long cursor;
if (parseScanCursorOrReply(c,c->argv[2],&cursor) == C_ERR) return; if (parseScanCursorOrReply(c,c->argv[2],&cursor) == C_ERR) return;
if ((set = lookupKeyReadOrReply(c,c->argv[1],shared.emptyscan)) == NULL || if ((set = lookupKeyReadOrReply(c,c->argv[1],shared.emptyscan)) == NULL ||
......
...@@ -3816,7 +3816,7 @@ void zrevrankCommand(client *c) { ...@@ -3816,7 +3816,7 @@ void zrevrankCommand(client *c) {
void zscanCommand(client *c) { void zscanCommand(client *c) {
robj *o; robj *o;
unsigned long cursor; unsigned long long cursor;
if (parseScanCursorOrReply(c,c->argv[2],&cursor) == C_ERR) return; if (parseScanCursorOrReply(c,c->argv[2],&cursor) == C_ERR) return;
if ((o = lookupKeyReadOrReply(c,c->argv[1],shared.emptyscan)) == NULL || if ((o = lookupKeyReadOrReply(c,c->argv[1],shared.emptyscan)) == NULL ||
......
Markdown is supported
0% or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment