Use madvise(MADV_DONTNEED) to release memory to reduce COW (#8974)

## Backgroud
As we know, after `fork`, one process will copy pages when writing data to these
pages(CoW), and another process still keep old pages, they totally cost more memory.
For redis, we suffered that redis consumed much memory when the fork child is serializing
key/values, even that maybe cause OOM.

But actually we find, in redis fork child process, the child process don't need to keep some
memory and parent process may write or update that, for example, child process will never
access the key-value that is serialized but users may update it in parent process.
So we think it may reduce COW if the child process release memory that it is not needed.

## Implementation
For releasing key value in child process, we may think we call `decrRefCount` to free memory,
but i find the fork child process still use much memory when we don't write any data to redis,
and it costs much more time that slows down bgsave. Maybe because memory allocator doesn't
really release memory to OS, and it may modify some inner data for this free operation, especially
when we free small objects.

Moreover, CoW is based on  pages, so it is a easy way that we only free the memory bulk that is
not less than kernel page size. madvise(MADV_DONTNEED) can quickly release specified region
pages to OS bypassing memory allocator, and allocator still consider that this memory still is used
and don't change its inner data.

There are some buffers we can release in the fork child process:
- **Serialized key-values**
  the fork child process never access serialized key-values, so we try to free them.
  Because we only can release big bulk memory, and it is time consumed to iterate all
  items/members/fields/entries of complex data type. So we decide to iterate them and
  try to release them only when their average size of item/member/field/entry is more
  than page size of OS.
- **Replication backlog**
  Because replication backlog is a cycle buffer, it will be changed quickly if redis has heavy
  write traffic, but in fork child process, we don't need to access that.
- **Client buffers**
  If clients have requests during having the fork child process, clients' buffer also be changed
  frequently. The memory includes client query buffer, output buffer, and client struct used memory.

To get child process peak private dirty memory, we need to count peak memory instead
of last used memory, because the child process may continue to release memory (since
COW used to only grow till now, the last was equivalent to the peak).
Also we're adding a new `current_cow_peak` info variable (to complement the existing
`current_cow_size`)

Co-authored-by: Oran Agra <oran@redislabs.com>
This commit is contained in:
Wang Yuan 2021-08-05 04:01:46 +08:00 committed by GitHub
parent 56eb7f7de4
commit d4bca53cd9
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 432 additions and 9 deletions

View File

