diff --git a/src/aof.c b/src/aof.c index e0ca6fbb61..c04d97dc64 100644 --- a/src/aof.c +++ b/src/aof.c @@ -2190,7 +2190,7 @@ static int rewriteFunctions(rio *aof) { } int rewriteAppendOnlyFileRio(rio *aof) { - dictEntry *de; + valkey *o; int j; long key_count = 0; long long updated_time = 0; @@ -2219,17 +2219,16 @@ int rewriteAppendOnlyFileRio(rio *aof) { kvs_it = kvstoreIteratorInit(db->keys); /* Iterate this DB writing every entry */ - while ((de = kvstoreIteratorNext(kvs_it)) != NULL) { + while (kvstoreIteratorNext(kvs_it, (void **)&o)) { sds keystr; - robj key, *o; + robj key; long long expiretime; size_t aof_bytes_before_key = aof->processed_bytes; - keystr = dictGetKey(de); - o = dictGetVal(de); + keystr = objectGetKey(o); initStaticStringObject(key, keystr); - expiretime = getExpire(db, &key); + expiretime = objectGetExpire(o); /* Save the key and associated value */ if (o->type == OBJ_STRING) { diff --git a/src/bitops.c b/src/bitops.c index 10c383b270..e4aba7558f 100644 --- a/src/bitops.c +++ b/src/bitops.c @@ -486,7 +486,7 @@ robj *lookupStringForBitCommand(client *c, uint64_t maxbit, int *dirty) { if (o == NULL) { o = createObject(OBJ_STRING, sdsnewlen(NULL, byte + 1)); - dbAdd(c->db, c->argv[1], o); + o = dbAdd(c->db, c->argv[1], o); if (dirty) *dirty = 1; } else { o = dbUnshareStringValue(c->db, c->argv[1], o); @@ -772,9 +772,8 @@ void bitopCommand(client *c) { /* Store the computed value into the target key */ if (maxlen) { o = createObject(OBJ_STRING, res); - setKey(c, c->db, targetkey, o, 0); + o = setKey(c, c->db, targetkey, o, 0); notifyKeyspaceEvent(NOTIFY_STRING, "set", targetkey, c->db->id); - decrRefCount(o); server.dirty++; } else if (dbDelete(c->db, targetkey)) { signalModifiedKey(c, c->db, targetkey); diff --git a/src/cluster.c b/src/cluster.c index 9154ac3207..bb0ce662f5 100644 --- a/src/cluster.c +++ b/src/cluster.c @@ -276,9 +276,9 @@ void restoreCommand(client *c) { } /* Create the key and set the TTL if any */ - dbAdd(c->db, key, obj); + obj = dbAdd(c->db, key, obj); if (ttl) { - setExpire(c, c->db, key, ttl); + obj = setExpire(c, c->db, key, ttl); if (!absttl) { /* Propagate TTL as absolute timestamp */ robj *ttl_obj = createStringObjectFromLongLong(ttl); @@ -811,7 +811,7 @@ static int shouldReturnTlsInfo(void) { } unsigned int countKeysInSlot(unsigned int slot) { - return kvstoreDictSize(server.db->keys, slot); + return kvstoreHashtableSize(server.db->keys, slot); } void clusterCommandHelp(client *c) { @@ -908,16 +908,15 @@ void clusterCommand(client *c) { unsigned int keys_in_slot = countKeysInSlot(slot); unsigned int numkeys = maxkeys > keys_in_slot ? keys_in_slot : maxkeys; addReplyArrayLen(c, numkeys); - kvstoreDictIterator *kvs_di = NULL; - dictEntry *de = NULL; - kvs_di = kvstoreGetDictIterator(server.db->keys, slot); + kvstoreHashtableIterator *kvs_di = NULL; + valkey *valkey = NULL; + kvs_di = kvstoreGetHashtableIterator(server.db->keys, slot); for (unsigned int i = 0; i < numkeys; i++) { - de = kvstoreDictIteratorNext(kvs_di); - serverAssert(de != NULL); - sds sdskey = dictGetKey(de); + serverAssert(kvstoreHashtableIteratorNext(kvs_di, (void **)&valkey)); + sds sdskey = objectGetKey(valkey); addReplyBulkCBuffer(c, sdskey, sdslen(sdskey)); } - kvstoreReleaseDictIterator(kvs_di); + kvstoreReleaseHashtableIterator(kvs_di); } else if ((!strcasecmp(c->argv[1]->ptr, "slaves") || !strcasecmp(c->argv[1]->ptr, "replicas")) && c->argc == 3) { /* CLUSTER REPLICAS */ clusterNode *n = clusterLookupNode(c->argv[2]->ptr, sdslen(c->argv[2]->ptr)); diff --git a/src/cluster_legacy.c b/src/cluster_legacy.c index 69af65f1e8..e4b2be422d 100644 --- a/src/cluster_legacy.c +++ b/src/cluster_legacy.c @@ -6089,12 +6089,12 @@ unsigned int delKeysInSlot(unsigned int hashslot) { server.server_del_keys_in_slot = 1; unsigned int j = 0; - kvstoreDictIterator *kvs_di = NULL; - dictEntry *de = NULL; - kvs_di = kvstoreGetDictSafeIterator(server.db->keys, hashslot); - while ((de = kvstoreDictIteratorNext(kvs_di)) != NULL) { + kvstoreHashtableIterator *kvs_di = NULL; + valkey *valkey = NULL; + kvs_di = kvstoreGetHashtableSafeIterator(server.db->keys, hashslot); + while (kvstoreHashtableIteratorNext(kvs_di, (void **)&valkey)) { enterExecutionUnit(1, 0); - sds sdskey = dictGetKey(de); + sds sdskey = objectGetKey(valkey); robj *key = createStringObject(sdskey, sdslen(sdskey)); dbDelete(&server.db[0], key); propagateDeletion(&server.db[0], key, server.lazyfree_lazy_server_del); @@ -6109,7 +6109,7 @@ unsigned int delKeysInSlot(unsigned int hashslot) { j++; server.dirty++; } - kvstoreReleaseDictIterator(kvs_di); + kvstoreReleaseHashtableIterator(kvs_di); server.server_del_keys_in_slot = 0; serverAssert(server.execution_nesting == 0); @@ -6118,7 +6118,7 @@ unsigned int delKeysInSlot(unsigned int hashslot) { /* Get the count of the channels for a given slot. */ unsigned int countChannelsInSlot(unsigned int hashslot) { - return kvstoreDictSize(server.pubsubshard_channels, hashslot); + return kvstoreHashtableSize(server.pubsubshard_channels, hashslot); } clusterNode *getMyClusterNode(void) { diff --git a/src/db.c b/src/db.c index b59c7727b2..7869d013ca 100644 --- a/src/db.c +++ b/src/db.c @@ -56,9 +56,9 @@ keyStatus expireIfNeededWithDictIndex(serverDb *db, robj *key, int flags, int di keyStatus expireIfNeeded(serverDb *db, robj *key, int flags); int keyIsExpiredWithDictIndex(serverDb *db, robj *key, int dict_index); int keyIsExpired(serverDb *db, robj *key); -static void dbSetValue(serverDb *db, robj *key, robj *val, int overwrite, dictEntry *de); +static valkey *dbSetValue(serverDb *db, robj *key, robj *val, int overwrite, void **oldref); static int getKVStoreIndexForKey(sds key); -dictEntry *dbFindExpiresWithDictIndex(serverDb *db, void *key, int dict_index); +valkey *dbFindExpiresWithDictIndex(serverDb *db, sds key, int dict_index); /* Update LFU when an object is accessed. * Firstly, decrement the counter if the decrement time is reached. @@ -97,10 +97,8 @@ void updateLFU(robj *val) { * expired on replicas even if the primary is lagging expiring our key via DELs * in the replication link. */ robj *lookupKey(serverDb *db, robj *key, int flags) { - dictEntry *de = dbFind(db, key->ptr); - robj *val = NULL; - if (de) { - val = dictGetVal(de); + valkey *val = dbFind(db, key->ptr); + if (val) { /* Forcing deletion of expired keys on a replica makes the replica * inconsistent with the primary. We forbid it on readonly replicas, but * we have to allow it on writable replicas to make write commands @@ -113,7 +111,9 @@ robj *lookupKey(serverDb *db, robj *key, int flags) { int expire_flags = 0; if (flags & LOOKUP_WRITE && !is_ro_replica) expire_flags |= EXPIRE_FORCE_DELETE_EXPIRED; if (flags & LOOKUP_NOEXPIRE) expire_flags |= EXPIRE_AVOID_DELETE_EXPIRED; - if (expireIfNeeded(db, key, expire_flags) != KEY_VALID) { + /* TODO: Use a better design for expireIfNeeded, now that the expire is + * stored in the value, for example passing val to expireIfNeeded. */ + if (objectGetExpire(val) != -1 && expireIfNeeded(db, key, expire_flags) != KEY_VALID) { /* The key is no longer valid. */ val = NULL; } @@ -127,10 +127,8 @@ robj *lookupKey(serverDb *db, robj *key, int flags) { server.current_client->cmd->proc != touchCommand) flags |= LOOKUP_NOTOUCH; if (!hasActiveChildProcess() && !(flags & LOOKUP_NOTOUCH)) { - if (!canUseSharedObject() && val->refcount == OBJ_SHARED_REFCOUNT) { - val = dupStringObject(val); - kvstoreDictSetVal(db->keys, getKVStoreIndexForKey(key->ptr), de, val); - } + /* Shared objects can't be stored in the database. */ + serverAssert(val->refcount != OBJ_SHARED_REFCOUNT); if (server.maxmemory_policy & MAXMEMORY_FLAG_LFU) { updateLFU(val); } else { @@ -195,32 +193,45 @@ robj *lookupKeyWriteOrReply(client *c, robj *key, robj *reply) { return o; } -/* Add the key to the DB. +/* Add a key-value entry to the DB. * - * In this case a copy of `key` is copied in kvstore, the caller must ensure the `key` is properly freed. + * A copy of 'key' is stored in the database. The caller must ensure the + * `key` is properly freed by calling decrRefcount(key). * - * It's up to the caller to increment the reference - * counter of the value if needed. + * The value 'val' may (if its reference counter == 1) be reallocated and become + * invalid after a call to this function. The (possibly reallocated) value is + * stored in the database and also returned by this function, so the caller must + * use the returned pointer rather than 'val' after calling this function. + * + * The reference counter of the returned value is not incremented, so the caller + * should not free the value using decrRefcount after calling this function. * * If the update_if_existing argument is false, the program is aborted * if the key already exists, otherwise, it can fall back to dbOverwrite. */ -static void dbAddInternal(serverDb *db, robj *key, robj *val, int update_if_existing) { - dictEntry *existing; +static valkey *dbAddInternal(serverDb *db, robj *key, robj *val, int update_if_existing) { int dict_index = getKVStoreIndexForKey(key->ptr); - dictEntry *de = kvstoreDictAddRaw(db->keys, dict_index, key->ptr, &existing); - if (update_if_existing && existing) { - dbSetValue(db, key, val, 1, existing); - return; + void **oldref = NULL; + if (update_if_existing) { + oldref = kvstoreHashtableFindRef(db->keys, dict_index, key->ptr); + if (oldref != NULL) { + val = dbSetValue(db, key, val, 1, oldref); + return val; + } + } else { + debugServerAssertWithInfo(NULL, key, kvstoreHashtableFindRef(db->keys, dict_index, key->ptr) == NULL); } - serverAssertWithInfo(NULL, key, de != NULL); + + /* Not existing. Convert val to valkey object and insert. */ + val = objectSetKeyAndExpire(val, key->ptr, -1); initObjectLRUOrLFU(val); - kvstoreDictSetVal(db->keys, dict_index, de, val); + kvstoreHashtableAdd(db->keys, dict_index, val); signalKeyAsReady(db, key, val->type); notifyKeyspaceEvent(NOTIFY_NEW, "new", key, db->id); + return val; } -void dbAdd(serverDb *db, robj *key, robj *val) { - dbAddInternal(db, key, val, 0); +valkey *dbAdd(serverDb *db, robj *key, robj *val) { + return dbAddInternal(db, key, val, 0); } /* Returns which dict index should be used with kvstore for a given key. */ @@ -268,40 +279,46 @@ int getKeySlot(sds key) { * since it is not useful in this context. * * The function returns 1 if the key was added to the database, otherwise 0 is returned. - * - * In this case a copy of `key` is copied in kvstore, the caller must ensure the `key` is properly freed. */ -int dbAddRDBLoad(serverDb *db, sds key, robj *val) { +valkey *dbAddRDBLoad(serverDb *db, sds key, robj *val) { int dict_index = getKVStoreIndexForKey(key); - dictEntry *de = kvstoreDictAddRaw(db->keys, dict_index, key, NULL); - if (de == NULL) return 0; + hashtablePosition pos; + if (!kvstoreHashtableFindPositionForInsert(db->keys, dict_index, key, &pos, NULL)) { + return NULL; + } + val = objectSetKeyAndExpire(val, key, -1); + kvstoreHashtableInsertAtPosition(db->keys, dict_index, val, &pos); initObjectLRUOrLFU(val); - kvstoreDictSetVal(db->keys, dict_index, de, val); - return 1; + return val; } /* Overwrite an existing key with a new value. Incrementing the reference - * count of the new value is up to the caller. - * This function does not modify the expire time of the existing key. + * counter of the new value is up to the caller. The 'val' may be reallocated + * and the new pointer is returned. This function does not modify the expire + * time of the existing key. * * The 'overwrite' flag is an indication whether this is done as part of a * complete replacement of their key, which can be thought as a deletion and * replacement (in which case we need to emit deletion signals), or just an * update of a value of an existing key (when false). * - * The dictEntry input is optional, can be used if we already have one. + * The 'oldref' argument is optional. If provided, it is a pointer to the + * location within the hash table where the old value is stored and the new + * value should be stored. * * The program is aborted if the key was not already present. */ -static void dbSetValue(serverDb *db, robj *key, robj *val, int overwrite, dictEntry *de) { - int dict_index = getKVStoreIndexForKey(key->ptr); - if (!de) de = kvstoreDictFind(db->keys, dict_index, key->ptr); - serverAssertWithInfo(NULL, key, de != NULL); - robj *old = dictGetVal(de); +static valkey *dbSetValue(serverDb *db, robj *key, robj *val, int overwrite, void **oldref) { + if (oldref == NULL) { + int dict_index = getKVStoreIndexForKey(key->ptr); + oldref = kvstoreHashtableFindRef(db->keys, dict_index, key->ptr); + } + serverAssertWithInfo(NULL, key, oldref != NULL); + valkey *old = *oldref; val->lru = old->lru; if (overwrite) { - /* RM_StringDMA may call dbUnshareStringValue which may free val, so we + /* VM_StringDMA may call dbUnshareStringValue which may free val, so we * need to incr to retain old */ incrRefCount(old); /* Although the key is not really deleted from the database, we regard @@ -311,10 +328,20 @@ static void dbSetValue(serverDb *db, robj *key, robj *val, int overwrite, dictEn /* We want to try to unblock any module clients or clients using a blocking XREADGROUP */ signalDeletedKeyAsReady(db, key, old->type); decrRefCount(old); - /* Because of RM_StringDMA, old may be changed, so we need get old again */ - old = dictGetVal(de); + /* Because of VM_StringDMA, old may be changed, so we need get old again */ + old = *oldref; + } + /* Replace the old value at its location in the key space. */ + long long expire = objectGetExpire(old); + valkey *new = objectSetKeyAndExpire(val, key->ptr, expire); + *oldref = new; + /* Replace the old value at its location in the expire space. */ + if (expire >= 0) { + int dict_index = getKVStoreIndexForKey(key->ptr); + void **expireref = kvstoreHashtableFindRef(db->expires, dict_index, key->ptr); + serverAssert(expireref != NULL); + *expireref = new; } - kvstoreDictSetVal(db->keys, dict_index, de, val); /* For efficiency, let the I/O thread that allocated an object also deallocate it. */ if (tryOffloadFreeObjToIOThreads(old) == C_OK) { /* OK */ @@ -323,18 +350,23 @@ static void dbSetValue(serverDb *db, robj *key, robj *val, int overwrite, dictEn } else { decrRefCount(old); } + return new; } /* Replace an existing key with a new value, we just replace value and don't * emit any events */ -void dbReplaceValue(serverDb *db, robj *key, robj *val) { - dbSetValue(db, key, val, 0, NULL); +valkey *dbReplaceValue(serverDb *db, robj *key, robj *val) { + return dbSetValue(db, key, val, 0, NULL); } /* High level Set operation. This function can be used in order to set * a key, whatever it was existing or not, to a new object. * - * 1) The ref count of the value object is incremented. + * 1) The ref count of the value object is *not* incremented. The value + * may be reallocated when adding it to the database. A pointer to + * the reallocated object is returned. If you need to retain 'val', + * increment its reference counter before calling this function. + * Otherwise, use the returned value instead of 'val' after this call. * 2) clients WATCHing for the destination key notified. * 3) The expire time of the key is reset (the key is made persistent), * unless 'SETKEY_KEEPTTL' is enabled in flags. @@ -344,7 +376,7 @@ void dbReplaceValue(serverDb *db, robj *key, robj *val) { * All the new keys in the database should be created via this interface. * The client 'c' argument may be set to NULL if the operation is performed * in a context where there is no clear client performing the operation. */ -void setKey(client *c, serverDb *db, robj *key, robj *val, int flags) { +robj *setKey(client *c, serverDb *db, robj *key, robj *val, int flags) { int keyfound = 0; if (flags & SETKEY_ALREADY_EXIST) @@ -355,15 +387,15 @@ void setKey(client *c, serverDb *db, robj *key, robj *val, int flags) { keyfound = (lookupKeyWrite(db, key) != NULL); if (!keyfound) { - dbAdd(db, key, val); + val = dbAdd(db, key, val); } else if (keyfound < 0) { - dbAddInternal(db, key, val, 1); + val = dbAddInternal(db, key, val, 1); } else { - dbSetValue(db, key, val, 1, NULL); + val = dbSetValue(db, key, val, 1, NULL); } - incrRefCount(val); if (!(flags & SETKEY_KEEPTTL)) removeExpire(db, key); if (!(flags & SETKEY_NO_SIGNAL)) signalModifiedKey(c, db, key); + return val; } /* Return a random key, in form of an Object. @@ -371,18 +403,18 @@ void setKey(client *c, serverDb *db, robj *key, robj *val, int flags) { * * The function makes sure to return keys not already expired. */ robj *dbRandomKey(serverDb *db) { - dictEntry *de; + valkey *valkey; int maxtries = 100; int allvolatile = kvstoreSize(db->keys) == kvstoreSize(db->expires); while (1) { sds key; robj *keyobj; - int randomDictIndex = kvstoreGetFairRandomDictIndex(db->keys); - de = kvstoreDictGetFairRandomKey(db->keys, randomDictIndex); - if (de == NULL) return NULL; + int randomDictIndex = kvstoreGetFairRandomHashtableIndex(db->keys); + int ok = kvstoreHashtableFairRandomEntry(db->keys, randomDictIndex, (void **)&valkey); + if (!ok) return NULL; - key = dictGetKey(de); + key = objectGetKey(valkey); keyobj = createStringObject(key, sdslen(key)); if (dbFindExpiresWithDictIndex(db, key, randomDictIndex)) { if (allvolatile && server.primary_host && --maxtries == 0) { @@ -406,31 +438,38 @@ robj *dbRandomKey(serverDb *db) { } int dbGenericDeleteWithDictIndex(serverDb *db, robj *key, int async, int flags, int dict_index) { - dictEntry **plink; - int table; - dictEntry *de = kvstoreDictTwoPhaseUnlinkFind(db->keys, dict_index, key->ptr, &plink, &table); - if (de) { - robj *val = dictGetVal(de); - /* RM_StringDMA may call dbUnshareStringValue which may free val, so we + hashtablePosition pos; + void **ref = kvstoreHashtableTwoPhasePopFindRef(db->keys, dict_index, key->ptr, &pos); + if (ref != NULL) { + valkey *val = *ref; + /* VM_StringDMA may call dbUnshareStringValue which may free val, so we * need to incr to retain val */ incrRefCount(val); /* Tells the module that the key has been unlinked from the database. */ moduleNotifyKeyUnlink(key, val, db->id, flags); /* We want to try to unblock any module clients or clients using a blocking XREADGROUP */ signalDeletedKeyAsReady(db, key, val->type); - /* We should call decr before freeObjAsync. If not, the refcount may be - * greater than 1, so freeObjAsync doesn't work */ + /* Match the incrRefCount above. */ decrRefCount(val); + /* Because of dbUnshareStringValue, the val in de may change. */ + val = *ref; + + /* Delete from keys and expires tables. This will not free the object. + * (The expires table has no destructor callback.) */ + kvstoreHashtableTwoPhasePopDelete(db->keys, dict_index, &pos); + if (objectGetExpire(val) != -1) { + int deleted = kvstoreHashtableDelete(db->expires, dict_index, key->ptr); + serverAssert(deleted); + } else { + debugServerAssert(0 == kvstoreHashtableDelete(db->expires, dict_index, key->ptr)); + } + if (async) { - /* Because of dbUnshareStringValue, the val in de may change. */ - freeObjAsync(key, dictGetVal(de), db->id); - kvstoreDictSetVal(db->keys, dict_index, de, NULL); + freeObjAsync(key, val, db->id); + } else { + decrRefCount(val); } - /* Deleting an entry from the expires dict will not free the sds of - * the key, because it is shared with the main dictionary. */ - kvstoreDictDelete(db->expires, dict_index, key->ptr); - kvstoreDictTwoPhaseUnlinkFree(db->keys, dict_index, de, plink, table); return 1; } else { return 0; @@ -493,7 +532,7 @@ robj *dbUnshareStringValue(serverDb *db, robj *key, robj *o) { robj *decoded = getDecodedObject(o); o = createRawStringObject(decoded->ptr, sdslen(decoded->ptr)); decrRefCount(decoded); - dbReplaceValue(db, key, o); + o = dbReplaceValue(db, key, o); } return o; } @@ -504,7 +543,7 @@ robj *dbUnshareStringValue(serverDb *db, robj *key, robj *o) { * The dbnum can be -1 if all the DBs should be emptied, or the specified * DB index if we want to empty only a single database. * The function returns the number of keys removed from the database(s). */ -long long emptyDbStructure(serverDb *dbarray, int dbnum, int async, void(callback)(dict *)) { +long long emptyDbStructure(serverDb *dbarray, int dbnum, int async, void(callback)(hashtable *)) { long long removed = 0; int startdb, enddb; @@ -546,7 +585,7 @@ long long emptyDbStructure(serverDb *dbarray, int dbnum, int async, void(callbac * On success the function returns the number of keys removed from the * database(s). Otherwise -1 is returned in the specific case the * DB number is out of range, and errno is set to EINVAL. */ -long long emptyData(int dbnum, int flags, void(callback)(dict *)) { +long long emptyData(int dbnum, int flags, void(callback)(hashtable *)) { int async = (flags & EMPTYDB_ASYNC); int with_functions = !(flags & EMPTYDB_NOFUNCTIONS); ValkeyModuleFlushInfoV1 fi = {VALKEYMODULE_FLUSHINFO_VERSION, !async, dbnum}; @@ -585,23 +624,23 @@ long long emptyData(int dbnum, int flags, void(callback)(dict *)) { /* Initialize temporary db on replica for use during diskless replication. */ serverDb *initTempDb(void) { int slot_count_bits = 0; - int flags = KVSTORE_ALLOCATE_DICTS_ON_DEMAND; + int flags = KVSTORE_ALLOCATE_HASHTABLES_ON_DEMAND; if (server.cluster_enabled) { slot_count_bits = CLUSTER_SLOT_MASK_BITS; - flags |= KVSTORE_FREE_EMPTY_DICTS; + flags |= KVSTORE_FREE_EMPTY_HASHTABLES; } serverDb *tempDb = zcalloc(sizeof(serverDb) * server.dbnum); for (int i = 0; i < server.dbnum; i++) { tempDb[i].id = i; - tempDb[i].keys = kvstoreCreate(&kvstoreKeysDictType, slot_count_bits, flags); - tempDb[i].expires = kvstoreCreate(&kvstoreExpiresDictType, slot_count_bits, flags); + tempDb[i].keys = kvstoreCreate(&kvstoreKeysHashtableType, slot_count_bits, flags); + tempDb[i].expires = kvstoreCreate(&kvstoreExpiresHashtableType, slot_count_bits, flags); } return tempDb; } /* Discard tempDb, this can be slow (similar to FLUSHALL), but it's always async. */ -void discardTempDb(serverDb *tempDb, void(callback)(dict *)) { +void discardTempDb(serverDb *tempDb, void(callback)(hashtable *)) { int async = 1; /* Release temp DBs. */ @@ -818,7 +857,7 @@ void randomkeyCommand(client *c) { } void keysCommand(client *c) { - dictEntry *de; + valkey *val; sds pattern = c->argv[1]->ptr; int plen = sdslen(pattern), allkeys, pslot = -1; unsigned long numkeys = 0; @@ -827,16 +866,16 @@ void keysCommand(client *c) { if (server.cluster_enabled && !allkeys) { pslot = patternHashSlot(pattern, plen); } - kvstoreDictIterator *kvs_di = NULL; + kvstoreHashtableIterator *kvs_di = NULL; kvstoreIterator *kvs_it = NULL; if (pslot != -1) { - kvs_di = kvstoreGetDictSafeIterator(c->db->keys, pslot); + kvs_di = kvstoreGetHashtableSafeIterator(c->db->keys, pslot); } else { kvs_it = kvstoreIteratorInit(c->db->keys); } robj keyobj; - while ((de = kvs_di ? kvstoreDictIteratorNext(kvs_di) : kvstoreIteratorNext(kvs_it)) != NULL) { - sds key = dictGetKey(de); + while (kvs_di ? kvstoreHashtableIteratorNext(kvs_di, (void **)&val) : kvstoreIteratorNext(kvs_it, (void **)&val)) { + sds key = objectGetKey(val); if (allkeys || stringmatchlen(pattern, plen, key, sdslen(key), 0)) { initStaticStringObject(keyobj, key); @@ -847,7 +886,7 @@ void keysCommand(client *c) { } if (c->flag.close_asap) break; } - if (kvs_di) kvstoreReleaseDictIterator(kvs_di); + if (kvs_di) kvstoreReleaseHashtableIterator(kvs_di); if (kvs_it) kvstoreIteratorRelease(kvs_it); setDeferredArrayLen(c, replylen, numkeys); } @@ -877,6 +916,32 @@ int objectTypeCompare(robj *o, long long target) { else return 1; } + +/* Hashtable scan callback used by scanCallback when scanning the keyspace. */ +void keysScanCallback(void *privdata, void *entry) { + scanData *data = (scanData *)privdata; + valkey *obj = entry; + data->sampled++; + + /* Filter an object if it isn't the type we want. */ + if (data->type != LLONG_MAX) { + if (!objectTypeCompare(obj, data->type)) return; + } + + sds key = objectGetKey(obj); + + /* Filter object if its key does not match the pattern. */ + if (data->pattern) { + if (!stringmatchlen(data->pattern, sdslen(data->pattern), key, sdslen(key), 0)) { + return; + } + } + + /* Keep this key. */ + list *keys = data->keys; + listAddNodeTail(keys, key); +} + /* This callback is used by scanGenericCommand in order to collect elements * returned by the dictionary iterator into a list. */ void scanCallback(void *privdata, const dictEntry *de) { @@ -887,14 +952,9 @@ void scanCallback(void *privdata, const dictEntry *de) { sds key = NULL; data->sampled++; - /* o and typename can not have values at the same time. */ - serverAssert(!((data->type != LLONG_MAX) && o)); - - /* Filter an element if it isn't the type we want. */ - if (!o && data->type != LLONG_MAX) { - robj *rval = dictGetVal(de); - if (!objectTypeCompare(rval, data->type)) return; - } + /* This callback is only used for scanning elements within a key (hash + * fields, set elements, etc.) so o must be set here. */ + serverAssert(o != NULL); /* Filter element if it does not match the pattern. */ sds keysds = dictGetKey(de); @@ -904,9 +964,7 @@ void scanCallback(void *privdata, const dictEntry *de) { } } - if (o == NULL) { - key = keysds; - } else if (o->type == OBJ_SET) { + if (o->type == OBJ_SET) { key = keysds; } else if (o->type == OBJ_HASH) { key = keysds; @@ -1125,7 +1183,7 @@ void scanGenericCommand(client *c, robj *o, unsigned long long cursor) { /* In cluster mode there is a separate dictionary for each slot. * If cursor is empty, we should try exploring next non-empty slot. */ if (o == NULL) { - cursor = kvstoreScan(c->db->keys, cursor, onlydidx, scanCallback, NULL, &data); + cursor = kvstoreScan(c->db->keys, cursor, onlydidx, keysScanCallback, NULL, &data); } else { cursor = dictScan(ht, cursor, scanCallback, &data); } @@ -1178,6 +1236,9 @@ void scanGenericCommand(client *c, robj *o, unsigned long long cursor) { } /* Step 3: Filter the expired keys */ + /* TODO: Do this in the keysScanCallback where we have the valkey objects + * that contain the TTL (or add valkey object to the list instead of just + * the keys). Then we don't need to look them up again here. */ if (o == NULL && listLength(keys)) { robj kobj; listIter li; @@ -1315,9 +1376,9 @@ void renameGenericCommand(client *c, int nx) { * with the same name. */ dbDelete(c->db, c->argv[2]); } - dbAdd(c->db, c->argv[2], o); - if (expire != -1) setExpire(c, c->db, c->argv[2], expire); dbDelete(c->db, c->argv[1]); + o = dbAdd(c->db, c->argv[2], o); + if (expire != -1) o = setExpire(c, c->db, c->argv[2], expire); signalModifiedKey(c, c->db, c->argv[1]); signalModifiedKey(c, c->db, c->argv[2]); notifyKeyspaceEvent(NOTIFY_GENERIC, "rename_from", c->argv[1], c->db->id); @@ -1378,12 +1439,14 @@ void moveCommand(client *c) { addReply(c, shared.czero); return; } - dbAdd(dst, c->argv[1], o); - if (expire != -1) setExpire(c, dst, c->argv[1], expire); - incrRefCount(o); - /* OK! key moved, free the entry in the source DB */ - dbDelete(src, c->argv[1]); + incrRefCount(o); /* ref counter = 2 */ + dbDelete(src, c->argv[1]); /* ref counter = 1 */ + + o = dbAdd(dst, c->argv[1], o); + if (expire != -1) o = setExpire(c, dst, c->argv[1], expire); + + /* OK! key moved */ signalModifiedKey(c, src, c->argv[1]); signalModifiedKey(c, dst, c->argv[1]); notifyKeyspaceEvent(NOTIFY_GENERIC, "move_from", c->argv[1], src->id); @@ -1481,8 +1544,8 @@ void copyCommand(client *c) { dbDelete(dst, newkey); } - dbAdd(dst, newkey, newobj); - if (expire != -1) setExpire(c, dst, newkey, expire); + newobj = dbAdd(dst, newkey, newobj); + if (expire != -1) newobj = setExpire(c, dst, newkey, expire); /* OK! key copied */ signalModifiedKey(c, dst, c->argv[2]); @@ -1501,9 +1564,8 @@ void scanDatabaseForReadyKeys(serverDb *db) { dictIterator *di = dictGetSafeIterator(db->blocking_keys); while ((de = dictNext(di)) != NULL) { robj *key = dictGetKey(de); - dictEntry *kde = dbFind(db, key->ptr); - if (kde) { - robj *value = dictGetVal(kde); + valkey *value = dbFind(db, key->ptr); + if (value) { signalKeyAsReady(db, key, value->type); } } @@ -1521,17 +1583,15 @@ void scanDatabaseForDeletedKeys(serverDb *emptied, serverDb *replaced_with) { int existed = 0, exists = 0; int original_type = -1, curr_type = -1; - dictEntry *kde = dbFind(emptied, key->ptr); - if (kde) { - robj *value = dictGetVal(kde); + valkey *value = dbFind(emptied, key->ptr); + if (value) { original_type = value->type; existed = 1; } if (replaced_with) { - kde = dbFind(replaced_with, key->ptr); - if (kde) { - robj *value = dictGetVal(kde); + value = dbFind(replaced_with, key->ptr); + if (value) { curr_type = value->type; exists = 1; } @@ -1668,39 +1728,62 @@ void swapdbCommand(client *c) { *----------------------------------------------------------------------------*/ int removeExpire(serverDb *db, robj *key) { - return kvstoreDictDelete(db->expires, getKVStoreIndexForKey(key->ptr), key->ptr) == DICT_OK; + valkey *val; + int dict_index = getKVStoreIndexForKey(key->ptr); + if (kvstoreHashtablePop(db->expires, dict_index, key->ptr, (void **)&val)) { + valkey *newval = objectSetExpire(val, -1); + serverAssert(newval == val); + debugServerAssert(getExpire(db, key) == -1); + return 1; + } + return 0; } /* Set an expire to the specified key. If the expire is set in the context * of an user calling a command 'c' is the client, otherwise 'c' is set * to NULL. The 'when' parameter is the absolute unix time in milliseconds * after which the key will no longer be considered valid. */ -void setExpire(client *c, serverDb *db, robj *key, long long when) { - dictEntry *kde, *de, *existing; +robj *setExpire(client *c, serverDb *db, robj *key, long long when) { + /* TODO: Add val as a parameter to this function, to avoid looking it up. */ + valkey *val; - /* Reuse the sds from the main dict in the expire dict */ + /* Reuse the object from the main dict in the expire dict. When setting + * expire in an robj, it's potentially reallocated. We need to updates the + * pointer(s) to it. */ int dict_index = getKVStoreIndexForKey(key->ptr); - kde = kvstoreDictFind(db->keys, dict_index, key->ptr); - serverAssertWithInfo(NULL, key, kde != NULL); - de = kvstoreDictAddRaw(db->expires, dict_index, dictGetKey(kde), &existing); - if (existing) { - dictSetSignedIntegerVal(existing, when); + void **valref = kvstoreHashtableFindRef(db->keys, dict_index, key->ptr); + serverAssertWithInfo(NULL, key, valref != NULL); + val = *valref; + long long old_when = objectGetExpire(val); + valkey *newval = objectSetExpire(val, when); + if (old_when != -1) { + /* Val already had an expire field, so it was not reallocated. */ + serverAssert(newval == val); + /* It already exists in set of keys with expire. */ + debugServerAssert(!kvstoreHashtableAdd(db->expires, dict_index, newval)); } else { - dictSetSignedIntegerVal(de, when); + /* No old expire. Update the pointer in the keys hashtab, if needed, and + * add it to the expires hashtable. */ + if (newval != val) { + val = *valref = newval; + } + int added = kvstoreHashtableAdd(db->expires, dict_index, newval); + serverAssert(added); } int writable_replica = server.primary_host && server.repl_replica_ro == 0; if (c && writable_replica && !c->flag.primary) rememberReplicaKeyWithExpire(db, key); + return val; } /* Return the expire time of the specified key, or -1 if no expire * is associated with this key (i.e. the key is non volatile) */ long long getExpireWithDictIndex(serverDb *db, robj *key, int dict_index) { - dictEntry *de; + valkey *val; - if ((de = dbFindExpiresWithDictIndex(db, key->ptr, dict_index)) == NULL) return -1; + if ((val = dbFindExpiresWithDictIndex(db, key->ptr, dict_index)) == NULL) return -1; - return dictGetSignedIntegerVal(de); + return objectGetExpire(val); } /* Return the expire time of the specified key, or -1 if no expire @@ -1892,10 +1975,11 @@ static int dbExpandSkipSlot(int slot) { * In cluster mode resizes all individual dictionaries for slots that this node owns. * * Based on the parameter `try_expand`, appropriate dict expand API is invoked. - * if try_expand is set to 1, `dictTryExpand` is used else `dictExpand`. - * The return code is either `DICT_OK`/`DICT_ERR` for both the API(s). - * `DICT_OK` response is for successful expansion. However ,`DICT_ERR` response signifies failure in allocation in - * `dictTryExpand` call and in case of `dictExpand` call it signifies no expansion was performed. + * if try_expand is non-zero, `hashtableTryExpand` is used else `hashtableExpand`. + * + * Returns C_OK or C_ERR. C_OK response is for successful expansion. C_ERR + * signifies failure in allocation if try_expand is non-zero. Otherwise it + * signifies that no expansion was performed. */ static int dbExpandGeneric(kvstore *kvs, uint64_t db_size, int try_expand) { int ret; @@ -1921,20 +2005,24 @@ int dbExpandExpires(serverDb *db, uint64_t db_size, int try_expand) { return dbExpandGeneric(db->expires, db_size, try_expand); } -dictEntry *dbFindWithDictIndex(serverDb *db, void *key, int dict_index) { - return kvstoreDictFind(db->keys, dict_index, key); +valkey *dbFindWithDictIndex(serverDb *db, sds key, int dict_index) { + void *existing = NULL; + kvstoreHashtableFind(db->keys, dict_index, key, &existing); + return existing; } -dictEntry *dbFind(serverDb *db, void *key) { +valkey *dbFind(serverDb *db, sds key) { int dict_index = getKVStoreIndexForKey(key); return dbFindWithDictIndex(db, key, dict_index); } -dictEntry *dbFindExpiresWithDictIndex(serverDb *db, void *key, int dict_index) { - return kvstoreDictFind(db->expires, dict_index, key); +valkey *dbFindExpiresWithDictIndex(serverDb *db, sds key, int dict_index) { + void *existing = NULL; + kvstoreHashtableFind(db->expires, dict_index, key, &existing); + return existing; } -dictEntry *dbFindExpires(serverDb *db, void *key) { +valkey *dbFindExpires(serverDb *db, sds key) { int dict_index = getKVStoreIndexForKey(key); return dbFindExpiresWithDictIndex(db, key, dict_index); } @@ -1943,7 +2031,7 @@ unsigned long long dbSize(serverDb *db) { return kvstoreSize(db->keys); } -unsigned long long dbScan(serverDb *db, unsigned long long cursor, dictScanFunction *scan_cb, void *privdata) { +unsigned long long dbScan(serverDb *db, unsigned long long cursor, hashtableScanFunction scan_cb, void *privdata) { return kvstoreScan(db->keys, cursor, -1, scan_cb, NULL, privdata); } diff --git a/src/debug.c b/src/debug.c index 13da7bcc93..90f4fcb3c0 100644 --- a/src/debug.c +++ b/src/debug.c @@ -281,7 +281,7 @@ void xorObjectDigest(serverDb *db, robj *keyobj, unsigned char *digest, robj *o) * a different digest. */ void computeDatasetDigest(unsigned char *final) { unsigned char digest[20]; - dictEntry *de; + valkey *o; int j; uint32_t aux; @@ -297,17 +297,16 @@ void computeDatasetDigest(unsigned char *final) { mixDigest(final, &aux, sizeof(aux)); /* Iterate this DB writing every entry */ - while ((de = kvstoreIteratorNext(kvs_it)) != NULL) { + while (kvstoreIteratorNext(kvs_it, (void **)&o)) { sds key; - robj *keyobj, *o; + robj *keyobj; memset(digest, 0, 20); /* This key-val digest */ - key = dictGetKey(de); + key = objectGetKey(o); keyobj = createStringObject(key, sdslen(key)); mixDigest(digest, key, sdslen(key)); - o = dictGetVal(de); xorObjectDigest(db, keyobj, digest, o); /* We can finally xor the key-val digest to the final digest */ @@ -608,18 +607,16 @@ void debugCommand(client *c) { server.debug_cluster_close_link_on_packet_drop = atoi(c->argv[2]->ptr); addReply(c, shared.ok); } else if (!strcasecmp(c->argv[1]->ptr, "object") && (c->argc == 3 || c->argc == 4)) { - dictEntry *de; robj *val; char *strenc; int fast = 0; if (c->argc == 4 && !strcasecmp(c->argv[3]->ptr, "fast")) fast = 1; - if ((de = dbFind(c->db, c->argv[2]->ptr)) == NULL) { + if ((val = dbFind(c->db, c->argv[2]->ptr)) == NULL) { addReplyErrorObject(c, shared.nokeyerr); return; } - val = dictGetVal(de); strenc = strEncoding(val->encoding); char extra[138] = {0}; @@ -667,16 +664,14 @@ void debugCommand(client *c) { addReplyStatusLength(c, s, sdslen(s)); sdsfree(s); } else if (!strcasecmp(c->argv[1]->ptr, "sdslen") && c->argc == 3) { - dictEntry *de; robj *val; sds key; - if ((de = dbFind(c->db, c->argv[2]->ptr)) == NULL) { + if ((val = dbFind(c->db, c->argv[2]->ptr)) == NULL) { addReplyErrorObject(c, shared.nokeyerr); return; } - val = dictGetVal(de); - key = dictGetKey(de); + key = objectGetKey(val); if (val->type != OBJ_STRING || !sdsEncodedObject(val)) { addReplyError(c, "Not an sds encoded string."); @@ -746,7 +741,7 @@ void debugCommand(client *c) { val = createStringObject(NULL, valsize); memcpy(val->ptr, buf, valsize <= buflen ? valsize : buflen); } - dbAdd(c->db, key, val); + val = dbAdd(c->db, key, val); signalModifiedKey(c, c->db, key); decrRefCount(key); } @@ -769,8 +764,7 @@ void debugCommand(client *c) { /* We don't use lookupKey because a debug command should * work on logically expired keys */ - dictEntry *de; - robj *o = ((de = dbFind(c->db, c->argv[j]->ptr)) == NULL) ? NULL : dictGetVal(de); + robj *o = dbFind(c->db, c->argv[j]->ptr); if (o) xorObjectDigest(c->db, c->argv[j], digest, o); sds d = sdsempty(); @@ -1905,12 +1899,10 @@ void logCurrentClient(client *cc, const char *title) { * selected DB, and if so print info about the associated object. */ if (cc->argc > 1) { robj *val, *key; - dictEntry *de; key = getDecodedObject(cc->argv[1]); - de = dbFind(cc->db, key->ptr); - if (de) { - val = dictGetVal(de); + val = dbFind(cc->db, key->ptr); + if (val) { serverLog(LL_WARNING, "key '%s' found in DB containing the following object:", (char *)key->ptr); serverLogObjectDebugInfo(val); } diff --git a/src/defrag.c b/src/defrag.c index 4d34009f8b..1893ef3825 100644 --- a/src/defrag.c +++ b/src/defrag.c @@ -41,7 +41,6 @@ typedef struct defragCtx { void *privdata; int slot; - void *aux; } defragCtx; typedef struct defragPubSubCtx { @@ -76,36 +75,6 @@ void *activeDefragAlloc(void *ptr) { return newptr; } -/* This method captures the expiry db dict entry which refers to data stored in keys db dict entry. */ -void defragEntryStartCbForKeys(void *ctx, void *oldptr) { - defragCtx *defragctx = (defragCtx *)ctx; - serverDb *db = defragctx->privdata; - sds oldsds = (sds)dictGetKey((dictEntry *)oldptr); - int slot = defragctx->slot; - if (kvstoreDictSize(db->expires, slot)) { - dictEntry *expire_de = kvstoreDictFind(db->expires, slot, oldsds); - defragctx->aux = expire_de; - } -} - -/* This method updates the key of expiry db dict entry. The key might be no longer valid - * as it could have been cleaned up during the defrag-realloc of the main dictionary. */ -void defragEntryFinishCbForKeys(void *ctx, void *newptr) { - defragCtx *defragctx = (defragCtx *)ctx; - dictEntry *expire_de = (dictEntry *)defragctx->aux; - /* Item doesn't have TTL associated to it. */ - if (!expire_de) return; - /* No reallocation happened. */ - if (!newptr) { - expire_de = NULL; - return; - } - serverDb *db = defragctx->privdata; - sds newsds = (sds)dictGetKey((dictEntry *)newptr); - int slot = defragctx->slot; - kvstoreDictSetKey(db->expires, slot, expire_de, newsds); -} - /*Defrag helper for sds strings * * returns NULL in case the allocation wasn't moved. @@ -365,8 +334,8 @@ void activeDefragQuickListNodes(quicklist *ql) { /* when the value has lots of elements, we want to handle it later and not as * part of the main dictionary scan. this is needed in order to prevent latency * spikes when handling large items */ -void defragLater(serverDb *db, dictEntry *kde) { - sds key = sdsdup(dictGetKey(kde)); +void defragLater(serverDb *db, valkey *obj) { + sds key = sdsdup(objectGetKey(obj)); listAddNodeTail(db->defrag_later, key); } @@ -433,7 +402,15 @@ void scanLaterZset(robj *ob, unsigned long *cursor) { *cursor = dictScanDefrag(d, *cursor, scanLaterZsetCallback, &defragfns, &data); } -/* Used as scan callback when all the work is done in the dictDefragFunctions. */ +/* Used as hashtable scan callback when all we need is to defrag the hashtable + * internals (the allocated buckets) and not the elements. */ +void scanHashtableCallbackCountScanned(void *privdata, void *elemref) { + UNUSED(privdata); + UNUSED(elemref); + server.stat_active_defrag_scanned++; +} + +/* Used as dict scan callback when all the work is done in the dictDefragFunctions. */ void scanCallbackCountScanned(void *privdata, const dictEntry *de) { UNUSED(privdata); UNUSED(de); @@ -457,19 +434,17 @@ void scanLaterHash(robj *ob, unsigned long *cursor) { *cursor = dictScanDefrag(d, *cursor, scanCallbackCountScanned, &defragfns, NULL); } -void defragQuicklist(serverDb *db, dictEntry *kde) { - robj *ob = dictGetVal(kde); +void defragQuicklist(serverDb *db, valkey *ob) { quicklist *ql = ob->ptr, *newql; serverAssert(ob->type == OBJ_LIST && ob->encoding == OBJ_ENCODING_QUICKLIST); if ((newql = activeDefragAlloc(ql))) ob->ptr = ql = newql; if (ql->len > server.active_defrag_max_scan_fields) - defragLater(db, kde); + defragLater(db, ob); else activeDefragQuickListNodes(ql); } -void defragZsetSkiplist(serverDb *db, dictEntry *kde) { - robj *ob = dictGetVal(kde); +void defragZsetSkiplist(serverDb *db, valkey *ob) { zset *zs = (zset *)ob->ptr; zset *newzs; zskiplist *newzsl; @@ -481,7 +456,7 @@ void defragZsetSkiplist(serverDb *db, dictEntry *kde) { if ((newzsl = activeDefragAlloc(zs->zsl))) zs->zsl = newzsl; if ((newheader = activeDefragAlloc(zs->zsl->header))) zs->zsl->header = newheader; if (dictSize(zs->dict) > server.active_defrag_max_scan_fields) - defragLater(db, kde); + defragLater(db, ob); else { dictIterator *di = dictGetIterator(zs->dict); while ((de = dictNext(di)) != NULL) { @@ -493,26 +468,24 @@ void defragZsetSkiplist(serverDb *db, dictEntry *kde) { if ((newdict = dictDefragTables(zs->dict))) zs->dict = newdict; } -void defragHash(serverDb *db, dictEntry *kde) { - robj *ob = dictGetVal(kde); +void defragHash(serverDb *db, valkey *ob) { dict *d, *newd; serverAssert(ob->type == OBJ_HASH && ob->encoding == OBJ_ENCODING_HT); d = ob->ptr; if (dictSize(d) > server.active_defrag_max_scan_fields) - defragLater(db, kde); + defragLater(db, ob); else activeDefragSdsDict(d, DEFRAG_SDS_DICT_VAL_IS_SDS); /* defrag the dict struct and tables */ if ((newd = dictDefragTables(ob->ptr))) ob->ptr = newd; } -void defragSet(serverDb *db, dictEntry *kde) { - robj *ob = dictGetVal(kde); +void defragSet(serverDb *db, valkey *ob) { dict *d, *newd; serverAssert(ob->type == OBJ_SET && ob->encoding == OBJ_ENCODING_HT); d = ob->ptr; if (dictSize(d) > server.active_defrag_max_scan_fields) - defragLater(db, kde); + defragLater(db, ob); else activeDefragSdsDict(d, DEFRAG_SDS_DICT_NO_VAL); /* defrag the dict struct and tables */ @@ -650,8 +623,7 @@ void *defragStreamConsumerGroup(raxIterator *ri, void *privdata) { return NULL; } -void defragStream(serverDb *db, dictEntry *kde) { - robj *ob = dictGetVal(kde); +void defragStream(serverDb *db, valkey *ob) { serverAssert(ob->type == OBJ_STREAM && ob->encoding == OBJ_ENCODING_STREAM); stream *s = ob->ptr, *news; @@ -661,7 +633,7 @@ void defragStream(serverDb *db, dictEntry *kde) { if (raxSize(s->rax) > server.active_defrag_max_scan_fields) { rax *newrax = activeDefragAlloc(s->rax); if (newrax) s->rax = newrax; - defragLater(db, kde); + defragLater(db, ob); } else defragRadixTree(&s->rax, 1, NULL, NULL); @@ -671,33 +643,47 @@ void defragStream(serverDb *db, dictEntry *kde) { /* Defrag a module key. This is either done immediately or scheduled * for later. Returns then number of pointers defragged. */ -void defragModule(serverDb *db, dictEntry *kde) { - robj *obj = dictGetVal(kde); +void defragModule(serverDb *db, valkey *obj) { serverAssert(obj->type == OBJ_MODULE); - - if (!moduleDefragValue(dictGetKey(kde), obj, db->id)) defragLater(db, kde); + void *sds_key_passed_as_robj = objectGetKey(obj); + /* Fun fact (and a bug since forever): The key is passed to + * moduleDefragValue as an sds string, but the parameter is declared to be + * an robj and it's passed as such to the module type defrag callbacks. + * Nobody can ever have used this, i.e. accessed the key name in the defrag + * or free_effort module type callbacks. */ + if (!moduleDefragValue(sds_key_passed_as_robj, obj, db->id)) defragLater(db, obj); } /* for each key we scan in the main dict, this function will attempt to defrag * all the various pointers it has. */ -void defragKey(defragCtx *ctx, dictEntry *de) { +void defragKey(defragCtx *ctx, valkey **elemref) { serverDb *db = ctx->privdata; int slot = ctx->slot; robj *newob, *ob; unsigned char *newzl; + ob = *elemref; + + /* Find the pointer in the expire table to this object, if any. */ + /* TODO: Only lookup the expire table when the object has actually been + * reallocated. A trick is hashtableFindRefByKeyAndOldValue(s, key, ob). */ + void **expireref = NULL; + if (objectGetExpire(ob) >= 0) { + expireref = kvstoreHashtableFindRef(db->expires, slot, objectGetKey(ob)); + serverAssert(expireref != NULL); + } /* Try to defrag robj and / or string value. */ - ob = dictGetVal(de); if ((newob = activeDefragStringOb(ob))) { - kvstoreDictSetVal(db->keys, slot, de, newob); + *elemref = newob; ob = newob; + if (expireref != NULL) *expireref = newob; } if (ob->type == OBJ_STRING) { /* Already handled in activeDefragStringOb. */ } else if (ob->type == OBJ_LIST) { if (ob->encoding == OBJ_ENCODING_QUICKLIST) { - defragQuicklist(db, de); + defragQuicklist(db, ob); } else if (ob->encoding == OBJ_ENCODING_LISTPACK) { if ((newzl = activeDefragAlloc(ob->ptr))) ob->ptr = newzl; } else { @@ -705,7 +691,7 @@ void defragKey(defragCtx *ctx, dictEntry *de) { } } else if (ob->type == OBJ_SET) { if (ob->encoding == OBJ_ENCODING_HT) { - defragSet(db, de); + defragSet(db, ob); } else if (ob->encoding == OBJ_ENCODING_INTSET || ob->encoding == OBJ_ENCODING_LISTPACK) { void *newptr, *ptr = ob->ptr; if ((newptr = activeDefragAlloc(ptr))) ob->ptr = newptr; @@ -716,7 +702,7 @@ void defragKey(defragCtx *ctx, dictEntry *de) { if (ob->encoding == OBJ_ENCODING_LISTPACK) { if ((newzl = activeDefragAlloc(ob->ptr))) ob->ptr = newzl; } else if (ob->encoding == OBJ_ENCODING_SKIPLIST) { - defragZsetSkiplist(db, de); + defragZsetSkiplist(db, ob); } else { serverPanic("Unknown sorted set encoding"); } @@ -724,23 +710,23 @@ void defragKey(defragCtx *ctx, dictEntry *de) { if (ob->encoding == OBJ_ENCODING_LISTPACK) { if ((newzl = activeDefragAlloc(ob->ptr))) ob->ptr = newzl; } else if (ob->encoding == OBJ_ENCODING_HT) { - defragHash(db, de); + defragHash(db, ob); } else { serverPanic("Unknown hash encoding"); } } else if (ob->type == OBJ_STREAM) { - defragStream(db, de); + defragStream(db, ob); } else if (ob->type == OBJ_MODULE) { - defragModule(db, de); + defragModule(db, ob); } else { serverPanic("Unknown object type"); } } /* Defrag scan callback for the main db dictionary. */ -void defragScanCallback(void *privdata, const dictEntry *de) { +void defragScanCallback(void *privdata, void *elemref) { long long hits_before = server.stat_active_defrag_hits; - defragKey((defragCtx *)privdata, (dictEntry *)de); + defragKey((defragCtx *)privdata, (valkey **)elemref); if (server.stat_active_defrag_hits != hits_before) server.stat_active_defrag_key_hits++; else @@ -771,19 +757,19 @@ float getAllocatorFragmentation(size_t *out_frag_bytes) { return frag_pct; } -/* Defrag scan callback for the pubsub dictionary. */ -void defragPubsubScanCallback(void *privdata, const dictEntry *de) { +/* Defrag scan callback for a pubsub channels hashtable. */ +void defragPubsubScanCallback(void *privdata, void *elemref) { defragCtx *ctx = privdata; defragPubSubCtx *pubsub_ctx = ctx->privdata; - kvstore *pubsub_channels = pubsub_ctx->pubsub_channels; - robj *newchannel, *channel = dictGetKey(de); - dict *newclients, *clients = dictGetVal(de); + void **channel_dict_ref = (void **)elemref; + dict *newclients, *clients = *channel_dict_ref; + robj *newchannel, *channel = *(robj **)clients->metadata; /* Try to defrag the channel name. */ serverAssert(channel->refcount == (int)dictSize(clients) + 1); newchannel = activeDefragStringObEx(channel, dictSize(clients) + 1); if (newchannel) { - kvstoreDictSetKey(pubsub_channels, ctx->slot, (dictEntry *)de, newchannel); + *(robj **)clients->metadata = newchannel; /* 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 @@ -800,8 +786,9 @@ void defragPubsubScanCallback(void *privdata, const dictEntry *de) { } /* Try to defrag the dictionary of clients that is stored as the value part. */ - if ((newclients = dictDefragTables(clients))) - kvstoreDictSetVal(pubsub_channels, ctx->slot, (dictEntry *)de, newclients); + if ((newclients = dictDefragTables(clients))) { + *channel_dict_ref = newclients; + } server.stat_active_defrag_scanned++; } @@ -814,15 +801,14 @@ void defragOtherGlobals(void) { * that remain static for a long time */ activeDefragSdsDict(evalScriptsDict(), DEFRAG_SDS_DICT_VAL_LUA_SCRIPT); moduleDefragGlobals(); - kvstoreDictLUTDefrag(server.pubsub_channels, dictDefragTables); - kvstoreDictLUTDefrag(server.pubsubshard_channels, dictDefragTables); + kvstoreHashtableDefragTables(server.pubsub_channels, activeDefragAlloc); + kvstoreHashtableDefragTables(server.pubsubshard_channels, activeDefragAlloc); } /* returns 0 more work may or may not be needed (see non-zero cursor), * and 1 if time is up and more work is needed. */ -int defragLaterItem(dictEntry *de, unsigned long *cursor, long long endtime, int dbid) { - if (de) { - robj *ob = dictGetVal(de); +int defragLaterItem(valkey *ob, unsigned long *cursor, long long endtime, int dbid) { + if (ob) { if (ob->type == OBJ_LIST) { return scanLaterList(ob, cursor, endtime); } else if (ob->type == OBJ_SET) { @@ -834,7 +820,13 @@ int defragLaterItem(dictEntry *de, unsigned long *cursor, long long endtime, int } else if (ob->type == OBJ_STREAM) { return scanLaterStreamListpacks(ob, cursor, endtime); } else if (ob->type == OBJ_MODULE) { - return moduleLateDefrag(dictGetKey(de), ob, cursor, endtime, dbid); + void *sds_key_passed_as_robj = objectGetKey(ob); + /* Fun fact (and a bug since forever): The key is passed to + * moduleLateDefrag as an sds string, but the parameter is declared + * to be an robj and it's passed as such to the module type defrag + * callbacks. Nobody can ever have used this, i.e. accessed the key + * name in the defrag module type callback. */ + return moduleLateDefrag(sds_key_passed_as_robj, ob, cursor, endtime, dbid); } else { *cursor = 0; /* object type may have changed since we schedule it for later */ } @@ -877,12 +869,13 @@ int defragLaterStep(serverDb *db, int slot, long long endtime) { defrag_later_cursor = 0; } - /* each time we enter this function we need to fetch the key from the dict again (if it still exists) */ - dictEntry *de = kvstoreDictFind(db->keys, slot, defrag_later_current_key); + /* each time we enter this function we need to fetch the object again (if it still exists) */ + valkey *ob = NULL; + kvstoreHashtableFind(db->keys, slot, defrag_later_current_key, (void **)&ob); key_defragged = server.stat_active_defrag_hits; do { int quit = 0; - if (defragLaterItem(de, &defrag_later_cursor, endtime, db->id)) + if (defragLaterItem(ob, &defrag_later_cursor, endtime, db->id)) quit = 1; /* time is up, we didn't finish all the work */ /* Once in 16 scan iterations, 512 pointer reallocations, or 64 fields @@ -1000,9 +993,6 @@ void activeDefragCycle(void) { endtime = start + timelimit; latencyStartMonitor(latency); - dictDefragFunctions defragfns = {.defragAlloc = activeDefragAlloc, - .defragEntryStartCb = defragEntryStartCbForKeys, - .defragEntryFinishCb = defragEntryFinishCbForKeys}; do { /* if we're not continuing a scan from the last call or loop, start a new one */ if (!defrag_stage && !defrag_cursor && (slot < 0)) { @@ -1043,8 +1033,8 @@ void activeDefragCycle(void) { } db = &server.db[current_db]; - kvstoreDictLUTDefrag(db->keys, dictDefragTables); - kvstoreDictLUTDefrag(db->expires, dictDefragTables); + kvstoreHashtableDefragTables(db->keys, activeDefragAlloc); + kvstoreHashtableDefragTables(db->expires, activeDefragAlloc); defrag_stage = 0; defrag_cursor = 0; slot = -1; @@ -1054,12 +1044,12 @@ void activeDefragCycle(void) { /* This array of structures holds the parameters for all defragmentation stages. */ typedef struct defragStage { kvstore *kvs; - dictScanFunction *scanfn; + hashtableScanFunction scanfn; void *privdata; } defragStage; defragStage defrag_stages[] = { {db->keys, defragScanCallback, db}, - {db->expires, scanCallbackCountScanned, NULL}, + {db->expires, scanHashtableCallbackCountScanned, NULL}, {server.pubsub_channels, defragPubsubScanCallback, &(defragPubSubCtx){server.pubsub_channels, getClientPubSubChannels}}, {server.pubsubshard_channels, defragPubsubScanCallback, @@ -1079,9 +1069,14 @@ void activeDefragCycle(void) { if (!defrag_later_item_in_progress) { /* Continue defragmentation from the previous stage. * If slot is -1, it means this stage starts from the first non-empty slot. */ - if (slot == -1) slot = kvstoreGetFirstNonEmptyDictIndex(current_stage->kvs); - defrag_cursor = kvstoreDictScanDefrag(current_stage->kvs, slot, defrag_cursor, current_stage->scanfn, - &defragfns, &(defragCtx){current_stage->privdata, slot}); + if (slot == -1) slot = kvstoreGetFirstNonEmptyHashtableIndex(current_stage->kvs); + defrag_cursor = kvstoreHashtableScanDefrag(current_stage->kvs, + slot, + defrag_cursor, + current_stage->scanfn, + &(defragCtx){current_stage->privdata, slot}, + activeDefragAlloc, + HASHTABLE_SCAN_EMIT_REF); } if (!defrag_cursor) { @@ -1092,7 +1087,7 @@ void activeDefragCycle(void) { } /* Move to the next slot in the current stage. If we've reached the end, move to the next stage. */ - if ((slot = kvstoreGetNextNonEmptyDictIndex(current_stage->kvs, slot)) == -1) defrag_stage++; + if ((slot = kvstoreGetNextNonEmptyHashtableIndex(current_stage->kvs, slot)) == -1) defrag_stage++; defrag_later_item_in_progress = 0; } diff --git a/src/evict.c b/src/evict.c index 5e4b6220eb..06923ec426 100644 --- a/src/evict.c +++ b/src/evict.c @@ -143,26 +143,14 @@ void evictionPoolAlloc(void) { * right. */ int evictionPoolPopulate(serverDb *db, kvstore *samplekvs, struct evictionPoolEntry *pool) { int j, k, count; - dictEntry *samples[server.maxmemory_samples]; + void *samples[server.maxmemory_samples]; - int slot = kvstoreGetFairRandomDictIndex(samplekvs); - count = kvstoreDictGetSomeKeys(samplekvs, slot, samples, server.maxmemory_samples); + int slot = kvstoreGetFairRandomHashtableIndex(samplekvs); + count = kvstoreHashtableSampleEntries(samplekvs, slot, (void **)&samples, server.maxmemory_samples); for (j = 0; j < count; j++) { unsigned long long idle; - sds key; - robj *o; - dictEntry *de; - - de = samples[j]; - key = dictGetKey(de); - - /* If the dictionary we are sampling from is not the main - * dictionary (but the expires one) we need to lookup the key - * again in the key dictionary to obtain the value object. */ - if (server.maxmemory_policy != MAXMEMORY_VOLATILE_TTL) { - if (samplekvs != db->keys) de = kvstoreDictFind(db->keys, slot, key); - o = dictGetVal(de); - } + valkey *o = samples[j]; + sds key = objectGetKey(o); /* Calculate the idle time according to the policy. This is called * idle just because the code initially handled LRU, but is in fact @@ -180,7 +168,7 @@ int evictionPoolPopulate(serverDb *db, kvstore *samplekvs, struct evictionPoolEn idle = 255 - LFUDecrAndReturn(o); } else if (server.maxmemory_policy == MAXMEMORY_VOLATILE_TTL) { /* In this case the sooner the expire the better. */ - idle = ULLONG_MAX - (long)dictGetVal(de); + idle = ULLONG_MAX - objectGetExpire(o); } else { serverPanic("Unknown eviction policy in evictionPoolPopulate()"); } @@ -568,7 +556,7 @@ int performEvictions(void) { sds bestkey = NULL; int bestdbid; serverDb *db; - dictEntry *de; + valkey *valkey; if (server.maxmemory_policy & (MAXMEMORY_FLAG_LRU | MAXMEMORY_FLAG_LFU) || server.maxmemory_policy == MAXMEMORY_VOLATILE_TTL) { @@ -592,7 +580,7 @@ int performEvictions(void) { if (current_db_keys == 0) continue; total_keys += current_db_keys; - int l = kvstoreNumNonEmptyDicts(kvs); + int l = kvstoreNumNonEmptyHashtables(kvs); /* Do not exceed the number of non-empty slots when looping. */ while (l--) { sampled_keys += evictionPoolPopulate(db, kvs, pool); @@ -617,7 +605,7 @@ int performEvictions(void) { } else { kvs = server.db[bestdbid].expires; } - de = kvstoreDictFind(kvs, pool[k].slot, pool[k].key); + int found = kvstoreHashtableFind(kvs, pool[k].slot, pool[k].key, (void **)&valkey); /* Remove the entry from the pool. */ if (pool[k].key != pool[k].cached) sdsfree(pool[k].key); @@ -626,8 +614,8 @@ int performEvictions(void) { /* If the key exists, is our pick. Otherwise it is * a ghost and we need to try the next element. */ - if (de) { - bestkey = dictGetKey(de); + if (found) { + bestkey = objectGetKey(valkey); break; } else { /* Ghost... Iterate again. */ @@ -651,10 +639,10 @@ int performEvictions(void) { } else { kvs = db->expires; } - int slot = kvstoreGetFairRandomDictIndex(kvs); - de = kvstoreDictGetRandomKey(kvs, slot); - if (de) { - bestkey = dictGetKey(de); + int slot = kvstoreGetFairRandomHashtableIndex(kvs); + int found = kvstoreHashtableRandomEntry(kvs, slot, (void **)&valkey); + if (found) { + bestkey = objectGetKey(valkey); bestdbid = j; break; } diff --git a/src/expire.c b/src/expire.c index 928bb58d86..7edaf6203e 100644 --- a/src/expire.c +++ b/src/expire.c @@ -46,8 +46,7 @@ static double avg_ttl_factor[16] = {0.98, 0.9604, 0.941192, 0.922368, 0.903921, 0.833748, 0.817073, 0.800731, 0.784717, 0.769022, 0.753642, 0.738569, 0.723798}; /* Helper function for the activeExpireCycle() function. - * This function will try to expire the key that is stored in the hash table - * entry 'de' of the 'expires' hash table of a database. + * This function will try to expire the key-value entry 'val'. * * If the key is found to be expired, it is removed from the database and * 1 is returned. Otherwise no operation is performed and 0 is returned. @@ -56,11 +55,12 @@ static double avg_ttl_factor[16] = {0.98, 0.9604, 0.941192, 0.922368, 0.903921, * * The parameter 'now' is the current time in milliseconds as is passed * to the function to avoid too many gettimeofday() syscalls. */ -int activeExpireCycleTryExpire(serverDb *db, dictEntry *de, long long now) { - long long t = dictGetSignedIntegerVal(de); +int activeExpireCycleTryExpire(serverDb *db, valkey *val, long long now) { + long long t = objectGetExpire(val); + serverAssert(t >= 0); if (now > t) { enterExecutionUnit(1, 0); - sds key = dictGetKey(de); + sds key = objectGetKey(val); robj *keyobj = createStringObject(key, sdslen(key)); deleteExpiredKeyAndPropagate(db, keyobj); decrRefCount(keyobj); @@ -127,11 +127,11 @@ typedef struct { int ttl_samples; /* num keys with ttl not yet expired */ } expireScanData; -void expireScanCallback(void *privdata, const dictEntry *const_de) { - dictEntry *de = (dictEntry *)const_de; +void expireScanCallback(void *privdata, void *element) { + valkey *val = element; expireScanData *data = privdata; - long long ttl = dictGetSignedIntegerVal(de) - data->now; - if (activeExpireCycleTryExpire(data->db, de, data->now)) { + long long ttl = objectGetExpire(val) - data->now; + if (activeExpireCycleTryExpire(data->db, val, data->now)) { data->expired++; /* Propagate the DEL command */ postExecutionUnitOperations(); @@ -144,13 +144,13 @@ void expireScanCallback(void *privdata, const dictEntry *const_de) { data->sampled++; } -static inline int isExpiryDictValidForSamplingCb(dict *d) { - long long numkeys = dictSize(d); - unsigned long buckets = dictBuckets(d); +static inline int isExpiryTableValidForSamplingCb(hashtable *s) { + long long numkeys = hashtableSize(s); + unsigned long buckets = hashtableBuckets(s); /* When there are less than 1% filled buckets, sampling the key * space is expensive, so stop here waiting for better times... * The dictionary will be resized asap. */ - if (buckets > DICT_HT_INITIAL_SIZE && (numkeys * 100 / buckets < 1)) { + if (buckets > 0 && (numkeys * 100 / buckets < 1)) { return C_ERR; } return C_OK; @@ -279,14 +279,14 @@ void activeExpireCycle(int type) { * is very fast: we are in the cache line scanning a sequential * array of NULL pointers, so we can scan a lot more buckets * than keys in the same time. */ - long max_buckets = num * 20; + long max_buckets = num * 10; long checked_buckets = 0; int origin_ttl_samples = data.ttl_samples; while (data.sampled < num && checked_buckets < max_buckets) { db->expires_cursor = kvstoreScan(db->expires, db->expires_cursor, -1, expireScanCallback, - isExpiryDictValidForSamplingCb, &data); + isExpiryTableValidForSamplingCb, &data); if (db->expires_cursor == 0) { db_done = 1; break; @@ -422,7 +422,7 @@ void expireReplicaKeys(void) { while (dbids && dbid < server.dbnum) { if ((dbids & 1) != 0) { serverDb *db = server.db + dbid; - dictEntry *expire = dbFindExpires(db, keyname); + valkey *expire = dbFindExpires(db, keyname); int expired = 0; if (expire && activeExpireCycleTryExpire(server.db + dbid, expire, start)) { @@ -616,14 +616,16 @@ void expireGenericCommand(client *c, long long basetime, int unit) { } when += basetime; + robj *obj = lookupKeyWrite(c->db, key); + /* No key, return zero. */ - if (lookupKeyWrite(c->db, key) == NULL) { + if (obj == NULL) { addReply(c, shared.czero); return; } if (flag) { - current_expire = getExpire(c->db, key); + current_expire = objectGetExpire(obj); /* NX option is set, check current expiry */ if (flag & EXPIRE_NX) { @@ -671,7 +673,7 @@ void expireGenericCommand(client *c, long long basetime, int unit) { addReply(c, shared.cone); return; } else { - setExpire(c, c->db, key, when); + obj = setExpire(c, c->db, key, when); addReply(c, shared.cone); /* Propagate as PEXPIREAT millisecond-timestamp * Only rewrite the command arg if not already PEXPIREAT */ diff --git a/src/geo.c b/src/geo.c index 9e43a6e93b..8034e42826 100644 --- a/src/geo.c +++ b/src/geo.c @@ -780,8 +780,7 @@ void georadiusGeneric(client *c, int srcKeyIndex, int flags) { if (returned_items) { zsetConvertToListpackIfNeeded(zobj, maxelelen, totelelen); - setKey(c, c->db, storekey, zobj, 0); - decrRefCount(zobj); + zobj = setKey(c, c->db, storekey, zobj, 0); notifyKeyspaceEvent(NOTIFY_ZSET, flags & GEOSEARCH ? "geosearchstore" : "georadiusstore", storekey, c->db->id); server.dirty += returned_items; diff --git a/src/hyperloglog.c b/src/hyperloglog.c index 563c5e7941..fcdd492224 100644 --- a/src/hyperloglog.c +++ b/src/hyperloglog.c @@ -1189,7 +1189,7 @@ void pfaddCommand(client *c) { * hold our HLL data structure. sdsnewlen() when NULL is passed * is guaranteed to return bytes initialized to zero. */ o = createHLLObject(); - dbAdd(c->db, c->argv[1], o); + o = dbAdd(c->db, c->argv[1], o); updated++; } else { if (isHLLObjectOrReply(c, o) != C_OK) return; @@ -1346,7 +1346,7 @@ void pfmergeCommand(client *c) { * hold our HLL data structure. sdsnewlen() when NULL is passed * is guaranteed to return bytes initialized to zero. */ o = createHLLObject(); - dbAdd(c->db, c->argv[1], o); + o = dbAdd(c->db, c->argv[1], o); } else { /* If key exists we are sure it's of the right type/size * since we checked when merging the different HLLs, so we diff --git a/src/kvstore.c b/src/kvstore.c index 49662f330a..41489a6e3e 100644 --- a/src/kvstore.c +++ b/src/kvstore.c @@ -1,11 +1,11 @@ /* * Index-based KV store implementation - * This file implements a KV store comprised of an array of dicts (see dict.c) + * This file implements a KV store comprised of an array of hash tables (see hashtable.c) * The purpose of this KV store is to have easy access to all keys that belong - * in the same dict (i.e. are in the same dict-index) + * in the same hash table (i.e. are in the same hashtable-index) * * For example, when the server is running in cluster mode, we use kvstore to save - * all keys that map to the same hash-slot in a separate dict within the kvstore + * all keys that map to the same hash-slot in a separate hash table within the kvstore * struct. * This enables us to easily access all keys that map to a specific hash-slot. * @@ -40,6 +40,7 @@ #include #include +#include #include "zmalloc.h" #include "kvstore.h" @@ -48,236 +49,248 @@ #define UNUSED(V) ((void)V) -static dict *kvstoreIteratorNextDict(kvstoreIterator *kvs_it); +static hashtable *kvstoreIteratorNextHashtable(kvstoreIterator *kvs_it); struct _kvstore { int flags; - dictType *dtype; - dict **dicts; - int num_dicts; - int num_dicts_bits; - list *rehashing; /* List of dictionaries in this kvstore that are currently rehashing. */ - int resize_cursor; /* Cron job uses this cursor to gradually resize dictionaries (only used if num_dicts > 1). */ - int allocated_dicts; /* The number of allocated dicts. */ - int non_empty_dicts; /* The number of non-empty dicts. */ - unsigned long long key_count; /* Total number of keys in this kvstore. */ - unsigned long long bucket_count; /* Total number of buckets in this kvstore across dictionaries. */ - unsigned long long *dict_size_index; /* Binary indexed tree (BIT) that describes cumulative key frequencies up until - given dict-index. */ - size_t overhead_hashtable_lut; /* The overhead of all dictionaries. */ - size_t overhead_hashtable_rehashing; /* The overhead of dictionaries rehashing. */ + hashtableType *dtype; + hashtable **hashtables; + int num_hashtables; + int num_hashtables_bits; + list *rehashing; /* List of hash tables in this kvstore that are currently rehashing. */ + int resize_cursor; /* Cron job uses this cursor to gradually resize hash tables (only used if num_hashtables > 1). */ + int allocated_hashtables; /* The number of allocated hashtables. */ + int non_empty_hashtables; /* The number of non-empty hashtables. */ + unsigned long long key_count; /* Total number of keys in this kvstore. */ + unsigned long long bucket_count; /* Total number of buckets in this kvstore across hash tables. */ + unsigned long long *hashtable_size_index; /* Binary indexed tree (BIT) that describes cumulative key frequencies up until + * given hashtable-index. */ + size_t overhead_hashtable_lut; /* Overhead of all hashtables in bytes. */ + size_t overhead_hashtable_rehashing; /* Overhead of hash tables rehashing in bytes. */ }; -/* Structure for kvstore iterator that allows iterating across multiple dicts. */ +/* Structure for kvstore iterator that allows iterating across multiple hashtables. */ struct _kvstoreIterator { kvstore *kvs; long long didx; long long next_didx; - dictIterator di; + hashtableIterator di; }; -/* Structure for kvstore dict iterator that allows iterating the corresponding dict. */ -struct _kvstoreDictIterator { +/* Structure for kvstore hashtable iterator that allows iterating the corresponding hashtable. */ +struct _kvstoreHashtableIterator { kvstore *kvs; long long didx; - dictIterator di; + hashtableIterator di; }; -/* Dict metadata for database, used for record the position in rehashing list. */ +/* Hashtable metadata for database, used for record the position in rehashing list. */ typedef struct { listNode *rehashing_node; /* list node in rehashing list */ kvstore *kvs; -} kvstoreDictMetadata; +} kvstoreHashtableMetadata; /**********************************/ /*** Helpers **********************/ /**********************************/ -/* Get the dictionary pointer based on dict-index. */ -dict *kvstoreGetDict(kvstore *kvs, int didx) { - return kvs->dicts[didx]; +/* Get the hash table pointer based on hashtable-index. */ +hashtable *kvstoreGetHashtable(kvstore *kvs, int didx) { + return kvs->hashtables[didx]; } -static dict **kvstoreGetDictRef(kvstore *kvs, int didx) { - return &kvs->dicts[didx]; +static hashtable **kvstoreGetHashtableRef(kvstore *kvs, int didx) { + return &kvs->hashtables[didx]; } -static int kvstoreDictIsRehashingPaused(kvstore *kvs, int didx) { - dict *d = kvstoreGetDict(kvs, didx); - return d ? dictIsRehashingPaused(d) : 0; +static int kvstoreHashtableIsRehashingPaused(kvstore *kvs, int didx) { + hashtable *d = kvstoreGetHashtable(kvs, didx); + return d ? hashtableIsRehashingPaused(d) : 0; } -/* Returns total (cumulative) number of keys up until given dict-index (inclusive). - * Time complexity is O(log(kvs->num_dicts)). */ +/* Returns total (cumulative) number of keys up until given hashtable-index (inclusive). + * Time complexity is O(log(kvs->num_hashtables)). */ static unsigned long long cumulativeKeyCountRead(kvstore *kvs, int didx) { - if (kvs->num_dicts == 1) { + if (kvs->num_hashtables == 1) { assert(didx == 0); return kvstoreSize(kvs); } int idx = didx + 1; unsigned long long sum = 0; while (idx > 0) { - sum += kvs->dict_size_index[idx]; + sum += kvs->hashtable_size_index[idx]; idx -= (idx & -idx); } return sum; } -static void addDictIndexToCursor(kvstore *kvs, int didx, unsigned long long *cursor) { - if (kvs->num_dicts == 1) return; - /* didx can be -1 when iteration is over and there are no more dicts to visit. */ +static void addHashtableIndexToCursor(kvstore *kvs, int didx, unsigned long long *cursor) { + if (kvs->num_hashtables == 1) return; + /* didx can be -1 when iteration is over and there are no more hashtables to visit. */ if (didx < 0) return; - *cursor = (*cursor << kvs->num_dicts_bits) | didx; + *cursor = (*cursor << kvs->num_hashtables_bits) | didx; } -static int getAndClearDictIndexFromCursor(kvstore *kvs, unsigned long long *cursor) { - if (kvs->num_dicts == 1) return 0; - int didx = (int)(*cursor & (kvs->num_dicts - 1)); - *cursor = *cursor >> kvs->num_dicts_bits; +static int getAndClearHashtableIndexFromCursor(kvstore *kvs, unsigned long long *cursor) { + if (kvs->num_hashtables == 1) return 0; + int didx = (int)(*cursor & (kvs->num_hashtables - 1)); + *cursor = *cursor >> kvs->num_hashtables_bits; return didx; } -/* Updates binary index tree (also known as Fenwick tree), increasing key count for a given dict. +/* Updates binary index tree (also known as Fenwick tree), increasing key count for a given hashtable. * You can read more about this data structure here https://en.wikipedia.org/wiki/Fenwick_tree - * Time complexity is O(log(kvs->num_dicts)). */ + * Time complexity is O(log(kvs->num_hashtables)). */ static void cumulativeKeyCountAdd(kvstore *kvs, int didx, long delta) { kvs->key_count += delta; - dict *d = kvstoreGetDict(kvs, didx); - size_t dsize = dictSize(d); - int non_empty_dicts_delta = dsize == 1 ? 1 : dsize == 0 ? -1 - : 0; - kvs->non_empty_dicts += non_empty_dicts_delta; + hashtable *ht = kvstoreGetHashtable(kvs, didx); + size_t size = hashtableSize(ht); + if (delta < 0 && size == 0) { + kvs->non_empty_hashtables--; /* It became empty. */ + } else if (delta > 0 && size == (size_t)delta) { + kvs->non_empty_hashtables++; /* It was empty before. */ + } - /* BIT does not need to be calculated when there's only one dict. */ - if (kvs->num_dicts == 1) return; + /* BIT does not need to be calculated when there's only one hashtable. */ + if (kvs->num_hashtables == 1) return; /* Update the BIT */ - int idx = didx + 1; /* Unlike dict indices, BIT is 1-based, so we need to add 1. */ - while (idx <= kvs->num_dicts) { + int idx = didx + 1; /* Unlike hashtable indices, BIT is 1-based, so we need to add 1. */ + while (idx <= kvs->num_hashtables) { if (delta < 0) { - assert(kvs->dict_size_index[idx] >= (unsigned long long)labs(delta)); + assert(kvs->hashtable_size_index[idx] >= (unsigned long long)labs(delta)); } - kvs->dict_size_index[idx] += delta; + kvs->hashtable_size_index[idx] += delta; idx += (idx & -idx); } } -/* Create the dict if it does not exist and return it. */ -static dict *createDictIfNeeded(kvstore *kvs, int didx) { - dict *d = kvstoreGetDict(kvs, didx); - if (d) return d; +/* Create the hashtable if it does not exist and return it. */ +static hashtable *createHashtableIfNeeded(kvstore *kvs, int didx) { + hashtable *ht = kvstoreGetHashtable(kvs, didx); + if (ht) return ht; - kvs->dicts[didx] = dictCreate(kvs->dtype); - kvstoreDictMetadata *metadata = (kvstoreDictMetadata *)dictMetadata(kvs->dicts[didx]); + kvs->hashtables[didx] = hashtableCreate(kvs->dtype); + kvstoreHashtableMetadata *metadata = (kvstoreHashtableMetadata *)hashtableMetadata(kvs->hashtables[didx]); metadata->kvs = kvs; - kvs->allocated_dicts++; - return kvs->dicts[didx]; + /* Memory is counted by kvstoreHashtableTrackMemUsage, but when it's invoked + * by hashtableCreate above, we don't know which hashtable it is for, because + * the metadata has yet been initialized. Account for the newly created + * hashtable here instead. */ + kvs->overhead_hashtable_lut += hashtableMemUsage(kvs->hashtables[didx]); + kvs->allocated_hashtables++; + return kvs->hashtables[didx]; } -/* Called when the dict will delete entries, the function will check - * KVSTORE_FREE_EMPTY_DICTS to determine whether the empty dict needs +/* Called when the hashtable will delete entries, the function will check + * KVSTORE_FREE_EMPTY_HASHTABLES to determine whether the empty hashtable needs * to be freed. * - * Note that for rehashing dicts, that is, in the case of safe iterators - * and Scan, we won't delete the dict. We will check whether it needs + * Note that for rehashing hashtables, that is, in the case of safe iterators + * and Scan, we won't delete the hashtable. We will check whether it needs * to be deleted when we're releasing the iterator. */ -static void freeDictIfNeeded(kvstore *kvs, int didx) { - if (!(kvs->flags & KVSTORE_FREE_EMPTY_DICTS) || !kvstoreGetDict(kvs, didx) || kvstoreDictSize(kvs, didx) != 0 || - kvstoreDictIsRehashingPaused(kvs, didx)) +static void freeHashtableIfNeeded(kvstore *kvs, int didx) { + if (!(kvs->flags & KVSTORE_FREE_EMPTY_HASHTABLES) || !kvstoreGetHashtable(kvs, didx) || kvstoreHashtableSize(kvs, didx) != 0 || + kvstoreHashtableIsRehashingPaused(kvs, didx)) return; - dictRelease(kvs->dicts[didx]); - kvs->dicts[didx] = NULL; - kvs->allocated_dicts--; + hashtableRelease(kvs->hashtables[didx]); + kvs->hashtables[didx] = NULL; + kvs->allocated_hashtables--; } -/**********************************/ -/*** dict callbacks ***************/ -/**********************************/ +/*************************************/ +/*** hashtable callbacks ***************/ +/*************************************/ -/* Adds dictionary to the rehashing list, which allows us +/* Adds hash table to the rehashing list, which allows us * to quickly find rehash targets during incremental rehashing. * - * If there are multiple dicts, updates the bucket count for the given dictionary + * If there are multiple hashtables, updates the bucket count for the given hash table * in a DB, bucket count incremented with the new ht size during the rehashing phase. - * If there's one dict, bucket count can be retrieved directly from single dict bucket. */ -void kvstoreDictRehashingStarted(dict *d) { - kvstoreDictMetadata *metadata = (kvstoreDictMetadata *)dictMetadata(d); + * If there's one hashtable, bucket count can be retrieved directly from single hashtable bucket. */ +void kvstoreHashtableRehashingStarted(hashtable *d) { + kvstoreHashtableMetadata *metadata = (kvstoreHashtableMetadata *)hashtableMetadata(d); kvstore *kvs = metadata->kvs; listAddNodeTail(kvs->rehashing, d); metadata->rehashing_node = listLast(kvs->rehashing); - unsigned long long from, to; - dictRehashingInfo(d, &from, &to); + size_t from, to; + hashtableRehashingInfo(d, &from, &to); kvs->bucket_count += to; /* Started rehashing (Add the new ht size) */ - kvs->overhead_hashtable_lut += to; - kvs->overhead_hashtable_rehashing += from; + kvs->overhead_hashtable_rehashing += from * HASHTABLE_BUCKET_SIZE; } -/* Remove dictionary from the rehashing list. +/* Remove hash table from the rehashing list. * - * Updates the bucket count for the given dictionary in a DB. It removes - * the old ht size of the dictionary from the total sum of buckets for a DB. */ -void kvstoreDictRehashingCompleted(dict *d) { - kvstoreDictMetadata *metadata = (kvstoreDictMetadata *)dictMetadata(d); + * Updates the bucket count for the given hash table in a DB. It removes + * the old ht size of the hash table from the total sum of buckets for a DB. */ +void kvstoreHashtableRehashingCompleted(hashtable *d) { + kvstoreHashtableMetadata *metadata = (kvstoreHashtableMetadata *)hashtableMetadata(d); kvstore *kvs = metadata->kvs; if (metadata->rehashing_node) { listDelNode(kvs->rehashing, metadata->rehashing_node); metadata->rehashing_node = NULL; } - unsigned long long from, to; - dictRehashingInfo(d, &from, &to); + size_t from, to; + hashtableRehashingInfo(d, &from, &to); kvs->bucket_count -= from; /* Finished rehashing (Remove the old ht size) */ - kvs->overhead_hashtable_lut -= from; - kvs->overhead_hashtable_rehashing -= from; + kvs->overhead_hashtable_rehashing -= from * HASHTABLE_BUCKET_SIZE; +} + +/* Hashtable callback to keep track of memory usage. */ +void kvstoreHashtableTrackMemUsage(hashtable *ht, ssize_t delta) { + kvstoreHashtableMetadata *metadata = (kvstoreHashtableMetadata *)hashtableMetadata(ht); + if (metadata->kvs == NULL) { + /* This is the initial allocation by hashtableCreate, when the metadata + * hasn't been initialized yet. */ + return; + } + metadata->kvs->overhead_hashtable_lut += delta; } -/* Returns the size of the DB dict metadata in bytes. */ -size_t kvstoreDictMetadataSize(dict *d) { - UNUSED(d); - return sizeof(kvstoreDictMetadata); +/* Returns the size of the DB hashtable metadata in bytes. */ +size_t kvstoreHashtableMetadataSize(void) { + return sizeof(kvstoreHashtableMetadata); } /**********************************/ /*** API **************************/ /**********************************/ -/* Create an array of dictionaries - * num_dicts_bits is the log2 of the amount of dictionaries needed (e.g. 0 for 1 dict, - * 3 for 8 dicts, etc.) - * - * The kvstore handles `key` based on `dictType` during initialization: - * - If `dictType.embedded-entry` is 1, it clones the `key`. - * - Otherwise, it assumes ownership of the `key`. +/* Create an array of hash tables + * num_hashtables_bits is the log2 of the amount of hash tables needed (e.g. 0 for 1 hashtable, + * 3 for 8 hashtables, etc.) */ -kvstore *kvstoreCreate(dictType *type, int num_dicts_bits, int flags) { - /* We can't support more than 2^16 dicts because we want to save 48 bits - * for the dict cursor, see kvstoreScan */ - assert(num_dicts_bits <= 16); +kvstore *kvstoreCreate(hashtableType *type, int num_hashtables_bits, int flags) { + /* We can't support more than 2^16 hashtables because we want to save 48 bits + * for the hashtable cursor, see kvstoreScan */ + assert(num_hashtables_bits <= 16); - /* The dictType of kvstore needs to use the specific callbacks. + /* The hashtableType of kvstore needs to use the specific callbacks. * If there are any changes in the future, it will need to be modified. */ - assert(type->rehashingStarted == kvstoreDictRehashingStarted); - assert(type->rehashingCompleted == kvstoreDictRehashingCompleted); - assert(type->dictMetadataBytes == kvstoreDictMetadataSize); + assert(type->rehashingStarted == kvstoreHashtableRehashingStarted); + assert(type->rehashingCompleted == kvstoreHashtableRehashingCompleted); + assert(type->trackMemUsage == kvstoreHashtableTrackMemUsage); + assert(type->getMetadataSize == kvstoreHashtableMetadataSize); kvstore *kvs = zcalloc(sizeof(*kvs)); kvs->dtype = type; kvs->flags = flags; - kvs->num_dicts_bits = num_dicts_bits; - kvs->num_dicts = 1 << kvs->num_dicts_bits; - kvs->dicts = zcalloc(sizeof(dict *) * kvs->num_dicts); - if (!(kvs->flags & KVSTORE_ALLOCATE_DICTS_ON_DEMAND)) { - for (int i = 0; i < kvs->num_dicts; i++) createDictIfNeeded(kvs, i); + kvs->num_hashtables_bits = num_hashtables_bits; + kvs->num_hashtables = 1 << kvs->num_hashtables_bits; + kvs->hashtables = zcalloc(sizeof(hashtable *) * kvs->num_hashtables); + if (!(kvs->flags & KVSTORE_ALLOCATE_HASHTABLES_ON_DEMAND)) { + for (int i = 0; i < kvs->num_hashtables; i++) createHashtableIfNeeded(kvs, i); } kvs->rehashing = listCreate(); kvs->key_count = 0; - kvs->non_empty_dicts = 0; + kvs->non_empty_hashtables = 0; kvs->resize_cursor = 0; - kvs->dict_size_index = kvs->num_dicts > 1 ? zcalloc(sizeof(unsigned long long) * (kvs->num_dicts + 1)) : NULL; + kvs->hashtable_size_index = kvs->num_hashtables > 1 ? zcalloc(sizeof(unsigned long long) * (kvs->num_hashtables + 1)) : NULL; kvs->bucket_count = 0; kvs->overhead_hashtable_lut = 0; kvs->overhead_hashtable_rehashing = 0; @@ -285,105 +298,102 @@ kvstore *kvstoreCreate(dictType *type, int num_dicts_bits, int flags) { return kvs; } -void kvstoreEmpty(kvstore *kvs, void(callback)(dict *)) { - for (int didx = 0; didx < kvs->num_dicts; didx++) { - dict *d = kvstoreGetDict(kvs, didx); +void kvstoreEmpty(kvstore *kvs, void(callback)(hashtable *)) { + for (int didx = 0; didx < kvs->num_hashtables; didx++) { + hashtable *d = kvstoreGetHashtable(kvs, didx); if (!d) continue; - kvstoreDictMetadata *metadata = (kvstoreDictMetadata *)dictMetadata(d); + kvstoreHashtableMetadata *metadata = (kvstoreHashtableMetadata *)hashtableMetadata(d); if (metadata->rehashing_node) metadata->rehashing_node = NULL; - dictEmpty(d, callback); - freeDictIfNeeded(kvs, didx); + hashtableEmpty(d, callback); + freeHashtableIfNeeded(kvs, didx); } listEmpty(kvs->rehashing); kvs->key_count = 0; - kvs->non_empty_dicts = 0; + kvs->non_empty_hashtables = 0; kvs->resize_cursor = 0; kvs->bucket_count = 0; - if (kvs->dict_size_index) memset(kvs->dict_size_index, 0, sizeof(unsigned long long) * (kvs->num_dicts + 1)); - kvs->overhead_hashtable_lut = 0; + if (kvs->hashtable_size_index) memset(kvs->hashtable_size_index, 0, sizeof(unsigned long long) * (kvs->num_hashtables + 1)); kvs->overhead_hashtable_rehashing = 0; } void kvstoreRelease(kvstore *kvs) { - for (int didx = 0; didx < kvs->num_dicts; didx++) { - dict *d = kvstoreGetDict(kvs, didx); + for (int didx = 0; didx < kvs->num_hashtables; didx++) { + hashtable *d = kvstoreGetHashtable(kvs, didx); if (!d) continue; - kvstoreDictMetadata *metadata = (kvstoreDictMetadata *)dictMetadata(d); + kvstoreHashtableMetadata *metadata = (kvstoreHashtableMetadata *)hashtableMetadata(d); if (metadata->rehashing_node) metadata->rehashing_node = NULL; - dictRelease(d); + hashtableRelease(d); } - zfree(kvs->dicts); + assert(kvs->overhead_hashtable_lut == 0); + zfree(kvs->hashtables); listRelease(kvs->rehashing); - if (kvs->dict_size_index) zfree(kvs->dict_size_index); + if (kvs->hashtable_size_index) zfree(kvs->hashtable_size_index); zfree(kvs); } unsigned long long int kvstoreSize(kvstore *kvs) { - if (kvs->num_dicts != 1) { + if (kvs->num_hashtables != 1) { return kvs->key_count; } else { - return kvs->dicts[0] ? dictSize(kvs->dicts[0]) : 0; + return kvs->hashtables[0] ? hashtableSize(kvs->hashtables[0]) : 0; } } -/* This method provides the cumulative sum of all the dictionary buckets - * across dictionaries in a database. */ +/* This method provides the cumulative sum of all the hash table buckets + * across hash tables in a database. */ unsigned long kvstoreBuckets(kvstore *kvs) { - if (kvs->num_dicts != 1) { + if (kvs->num_hashtables != 1) { return kvs->bucket_count; } else { - return kvs->dicts[0] ? dictBuckets(kvs->dicts[0]) : 0; + return kvs->hashtables[0] ? hashtableBuckets(kvs->hashtables[0]) : 0; } } size_t kvstoreMemUsage(kvstore *kvs) { size_t mem = sizeof(*kvs); + mem += kvs->overhead_hashtable_lut; - unsigned long long keys_count = kvstoreSize(kvs); - mem += keys_count * dictEntryMemUsage(NULL) + kvstoreBuckets(kvs) * sizeof(dictEntry *) + - kvs->allocated_dicts * (sizeof(dict) + kvstoreDictMetadataSize(NULL)); - - /* Values are dict* shared with kvs->dicts */ + /* Values are hashtable* shared with kvs->hashtables */ mem += listLength(kvs->rehashing) * sizeof(listNode); - if (kvs->dict_size_index) mem += sizeof(unsigned long long) * (kvs->num_dicts + 1); + if (kvs->hashtable_size_index) mem += sizeof(unsigned long long) * (kvs->num_hashtables + 1); return mem; } /* - * This method is used to iterate over the elements of the entire kvstore specifically across dicts. + * This method is used to iterate over the elements of the entire kvstore specifically across hashtables. * It's a three pronged approach. * - * 1. It uses the provided cursor `cursor` to retrieve the dict index from it. - * 2. If the dictionary is in a valid state checked through the provided callback `dictScanValidFunction`, - * it performs a dictScan over the appropriate `keyType` dictionary of `db`. - * 3. If the dict is entirely scanned i.e. the cursor has reached 0, the next non empty dict is discovered. - * The dict information is embedded into the cursor and returned. + * 1. It uses the provided cursor `cursor` to retrieve the hashtable index from it. + * 2. If the hash table is in a valid state checked through the provided callback `hashtableScanValidFunction`, + * it performs a hashtableScan over the appropriate `keyType` hash table of `db`. + * 3. If the hashtable is entirely scanned i.e. the cursor has reached 0, the next non empty hashtable is discovered. + * The hashtable information is embedded into the cursor and returned. * - * To restrict the scan to a single dict, pass a valid dict index as + * To restrict the scan to a single hashtable, pass a valid hashtable index as * 'onlydidx', otherwise pass -1. */ unsigned long long kvstoreScan(kvstore *kvs, unsigned long long cursor, int onlydidx, - dictScanFunction *scan_cb, - kvstoreScanShouldSkipDict *skip_cb, + hashtableScanFunction scan_cb, + kvstoreScanShouldSkipHashtable *skip_cb, void *privdata) { - unsigned long long _cursor = 0; - /* During dictionary traversal, 48 upper bits in the cursor are used for positioning in the HT. - * Following lower bits are used for the dict index number, ranging from 0 to 2^num_dicts_bits-1. - * Dict index is always 0 at the start of iteration and can be incremented only if there are - * multiple dicts. */ - int didx = getAndClearDictIndexFromCursor(kvs, &cursor); + unsigned long long next_cursor = 0; + /* During hash table traversal, 48 upper bits in the cursor are used for positioning in the HT. + * Following lower bits are used for the hashtable index number, ranging from 0 to 2^num_hashtables_bits-1. + * Hashtable index is always 0 at the start of iteration and can be incremented only if there are + * multiple hashtables. */ + int didx = getAndClearHashtableIndexFromCursor(kvs, &cursor); if (onlydidx >= 0) { if (didx < onlydidx) { /* Fast-forward to onlydidx. */ - assert(onlydidx < kvs->num_dicts); + assert(onlydidx < kvs->num_hashtables); didx = onlydidx; cursor = 0; } else if (didx > onlydidx) { @@ -392,55 +402,60 @@ unsigned long long kvstoreScan(kvstore *kvs, } } - dict *d = kvstoreGetDict(kvs, didx); + hashtable *d = kvstoreGetHashtable(kvs, didx); int skip = !d || (skip_cb && skip_cb(d)); if (!skip) { - _cursor = dictScan(d, cursor, scan_cb, privdata); - /* In dictScan, scan_cb may delete entries (e.g., in active expire case). */ - freeDictIfNeeded(kvs, didx); + next_cursor = hashtableScan(d, cursor, scan_cb, privdata); + /* In hashtableScan, scan_cb may delete entries (e.g., in active expire case). */ + freeHashtableIfNeeded(kvs, didx); } - /* scanning done for the current dictionary or if the scanning wasn't possible, move to the next dict index. */ - if (_cursor == 0 || skip) { + /* scanning done for the current hash table or if the scanning wasn't possible, move to the next hashtable index. */ + if (next_cursor == 0 || skip) { if (onlydidx >= 0) return 0; - didx = kvstoreGetNextNonEmptyDictIndex(kvs, didx); + didx = kvstoreGetNextNonEmptyHashtableIndex(kvs, didx); } if (didx == -1) { return 0; } - addDictIndexToCursor(kvs, didx, &_cursor); - return _cursor; + addHashtableIndexToCursor(kvs, didx, &next_cursor); + return next_cursor; } /* * This functions increases size of kvstore to match desired number. - * It resizes all individual dictionaries, unless skip_cb indicates otherwise. + * It resizes all individual hash tables, unless skip_cb indicates otherwise. * - * Based on the parameter `try_expand`, appropriate dict expand API is invoked. - * if try_expand is set to 1, `dictTryExpand` is used else `dictExpand`. - * The return code is either `DICT_OK`/`DICT_ERR` for both the API(s). - * `DICT_OK` response is for successful expansion. However, `DICT_ERR` response signifies failure in allocation in - * `dictTryExpand` call and in case of `dictExpand` call it signifies no expansion was performed. + * Based on the parameter `try_expand`, appropriate hashtable expand API is invoked. + * if try_expand is set to 1, `hashtableTryExpand` is used else `hashtableExpand`. + * The return code is either 1 or 0 for both the API(s). + * 1 response is for successful expansion. However, 0 response signifies failure in allocation in + * `hashtableTryExpand` call and in case of `hashtableExpand` call it signifies no expansion was performed. */ -int kvstoreExpand(kvstore *kvs, uint64_t newsize, int try_expand, kvstoreExpandShouldSkipDictIndex *skip_cb) { +int kvstoreExpand(kvstore *kvs, uint64_t newsize, int try_expand, kvstoreExpandShouldSkipHashtableIndex *skip_cb) { if (newsize == 0) return 1; - for (int i = 0; i < kvs->num_dicts; i++) { + for (int i = 0; i < kvs->num_hashtables; i++) { if (skip_cb && skip_cb(i)) continue; - /* If the dictionary doesn't exist, create it */ - dict *d = createDictIfNeeded(kvs, i); - int result = try_expand ? dictTryExpand(d, newsize) : dictExpand(d, newsize); - if (try_expand && result == DICT_ERR) return 0; + /* If the hash table doesn't exist, create it. */ + hashtable *ht = createHashtableIfNeeded(kvs, i); + if (try_expand) { + if (!hashtableTryExpand(ht, newsize)) return 0; + } else { + hashtableExpand(ht, newsize); + } } return 1; } -/* Returns fair random dict index, probability of each dict being returned is proportional to the number of elements - * that dictionary holds. This function guarantees that it returns a dict-index of a non-empty dict, unless the entire - * kvstore is empty. Time complexity of this function is O(log(kvs->num_dicts)). */ -int kvstoreGetFairRandomDictIndex(kvstore *kvs) { - unsigned long target = kvstoreSize(kvs) ? (randomULong() % kvstoreSize(kvs)) + 1 : 0; - return kvstoreFindDictIndexByKeyIndex(kvs, target); +/* Returns fair random hashtable index, probability of each hashtable being + * returned is proportional to the number of elements that hash table holds. + * This function guarantees that it returns a hashtable-index of a non-empty + * hashtable, unless the entire kvstore is empty. Time complexity of this + * function is O(log(kvs->num_hashtables)). */ +int kvstoreGetFairRandomHashtableIndex(kvstore *kvs) { + unsigned long target = kvstoreSize(kvs) ? (random() % kvstoreSize(kvs)) + 1 : 0; + return kvstoreFindHashtableIndexByKeyIndex(kvs, target); } void kvstoreGetStats(kvstore *kvs, char *buf, size_t bufsize, int full) { @@ -449,40 +464,40 @@ void kvstoreGetStats(kvstore *kvs, char *buf, size_t bufsize, int full) { size_t l; char *orig_buf = buf; size_t orig_bufsize = bufsize; - dictStats *mainHtStats = NULL; - dictStats *rehashHtStats = NULL; - dict *d; + hashtableStats *mainHtStats = NULL; + hashtableStats *rehashHtStats = NULL; + hashtable *d; kvstoreIterator *kvs_it = kvstoreIteratorInit(kvs); - while ((d = kvstoreIteratorNextDict(kvs_it))) { - dictStats *stats = dictGetStatsHt(d, 0, full); + while ((d = kvstoreIteratorNextHashtable(kvs_it))) { + hashtableStats *stats = hashtableGetStatsHt(d, 0, full); if (!mainHtStats) { mainHtStats = stats; } else { - dictCombineStats(stats, mainHtStats); - dictFreeStats(stats); + hashtableCombineStats(stats, mainHtStats); + hashtableFreeStats(stats); } - if (dictIsRehashing(d)) { - stats = dictGetStatsHt(d, 1, full); + if (hashtableIsRehashing(d)) { + stats = hashtableGetStatsHt(d, 1, full); if (!rehashHtStats) { rehashHtStats = stats; } else { - dictCombineStats(stats, rehashHtStats); - dictFreeStats(stats); + hashtableCombineStats(stats, rehashHtStats); + hashtableFreeStats(stats); } } } kvstoreIteratorRelease(kvs_it); if (mainHtStats && bufsize > 0) { - l = dictGetStatsMsg(buf, bufsize, mainHtStats, full); - dictFreeStats(mainHtStats); + l = hashtableGetStatsMsg(buf, bufsize, mainHtStats, full); + hashtableFreeStats(mainHtStats); buf += l; bufsize -= l; } if (rehashHtStats && bufsize > 0) { - l = dictGetStatsMsg(buf, bufsize, rehashHtStats, full); - dictFreeStats(rehashHtStats); + l = hashtableGetStatsMsg(buf, bufsize, rehashHtStats, full); + hashtableFreeStats(rehashHtStats); buf += l; bufsize -= l; } @@ -490,142 +505,143 @@ void kvstoreGetStats(kvstore *kvs, char *buf, size_t bufsize, int full) { if (orig_bufsize) orig_buf[orig_bufsize - 1] = '\0'; } -/* Finds a dict containing target element in a key space ordered by dict index. - * Consider this example. Dictionaries are represented by brackets and keys by dots: +/* Finds a hashtable containing target element in a key space ordered by hashtable index. + * Consider this example. Hash Tables are represented by brackets and keys by dots: * #0 #1 #2 #3 #4 * [..][....][...][.......][.] * ^ * target * - * In this case dict #3 contains key that we are trying to find. + * In this case hashtable #3 contains key that we are trying to find. * - * The return value is 0 based dict-index, and the range of the target is [1..kvstoreSize], kvstoreSize inclusive. + * The return value is 0 based hashtable-index, and the range of the target is [1..kvstoreSize], kvstoreSize inclusive. * - * To find the dict, we start with the root node of the binary index tree and search through its children - * from the highest index (2^num_dicts_bits in our case) to the lowest index. At each node, we check if the target + * To find the hashtable, we start with the root node of the binary index tree and search through its children + * from the highest index (2^num_hashtables_bits in our case) to the lowest index. At each node, we check if the target * value is greater than the node's value. If it is, we remove the node's value from the target and recursively * search for the new target using the current node as the parent. - * Time complexity of this function is O(log(kvs->num_dicts)) + * Time complexity of this function is O(log(kvs->num_hashtables)) */ -int kvstoreFindDictIndexByKeyIndex(kvstore *kvs, unsigned long target) { - if (kvs->num_dicts == 1 || kvstoreSize(kvs) == 0) return 0; +int kvstoreFindHashtableIndexByKeyIndex(kvstore *kvs, unsigned long target) { + if (kvs->num_hashtables == 1 || kvstoreSize(kvs) == 0) return 0; assert(target <= kvstoreSize(kvs)); - int result = 0, bit_mask = 1 << kvs->num_dicts_bits; + int result = 0, bit_mask = 1 << kvs->num_hashtables_bits; for (int i = bit_mask; i != 0; i >>= 1) { int current = result + i; /* When the target index is greater than 'current' node value the we will update * the target and search in the 'current' node tree. */ - if (target > kvs->dict_size_index[current]) { - target -= kvs->dict_size_index[current]; + if (target > kvs->hashtable_size_index[current]) { + target -= kvs->hashtable_size_index[current]; result = current; } } - /* Adjust the result to get the correct dict: + /* Adjust the result to get the correct hashtable: * 1. result += 1; - * After the calculations, the index of target in dict_size_index should be the next one, + * After the calculations, the index of target in hashtable_size_index should be the next one, * so we should add 1. * 2. result -= 1; - * Unlike BIT(dict_size_index is 1-based), dict indices are 0-based, so we need to subtract 1. + * Unlike BIT(hashtable_size_index is 1-based), hashtable indices are 0-based, so we need to subtract 1. * As the addition and subtraction cancel each other out, we can simply return the result. */ return result; } -/* Wrapper for kvstoreFindDictIndexByKeyIndex to get the first non-empty dict index in the kvstore. */ -int kvstoreGetFirstNonEmptyDictIndex(kvstore *kvs) { - return kvstoreFindDictIndexByKeyIndex(kvs, 1); +/* Wrapper for kvstoreFindHashtableIndexByKeyIndex to get the first non-empty hashtable index in the kvstore. */ +int kvstoreGetFirstNonEmptyHashtableIndex(kvstore *kvs) { + return kvstoreFindHashtableIndexByKeyIndex(kvs, 1); } -/* Returns next non-empty dict index strictly after given one, or -1 if provided didx is the last one. */ -int kvstoreGetNextNonEmptyDictIndex(kvstore *kvs, int didx) { - if (kvs->num_dicts == 1) { +/* Returns next non-empty hashtable index strictly after given one, or -1 if provided didx is the last one. */ +int kvstoreGetNextNonEmptyHashtableIndex(kvstore *kvs, int didx) { + if (kvs->num_hashtables == 1) { assert(didx == 0); return -1; } unsigned long long next_key = cumulativeKeyCountRead(kvs, didx) + 1; - return next_key <= kvstoreSize(kvs) ? kvstoreFindDictIndexByKeyIndex(kvs, next_key) : -1; + return next_key <= kvstoreSize(kvs) ? kvstoreFindHashtableIndexByKeyIndex(kvs, next_key) : -1; } -int kvstoreNumNonEmptyDicts(kvstore *kvs) { - return kvs->non_empty_dicts; +int kvstoreNumNonEmptyHashtables(kvstore *kvs) { + return kvs->non_empty_hashtables; } -int kvstoreNumAllocatedDicts(kvstore *kvs) { - return kvs->allocated_dicts; +int kvstoreNumAllocatedHashtables(kvstore *kvs) { + return kvs->allocated_hashtables; } -int kvstoreNumDicts(kvstore *kvs) { - return kvs->num_dicts; +int kvstoreNumHashtables(kvstore *kvs) { + return kvs->num_hashtables; } -/* Returns kvstore iterator that can be used to iterate through sub-dictionaries. +/* Returns kvstore iterator that can be used to iterate through sub-hash tables. * * The caller should free the resulting kvs_it with kvstoreIteratorRelease. */ kvstoreIterator *kvstoreIteratorInit(kvstore *kvs) { kvstoreIterator *kvs_it = zmalloc(sizeof(*kvs_it)); kvs_it->kvs = kvs; kvs_it->didx = -1; - kvs_it->next_didx = kvstoreGetFirstNonEmptyDictIndex(kvs_it->kvs); /* Finds first non-empty dict index. */ - dictInitSafeIterator(&kvs_it->di, NULL); + kvs_it->next_didx = kvstoreGetFirstNonEmptyHashtableIndex(kvs_it->kvs); /* Finds first non-empty hashtable index. */ + hashtableInitSafeIterator(&kvs_it->di, NULL); return kvs_it; } /* Free the kvs_it returned by kvstoreIteratorInit. */ void kvstoreIteratorRelease(kvstoreIterator *kvs_it) { - dictIterator *iter = &kvs_it->di; - dictResetIterator(iter); + hashtableIterator *iter = &kvs_it->di; + hashtableResetIterator(iter); /* In the safe iterator context, we may delete entries. */ - freeDictIfNeeded(kvs_it->kvs, kvs_it->didx); + freeHashtableIfNeeded(kvs_it->kvs, kvs_it->didx); zfree(kvs_it); } -/* Returns next dictionary from the iterator, or NULL if iteration is complete. */ -static dict *kvstoreIteratorNextDict(kvstoreIterator *kvs_it) { +/* Returns next hash table from the iterator, or NULL if iteration is complete. */ +static hashtable *kvstoreIteratorNextHashtable(kvstoreIterator *kvs_it) { if (kvs_it->next_didx == -1) return NULL; - /* The dict may be deleted during the iteration process, so here need to check for NULL. */ - if (kvs_it->didx != -1 && kvstoreGetDict(kvs_it->kvs, kvs_it->didx)) { - /* Before we move to the next dict, reset the iter of the previous dict. */ - dictIterator *iter = &kvs_it->di; - dictResetIterator(iter); + /* The hashtable may be deleted during the iteration process, so here need to check for NULL. */ + if (kvs_it->didx != -1 && kvstoreGetHashtable(kvs_it->kvs, kvs_it->didx)) { + /* Before we move to the next hashtable, reset the iter of the previous hashtable. */ + hashtableIterator *iter = &kvs_it->di; + hashtableResetIterator(iter); /* In the safe iterator context, we may delete entries. */ - freeDictIfNeeded(kvs_it->kvs, kvs_it->didx); + freeHashtableIfNeeded(kvs_it->kvs, kvs_it->didx); } kvs_it->didx = kvs_it->next_didx; - kvs_it->next_didx = kvstoreGetNextNonEmptyDictIndex(kvs_it->kvs, kvs_it->didx); - return kvs_it->kvs->dicts[kvs_it->didx]; + kvs_it->next_didx = kvstoreGetNextNonEmptyHashtableIndex(kvs_it->kvs, kvs_it->didx); + return kvs_it->kvs->hashtables[kvs_it->didx]; } -int kvstoreIteratorGetCurrentDictIndex(kvstoreIterator *kvs_it) { - assert(kvs_it->didx >= 0 && kvs_it->didx < kvs_it->kvs->num_dicts); +int kvstoreIteratorGetCurrentHashtableIndex(kvstoreIterator *kvs_it) { + assert(kvs_it->didx >= 0 && kvs_it->didx < kvs_it->kvs->num_hashtables); return kvs_it->didx; } -/* Returns next entry. */ -dictEntry *kvstoreIteratorNext(kvstoreIterator *kvs_it) { - dictEntry *de = kvs_it->di.d ? dictNext(&kvs_it->di) : NULL; - if (!de) { /* No current dict or reached the end of the dictionary. */ - dict *d = kvstoreIteratorNextDict(kvs_it); - if (!d) return NULL; - dictInitSafeIterator(&kvs_it->di, d); - de = dictNext(&kvs_it->di); +/* Fetches the next element and returns 1. Returns 0 if there are no more elements. */ +int kvstoreIteratorNext(kvstoreIterator *kvs_it, void **next) { + if (kvs_it->didx != -1 && hashtableNext(&kvs_it->di, next)) { + return 1; + } else { + /* No current hashtable or reached the end of the hash table. */ + hashtable *d = kvstoreIteratorNextHashtable(kvs_it); + if (!d) return 0; + hashtableInitSafeIterator(&kvs_it->di, d); + return hashtableNext(&kvs_it->di, next); } - return de; } -/* This method traverses through kvstore dictionaries and triggers a resize. +/* This method traverses through kvstore hash tables and triggers a resize. * It first tries to shrink if needed, and if it isn't, it tries to expand. */ -void kvstoreTryResizeDicts(kvstore *kvs, int limit) { - if (limit > kvs->num_dicts) limit = kvs->num_dicts; +void kvstoreTryResizeHashtables(kvstore *kvs, int limit) { + if (limit > kvs->num_hashtables) limit = kvs->num_hashtables; for (int i = 0; i < limit; i++) { int didx = kvs->resize_cursor; - dict *d = kvstoreGetDict(kvs, didx); - if (d && dictShrinkIfNeeded(d) == DICT_ERR) { - dictExpandIfNeeded(d); + hashtable *d = kvstoreGetHashtable(kvs, didx); + if (d && !hashtableShrinkIfNeeded(d)) { + hashtableExpandIfNeeded(d); } - kvs->resize_cursor = (didx + 1) % kvs->num_dicts; + kvs->resize_cursor = (didx + 1) % kvs->num_hashtables; } } @@ -639,14 +655,14 @@ void kvstoreTryResizeDicts(kvstore *kvs, int limit) { uint64_t kvstoreIncrementallyRehash(kvstore *kvs, uint64_t threshold_us) { if (listLength(kvs->rehashing) == 0) return 0; - /* Our goal is to rehash as many dictionaries as we can before reaching threshold_us, - * after each dictionary completes rehashing, it removes itself from the list. */ + /* Our goal is to rehash as many hash tables as we can before reaching threshold_us, + * after each hash table completes rehashing, it removes itself from the list. */ listNode *node; monotime timer; uint64_t elapsed_us = 0; elapsedStart(&timer); while ((node = listFirst(kvs->rehashing))) { - dictRehashMicroseconds(listNodeValue(node), threshold_us - elapsed_us); + hashtableRehashMicroseconds(listNodeValue(node), threshold_us - elapsed_us); elapsed_us = elapsedUs(timer); if (elapsed_us >= threshold_us) { @@ -656,107 +672,114 @@ uint64_t kvstoreIncrementallyRehash(kvstore *kvs, uint64_t threshold_us) { return elapsed_us; } +/* Size in bytes of hash tables used by the hashtables. */ size_t kvstoreOverheadHashtableLut(kvstore *kvs) { - return kvs->overhead_hashtable_lut * sizeof(dictEntry *); + return kvs->overhead_hashtable_lut; } size_t kvstoreOverheadHashtableRehashing(kvstore *kvs) { - return kvs->overhead_hashtable_rehashing * sizeof(dictEntry *); + return kvs->overhead_hashtable_rehashing; } -unsigned long kvstoreDictRehashingCount(kvstore *kvs) { +unsigned long kvstoreHashtableRehashingCount(kvstore *kvs) { return listLength(kvs->rehashing); } -unsigned long kvstoreDictSize(kvstore *kvs, int didx) { - dict *d = kvstoreGetDict(kvs, didx); +unsigned long kvstoreHashtableSize(kvstore *kvs, int didx) { + hashtable *d = kvstoreGetHashtable(kvs, didx); if (!d) return 0; - return dictSize(d); + return hashtableSize(d); } -kvstoreDictIterator *kvstoreGetDictIterator(kvstore *kvs, int didx) { - kvstoreDictIterator *kvs_di = zmalloc(sizeof(*kvs_di)); +kvstoreHashtableIterator *kvstoreGetHashtableIterator(kvstore *kvs, int didx) { + kvstoreHashtableIterator *kvs_di = zmalloc(sizeof(*kvs_di)); kvs_di->kvs = kvs; kvs_di->didx = didx; - dictInitIterator(&kvs_di->di, kvstoreGetDict(kvs, didx)); + hashtableInitIterator(&kvs_di->di, kvstoreGetHashtable(kvs, didx)); return kvs_di; } -kvstoreDictIterator *kvstoreGetDictSafeIterator(kvstore *kvs, int didx) { - kvstoreDictIterator *kvs_di = zmalloc(sizeof(*kvs_di)); +kvstoreHashtableIterator *kvstoreGetHashtableSafeIterator(kvstore *kvs, int didx) { + kvstoreHashtableIterator *kvs_di = zmalloc(sizeof(*kvs_di)); kvs_di->kvs = kvs; kvs_di->didx = didx; - dictInitSafeIterator(&kvs_di->di, kvstoreGetDict(kvs, didx)); + hashtableInitSafeIterator(&kvs_di->di, kvstoreGetHashtable(kvs, didx)); return kvs_di; } -/* Free the kvs_di returned by kvstoreGetDictIterator and kvstoreGetDictSafeIterator. */ -void kvstoreReleaseDictIterator(kvstoreDictIterator *kvs_di) { - /* The dict may be deleted during the iteration process, so here need to check for NULL. */ - if (kvstoreGetDict(kvs_di->kvs, kvs_di->didx)) { - dictResetIterator(&kvs_di->di); +/* Free the kvs_di returned by kvstoreGetHashtableIterator and kvstoreGetHashtableSafeIterator. */ +void kvstoreReleaseHashtableIterator(kvstoreHashtableIterator *kvs_di) { + /* The hashtable may be deleted during the iteration process, so here need to check for NULL. */ + if (kvstoreGetHashtable(kvs_di->kvs, kvs_di->didx)) { + hashtableResetIterator(&kvs_di->di); /* In the safe iterator context, we may delete entries. */ - freeDictIfNeeded(kvs_di->kvs, kvs_di->didx); + freeHashtableIfNeeded(kvs_di->kvs, kvs_di->didx); } zfree(kvs_di); } -/* Get the next element of the dict through kvstoreDictIterator and dictNext. */ -dictEntry *kvstoreDictIteratorNext(kvstoreDictIterator *kvs_di) { - /* The dict may be deleted during the iteration process, so here need to check for NULL. */ - dict *d = kvstoreGetDict(kvs_di->kvs, kvs_di->didx); - if (!d) return NULL; - - return dictNext(&kvs_di->di); +/* Get the next element of the hashtable through kvstoreHashtableIterator and hashtableNext. */ +int kvstoreHashtableIteratorNext(kvstoreHashtableIterator *kvs_di, void **next) { + /* The hashtable may be deleted during the iteration process, so here need to check for NULL. */ + hashtable *ht = kvstoreGetHashtable(kvs_di->kvs, kvs_di->didx); + if (!ht) return 0; + return hashtableNext(&kvs_di->di, next); } -dictEntry *kvstoreDictGetRandomKey(kvstore *kvs, int didx) { - dict *d = kvstoreGetDict(kvs, didx); - if (!d) return NULL; - return dictGetRandomKey(d); +int kvstoreHashtableRandomEntry(kvstore *kvs, int didx, void **entry) { + hashtable *ht = kvstoreGetHashtable(kvs, didx); + if (!ht) return 0; + return hashtableRandomEntry(ht, entry); } -dictEntry *kvstoreDictGetFairRandomKey(kvstore *kvs, int didx) { - dict *d = kvstoreGetDict(kvs, didx); - if (!d) return NULL; - return dictGetFairRandomKey(d); +int kvstoreHashtableFairRandomEntry(kvstore *kvs, int didx, void **entry) { + hashtable *ht = kvstoreGetHashtable(kvs, didx); + if (!ht) return 0; + return hashtableFairRandomEntry(ht, entry); } -unsigned int kvstoreDictGetSomeKeys(kvstore *kvs, int didx, dictEntry **des, unsigned int count) { - dict *d = kvstoreGetDict(kvs, didx); - if (!d) return 0; - return dictGetSomeKeys(d, des, count); +unsigned int kvstoreHashtableSampleEntries(kvstore *kvs, int didx, void **dst, unsigned int count) { + hashtable *ht = kvstoreGetHashtable(kvs, didx); + if (!ht) return 0; + return hashtableSampleEntries(ht, dst, count); } -int kvstoreDictExpand(kvstore *kvs, int didx, unsigned long size) { - dict *d = kvstoreGetDict(kvs, didx); - if (!d) return DICT_ERR; - return dictExpand(d, size); +int kvstoreHashtableExpand(kvstore *kvs, int didx, unsigned long size) { + hashtable *ht = kvstoreGetHashtable(kvs, didx); + if (!ht) return 0; + return hashtableExpand(ht, size); } -unsigned long kvstoreDictScanDefrag(kvstore *kvs, - int didx, - unsigned long v, - dictScanFunction *fn, - dictDefragFunctions *defragfns, - void *privdata) { - dict *d = kvstoreGetDict(kvs, didx); - if (!d) return 0; - return dictScanDefrag(d, v, fn, defragfns, privdata); +unsigned long kvstoreHashtableScanDefrag(kvstore *kvs, + int didx, + unsigned long v, + hashtableScanFunction fn, + void *privdata, + void *(*defragfn)(void *), + int flags) { + hashtable *ht = kvstoreGetHashtable(kvs, didx); + if (!ht) return 0; + return hashtableScanDefrag(ht, v, fn, privdata, defragfn, flags); } -/* Unlike kvstoreDictScanDefrag(), this method doesn't defrag the data(keys and values) - * within dict, it only reallocates the memory used by the dict structure itself using - * the provided allocation function. This feature was added for the active defrag feature. +/* This function doesn't defrag the data (keys and values) within hashtable. It + * only reallocates the memory used by the hashtable structure itself using the + * provided allocation function. This feature was added for the active defrag + * feature. * - * The 'defragfn' callback is called with a reference to the dict - * that callback can reallocate. */ -void kvstoreDictLUTDefrag(kvstore *kvs, kvstoreDictLUTDefragFunction *defragfn) { - for (int didx = 0; didx < kvs->num_dicts; didx++) { - dict **d = kvstoreGetDictRef(kvs, didx), *newd; - if (!*d) continue; - if ((newd = defragfn(*d))) *d = newd; + * The provided defragfn callback should either return NULL (if reallocation is + * not necessary) or reallocate the memory like realloc() would do. */ +void kvstoreHashtableDefragTables(kvstore *kvs, void *(*defragfn)(void *)) { + for (int didx = 0; didx < kvs->num_hashtables; didx++) { + hashtable **ref = kvstoreGetHashtableRef(kvs, didx), *new; + if (!*ref) continue; + new = hashtableDefragTables(*ref, defragfn); + if (new) { + *ref = new; + kvstoreHashtableMetadata *metadata = hashtableMetadata(new); + if (metadata->rehashing_node) metadata->rehashing_node->value = new; + } } } @@ -764,68 +787,77 @@ uint64_t kvstoreGetHash(kvstore *kvs, const void *key) { return kvs->dtype->hashFunction(key); } -void *kvstoreDictFetchValue(kvstore *kvs, int didx, const void *key) { - dict *d = kvstoreGetDict(kvs, didx); - if (!d) return NULL; - return dictFetchValue(d, key); +int kvstoreHashtableFind(kvstore *kvs, int didx, void *key, void **found) { + hashtable *ht = kvstoreGetHashtable(kvs, didx); + if (!ht) return 0; + return hashtableFind(ht, key, found); } -dictEntry *kvstoreDictFind(kvstore *kvs, int didx, void *key) { - dict *d = kvstoreGetDict(kvs, didx); - if (!d) return NULL; - return dictFind(d, key); +void **kvstoreHashtableFindRef(kvstore *kvs, int didx, const void *key) { + hashtable *ht = kvstoreGetHashtable(kvs, didx); + if (!ht) return NULL; + return hashtableFindRef(ht, key); } -/* - * The kvstore handles `key` based on `dictType` during initialization: - * - If `dictType.embedded-entry` is 1, it clones the `key`. - * - Otherwise, it assumes ownership of the `key`. - * The caller must ensure the `key` is properly freed. - * - * kvstore current usage: - * - * 1. keyspace (db.keys) kvstore - creates a copy of the key. - * 2. expiry (db.expires), pubsub_channels and pubsubshard_channels kvstore - takes ownership of the key. - */ -dictEntry *kvstoreDictAddRaw(kvstore *kvs, int didx, void *key, dictEntry **existing) { - dict *d = createDictIfNeeded(kvs, didx); - dictEntry *ret = dictAddRaw(d, key, existing); +/* was AddRaw */ +int kvstoreHashtableAddOrFind(kvstore *kvs, int didx, void *key, void **existing) { + hashtable *ht = createHashtableIfNeeded(kvs, didx); + int ret = hashtableAddOrFind(ht, key, existing); if (ret) cumulativeKeyCountAdd(kvs, didx, 1); return ret; } -void kvstoreDictSetKey(kvstore *kvs, int didx, dictEntry *de, void *key) { - dict *d = kvstoreGetDict(kvs, didx); - dictSetKey(d, de, key); +int kvstoreHashtableAdd(kvstore *kvs, int didx, void *entry) { + hashtable *ht = createHashtableIfNeeded(kvs, didx); + int ret = hashtableAdd(ht, entry); + if (ret) cumulativeKeyCountAdd(kvs, didx, 1); + return ret; } -void kvstoreDictSetVal(kvstore *kvs, int didx, dictEntry *de, void *val) { - UNUSED(kvs); - UNUSED(didx); - dictSetVal(NULL, de, val); +int kvstoreHashtableFindPositionForInsert(kvstore *kvs, int didx, void *key, hashtablePosition *position, void **existing) { + hashtable *ht = createHashtableIfNeeded(kvs, didx); + return hashtableFindPositionForInsert(ht, key, position, existing); } -dictEntry * -kvstoreDictTwoPhaseUnlinkFind(kvstore *kvs, int didx, const void *key, dictEntry ***plink, int *table_index) { - dict *d = kvstoreGetDict(kvs, didx); - if (!d) return NULL; - return dictTwoPhaseUnlinkFind(kvstoreGetDict(kvs, didx), key, plink, table_index); +/* Must be used together with kvstoreHashtableFindPositionForInsert, with returned + * position and with the same didx. */ +void kvstoreHashtableInsertAtPosition(kvstore *kvs, int didx, void *entry, void *position) { + hashtable *ht = kvstoreGetHashtable(kvs, didx); + hashtableInsertAtPosition(ht, entry, position); + cumulativeKeyCountAdd(kvs, didx, 1); } -void kvstoreDictTwoPhaseUnlinkFree(kvstore *kvs, int didx, dictEntry *he, dictEntry **plink, int table_index) { - dict *d = kvstoreGetDict(kvs, didx); - dictTwoPhaseUnlinkFree(d, he, plink, table_index); +void **kvstoreHashtableTwoPhasePopFindRef(kvstore *kvs, int didx, const void *key, void *position) { + hashtable *ht = kvstoreGetHashtable(kvs, didx); + if (!ht) return NULL; + return hashtableTwoPhasePopFindRef(ht, key, position); +} + +void kvstoreHashtableTwoPhasePopDelete(kvstore *kvs, int didx, void *position) { + hashtable *ht = kvstoreGetHashtable(kvs, didx); + hashtableTwoPhasePopDelete(ht, position); cumulativeKeyCountAdd(kvs, didx, -1); - freeDictIfNeeded(kvs, didx); + freeHashtableIfNeeded(kvs, didx); +} + +int kvstoreHashtablePop(kvstore *kvs, int didx, const void *key, void **popped) { + hashtable *ht = kvstoreGetHashtable(kvs, didx); + if (!ht) return 0; + int ret = hashtablePop(ht, key, popped); + if (ret) { + cumulativeKeyCountAdd(kvs, didx, -1); + freeHashtableIfNeeded(kvs, didx); + } + return ret; } -int kvstoreDictDelete(kvstore *kvs, int didx, const void *key) { - dict *d = kvstoreGetDict(kvs, didx); - if (!d) return DICT_ERR; - int ret = dictDelete(d, key); - if (ret == DICT_OK) { +int kvstoreHashtableDelete(kvstore *kvs, int didx, const void *key) { + hashtable *ht = kvstoreGetHashtable(kvs, didx); + if (!ht) return 0; + int ret = hashtableDelete(ht, key); + if (ret) { cumulativeKeyCountAdd(kvs, didx, -1); - freeDictIfNeeded(kvs, didx); + freeHashtableIfNeeded(kvs, didx); } return ret; } diff --git a/src/kvstore.h b/src/kvstore.h index 81a0d9a96e..ed1fec6d64 100644 --- a/src/kvstore.h +++ b/src/kvstore.h @@ -1,20 +1,20 @@ -#ifndef DICTARRAY_H_ -#define DICTARRAY_H_ +#ifndef KVSTORE_H +#define KVSTORE_H -#include "dict.h" +#include "hashtable.h" #include "adlist.h" typedef struct _kvstore kvstore; typedef struct _kvstoreIterator kvstoreIterator; -typedef struct _kvstoreDictIterator kvstoreDictIterator; +typedef struct _kvstoreHashtableIterator kvstoreHashtableIterator; -typedef int(kvstoreScanShouldSkipDict)(dict *d); -typedef int(kvstoreExpandShouldSkipDictIndex)(int didx); +typedef int(kvstoreScanShouldSkipHashtable)(hashtable *d); +typedef int(kvstoreExpandShouldSkipHashtableIndex)(int didx); -#define KVSTORE_ALLOCATE_DICTS_ON_DEMAND (1 << 0) -#define KVSTORE_FREE_EMPTY_DICTS (1 << 1) -kvstore *kvstoreCreate(dictType *type, int num_dicts_bits, int flags); -void kvstoreEmpty(kvstore *kvs, void(callback)(dict *)); +#define KVSTORE_ALLOCATE_HASHTABLES_ON_DEMAND (1 << 0) +#define KVSTORE_FREE_EMPTY_HASHTABLES (1 << 1) +kvstore *kvstoreCreate(hashtableType *type, int num_hashtables_bits, int flags); +void kvstoreEmpty(kvstore *kvs, void(callback)(hashtable *)); void kvstoreRelease(kvstore *kvs); unsigned long long kvstoreSize(kvstore *kvs); unsigned long kvstoreBuckets(kvstore *kvs); @@ -22,64 +22,69 @@ size_t kvstoreMemUsage(kvstore *kvs); unsigned long long kvstoreScan(kvstore *kvs, unsigned long long cursor, int onlydidx, - dictScanFunction *scan_cb, - kvstoreScanShouldSkipDict *skip_cb, + hashtableScanFunction scan_cb, + kvstoreScanShouldSkipHashtable *skip_cb, void *privdata); -int kvstoreExpand(kvstore *kvs, uint64_t newsize, int try_expand, kvstoreExpandShouldSkipDictIndex *skip_cb); -int kvstoreGetFairRandomDictIndex(kvstore *kvs); +int kvstoreExpand(kvstore *kvs, uint64_t newsize, int try_expand, kvstoreExpandShouldSkipHashtableIndex *skip_cb); +int kvstoreGetFairRandomHashtableIndex(kvstore *kvs); void kvstoreGetStats(kvstore *kvs, char *buf, size_t bufsize, int full); -int kvstoreFindDictIndexByKeyIndex(kvstore *kvs, unsigned long target); -int kvstoreGetFirstNonEmptyDictIndex(kvstore *kvs); -int kvstoreGetNextNonEmptyDictIndex(kvstore *kvs, int didx); -int kvstoreNumNonEmptyDicts(kvstore *kvs); -int kvstoreNumAllocatedDicts(kvstore *kvs); -int kvstoreNumDicts(kvstore *kvs); +int kvstoreFindHashtableIndexByKeyIndex(kvstore *kvs, unsigned long target); +int kvstoreGetFirstNonEmptyHashtableIndex(kvstore *kvs); +int kvstoreGetNextNonEmptyHashtableIndex(kvstore *kvs, int didx); +int kvstoreNumNonEmptyHashtables(kvstore *kvs); +int kvstoreNumAllocatedHashtables(kvstore *kvs); +int kvstoreNumHashtables(kvstore *kvs); uint64_t kvstoreGetHash(kvstore *kvs, const void *key); -void kvstoreDictRehashingStarted(dict *d); -void kvstoreDictRehashingCompleted(dict *d); -size_t kvstoreDictMetadataSize(dict *d); +void kvstoreHashtableRehashingStarted(hashtable *d); +void kvstoreHashtableRehashingCompleted(hashtable *d); +void kvstoreHashtableTrackMemUsage(hashtable *s, ssize_t delta); +size_t kvstoreHashtableMetadataSize(void); /* kvstore iterator specific functions */ kvstoreIterator *kvstoreIteratorInit(kvstore *kvs); void kvstoreIteratorRelease(kvstoreIterator *kvs_it); -int kvstoreIteratorGetCurrentDictIndex(kvstoreIterator *kvs_it); -dictEntry *kvstoreIteratorNext(kvstoreIterator *kvs_it); +int kvstoreIteratorGetCurrentHashtableIndex(kvstoreIterator *kvs_it); +int kvstoreIteratorNext(kvstoreIterator *kvs_it, void **next); /* Rehashing */ -void kvstoreTryResizeDicts(kvstore *kvs, int limit); +void kvstoreTryResizeHashtables(kvstore *kvs, int limit); uint64_t kvstoreIncrementallyRehash(kvstore *kvs, uint64_t threshold_us); size_t kvstoreOverheadHashtableLut(kvstore *kvs); size_t kvstoreOverheadHashtableRehashing(kvstore *kvs); -unsigned long kvstoreDictRehashingCount(kvstore *kvs); +unsigned long kvstoreHashtableRehashingCount(kvstore *kvs); -/* Specific dict access by dict-index */ -unsigned long kvstoreDictSize(kvstore *kvs, int didx); -kvstoreDictIterator *kvstoreGetDictIterator(kvstore *kvs, int didx); -kvstoreDictIterator *kvstoreGetDictSafeIterator(kvstore *kvs, int didx); -void kvstoreReleaseDictIterator(kvstoreDictIterator *kvs_id); -dictEntry *kvstoreDictIteratorNext(kvstoreDictIterator *kvs_di); -dictEntry *kvstoreDictGetRandomKey(kvstore *kvs, int didx); -dictEntry *kvstoreDictGetFairRandomKey(kvstore *kvs, int didx); -unsigned int kvstoreDictGetSomeKeys(kvstore *kvs, int didx, dictEntry **des, unsigned int count); -int kvstoreDictExpand(kvstore *kvs, int didx, unsigned long size); -unsigned long kvstoreDictScanDefrag(kvstore *kvs, - int didx, - unsigned long v, - dictScanFunction *fn, - dictDefragFunctions *defragfns, - void *privdata); -typedef dict *(kvstoreDictLUTDefragFunction)(dict *d); -void kvstoreDictLUTDefrag(kvstore *kvs, kvstoreDictLUTDefragFunction *defragfn); -void *kvstoreDictFetchValue(kvstore *kvs, int didx, const void *key); -dictEntry *kvstoreDictFind(kvstore *kvs, int didx, void *key); -dictEntry *kvstoreDictAddRaw(kvstore *kvs, int didx, void *key, dictEntry **existing); -void kvstoreDictSetKey(kvstore *kvs, int didx, dictEntry *de, void *key); -void kvstoreDictSetVal(kvstore *kvs, int didx, dictEntry *de, void *val); -dictEntry *kvstoreDictTwoPhaseUnlinkFind(kvstore *kvs, int didx, const void *key, dictEntry ***plink, int *table_index); -void kvstoreDictTwoPhaseUnlinkFree(kvstore *kvs, int didx, dictEntry *he, dictEntry **plink, int table_index); -int kvstoreDictDelete(kvstore *kvs, int didx, const void *key); -dict *kvstoreGetDict(kvstore *kvs, int didx); +/* Specific hashtable access by hashtable-index */ +unsigned long kvstoreHashtableSize(kvstore *kvs, int didx); +kvstoreHashtableIterator *kvstoreGetHashtableIterator(kvstore *kvs, int didx); +kvstoreHashtableIterator *kvstoreGetHashtableSafeIterator(kvstore *kvs, int didx); +void kvstoreReleaseHashtableIterator(kvstoreHashtableIterator *kvs_id); +int kvstoreHashtableIteratorNext(kvstoreHashtableIterator *kvs_di, void **next); +int kvstoreHashtableRandomEntry(kvstore *kvs, int didx, void **found); +int kvstoreHashtableFairRandomEntry(kvstore *kvs, int didx, void **found); +unsigned int kvstoreHashtableSampleEntries(kvstore *kvs, int didx, void **dst, unsigned int count); +int kvstoreHashtableExpand(kvstore *kvs, int didx, unsigned long size); +unsigned long kvstoreHashtableScanDefrag(kvstore *kvs, + int didx, + unsigned long v, + hashtableScanFunction fn, + void *privdata, + void *(*defragfn)(void *), + int flags); +void kvstoreHashtableDefragTables(kvstore *kvs, void *(*defragfn)(void *)); +int kvstoreHashtableFind(kvstore *kvs, int didx, void *key, void **found); +void **kvstoreHashtableFindRef(kvstore *kvs, int didx, const void *key); +int kvstoreHashtableAddOrFind(kvstore *kvs, int didx, void *key, void **existing); +int kvstoreHashtableAdd(kvstore *kvs, int didx, void *entry); -#endif /* DICTARRAY_H_ */ +int kvstoreHashtableFindPositionForInsert(kvstore *kvs, int didx, void *key, hashtablePosition *position, void **existing); +void kvstoreHashtableInsertAtPosition(kvstore *kvs, int didx, void *entry, void *position); + +void **kvstoreHashtableTwoPhasePopFindRef(kvstore *kvs, int didx, const void *key, void *position); +void kvstoreHashtableTwoPhasePopDelete(kvstore *kvs, int didx, void *position); +int kvstoreHashtablePop(kvstore *kvs, int didx, const void *key, void **popped); +int kvstoreHashtableDelete(kvstore *kvs, int didx, const void *key); +hashtable *kvstoreGetHashtable(kvstore *kvs, int didx); + +#endif /* KVSTORE_H */ diff --git a/src/lazyfree.c b/src/lazyfree.c index 6176b43440..14a4454d7a 100644 --- a/src/lazyfree.c +++ b/src/lazyfree.c @@ -186,14 +186,14 @@ void freeObjAsync(robj *key, robj *obj, int dbid) { * lazy freeing. */ void emptyDbAsync(serverDb *db) { int slot_count_bits = 0; - int flags = KVSTORE_ALLOCATE_DICTS_ON_DEMAND; + int flags = KVSTORE_ALLOCATE_HASHTABLES_ON_DEMAND; if (server.cluster_enabled) { slot_count_bits = CLUSTER_SLOT_MASK_BITS; - flags |= KVSTORE_FREE_EMPTY_DICTS; + flags |= KVSTORE_FREE_EMPTY_HASHTABLES; } kvstore *oldkeys = db->keys, *oldexpires = db->expires; - db->keys = kvstoreCreate(&kvstoreKeysDictType, slot_count_bits, flags); - db->expires = kvstoreCreate(&kvstoreExpiresDictType, slot_count_bits, flags); + db->keys = kvstoreCreate(&kvstoreKeysHashtableType, slot_count_bits, flags); + db->expires = kvstoreCreate(&kvstoreExpiresHashtableType, slot_count_bits, flags); atomic_fetch_add_explicit(&lazyfree_objects, kvstoreSize(oldkeys), memory_order_relaxed); bioCreateLazyFreeJob(lazyfreeFreeDatabase, 2, oldkeys, oldexpires); } diff --git a/src/memory_prefetch.c b/src/memory_prefetch.c index d888170176..5f75652fb0 100644 --- a/src/memory_prefetch.c +++ b/src/memory_prefetch.c @@ -8,411 +8,15 @@ */ #include "memory_prefetch.h" -#include "server.h" -#include "dict.h" - -/* Forward declarations of dict.c functions */ -dictEntry *dictGetNext(const dictEntry *de); - -/* Forward declarations of kvstore.c functions */ -dict *kvstoreGetDict(kvstore *kvs, int didx); - -typedef enum { - HT_IDX_FIRST = 0, - HT_IDX_SECOND = 1, - HT_IDX_INVALID = -1 -} HashTableIndex; - -typedef enum { - PREFETCH_BUCKET, /* Initial state, determines which hash table to use and prefetch the table's bucket */ - PREFETCH_ENTRY, /* prefetch entries associated with the given key's hash */ - PREFETCH_VALUE, /* prefetch the value object of the entry found in the previous step */ - PREFETCH_VALUE_DATA, /* prefetch the value object's data (if applicable) */ - PREFETCH_DONE /* Indicates that prefetching for this key is complete */ -} PrefetchState; - - -/************************************ State machine diagram for the prefetch operation. ******************************** - │ - start - │ - ┌────────▼─────────┐ - ┌─────────►│ PREFETCH_BUCKET ├────►────────┐ - │ └────────┬─────────┘ no more tables -> done - | bucket|found | - │ | │ - entry not found - goto next table ┌────────▼────────┐ │ - └────◄─────┤ PREFETCH_ENTRY | ▼ - ┌────────────►└────────┬────────┘ │ - | Entry│found │ - │ | │ - value not found - goto next entry ┌───────▼────────┐ | - └───────◄──────┤ PREFETCH_VALUE | ▼ - └───────┬────────┘ │ - Value│found │ - | | - ┌───────────▼──────────────┐ │ - │ PREFETCH_VALUE_DATA │ ▼ - └───────────┬──────────────┘ │ - | │ - ┌───────-─▼─────────────┐ │ - │ PREFETCH_DONE │◄────────┘ - └───────────────────────┘ -**********************************************************************************************************************/ - -typedef void *(*GetValueDataFunc)(const void *val); - -typedef struct KeyPrefetchInfo { - PrefetchState state; /* Current state of the prefetch operation */ - HashTableIndex ht_idx; /* Index of the current hash table (0 or 1 for rehashing) */ - uint64_t bucket_idx; /* Index of the bucket in the current hash table */ - uint64_t key_hash; /* Hash value of the key being prefetched */ - dictEntry *current_entry; /* Pointer to the current entry being processed */ -} KeyPrefetchInfo; - -/* PrefetchCommandsBatch structure holds the state of the current batch of client commands being processed. */ -typedef struct PrefetchCommandsBatch { - size_t cur_idx; /* Index of the current key being processed */ - size_t keys_done; /* Number of keys that have been prefetched */ - size_t key_count; /* Number of keys in the current batch */ - size_t client_count; /* Number of clients in the current batch */ - size_t max_prefetch_size; /* Maximum number of keys to prefetch in a batch */ - size_t executed_commands; /* Number of commands executed in the current batch */ - int *slots; /* Array of slots for each key */ - void **keys; /* Array of keys to prefetch in the current batch */ - client **clients; /* Array of clients in the current batch */ - dict **keys_dicts; /* Main dict for each key */ - dict **expire_dicts; /* Expire dict for each key */ - dict **current_dicts; /* Points to either keys_dicts or expire_dicts */ - KeyPrefetchInfo *prefetch_info; /* Prefetch info for each key */ -} PrefetchCommandsBatch; - -static PrefetchCommandsBatch *batch = NULL; - -void freePrefetchCommandsBatch(void) { - if (batch == NULL) { - return; - } - - zfree(batch->clients); - zfree(batch->keys); - zfree(batch->keys_dicts); - zfree(batch->expire_dicts); - zfree(batch->slots); - zfree(batch->prefetch_info); - zfree(batch); - batch = NULL; -} void prefetchCommandsBatchInit(void) { - serverAssert(!batch); - size_t max_prefetch_size = server.prefetch_batch_max_size; - - if (max_prefetch_size == 0) { - return; - } - - batch = zcalloc(sizeof(PrefetchCommandsBatch)); - batch->max_prefetch_size = max_prefetch_size; - batch->clients = zcalloc(max_prefetch_size * sizeof(client *)); - batch->keys = zcalloc(max_prefetch_size * sizeof(void *)); - batch->keys_dicts = zcalloc(max_prefetch_size * sizeof(dict *)); - batch->expire_dicts = zcalloc(max_prefetch_size * sizeof(dict *)); - batch->slots = zcalloc(max_prefetch_size * sizeof(int)); - batch->prefetch_info = zcalloc(max_prefetch_size * sizeof(KeyPrefetchInfo)); -} - -void onMaxBatchSizeChange(void) { - if (batch && batch->client_count > 0) { - /* We need to process the current batch before updating the size */ - return; - } - - freePrefetchCommandsBatch(); - prefetchCommandsBatchInit(); } - -/* Prefetch the given pointer and move to the next key in the batch. */ -static void prefetchAndMoveToNextKey(void *addr) { - valkey_prefetch(addr); - /* While the prefetch is in progress, we can continue to the next key */ - batch->cur_idx = (batch->cur_idx + 1) % batch->key_count; -} - -static void markKeyAsdone(KeyPrefetchInfo *info) { - info->state = PREFETCH_DONE; - server.stat_total_prefetch_entries++; - batch->keys_done++; -} - -/* Returns the next KeyPrefetchInfo structure that needs to be processed. */ -static KeyPrefetchInfo *getNextPrefetchInfo(void) { - size_t start_idx = batch->cur_idx; - do { - KeyPrefetchInfo *info = &batch->prefetch_info[batch->cur_idx]; - if (info->state != PREFETCH_DONE) return info; - batch->cur_idx = (batch->cur_idx + 1) % batch->key_count; - } while (batch->cur_idx != start_idx); - return NULL; -} - -static void initBatchInfo(dict **dicts) { - batch->current_dicts = dicts; - - /* Initialize the prefetch info */ - for (size_t i = 0; i < batch->key_count; i++) { - KeyPrefetchInfo *info = &batch->prefetch_info[i]; - if (!batch->current_dicts[i] || dictSize(batch->current_dicts[i]) == 0) { - info->state = PREFETCH_DONE; - batch->keys_done++; - continue; - } - info->ht_idx = HT_IDX_INVALID; - info->current_entry = NULL; - info->state = PREFETCH_BUCKET; - info->key_hash = dictHashKey(batch->current_dicts[i], batch->keys[i]); - } -} - -/* Prefetch the bucket of the next hash table index. - * If no tables are left, move to the PREFETCH_DONE state. */ -static void prefetchBucket(KeyPrefetchInfo *info) { - size_t i = batch->cur_idx; - - /* Determine which hash table to use */ - if (info->ht_idx == HT_IDX_INVALID) { - info->ht_idx = HT_IDX_FIRST; - } else if (info->ht_idx == HT_IDX_FIRST && dictIsRehashing(batch->current_dicts[i])) { - info->ht_idx = HT_IDX_SECOND; - } else { - /* No more tables left - mark as done. */ - markKeyAsdone(info); - return; - } - - /* Prefetch the bucket */ - info->bucket_idx = info->key_hash & DICTHT_SIZE_MASK(batch->current_dicts[i]->ht_size_exp[info->ht_idx]); - prefetchAndMoveToNextKey(&batch->current_dicts[i]->ht_table[info->ht_idx][info->bucket_idx]); - info->current_entry = NULL; - info->state = PREFETCH_ENTRY; -} - -/* Prefetch the next entry in the bucket and move to the PREFETCH_VALUE state. - * If no more entries in the bucket, move to the PREFETCH_BUCKET state to look at the next table. */ -static void prefetchEntry(KeyPrefetchInfo *info) { - size_t i = batch->cur_idx; - - if (info->current_entry) { - /* We already found an entry in the bucket - move to the next entry */ - info->current_entry = dictGetNext(info->current_entry); - } else { - /* Go to the first entry in the bucket */ - info->current_entry = batch->current_dicts[i]->ht_table[info->ht_idx][info->bucket_idx]; - } - - if (info->current_entry) { - prefetchAndMoveToNextKey(info->current_entry); - info->state = PREFETCH_VALUE; - } else { - /* No entry found in the bucket - try the bucket in the next table */ - info->state = PREFETCH_BUCKET; - } -} - -/* Prefetch the entry's value. If the value is found, move to the PREFETCH_VALUE_DATA state. - * If the value is not found, move to the PREFETCH_ENTRY state to look at the next entry in the bucket. */ -static void prefetchValue(KeyPrefetchInfo *info) { - size_t i = batch->cur_idx; - void *value = dictGetVal(info->current_entry); - - if (dictGetNext(info->current_entry) == NULL && !dictIsRehashing(batch->current_dicts[i])) { - /* If this is the last element, we assume a hit and don't compare the keys */ - prefetchAndMoveToNextKey(value); - info->state = PREFETCH_VALUE_DATA; - return; - } - - void *current_entry_key = dictGetKey(info->current_entry); - if (batch->keys[i] == current_entry_key || - dictCompareKeys(batch->current_dicts[i], batch->keys[i], current_entry_key)) { - /* If the key is found, prefetch the value */ - prefetchAndMoveToNextKey(value); - info->state = PREFETCH_VALUE_DATA; - } else { - /* Move to the next entry */ - info->state = PREFETCH_ENTRY; - } -} - -/* Prefetch the value data if available. */ -static void prefetchValueData(KeyPrefetchInfo *info, GetValueDataFunc get_val_data_func) { - if (get_val_data_func) { - void *value_data = get_val_data_func(dictGetVal(info->current_entry)); - if (value_data) prefetchAndMoveToNextKey(value_data); - } - markKeyAsdone(info); -} - -/* Prefetch dictionary data for an array of keys. - * - * This function takes an array of dictionaries and keys, attempting to bring - * data closer to the L1 cache that might be needed for dictionary operations - * on those keys. - * - * The dictFind algorithm: - * 1. Evaluate the hash of the key - * 2. Access the index in the first table - * 3. Walk the entries linked list until the key is found - * If the key hasn't been found and the dictionary is in the middle of rehashing, - * access the index on the second table and repeat step 3 - * - * dictPrefetch executes the same algorithm as dictFind, but one step at a time - * for each key. Instead of waiting for data to be read from memory, it prefetches - * the data and then moves on to execute the next prefetch for another key. - * - * dicts - An array of dictionaries to prefetch data from. - * get_val_data_func - A callback function that dictPrefetch can invoke - * to bring the key's value data closer to the L1 cache as well. - */ -static void dictPrefetch(dict **dicts, GetValueDataFunc get_val_data_func) { - initBatchInfo(dicts); - KeyPrefetchInfo *info; - while ((info = getNextPrefetchInfo())) { - switch (info->state) { - case PREFETCH_BUCKET: prefetchBucket(info); break; - case PREFETCH_ENTRY: prefetchEntry(info); break; - case PREFETCH_VALUE: prefetchValue(info); break; - case PREFETCH_VALUE_DATA: prefetchValueData(info, get_val_data_func); break; - default: serverPanic("Unknown prefetch state %d", info->state); - } - } -} - -/* Helper function to get the value pointer of an object. */ -static void *getObjectValuePtr(const void *val) { - robj *o = (robj *)val; - return (o->type == OBJ_STRING && o->encoding == OBJ_ENCODING_RAW) ? o->ptr : NULL; -} - -static void resetCommandsBatch(void) { - batch->cur_idx = 0; - batch->keys_done = 0; - batch->key_count = 0; - batch->client_count = 0; - batch->executed_commands = 0; -} - -/* Prefetch command-related data: - * 1. Prefetch the command arguments allocated by the I/O thread to bring them closer to the L1 cache. - * 2. Prefetch the keys and values for all commands in the current batch from the main and expires dictionaries. */ -static void prefetchCommands(void) { - /* Prefetch argv's for all clients */ - for (size_t i = 0; i < batch->client_count; i++) { - client *c = batch->clients[i]; - if (!c || c->argc <= 1) continue; - /* Skip prefetching first argv (cmd name) it was already looked up by the I/O thread. */ - for (int j = 1; j < c->argc; j++) { - valkey_prefetch(c->argv[j]); - } - } - - /* Prefetch the argv->ptr if required */ - for (size_t i = 0; i < batch->client_count; i++) { - client *c = batch->clients[i]; - if (!c || c->argc <= 1) continue; - for (int j = 1; j < c->argc; j++) { - if (c->argv[j]->encoding == OBJ_ENCODING_RAW) { - valkey_prefetch(c->argv[j]->ptr); - } - } - } - - /* Get the keys ptrs - we do it here after the key obj was prefetched. */ - for (size_t i = 0; i < batch->key_count; i++) { - batch->keys[i] = ((robj *)batch->keys[i])->ptr; - } - - /* Prefetch dict keys for all commands. Prefetching is beneficial only if there are more than one key. */ - if (batch->key_count > 1) { - server.stat_total_prefetch_batches++; - /* Prefetch keys from the main dict */ - dictPrefetch(batch->keys_dicts, getObjectValuePtr); - /* Prefetch keys from the expires dict - no value data to prefetch */ - dictPrefetch(batch->expire_dicts, NULL); - } -} - -/* Processes all the prefetched commands in the current batch. */ void processClientsCommandsBatch(void) { - if (!batch || batch->client_count == 0) return; - - /* If executed_commands is not 0, - * it means that we are in the middle of processing a batch and this is a recursive call */ - if (batch->executed_commands == 0) { - prefetchCommands(); - } - - /* Process the commands */ - for (size_t i = 0; i < batch->client_count; i++) { - client *c = batch->clients[i]; - if (c == NULL) continue; - - /* Set the client to null immediately to avoid accessing it again recursively when ProcessingEventsWhileBlocked */ - batch->clients[i] = NULL; - batch->executed_commands++; - if (processPendingCommandAndInputBuffer(c) != C_ERR) beforeNextClient(c); - } - - resetCommandsBatch(); - - /* Handle the case where the max prefetch size has been changed. */ - if (batch->max_prefetch_size != (size_t)server.prefetch_batch_max_size) { - onMaxBatchSizeChange(); - } } - -/* Adds the client's command to the current batch and processes the batch - * if it becomes full. - * - * Returns C_OK if the command was added successfully, C_ERR otherwise. */ -int addCommandToBatchAndProcessIfFull(client *c) { - if (!batch) return C_ERR; - - batch->clients[batch->client_count++] = c; - - /* Get command's keys positions */ - if (c->io_parsed_cmd) { - getKeysResult result; - initGetKeysResult(&result); - int num_keys = getKeysFromCommand(c->io_parsed_cmd, c->argv, c->argc, &result); - for (int i = 0; i < num_keys && batch->key_count < batch->max_prefetch_size; i++) { - batch->keys[batch->key_count] = c->argv[result.keys[i].pos]; - batch->slots[batch->key_count] = c->slot > 0 ? c->slot : 0; - batch->keys_dicts[batch->key_count] = kvstoreGetDict(c->db->keys, batch->slots[batch->key_count]); - batch->expire_dicts[batch->key_count] = kvstoreGetDict(c->db->expires, batch->slots[batch->key_count]); - batch->key_count++; - } - getKeysFreeResult(&result); - } - - /* If the batch is full, process it. - * We also check the client count to handle cases where - * no keys exist for the clients' commands. */ - if (batch->client_count == batch->max_prefetch_size || batch->key_count == batch->max_prefetch_size) { - processClientsCommandsBatch(); - } - - return C_OK; +int addCommandToBatchAndProcessIfFull(struct client *c) { + (void)c; + return -1; } - -/* Removes the given client from the pending prefetch batch, if present. */ -void removeClientFromPendingCommandsBatch(client *c) { - if (!batch) return; - - for (size_t i = 0; i < batch->client_count; i++) { - if (batch->clients[i] == c) { - batch->clients[i] = NULL; - return; - } - } +void removeClientFromPendingCommandsBatch(struct client *c) { + (void)c; } diff --git a/src/module.c b/src/module.c index d1cd55c501..597f203872 100644 --- a/src/module.c +++ b/src/module.c @@ -717,7 +717,7 @@ int moduleCreateEmptyKey(ValkeyModuleKey *key, int type) { case VALKEYMODULE_KEYTYPE_STREAM: obj = createStreamObject(); break; default: return VALKEYMODULE_ERR; } - dbAdd(key->db, key->key, obj); + obj = dbAdd(key->db, key->key, obj); key->value = obj; moduleInitKeyTypeSpecific(key); return VALKEYMODULE_OK; @@ -4174,7 +4174,7 @@ int VM_SetExpire(ValkeyModuleKey *key, mstime_t expire) { return VALKEYMODULE_ERR; if (expire != VALKEYMODULE_NO_EXPIRE) { expire += commandTimeSnapshot(); - setExpire(key->ctx->client, key->db, key->key, expire); + key->value = setExpire(key->ctx->client, key->db, key->key, expire); } else { removeExpire(key->db, key->key); } @@ -4203,7 +4203,7 @@ int VM_SetAbsExpire(ValkeyModuleKey *key, mstime_t expire) { if (!(key->mode & VALKEYMODULE_WRITE) || key->value == NULL || (expire < 0 && expire != VALKEYMODULE_NO_EXPIRE)) return VALKEYMODULE_ERR; if (expire != VALKEYMODULE_NO_EXPIRE) { - setExpire(key->ctx->client, key->db, key->key, expire); + key->value = setExpire(key->ctx->client, key->db, key->key, expire); } else { removeExpire(key->db, key->key); } @@ -4264,8 +4264,8 @@ int VM_GetToDbIdFromOptCtx(ValkeyModuleKeyOptCtx *ctx) { int VM_StringSet(ValkeyModuleKey *key, ValkeyModuleString *str) { if (!(key->mode & VALKEYMODULE_WRITE) || key->iter) return VALKEYMODULE_ERR; VM_DeleteKey(key); - setKey(key->ctx->client, key->db, key->key, str, SETKEY_NO_SIGNAL); - key->value = str; + incrRefCount(str); + key->value = setKey(key->ctx->client, key->db, key->key, str, SETKEY_NO_SIGNAL); return VALKEYMODULE_OK; } @@ -4344,9 +4344,8 @@ int VM_StringTruncate(ValkeyModuleKey *key, size_t newlen) { if (key->value == NULL) { /* Empty key: create it with the new size. */ robj *o = createObject(OBJ_STRING, sdsnewlen(NULL, newlen)); - setKey(key->ctx->client, key->db, key->key, o, SETKEY_NO_SIGNAL); + o = setKey(key->ctx->client, key->db, key->key, o, SETKEY_NO_SIGNAL); key->value = o; - decrRefCount(o); } else { /* Unshare and resize. */ key->value = dbUnshareStringValue(key->db, key->key, key->value); @@ -6911,8 +6910,7 @@ int VM_ModuleTypeSetValue(ValkeyModuleKey *key, moduleType *mt, void *value) { if (!(key->mode & VALKEYMODULE_WRITE) || key->iter) return VALKEYMODULE_ERR; VM_DeleteKey(key); robj *o = createModuleObject(mt, value); - setKey(key->ctx->client, key->db, key->key, o, SETKEY_NO_SIGNAL); - decrRefCount(o); + o = setKey(key->ctx->client, key->db, key->key, o, SETKEY_NO_SIGNAL); key->value = o; return VALKEYMODULE_OK; } @@ -10878,10 +10876,10 @@ typedef struct ValkeyModuleScanCursor { int done; } ValkeyModuleScanCursor; -static void moduleScanCallback(void *privdata, const dictEntry *de) { +static void moduleScanCallback(void *privdata, void *element) { ScanCBData *data = privdata; - sds key = dictGetKey(de); - robj *val = dictGetVal(de); + valkey *val = element; + sds key = objectGetKey(val); ValkeyModuleString *keyname = createObject(OBJ_STRING, sdsdup(key)); /* Setup the key handle. */ diff --git a/src/object.c b/src/object.c index 8c1cf64892..fdd01eed7c 100644 --- a/src/object.c +++ b/src/object.c @@ -43,16 +43,60 @@ /* ===================== Creation and parsing of objects ==================== */ -robj *createObject(int type, void *ptr) { - robj *o = zmalloc(sizeof(*o)); +/* Creates an object, optionally with embedded key and expire fields. The key + * and expire fields can be omitted by passing NULL and -1, respectively. */ +robj *createObjectWithKeyAndExpire(int type, void *ptr, const sds key, long long expire) { + /* Calculate sizes */ + size_t key_sds_size = 0; + size_t min_size = sizeof(robj); + if (expire != -1) { + min_size += sizeof(long long); + } + if (key != NULL) { + /* Size of embedded key, incl. 1 byte for prefixed sds hdr size. */ + key_sds_size = sdscopytobuffer(NULL, 0, key, NULL); + min_size += 1 + key_sds_size; + } + /* Allocate and set the declared fields. */ + size_t bufsize = 0; + robj *o = zmalloc_usable(min_size, &bufsize); o->type = type; o->encoding = OBJ_ENCODING_RAW; o->ptr = ptr; o->refcount = 1; o->lru = 0; + o->hasexpire = (expire != -1); + o->hasembkey = (key != NULL); + + /* If the allocation has enough space for an expire field, add it even if we + * don't need it now. Then we don't need to realloc if it's needed later. */ + if (key != NULL && !o->hasexpire && bufsize >= min_size + sizeof(long long)) { + o->hasexpire = 1; + min_size += sizeof(long long); + } + + /* The memory after the struct where we embedded data. */ + unsigned char *data = (void *)(o + 1); + + /* Set the expire field. */ + if (o->hasexpire) { + *(long long *)data = expire; + data += sizeof(long long); + } + + /* Copy embedded key. */ + if (o->hasembkey) { + sdscopytobuffer(data + 1, key_sds_size, key, data); + data += 1 + key_sds_size; + } + return o; } +robj *createObject(int type, void *ptr) { + return createObjectWithKeyAndExpire(type, ptr, NULL, -1); +} + void initObjectLRUOrLFU(robj *o) { if (o->refcount == OBJ_SHARED_REFCOUNT) return; /* Set the LRU to the current lruclock (minutes resolution), or @@ -88,39 +132,85 @@ robj *createRawStringObject(const char *ptr, size_t len) { return createObject(OBJ_STRING, sdsnewlen(ptr, len)); } -/* Create a string object with encoding OBJ_ENCODING_EMBSTR, that is - * an object where the sds string is actually an unmodifiable string - * allocated in the same chunk as the object itself. */ -robj *createEmbeddedStringObject(const char *ptr, size_t len) { - size_t bufsize = 0; - size_t sds_hdrlen = sizeof(struct sdshdr8); - robj *o = zmalloc_usable(sizeof(robj) + sds_hdrlen + len + 1, &bufsize); - struct sdshdr8 *sh = (void *)(o + 1); +/* Creates a new embedded string object and copies the content of key, val and + * expire to the new object. LRU is set to 0. */ +static valkey *createEmbeddedStringObjectWithKeyAndExpire(const char *val_ptr, + size_t val_len, + const sds key, + long long expire) { + /* Calculate sizes */ + size_t key_sds_size = 0; + size_t min_size = sizeof(robj); + if (expire != -1) { + min_size += sizeof(long long); + } + if (key != NULL) { + /* Size of embedded key, incl. 1 byte for prefixed sds hdr size. */ + key_sds_size = sdscopytobuffer(NULL, 0, key, NULL); + min_size += 1 + key_sds_size; + } + /* Size of embedded value (EMBSTR) including \0 term. */ + min_size += sizeof(struct sdshdr8) + val_len + 1; + /* Allocate and set the declared fields. */ + size_t bufsize = 0; + valkey *o = zmalloc_usable(min_size, &bufsize); o->type = OBJ_STRING; o->encoding = OBJ_ENCODING_EMBSTR; - o->ptr = sh + 1; o->refcount = 1; o->lru = 0; + o->hasexpire = (expire != -1); + o->hasembkey = (key != NULL); + + /* If the allocation has enough space for an expire field, add it even if we + * don't need it now. Then we don't need to realloc if it's needed later. */ + if (!o->hasexpire && bufsize >= min_size + sizeof(long long)) { + o->hasexpire = 1; + min_size += sizeof(long long); + } + + /* The memory after the struct where we embedded data. */ + unsigned char *data = (void *)(o + 1); - sh->len = len; - size_t usable = bufsize - (sizeof(robj) + sds_hdrlen + 1); - sh->alloc = usable; - /* Overflow check. This must not happen as we use embedded strings only - * for sds strings that fit into SDS_TYPE_8. */ - serverAssert(usable == sh->alloc); + /* Set the expire field. */ + if (o->hasexpire) { + *(long long *)data = expire; + data += sizeof(long long); + } + + /* Copy embedded key. */ + if (o->hasembkey) { + sdscopytobuffer(data + 1, key_sds_size, key, data); + data += 1 + key_sds_size; + } + + /* Copy embedded value (EMBSTR). */ + struct sdshdr8 *sh = (void *)data; sh->flags = SDS_TYPE_8; - if (ptr == SDS_NOINIT) - sh->buf[len] = '\0'; - else if (ptr) { - memcpy(sh->buf, ptr, len); - sh->buf[len] = '\0'; + sh->len = val_len; + size_t capacity = bufsize - (min_size - val_len); + sh->alloc = capacity; + serverAssert(capacity == sh->alloc); /* Overflow check. */ + if (val_ptr == SDS_NOINIT) { + sh->buf[val_len] = '\0'; + } else if (val_ptr != NULL) { + memcpy(sh->buf, val_ptr, val_len); + sh->buf[val_len] = '\0'; } else { - memset(sh->buf, 0, len + 1); + memset(sh->buf, 0, val_len + 1); } + o->ptr = sh->buf; + return o; } +/* Create a string object with encoding OBJ_ENCODING_EMBSTR, that is + * an object where the sds string is actually an unmodifiable string + * allocated in the same chunk as the object itself. */ +robj *createEmbeddedStringObject(const char *ptr, size_t len) { + return createEmbeddedStringObjectWithKeyAndExpire(ptr, len, NULL, -1); +} + /* Create a string object with EMBSTR encoding if it is smaller than * OBJ_ENCODING_EMBSTR_SIZE_LIMIT, otherwise the RAW encoding is * used. @@ -135,6 +225,96 @@ robj *createStringObject(const char *ptr, size_t len) { return createRawStringObject(ptr, len); } +robj *createStringObjectWithKeyAndExpire(const char *ptr, size_t len, const sds key, long long expire) { + /* When to embed? Embed when the sum is up to 64 bytes. There may be better + * heuristics, e.g. we can look at the jemalloc sizes (16-byte intervals up + * to 128 bytes). */ + size_t size = sizeof(robj); + size += (key != NULL) * (sdslen(key) + 3); /* hdr size (1) + hdr (1) + nullterm (1) */ + size += (expire != -1) * sizeof(long long); + size += 4 + len; /* embstr header (3) + nullterm (1) */ + if (size <= 64) { + return createEmbeddedStringObjectWithKeyAndExpire(ptr, len, key, expire); + } else { + return createObjectWithKeyAndExpire(OBJ_STRING, sdsnewlen(ptr, len), key, expire); + } +} + +sds objectGetKey(const valkey *val) { + unsigned char *data = (void *)(val + 1); + if (val->hasexpire) { + /* Skip expire field */ + data += sizeof(long long); + } + if (val->hasembkey) { + uint8_t hdr_size = *(uint8_t *)data; + data += 1 + hdr_size; + return (sds)data; + } + return NULL; +} + +long long objectGetExpire(const valkey *val) { + unsigned char *data = (void *)(val + 1); + if (val->hasexpire) { + return *(long long *)data; + } else { + return -1; + } +} + +/* This functions may reallocate the value. The new allocation is returned and + * the old object's reference counter is decremented and possibly freed. Use the + * returned object instead of 'val' after calling this function. */ +valkey *objectSetExpire(valkey *val, long long expire) { + if (val->hasexpire) { + /* Update existing expire field. */ + unsigned char *data = (void *)(val + 1); + *(long long *)data = expire; + return val; + } else if (expire == -1) { + return val; + } else { + return objectSetKeyAndExpire(val, objectGetKey(val), expire); + } +} + +/* This functions may reallocate the value. The new allocation is returned and + * the old object's reference counter is decremented and possibly freed. Use the + * returned object instead of 'val' after calling this function. */ +valkey *objectSetKeyAndExpire(valkey *val, sds key, long long expire) { + if (val->type == OBJ_STRING && val->encoding == OBJ_ENCODING_EMBSTR) { + valkey *new = createStringObjectWithKeyAndExpire(val->ptr, sdslen(val->ptr), key, expire); + new->lru = val->lru; + decrRefCount(val); + return new; + } + + /* Create a new object with embedded key. Reuse ptr if possible. */ + void *ptr; + if (val->refcount == 1) { + /* Reuse the ptr. There are no other references to val. */ + ptr = val->ptr; + val->ptr = NULL; + } else if (val->type == OBJ_STRING && val->encoding == OBJ_ENCODING_INT) { + /* The pointer is not allocated memory. We can just copy the pointer. */ + ptr = val->ptr; + } else if (val->type == OBJ_STRING && val->encoding == OBJ_ENCODING_RAW) { + /* Dup the string. */ + ptr = sdsdup(val->ptr); + } else { + serverAssert(val->type != OBJ_STRING); + /* There are multiple references to this non-string object. Most types + * can be duplicated, but for a module type is not always possible. */ + serverPanic("Not implemented"); + } + valkey *new = createObjectWithKeyAndExpire(val->type, ptr, key, expire); + new->encoding = val->encoding; + new->lru = val->lru; + decrRefCount(val); + return new; +} + /* Same as CreateRawStringObject, can return NULL if allocation fails */ robj *tryCreateRawStringObject(const char *ptr, size_t len) { sds str = sdstrynewlen(ptr, len); @@ -179,18 +359,10 @@ robj *createStringObjectFromLongLong(long long value) { return createStringObjectFromLongLongWithOptions(value, LL2STROBJ_AUTO); } -/* The function avoids returning a shared integer when LFU/LRU info - * are needed, that is, when the object is used as a value in the key - * space(for instance when the INCR command is used), and the server is - * configured to evict based on LFU/LRU, so we want LFU/LRU values - * specific for each key. */ +/* The function doesn't return a shared integer when the object is used as a + * value in the key space (for instance when the INCR command is used). */ robj *createStringObjectFromLongLongForValue(long long value) { - if (server.maxmemory == 0 || !(server.maxmemory_policy & MAXMEMORY_FLAG_NO_SHARED_INTEGERS)) { - /* If the maxmemory policy permits, we can still return shared integers */ - return createStringObjectFromLongLongWithOptions(value, LL2STROBJ_AUTO); - } else { - return createStringObjectFromLongLongWithOptions(value, LL2STROBJ_NO_SHARED); - } + return createStringObjectFromLongLongWithOptions(value, LL2STROBJ_NO_SHARED); } /* Create a string object that contains an sds inside it. That means it can't be @@ -381,15 +553,17 @@ void incrRefCount(robj *o) { void decrRefCount(robj *o) { if (o->refcount == 1) { - switch (o->type) { - case OBJ_STRING: freeStringObject(o); break; - case OBJ_LIST: freeListObject(o); break; - case OBJ_SET: freeSetObject(o); break; - case OBJ_ZSET: freeZsetObject(o); break; - case OBJ_HASH: freeHashObject(o); break; - case OBJ_MODULE: freeModuleObject(o); break; - case OBJ_STREAM: freeStreamObject(o); break; - default: serverPanic("Unknown object type"); break; + if (o->ptr != NULL) { + switch (o->type) { + case OBJ_STRING: freeStringObject(o); break; + case OBJ_LIST: freeListObject(o); break; + case OBJ_SET: freeSetObject(o); break; + case OBJ_ZSET: freeZsetObject(o); break; + case OBJ_HASH: freeHashObject(o); break; + case OBJ_MODULE: freeModuleObject(o); break; + case OBJ_STREAM: freeStreamObject(o); break; + default: serverPanic("Unknown object type"); break; + } } zfree(o); } else { @@ -648,23 +822,15 @@ robj *tryObjectEncodingEx(robj *o, int try_trim) { * representable as a 32 nor 64 bit integer. */ len = sdslen(s); if (len <= 20 && string2l(s, len, &value)) { - /* This object is encodable as a long. Try to use a shared object. - * Note that we avoid using shared integers when maxmemory is used - * because every object needs to have a private LRU field for the LRU - * algorithm to work well. */ - if (canUseSharedObject() && value >= 0 && value < OBJ_SHARED_INTEGERS) { + /* This object is encodable as a long. */ + if (o->encoding == OBJ_ENCODING_RAW) { + sdsfree(o->ptr); + o->encoding = OBJ_ENCODING_INT; + o->ptr = (void *)value; + return o; + } else if (o->encoding == OBJ_ENCODING_EMBSTR) { decrRefCount(o); - return shared.integers[value]; - } else { - if (o->encoding == OBJ_ENCODING_RAW) { - sdsfree(o->ptr); - o->encoding = OBJ_ENCODING_INT; - o->ptr = (void *)value; - return o; - } else if (o->encoding == OBJ_ENCODING_EMBSTR) { - decrRefCount(o); - return createStringObjectFromLongLongForValue(value); - } + return createStringObjectFromLongLongForValue(value); } } @@ -1194,7 +1360,7 @@ struct serverMemOverhead *getMemoryOverheadData(void) { for (j = 0; j < server.dbnum; j++) { serverDb *db = server.db + j; - if (!kvstoreNumAllocatedDicts(db->keys)) continue; + if (!kvstoreNumAllocatedHashtables(db->keys)) continue; unsigned long long keyscount = kvstoreSize(db->keys); @@ -1216,8 +1382,8 @@ struct serverMemOverhead *getMemoryOverheadData(void) { mh->overhead_db_hashtable_lut += kvstoreOverheadHashtableLut(db->expires); mh->overhead_db_hashtable_rehashing += kvstoreOverheadHashtableRehashing(db->keys); mh->overhead_db_hashtable_rehashing += kvstoreOverheadHashtableRehashing(db->expires); - mh->db_dict_rehashing_count += kvstoreDictRehashingCount(db->keys); - mh->db_dict_rehashing_count += kvstoreDictRehashingCount(db->expires); + mh->db_dict_rehashing_count += kvstoreHashtableRehashingCount(db->keys); + mh->db_dict_rehashing_count += kvstoreHashtableRehashingCount(db->expires); } mh->overhead_total = mem_total; @@ -1515,7 +1681,6 @@ void memoryCommand(client *c) { }; addReplyHelp(c, help); } else if (!strcasecmp(c->argv[1]->ptr, "usage") && c->argc >= 3) { - dictEntry *de; long long samples = OBJ_COMPUTE_SIZE_DEF_SAMPLES; for (int j = 3; j < c->argc; j++) { if (!strcasecmp(c->argv[j]->ptr, "samples") && j + 1 < c->argc) { @@ -1531,12 +1696,12 @@ void memoryCommand(client *c) { return; } } - if ((de = dbFind(c->db, c->argv[2]->ptr)) == NULL) { + valkey *obj = dbFind(c->db, c->argv[2]->ptr); + if (obj == NULL) { addReplyNull(c); return; } - size_t usage = objectComputeSize(c->argv[2], dictGetVal(de), samples, c->db->id); - usage += dictEntryMemUsage(de); + size_t usage = objectComputeSize(c->argv[2], obj, samples, c->db->id); addReplyLongLong(c, usage); } else if (!strcasecmp(c->argv[1]->ptr, "stats") && c->argc == 2) { struct serverMemOverhead *mh = getMemoryOverheadData(); diff --git a/src/pubsub.c b/src/pubsub.c index 5b037b5721..296af1ffd6 100644 --- a/src/pubsub.c +++ b/src/pubsub.c @@ -258,7 +258,6 @@ void unmarkClientAsPubSub(client *c) { /* Subscribe a client to a channel. Returns 1 if the operation succeeded, or * 0 if the client was already subscribed to that channel. */ int pubsubSubscribeChannel(client *c, robj *channel, pubsubtype type) { - dictEntry *de, *existing; dict *clients = NULL; int retval = 0; unsigned int slot = 0; @@ -272,15 +271,18 @@ int pubsubSubscribeChannel(client *c, robj *channel, pubsubtype type) { slot = getKeySlot(channel->ptr); } - de = kvstoreDictAddRaw(*type.serverPubSubChannels, slot, channel, &existing); - - if (existing) { - clients = dictGetVal(existing); - channel = dictGetKey(existing); + hashtablePosition pos; + void *existing; + if (!kvstoreHashtableFindPositionForInsert(*type.serverPubSubChannels, slot, channel, &pos, &existing)) { + clients = existing; + channel = *(robj **)clients->metadata; } else { + /* Store pointer to channel name in the dict's metadata. */ clients = dictCreate(&clientDictType); - kvstoreDictSetVal(*type.serverPubSubChannels, slot, de, clients); + memcpy(clients->metadata, (void *)&channel, sizeof(void *)); incrRefCount(channel); + /* Insert this dict in the kvstore at the position returned above. */ + kvstoreHashtableInsertAtPosition(*type.serverPubSubChannels, slot, clients, &pos); } serverAssert(dictAdd(clients, c, NULL) != DICT_ERR); @@ -295,7 +297,6 @@ int pubsubSubscribeChannel(client *c, robj *channel, pubsubtype type) { /* Unsubscribe a client from a channel. Returns 1 if the operation succeeded, or * 0 if the client was not subscribed to the specified channel. */ int pubsubUnsubscribeChannel(client *c, robj *channel, int notify, pubsubtype type) { - dictEntry *de; dict *clients; int retval = 0; int slot = 0; @@ -309,15 +310,16 @@ int pubsubUnsubscribeChannel(client *c, robj *channel, int notify, pubsubtype ty if (server.cluster_enabled && type.shard) { slot = getKeySlot(channel->ptr); } - de = kvstoreDictFind(*type.serverPubSubChannels, slot, channel); - serverAssertWithInfo(c, NULL, de != NULL); - clients = dictGetVal(de); + void *found; + kvstoreHashtableFind(*type.serverPubSubChannels, slot, channel, &found); + serverAssertWithInfo(c, NULL, found); + clients = found; serverAssertWithInfo(c, NULL, dictDelete(clients, c) == DICT_OK); if (dictSize(clients) == 0) { /* Free the dict and associated hash entry at all if this was * the latest client, so that it will be possible to abuse * PUBSUB creating millions of channels. */ - kvstoreDictDelete(*type.serverPubSubChannels, slot, channel); + kvstoreHashtableDelete(*type.serverPubSubChannels, slot, channel); } } /* Notify the client */ @@ -330,13 +332,13 @@ int pubsubUnsubscribeChannel(client *c, robj *channel, int notify, pubsubtype ty /* Unsubscribe all shard channels in a slot. */ void pubsubShardUnsubscribeAllChannelsInSlot(unsigned int slot) { - if (!kvstoreDictSize(server.pubsubshard_channels, slot)) return; + if (!kvstoreHashtableSize(server.pubsubshard_channels, slot)) return; - kvstoreDictIterator *kvs_di = kvstoreGetDictSafeIterator(server.pubsubshard_channels, slot); - dictEntry *de; - while ((de = kvstoreDictIteratorNext(kvs_di)) != NULL) { - robj *channel = dictGetKey(de); - dict *clients = dictGetVal(de); + kvstoreHashtableIterator *kvs_di = kvstoreGetHashtableSafeIterator(server.pubsubshard_channels, slot); + void *element; + while (kvstoreHashtableIteratorNext(kvs_di, &element)) { + dict *clients = element; + robj *channel = *(robj **)clients->metadata; /* For each client subscribed to the channel, unsubscribe it. */ dictIterator *iter = dictGetIterator(clients); dictEntry *entry; @@ -352,9 +354,9 @@ void pubsubShardUnsubscribeAllChannelsInSlot(unsigned int slot) { } } dictReleaseIterator(iter); - kvstoreDictDelete(server.pubsubshard_channels, slot, channel); + kvstoreHashtableDelete(server.pubsubshard_channels, slot, channel); } - kvstoreReleaseDictIterator(kvs_di); + kvstoreReleaseHashtableIterator(kvs_di); } /* Subscribe a client to a pattern. Returns 1 if the operation succeeded, or 0 if the client was already subscribed to @@ -474,6 +476,7 @@ int pubsubUnsubscribeAllPatterns(client *c, int notify) { */ int pubsubPublishMessageInternal(robj *channel, robj *message, pubsubtype type) { int receivers = 0; + void *element; dictEntry *de; dictIterator *di; int slot = -1; @@ -482,9 +485,8 @@ int pubsubPublishMessageInternal(robj *channel, robj *message, pubsubtype type) if (server.cluster_enabled && type.shard) { slot = keyHashSlot(channel->ptr, sdslen(channel->ptr)); } - de = kvstoreDictFind(*type.serverPubSubChannels, (slot == -1) ? 0 : slot, channel); - if (de) { - dict *clients = dictGetVal(de); + if (kvstoreHashtableFind(*type.serverPubSubChannels, (slot == -1) ? 0 : slot, channel, &element)) { + dict *clients = element; dictEntry *entry; dictIterator *iter = dictGetIterator(clients); while ((entry = dictNext(iter)) != NULL) { @@ -650,8 +652,8 @@ void pubsubCommand(client *c) { addReplyArrayLen(c, (c->argc - 2) * 2); for (j = 2; j < c->argc; j++) { - dict *d = kvstoreDictFetchValue(server.pubsub_channels, 0, c->argv[j]); - + dict *d = NULL; + kvstoreHashtableFind(server.pubsub_channels, 0, c->argv[j], (void **)&d); addReplyBulk(c, c->argv[j]); addReplyLongLong(c, d ? dictSize(d) : 0); } @@ -669,7 +671,8 @@ void pubsubCommand(client *c) { for (j = 2; j < c->argc; j++) { sds key = c->argv[j]->ptr; unsigned int slot = server.cluster_enabled ? keyHashSlot(key, (int)sdslen(key)) : 0; - dict *clients = kvstoreDictFetchValue(server.pubsubshard_channels, slot, c->argv[j]); + dict *clients = NULL; + kvstoreHashtableFind(server.pubsubshard_channels, slot, c->argv[j], (void **)&clients); addReplyBulk(c, c->argv[j]); addReplyLongLong(c, clients ? dictSize(clients) : 0); @@ -682,15 +685,16 @@ void pubsubCommand(client *c) { void channelList(client *c, sds pat, kvstore *pubsub_channels) { long mblen = 0; void *replylen; - unsigned int slot_cnt = kvstoreNumDicts(pubsub_channels); + unsigned int slot_cnt = kvstoreNumHashtables(pubsub_channels); replylen = addReplyDeferredLen(c); for (unsigned int i = 0; i < slot_cnt; i++) { - if (!kvstoreDictSize(pubsub_channels, i)) continue; - kvstoreDictIterator *kvs_di = kvstoreGetDictIterator(pubsub_channels, i); - dictEntry *de; - while ((de = kvstoreDictIteratorNext(kvs_di)) != NULL) { - robj *cobj = dictGetKey(de); + if (!kvstoreHashtableSize(pubsub_channels, i)) continue; + kvstoreHashtableIterator *kvs_di = kvstoreGetHashtableIterator(pubsub_channels, i); + void *next; + while (kvstoreHashtableIteratorNext(kvs_di, &next)) { + dict *clients = next; + robj *cobj = *(robj **)clients->metadata; sds channel = cobj->ptr; if (!pat || stringmatchlen(pat, sdslen(pat), channel, sdslen(channel), 0)) { @@ -698,7 +702,7 @@ void channelList(client *c, sds pat, kvstore *pubsub_channels) { mblen++; } } - kvstoreReleaseDictIterator(kvs_di); + kvstoreReleaseHashtableIterator(kvs_di); } setDeferredArrayLen(c, replylen, mblen); } diff --git a/src/rdb.c b/src/rdb.c index 1c200e54f5..408b023b72 100644 --- a/src/rdb.c +++ b/src/rdb.c @@ -1316,7 +1316,7 @@ ssize_t rdbSaveFunctions(rio *rdb) { } ssize_t rdbSaveDb(rio *rdb, int dbid, int rdbflags, long *key_counter) { - dictEntry *de; + valkey *o; ssize_t written = 0; ssize_t res; kvstoreIterator *kvs_it = NULL; @@ -1345,12 +1345,12 @@ ssize_t rdbSaveDb(rio *rdb, int dbid, int rdbflags, long *key_counter) { kvs_it = kvstoreIteratorInit(db->keys); int last_slot = -1; /* Iterate this DB writing every entry */ - while ((de = kvstoreIteratorNext(kvs_it)) != NULL) { - int curr_slot = kvstoreIteratorGetCurrentDictIndex(kvs_it); + while (kvstoreIteratorNext(kvs_it, (void **)&o)) { + int curr_slot = kvstoreIteratorGetCurrentHashtableIndex(kvs_it); /* Save slot info. */ if (server.cluster_enabled && curr_slot != last_slot) { - sds slot_info = sdscatprintf(sdsempty(), "%i,%lu,%lu", curr_slot, kvstoreDictSize(db->keys, curr_slot), - kvstoreDictSize(db->expires, curr_slot)); + sds slot_info = sdscatprintf(sdsempty(), "%i,%lu,%lu", curr_slot, kvstoreHashtableSize(db->keys, curr_slot), + kvstoreHashtableSize(db->expires, curr_slot)); if ((res = rdbSaveAuxFieldStrStr(rdb, "slot-info", slot_info)) < 0) { sdsfree(slot_info); goto werr; @@ -1359,8 +1359,8 @@ ssize_t rdbSaveDb(rio *rdb, int dbid, int rdbflags, long *key_counter) { last_slot = curr_slot; sdsfree(slot_info); } - sds keystr = dictGetKey(de); - robj key, *o = dictGetVal(de); + sds keystr = objectGetKey(o); + robj key; long long expire; size_t rdb_bytes_before_key = rdb->processed_bytes; @@ -3141,8 +3141,8 @@ int rdbLoadRioWithLoadingCtx(rio *rdb, int rdbflags, rdbSaveInfo *rsi, rdbLoadin if (server.cluster_enabled) { /* In cluster mode we resize individual slot specific dictionaries based on the number of keys that * slot holds. */ - kvstoreDictExpand(db->keys, slot_id, slot_size); - kvstoreDictExpand(db->expires, slot_id, expires_slot_size); + kvstoreHashtableExpand(db->keys, slot_id, slot_size); + kvstoreHashtableExpand(db->expires, slot_id, expires_slot_size); should_expand_db = 0; } } else { @@ -3300,7 +3300,7 @@ int rdbLoadRioWithLoadingCtx(rio *rdb, int rdbflags, rdbSaveInfo *rsi, rdbLoadin initStaticStringObject(keyobj, key); /* Add the new object in the hash table */ - int added = dbAddRDBLoad(db, key, val); + valkey *added = dbAddRDBLoad(db, key, val); server.rdb_last_load_keys_loaded++; if (!added) { if (rdbflags & RDBFLAGS_ALLOW_DUP) { @@ -3308,16 +3308,18 @@ int rdbLoadRioWithLoadingCtx(rio *rdb, int rdbflags, rdbSaveInfo *rsi, rdbLoadin * When it's set we allow new keys to replace the current * keys with the same name. */ dbSyncDelete(db, &keyobj); - dbAddRDBLoad(db, key, val); + added = dbAddRDBLoad(db, key, val); + serverAssert(added != NULL); } else { serverLog(LL_WARNING, "RDB has duplicated key '%s' in DB %d", key, db->id); serverPanic("Duplicated key found in RDB file"); } } + val = added; /* Set the expire time if needed */ if (expiretime != -1) { - setExpire(NULL, db, &keyobj, expiretime); + val = setExpire(NULL, db, &keyobj, expiretime); } /* Set usage information (for eviction). */ diff --git a/src/replication.c b/src/replication.c index 48f02cf658..d7b9d70ca1 100644 --- a/src/replication.c +++ b/src/replication.c @@ -1885,7 +1885,7 @@ void replicationSendNewlineToPrimary(void) { /* Callback used by emptyData() while flushing away old data to load * the new dataset received by the primary and by discardTempDb() * after loading succeeded or failed. */ -void replicationEmptyDbCallback(dict *d) { +void replicationEmptyDbCallback(hashtable *d) { UNUSED(d); if (server.repl_state == REPL_STATE_TRANSFER) replicationSendNewlineToPrimary(); } diff --git a/src/sds.c b/src/sds.c index 4dd7d709aa..cc5a4d3bc8 100644 --- a/src/sds.c +++ b/src/sds.c @@ -194,12 +194,12 @@ sds sdsdup(const sds s) { /* * This method returns the minimum amount of bytes required to store the sds (header + data + NULL terminator). */ -static inline size_t sdsminlen(sds s) { +static inline size_t sdsminlen(const sds s) { return sdslen(s) + sdsHdrSize(s[-1]) + 1; } /* This method copies the sds `s` into `buf` which is the target character buffer. */ -size_t sdscopytobuffer(unsigned char *buf, size_t buf_len, sds s, uint8_t *hdr_size) { +size_t sdscopytobuffer(unsigned char *buf, size_t buf_len, const sds s, uint8_t *hdr_size) { size_t required_keylen = sdsminlen(s); if (buf == NULL) { return required_keylen; diff --git a/src/server.c b/src/server.c index 27936c0fde..e2c926450a 100644 --- a/src/server.c +++ b/src/server.c @@ -494,21 +494,20 @@ uint64_t dictEncObjHash(const void *key) { } } -/* Return 1 if currently we allow dict to expand. Dict may allocate huge - * memory to contain hash buckets when dict expands, that may lead the server to - * reject user's requests or evict some keys, we can stop dict to expand - * provisionally if used memory will be over maxmemory after dict expands, - * but to guarantee the performance of the server, we still allow dict to expand - * if dict load factor exceeds HASHTABLE_MAX_LOAD_FACTOR. */ -int dictResizeAllowed(size_t moreMem, double usedRatio) { - /* for debug purposes: dict is not allowed to be resized. */ +/* Return 1 if we allow a hash table to expand. It may allocate a huge amount of + * memory to contain hash buckets when it expands, that may lead the server to + * reject user's requests or evict some keys. We can prevent expansion + * provisionally if used memory will be over maxmemory after it expands, + * but to guarantee the performance of the server, we still allow it to expand + * if the load factor exceeds the hard limit defined in hashtable.c. */ +int hashtableResizeAllowed(size_t moreMem, double usedRatio) { + UNUSED(usedRatio); + + /* For debug purposes, not allowed to be resized. */ if (!server.dict_resizing) return 0; - if (usedRatio <= HASHTABLE_MAX_LOAD_FACTOR) { - return !overMaxmemoryAfterAlloc(moreMem); - } else { - return 1; - } + /* Avoid resizing over max memory. */ + return !overMaxmemoryAfterAlloc(moreMem); } const void *hashtableCommandGetKey(const void *element) { @@ -565,32 +564,53 @@ dictType zsetDictType = { NULL, /* allow to expand */ }; +uint64_t hashtableSdsHash(const void *key) { + return hashtableGenHashFunction((const char *)key, sdslen((char *)key)); +} + +const void *hashtableValkeyObjectGetKey(const void *entry) { + return objectGetKey(entry); +} + +int hashtableSdsKeyCompare(const void *key1, const void *key2) { + const sds sds1 = (const sds)key1, sds2 = (const sds)key2; + return sdslen(sds1) != sdslen(sds2) || sdscmp(sds1, sds2); +} + +int hashtableObjKeyCompare(const void *key1, const void *key2) { + const robj *o1 = key1, *o2 = key2; + return hashtableSdsKeyCompare(o1->ptr, o2->ptr); +} + +void hashtableObjectDestructor(void *val) { + if (val == NULL) return; /* Lazy freeing will set value to NULL. */ + decrRefCount(val); +} + /* Kvstore->keys, keys are sds strings, vals are Objects. */ -dictType kvstoreKeysDictType = { - dictSdsHash, /* hash function */ - NULL, /* key dup */ - dictSdsKeyCompare, /* key compare */ - NULL, /* key is embedded in the dictEntry and freed internally */ - dictObjectDestructor, /* val destructor */ - dictResizeAllowed, /* allow to resize */ - kvstoreDictRehashingStarted, - kvstoreDictRehashingCompleted, - kvstoreDictMetadataSize, - .embedKey = dictSdsEmbedKey, - .embedded_entry = 1, +hashtableType kvstoreKeysHashtableType = { + .entryGetKey = hashtableValkeyObjectGetKey, + .hashFunction = hashtableSdsHash, + .keyCompare = hashtableSdsKeyCompare, + .entryDestructor = hashtableObjectDestructor, + .resizeAllowed = hashtableResizeAllowed, + .rehashingStarted = kvstoreHashtableRehashingStarted, + .rehashingCompleted = kvstoreHashtableRehashingCompleted, + .trackMemUsage = kvstoreHashtableTrackMemUsage, + .getMetadataSize = kvstoreHashtableMetadataSize, }; /* Kvstore->expires */ -dictType kvstoreExpiresDictType = { - dictSdsHash, /* hash function */ - NULL, /* key dup */ - dictSdsKeyCompare, /* key compare */ - NULL, /* key destructor */ - NULL, /* val destructor */ - dictResizeAllowed, /* allow to resize */ - kvstoreDictRehashingStarted, - kvstoreDictRehashingCompleted, - kvstoreDictMetadataSize, +hashtableType kvstoreExpiresHashtableType = { + .entryGetKey = hashtableValkeyObjectGetKey, + .hashFunction = hashtableSdsHash, + .keyCompare = hashtableSdsKeyCompare, + .entryDestructor = NULL, /* shared with keyspace table */ + .resizeAllowed = hashtableResizeAllowed, + .rehashingStarted = kvstoreHashtableRehashingStarted, + .rehashingCompleted = kvstoreHashtableRehashingCompleted, + .trackMemUsage = kvstoreHashtableTrackMemUsage, + .getMetadataSize = kvstoreHashtableMetadataSize, }; /* Command set, hashed by sds string, stores serverCommand structs. */ @@ -648,18 +668,33 @@ dictType objToDictDictType = { NULL /* allow to expand */ }; -/* Same as objToDictDictType, added some kvstore callbacks, it's used - * for PUBSUB command to track clients subscribing the channels. */ -dictType kvstoreChannelDictType = { - dictObjHash, /* hash function */ - NULL, /* key dup */ - dictObjKeyCompare, /* key compare */ - dictObjectDestructor, /* key destructor */ - dictDictDestructor, /* val destructor */ - NULL, /* allow to expand */ - kvstoreDictRehashingStarted, - kvstoreDictRehashingCompleted, - kvstoreDictMetadataSize, +/* Callback used for hash tables where the entries are dicts and the key + * (channel name) is stored in each dict's metadata. */ +const void *hashtableChannelsDictGetKey(const void *entry) { + const dict *d = entry; + return *((const void **)d->metadata); +} + +void hashtableChannelsDictDestructor(void *entry) { + dict *d = entry; + robj *channel = *((void **)d->metadata); + decrRefCount(channel); + dictRelease(d); +} + +/* Similar to objToDictDictType, but changed to hashtable and added some kvstore + * callbacks, it's used for PUBSUB command to track clients subscribing the + * channels. The elements are dicts where the keys are clients. The metadata in + * each dict stores a pointer to the channel name. */ +hashtableType kvstoreChannelHashtableType = { + .entryGetKey = hashtableChannelsDictGetKey, + .hashFunction = dictObjHash, + .keyCompare = hashtableObjKeyCompare, + .entryDestructor = hashtableChannelsDictDestructor, + .rehashingStarted = kvstoreHashtableRehashingStarted, + .rehashingCompleted = kvstoreHashtableRehashingCompleted, + .trackMemUsage = kvstoreHashtableTrackMemUsage, + .getMetadataSize = kvstoreHashtableMetadataSize, }; /* Modules system dictionary type. Keys are module name, @@ -716,12 +751,18 @@ dictType sdsHashDictType = { NULL /* allow to expand */ }; +size_t clientSetDictTypeMetadataBytes(dict *d) { + UNUSED(d); + return sizeof(void *); +} + /* Client Set dictionary type. Keys are client, values are not used. */ dictType clientDictType = { dictClientHash, /* hash function */ NULL, /* key dup */ dictClientKeyCompare, /* key compare */ - .no_value = 1 /* no values in this dict */ + .dictMetadataBytes = clientSetDictTypeMetadataBytes, + .no_value = 1 /* no values in this dict */ }; /* This function is called once a background process of some kind terminates, @@ -731,12 +772,16 @@ dictType clientDictType = { * for dict.c to resize or rehash the tables accordingly to the fact we have an * active fork child running. */ void updateDictResizePolicy(void) { - if (server.in_fork_child != CHILD_TYPE_NONE) + if (server.in_fork_child != CHILD_TYPE_NONE) { dictSetResizeEnabled(DICT_RESIZE_FORBID); - else if (hasActiveChildProcess()) + hashtableSetResizePolicy(HASHTABLE_RESIZE_FORBID); + } else if (hasActiveChildProcess()) { dictSetResizeEnabled(DICT_RESIZE_AVOID); - else + hashtableSetResizePolicy(HASHTABLE_RESIZE_AVOID); + } else { dictSetResizeEnabled(DICT_RESIZE_ENABLE); + hashtableSetResizePolicy(HASHTABLE_RESIZE_ALLOW); + } } const char *strChildType(int type) { @@ -1174,8 +1219,8 @@ void databasesCron(void) { for (j = 0; j < dbs_per_call; j++) { serverDb *db = &server.db[resize_db % server.dbnum]; - kvstoreTryResizeDicts(db->keys, CRON_DICTS_PER_DB); - kvstoreTryResizeDicts(db->expires, CRON_DICTS_PER_DB); + kvstoreTryResizeHashtables(db->keys, CRON_DICTS_PER_DB); + kvstoreTryResizeHashtables(db->expires, CRON_DICTS_PER_DB); resize_db++; } @@ -2732,14 +2777,14 @@ void initServer(void) { /* Create the databases, and initialize other internal state. */ int slot_count_bits = 0; - int flags = KVSTORE_ALLOCATE_DICTS_ON_DEMAND; + int flags = KVSTORE_ALLOCATE_HASHTABLES_ON_DEMAND; if (server.cluster_enabled) { slot_count_bits = CLUSTER_SLOT_MASK_BITS; - flags |= KVSTORE_FREE_EMPTY_DICTS; + flags |= KVSTORE_FREE_EMPTY_HASHTABLES; } for (j = 0; j < server.dbnum; j++) { - server.db[j].keys = kvstoreCreate(&kvstoreKeysDictType, slot_count_bits, flags); - server.db[j].expires = kvstoreCreate(&kvstoreExpiresDictType, slot_count_bits, flags); + server.db[j].keys = kvstoreCreate(&kvstoreKeysHashtableType, slot_count_bits, flags); + server.db[j].expires = kvstoreCreate(&kvstoreExpiresHashtableType, slot_count_bits, flags); server.db[j].expires_cursor = 0; server.db[j].blocking_keys = dictCreate(&keylistDictType); server.db[j].blocking_keys_unblock_on_nokey = dictCreate(&objectKeyPointerValueDictType); @@ -2754,10 +2799,10 @@ void initServer(void) { /* Note that server.pubsub_channels was chosen to be a kvstore (with only one dict, which * seems odd) just to make the code cleaner by making it be the same type as server.pubsubshard_channels * (which has to be kvstore), see pubsubtype.serverPubSubChannels */ - server.pubsub_channels = kvstoreCreate(&kvstoreChannelDictType, 0, KVSTORE_ALLOCATE_DICTS_ON_DEMAND); + server.pubsub_channels = kvstoreCreate(&kvstoreChannelHashtableType, 0, KVSTORE_ALLOCATE_HASHTABLES_ON_DEMAND); server.pubsub_patterns = dictCreate(&objToDictDictType); - server.pubsubshard_channels = kvstoreCreate(&kvstoreChannelDictType, slot_count_bits, - KVSTORE_ALLOCATE_DICTS_ON_DEMAND | KVSTORE_FREE_EMPTY_DICTS); + server.pubsubshard_channels = kvstoreCreate(&kvstoreChannelHashtableType, slot_count_bits, + KVSTORE_ALLOCATE_HASHTABLES_ON_DEMAND | KVSTORE_FREE_EMPTY_HASHTABLES); server.pubsub_clients = 0; server.watching_clients = 0; server.cronloops = 0; @@ -6825,6 +6870,7 @@ __attribute__((weak)) int main(int argc, char **argv) { uint8_t hashseed[16]; getRandomBytes(hashseed, sizeof(hashseed)); dictSetHashFunctionSeed(hashseed); + hashtableSetHashFunctionSeed(hashseed); char *exec_name = strrchr(argv[0], '/'); if (exec_name == NULL) exec_name = argv[0]; diff --git a/src/server.h b/src/server.h index 70611e4c2b..f71d417855 100644 --- a/src/server.h +++ b/src/server.h @@ -84,6 +84,7 @@ typedef long long ustime_t; /* microsecond time type. */ #define VALKEYMODULE_CORE 1 typedef struct serverObject robj; +typedef struct serverObject valkey; #include "valkeymodule.h" /* Modules API defines. */ /* Following includes allow test functions to be called from main() */ @@ -209,9 +210,6 @@ struct hdr_histogram; extern int configOOMScoreAdjValuesDefaults[CONFIG_OOM_COUNT]; -/* Hash table parameters */ -#define HASHTABLE_MAX_LOAD_FACTOR 1.618 /* Maximum hash table load factor. */ - /* Command flags. Please check the definition of struct serverCommand in this file * for more information about the meaning of every flag. */ #define CMD_WRITE (1ULL << 0) @@ -880,8 +878,9 @@ struct ValkeyModuleDigest { #define LRU_CLOCK_MAX ((1 << LRU_BITS) - 1) /* Max value of obj->lru */ #define LRU_CLOCK_RESOLUTION 1000 /* LRU clock resolution in ms */ -#define OBJ_SHARED_REFCOUNT INT_MAX /* Global object never destroyed. */ -#define OBJ_STATIC_REFCOUNT (INT_MAX - 1) /* Object allocated in the stack. */ +#define OBJ_REFCOUNT_BITS 30 +#define OBJ_SHARED_REFCOUNT ((1 << OBJ_REFCOUNT_BITS) - 1) /* Global object never destroyed. */ +#define OBJ_STATIC_REFCOUNT ((1 << OBJ_REFCOUNT_BITS) - 2) /* Object allocated in the stack. */ #define OBJ_FIRST_SPECIAL_REFCOUNT OBJ_STATIC_REFCOUNT struct serverObject { unsigned type : 4; @@ -889,7 +888,9 @@ struct serverObject { unsigned lru : LRU_BITS; /* LRU time (relative to global lru_clock) or * LFU data (least significant 8 bits frequency * and most significant 16 bits access time). */ - int refcount; + unsigned hasexpire : 1; + unsigned hasembkey : 1; + unsigned refcount : OBJ_REFCOUNT_BITS; void *ptr; }; @@ -907,6 +908,8 @@ char *getObjectTypeName(robj *); _var.refcount = OBJ_STATIC_REFCOUNT; \ _var.type = OBJ_STRING; \ _var.encoding = OBJ_ENCODING_RAW; \ + _var.hasexpire = 0; \ + _var.hasembkey = 0; \ _var.ptr = _ptr; \ } while (0) @@ -2644,8 +2647,8 @@ extern dictType objectKeyHeapPointerValueDictType; extern dictType setDictType; extern dictType BenchmarkDictType; extern dictType zsetDictType; -extern dictType kvstoreKeysDictType; -extern dictType kvstoreExpiresDictType; +extern hashtableType kvstoreKeysHashtableType; +extern hashtableType kvstoreExpiresHashtableType; extern double R_Zero, R_PosInf, R_NegInf, R_Nan; extern dictType hashDictType; extern dictType stringSetDictType; @@ -2653,7 +2656,7 @@ extern dictType externalStringType; extern dictType sdsHashDictType; extern dictType clientDictType; extern dictType objToDictDictType; -extern dictType kvstoreChannelDictType; +extern hashtableType kvstoreChannelHashtableType; extern dictType modulesDictType; extern dictType sdsReplyDictType; extern dictType keylistDictType; @@ -2989,7 +2992,6 @@ robj *createObject(int type, void *ptr); void initObjectLRUOrLFU(robj *o); robj *createStringObject(const char *ptr, size_t len); robj *createRawStringObject(const char *ptr, size_t len); -robj *createEmbeddedStringObject(const char *ptr, size_t len); robj *tryCreateRawStringObject(const char *ptr, size_t len); robj *tryCreateStringObject(const char *ptr, size_t len); robj *dupStringObject(const robj *o); @@ -3030,11 +3032,15 @@ int collateStringObjects(const robj *a, const robj *b); int equalStringObjects(robj *a, robj *b); unsigned long long estimateObjectIdleTime(robj *o); void trimStringObjectIfNeeded(robj *o, int trim_small_values); -static inline int canUseSharedObject(void) { - return server.maxmemory == 0 || !(server.maxmemory_policy & MAXMEMORY_FLAG_NO_SHARED_INTEGERS); -} #define sdsEncodedObject(objptr) (objptr->encoding == OBJ_ENCODING_RAW || objptr->encoding == OBJ_ENCODING_EMBSTR) +/* Objects with key attached, AKA valkey objects */ +valkey *createObjectWithKeyAndExpire(int type, void *ptr, const sds key, long long expire); +valkey *objectSetKeyAndExpire(valkey *val, sds key, long long expire); +valkey *objectSetExpire(valkey *val, long long expire); +sds objectGetKey(const valkey *val); +long long objectGetExpire(const valkey *val); + /* Synchronous I/O with timeout */ ssize_t syncWrite(int fd, char *ptr, ssize_t size, long long timeout); ssize_t syncRead(int fd, char *ptr, ssize_t size, long long timeout); @@ -3360,10 +3366,10 @@ int calculateKeySlot(sds key); /* kvstore wrappers */ int dbExpand(serverDb *db, uint64_t db_size, int try_expand); int dbExpandExpires(serverDb *db, uint64_t db_size, int try_expand); -dictEntry *dbFind(serverDb *db, void *key); -dictEntry *dbFindExpires(serverDb *db, void *key); +valkey *dbFind(serverDb *db, sds key); +valkey *dbFindExpires(serverDb *db, sds key); unsigned long long dbSize(serverDb *db); -unsigned long long dbScan(serverDb *db, unsigned long long cursor, dictScanFunction *scan_cb, void *privdata); +unsigned long long dbScan(serverDb *db, unsigned long long cursor, hashtableScanFunction scan_cb, void *privdata); /* Set data type */ robj *setTypeCreate(sds value, size_t size_hint); @@ -3522,7 +3528,7 @@ void deleteExpiredKeyFromOverwriteAndPropagate(client *c, robj *keyobj); void propagateDeletion(serverDb *db, robj *key, int lazy); int keyIsExpired(serverDb *db, robj *key); long long getExpire(serverDb *db, robj *key); -void setExpire(client *c, serverDb *db, robj *key, long long when); +robj *setExpire(client *c, serverDb *db, robj *key, long long when); int checkAlreadyExpired(long long when); robj *lookupKeyRead(serverDb *db, robj *key); robj *lookupKeyWrite(serverDb *db, robj *key); @@ -3542,16 +3548,16 @@ int objectSetLRUOrLFU(robj *val, long long lfu_freq, long long lru_idle, long lo #define LOOKUP_NOEFFECTS \ (LOOKUP_NONOTIFY | LOOKUP_NOSTATS | LOOKUP_NOTOUCH | LOOKUP_NOEXPIRE) /* Avoid any effects from fetching the key */ -void dbAdd(serverDb *db, robj *key, robj *val); -int dbAddRDBLoad(serverDb *db, sds key, robj *val); -void dbReplaceValue(serverDb *db, robj *key, robj *val); +valkey *dbAdd(serverDb *db, robj *key, robj *val); +valkey *dbAddRDBLoad(serverDb *db, sds key, robj *val); +valkey *dbReplaceValue(serverDb *db, robj *key, robj *val); #define SETKEY_KEEPTTL 1 #define SETKEY_NO_SIGNAL 2 #define SETKEY_ALREADY_EXIST 4 #define SETKEY_DOESNT_EXIST 8 #define SETKEY_ADD_OR_UPDATE 16 /* Key most likely doesn't exists */ -void setKey(client *c, serverDb *db, robj *key, robj *val, int flags); +robj *setKey(client *c, serverDb *db, robj *key, robj *val, int flags); robj *dbRandomKey(serverDb *db); int dbGenericDelete(serverDb *db, robj *key, int async, int flags); int dbSyncDelete(serverDb *db, robj *key); @@ -3561,12 +3567,12 @@ robj *dbUnshareStringValue(serverDb *db, robj *key, robj *o); #define EMPTYDB_NO_FLAGS 0 /* No flags. */ #define EMPTYDB_ASYNC (1 << 0) /* Reclaim memory in another thread. */ #define EMPTYDB_NOFUNCTIONS (1 << 1) /* Indicate not to flush the functions. */ -long long emptyData(int dbnum, int flags, void(callback)(dict *)); -long long emptyDbStructure(serverDb *dbarray, int dbnum, int async, void(callback)(dict *)); +long long emptyData(int dbnum, int flags, void(callback)(hashtable *)); +long long emptyDbStructure(serverDb *dbarray, int dbnum, int async, void(callback)(hashtable *)); void flushAllDataAndResetRDB(int flags); long long dbTotalServerKeyCount(void); serverDb *initTempDb(void); -void discardTempDb(serverDb *tempDb, void(callback)(dict *)); +void discardTempDb(serverDb *tempDb, void(callback)(hashtable *)); int selectDb(client *c, int id); diff --git a/src/sort.c b/src/sort.c index 92777b068c..86487a1fb9 100644 --- a/src/sort.c +++ b/src/sort.c @@ -578,6 +578,9 @@ void sortCommandGeneric(client *c, int readonly) { if (outputlen) { listTypeTryConversion(sobj, LIST_CONV_AUTO, NULL, NULL); setKey(c, c->db, storekey, sobj, 0); + /* Ownership of sobj transferred to the db. Set to NULL to prevent + * freeing it below. */ + sobj = NULL; notifyKeyspaceEvent(NOTIFY_LIST, "sortstore", storekey, c->db->id); server.dirty += outputlen; } else if (dbDelete(c->db, storekey)) { @@ -585,7 +588,7 @@ void sortCommandGeneric(client *c, int readonly) { notifyKeyspaceEvent(NOTIFY_GENERIC, "del", storekey, c->db->id); server.dirty++; } - decrRefCount(sobj); + if (sobj != NULL) decrRefCount(sobj); addReplyLongLong(c, outputlen); } diff --git a/src/t_hash.c b/src/t_hash.c index dabe279808..375aa9f3c9 100644 --- a/src/t_hash.c +++ b/src/t_hash.c @@ -432,7 +432,7 @@ robj *hashTypeLookupWriteOrCreate(client *c, robj *key) { if (o == NULL) { o = createHashObject(); - dbAdd(c->db, key, o); + o = dbAdd(c->db, key, o); } return o; } diff --git a/src/t_list.c b/src/t_list.c index ffe3e9b08a..6791a59dfa 100644 --- a/src/t_list.c +++ b/src/t_list.c @@ -471,7 +471,7 @@ void pushGenericCommand(client *c, int where, int xx) { } lobj = createListListpackObject(); - dbAdd(c->db, c->argv[1], lobj); + lobj = dbAdd(c->db, c->argv[1], lobj); } listTypeTryConversionAppend(lobj, c->argv, 2, c->argc - 1, NULL, NULL); @@ -1068,7 +1068,7 @@ void lmoveHandlePush(client *c, robj *dstkey, robj *dstobj, robj *value, int whe /* Create the list if the key does not exist */ if (!dstobj) { dstobj = createListListpackObject(); - dbAdd(c->db, dstkey, dstobj); + dstobj = dbAdd(c->db, dstkey, dstobj); } listTypeTryConversionAppend(dstobj, &value, 0, 0, NULL, NULL); listTypePush(dstobj, value, where); diff --git a/src/t_set.c b/src/t_set.c index a540c3c49b..a4acb58fcf 100644 --- a/src/t_set.c +++ b/src/t_set.c @@ -595,7 +595,7 @@ void saddCommand(client *c) { if (set == NULL) { set = setTypeCreate(c->argv[2]->ptr, c->argc - 2); - dbAdd(c->db, c->argv[1], set); + set = dbAdd(c->db, c->argv[1], set); } else { setTypeMaybeConvert(set, c->argc - 2); } @@ -674,7 +674,7 @@ void smoveCommand(client *c) { /* Create the destination set when it doesn't exist */ if (!dstset) { dstset = setTypeCreate(ele->ptr, 1); - dbAdd(c->db, c->argv[2], dstset); + dstset = dbAdd(c->db, c->argv[2], dstset); } signalModifiedKey(c, c->db, c->argv[1]); @@ -919,7 +919,7 @@ void spopWithCountCommand(client *c) { setTypeReleaseIterator(si); /* Assign the new set as the key value. */ - dbReplaceValue(c->db, c->argv[1], newset); + newset = dbReplaceValue(c->db, c->argv[1], newset); } /* Replicate/AOF the remaining elements as an SREM operation */ @@ -1383,7 +1383,7 @@ void sinterGenericCommand(client *c, * frequent reallocs. Therefore, we shrink it now. */ dstset->ptr = lpShrinkToFit(dstset->ptr); } - setKey(c, c->db, dstkey, dstset, 0); + dstset = setKey(c, c->db, dstkey, dstset, 0); addReplyLongLong(c, setTypeSize(dstset)); notifyKeyspaceEvent(NOTIFY_SET, "sinterstore", dstkey, c->db->id); server.dirty++; @@ -1394,8 +1394,8 @@ void sinterGenericCommand(client *c, signalModifiedKey(c, c->db, dstkey); notifyKeyspaceEvent(NOTIFY_GENERIC, "del", dstkey, c->db->id); } + decrRefCount(dstset); } - decrRefCount(dstset); } else { setDeferredSetLen(c, replylen, cardinality); } @@ -1607,7 +1607,7 @@ void sunionDiffGenericCommand(client *c, robj **setkeys, int setnum, robj *dstke /* If we have a target key where to store the resulting set * create this key with the result set inside */ if (setTypeSize(dstset) > 0) { - setKey(c, c->db, dstkey, dstset, 0); + dstset = setKey(c, c->db, dstkey, dstset, 0); addReplyLongLong(c, setTypeSize(dstset)); notifyKeyspaceEvent(NOTIFY_SET, op == SET_OP_UNION ? "sunionstore" : "sdiffstore", dstkey, c->db->id); server.dirty++; @@ -1618,8 +1618,8 @@ void sunionDiffGenericCommand(client *c, robj **setkeys, int setnum, robj *dstke signalModifiedKey(c, c->db, dstkey); notifyKeyspaceEvent(NOTIFY_GENERIC, "del", dstkey, c->db->id); } + decrRefCount(dstset); } - decrRefCount(dstset); } zfree(sets); } diff --git a/src/t_stream.c b/src/t_stream.c index a42822dabc..2d19825a98 100644 --- a/src/t_stream.c +++ b/src/t_stream.c @@ -1839,7 +1839,7 @@ robj *streamTypeLookupWriteOrCreate(client *c, robj *key, int no_create) { return NULL; } o = createStreamObject(); - dbAdd(c->db, key, o); + o = dbAdd(c->db, key, o); } return o; } @@ -2645,7 +2645,7 @@ void xgroupCommand(client *c) { if (s == NULL) { serverAssert(mkstream); o = createStreamObject(); - dbAdd(c->db, c->argv[2], o); + o = dbAdd(c->db, c->argv[2], o); s = o->ptr; signalModifiedKey(c, c->db, c->argv[2]); } diff --git a/src/t_string.c b/src/t_string.c index 1c90eabf3e..b961daaddd 100644 --- a/src/t_string.c +++ b/src/t_string.c @@ -76,6 +76,8 @@ static int checkStringLength(client *c, long long size, long long append) { #define OBJ_EXAT (1 << 6) /* Set if timestamp in second is given */ #define OBJ_PXAT (1 << 7) /* Set if timestamp in ms is given */ #define OBJ_PERSIST (1 << 8) /* Set if we need to remove the ttl */ +#define OBJ_ARGV3 (1 << 9) /* Set if the value is at argv[3]; otherwise it's \ + * at argv[2]. */ /* Forward declaration */ static int getExpireMillisecondsOrReply(client *c, robj *expire, int flags, int unit, long long *milliseconds); @@ -123,12 +125,19 @@ void setGenericCommand(client *c, setkey_flags |= ((flags & OBJ_KEEPTTL) || expire) ? SETKEY_KEEPTTL : 0; setkey_flags |= found ? SETKEY_ALREADY_EXIST : SETKEY_DOESNT_EXIST; - setKey(c, c->db, key, val, setkey_flags); + /* Retain val, c->argv refers to it. This causes setVal to duplicate the string. */ + val = setKey(c, c->db, key, val, setkey_flags); + if (expire) val = setExpire(c, c->db, key, milliseconds); + + /* By setting the reallocated value back into argv, we can avoid duplicating + * a large string value when adding it to the db. */ + c->argv[(flags & OBJ_ARGV3) ? 3 : 2] = val; + incrRefCount(val); + server.dirty++; notifyKeyspaceEvent(NOTIFY_STRING, "set", key, c->db->id); if (expire) { - setExpire(c, c->db, key, milliseconds); /* Propagate as SET Key Value PXAT millisecond-timestamp if there is * EX/PX/EXAT flag. */ if (!(flags & OBJ_PXAT)) { @@ -326,12 +335,12 @@ void setnxCommand(client *c) { void setexCommand(client *c) { c->argv[3] = tryObjectEncoding(c->argv[3]); - setGenericCommand(c, OBJ_EX, c->argv[1], c->argv[3], c->argv[2], UNIT_SECONDS, NULL, NULL); + setGenericCommand(c, OBJ_EX | OBJ_ARGV3, c->argv[1], c->argv[3], c->argv[2], UNIT_SECONDS, NULL, NULL); } void psetexCommand(client *c) { c->argv[3] = tryObjectEncoding(c->argv[3]); - setGenericCommand(c, OBJ_PX, c->argv[1], c->argv[3], c->argv[2], UNIT_MILLISECONDS, NULL, NULL); + setGenericCommand(c, OBJ_PX | OBJ_ARGV3, c->argv[1], c->argv[3], c->argv[2], UNIT_MILLISECONDS, NULL, NULL); } int getGenericCommand(client *c) { @@ -406,7 +415,7 @@ void getexCommand(client *c) { * has already elapsed so delete the key in that case. */ deleteExpiredKeyFromOverwriteAndPropagate(c, c->argv[1]); } else if (expire) { - setExpire(c, c->db, c->argv[1], milliseconds); + o = setExpire(c, c->db, c->argv[1], milliseconds); /* Propagate as PXEXPIREAT millisecond-timestamp if there is * EX/PX/EXAT/PXAT flag and the key has not expired. */ robj *milliseconds_obj = createStringObjectFromLongLong(milliseconds); @@ -439,7 +448,8 @@ void getdelCommand(client *c) { void getsetCommand(client *c) { if (getGenericCommand(c) == C_ERR) return; c->argv[2] = tryObjectEncoding(c->argv[2]); - setKey(c, c->db, c->argv[1], c->argv[2], 0); + c->argv[2] = setKey(c, c->db, c->argv[1], c->argv[2], 0); + incrRefCount(c->argv[2]); notifyKeyspaceEvent(NOTIFY_STRING, "set", c->argv[1], c->db->id); server.dirty++; @@ -473,7 +483,7 @@ void setrangeCommand(client *c) { return; o = createObject(OBJ_STRING, sdsnewlen(NULL, offset + sdslen(value))); - dbAdd(c->db, c->argv[1], o); + o = dbAdd(c->db, c->argv[1], o); } else { size_t olen; @@ -587,8 +597,10 @@ void msetGenericCommand(client *c, int nx) { int setkey_flags = nx ? SETKEY_DOESNT_EXIST : 0; for (j = 1; j < c->argc; j += 2) { - c->argv[j + 1] = tryObjectEncoding(c->argv[j + 1]); - setKey(c, c->db, c->argv[j], c->argv[j + 1], setkey_flags); + robj *val = tryObjectEncoding(c->argv[j + 1]); + val = setKey(c, c->db, c->argv[j], val, setkey_flags); + incrRefCount(val); + c->argv[j + 1] = val; notifyKeyspaceEvent(NOTIFY_STRING, "set", c->argv[j], c->db->id); /* In MSETNX, It could be that we're overriding the same key, we can't be sure it doesn't exist. */ if (nx) @@ -623,16 +635,15 @@ void incrDecrCommand(client *c, long long incr) { value += incr; if (o && o->refcount == 1 && o->encoding == OBJ_ENCODING_INT && - (value < 0 || value >= OBJ_SHARED_INTEGERS) && value >= LONG_MIN && value <= LONG_MAX) { new = o; o->ptr = (void *)((long)value); } else { new = createStringObjectFromLongLongForValue(value); if (o) { - dbReplaceValue(c->db, c->argv[1], new); + new = dbReplaceValue(c->db, c->argv[1], new); } else { - dbAdd(c->db, c->argv[1], new); + new = dbAdd(c->db, c->argv[1], new); } } signalModifiedKey(c, c->db, c->argv[1]); @@ -685,9 +696,9 @@ void incrbyfloatCommand(client *c) { } new = createStringObjectFromLongDouble(value, 1); if (o) - dbReplaceValue(c->db, c->argv[1], new); + new = dbReplaceValue(c->db, c->argv[1], new); else - dbAdd(c->db, c->argv[1], new); + new = dbAdd(c->db, c->argv[1], new); signalModifiedKey(c, c->db, c->argv[1]); notifyKeyspaceEvent(NOTIFY_STRING, "incrbyfloat", c->argv[1], c->db->id); server.dirty++; @@ -709,7 +720,7 @@ void appendCommand(client *c) { if (o == NULL) { /* Create the key */ c->argv[2] = tryObjectEncoding(c->argv[2]); - dbAdd(c->db, c->argv[1], c->argv[2]); + c->argv[2] = dbAdd(c->db, c->argv[1], c->argv[2]); incrRefCount(c->argv[2]); totlen = stringObjectLen(c->argv[2]); } else { diff --git a/src/t_zset.c b/src/t_zset.c index 069ab0924a..18fbba6ec7 100644 --- a/src/t_zset.c +++ b/src/t_zset.c @@ -1803,7 +1803,7 @@ void zaddGenericCommand(client *c, int flags) { if (zobj == NULL) { if (xx) goto reply_to_client; /* No key + XX option: nothing to do. */ zobj = zsetTypeCreate(elements, maxelelen); - dbAdd(c->db, key, zobj); + zobj = dbAdd(c->db, key, zobj); } else { zsetTypeMaybeConvert(zobj, elements, maxelelen); } @@ -2782,7 +2782,7 @@ void zunionInterDiffGenericCommand(client *c, robj *dstkey, int numkeysIndex, in if (dstkey) { if (dstzset->zsl->length) { zsetConvertToListpackIfNeeded(dstobj, maxelelen, totelelen); - setKey(c, c->db, dstkey, dstobj, 0); + dstobj = setKey(c, c->db, dstkey, dstobj, 0); addReplyLongLong(c, zsetLength(dstobj)); notifyKeyspaceEvent( NOTIFY_ZSET, (op == SET_OP_UNION) ? "zunionstore" : (op == SET_OP_INTER ? "zinterstore" : "zdiffstore"), @@ -2795,8 +2795,8 @@ void zunionInterDiffGenericCommand(client *c, robj *dstkey, int numkeysIndex, in notifyKeyspaceEvent(NOTIFY_GENERIC, "del", dstkey, c->db->id); server.dirty++; } + decrRefCount(dstobj); } - decrRefCount(dstobj); } else if (cardinality_only) { addReplyLongLong(c, cardinality); } else { @@ -2985,7 +2985,7 @@ static void zrangeResultEmitLongLongForStore(zrange_result_handler *handler, lon static void zrangeResultFinalizeStore(zrange_result_handler *handler, size_t result_count) { if (result_count) { - setKey(handler->client, handler->client->db, handler->dstkey, handler->dstobj, 0); + handler->dstobj = setKey(handler->client, handler->client->db, handler->dstkey, handler->dstobj, 0); addReplyLongLong(handler->client, result_count); notifyKeyspaceEvent(NOTIFY_ZSET, "zrangestore", handler->dstkey, handler->client->db->id); server.dirty++; @@ -2996,8 +2996,8 @@ static void zrangeResultFinalizeStore(zrange_result_handler *handler, size_t res notifyKeyspaceEvent(NOTIFY_GENERIC, "del", handler->dstkey, handler->client->db->id); server.dirty++; } + decrRefCount(handler->dstobj); } - decrRefCount(handler->dstobj); } /* Initialize the consumer interface type with the requested type. */ diff --git a/src/unit/test_files.h b/src/unit/test_files.h index 7a3a505051..bc0c9bbcad 100644 --- a/src/unit/test_files.h +++ b/src/unit/test_files.h @@ -43,10 +43,10 @@ int test_intsetUpgradeFromint32Toint64(int argc, char **argv, int flags); int test_intsetStressLookups(int argc, char **argv, int flags); int test_intsetStressAddDelete(int argc, char **argv, int flags); int test_kvstoreAdd16Keys(int argc, char **argv, int flags); -int test_kvstoreIteratorRemoveAllKeysNoDeleteEmptyDict(int argc, char **argv, int flags); -int test_kvstoreIteratorRemoveAllKeysDeleteEmptyDict(int argc, char **argv, int flags); -int test_kvstoreDictIteratorRemoveAllKeysNoDeleteEmptyDict(int argc, char **argv, int flags); -int test_kvstoreDictIteratorRemoveAllKeysDeleteEmptyDict(int argc, char **argv, int flags); +int test_kvstoreIteratorRemoveAllKeysNoDeleteEmptyHashtable(int argc, char **argv, int flags); +int test_kvstoreIteratorRemoveAllKeysDeleteEmptyHashtable(int argc, char **argv, int flags); +int test_kvstoreHashtableIteratorRemoveAllKeysNoDeleteEmptyHashtable(int argc, char **argv, int flags); +int test_kvstoreHashtableIteratorRemoveAllKeysDeleteEmptyHashtable(int argc, char **argv, int flags); int test_listpackCreateIntList(int argc, char **argv, int flags); int test_listpackCreateList(int argc, char **argv, int flags); int test_listpackLpPrepend(int argc, char **argv, int flags); @@ -99,6 +99,7 @@ int test_listpackBenchmarkLpValidateIntegrity(int argc, char **argv, int flags); int test_listpackBenchmarkLpCompareWithString(int argc, char **argv, int flags); int test_listpackBenchmarkLpCompareWithNumber(int argc, char **argv, int flags); int test_listpackBenchmarkFree(int argc, char **argv, int flags); +int test_object_with_key(int argc, char **argv, int flags); int test_quicklistCreateList(int argc, char **argv, int flags); int test_quicklistAddToTailOfEmptyList(int argc, char **argv, int flags); int test_quicklistAddToHeadOfEmptyList(int argc, char **argv, int flags); @@ -229,8 +230,9 @@ unitTest __test_dict_c[] = {{"test_dictCreate", test_dictCreate}, {"test_dictAdd unitTest __test_endianconv_c[] = {{"test_endianconv", test_endianconv}, {NULL, NULL}}; unitTest __test_hashtable_c[] = {{"test_cursor", test_cursor}, {"test_set_hash_function_seed", test_set_hash_function_seed}, {"test_add_find_delete", test_add_find_delete}, {"test_add_find_delete_avoid_resize", test_add_find_delete_avoid_resize}, {"test_instant_rehashing", test_instant_rehashing}, {"test_bucket_chain_length", test_bucket_chain_length}, {"test_two_phase_insert_and_pop", test_two_phase_insert_and_pop}, {"test_incremental_find", test_incremental_find}, {"test_scan", test_scan}, {"test_iterator", test_iterator}, {"test_safe_iterator", test_safe_iterator}, {"test_compact_bucket_chain", test_compact_bucket_chain}, {"test_random_entry", test_random_entry}, {"test_random_entry_with_long_chain", test_random_entry_with_long_chain}, {"test_all_memory_freed", test_all_memory_freed}, {NULL, NULL}}; unitTest __test_intset_c[] = {{"test_intsetValueEncodings", test_intsetValueEncodings}, {"test_intsetBasicAdding", test_intsetBasicAdding}, {"test_intsetLargeNumberRandomAdd", test_intsetLargeNumberRandomAdd}, {"test_intsetUpgradeFromint16Toint32", test_intsetUpgradeFromint16Toint32}, {"test_intsetUpgradeFromint16Toint64", test_intsetUpgradeFromint16Toint64}, {"test_intsetUpgradeFromint32Toint64", test_intsetUpgradeFromint32Toint64}, {"test_intsetStressLookups", test_intsetStressLookups}, {"test_intsetStressAddDelete", test_intsetStressAddDelete}, {NULL, NULL}}; -unitTest __test_kvstore_c[] = {{"test_kvstoreAdd16Keys", test_kvstoreAdd16Keys}, {"test_kvstoreIteratorRemoveAllKeysNoDeleteEmptyDict", test_kvstoreIteratorRemoveAllKeysNoDeleteEmptyDict}, {"test_kvstoreIteratorRemoveAllKeysDeleteEmptyDict", test_kvstoreIteratorRemoveAllKeysDeleteEmptyDict}, {"test_kvstoreDictIteratorRemoveAllKeysNoDeleteEmptyDict", test_kvstoreDictIteratorRemoveAllKeysNoDeleteEmptyDict}, {"test_kvstoreDictIteratorRemoveAllKeysDeleteEmptyDict", test_kvstoreDictIteratorRemoveAllKeysDeleteEmptyDict}, {NULL, NULL}}; +unitTest __test_kvstore_c[] = {{"test_kvstoreAdd16Keys", test_kvstoreAdd16Keys}, {"test_kvstoreIteratorRemoveAllKeysNoDeleteEmptyHashtable", test_kvstoreIteratorRemoveAllKeysNoDeleteEmptyHashtable}, {"test_kvstoreIteratorRemoveAllKeysDeleteEmptyHashtable", test_kvstoreIteratorRemoveAllKeysDeleteEmptyHashtable}, {"test_kvstoreHashtableIteratorRemoveAllKeysNoDeleteEmptyHashtable", test_kvstoreHashtableIteratorRemoveAllKeysNoDeleteEmptyHashtable}, {"test_kvstoreHashtableIteratorRemoveAllKeysDeleteEmptyHashtable", test_kvstoreHashtableIteratorRemoveAllKeysDeleteEmptyHashtable}, {NULL, NULL}}; unitTest __test_listpack_c[] = {{"test_listpackCreateIntList", test_listpackCreateIntList}, {"test_listpackCreateList", test_listpackCreateList}, {"test_listpackLpPrepend", test_listpackLpPrepend}, {"test_listpackLpPrependInteger", test_listpackLpPrependInteger}, {"test_listpackGetELementAtIndex", test_listpackGetELementAtIndex}, {"test_listpackPop", test_listpackPop}, {"test_listpackGetELementAtIndex2", test_listpackGetELementAtIndex2}, {"test_listpackIterate0toEnd", test_listpackIterate0toEnd}, {"test_listpackIterate1toEnd", test_listpackIterate1toEnd}, {"test_listpackIterate2toEnd", test_listpackIterate2toEnd}, {"test_listpackIterateBackToFront", test_listpackIterateBackToFront}, {"test_listpackIterateBackToFrontWithDelete", test_listpackIterateBackToFrontWithDelete}, {"test_listpackDeleteWhenNumIsMinusOne", test_listpackDeleteWhenNumIsMinusOne}, {"test_listpackDeleteWithNegativeIndex", test_listpackDeleteWithNegativeIndex}, {"test_listpackDeleteInclusiveRange0_0", test_listpackDeleteInclusiveRange0_0}, {"test_listpackDeleteInclusiveRange0_1", test_listpackDeleteInclusiveRange0_1}, {"test_listpackDeleteInclusiveRange1_2", test_listpackDeleteInclusiveRange1_2}, {"test_listpackDeleteWitStartIndexOutOfRange", test_listpackDeleteWitStartIndexOutOfRange}, {"test_listpackDeleteWitNumOverflow", test_listpackDeleteWitNumOverflow}, {"test_listpackBatchDelete", test_listpackBatchDelete}, {"test_listpackDeleteFooWhileIterating", test_listpackDeleteFooWhileIterating}, {"test_listpackReplaceWithSameSize", test_listpackReplaceWithSameSize}, {"test_listpackReplaceWithDifferentSize", test_listpackReplaceWithDifferentSize}, {"test_listpackRegressionGt255Bytes", test_listpackRegressionGt255Bytes}, {"test_listpackCreateLongListAndCheckIndices", test_listpackCreateLongListAndCheckIndices}, {"test_listpackCompareStrsWithLpEntries", test_listpackCompareStrsWithLpEntries}, {"test_listpackLpMergeEmptyLps", test_listpackLpMergeEmptyLps}, {"test_listpackLpMergeLp1Larger", test_listpackLpMergeLp1Larger}, {"test_listpackLpMergeLp2Larger", test_listpackLpMergeLp2Larger}, {"test_listpackLpNextRandom", test_listpackLpNextRandom}, {"test_listpackLpNextRandomCC", test_listpackLpNextRandomCC}, {"test_listpackRandomPairWithOneElement", test_listpackRandomPairWithOneElement}, {"test_listpackRandomPairWithManyElements", test_listpackRandomPairWithManyElements}, {"test_listpackRandomPairsWithOneElement", test_listpackRandomPairsWithOneElement}, {"test_listpackRandomPairsWithManyElements", test_listpackRandomPairsWithManyElements}, {"test_listpackRandomPairsUniqueWithOneElement", test_listpackRandomPairsUniqueWithOneElement}, {"test_listpackRandomPairsUniqueWithManyElements", test_listpackRandomPairsUniqueWithManyElements}, {"test_listpackPushVariousEncodings", test_listpackPushVariousEncodings}, {"test_listpackLpFind", test_listpackLpFind}, {"test_listpackLpValidateIntegrity", test_listpackLpValidateIntegrity}, {"test_listpackNumberOfElementsExceedsLP_HDR_NUMELE_UNKNOWN", test_listpackNumberOfElementsExceedsLP_HDR_NUMELE_UNKNOWN}, {"test_listpackStressWithRandom", test_listpackStressWithRandom}, {"test_listpackSTressWithVariableSize", test_listpackSTressWithVariableSize}, {"test_listpackBenchmarkInit", test_listpackBenchmarkInit}, {"test_listpackBenchmarkLpAppend", test_listpackBenchmarkLpAppend}, {"test_listpackBenchmarkLpFindString", test_listpackBenchmarkLpFindString}, {"test_listpackBenchmarkLpFindNumber", test_listpackBenchmarkLpFindNumber}, {"test_listpackBenchmarkLpSeek", test_listpackBenchmarkLpSeek}, {"test_listpackBenchmarkLpValidateIntegrity", test_listpackBenchmarkLpValidateIntegrity}, {"test_listpackBenchmarkLpCompareWithString", test_listpackBenchmarkLpCompareWithString}, {"test_listpackBenchmarkLpCompareWithNumber", test_listpackBenchmarkLpCompareWithNumber}, {"test_listpackBenchmarkFree", test_listpackBenchmarkFree}, {NULL, NULL}}; +unitTest __test_object_c[] = {{"test_object_with_key", test_object_with_key}, {NULL, NULL}}; unitTest __test_quicklist_c[] = {{"test_quicklistCreateList", test_quicklistCreateList}, {"test_quicklistAddToTailOfEmptyList", test_quicklistAddToTailOfEmptyList}, {"test_quicklistAddToHeadOfEmptyList", test_quicklistAddToHeadOfEmptyList}, {"test_quicklistAddToTail5xAtCompress", test_quicklistAddToTail5xAtCompress}, {"test_quicklistAddToHead5xAtCompress", test_quicklistAddToHead5xAtCompress}, {"test_quicklistAddToTail500xAtCompress", test_quicklistAddToTail500xAtCompress}, {"test_quicklistAddToHead500xAtCompress", test_quicklistAddToHead500xAtCompress}, {"test_quicklistRotateEmpty", test_quicklistRotateEmpty}, {"test_quicklistComprassionPlainNode", test_quicklistComprassionPlainNode}, {"test_quicklistNextPlainNode", test_quicklistNextPlainNode}, {"test_quicklistRotatePlainNode", test_quicklistRotatePlainNode}, {"test_quicklistRotateOneValOnce", test_quicklistRotateOneValOnce}, {"test_quicklistRotate500Val5000TimesAtCompress", test_quicklistRotate500Val5000TimesAtCompress}, {"test_quicklistPopEmpty", test_quicklistPopEmpty}, {"test_quicklistPop1StringFrom1", test_quicklistPop1StringFrom1}, {"test_quicklistPopHead1NumberFrom1", test_quicklistPopHead1NumberFrom1}, {"test_quicklistPopHead500From500", test_quicklistPopHead500From500}, {"test_quicklistPopHead5000From500", test_quicklistPopHead5000From500}, {"test_quicklistIterateForwardOver500List", test_quicklistIterateForwardOver500List}, {"test_quicklistIterateReverseOver500List", test_quicklistIterateReverseOver500List}, {"test_quicklistInsertAfter1Element", test_quicklistInsertAfter1Element}, {"test_quicklistInsertBefore1Element", test_quicklistInsertBefore1Element}, {"test_quicklistInsertHeadWhileHeadNodeIsFull", test_quicklistInsertHeadWhileHeadNodeIsFull}, {"test_quicklistInsertTailWhileTailNodeIsFull", test_quicklistInsertTailWhileTailNodeIsFull}, {"test_quicklistInsertOnceInElementsWhileIteratingAtCompress", test_quicklistInsertOnceInElementsWhileIteratingAtCompress}, {"test_quicklistInsertBefore250NewInMiddleOf500ElementsAtCompress", test_quicklistInsertBefore250NewInMiddleOf500ElementsAtCompress}, {"test_quicklistInsertAfter250NewInMiddleOf500ElementsAtCompress", test_quicklistInsertAfter250NewInMiddleOf500ElementsAtCompress}, {"test_quicklistDuplicateEmptyList", test_quicklistDuplicateEmptyList}, {"test_quicklistDuplicateListOf1Element", test_quicklistDuplicateListOf1Element}, {"test_quicklistDuplicateListOf500", test_quicklistDuplicateListOf500}, {"test_quicklistIndex1200From500ListAtFill", test_quicklistIndex1200From500ListAtFill}, {"test_quicklistIndex12From500ListAtFill", test_quicklistIndex12From500ListAtFill}, {"test_quicklistIndex100From500ListAtFill", test_quicklistIndex100From500ListAtFill}, {"test_quicklistIndexTooBig1From50ListAtFill", test_quicklistIndexTooBig1From50ListAtFill}, {"test_quicklistDeleteRangeEmptyList", test_quicklistDeleteRangeEmptyList}, {"test_quicklistDeleteRangeOfEntireNodeInListOfOneNode", test_quicklistDeleteRangeOfEntireNodeInListOfOneNode}, {"test_quicklistDeleteRangeOfEntireNodeWithOverflowCounts", test_quicklistDeleteRangeOfEntireNodeWithOverflowCounts}, {"test_quicklistDeleteMiddle100Of500List", test_quicklistDeleteMiddle100Of500List}, {"test_quicklistDeleteLessThanFillButAcrossNodes", test_quicklistDeleteLessThanFillButAcrossNodes}, {"test_quicklistDeleteNegative1From500List", test_quicklistDeleteNegative1From500List}, {"test_quicklistDeleteNegative1From500ListWithOverflowCounts", test_quicklistDeleteNegative1From500ListWithOverflowCounts}, {"test_quicklistDeleteNegative100From500List", test_quicklistDeleteNegative100From500List}, {"test_quicklistDelete10Count5From50List", test_quicklistDelete10Count5From50List}, {"test_quicklistNumbersOnlyListRead", test_quicklistNumbersOnlyListRead}, {"test_quicklistNumbersLargerListRead", test_quicklistNumbersLargerListRead}, {"test_quicklistNumbersLargerListReadB", test_quicklistNumbersLargerListReadB}, {"test_quicklistLremTestAtCompress", test_quicklistLremTestAtCompress}, {"test_quicklistIterateReverseDeleteAtCompress", test_quicklistIterateReverseDeleteAtCompress}, {"test_quicklistIteratorAtIndexTestAtCompress", test_quicklistIteratorAtIndexTestAtCompress}, {"test_quicklistLtrimTestAAtCompress", test_quicklistLtrimTestAAtCompress}, {"test_quicklistLtrimTestBAtCompress", test_quicklistLtrimTestBAtCompress}, {"test_quicklistLtrimTestCAtCompress", test_quicklistLtrimTestCAtCompress}, {"test_quicklistLtrimTestDAtCompress", test_quicklistLtrimTestDAtCompress}, {"test_quicklistVerifySpecificCompressionOfInteriorNodes", test_quicklistVerifySpecificCompressionOfInteriorNodes}, {"test_quicklistBookmarkGetUpdatedToNextItem", test_quicklistBookmarkGetUpdatedToNextItem}, {"test_quicklistBookmarkLimit", test_quicklistBookmarkLimit}, {"test_quicklistCompressAndDecompressQuicklistListpackNode", test_quicklistCompressAndDecompressQuicklistListpackNode}, {"test_quicklistCompressAndDecomressQuicklistPlainNodeLargeThanUINT32MAX", test_quicklistCompressAndDecomressQuicklistPlainNodeLargeThanUINT32MAX}, {NULL, NULL}}; unitTest __test_rax_c[] = {{"test_raxRandomWalk", test_raxRandomWalk}, {"test_raxIteratorUnitTests", test_raxIteratorUnitTests}, {"test_raxTryInsertUnitTests", test_raxTryInsertUnitTests}, {"test_raxRegressionTest1", test_raxRegressionTest1}, {"test_raxRegressionTest2", test_raxRegressionTest2}, {"test_raxRegressionTest3", test_raxRegressionTest3}, {"test_raxRegressionTest4", test_raxRegressionTest4}, {"test_raxRegressionTest5", test_raxRegressionTest5}, {"test_raxRegressionTest6", test_raxRegressionTest6}, {"test_raxBenchmark", test_raxBenchmark}, {"test_raxHugeKey", test_raxHugeKey}, {"test_raxFuzz", test_raxFuzz}, {NULL, NULL}}; unitTest __test_sds_c[] = {{"test_sds", test_sds}, {"test_typesAndAllocSize", test_typesAndAllocSize}, {"test_sdsHeaderSizes", test_sdsHeaderSizes}, {"test_sdssplitargs", test_sdssplitargs}, {NULL, NULL}}; @@ -252,6 +254,7 @@ struct unitTestSuite { {"test_intset.c", __test_intset_c}, {"test_kvstore.c", __test_kvstore_c}, {"test_listpack.c", __test_listpack_c}, + {"test_object.c", __test_object_c}, {"test_quicklist.c", __test_quicklist_c}, {"test_rax.c", __test_rax_c}, {"test_sds.c", __test_sds_c}, diff --git a/src/unit/test_kvstore.c b/src/unit/test_kvstore.c index 062b9f32fc..d4cc91d6d8 100644 --- a/src/unit/test_kvstore.c +++ b/src/unit/test_kvstore.c @@ -2,22 +2,26 @@ #include "test_help.h" uint64_t hashTestCallback(const void *key) { - return dictGenHashFunction((unsigned char *)key, strlen((char *)key)); + return hashtableGenHashFunction((char *)key, strlen((char *)key)); +} + +int cmpTestCallback(const void *k1, const void *k2) { + return strcmp(k1, k2); } void freeTestCallback(void *val) { zfree(val); } -dictType KvstoreDictTestType = {hashTestCallback, - NULL, - NULL, - freeTestCallback, - NULL, - NULL, - kvstoreDictRehashingStarted, - kvstoreDictRehashingCompleted, - kvstoreDictMetadataSize}; +hashtableType KvstoreHashtableTestType = { + .hashFunction = hashTestCallback, + .keyCompare = cmpTestCallback, + .entryDestructor = freeTestCallback, + .rehashingStarted = kvstoreHashtableRehashingStarted, + .rehashingCompleted = kvstoreHashtableRehashingCompleted, + .trackMemUsage = kvstoreHashtableTrackMemUsage, + .getMetadataSize = kvstoreHashtableMetadataSize, +}; char *stringFromInt(int value) { char buf[32]; @@ -37,21 +41,18 @@ int test_kvstoreAdd16Keys(int argc, char **argv, int flags) { UNUSED(flags); int i; - dictEntry *de; int didx = 0; - kvstore *kvs1 = kvstoreCreate(&KvstoreDictTestType, 0, KVSTORE_ALLOCATE_DICTS_ON_DEMAND); - kvstore *kvs2 = kvstoreCreate(&KvstoreDictTestType, 0, KVSTORE_ALLOCATE_DICTS_ON_DEMAND | KVSTORE_FREE_EMPTY_DICTS); + kvstore *kvs1 = kvstoreCreate(&KvstoreHashtableTestType, 0, KVSTORE_ALLOCATE_HASHTABLES_ON_DEMAND); + kvstore *kvs2 = kvstoreCreate(&KvstoreHashtableTestType, 0, KVSTORE_ALLOCATE_HASHTABLES_ON_DEMAND | KVSTORE_FREE_EMPTY_HASHTABLES); for (i = 0; i < 16; i++) { - de = kvstoreDictAddRaw(kvs1, didx, stringFromInt(i), NULL); - TEST_ASSERT(de != NULL); - de = kvstoreDictAddRaw(kvs2, didx, stringFromInt(i), NULL); - TEST_ASSERT(de != NULL); + TEST_ASSERT(kvstoreHashtableAdd(kvs1, didx, stringFromInt(i))); + TEST_ASSERT(kvstoreHashtableAdd(kvs2, didx, stringFromInt(i))); } - TEST_ASSERT(kvstoreDictSize(kvs1, didx) == 16); + TEST_ASSERT(kvstoreHashtableSize(kvs1, didx) == 16); TEST_ASSERT(kvstoreSize(kvs1) == 16); - TEST_ASSERT(kvstoreDictSize(kvs2, didx) == 16); + TEST_ASSERT(kvstoreHashtableSize(kvs2, didx) == 16); TEST_ASSERT(kvstoreSize(kvs2) == 16); kvstoreRelease(kvs1); @@ -59,144 +60,132 @@ int test_kvstoreAdd16Keys(int argc, char **argv, int flags) { return 0; } -int test_kvstoreIteratorRemoveAllKeysNoDeleteEmptyDict(int argc, char **argv, int flags) { +int test_kvstoreIteratorRemoveAllKeysNoDeleteEmptyHashtable(int argc, char **argv, int flags) { UNUSED(argc); UNUSED(argv); UNUSED(flags); int i; void *key; - dictEntry *de; kvstoreIterator *kvs_it; int didx = 0; int curr_slot = 0; - kvstore *kvs1 = kvstoreCreate(&KvstoreDictTestType, 0, KVSTORE_ALLOCATE_DICTS_ON_DEMAND); + kvstore *kvs1 = kvstoreCreate(&KvstoreHashtableTestType, 0, KVSTORE_ALLOCATE_HASHTABLES_ON_DEMAND); for (i = 0; i < 16; i++) { - de = kvstoreDictAddRaw(kvs1, didx, stringFromInt(i), NULL); - TEST_ASSERT(de != NULL); + TEST_ASSERT(kvstoreHashtableAdd(kvs1, didx, stringFromInt(i))); } kvs_it = kvstoreIteratorInit(kvs1); - while ((de = kvstoreIteratorNext(kvs_it)) != NULL) { - curr_slot = kvstoreIteratorGetCurrentDictIndex(kvs_it); - key = dictGetKey(de); - TEST_ASSERT(kvstoreDictDelete(kvs1, curr_slot, key) == DICT_OK); + while (kvstoreIteratorNext(kvs_it, &key)) { + curr_slot = kvstoreIteratorGetCurrentHashtableIndex(kvs_it); + TEST_ASSERT(kvstoreHashtableDelete(kvs1, curr_slot, key)); } kvstoreIteratorRelease(kvs_it); - dict *d = kvstoreGetDict(kvs1, didx); - TEST_ASSERT(d != NULL); - TEST_ASSERT(kvstoreDictSize(kvs1, didx) == 0); + hashtable *ht = kvstoreGetHashtable(kvs1, didx); + TEST_ASSERT(ht != NULL); + TEST_ASSERT(kvstoreHashtableSize(kvs1, didx) == 0); TEST_ASSERT(kvstoreSize(kvs1) == 0); kvstoreRelease(kvs1); return 0; } -int test_kvstoreIteratorRemoveAllKeysDeleteEmptyDict(int argc, char **argv, int flags) { +int test_kvstoreIteratorRemoveAllKeysDeleteEmptyHashtable(int argc, char **argv, int flags) { UNUSED(argc); UNUSED(argv); UNUSED(flags); int i; void *key; - dictEntry *de; kvstoreIterator *kvs_it; int didx = 0; int curr_slot = 0; - kvstore *kvs2 = kvstoreCreate(&KvstoreDictTestType, 0, KVSTORE_ALLOCATE_DICTS_ON_DEMAND | KVSTORE_FREE_EMPTY_DICTS); + kvstore *kvs2 = kvstoreCreate(&KvstoreHashtableTestType, 0, KVSTORE_ALLOCATE_HASHTABLES_ON_DEMAND | KVSTORE_FREE_EMPTY_HASHTABLES); for (i = 0; i < 16; i++) { - de = kvstoreDictAddRaw(kvs2, didx, stringFromInt(i), NULL); - TEST_ASSERT(de != NULL); + TEST_ASSERT(kvstoreHashtableAdd(kvs2, didx, stringFromInt(i))); } kvs_it = kvstoreIteratorInit(kvs2); - while ((de = kvstoreIteratorNext(kvs_it)) != NULL) { - curr_slot = kvstoreIteratorGetCurrentDictIndex(kvs_it); - key = dictGetKey(de); - TEST_ASSERT(kvstoreDictDelete(kvs2, curr_slot, key) == DICT_OK); + while (kvstoreIteratorNext(kvs_it, &key)) { + curr_slot = kvstoreIteratorGetCurrentHashtableIndex(kvs_it); + TEST_ASSERT(kvstoreHashtableDelete(kvs2, curr_slot, key)); } kvstoreIteratorRelease(kvs_it); - /* Make sure the dict was removed from the rehashing list. */ + /* Make sure the hashtable was removed from the rehashing list. */ while (kvstoreIncrementallyRehash(kvs2, 1000)) { } - dict *d = kvstoreGetDict(kvs2, didx); - TEST_ASSERT(d == NULL); - TEST_ASSERT(kvstoreDictSize(kvs2, didx) == 0); + hashtable *ht = kvstoreGetHashtable(kvs2, didx); + TEST_ASSERT(ht == NULL); + TEST_ASSERT(kvstoreHashtableSize(kvs2, didx) == 0); TEST_ASSERT(kvstoreSize(kvs2) == 0); kvstoreRelease(kvs2); return 0; } -int test_kvstoreDictIteratorRemoveAllKeysNoDeleteEmptyDict(int argc, char **argv, int flags) { +int test_kvstoreHashtableIteratorRemoveAllKeysNoDeleteEmptyHashtable(int argc, char **argv, int flags) { UNUSED(argc); UNUSED(argv); UNUSED(flags); int i; void *key; - dictEntry *de; - kvstoreDictIterator *kvs_di; + kvstoreHashtableIterator *kvs_di; int didx = 0; - kvstore *kvs1 = kvstoreCreate(&KvstoreDictTestType, 0, KVSTORE_ALLOCATE_DICTS_ON_DEMAND); + kvstore *kvs1 = kvstoreCreate(&KvstoreHashtableTestType, 0, KVSTORE_ALLOCATE_HASHTABLES_ON_DEMAND); for (i = 0; i < 16; i++) { - de = kvstoreDictAddRaw(kvs1, didx, stringFromInt(i), NULL); - TEST_ASSERT(de != NULL); + TEST_ASSERT(kvstoreHashtableAdd(kvs1, didx, stringFromInt(i))); } - kvs_di = kvstoreGetDictSafeIterator(kvs1, didx); - while ((de = kvstoreDictIteratorNext(kvs_di)) != NULL) { - key = dictGetKey(de); - TEST_ASSERT(kvstoreDictDelete(kvs1, didx, key) == DICT_OK); + kvs_di = kvstoreGetHashtableSafeIterator(kvs1, didx); + while (kvstoreHashtableIteratorNext(kvs_di, &key)) { + TEST_ASSERT(kvstoreHashtableDelete(kvs1, didx, key)); } - kvstoreReleaseDictIterator(kvs_di); + kvstoreReleaseHashtableIterator(kvs_di); - dict *d = kvstoreGetDict(kvs1, didx); - TEST_ASSERT(d != NULL); - TEST_ASSERT(kvstoreDictSize(kvs1, didx) == 0); + hashtable *ht = kvstoreGetHashtable(kvs1, didx); + TEST_ASSERT(ht != NULL); + TEST_ASSERT(kvstoreHashtableSize(kvs1, didx) == 0); TEST_ASSERT(kvstoreSize(kvs1) == 0); kvstoreRelease(kvs1); return 0; } -int test_kvstoreDictIteratorRemoveAllKeysDeleteEmptyDict(int argc, char **argv, int flags) { +int test_kvstoreHashtableIteratorRemoveAllKeysDeleteEmptyHashtable(int argc, char **argv, int flags) { UNUSED(argc); UNUSED(argv); UNUSED(flags); int i; void *key; - dictEntry *de; - kvstoreDictIterator *kvs_di; + kvstoreHashtableIterator *kvs_di; int didx = 0; - kvstore *kvs2 = kvstoreCreate(&KvstoreDictTestType, 0, KVSTORE_ALLOCATE_DICTS_ON_DEMAND | KVSTORE_FREE_EMPTY_DICTS); + kvstore *kvs2 = kvstoreCreate(&KvstoreHashtableTestType, 0, KVSTORE_ALLOCATE_HASHTABLES_ON_DEMAND | KVSTORE_FREE_EMPTY_HASHTABLES); for (i = 0; i < 16; i++) { - de = kvstoreDictAddRaw(kvs2, didx, stringFromInt(i), NULL); - TEST_ASSERT(de != NULL); + TEST_ASSERT(kvstoreHashtableAdd(kvs2, didx, stringFromInt(i))); } - kvs_di = kvstoreGetDictSafeIterator(kvs2, didx); - while ((de = kvstoreDictIteratorNext(kvs_di)) != NULL) { - key = dictGetKey(de); - TEST_ASSERT(kvstoreDictDelete(kvs2, didx, key) == DICT_OK); + kvs_di = kvstoreGetHashtableSafeIterator(kvs2, didx); + while (kvstoreHashtableIteratorNext(kvs_di, &key)) { + TEST_ASSERT(kvstoreHashtableDelete(kvs2, didx, key)); } - kvstoreReleaseDictIterator(kvs_di); + kvstoreReleaseHashtableIterator(kvs_di); - dict *d = kvstoreGetDict(kvs2, didx); - TEST_ASSERT(d == NULL); - TEST_ASSERT(kvstoreDictSize(kvs2, didx) == 0); + hashtable *ht = kvstoreGetHashtable(kvs2, didx); + TEST_ASSERT(ht == NULL); + TEST_ASSERT(kvstoreHashtableSize(kvs2, didx) == 0); TEST_ASSERT(kvstoreSize(kvs2) == 0); kvstoreRelease(kvs2); diff --git a/src/unit/test_object.c b/src/unit/test_object.c new file mode 100644 index 0000000000..0d65b4691f --- /dev/null +++ b/src/unit/test_object.c @@ -0,0 +1,50 @@ +#include "../object.c" +#include "test_help.h" + +#include +#include +#include +#include +#include + + +int test_object_with_key(int argc, char **argv, int flags) { + UNUSED(argc); + UNUSED(argv); + UNUSED(flags); + sds key = sdsnew("foo"); + robj *val = createStringObject("bar", strlen("bar")); + TEST_ASSERT(val->encoding == OBJ_ENCODING_EMBSTR); + + /* Prevent objectSetKeyAndExpire from freeing the old val when reallocating it. */ + incrRefCount(val); + + /* Create valkey: val with key. */ + valkey *valkey = objectSetKeyAndExpire(val, key, -1); + TEST_ASSERT(valkey->encoding == OBJ_ENCODING_EMBSTR); + TEST_ASSERT(objectGetKey(valkey) != NULL); + + /* Check embedded key "foo" */ + TEST_ASSERT(sdslen(objectGetKey(valkey)) == 3); + TEST_ASSERT(sdslen(key) == 3); + TEST_ASSERT(sdscmp(objectGetKey(valkey), key) == 0); + TEST_ASSERT(strcmp(objectGetKey(valkey), "foo") == 0); + + /* Check embedded value "bar" (EMBSTR content) */ + TEST_ASSERT(sdscmp(valkey->ptr, val->ptr) == 0); + TEST_ASSERT(strcmp(valkey->ptr, "bar") == 0); + + /* Either they're two separate objects, or one object with refcount == 2. */ + if (valkey == val) { + TEST_ASSERT(valkey->refcount == 2); + } else { + TEST_ASSERT(valkey->refcount == 1); + TEST_ASSERT(val->refcount == 1); + } + + /* Free them. */ + sdsfree(key); + decrRefCount(val); + decrRefCount(valkey); + return 0; +} diff --git a/tests/integration/valkey-cli.tcl b/tests/integration/valkey-cli.tcl index 6344215a25..0c15af74f9 100644 --- a/tests/integration/valkey-cli.tcl +++ b/tests/integration/valkey-cli.tcl @@ -499,10 +499,10 @@ if {!$::tls} { ;# fake_redis_node doesn't support TLS assert_equal 1000 [llength [split [run_cli --scan]]] # pattern - assert_equal {key:2} [run_cli --scan --pattern "*:2"] + assert_equal {key:2} [split [run_cli --scan --pattern "*:2"]] # pattern matching with a quoted string - assert_equal {key:2} [run_cli --scan --quoted-pattern {"*:\x32"}] + assert_equal {key:2} [split [run_cli --scan --quoted-pattern {"*:\x32"}]] } proc test_valkey_cli_repl {} { diff --git a/tests/unit/expire.tcl b/tests/unit/expire.tcl index d85ce7ee68..f573943780 100644 --- a/tests/unit/expire.tcl +++ b/tests/unit/expire.tcl @@ -847,7 +847,7 @@ start_cluster 1 0 {tags {"expire external:skip cluster"}} { # hashslot(foo) is 12182 # fill data across different slots with expiration - for {set j 1} {$j <= 100} {incr j} { + for {set j 1} {$j <= 1000} {incr j} { r psetex "{foo}$j" 500 a } # hashslot(key) is 12539 @@ -858,7 +858,7 @@ start_cluster 1 0 {tags {"expire external:skip cluster"}} { r debug dict-resizing 0 # delete data to have lot's (99%) of empty buckets (slot 12182 should be skipped) - for {set j 1} {$j <= 99} {incr j} { + for {set j 1} {$j <= 999} {incr j} { r del "{foo}$j" } @@ -884,7 +884,9 @@ start_cluster 1 0 {tags {"expire external:skip cluster"}} { r debug dict-resizing 1 # put some data into slot 12182 and trigger the resize + # by deleting it to trigger shrink r psetex "{foo}0" 500 a + r del "{foo}0" # Verify all keys have expired wait_for_condition 400 100 { diff --git a/tests/unit/info.tcl b/tests/unit/info.tcl index 278a1d8e33..cf7f633a8c 100644 --- a/tests/unit/info.tcl +++ b/tests/unit/info.tcl @@ -515,23 +515,43 @@ start_server {tags {"info" "external:skip"}} { set info_mem [r info memory] set mem_stats [r memory stats] assert_equal [getInfoProperty $info_mem mem_overhead_db_hashtable_rehashing] {0} - assert_range [dict get $mem_stats overhead.db.hashtable.lut] 1 64 + # overhead.db.hashtable.lut = memory overhead of hashset including hashset struct and tables + set hashset_overhead [dict get $mem_stats overhead.db.hashtable.lut] + if {$hashset_overhead < 140} { + # 32-bit version (hashset struct + 1 bucket of 64 bytes) + set bits 32 + } else { + set bits 64 + } + assert_range [dict get $mem_stats overhead.db.hashtable.lut] 1 256 assert_equal [dict get $mem_stats overhead.db.hashtable.rehashing] {0} assert_equal [dict get $mem_stats db.dict.rehashing.count] {0} - # set 4 more keys to trigger rehashing + # set 7 more keys to trigger rehashing # get the info within a transaction to make sure the rehashing is not completed - r multi + r multi r set b c r set c d r set d e r set e f + r set f g + r set g h + r set h i + if {$bits == 32} { + # In 32-bit mode, we have 12 elements per bucket. Insert five more + # to trigger rehashing. + r set aa aa + r set bb bb + r set cc cc + r set dd dd + r set ee ee + } r info memory r memory stats set res [r exec] - set info_mem [lindex $res 4] - set mem_stats [lindex $res 5] + set info_mem [lindex $res end-1] + set mem_stats [lindex $res end] assert_range [getInfoProperty $info_mem mem_overhead_db_hashtable_rehashing] 1 64 - assert_range [dict get $mem_stats overhead.db.hashtable.lut] 1 192 + assert_range [dict get $mem_stats overhead.db.hashtable.lut] 1 300 assert_range [dict get $mem_stats overhead.db.hashtable.rehashing] 1 64 assert_equal [dict get $mem_stats db.dict.rehashing.count] {1} } diff --git a/tests/unit/maxmemory.tcl b/tests/unit/maxmemory.tcl index d4e62246f1..ef993cdd43 100644 --- a/tests/unit/maxmemory.tcl +++ b/tests/unit/maxmemory.tcl @@ -145,45 +145,6 @@ start_server {tags {"maxmemory" "external:skip"}} { } start_server {tags {"maxmemory external:skip"}} { - test "Without maxmemory small integers are shared" { - r config set maxmemory 0 - r set a 1 - assert_refcount_morethan a 1 - } - - test "With maxmemory and non-LRU policy integers are still shared" { - r config set maxmemory 1073741824 - r config set maxmemory-policy allkeys-random - r set a 1 - assert_refcount_morethan a 1 - } - - test "With maxmemory and LRU policy integers are not shared" { - r config set maxmemory 1073741824 - r config set maxmemory-policy allkeys-lru - r set a 1 - r config set maxmemory-policy volatile-lru - r set b 1 - assert_refcount 1 a - assert_refcount 1 b - r config set maxmemory 0 - } - - test "Shared integers are unshared with maxmemory and LRU policy" { - r set a 1 - r set b 1 - assert_refcount_morethan a 1 - assert_refcount_morethan b 1 - r config set maxmemory 1073741824 - r config set maxmemory-policy allkeys-lru - r get a - assert_refcount 1 a - r config set maxmemory-policy volatile-lru - r get b - assert_refcount 1 b - r config set maxmemory 0 - } - foreach policy { allkeys-random allkeys-lru allkeys-lfu volatile-lru volatile-lfu volatile-random volatile-ttl } { @@ -265,10 +226,10 @@ start_server {tags {"maxmemory external:skip"}} { # make sure to start with a blank instance r flushall # Get the current memory limit and calculate a new limit. - # We just add 100k to the current memory size so that it is + # We just add 400KiB to the current memory size so that it is # fast for us to reach that limit. set used [s used_memory] - set limit [expr {$used+100*1024}] + set limit [expr {$used+400*1024}] r config set maxmemory $limit r config set maxmemory-policy $policy # Now add keys until the limit is almost reached. diff --git a/tests/unit/other.tcl b/tests/unit/other.tcl index 6e6230fc19..9a34e848f2 100644 --- a/tests/unit/other.tcl +++ b/tests/unit/other.tcl @@ -391,12 +391,34 @@ start_server {tags {"other"}} { } } +proc table_size {dbnum} { + regexp {Hash table 0 stats \(main hash table\):\n *table size: (\d+)} [r DEBUG HTSTATS $dbnum] -> table_size + return $table_size +} + start_server {tags {"other external:skip"}} { test {Don't rehash if server has child process} { r config set save "" r config set rdb-key-save-delay 1000000 - populate 4095 "" 1 + # This fill factor is defined internally in hashset.c and duplicated + # here. If we change the fill factor, this test case will fail and will + # need to be updated accordingly. + # + # TODO: Find a better way to detect the limit where resize happens. + set MAX_FILL_PERCENT_SOFT 100 + + # Populate some, then check table size and populate more up to the soft + # maximum fill factor. + set initial 2000 + populate $initial a 1 + set table_size [table_size 9] + set more [expr {$table_size * $MAX_FILL_PERCENT_SOFT / 100 - $initial}] + populate $more b 1 + assert_equal $table_size [table_size 9] + assert_no_match "*Hash table 1 stats*" [r DEBUG HTSTATS 9] + + # Now we are close to resizing. r bgsave wait_for_condition 10 100 { [s rdb_bgsave_in_progress] eq 1 @@ -406,14 +428,15 @@ start_server {tags {"other external:skip"}} { r mset k1 v1 k2 v2 # Hash table should not rehash - assert_no_match "*table size: 8192*" [r debug HTSTATS 9] + assert_equal $table_size [table_size 9] + assert_no_match "*Hash table 1 stats*" [r DEBUG HTSTATS 9] exec kill -9 [get_child_pid 0] waitForBgsave r # Hash table should rehash since there is no child process, - # size is power of two and over 4096, so it is 8192 + # so the resize limit is restored. wait_for_condition 50 100 { - [string match "*table size: 8192*" [r debug HTSTATS 9]] + [table_size 9] > $table_size } else { fail "hash table did not rehash after child process killed" } @@ -472,7 +495,7 @@ start_cluster 1 0 {tags {"other external:skip cluster slow"}} { for {set j 1} {$j <= 128} {incr j} { r set "{foo}$j" a } - assert_match "*table size: 128*" [r debug HTSTATS 0] + set table_size [table_size 0] # disable resizing, the reason for not using slow bgsave is because # it will hit the dict_force_resize_ratio. @@ -482,14 +505,14 @@ start_cluster 1 0 {tags {"other external:skip cluster slow"}} { for {set j 1} {$j <= 123} {incr j} { r del "{foo}$j" } - assert_match "*table size: 128*" [r debug HTSTATS 0] + assert_equal $table_size [table_size 0] # enable resizing r debug dict-resizing 1 # waiting for serverCron to resize the tables wait_for_condition 1000 10 { - [string match {*table size: 8*} [r debug HTSTATS 0]] + [table_size 0] < $table_size } else { puts [r debug HTSTATS 0] fail "hash tables weren't resize." @@ -503,6 +526,7 @@ start_cluster 1 0 {tags {"other external:skip cluster slow"}} { for {set j 1} {$j <= 128} {incr j} { r set "{alice}$j" a } + set table_size [table_size 0] # disable resizing, the reason for not using slow bgsave is because # it will hit the dict_force_resize_ratio. @@ -517,7 +541,7 @@ start_cluster 1 0 {tags {"other external:skip cluster slow"}} { # waiting for serverCron to resize the tables wait_for_condition 1000 10 { - [string match {*table size: 16*} [r debug HTSTATS 0]] + [table_size 0] < $table_size } else { puts [r debug HTSTATS 0] fail "hash tables weren't resize." diff --git a/tests/unit/type/incr.tcl b/tests/unit/type/incr.tcl index 4bc130bcb1..fd0a8d02d8 100644 --- a/tests/unit/type/incr.tcl +++ b/tests/unit/type/incr.tcl @@ -75,17 +75,6 @@ start_server {tags {"incr"}} { assert_equal {-1} [r decrby key_not_exist 1] } - test {INCR uses shared objects in the 0-9999 range} { - r set foo -1 - r incr foo - assert_refcount_morethan foo 1 - r set foo 9998 - r incr foo - assert_refcount_morethan foo 1 - r incr foo - assert_refcount 1 foo - } - test {INCR can modify objects in-place} { r set foo 20000 r incr foo