mirror of https://github.com/redis/redis.git
				
				
				
			Fix ziplist and listpack overflows and truncations (CVE-2021-32627, CVE-2021-32628) (#9589)
- fix possible heap corruption in ziplist and listpack resulting by trying to allocate more than the maximum size of 4GB. - prevent ziplist (hash and zset) from reaching size of above 1GB, will be converted to HT encoding, that's not a useful size. - prevent listpack (stream) from reaching size of above 1GB. - XADD will start a new listpack if the new record may cause the previous listpack to grow over 1GB. - XADD will respond with an error if a single stream record is over 1GB - List type (ziplist in quicklist) was truncating strings that were over 4GB, now it'll respond with an error. Co-authored-by: sundb <sundbcn@gmail.com>
This commit is contained in:
		
							parent
							
								
									fba15850e5
								
							
						
					
					
						commit
						c5e6a6204c
					
				|  | @ -797,7 +797,7 @@ void georadiusGeneric(client *c, int srcKeyIndex, int flags) { | |||
|         robj *zobj; | ||||
|         zset *zs; | ||||
|         int i; | ||||
|         size_t maxelelen = 0; | ||||
|         size_t maxelelen = 0, totelelen = 0; | ||||
| 
 | ||||
|         if (returned_items) { | ||||
|             zobj = createZsetObject(); | ||||
|  | @ -812,13 +812,14 @@ void georadiusGeneric(client *c, int srcKeyIndex, int flags) { | |||
|             size_t elelen = sdslen(gp->member); | ||||
| 
 | ||||
|             if (maxelelen < elelen) maxelelen = elelen; | ||||
|             totelelen += elelen; | ||||
|             znode = zslInsert(zs->zsl,score,gp->member); | ||||
|             serverAssert(dictAdd(zs->dict,gp->member,&znode->score) == DICT_OK); | ||||
|             gp->member = NULL; | ||||
|         } | ||||
| 
 | ||||
|         if (returned_items) { | ||||
|             zsetConvertToListpackIfNeeded(zobj,maxelelen); | ||||
|             zsetConvertToListpackIfNeeded(zobj,maxelelen,totelelen); | ||||
|             setKey(c,c->db,storekey,zobj); | ||||
|             decrRefCount(zobj); | ||||
|             notifyKeyspaceEvent(NOTIFY_ZSET,flags & GEOSEARCH ? "geosearchstore" : "georadiusstore",storekey, | ||||
|  |  | |||
|  | @ -140,6 +140,16 @@ | |||
| 
 | ||||
| static inline void lpAssertValidEntry(unsigned char* lp, size_t lpbytes, unsigned char *p); | ||||
| 
 | ||||
| /* Don't let listpacks grow over 1GB in any case, don't wanna risk overflow in
 | ||||
|  * Total Bytes header field */ | ||||
| #define LISTPACK_MAX_SAFETY_SIZE (1<<30) | ||||
| int lpSafeToAdd(unsigned char* lp, size_t add) { | ||||
|     size_t len = lp? lpGetTotalBytes(lp): 0; | ||||
|     if (len + add > LISTPACK_MAX_SAFETY_SIZE) | ||||
|         return 0; | ||||
|     return 1; | ||||
| } | ||||
| 
 | ||||
| /* Convert a string into a signed 64 bit integer.
 | ||||
|  * The function returns 1 if the string could be parsed into a (non-overflowing) | ||||
|  * signed 64 bit int, 0 otherwise. The 'value' will be set to the parsed value | ||||
|  | @ -324,7 +334,7 @@ static inline int lpEncodeGetType(unsigned char *ele, uint32_t size, unsigned ch | |||
|     } else { | ||||
|         if (size < 64) *enclen = 1+size; | ||||
|         else if (size < 4096) *enclen = 2+size; | ||||
|         else *enclen = 5+size; | ||||
|         else *enclen = 5+(uint64_t)size; | ||||
|         return LP_ENCODING_STRING; | ||||
|     } | ||||
| } | ||||
|  |  | |||
|  | @ -87,6 +87,7 @@ unsigned int lpCompare(unsigned char *p, unsigned char *s, uint32_t slen); | |||
| void lpRandomPair(unsigned char *lp, unsigned long total_count, listpackEntry *key, listpackEntry *val); | ||||
| void lpRandomPairs(unsigned char *lp, unsigned int count, listpackEntry *keys, listpackEntry *vals); | ||||
| unsigned int lpRandomPairsUnique(unsigned char *lp, unsigned int count, listpackEntry *keys, listpackEntry *vals); | ||||
| int lpSafeToAdd(unsigned char* lp, size_t add); | ||||
| 
 | ||||
| #ifdef REDIS_TEST | ||||
| int listpackTest(int argc, char *argv[], int accurate); | ||||
|  |  | |||
|  | @ -4064,6 +4064,7 @@ int RM_HashGet(RedisModuleKey *key, int flags, ...) { | |||
|  * - EDOM if the given ID was 0-0 or not greater than all other IDs in the | ||||
|  *   stream (only if the AUTOID flag is unset) | ||||
|  * - EFBIG if the stream has reached the last possible ID | ||||
|  * - ERANGE if the elements are too large to be stored. | ||||
|  */ | ||||
| int RM_StreamAdd(RedisModuleKey *key, int flags, RedisModuleStreamID *id, RedisModuleString **argv, long numfields) { | ||||
|     /* Validate args */ | ||||
|  | @ -4107,8 +4108,9 @@ int RM_StreamAdd(RedisModuleKey *key, int flags, RedisModuleStreamID *id, RedisM | |||
|         use_id_ptr = &use_id; | ||||
|     } | ||||
|     if (streamAppendItem(s, argv, numfields, &added_id, use_id_ptr) == C_ERR) { | ||||
|         /* ID not greater than all existing IDs in the stream */ | ||||
|         errno = EDOM; | ||||
|         /* Either the ID not greater than all existing IDs in the stream, or
 | ||||
|          * the elements are too large to be stored. either way, errno is already | ||||
|          * set by streamAppendItem. */ | ||||
|         return REDISMODULE_ERR; | ||||
|     } | ||||
|     /* Postponed signalKeyAsReady(). Done implicitly by moduleCreateEmptyKey()
 | ||||
|  |  | |||
|  | @ -45,11 +45,16 @@ | |||
| #define REDIS_STATIC static | ||||
| #endif | ||||
| 
 | ||||
| /* Optimization levels for size-based filling */ | ||||
| /* Optimization levels for size-based filling.
 | ||||
|  * Note that the largest possible limit is 16k, so even if each record takes | ||||
|  * just one byte, it still won't overflow the 16 bit count field. */ | ||||
| static const size_t optimization_level[] = {4096, 8192, 16384, 32768, 65536}; | ||||
| 
 | ||||
| /* Maximum size in bytes of any multi-element ziplist.
 | ||||
|  * Larger values will live in their own isolated ziplists. */ | ||||
|  * Larger values will live in their own isolated ziplists. | ||||
|  * This is used only if we're limited by record count. when we're limited by | ||||
|  * size, the maximum limit is bigger, but still safe. | ||||
|  * 8k is a recommended / default size limit */ | ||||
| #define SIZE_SAFETY_LIMIT 8192 | ||||
| 
 | ||||
| /* Minimum ziplist size in bytes for attempting compression. */ | ||||
|  | @ -444,6 +449,8 @@ REDIS_STATIC int _quicklistNodeAllowInsert(const quicklistNode *node, | |||
|     unsigned int new_sz = node->sz + sz + ziplist_overhead; | ||||
|     if (likely(_quicklistNodeSizeMeetsOptimizationRequirement(new_sz, fill))) | ||||
|         return 1; | ||||
|     /* when we return 1 above we know that the limit is a size limit (which is
 | ||||
|      * safe, see comments next to optimization_level and SIZE_SAFETY_LIMIT) */ | ||||
|     else if (!sizeMeetsSafetyLimit(new_sz)) | ||||
|         return 0; | ||||
|     else if ((int)node->count < fill) | ||||
|  | @ -463,6 +470,8 @@ REDIS_STATIC int _quicklistNodeAllowMerge(const quicklistNode *a, | |||
|     unsigned int merge_sz = a->sz + b->sz - 11; | ||||
|     if (likely(_quicklistNodeSizeMeetsOptimizationRequirement(merge_sz, fill))) | ||||
|         return 1; | ||||
|     /* when we return 1 above we know that the limit is a size limit (which is
 | ||||
|      * safe, see comments next to optimization_level and SIZE_SAFETY_LIMIT) */ | ||||
|     else if (!sizeMeetsSafetyLimit(merge_sz)) | ||||
|         return 0; | ||||
|     else if ((int)(a->count + b->count) <= fill) | ||||
|  | @ -482,6 +491,7 @@ REDIS_STATIC int _quicklistNodeAllowMerge(const quicklistNode *a, | |||
|  * Returns 1 if new head created. */ | ||||
| int quicklistPushHead(quicklist *quicklist, void *value, size_t sz) { | ||||
|     quicklistNode *orig_head = quicklist->head; | ||||
|     assert(sz < UINT32_MAX); /* TODO: add support for quicklist nodes that are sds encoded (not zipped) */ | ||||
|     if (likely( | ||||
|             _quicklistNodeAllowInsert(quicklist->head, quicklist->fill, sz))) { | ||||
|         quicklist->head->zl = | ||||
|  | @ -505,6 +515,7 @@ int quicklistPushHead(quicklist *quicklist, void *value, size_t sz) { | |||
|  * Returns 1 if new tail created. */ | ||||
| int quicklistPushTail(quicklist *quicklist, void *value, size_t sz) { | ||||
|     quicklistNode *orig_tail = quicklist->tail; | ||||
|     assert(sz < UINT32_MAX); /* TODO: add support for quicklist nodes that are sds encoded (not zipped) */ | ||||
|     if (likely( | ||||
|             _quicklistNodeAllowInsert(quicklist->tail, quicklist->fill, sz))) { | ||||
|         quicklist->tail->zl = | ||||
|  | @ -853,6 +864,7 @@ REDIS_STATIC void _quicklistInsert(quicklist *quicklist, quicklistEntry *entry, | |||
|     int fill = quicklist->fill; | ||||
|     quicklistNode *node = entry->node; | ||||
|     quicklistNode *new_node = NULL; | ||||
|     assert(sz < UINT32_MAX); /* TODO: add support for quicklist nodes that are sds encoded (not zipped) */ | ||||
| 
 | ||||
|     if (!node) { | ||||
|         /* we have no reference node, so let's create only node in the list */ | ||||
|  |  | |||
							
								
								
									
										49
									
								
								src/rdb.c
								
								
								
								
							
							
						
						
									
										49
									
								
								src/rdb.c
								
								
								
								
							|  | @ -1753,7 +1753,7 @@ robj *rdbLoadObject(int rdbtype, rio *rdb, sds key, int dbid, int *error) { | |||
|     } else if (rdbtype == RDB_TYPE_ZSET_2 || rdbtype == RDB_TYPE_ZSET) { | ||||
|         /* Read sorted set value. */ | ||||
|         uint64_t zsetlen; | ||||
|         size_t maxelelen = 0; | ||||
|         size_t maxelelen = 0, totelelen = 0; | ||||
|         zset *zs; | ||||
| 
 | ||||
|         if ((zsetlen = rdbLoadLen(rdb,NULL)) == RDB_LENERR) return NULL; | ||||
|  | @ -1795,6 +1795,7 @@ robj *rdbLoadObject(int rdbtype, rio *rdb, sds key, int dbid, int *error) { | |||
| 
 | ||||
|             /* Don't care about integer-encoded strings. */ | ||||
|             if (sdslen(sdsele) > maxelelen) maxelelen = sdslen(sdsele); | ||||
|             totelelen += sdslen(sdsele); | ||||
| 
 | ||||
|             znode = zslInsert(zs->zsl,score,sdsele); | ||||
|             if (dictAdd(zs->dict,sdsele,&znode->score) != DICT_OK) { | ||||
|  | @ -1807,8 +1808,11 @@ robj *rdbLoadObject(int rdbtype, rio *rdb, sds key, int dbid, int *error) { | |||
| 
 | ||||
|         /* Convert *after* loading, since sorted sets are not stored ordered. */ | ||||
|         if (zsetLength(o) <= server.zset_max_listpack_entries && | ||||
|             maxelelen <= server.zset_max_listpack_value) | ||||
|                 zsetConvert(o,OBJ_ENCODING_LISTPACK); | ||||
|             maxelelen <= server.zset_max_listpack_value && | ||||
|             lpSafeToAdd(NULL, totelelen)) | ||||
|         { | ||||
|             zsetConvert(o,OBJ_ENCODING_LISTPACK); | ||||
|         } | ||||
|     } else if (rdbtype == RDB_TYPE_HASH) { | ||||
|         uint64_t len; | ||||
|         int ret; | ||||
|  | @ -1862,19 +1866,28 @@ robj *rdbLoadObject(int rdbtype, rio *rdb, sds key, int dbid, int *error) { | |||
|                 } | ||||
|             } | ||||
| 
 | ||||
|             /* Convert to hash table if size threshold is exceeded */ | ||||
|             if (sdslen(field) > server.hash_max_listpack_value || | ||||
|                 sdslen(value) > server.hash_max_listpack_value || | ||||
|                 !lpSafeToAdd(o->ptr, sdslen(field)+sdslen(value))) | ||||
|             { | ||||
|                 hashTypeConvert(o, OBJ_ENCODING_HT); | ||||
|                 ret = dictAdd((dict*)o->ptr, field, value); | ||||
|                 if (ret == DICT_ERR) { | ||||
|                     rdbReportCorruptRDB("Duplicate hash fields detected"); | ||||
|                     if (dupSearchDict) dictRelease(dupSearchDict); | ||||
|                     sdsfree(value); | ||||
|                     sdsfree(field); | ||||
|                     decrRefCount(o); | ||||
|                     return NULL; | ||||
|                 } | ||||
|                 break; | ||||
|             } | ||||
| 
 | ||||
|             /* Add pair to listpack */ | ||||
|             o->ptr = lpAppend(o->ptr, (unsigned char*)field, sdslen(field)); | ||||
|             o->ptr = lpAppend(o->ptr, (unsigned char*)value, sdslen(value)); | ||||
| 
 | ||||
|             /* Convert to hash table if size threshold is exceeded */ | ||||
|             if (sdslen(field) > server.hash_max_listpack_value || | ||||
|                 sdslen(value) > server.hash_max_listpack_value) | ||||
|             { | ||||
|                 sdsfree(field); | ||||
|                 sdsfree(value); | ||||
|                 hashTypeConvert(o, OBJ_ENCODING_HT); | ||||
|                 break; | ||||
|             } | ||||
|             sdsfree(field); | ||||
|             sdsfree(value); | ||||
|         } | ||||
|  | @ -1993,7 +2006,7 @@ robj *rdbLoadObject(int rdbtype, rio *rdb, sds key, int dbid, int *error) { | |||
|                 /* Convert to ziplist encoded hash. This must be deprecated
 | ||||
|                  * when loading dumps created by Redis 2.4 gets deprecated. */ | ||||
|                 { | ||||
|                     unsigned char *zl = lpNew(0); | ||||
|                     unsigned char *lp = lpNew(0); | ||||
|                     unsigned char *zi = zipmapRewind(o->ptr); | ||||
|                     unsigned char *fstr, *vstr; | ||||
|                     unsigned int flen, vlen; | ||||
|  | @ -2003,12 +2016,11 @@ robj *rdbLoadObject(int rdbtype, rio *rdb, sds key, int dbid, int *error) { | |||
|                     while ((zi = zipmapNext(zi, &fstr, &flen, &vstr, &vlen)) != NULL) { | ||||
|                         if (flen > maxlen) maxlen = flen; | ||||
|                         if (vlen > maxlen) maxlen = vlen; | ||||
|                         zl = lpAppend(zl, fstr, flen); | ||||
|                         zl = lpAppend(zl, vstr, vlen); | ||||
| 
 | ||||
|                         /* search for duplicate records */ | ||||
|                         sds field = sdstrynewlen(fstr, flen); | ||||
|                         if (!field || dictAdd(dupSearchDict, field, NULL) != DICT_OK) { | ||||
|                         if (!field || dictAdd(dupSearchDict, field, NULL) != DICT_OK || | ||||
|                             !lpSafeToAdd(lp, (size_t)flen + vlen)) { | ||||
|                             rdbReportCorruptRDB("Hash zipmap with dup elements, or big length (%u)", flen); | ||||
|                             dictRelease(dupSearchDict); | ||||
|                             sdsfree(field); | ||||
|  | @ -2017,11 +2029,14 @@ robj *rdbLoadObject(int rdbtype, rio *rdb, sds key, int dbid, int *error) { | |||
|                             decrRefCount(o); | ||||
|                             return NULL; | ||||
|                         } | ||||
| 
 | ||||
|                         lp = lpAppend(lp, fstr, flen); | ||||
|                         lp = lpAppend(lp, vstr, vlen); | ||||
|                     } | ||||
| 
 | ||||
|                     dictRelease(dupSearchDict); | ||||
|                     zfree(o->ptr); | ||||
|                     o->ptr = zl; | ||||
|                     o->ptr = lp; | ||||
|                     o->type = OBJ_HASH; | ||||
|                     o->encoding = OBJ_ENCODING_LISTPACK; | ||||
| 
 | ||||
|  |  | |||
|  | @ -2384,7 +2384,7 @@ unsigned char *zzlFirstInRange(unsigned char *zl, zrangespec *range); | |||
| unsigned char *zzlLastInRange(unsigned char *zl, zrangespec *range); | ||||
| unsigned long zsetLength(const robj *zobj); | ||||
| void zsetConvert(robj *zobj, int encoding); | ||||
| void zsetConvertToListpackIfNeeded(robj *zobj, size_t maxelelen); | ||||
| void zsetConvertToListpackIfNeeded(robj *zobj, size_t maxelelen, size_t totelelen); | ||||
| int zsetScore(robj *zobj, sds member, double *score); | ||||
| unsigned long zslGetRank(zskiplist *zsl, double score, sds o); | ||||
| int zsetAdd(robj *zobj, double score, sds ele, int in_flags, int *out_flags, double *newscore); | ||||
|  |  | |||
							
								
								
									
										13
									
								
								src/t_hash.c
								
								
								
								
							
							
						
						
									
										13
									
								
								src/t_hash.c
								
								
								
								
							|  | @ -39,17 +39,22 @@ | |||
|  * as their string length can be queried in constant time. */ | ||||
| void hashTypeTryConversion(robj *o, robj **argv, int start, int end) { | ||||
|     int i; | ||||
|     size_t sum = 0; | ||||
| 
 | ||||
|     if (o->encoding != OBJ_ENCODING_LISTPACK) return; | ||||
| 
 | ||||
|     for (i = start; i <= end; i++) { | ||||
|         if (sdsEncodedObject(argv[i]) && | ||||
|             sdslen(argv[i]->ptr) > server.hash_max_listpack_value) | ||||
|         { | ||||
|         if (!sdsEncodedObject(argv[i])) | ||||
|             continue; | ||||
|         size_t len = sdslen(argv[i]->ptr); | ||||
|         if (len > server.hash_max_listpack_value) { | ||||
|             hashTypeConvert(o, OBJ_ENCODING_HT); | ||||
|             break; | ||||
|             return; | ||||
|         } | ||||
|         sum += len; | ||||
|     } | ||||
|     if (!lpSafeToAdd(o->ptr, sum)) | ||||
|         hashTypeConvert(o, OBJ_ENCODING_HT); | ||||
| } | ||||
| 
 | ||||
| /* Get the value from a listpack encoded hash, identified by field.
 | ||||
|  |  | |||
							
								
								
									
										29
									
								
								src/t_list.c
								
								
								
								
							
							
						
						
									
										29
									
								
								src/t_list.c
								
								
								
								
							|  | @ -29,6 +29,8 @@ | |||
| 
 | ||||
| #include "server.h" | ||||
| 
 | ||||
| #define LIST_MAX_ITEM_SIZE ((1ull<<32)-1024) | ||||
| 
 | ||||
| /*-----------------------------------------------------------------------------
 | ||||
|  * List API | ||||
|  *----------------------------------------------------------------------------*/ | ||||
|  | @ -254,6 +256,13 @@ int listTypeDelRange(robj *subject, long start, long count) { | |||
| void pushGenericCommand(client *c, int where, int xx) { | ||||
|     int j; | ||||
| 
 | ||||
|     for (j = 2; j < c->argc; j++) { | ||||
|         if (sdslen(c->argv[j]->ptr) > LIST_MAX_ITEM_SIZE) { | ||||
|             addReplyError(c, "Element too large"); | ||||
|             return; | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|     robj *lobj = lookupKeyWrite(c->db, c->argv[1]); | ||||
|     if (checkType(c,lobj,OBJ_LIST)) return; | ||||
|     if (!lobj) { | ||||
|  | @ -317,6 +326,11 @@ void linsertCommand(client *c) { | |||
|         return; | ||||
|     } | ||||
| 
 | ||||
|     if (sdslen(c->argv[4]->ptr) > LIST_MAX_ITEM_SIZE) { | ||||
|         addReplyError(c, "Element too large"); | ||||
|         return; | ||||
|     } | ||||
| 
 | ||||
|     if ((subject = lookupKeyWriteOrReply(c,c->argv[1],shared.czero)) == NULL || | ||||
|         checkType(c,subject,OBJ_LIST)) return; | ||||
| 
 | ||||
|  | @ -384,6 +398,11 @@ void lsetCommand(client *c) { | |||
|     long index; | ||||
|     robj *value = c->argv[3]; | ||||
| 
 | ||||
|     if (sdslen(value->ptr) > LIST_MAX_ITEM_SIZE) { | ||||
|         addReplyError(c, "Element too large"); | ||||
|         return; | ||||
|     } | ||||
| 
 | ||||
|     if ((getLongFromObjectOrReply(c, c->argv[2], &index, NULL) != C_OK)) | ||||
|         return; | ||||
| 
 | ||||
|  | @ -683,6 +702,11 @@ void lposCommand(client *c) { | |||
|     int direction = LIST_TAIL; | ||||
|     long rank = 1, count = -1, maxlen = 0; /* Count -1: option not given. */ | ||||
| 
 | ||||
|     if (sdslen(ele->ptr) > LIST_MAX_ITEM_SIZE) { | ||||
|         addReplyError(c, "Element too large"); | ||||
|         return; | ||||
|     } | ||||
| 
 | ||||
|     /* Parse the optional arguments. */ | ||||
|     for (int j = 3; j < c->argc; j++) { | ||||
|         char *opt = c->argv[j]->ptr; | ||||
|  | @ -778,6 +802,11 @@ void lremCommand(client *c) { | |||
|     long toremove; | ||||
|     long removed = 0; | ||||
| 
 | ||||
|     if (sdslen(obj->ptr) > LIST_MAX_ITEM_SIZE) { | ||||
|         addReplyError(c, "Element too large"); | ||||
|         return; | ||||
|     } | ||||
| 
 | ||||
|     if ((getLongFromObjectOrReply(c, c->argv[2], &toremove, NULL) != C_OK)) | ||||
|         return; | ||||
| 
 | ||||
|  |  | |||
|  | @ -47,6 +47,12 @@ | |||
|  * setting stream_node_max_bytes to a huge number. */ | ||||
| #define STREAM_LISTPACK_MAX_PRE_ALLOCATE 4096 | ||||
| 
 | ||||
| /* Don't let listpacks grow too big, even if the user config allows it.
 | ||||
|  * doing so can lead to an overflow (trying to store more than 32bit length | ||||
|  * into the listpack header), or actually an assertion since lpInsert | ||||
|  * will return NULL. */ | ||||
| #define STREAM_LISTPACK_MAX_SIZE (1<<30) | ||||
| 
 | ||||
| void streamFreeCG(streamCG *cg); | ||||
| void streamFreeNACK(streamNACK *na); | ||||
| size_t streamReplyWithRangeFromConsumerPEL(client *c, stream *s, streamID *start, streamID *end, size_t count, streamConsumer *consumer); | ||||
|  | @ -415,8 +421,11 @@ void streamGetEdgeID(stream *s, int first, streamID *edge_id) | |||
|  * | ||||
|  * The function returns C_OK if the item was added, this is always true | ||||
|  * if the ID was generated by the function. However the function may return | ||||
|  * C_ERR if an ID was given via 'use_id', but adding it failed since the | ||||
|  * current top ID is greater or equal. */ | ||||
|  * C_ERR in several cases: | ||||
|  * 1. If an ID was given via 'use_id', but adding it failed since the | ||||
|  *    current top ID is greater or equal. errno will be set to EDOM. | ||||
|  * 2. If a size of a single element or the sum of the elements is too big to | ||||
|  *    be stored into the stream. errno will be set to ERANGE. */ | ||||
| int streamAppendItem(stream *s, robj **argv, int64_t numfields, streamID *added_id, streamID *use_id) { | ||||
| 
 | ||||
|     /* Generate the new entry ID. */ | ||||
|  | @ -430,7 +439,23 @@ int streamAppendItem(stream *s, robj **argv, int64_t numfields, streamID *added_ | |||
|      * or return an error. Automatically generated IDs might | ||||
|      * overflow (and wrap-around) when incrementing the sequence | ||||
|        part. */ | ||||
|     if (streamCompareID(&id,&s->last_id) <= 0) return C_ERR; | ||||
|     if (streamCompareID(&id,&s->last_id) <= 0) { | ||||
|         errno = EDOM; | ||||
|         return C_ERR; | ||||
|     } | ||||
| 
 | ||||
|     /* Avoid overflow when trying to add an element to the stream (listpack
 | ||||
|      * can only host up to 32bit length sttrings, and also a total listpack size | ||||
|      * can't be bigger than 32bit length. */ | ||||
|     size_t totelelen = 0; | ||||
|     for (int64_t i = 0; i < numfields*2; i++) { | ||||
|         sds ele = argv[i]->ptr; | ||||
|         totelelen += sdslen(ele); | ||||
|     } | ||||
|     if (totelelen > STREAM_LISTPACK_MAX_SIZE) { | ||||
|         errno = ERANGE; | ||||
|         return C_ERR; | ||||
|     } | ||||
| 
 | ||||
|     /* Add the new entry. */ | ||||
|     raxIterator ri; | ||||
|  | @ -489,9 +514,10 @@ int streamAppendItem(stream *s, robj **argv, int64_t numfields, streamID *added_ | |||
|      * if we need to switch to the next one. 'lp' will be set to NULL if | ||||
|      * the current node is full. */ | ||||
|     if (lp != NULL) { | ||||
|         if (server.stream_node_max_bytes && | ||||
|             lp_bytes >= server.stream_node_max_bytes) | ||||
|         { | ||||
|         size_t node_max_bytes = server.stream_node_max_bytes; | ||||
|         if (node_max_bytes == 0 || node_max_bytes > STREAM_LISTPACK_MAX_SIZE) | ||||
|             node_max_bytes = STREAM_LISTPACK_MAX_SIZE; | ||||
|         if (lp_bytes + totelelen >= node_max_bytes) { | ||||
|             lp = NULL; | ||||
|         } else if (server.stream_node_max_entries) { | ||||
|             unsigned char *lp_ele = lpFirst(lp); | ||||
|  | @ -1782,11 +1808,13 @@ void xaddCommand(client *c) { | |||
|     /* Append using the low level function and return the ID. */ | ||||
|     streamID id; | ||||
|     if (streamAppendItem(s,c->argv+field_pos,(c->argc-field_pos)/2, | ||||
|         &id, parsed_args.id_given ? &parsed_args.id : NULL) | ||||
|         == C_ERR) | ||||
|         &id, parsed_args.id_given ? &parsed_args.id : NULL) == C_ERR) | ||||
|     { | ||||
|         addReplyError(c,"The ID specified in XADD is equal or smaller than the " | ||||
|                         "target stream top item"); | ||||
|         if (errno == EDOM) | ||||
|             addReplyError(c,"The ID specified in XADD is equal or smaller than " | ||||
|                             "the target stream top item"); | ||||
|         else | ||||
|             addReplyError(c,"Elements are too large to be stored"); | ||||
|         return; | ||||
|     } | ||||
|     addReplyStreamID(c,&id); | ||||
|  |  | |||
							
								
								
									
										62
									
								
								src/t_zset.c
								
								
								
								
							
							
						
						
									
										62
									
								
								src/t_zset.c
								
								
								
								
							|  | @ -1227,15 +1227,18 @@ void zsetConvert(robj *zobj, int encoding) { | |||
| } | ||||
| 
 | ||||
| /* Convert the sorted set object into a listpack if it is not already a listpack
 | ||||
|  * and if the number of elements and the maximum element size is within the | ||||
|  * expected ranges. */ | ||||
| void zsetConvertToListpackIfNeeded(robj *zobj, size_t maxelelen) { | ||||
|  * and if the number of elements and the maximum element size and total elements size | ||||
|  * are within the expected ranges. */ | ||||
| void zsetConvertToListpackIfNeeded(robj *zobj, size_t maxelelen, size_t totelelen) { | ||||
|     if (zobj->encoding == OBJ_ENCODING_LISTPACK) return; | ||||
|     zset *zset = zobj->ptr; | ||||
| 
 | ||||
|     if (zset->zsl->length <= server.zset_max_listpack_entries && | ||||
|         maxelelen <= server.zset_max_listpack_value) | ||||
|             zsetConvert(zobj,OBJ_ENCODING_LISTPACK); | ||||
|         maxelelen <= server.zset_max_listpack_value && | ||||
|         lpSafeToAdd(NULL, totelelen)) | ||||
|     { | ||||
|         zsetConvert(zobj,OBJ_ENCODING_LISTPACK); | ||||
|     } | ||||
| } | ||||
| 
 | ||||
| /* Return (by reference) the score of the specified member of the sorted set
 | ||||
|  | @ -1355,20 +1358,28 @@ int zsetAdd(robj *zobj, double score, sds ele, int in_flags, int *out_flags, dou | |||
|             } | ||||
|             return 1; | ||||
|         } else if (!xx) { | ||||
|             /* Optimize: check if the element is too large or the list
 | ||||
|             /* check if the element is too large or the list
 | ||||
|              * becomes too long *before* executing zzlInsert. */ | ||||
|             zobj->ptr = zzlInsert(zobj->ptr,ele,score); | ||||
|             if (zzlLength(zobj->ptr) > server.zset_max_listpack_entries || | ||||
|                 sdslen(ele) > server.zset_max_listpack_value) | ||||
|             if (zzlLength(zobj->ptr)+1 > server.zset_max_listpack_entries || | ||||
|                 sdslen(ele) > server.zset_max_listpack_value || | ||||
|                 !lpSafeToAdd(zobj->ptr, sdslen(ele))) | ||||
|             { | ||||
|                 zsetConvert(zobj,OBJ_ENCODING_SKIPLIST); | ||||
|             if (newscore) *newscore = score; | ||||
|             *out_flags |= ZADD_OUT_ADDED; | ||||
|             return 1; | ||||
|             } else { | ||||
|                 zobj->ptr = zzlInsert(zobj->ptr,ele,score); | ||||
|                 if (newscore) *newscore = score; | ||||
|                 *out_flags |= ZADD_OUT_ADDED; | ||||
|                 return 1; | ||||
|             } | ||||
|         } else { | ||||
|             *out_flags |= ZADD_OUT_NOP; | ||||
|             return 1; | ||||
|         } | ||||
|     } else if (zobj->encoding == OBJ_ENCODING_SKIPLIST) { | ||||
|     } | ||||
| 
 | ||||
|     /* Note that the above block handling ziplist would have either returned or
 | ||||
|      * converted the key to skiplist. */ | ||||
|     if (zobj->encoding == OBJ_ENCODING_SKIPLIST) { | ||||
|         zset *zs = zobj->ptr; | ||||
|         zskiplistNode *znode; | ||||
|         dictEntry *de; | ||||
|  | @ -2304,7 +2315,7 @@ inline static void zunionInterAggregate(double *target, double val, int aggregat | |||
|     } | ||||
| } | ||||
| 
 | ||||
| static int zsetDictGetMaxElementLength(dict *d) { | ||||
| static size_t zsetDictGetMaxElementLength(dict *d, size_t *totallen) { | ||||
|     dictIterator *di; | ||||
|     dictEntry *de; | ||||
|     size_t maxelelen = 0; | ||||
|  | @ -2314,6 +2325,8 @@ static int zsetDictGetMaxElementLength(dict *d) { | |||
|     while((de = dictNext(di)) != NULL) { | ||||
|         sds ele = dictGetKey(de); | ||||
|         if (sdslen(ele) > maxelelen) maxelelen = sdslen(ele); | ||||
|         if (totallen) | ||||
|             (*totallen) += sdslen(ele); | ||||
|     } | ||||
| 
 | ||||
|     dictReleaseIterator(di); | ||||
|  | @ -2321,7 +2334,7 @@ static int zsetDictGetMaxElementLength(dict *d) { | |||
|     return maxelelen; | ||||
| } | ||||
| 
 | ||||
| static void zdiffAlgorithm1(zsetopsrc *src, long setnum, zset *dstzset, size_t *maxelelen) { | ||||
| static void zdiffAlgorithm1(zsetopsrc *src, long setnum, zset *dstzset, size_t *maxelelen, size_t *totelelen) { | ||||
|     /* DIFF Algorithm 1:
 | ||||
|      * | ||||
|      * We perform the diff by iterating all the elements of the first set, | ||||
|  | @ -2369,13 +2382,14 @@ static void zdiffAlgorithm1(zsetopsrc *src, long setnum, zset *dstzset, size_t * | |||
|             znode = zslInsert(dstzset->zsl,zval.score,tmp); | ||||
|             dictAdd(dstzset->dict,tmp,&znode->score); | ||||
|             if (sdslen(tmp) > *maxelelen) *maxelelen = sdslen(tmp); | ||||
|             (*totelelen) += sdslen(tmp); | ||||
|         } | ||||
|     } | ||||
|     zuiClearIterator(&src[0]); | ||||
| } | ||||
| 
 | ||||
| 
 | ||||
| static void zdiffAlgorithm2(zsetopsrc *src, long setnum, zset *dstzset, size_t *maxelelen) { | ||||
| static void zdiffAlgorithm2(zsetopsrc *src, long setnum, zset *dstzset, size_t *maxelelen, size_t *totelelen) { | ||||
|     /* DIFF Algorithm 2:
 | ||||
|      * | ||||
|      * Add all the elements of the first set to the auxiliary set. | ||||
|  | @ -2429,7 +2443,7 @@ static void zdiffAlgorithm2(zsetopsrc *src, long setnum, zset *dstzset, size_t * | |||
| 
 | ||||
|     /* Using this algorithm, we can't calculate the max element as we go,
 | ||||
|      * we have to iterate through all elements to find the max one after. */ | ||||
|     *maxelelen = zsetDictGetMaxElementLength(dstzset->dict); | ||||
|     *maxelelen = zsetDictGetMaxElementLength(dstzset->dict, totelelen); | ||||
| } | ||||
| 
 | ||||
| static int zsetChooseDiffAlgorithm(zsetopsrc *src, long setnum) { | ||||
|  | @ -2466,14 +2480,14 @@ static int zsetChooseDiffAlgorithm(zsetopsrc *src, long setnum) { | |||
|     return (algo_one_work <= algo_two_work) ? 1 : 2; | ||||
| } | ||||
| 
 | ||||
| static void zdiff(zsetopsrc *src, long setnum, zset *dstzset, size_t *maxelelen) { | ||||
| static void zdiff(zsetopsrc *src, long setnum, zset *dstzset, size_t *maxelelen, size_t *totelelen) { | ||||
|     /* Skip everything if the smallest input is empty. */ | ||||
|     if (zuiLength(&src[0]) > 0) { | ||||
|         int diff_algo = zsetChooseDiffAlgorithm(src, setnum); | ||||
|         if (diff_algo == 1) { | ||||
|             zdiffAlgorithm1(src, setnum, dstzset, maxelelen); | ||||
|             zdiffAlgorithm1(src, setnum, dstzset, maxelelen, totelelen); | ||||
|         } else if (diff_algo == 2) { | ||||
|             zdiffAlgorithm2(src, setnum, dstzset, maxelelen); | ||||
|             zdiffAlgorithm2(src, setnum, dstzset, maxelelen, totelelen); | ||||
|         } else if (diff_algo != 0) { | ||||
|             serverPanic("Unknown algorithm"); | ||||
|         } | ||||
|  | @ -2510,7 +2524,7 @@ void zunionInterDiffGenericCommand(client *c, robj *dstkey, int numkeysIndex, in | |||
|     zsetopsrc *src; | ||||
|     zsetopval zval; | ||||
|     sds tmp; | ||||
|     size_t maxelelen = 0; | ||||
|     size_t maxelelen = 0, totelelen = 0; | ||||
|     robj *dstobj; | ||||
|     zset *dstzset; | ||||
|     zskiplistNode *znode; | ||||
|  | @ -2668,6 +2682,7 @@ void zunionInterDiffGenericCommand(client *c, robj *dstkey, int numkeysIndex, in | |||
|                     tmp = zuiNewSdsFromValue(&zval); | ||||
|                     znode = zslInsert(dstzset->zsl,score,tmp); | ||||
|                     dictAdd(dstzset->dict,tmp,&znode->score); | ||||
|                     totelelen += sdslen(tmp); | ||||
|                     if (sdslen(tmp) > maxelelen) maxelelen = sdslen(tmp); | ||||
|                 } | ||||
|             } | ||||
|  | @ -2704,6 +2719,7 @@ void zunionInterDiffGenericCommand(client *c, robj *dstkey, int numkeysIndex, in | |||
|                     /* Remember the longest single element encountered,
 | ||||
|                      * to understand if it's possible to convert to listpack | ||||
|                      * at the end. */ | ||||
|                      totelelen += sdslen(tmp); | ||||
|                      if (sdslen(tmp) > maxelelen) maxelelen = sdslen(tmp); | ||||
|                     /* Update the element with its initial score. */ | ||||
|                     dictSetKey(accumulator, de, tmp); | ||||
|  | @ -2738,14 +2754,14 @@ void zunionInterDiffGenericCommand(client *c, robj *dstkey, int numkeysIndex, in | |||
|         dictReleaseIterator(di); | ||||
|         dictRelease(accumulator); | ||||
|     } else if (op == SET_OP_DIFF) { | ||||
|         zdiff(src, setnum, dstzset, &maxelelen); | ||||
|         zdiff(src, setnum, dstzset, &maxelelen, &totelelen); | ||||
|     } else { | ||||
|         serverPanic("Unknown operator"); | ||||
|     } | ||||
| 
 | ||||
|     if (dstkey) { | ||||
|         if (dstzset->zsl->length) { | ||||
|             zsetConvertToListpackIfNeeded(dstobj, maxelelen); | ||||
|             zsetConvertToListpackIfNeeded(dstobj, maxelelen, totelelen); | ||||
|             setKey(c, c->db, dstkey, dstobj); | ||||
|             addReplyLongLong(c, zsetLength(dstobj)); | ||||
|             notifyKeyspaceEvent(NOTIFY_ZSET, | ||||
|  |  | |||
|  | @ -267,6 +267,17 @@ | |||
|         ZIPLIST_LENGTH(zl) = intrev16ifbe(intrev16ifbe(ZIPLIST_LENGTH(zl))+incr); \ | ||||
| } | ||||
| 
 | ||||
| /* Don't let ziplists grow over 1GB in any case, don't wanna risk overflow in
 | ||||
|  * zlbytes */ | ||||
| #define ZIPLIST_MAX_SAFETY_SIZE (1<<30) | ||||
| int ziplistSafeToAdd(unsigned char* zl, size_t add) { | ||||
|     size_t len = zl? ziplistBlobLen(zl): 0; | ||||
|     if (len + add > ZIPLIST_MAX_SAFETY_SIZE) | ||||
|         return 0; | ||||
|     return 1; | ||||
| } | ||||
| 
 | ||||
| 
 | ||||
| /* We use this function to receive information about a ziplist entry.
 | ||||
|  * Note that this is not how the data is actually encoded, is just what we | ||||
|  * get filled by a function in order to operate more easily. */ | ||||
|  | @ -709,7 +720,8 @@ unsigned char *ziplistNew(void) { | |||
| } | ||||
| 
 | ||||
| /* Resize the ziplist. */ | ||||
| unsigned char *ziplistResize(unsigned char *zl, unsigned int len) { | ||||
| unsigned char *ziplistResize(unsigned char *zl, size_t len) { | ||||
|     assert(len < UINT32_MAX); | ||||
|     zl = zrealloc(zl,len); | ||||
|     ZIPLIST_BYTES(zl) = intrev32ifbe(len); | ||||
|     zl[len-1] = ZIP_END; | ||||
|  | @ -1070,6 +1082,9 @@ unsigned char *ziplistMerge(unsigned char **first, unsigned char **second) { | |||
|     /* Combined zl length should be limited within UINT16_MAX */ | ||||
|     zllength = zllength < UINT16_MAX ? zllength : UINT16_MAX; | ||||
| 
 | ||||
|     /* larger values can't be stored into ZIPLIST_BYTES */ | ||||
|     assert(zlbytes < UINT32_MAX); | ||||
| 
 | ||||
|     /* Save offset positions before we start ripping memory apart. */ | ||||
|     size_t first_offset = intrev32ifbe(ZIPLIST_TAIL_OFFSET(*first)); | ||||
|     size_t second_offset = intrev32ifbe(ZIPLIST_TAIL_OFFSET(*second)); | ||||
|  |  | |||
|  | @ -65,6 +65,7 @@ int ziplistValidateIntegrity(unsigned char *zl, size_t size, int deep, | |||
| void ziplistRandomPair(unsigned char *zl, unsigned long total_count, ziplistEntry *key, ziplistEntry *val); | ||||
| void ziplistRandomPairs(unsigned char *zl, unsigned int count, ziplistEntry *keys, ziplistEntry *vals); | ||||
| unsigned int ziplistRandomPairsUnique(unsigned char *zl, unsigned int count, ziplistEntry *keys, ziplistEntry *vals); | ||||
| int ziplistSafeToAdd(unsigned char* zl, size_t add); | ||||
| 
 | ||||
| #ifdef REDIS_TEST | ||||
| int ziplistTest(int argc, char *argv[], int accurate); | ||||
|  |  | |||
|  | @ -0,0 +1,156 @@ | |||
| # These tests consume massive amounts of memory, and are not | ||||
| # suitable to be executed as part of the normal test suite | ||||
| set ::str500 [string repeat x 500000000] ;# 500mb | ||||
| 
 | ||||
| # Utility function to write big argument into redis client connection | ||||
| proc write_big_bulk {size} { | ||||
|     r write "\$$size\r\n" | ||||
|     while {$size >= 500000000} { | ||||
|         r write $::str500 | ||||
|         incr size -500000000 | ||||
|     } | ||||
|     if {$size > 0} { | ||||
|         r write [string repeat x $size] | ||||
|     } | ||||
|     r write "\r\n" | ||||
| } | ||||
| 
 | ||||
| # One XADD with one huge 5GB field | ||||
| # Expected to fail resulting in an empty stream | ||||
| start_server [list overrides [list save ""] ] { | ||||
|     test {XADD one huge field} { | ||||
|         r config set proto-max-bulk-len 10000000000 ;#10gb | ||||
|         r config set client-query-buffer-limit 10000000000 ;#10gb | ||||
|         r write "*5\r\n\$4\r\nXADD\r\n\$2\r\nS1\r\n\$1\r\n*\r\n" | ||||
|         r write "\$1\r\nA\r\n" | ||||
|         write_big_bulk 5000000000 ;#5gb | ||||
|         r flush | ||||
|         catch {r read} err | ||||
|         assert_match {*too large*} $err | ||||
|         r xlen S1 | ||||
|     } {0} | ||||
| } | ||||
| 
 | ||||
| # One XADD with one huge (exactly nearly) 4GB field | ||||
| # This uncovers the overflow in lpEncodeGetType | ||||
| # Expected to fail resulting in an empty stream | ||||
| start_server [list overrides [list save ""] ] { | ||||
|     test {XADD one huge field - 1} { | ||||
|         r config set proto-max-bulk-len 10000000000 ;#10gb | ||||
|         r config set client-query-buffer-limit 10000000000 ;#10gb | ||||
|         r write "*5\r\n\$4\r\nXADD\r\n\$2\r\nS1\r\n\$1\r\n*\r\n" | ||||
|         r write "\$1\r\nA\r\n" | ||||
|         write_big_bulk 4294967295 ;#4gb-1 | ||||
|         r flush | ||||
|         catch {r read} err | ||||
|         assert_match {*too large*} $err | ||||
|         r xlen S1 | ||||
|     } {0} | ||||
| } | ||||
| 
 | ||||
| # Gradually add big stream fields using repeated XADD calls | ||||
| start_server [list overrides [list save ""] ] { | ||||
|     test {several XADD big fields} { | ||||
|         r config set stream-node-max-bytes 0 | ||||
|         for {set j 0} {$j<10} {incr j} { | ||||
|             r xadd stream * 1 $::str500 2 $::str500 | ||||
|         } | ||||
|         r ping | ||||
|         r xlen stream | ||||
|     } {10} | ||||
| } | ||||
| 
 | ||||
| # Add over 4GB to a single stream listpack (one XADD command) | ||||
| # Expected to fail resulting in an empty stream | ||||
| start_server [list overrides [list save ""] ] { | ||||
|     test {single XADD big fields} { | ||||
|         r write "*23\r\n\$4\r\nXADD\r\n\$1\r\nS\r\n\$1\r\n*\r\n" | ||||
|         for {set j 0} {$j<10} {incr j} { | ||||
|             r write "\$1\r\n$j\r\n" | ||||
|             write_big_bulk 500000000 ;#500mb | ||||
|         } | ||||
|         r flush | ||||
|         catch {r read} err | ||||
|         assert_match {*too large*} $err | ||||
|         r xlen S | ||||
|     } {0} | ||||
| } | ||||
| 
 | ||||
| # Gradually add big hash fields using repeated HSET calls | ||||
| # This reproduces the overflow in the call to ziplistResize | ||||
| # Object will be converted to hashtable encoding | ||||
| start_server [list overrides [list save ""] ] { | ||||
|     r config set hash-max-ziplist-value 1000000000 ;#1gb | ||||
|     test {hash with many big fields} { | ||||
|         for {set j 0} {$j<10} {incr j} { | ||||
|             r hset h $j $::str500 | ||||
|         } | ||||
|         r object encoding h | ||||
|     } {hashtable} | ||||
| } | ||||
| 
 | ||||
| # Add over 4GB to a single hash field (one HSET command) | ||||
| # Object will be converted to hashtable encoding | ||||
| start_server [list overrides [list save ""] ] { | ||||
|     test {hash with one huge field} { | ||||
|         catch {r config set hash-max-ziplist-value 10000000000} ;#10gb | ||||
|         r config set proto-max-bulk-len 10000000000 ;#10gb | ||||
|         r config set client-query-buffer-limit 10000000000 ;#10gb | ||||
|         r write "*4\r\n\$4\r\nHSET\r\n\$2\r\nH1\r\n" | ||||
|         r write "\$1\r\nA\r\n" | ||||
|         write_big_bulk 5000000000 ;#5gb | ||||
|         r flush | ||||
|         r read | ||||
|         r object encoding H1 | ||||
|     } {hashtable} | ||||
| } | ||||
| 
 | ||||
| # Add over 4GB to a single list member (one LPUSH command) | ||||
| # Currently unsupported, and expected to fail rather than being truncated | ||||
| # Expected to fail resulting in a non-existing list | ||||
| start_server [list overrides [list save ""] ] { | ||||
|     test {list with one huge field} { | ||||
|         r config set proto-max-bulk-len 10000000000 ;#10gb | ||||
|         r config set client-query-buffer-limit 10000000000 ;#10gb | ||||
|         r write "*3\r\n\$5\r\nLPUSH\r\n\$2\r\nL1\r\n" | ||||
|         write_big_bulk 5000000000 ;#5gb | ||||
|         r flush | ||||
|         catch {r read} err | ||||
|         assert_match {*too large*} $err | ||||
|         r exists L1 | ||||
|     } {0} | ||||
| } | ||||
| 
 | ||||
| # SORT which attempts to store an element larger than 4GB into a list. | ||||
| # Currently unsupported and results in an assertion instead of truncation | ||||
| start_server [list overrides [list save ""] ] { | ||||
|     test {SORT adds huge field to list} { | ||||
|         r config set proto-max-bulk-len 10000000000 ;#10gb | ||||
|         r config set client-query-buffer-limit 10000000000 ;#10gb | ||||
|         r write "*3\r\n\$3\r\nSET\r\n\$2\r\nS1\r\n" | ||||
|         write_big_bulk 5000000000 ;#5gb | ||||
|         r flush | ||||
|         r read | ||||
|         assert_equal [r strlen S1] 5000000000 | ||||
|         r set S2 asdf | ||||
|         r sadd myset 1 2 | ||||
|         r mset D1 1 D2 2 | ||||
|         catch {r sort myset by D* get S* store mylist} | ||||
|         assert_equal [count_log_message 0 "crashed by signal"] 0 | ||||
|         assert_equal [count_log_message 0 "ASSERTION FAILED"] 1 | ||||
|     } | ||||
| } | ||||
| 
 | ||||
| # SORT which stores an integer encoded element into a list. | ||||
| # Just for coverage, no news here. | ||||
| start_server [list overrides [list save ""] ] { | ||||
|     test {SORT adds integer field to list} { | ||||
|         r set S1 asdf | ||||
|         r set S2 123 ;# integer encoded | ||||
|         assert_encoding "int" S2 | ||||
|         r sadd myset 1 2 | ||||
|         r mset D1 1 D2 2 | ||||
|         r sort myset by D* get S* store mylist | ||||
|         r llen mylist | ||||
|     } {2} | ||||
| } | ||||
		Loading…
	
		Reference in New Issue