@ -1440,6 +1440,7 @@ int rewriteAppendOnlyFileRio(rio *aof) {
sds keystr;
robj key, *o;
long long expiretime;
size_t aof_bytes_before_key = aof->processed_bytes;
keystr = dictGetKey(de);
o = dictGetVal(de);
@ -1470,6 +1471,13 @@ int rewriteAppendOnlyFileRio(rio *aof) {
} else {
serverPanic("Unknown object type");
}
/* In fork child process, we can try to release memory back to the
* OS and possibly avoid or decrease COW. We guve the dismiss
* mechanism a hint about an estimated size of the object we stored. */
size_t dump_size = aof->processed_bytes - aof_bytes_before_key;
if (server.in_fork_child) dismissObject(o, dump_size);
/* Save the expire time */
if (expiretime != -1) {
char cmd[]="*3\r\n$9\r\nPEXPIREAT\r\n";

View File

@ -73,6 +73,9 @@ void sendChildInfoGeneric(childInfoType info_type, size_t keys, double progress,
static monotime cow_updated = 0;
static uint64_t cow_update_cost = 0;
static size_t cow = 0;
static size_t peak_cow = 0;
static size_t update_count = 0;
static unsigned long long sum_cow = 0;
child_info_data data = {0}; /* zero everything, including padding to satisfy valgrind */
@ -89,11 +92,15 @@ void sendChildInfoGeneric(childInfoType info_type, size_t keys, double progress,
cow = zmalloc_get_private_dirty(-1);
cow_updated = getMonotonicUs();
cow_update_cost = cow_updated - now;
if (cow > peak_cow) peak_cow = cow;
sum_cow += cow;
update_count++;
if (cow) {
serverLog((info_type == CHILD_INFO_TYPE_CURRENT_INFO) ? LL_VERBOSE : LL_NOTICE,
"%s: %zu MB of memory used by copy-on-write",
pname, cow / (1024 * 1024));
int cow_info = (info_type != CHILD_INFO_TYPE_CURRENT_INFO);
if (cow || cow_info) {
serverLog(cow_info ? LL_NOTICE : LL_VERBOSE,
"Fork CoW for %s: current %zu MB, peak %zu MB, average %llu MB",
pname, cow>>20, peak_cow>>20, (sum_cow/update_count)>>20);
}
}
@ -112,17 +119,19 @@ void sendChildInfoGeneric(childInfoType info_type, size_t keys, double progress,
/* Update Child info. */
void updateChildInfo(childInfoType information_type, size_t cow, monotime cow_updated, size_t keys, double progress) {
if (cow > server.stat_current_cow_peak) server.stat_current_cow_peak = cow;
if (information_type == CHILD_INFO_TYPE_CURRENT_INFO) {
server.stat_current_cow_bytes = cow;
server.stat_current_cow_updated = cow_updated;
server.stat_current_save_keys_processed = keys;
if (progress != -1) server.stat_module_progress = progress;
} else if (information_type == CHILD_INFO_TYPE_AOF_COW_SIZE) {
server.stat_aof_cow_bytes = cow;
server.stat_aof_cow_bytes = server.stat_current_cow_peak;
} else if (information_type == CHILD_INFO_TYPE_RDB_COW_SIZE) {
server.stat_rdb_cow_bytes = cow;
server.stat_rdb_cow_bytes = server.stat_current_cow_peak;
} else if (information_type == CHILD_INFO_TYPE_MODULE_COW_SIZE) {
server.stat_module_cow_bytes = cow;
server.stat_module_cow_bytes = server.stat_current_cow_peak;
}
}

View File

@ -377,6 +377,169 @@ void decrRefCount(robj *o) {
}
}
/* See dismissObject() */
void dismissSds(sds s) {
dismissMemory(sdsAllocPtr(s), sdsAllocSize(s));
}
/* See dismissObject() */
void dismissStringObject(robj *o) {
if (o->encoding == OBJ_ENCODING_RAW) {
dismissSds(o->ptr);
}
}
/* See dismissObject() */
void dismissListObject(robj *o, size_t size_hint) {
if (o->encoding == OBJ_ENCODING_QUICKLIST) {
quicklist *ql = o->ptr;
serverAssert(ql->len != 0);
/* We iterate all nodes only when average node size is bigger than a
* page size, and there's a high chance we'll actually dismiss something. */
if (size_hint / ql->len >= server.page_size) {
quicklistNode *node = ql->head;
while (node) {
if (quicklistNodeIsCompressed(node)) {
dismissMemory(node->zl, ((quicklistLZF*)node->zl)->sz);
} else {
dismissMemory(node->zl, node->sz);
}
node = node->next;
}
}
}
}
/* See dismissObject() */
void dismissSetObject(robj *o, size_t size_hint) {
if (o->encoding == OBJ_ENCODING_HT) {
dict *set = o->ptr;
serverAssert(dictSize(set) != 0);
/* We iterate all nodes only when average member size is bigger than a
* page size, and there's a high chance we'll actually dismiss something. */
if (size_hint / dictSize(set) >= server.page_size) {
dictEntry *de;
dictIterator *di = dictGetIterator(set);
while ((de = dictNext(di)) != NULL) {
dismissSds(dictGetKey(de));
}
dictReleaseIterator(di);
}
/* Dismiss hash table memory. */
dismissMemory(set->ht[0].table, set->ht[0].size*sizeof(dictEntry*));
dismissMemory(set->ht[1].table, set->ht[1].size*sizeof(dictEntry*));
} else if (o->encoding == OBJ_ENCODING_INTSET) {
dismissMemory(o->ptr, intsetBlobLen((intset*)o->ptr));
}
}
/* See dismissObject() */
void dismissZsetObject(robj *o, size_t size_hint) {
if (o->encoding == OBJ_ENCODING_SKIPLIST) {
zset *zs = o->ptr;
zskiplist *zsl = zs->zsl;
serverAssert(zsl->length != 0);
/* We iterate all nodes only when average member size is bigger than a
* page size, and there's a high chance we'll actually dismiss something. */
if (size_hint / zsl->length >= server.page_size) {
zskiplistNode *zn = zsl->tail;
while (zn != NULL) {
dismissSds(zn->ele);
zn = zn->backward;
}
}
/* Dismiss hash table memory. */
dict *d = zs->dict;
dismissMemory(d->ht[0].table, d->ht[0].size*sizeof(dictEntry*));
dismissMemory(d->ht[1].table, d->ht[1].size*sizeof(dictEntry*));
} else if (o->encoding == OBJ_ENCODING_ZIPLIST) {
dismissMemory(o->ptr, ziplistBlobLen((unsigned char*)o->ptr));
}
}
/* See dismissObject() */
void dismissHashObject(robj *o, size_t size_hint) {
if (o->encoding == OBJ_ENCODING_HT) {
dict *d = o->ptr;
serverAssert(dictSize(d) != 0);
/* We iterate all fields only when average field/value size is bigger than
* a page size, and there's a high chance we'll actually dismiss something. */
if (size_hint / dictSize(d) >= server.page_size) {
dictEntry *de;
dictIterator *di = dictGetIterator(d);
while ((de = dictNext(di)) != NULL) {
/* Only dismiss values memory since the field size
* usually is small. */
dismissSds(dictGetVal(de));
}
dictReleaseIterator(di);
}
/* Dismiss hash table memory. */
dismissMemory(d->ht[0].table, d->ht[0].size*sizeof(dictEntry*));
dismissMemory(d->ht[1].table, d->ht[1].size*sizeof(dictEntry*));
} else if (o->encoding == OBJ_ENCODING_ZIPLIST) {
dismissMemory(o->ptr, ziplistBlobLen((unsigned char*)o->ptr));
}
}
/* See dismissObject() */
void dismissStreamObject(robj *o, size_t size_hint) {
stream *s = o->ptr;
rax *rax = s->rax;
if (raxSize(rax) == 0) return;
/* Iterate only on stream entries, although size_hint may include serialized
* consumer groups info, but usually, stream entries take up most of
* the space. */
if (size_hint / raxSize(rax) >= server.page_size) {
raxIterator ri;
raxStart(&ri,rax);
raxSeek(&ri,"^",NULL,0);
while (raxNext(&ri)) {
dismissMemory(ri.data, lpBytes(ri.data));
}
raxStop(&ri);
}
}
/* When creating a snapshot in a fork child process, the main process and child
* process share the same physical memory pages, and if / when the parent
* modifies any keys due to write traffic, it'll cause CoW which consume
* physical memory. In the child process, after serializing the key and value,
* the data is definitely not accessed again, so to avoid unnecessary CoW, we
* try to release their memory back to OS. see dismissMemory().
*
* Because of the cost of iterating all node/field/member/entry of complex data
* types, we iterate and dismiss them only when approximate average we estimate
* the size of an individual allocation is more than a page size of OS.
* 'size_hint' is the size of serialized value. This method is not accurate, but
* it can reduce unnecessary iteration for complex data types that are probably
* not going to release any memory. */
void dismissObject(robj *o, size_t size_hint) {
/* madvise(MADV_DONTNEED) may not work if Transparent Huge Pages is enabled. */
if (server.thp_enabled) return;
/* Currently we use zmadvise_dontneed only when we use jemalloc.
* so we avoid these pointless loops when they're not going to do anything. */
#if defined(USE_JEMALLOC)
if (o->refcount != 1) return;
switch(o->type) {
case OBJ_STRING: dismissStringObject(o); break;
case OBJ_LIST: dismissListObject(o, size_hint); break;
case OBJ_SET: dismissSetObject(o, size_hint); break;
case OBJ_ZSET: dismissZsetObject(o, size_hint); break;
case OBJ_HASH: dismissHashObject(o, size_hint); break;
case OBJ_STREAM: dismissStreamObject(o, size_hint); break;
default: break;
}
#else
UNUSED(o); UNUSED(size_hint);
#endif
}
/* This variant of decrRefCount() gets its argument as void, and is useful
* as free method in data structures that expect a 'void free_object(void*)'
* prototype for the free method. */

View File

@ -1248,11 +1248,18 @@ int rdbSaveRio(rio *rdb, int *error, int rdbflags, rdbSaveInfo *rsi) {
sds keystr = dictGetKey(de);
robj key, *o = dictGetVal(de);
long long expire;
size_t rdb_bytes_before_key = rdb->processed_bytes;
initStaticStringObject(key,keystr);
expire = getExpire(db,&key);
if (rdbSaveKeyValuePair(rdb,&key,o,expire,j) == -1) goto werr;
/* In fork child process, we can try to release memory back to the
* OS and possibly avoid or decrease COW. We give the dismiss
* mechanism a hint about an estimated size of the object we stored. */
size_t dump_size = rdb->processed_bytes - rdb_bytes_before_key;
if (server.in_fork_child) dismissObject(o, dump_size);
/* When this RDB is produced as part of an AOF rewrite, move
* accumulated diff from parent to child while rewriting in
* order to have a smaller final write. */

View File

@ -1615,6 +1615,7 @@ int hasActiveChildProcess() {
void resetChildState() {
server.child_type = CHILD_TYPE_NONE;
server.child_pid = -1;
server.stat_current_cow_peak = 0;
server.stat_current_cow_bytes = 0;
server.stat_current_cow_updated = 0;
server.stat_current_save_keys_processed = 0;
@ -2693,6 +2694,7 @@ void initServerConfig(void) {
server.migrate_cached_sockets = dictCreate(&migrateCacheDictType,NULL);
server.next_client_id = 1; /* Client IDs, start from 1 .*/
server.loading_process_events_interval_bytes = (1024*1024*2);
server.page_size = sysconf(_SC_PAGESIZE);
unsigned int lruclock = getLRUClock();
atomicSet(server.lruclock,lruclock);
@ -3186,6 +3188,7 @@ void initServer(void) {
server.system_memory_size = zmalloc_get_memory_size();
server.blocked_last_cron = 0;
server.blocking_op_nesting = 0;
server.thp_enabled = 0;
if ((server.tls_port || server.tls_replication || server.tls_cluster)
&& tlsConfigure(&server.tls_ctx_config) == C_ERR) {
@ -3281,6 +3284,7 @@ void initServer(void) {
/* A few stats we don't want to reset: server startup time, and peak mem. */
server.stat_starttime = time(NULL);
server.stat_peak_memory = 0;
server.stat_current_cow_peak = 0;
server.stat_current_cow_bytes = 0;
server.stat_current_cow_updated = 0;
server.stat_current_save_keys_processed = 0;
@ -4882,6 +4886,7 @@ sds genRedisInfoString(const char *section) {
info = sdscatprintf(info,
"# Persistence\r\n"
"loading:%d\r\n"
"current_cow_peak:%zu\r\n"
"current_cow_size:%zu\r\n"
"current_cow_size_age:%lu\r\n"
"current_fork_perc:%.2f\r\n"
@ -4905,6 +4910,7 @@ sds genRedisInfoString(const char *section) {
"module_fork_in_progress:%d\r\n"
"module_fork_last_cow_size:%zu\r\n",
(int)server.loading,
server.stat_current_cow_peak,
server.stat_current_cow_bytes,
server.stat_current_cow_updated ? (unsigned long) elapsedMs(server.stat_current_cow_updated) / 1000 : 0,
fork_perc,
@ -5410,7 +5416,12 @@ void linuxMemoryWarnings(void) {
if (linuxOvercommitMemoryValue() == 0) {
serverLog(LL_WARNING,"WARNING overcommit_memory is set to 0! Background save may fail under low memory condition. To fix this issue add 'vm.overcommit_memory = 1' to /etc/sysctl.conf and then reboot or run the command 'sysctl vm.overcommit_memory=1' for this to take effect.");
}
if (THPIsEnabled() && THPDisable()) {
if (THPIsEnabled()) {
server.thp_enabled = 1;
if (THPDisable() == 0) {
server.thp_enabled = 0;
return;
}
serverLog(LL_WARNING,"WARNING you have Transparent Huge Pages (THP) support enabled in your kernel. This will create latency and memory usage issues with Redis. To fix this issue run the command 'echo madvise > /sys/kernel/mm/transparent_hugepage/enabled' as root, and add it to your /etc/rc.local in order to retain the setting after a reboot. Redis must be restarted after THP is disabled (set to 'madvise' or 'never').");
}
}
@ -5864,6 +5875,7 @@ int redisFork(int purpose) {
if ((childpid = fork()) == 0) {
/* Child */
server.in_fork_child = purpose;
dismissMemoryInChild();
setOOMScoreAdj(CONFIG_OOM_BGCHILD);
setupChildSignalHandlers();
closeChildUnusedResourceAfterFork();
@ -5888,6 +5900,7 @@ int redisFork(int purpose) {
if (isMutuallyExclusiveChildType(purpose)) {
server.child_pid = childpid;
server.child_type = purpose;
server.stat_current_cow_peak = 0;
server.stat_current_cow_bytes = 0;
server.stat_current_cow_updated = 0;
server.stat_current_save_keys_processed = 0;
@ -5911,6 +5924,87 @@ void sendChildInfo(childInfoType info_type, size_t keys, char *pname) {
sendChildInfoGeneric(info_type, keys, -1, pname);
}
/* Try to release pages back to the OS directly (bypassing the allocator),
* in an effort to decrease CoW during fork. For small allocations, we can't
* release any full page, so in an effort to avoid getting the size of the
* allocation from the allocator (malloc_size) when we already know it's small,
* we check the size_hint. If the size is not already known, passing a size_hint
* of 0 will lead the checking the real size of the allocation.
* Also please note that the size may be not accurate, so in order to make this
* solution effective, the judgement for releasing memory pages should not be
* too strict. */
void dismissMemory(void* ptr, size_t size_hint) {
if (ptr == NULL) return;
/* madvise(MADV_DONTNEED) can not release pages if the size of memory
* is too small, we try to release only for the memory which the size
* is more than half of page size. */
if (size_hint && size_hint <= server.page_size/2) return;
zmadvise_dontneed(ptr);
}
/* Dismiss big chunks of memory inside a client structure, see dismissMemory() */
void dismissClientMemory(client *c) {
/* Dismiss client query buffer. */
dismissSds(c->querybuf);
dismissSds(c->pending_querybuf);
/* Dismiss argv array only if we estimate it contains a big buffer. */
if (c->argc && c->argv_len_sum/c->argc >= server.page_size) {
for (int i = 0; i < c->argc; i++) {
dismissObject(c->argv[i], 0);
}
}
if (c->argc) dismissMemory(c->argv, c->argc*sizeof(robj*));
/* Dismiss the reply array only if the average buffer size is bigger
* than a page. */
if (listLength(c->reply) &&
c->reply_bytes/listLength(c->reply) >= server.page_size)
{
listIter li;
listNode *ln;
listRewind(c->reply, &li);
while ((ln = listNext(&li))) {
clientReplyBlock *bulk = listNodeValue(ln);
/* Default bulk size is 16k, actually it has extra data, maybe it
* occupies 20k according to jemalloc bin size if using jemalloc. */
if (bulk) dismissMemory(bulk, bulk->size);
}
}
/* The client struct has a big static reply buffer in it. */
dismissMemory(c, 0);
}
/* In the child process, we don't need some buffers anymore, and these are
* likely to change in the parent when there's heavy write traffic.
* We dismis them right away, to avoid CoW.
* see dismissMemeory(). */
void dismissMemoryInChild(void) {
/* madvise(MADV_DONTNEED) may not work if Transparent Huge Pages is enabled. */
if (server.thp_enabled) return;
/* Currently we use zmadvise_dontneed only when we use jemalloc.
* so we avoid these pointless loops when they're not going to do anything. */
#if defined(USE_JEMALLOC)
/* Dismiss replication backlog. */
if (server.repl_backlog != NULL) {
dismissMemory(server.repl_backlog, server.repl_backlog_size);
}
/* Dismiss all clients memory. */
listIter li;
listNode *ln;
listRewind(server.clients, &li);
while((ln = listNext(&li))) {
client *c = listNodeValue(ln);
dismissClientMemory(c);
}
#endif
}
void memtest(size_t megabytes, int passes);
/* Returns 1 if there is --sentinel among the arguments or if

View File

@ -1245,6 +1245,8 @@ struct redisServer {
int propagate_in_transaction; /* Make sure we don't propagate nested MULTI/EXEC */
char *ignore_warnings; /* Config: warnings that should be ignored. */
int client_pause_in_transaction; /* Was a client pause executed during this Exec? */
int thp_enabled; /* If true, THP is enabled. */
size_t page_size; /* The page size of OS. */
/* Modules */
dict *moduleapi; /* Exported core APIs dictionary for modules. */
dict *sharedapi; /* Like moduleapi but containing the APIs that
@ -1336,6 +1338,7 @@ struct redisServer {
struct malloc_stats cron_malloc_stats; /* sampled in serverCron(). */
redisAtomic long long stat_net_input_bytes; /* Bytes read from network. */
redisAtomic long long stat_net_output_bytes; /* Bytes written to network. */
size_t stat_current_cow_peak; /* Peak size of copy on write bytes. */
size_t stat_current_cow_bytes; /* Copy on write bytes while child is active. */
monotime stat_current_cow_updated; /* Last update time of stat_current_cow_bytes */
size_t stat_current_save_keys_processed; /* Processed keys while child is active. */
@ -2017,6 +2020,7 @@ void freeListObject(robj *o);
void freeSetObject(robj *o);
void freeZsetObject(robj *o);
void freeHashObject(robj *o);
void dismissObject(robj *o, size_t dump_size);
robj *createObject(int type, void *ptr);
robj *createStringObject(const char *ptr, size_t len);
robj *createRawStringObject(const char *ptr, size_t len);
@ -2300,6 +2304,9 @@ int setOOMScoreAdj(int process_class);
void rejectCommandFormat(client *c, const char *fmt, ...);
void *activeDefragAlloc(void *ptr);
robj *activeDefragStringOb(robj* ob, long *defragged);
void dismissSds(sds s);
void dismissMemory(void* ptr, size_t size_hint);
void dismissMemoryInChild(void);
#define RESTART_SERVER_NONE 0
#define RESTART_SERVER_GRACEFULLY (1<<0) /* Do proper shutdown. */

View File

@ -28,12 +28,20 @@
* POSSIBILITY OF SUCH DAMAGE.
*/
#include "fmacros.h"
#include "config.h"
#include "solarisfixes.h"
#include <stdio.h>
#include <stdlib.h>
#include <stdint.h>
#include <unistd.h>
#include <assert.h>
#ifdef __linux__
#include <sys/mman.h>
#endif
/* This function provide us access to the original libc free(). This is useful
* for instance to free results obtained by backtrace_symbols(). We need
* to define this function before including zmalloc.h that may shadow the
@ -44,7 +52,6 @@ void zlibc_free(void *ptr) {
#include <string.h>
#include <pthread.h>
#include "config.h"
#include "zmalloc.h"
#include "atomicvar.h"
@ -335,6 +342,31 @@ void zmalloc_set_oom_handler(void (*oom_handler)(size_t)) {
zmalloc_oom_handler = oom_handler;
}
/* Use 'MADV_DONTNEED' to release memory to operating system quickly.
* We do that in a fork child process to avoid CoW when the parent modifies
* these shared pages. */
void zmadvise_dontneed(void *ptr) {
#if defined(USE_JEMALLOC)
static size_t page_size = 0;
if (page_size == 0) page_size = sysconf(_SC_PAGESIZE);
size_t page_size_mask = page_size - 1;
size_t real_size = zmalloc_size(ptr);
if (real_size < page_size) return;
/* We need to align the pointer upwards according to page size, because
* the memory address is increased upwards and we only can free memory
* based on page. */
char *aligned_ptr = (char *)(((size_t)ptr+page_size_mask) & ~page_size_mask);
real_size -= (aligned_ptr-(char*)ptr);
if (real_size >= page_size) {
madvise((void *)aligned_ptr, real_size&~page_size_mask, MADV_DONTNEED);
}
#else
(void)(ptr);
#endif
}
/* Get the RSS information in an OS-specific way.
*
* WARNING: the function zmalloc_get_rss() is not designed to be fast

View File

@ -121,6 +121,7 @@ size_t zmalloc_get_private_dirty(long pid);
size_t zmalloc_get_smap_bytes_by_field(char *field, long pid);
size_t zmalloc_get_memory_size(void);
void zlibc_free(void *ptr);
void zmadvise_dontneed(void *ptr);
#ifdef HAVE_DEFRAG
void zfree_no_tcache(void *ptr);

View File

@ -0,0 +1,101 @@
# The tests of this file aim to get coverage on all the "dismiss" methods
# that dismiss all data-types memory in the fork child. like client query
# buffer, client output buffer and replication backlog.
# Actually, we may not have many asserts in the test, since we just check for
# crashes and the dump file inconsistencies.
start_server {tags {"dismiss external:skip"}} {
# In other tests, although we test child process dumping RDB file, but
# memory allocations of key/values are usually small, they couldn't cover
# the "dismiss" object methods, in this test, we create big size key/values
# to satisfy the conditions for release memory pages, especially, we assume
# the page size of OS is 4KB in some cases.
test {dismiss all data types memory} {
set bigstr [string repeat A 8192]
set 64bytes [string repeat A 64]
# string
populate 100 bigstring 8192
# list
r lpush biglist1 $bigstr ; # uncompressed ziplist node
r config set list-compress-depth 1 ; # compressed ziplist nodes
for {set i 0} {$i < 16} {incr i} {
r lpush biglist2 $bigstr
}
# set
r sadd bigset1 $bigstr ; # hash encoding
set biginteger [string repeat 1 19]
for {set i 0} {$i < 512} {incr i} {
r sadd bigset2 $biginteger ; # intset encoding
}
# zset
r zadd bigzset1 1.0 $bigstr ; # skiplist encoding
for {set i 0} {$i < 128} {incr i} {
r zadd bigzset2 1.0 $64bytes ; # ziplist encoding
}
# hash
r hset bighash1 field1 $bigstr ; # hash encoding
for {set i 0} {$i < 128} {incr i} {
r hset bighash2 $i $64bytes ; # ziplist encoding
}
# stream
r xadd bigstream * entry1 $bigstr entry2 $bigstr
set digest [r debug digest]
r config set aof-use-rdb-preamble no
r bgrewriteaof
waitForBgrewriteaof r
r debug loadaof
set newdigest [r debug digest]
assert {$digest eq $newdigest}
}
test {dismiss client output buffer} {
# Big output buffer
set item [string repeat "x" 100000]
for {set i 0} {$i < 100} {incr i} {
r lpush mylist $item
}
set rd [redis_deferring_client]
$rd lrange mylist 0 -1
$rd flush
after 100
r bgsave
waitForBgsave r
assert_equal $item [r lpop mylist]
}
test {dismiss client query buffer} {
# Big pending query buffer
set bigstr [string repeat A 8192]
set rd [redis_deferring_client]
$rd write "*2\r\n\$8192\r\n"
$rd write $bigstr\r\n
$rd flush
after 100
r bgsave
waitForBgsave r
}
test {dismiss replication backlog} {
set master [srv 0 client]
start_server {} {
r slaveof [srv -1 host] [srv -1 port]
wait_for_sync r
set bigstr [string repeat A 8192]
for {set i 0} {$i < 20} {incr i} {
$master set $i $bigstr
}
$master bgsave
waitForBgsave $master
}
}
}

View File

@ -55,6 +55,7 @@ set ::all_tests {
integration/failover
integration/redis-cli
integration/redis-benchmark
integration/dismiss-mem
unit/pubsub
unit/slowlog
unit/scripting