src/transfer.sh
src/configs
src/redis-server.dSYM
+redis.ds
+src/redis.conf
# "no" that is the safest pick from the point of view of durability.
no-appendfsync-on-rewrite no
-################################ VIRTUAL MEMORY ###############################
-
-# Virtual Memory allows Redis to work with datasets bigger than the actual
-# amount of RAM needed to hold the whole dataset in memory.
-# In order to do so very used keys are taken in memory while the other keys
-# are swapped into a swap file, similarly to what operating systems do
-# with memory pages.
-#
-# To enable VM just set 'vm-enabled' to yes, and set the following three
-# VM parameters accordingly to your needs.
-
-vm-enabled no
-# vm-enabled yes
-
-# This is the path of the Redis swap file. As you can guess, swap files
-# can't be shared by different Redis instances, so make sure to use a swap
-# file for every redis process you are running. Redis will complain if the
-# swap file is already in use.
-#
-# The best kind of storage for the Redis swap file (that's accessed at random)
-# is a Solid State Disk (SSD).
-#
-# *** WARNING *** if you are using a shared hosting the default of putting
-# the swap file under /tmp is not secure. Create a dir with access granted
-# only to Redis user and configure Redis to create the swap file there.
-vm-swap-file /tmp/redis.swap
-
-# vm-max-memory configures the VM to use at max the specified amount of
-# RAM. Everything that deos not fit will be swapped on disk *if* possible, that
-# is, if there is still enough contiguous space in the swap file.
-#
-# With vm-max-memory 0 the system will swap everything it can. Not a good
-# default, just specify the max amount of RAM you can in bytes, but it's
-# better to leave some margin. For instance specify an amount of RAM
-# that's more or less between 60 and 80% of your free RAM.
-vm-max-memory 0
-
-# Redis swap files is split into pages. An object can be saved using multiple
-# contiguous pages, but pages can't be shared between different objects.
-# So if your page is too big, small objects swapped out on disk will waste
-# a lot of space. If you page is too small, there is less space in the swap
-# file (assuming you configured the same number of total swap file pages).
-#
-# If you use a lot of small objects, use a page size of 64 or 32 bytes.
-# If you use a lot of big objects, use a bigger page size.
-# If unsure, use the default :)
-vm-page-size 32
-
-# Number of total memory pages in the swap file.
-# Given that the page table (a bitmap of free/used pages) is taken in memory,
-# every 8 pages on disk will consume 1 byte of RAM.
-#
-# The total swap size is vm-page-size * vm-pages
-#
-# With the default of 32-bytes memory pages and 134217728 pages Redis will
-# use a 4 GB swap file, that will use 16 MB of RAM for the page table.
-#
-# It's better to use the smallest acceptable value for your application,
-# but the default is large in order to work in most conditions.
-vm-pages 134217728
-
-# Max number of VM I/O threads running at the same time.
-# This threads are used to read/write data from/to swap file, since they
-# also encode and decode objects from disk to memory or the reverse, a bigger
-# number of threads can help with big objects even if they can't help with
-# I/O itself as the physical device may not be able to couple with many
-# reads/writes operations at the same time.
-#
-# The special value of 0 turn off threaded I/O and enables the blocking
-# Virtual Memory implementation.
-vm-max-threads 4
+#################################### DISK STORE ###############################
+
+# When disk store is active Redis works as an on-disk database, where memory
+# is only used as a object cache.
+#
+# This mode is good for datasets that are bigger than memory, and in general
+# when you want to trade speed for:
+#
+# - less memory used
+# - immediate server restart
+# - per key durability, without need for backgrond savig
+#
+# On the other hand, with disk store enabled MULTI/EXEC are no longer
+# transactional from the point of view of the persistence on disk, that is,
+# Redis transactions will still guarantee that commands are either processed
+# all or nothing, but there is no guarantee that all the keys are flushed
+# on disk in an atomic way.
+#
+# Of course with disk store enabled Redis is not as fast as it is when
+# working with just the memory back end.
+
+diskstore-enabled no
+diskstore-path redis.ds
+cache-max-memory 0
+cache-flush-delay 0
############################### ADVANCED CONFIG ###############################
INSTALL_BIN= $(PREFIX)/bin
INSTALL= cp -p
-OBJ = adlist.o ae.o anet.o dict.o redis.o sds.o zmalloc.o lzf_c.o lzf_d.o pqsort.o zipmap.o sha1.o ziplist.o release.o networking.o util.o object.o db.o replication.o rdb.o t_string.o t_list.o t_set.o t_zset.o t_hash.o config.o aof.o vm.o pubsub.o multi.o debug.o sort.o intset.o syncio.o
+OBJ = adlist.o ae.o anet.o dict.o redis.o sds.o zmalloc.o lzf_c.o lzf_d.o pqsort.o zipmap.o sha1.o ziplist.o release.o networking.o util.o object.o db.o replication.o rdb.o t_string.o t_list.o t_set.o t_zset.o t_hash.o config.o aof.o dscache.o pubsub.o multi.o debug.o sort.o intset.o syncio.o diskstore.o
BENCHOBJ = ae.o anet.o redis-benchmark.o sds.o adlist.o zmalloc.o
CLIOBJ = anet.o sds.o adlist.o redis-cli.o zmalloc.o release.o
CHECKDUMPOBJ = redis-check-dump.o lzf_c.o lzf_d.o
anet.o: anet.c fmacros.h anet.h
aof.o: aof.c redis.h fmacros.h config.h ae.h sds.h dict.h adlist.h \
zmalloc.h anet.h zipmap.h ziplist.h intset.h version.h
-chprgname.o: chprgname.c
config.o: config.c redis.h fmacros.h config.h ae.h sds.h dict.h adlist.h \
zmalloc.h anet.h zipmap.h ziplist.h intset.h version.h
db.o: db.c redis.h fmacros.h config.h ae.h sds.h dict.h adlist.h \
debug.o: debug.c redis.h fmacros.h config.h ae.h sds.h dict.h adlist.h \
zmalloc.h anet.h zipmap.h ziplist.h intset.h version.h sha1.h
dict.o: dict.c fmacros.h dict.h zmalloc.h
+diskstore.o: diskstore.c redis.h fmacros.h config.h ae.h sds.h dict.h \
+ adlist.h zmalloc.h anet.h zipmap.h ziplist.h intset.h version.h
+dscache.o: dscache.c redis.h fmacros.h config.h ae.h sds.h dict.h \
+ adlist.h zmalloc.h anet.h zipmap.h ziplist.h intset.h version.h
intset.o: intset.c intset.h zmalloc.h
lzf_c.o: lzf_c.c lzfP.h
lzf_d.o: lzf_d.c lzfP.h
zmalloc.h anet.h zipmap.h ziplist.h intset.h version.h
rdb.o: rdb.c redis.h fmacros.h config.h ae.h sds.h dict.h adlist.h \
zmalloc.h anet.h zipmap.h ziplist.h intset.h version.h lzf.h
-redis-benchmark.o: redis-benchmark.c fmacros.h ae.h anet.h sds.h adlist.h \
- zmalloc.h
+redis-benchmark.o: redis-benchmark.c fmacros.h ae.h \
+ ../deps/hiredis/hiredis.h sds.h adlist.h zmalloc.h
redis-check-aof.o: redis-check-aof.c fmacros.h config.h
redis-check-dump.o: redis-check-dump.c lzf.h
-redis-cli.o: redis-cli.c fmacros.h version.h sds.h adlist.h zmalloc.h
+redis-cli.o: redis-cli.c fmacros.h version.h ../deps/hiredis/hiredis.h \
+ sds.h zmalloc.h ../deps/linenoise/linenoise.h help.h
redis.o: redis.c redis.h fmacros.h config.h ae.h sds.h dict.h adlist.h \
zmalloc.h anet.h zipmap.h ziplist.h intset.h version.h
release.o: release.c release.h
sha1.o: sha1.c sha1.h
sort.o: sort.c redis.h fmacros.h config.h ae.h sds.h dict.h adlist.h \
zmalloc.h anet.h zipmap.h ziplist.h intset.h version.h pqsort.h
-syncio.o: syncio.c
+syncio.o: syncio.c redis.h fmacros.h config.h ae.h sds.h dict.h adlist.h \
+ zmalloc.h anet.h zipmap.h ziplist.h intset.h version.h
t_hash.o: t_hash.c redis.h fmacros.h config.h ae.h sds.h dict.h adlist.h \
zmalloc.h anet.h zipmap.h ziplist.h intset.h version.h
t_list.o: t_list.c redis.h fmacros.h config.h ae.h sds.h dict.h adlist.h \
zmalloc.h anet.h zipmap.h ziplist.h intset.h version.h
util.o: util.c redis.h fmacros.h config.h ae.h sds.h dict.h adlist.h \
zmalloc.h anet.h zipmap.h ziplist.h intset.h version.h
-vm.o: vm.c redis.h fmacros.h config.h ae.h sds.h dict.h adlist.h \
- zmalloc.h anet.h zipmap.h ziplist.h intset.h version.h
ziplist.o: ziplist.c zmalloc.h ziplist.h
zipmap.o: zipmap.c zmalloc.h
zmalloc.o: zmalloc.c config.h
rm -rf $(PRGNAME) $(BENCHPRGNAME) $(CLIPRGNAME) $(CHECKDUMPPRGNAME) $(CHECKAOFPRGNAME) *.o *.gcda *.gcno *.gcov
dep:
- $(CC) -MM *.c
+ $(CC) -MM *.c -I ../deps/hiredis -I ../deps/linenoise
test: redis-server
(cd ..; tclsh8.5 tests/test_helper.tcl --tags "${TAGS}" --file "${FILE}")
char buf[128];
sds argsds;
struct redisCommand *cmd;
- int force_swapout;
/* Serve the clients from time to time */
if (!(loops++ % 1000)) {
/* Clean up, ready for the next command */
for (j = 0; j < argc; j++) decrRefCount(argv[j]);
zfree(argv);
-
- /* Handle swapping while loading big datasets when VM is on */
- force_swapout = 0;
- if ((zmalloc_used_memory() - server.vm_max_memory) > 1024*1024*32)
- force_swapout = 1;
-
- if (server.vm_enabled && force_swapout) {
- while (zmalloc_used_memory() > server.vm_max_memory) {
- if (vmSwapOneObjectBlocking() == REDIS_ERR) break;
- }
- }
}
/* This point can only be reached when EOF is reached without errors.
sds keystr = dictGetEntryKey(de);
robj key, *o;
time_t expiretime;
- int swapped;
keystr = dictGetEntryKey(de);
o = dictGetEntryVal(de);
initStaticStringObject(key,keystr);
- /* If the value for this key is swapped, load a preview in memory.
- * We use a "swapped" flag to remember if we need to free the
- * value object instead to just increment the ref count anyway
- * in order to avoid copy-on-write of pages if we are forked() */
- if (!server.vm_enabled || o->storage == REDIS_VM_MEMORY ||
- o->storage == REDIS_VM_SWAPPING) {
- swapped = 0;
- } else {
- o = vmPreviewObject(o);
- swapped = 1;
- }
+
expiretime = getExpire(db,&key);
/* Save the key and associated value */
if (fwriteBulkObject(fp,&key) == 0) goto werr;
if (fwriteBulkLongLong(fp,expiretime) == 0) goto werr;
}
- if (swapped) decrRefCount(o);
}
dictReleaseIterator(di);
}
pid_t childpid;
if (server.bgrewritechildpid != -1) return REDIS_ERR;
- if (server.vm_enabled) waitEmptyIOJobsQueue();
+ redisAssert(server.ds_enabled == 0);
if ((childpid = fork()) == 0) {
/* Child */
char tmpfile[256];
- if (server.vm_enabled) vmReopenSwapFile();
if (server.ipfd > 0) close(server.ipfd);
if (server.sofd > 0) close(server.sofd);
snprintf(tmpfile,256,"temp-rewriteaof-bg-%d.aof", (int) getpid());
} else if (!strcasecmp(argv[0],"dbfilename") && argc == 2) {
zfree(server.dbfilename);
server.dbfilename = zstrdup(argv[1]);
- } else if (!strcasecmp(argv[0],"vm-enabled") && argc == 2) {
- if ((server.vm_enabled = yesnotoi(argv[1])) == -1) {
+ } else if (!strcasecmp(argv[0],"diskstore-enabled") && argc == 2) {
+ if ((server.ds_enabled = yesnotoi(argv[1])) == -1) {
err = "argument must be 'yes' or 'no'"; goto loaderr;
}
- } else if (!strcasecmp(argv[0],"vm-swap-file") && argc == 2) {
- zfree(server.vm_swap_file);
- server.vm_swap_file = zstrdup(argv[1]);
- } else if (!strcasecmp(argv[0],"vm-max-memory") && argc == 2) {
- server.vm_max_memory = memtoll(argv[1],NULL);
- } else if (!strcasecmp(argv[0],"vm-page-size") && argc == 2) {
- server.vm_page_size = memtoll(argv[1], NULL);
- } else if (!strcasecmp(argv[0],"vm-pages") && argc == 2) {
- server.vm_pages = memtoll(argv[1], NULL);
- } else if (!strcasecmp(argv[0],"vm-max-threads") && argc == 2) {
- server.vm_max_threads = strtoll(argv[1], NULL, 10);
+ } else if (!strcasecmp(argv[0],"diskstore-path") && argc == 2) {
+ sdsfree(server.ds_path);
+ server.ds_path = sdsnew(argv[1]);
+ } else if (!strcasecmp(argv[0],"cache-max-memory") && argc == 2) {
+ server.cache_max_memory = memtoll(argv[1],NULL);
+ } else if (!strcasecmp(argv[0],"cache-flush-delay") && argc == 2) {
+ server.cache_flush_delay = atoi(argv[1]);
+ if (server.cache_flush_delay < 0) server.cache_flush_delay = 0;
} else if (!strcasecmp(argv[0],"hash-max-zipmap-entries") && argc == 2) {
server.hash_max_zipmap_entries = memtoll(argv[1], NULL);
} else if (!strcasecmp(argv[0],"hash-max-zipmap-value") && argc == 2) {
* C-level DB API
*----------------------------------------------------------------------------*/
+/* Important notes on lookup and disk store.
+ *
+ * When disk store is enabled on lookup we can have different cases.
+ *
+ * a) The key is in memory:
+ * - If the key is not in IO_SAVEINPROG state we can access it.
+ * As if it's just IO_SAVE this means we have the key in the IO queue
+ * but can't be accessed by the IO thread (it requires to be
+ * translated into an IO Job by the cache cron function.)
+ * - If the key is in IO_SAVEINPROG we can't touch the key and have
+ * to blocking wait completion of operations.
+ * b) The key is not in memory:
+ * - If it's marked as non existing on disk as well (negative cache)
+ * we don't need to perform the disk access.
+ * - if the key MAY EXIST, but is not in memory, and it is marked as IO_SAVE
+ * then the key can only be a deleted one. As IO_SAVE keys are never
+ * evicted (dirty state), so the only possibility is that key was deleted.
+ * - if the key MAY EXIST we need to blocking load it.
+ * We check that the key is not in IO_SAVEINPROG state before accessing
+ * the disk object. If it is in this state, we wait.
+ */
+
+void lookupWaitBusyKey(redisDb *db, robj *key) {
+ /* FIXME: wait just for this key, not everything */
+ waitEmptyIOJobsQueue();
+ processAllPendingIOJobs();
+ redisAssert((cacheScheduleIOGetFlags(db,key) & REDIS_IO_SAVEINPROG) == 0);
+}
+
robj *lookupKey(redisDb *db, robj *key) {
dictEntry *de = dictFind(db->dict,key->ptr);
if (de) {
if (server.bgsavechildpid == -1 && server.bgrewritechildpid == -1)
val->lru = server.lruclock;
- if (server.vm_enabled) {
- if (val->storage == REDIS_VM_MEMORY ||
- val->storage == REDIS_VM_SWAPPING)
- {
- /* If we were swapping the object out, cancel the operation */
- if (val->storage == REDIS_VM_SWAPPING)
- vmCancelThreadedIOJob(val);
- } else {
- int notify = (val->storage == REDIS_VM_LOADING);
-
- /* Our value was swapped on disk. Bring it at home. */
- redisAssert(val->type == REDIS_VMPOINTER);
- val = vmLoadObject(val);
- dictGetEntryVal(de) = val;
-
- /* Clients blocked by the VM subsystem may be waiting for
- * this key... */
- if (notify) handleClientsBlockedOnSwappedKey(db,key);
- }
+ if (server.ds_enabled &&
+ cacheScheduleIOGetFlags(db,key) & REDIS_IO_SAVEINPROG)
+ {
+ /* Need to wait for the key to get unbusy */
+ redisLog(REDIS_DEBUG,"Lookup found a key in SAVEINPROG state. Waiting. (Key was in the cache)");
+ lookupWaitBusyKey(db,key);
}
server.stat_keyspace_hits++;
return val;
} else {
+ time_t expire;
+ robj *val;
+
+ /* Key not found in the in memory hash table, but if disk store is
+ * enabled we may have this key on disk. If so load it in memory
+ * in a blocking way. */
+ if (server.ds_enabled && cacheKeyMayExist(db,key)) {
+ long flags = cacheScheduleIOGetFlags(db,key);
+
+ /* They key is not in cache, but it has a SAVE op in queue?
+ * The only possibility is that the key was deleted, since
+ * dirty keys are not evicted. */
+ if (flags & REDIS_IO_SAVE) {
+ server.stat_keyspace_misses++;
+ return NULL;
+ }
+
+ /* At this point we need to blocking load the key in memory.
+ * The first thing we do is waiting here if the key is busy. */
+ if (flags & REDIS_IO_SAVEINPROG) {
+ redisLog(REDIS_DEBUG,"Lookup found a key in SAVEINPROG state. Waiting (while force loading).");
+ lookupWaitBusyKey(db,key);
+ }
+
+ redisLog(REDIS_DEBUG,"Force loading key %s via lookup", key->ptr);
+ val = dsGet(db,key,&expire);
+ if (val) {
+ int retval = dbAdd(db,key,val);
+ redisAssert(retval == REDIS_OK);
+ if (expire != -1) setExpire(db,key,expire);
+ server.stat_keyspace_hits++;
+ return val;
+ } else {
+ cacheSetKeyDoesNotExist(db,key);
+ }
+ }
server.stat_keyspace_misses++;
return NULL;
}
} else {
sds copy = sdsdup(key->ptr);
dictAdd(db->dict, copy, val);
+ if (server.ds_enabled) cacheSetKeyMayExist(db,key);
return REDIS_OK;
}
}
*
* On update (key already existed) 0 is returned. Otherwise 1. */
int dbReplace(redisDb *db, robj *key, robj *val) {
- if (dictFind(db->dict,key->ptr) == NULL) {
+ robj *oldval;
+ int retval;
+
+ if ((oldval = dictFetchValue(db->dict,key->ptr)) == NULL) {
sds copy = sdsdup(key->ptr);
dictAdd(db->dict, copy, val);
- return 1;
+ retval = 1;
} else {
dictReplace(db->dict, key->ptr, val);
- return 0;
+ retval = 0;
}
+ if (server.ds_enabled) cacheSetKeyMayExist(db,key);
+ return retval;
}
int dbExists(redisDb *db, robj *key) {
/* Delete a key, value, and associated expiration entry if any, from the DB */
int dbDelete(redisDb *db, robj *key) {
- /* If VM is enabled make sure to awake waiting clients for this key:
- * deleting the key will kill the I/O thread bringing the key from swap
- * to memory, so the client will never be notified and unblocked if we
- * don't do it now. */
- if (server.vm_enabled) handleClientsBlockedOnSwappedKey(db,key);
+ /* If diskstore is enabled make sure to awake waiting clients for this key
+ * as it is not really useful to wait for a key already deleted to be
+ * loaded from disk. */
+ if (server.ds_enabled) {
+ handleClientsBlockedOnSwappedKey(db,key);
+ cacheSetKeyDoesNotExist(db,key);
+ }
+
/* Deleting an entry from the expires dict will not free the sds of
* the key, because it is shared with the main dictionary. */
if (dictSize(db->expires) > 0) dictDelete(db->expires,key->ptr);
return REDIS_OK;
}
+/*-----------------------------------------------------------------------------
+ * Hooks for key space changes.
+ *
+ * Every time a key in the database is modified the function
+ * signalModifiedKey() is called.
+ *
+ * Every time a DB is flushed the function signalFlushDb() is called.
+ *----------------------------------------------------------------------------*/
+
+void signalModifiedKey(redisDb *db, robj *key) {
+ touchWatchedKey(db,key);
+ if (server.ds_enabled)
+ cacheScheduleIO(db,key,REDIS_IO_SAVE);
+}
+
+void signalFlushedDb(int dbid) {
+ touchWatchedKeysOnFlush(dbid);
+}
+
/*-----------------------------------------------------------------------------
* Type agnostic commands operating on the key space
*----------------------------------------------------------------------------*/
void flushdbCommand(redisClient *c) {
server.dirty += dictSize(c->db->dict);
- touchWatchedKeysOnFlush(c->db->id);
+ signalFlushedDb(c->db->id);
dictEmpty(c->db->dict);
dictEmpty(c->db->expires);
+ if (server.ds_enabled) dsFlushDb(c->db->id);
addReply(c,shared.ok);
}
void flushallCommand(redisClient *c) {
- touchWatchedKeysOnFlush(-1);
+ signalFlushedDb(-1);
server.dirty += emptyDb();
addReply(c,shared.ok);
if (server.bgsavechildpid != -1) {
kill(server.bgsavechildpid,SIGKILL);
rdbRemoveTempFile(server.bgsavechildpid);
}
- rdbSave(server.dbfilename);
+ if (server.ds_enabled)
+ dsFlushDb(-1);
+ else
+ rdbSave(server.dbfilename);
server.dirty++;
}
int deleted = 0, j;
for (j = 1; j < c->argc; j++) {
+ if (server.ds_enabled) {
+ lookupKeyRead(c->db,c->argv[j]);
+ /* FIXME: this can be optimized a lot, no real need to load
+ * a possibly huge value. */
+ }
if (dbDelete(c->db,c->argv[j])) {
- touchWatchedKey(c->db,c->argv[j]);
+ signalModifiedKey(c->db,c->argv[j]);
server.dirty++;
deleted++;
+ } else if (server.ds_enabled) {
+ if (cacheKeyMayExist(c->db,c->argv[j]) &&
+ dsExists(c->db,c->argv[j]))
+ {
+ cacheScheduleIO(c->db,c->argv[j],REDIS_IO_SAVE);
+ deleted = 1;
+ }
}
}
addReplyLongLong(c,deleted);
dbReplace(c->db,c->argv[2],o);
}
dbDelete(c->db,c->argv[1]);
- touchWatchedKey(c->db,c->argv[1]);
- touchWatchedKey(c->db,c->argv[2]);
+ signalModifiedKey(c->db,c->argv[1]);
+ signalModifiedKey(c->db,c->argv[2]);
server.dirty++;
addReply(c,nx ? shared.cone : shared.ok);
}
if (seconds <= 0) {
if (dbDelete(c->db,key)) server.dirty++;
addReply(c, shared.cone);
- touchWatchedKey(c->db,key);
+ signalModifiedKey(c->db,key);
return;
} else {
time_t when = time(NULL)+seconds;
setExpire(c->db,key,when);
addReply(c,shared.cone);
- touchWatchedKey(c->db,key);
+ signalModifiedKey(c->db,key);
server.dirty++;
return;
}
} else if (!strcasecmp(c->argv[1]->ptr,"object") && c->argc == 3) {
dictEntry *de = dictFind(c->db->dict,c->argv[2]->ptr);
robj *val;
+ char *strenc;
if (!de) {
addReply(c,shared.nokeyerr);
return;
}
val = dictGetEntryVal(de);
- if (!server.vm_enabled || (val->storage == REDIS_VM_MEMORY ||
- val->storage == REDIS_VM_SWAPPING)) {
- char *strenc;
-
- strenc = strEncoding(val->encoding);
- addReplyStatusFormat(c,
- "Value at:%p refcount:%d "
- "encoding:%s serializedlength:%lld "
- "lru:%d lru_seconds_idle:%lu",
- (void*)val, val->refcount,
- strenc, (long long) rdbSavedObjectLen(val),
- val->lru, estimateObjectIdleTime(val));
- } else {
- vmpointer *vp = (vmpointer*) val;
- addReplyStatusFormat(c,
- "Value swapped at: page %llu "
- "using %llu pages",
- (unsigned long long) vp->page,
- (unsigned long long) vp->usedpages);
- }
- } else if (!strcasecmp(c->argv[1]->ptr,"swapin") && c->argc == 3) {
- lookupKeyRead(c->db,c->argv[2]);
- addReply(c,shared.ok);
- } else if (!strcasecmp(c->argv[1]->ptr,"swapout") && c->argc == 3) {
- dictEntry *de = dictFind(c->db->dict,c->argv[2]->ptr);
- robj *val;
- vmpointer *vp;
-
- if (!server.vm_enabled) {
- addReplyError(c,"Virtual Memory is disabled");
- return;
- }
- if (!de) {
- addReply(c,shared.nokeyerr);
- return;
- }
- val = dictGetEntryVal(de);
- /* Swap it */
- if (val->storage != REDIS_VM_MEMORY) {
- addReplyError(c,"This key is not in memory");
- } else if (val->refcount != 1) {
- addReplyError(c,"Object is shared");
- } else if ((vp = vmSwapObjectBlocking(val)) != NULL) {
- dictGetEntryVal(de) = vp;
- addReply(c,shared.ok);
- } else {
- addReply(c,shared.err);
- }
+ strenc = strEncoding(val->encoding);
+
+ addReplyStatusFormat(c,
+ "Value at:%p refcount:%d "
+ "encoding:%s serializedlength:%lld "
+ "lru:%d lru_seconds_idle:%lu",
+ (void*)val, val->refcount,
+ strenc, (long long) rdbSavedObjectLen(val),
+ val->lru, estimateObjectIdleTime(val));
} else if (!strcasecmp(c->argv[1]->ptr,"populate") && c->argc == 3) {
long keys, j;
robj *key, *val;
}
void _redisPanic(char *msg, char *file, int line) {
+ redisLog(REDIS_WARNING,"------------------------------------------------");
redisLog(REDIS_WARNING,"!!! Software Failure. Press left mouse button to continue");
redisLog(REDIS_WARNING,"Guru Meditation: %s #%s:%d",msg,file,line);
#ifdef HAVE_BACKTRACE
redisLog(REDIS_WARNING,"(forcing SIGSEGV in order to print the stack trace)");
+ redisLog(REDIS_WARNING,"------------------------------------------------");
*((char*)-1) = 'x';
#endif
}
--- /dev/null
+/* diskstore.c implements a very simple disk backed key-value store used
+ * by Redis for the "disk" backend. This implementation uses the filesystem
+ * to store key/value pairs. Every file represents a given key.
+ *
+ * The key path is calculated using the SHA1 of the key name. For instance
+ * the key "foo" is stored as a file name called:
+ *
+ * /0b/ee/0beec7b5ea3f0fdbc95d0dd47f3c5bc275da8a33
+ *
+ * The couples of characters from the hex output of SHA1 are also used
+ * to locate two two levels of directories to store the file (as most
+ * filesystems are not able to handle too many files in a single dir).
+ *
+ * In the end there are 65536 final directories (256 directories inside
+ * every 256 top level directories), so that with 1 billion of files every
+ * directory will contain in the average 15258 entires, that is ok with
+ * most filesystems implementation.
+ *
+ * Note that since Redis supports multiple databases, the actual key name
+ * is:
+ *
+ * /0b/ee/<dbid>_0beec7b5ea3f0fdbc95d0dd47f3c5bc275da8a33
+ *
+ * so for instance if the key is inside DB 0:
+ *
+ * /0b/ee/0_0beec7b5ea3f0fdbc95d0dd47f3c5bc275da8a33
+ *
+ * The actaul implementation of this disk store is highly dependant to the
+ * filesystem implementation itself. This implementation may be replaced by
+ * a B+TREE implementation in future implementations.
+ *
+ * Data ok every key is serialized using the same format used for .rdb
+ * serialization. Everything is serialized on every entry: key name,
+ * ttl information in case of keys with an associated expire time, and the
+ * serialized value itself.
+ *
+ * Because the format is the same of the .rdb files it is trivial to create
+ * an .rdb file starting from this format just by mean of scanning the
+ * directories and concatenating entries, with the sole addition of an
+ * .rdb header at the start and the end-of-db opcode at the end.
+ *
+ * -------------------------------------------------------------------------
+ *
+ * Copyright (c) 2010-2011, Salvatore Sanfilippo <antirez at gmail dot com>
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are met:
+ *
+ * * Redistributions of source code must retain the above copyright notice,
+ * this list of conditions and the following disclaimer.
+ * * Redistributions in binary form must reproduce the above copyright
+ * notice, this list of conditions and the following disclaimer in the
+ * documentation and/or other materials provided with the distribution.
+ * * Neither the name of Redis nor the names of its contributors may be used
+ * to endorse or promote products derived from this software without
+ * specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+ * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+ * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
+ * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+ * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+ * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+ * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+ * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+ * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
+ * POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include "redis.h"
+#include "sha1.h"
+
+#include <fcntl.h>
+#include <sys/stat.h>
+#include <dirent.h>
+
+int create256dir(char *prefix) {
+ char buf[1024];
+ int j;
+
+ for (j = 0; j < 256; j++) {
+ snprintf(buf,sizeof(buf),"%s%02x",prefix,j);
+ if (mkdir(buf,0755) == -1) {
+ redisLog(REDIS_WARNING,"Error creating dir %s for diskstore: %s",
+ buf,strerror(errno));
+ return REDIS_ERR;
+ }
+ }
+ return REDIS_OK;
+}
+
+int dsOpen(void) {
+ struct stat sb;
+ int retval, j;
+ char *path = server.ds_path;
+ char buf[1024];
+
+ if ((retval = stat(path,&sb) == -1) && errno != ENOENT) {
+ redisLog(REDIS_WARNING, "Error opening disk store at %s: %s",
+ path, strerror(errno));
+ return REDIS_ERR;
+ }
+
+ /* Directory already in place. Assume everything is ok. */
+ if (retval == 0 && S_ISDIR(sb.st_mode)) {
+ redisLog(REDIS_NOTICE,"Disk store %s exists", path);
+ return REDIS_OK;
+ }
+
+ /* File exists but it's not a directory */
+ if (retval == 0 && !S_ISDIR(sb.st_mode)) {
+ redisLog(REDIS_WARNING,"Disk store at %s is not a directory", path);
+ return REDIS_ERR;
+ }
+
+ /* New disk store, create the directory structure now, as creating
+ * them in a lazy way is not a good idea, after very few insertions
+ * we'll need most of the 65536 directories anyway. */
+ redisLog(REDIS_NOTICE,"Disk store %s does not exist: creating", path);
+ if (mkdir(path,0755) == -1) {
+ redisLog(REDIS_WARNING,"Disk store init failed creating dir %s: %s",
+ path, strerror(errno));
+ return REDIS_ERR;
+ }
+ /* Create the top level 256 directories */
+ snprintf(buf,sizeof(buf),"%s/",path);
+ if (create256dir(buf) == REDIS_ERR) return REDIS_ERR;
+
+ /* For every 256 top level dir, create 256 nested dirs */
+ for (j = 0; j < 256; j++) {
+ snprintf(buf,sizeof(buf),"%s/%02x/",path,j);
+ if (create256dir(buf) == REDIS_ERR) return REDIS_ERR;
+ }
+ return REDIS_OK;
+}
+
+int dsClose(void) {
+ return REDIS_OK;
+}
+
+/* Convert key into full path for this object. Dirty but hopefully
+ * is fast enough. Returns the length of the returned path. */
+int dsKeyToPath(redisDb *db, char *buf, robj *key) {
+ SHA1_CTX ctx;
+ unsigned char hash[20];
+ char hex[40], digits[] = "0123456789abcdef";
+ int j, l;
+ char *origbuf = buf;
+
+ SHA1Init(&ctx);
+ SHA1Update(&ctx,key->ptr,sdslen(key->ptr));
+ SHA1Final(hash,&ctx);
+
+ /* Convert the hash into hex format */
+ for (j = 0; j < 20; j++) {
+ hex[j*2] = digits[(hash[j]&0xF0)>>4];
+ hex[(j*2)+1] = digits[hash[j]&0x0F];
+ }
+
+ /* Create the object path. Start with server.ds_path that's the root dir */
+ l = sdslen(server.ds_path);
+ memcpy(buf,server.ds_path,l);
+ buf += l;
+ *buf++ = '/';
+
+ /* Then add xx/yy/ that is the two level directories */
+ buf[0] = hex[0];
+ buf[1] = hex[1];
+ buf[2] = '/';
+ buf[3] = hex[2];
+ buf[4] = hex[3];
+ buf[5] = '/';
+ buf += 6;
+
+ /* Add the database number followed by _ and finall the SHA1 hex */
+ l = ll2string(buf,64,db->id);
+ buf += l;
+ buf[0] = '_';
+ memcpy(buf+1,hex,40);
+ buf[41] = '\0';
+ return (buf-origbuf)+41;
+}
+
+int dsSet(redisDb *db, robj *key, robj *val) {
+ char buf[1024], buf2[1024];
+ FILE *fp;
+ int retval, len;
+
+ len = dsKeyToPath(db,buf,key);
+ memcpy(buf2,buf,len);
+ snprintf(buf2+len,sizeof(buf2)-len,"_%ld_%ld",(long)time(NULL),(long)val);
+ while ((fp = fopen(buf2,"w")) == NULL) {
+ if (errno == ENOSPC) {
+ redisLog(REDIS_WARNING,"Diskstore: No space left on device. Please make room and wait 30 seconds for Redis to continue.");
+ sleep(30);
+ } else {
+ redisLog(REDIS_WARNING,"diskstore error opening %s: %s",
+ buf2, strerror(errno));
+ redisPanic("Unrecoverable diskstore error. Exiting.");
+ }
+ }
+ if ((retval = rdbSaveKeyValuePair(fp,db,key,val,time(NULL))) == -1)
+ return REDIS_ERR;
+ fclose(fp);
+ if (retval == 0) {
+ /* Expired key. Unlink failing not critical */
+ unlink(buf);
+ unlink(buf2);
+ } else {
+ /* Use rename for atomic updadte of value */
+ if (rename(buf2,buf) == -1) {
+ redisLog(REDIS_WARNING,"rename(2) returned an error: %s",
+ strerror(errno));
+ redisPanic("Unrecoverable diskstore error. Exiting.");
+ }
+ }
+ return REDIS_OK;
+}
+
+robj *dsGet(redisDb *db, robj *key, time_t *expire) {
+ char buf[1024];
+ int type;
+ time_t expiretime = -1; /* -1 means: no expire */
+ robj *dskey; /* Key as loaded from disk. */
+ robj *val;
+ FILE *fp;
+
+ dsKeyToPath(db,buf,key);
+ fp = fopen(buf,"r");
+ if (fp == NULL && errno == ENOENT) return NULL; /* No such key */
+ if (fp == NULL) {
+ redisLog(REDIS_WARNING,"Disk store failed opening %s: %s",
+ buf, strerror(errno));
+ goto readerr;
+ }
+
+ if ((type = rdbLoadType(fp)) == -1) goto readerr;
+ if (type == REDIS_EXPIRETIME) {
+ if ((expiretime = rdbLoadTime(fp)) == -1) goto readerr;
+ /* We read the time so we need to read the object type again */
+ if ((type = rdbLoadType(fp)) == -1) goto readerr;
+ }
+ /* Read key */
+ if ((dskey = rdbLoadStringObject(fp)) == NULL) goto readerr;
+ /* Read value */
+ if ((val = rdbLoadObject(type,fp)) == NULL) goto readerr;
+ fclose(fp);
+
+ /* The key we asked, and the key returned, must be the same */
+ redisAssert(equalStringObjects(key,dskey));
+
+ /* Check if the key already expired */
+ decrRefCount(dskey);
+ if (expiretime != -1 && expiretime < time(NULL)) {
+ decrRefCount(val);
+ unlink(buf); /* This failing is non critical here */
+ return NULL;
+ }
+
+ /* Everything ok... */
+ *expire = expiretime;
+ return val;
+
+readerr:
+ redisLog(REDIS_WARNING,"Read error reading reading %s. Corrupted key?",
+ buf);
+ redisPanic("Unrecoverable error reading from disk store");
+ return NULL; /* unreached */
+}
+
+int dsDel(redisDb *db, robj *key) {
+ char buf[1024];
+
+ dsKeyToPath(db,buf,key);
+ if (unlink(buf) == -1) {
+ if (errno == ENOENT) {
+ return REDIS_ERR;
+ } else {
+ redisLog(REDIS_WARNING,"Disk store can't remove %s: %s",
+ buf, strerror(errno));
+ redisPanic("Unrecoverable Disk store errore. Existing.");
+ return REDIS_ERR; /* unreached */
+ }
+ } else {
+ return REDIS_OK;
+ }
+}
+
+int dsExists(redisDb *db, robj *key) {
+ char buf[1024];
+
+ dsKeyToPath(db,buf,key);
+ return access(buf,R_OK) == 0;
+}
+
+void dsFlushOneDir(char *path, int dbid) {
+ DIR *dir;
+ struct dirent *dp, de;
+
+ dir = opendir(path);
+ if (dir == NULL) {
+ redisLog(REDIS_WARNING,"Disk store can't open dir %s: %s",
+ path, strerror(errno));
+ redisPanic("Unrecoverable Disk store errore. Existing.");
+ }
+ while(1) {
+ char buf[1024];
+
+ readdir_r(dir,&de,&dp);
+ if (dp == NULL) break;
+ if (dp->d_name[0] == '.') continue;
+
+ /* Check if we need to remove this entry accordingly to the
+ * DB number */
+ if (dbid != -1) {
+ char id[64];
+ char *p = strchr(dp->d_name,'_');
+ int len = (p - dp->d_name);
+
+ redisAssert(p != NULL && len < 64);
+ memcpy(id,dp->d_name,len);
+ id[len] = '\0';
+ if (atoi(id) != dbid) continue; /* skip this file */
+ }
+
+ /* Finally unlink the file */
+ snprintf(buf,1024,"%s/%s",path,dp->d_name);
+ if (unlink(buf) == -1) {
+ redisLog(REDIS_WARNING,
+ "Can't unlink %s: %s", buf, strerror(errno));
+ redisPanic("Unrecoverable Disk store errore. Existing.");
+ }
+ }
+ closedir(dir);
+}
+
+void dsFlushDb(int dbid) {
+ char buf[1024];
+ int j, i;
+
+ redisLog(REDIS_NOTICE,"Flushing diskstore DB (%d)",dbid);
+ for (j = 0; j < 256; j++) {
+ for (i = 0; i < 256; i++) {
+ snprintf(buf,1024,"%s/%02x/%02x",server.ds_path,j,i);
+ dsFlushOneDir(buf,dbid);
+ }
+ }
+}
--- /dev/null
+#include "redis.h"
+
+#include <fcntl.h>
+#include <pthread.h>
+#include <math.h>
+#include <signal.h>
+
+/* dscache.c - Disk store cache for disk store backend.
+ *
+ * When Redis is configured for using disk as backend instead of memory, the
+ * memory is used as a cache, so that recently accessed keys are taken in
+ * memory for fast read and write operations.
+ *
+ * Modified keys are marked to be flushed on disk, and will be flushed
+ * as long as the maxium configured flush time elapsed.
+ *
+ * This file implements the whole caching subsystem and contains further
+ * documentation. */
+
+/* TODO:
+ *
+ * WARNING: most of the following todo items and design issues are no
+ * longer relevant with the new design. Here as a checklist to see if
+ * some old ideas still apply.
+ *
+ * - What happens when an object is destroyed?
+ *
+ * If the object is destroyed since semantically it was deleted or
+ * replaced with something new, we don't care if there was a SAVE
+ * job pending for it. Anyway when the IO JOb will be created we'll get
+ * the pointer of the current value.
+ *
+ * If the object is already a REDIS_IO_SAVEINPROG object, then it is
+ * impossible that we get a decrRefCount() that will reach refcount of zero
+ * since the object is both in the dataset and in the io job entry.
+ *
+ * - What happens with MULTI/EXEC?
+ *
+ * Good question. Without some kind of versioning with a global counter
+ * it is not possible to have trasactions on disk, but they are still
+ * useful since from the point of view of memory and client bugs it is
+ * a protection anyway. Also it's useful for WATCH.
+ *
+ * Btw there is to check what happens when WATCH gets combined to keys
+ * that gets removed from the object cache. Should be save but better
+ * to check.
+ *
+ * - Check if/why INCR will not update the LRU info for the object.
+ *
+ * - Fix/Check the following race condition: a key gets a DEL so there is
+ * a write operation scheduled against this key. Later the same key will
+ * be the argument of a GET, but the write operation was still not
+ * completed (to delete the file). If the GET will be for some reason
+ * a blocking loading (via lookup) we can load the old value on memory.
+ *
+ * This problems can be fixed with negative caching. We can use it
+ * to optimize the system, but also when a key is deleted we mark
+ * it as non existing on disk as well (in a way that this cache
+ * entry can't be evicted, setting time to 0), then we avoid looking at
+ * the disk at all if the key can't be there. When an IO Job complete
+ * a deletion, we set the time of the negative caching to a non zero
+ * value so it will be evicted later.
+ *
+ * Are there other patterns like this where we load stale data?
+ *
+ * Also, make sure that key preloading is ONLY done for keys that are
+ * not marked as cacheKeyDoesNotExist(), otherwise, again, we can load
+ * data from disk that should instead be deleted.
+ *
+ * - dsSet() should use rename(2) in order to avoid corruptions.
+ *
+ * - Don't add a LOAD if there is already a LOADINPROGRESS, or is this
+ * impossible since anyway the io_keys stuff will work as lock?
+ *
+ * - Serialize special encoded things in a raw form.
+ *
+ * - When putting IO read operations on top of the queue, do this only if
+ * the already-on-top operation is not a save or if it is a save that
+ * is scheduled for later execution. If there is a save that is ready to
+ * fire, let's insert the load operation just before the first save that
+ * is scheduled for later exection for instance.
+ *
+ * - Support MULTI/EXEC transactions via a journal file, that is played on
+ * startup to check if there is cleanup to do. This way we can implement
+ * transactions with our simple file based KV store.
+ */
+
+/* Virtual Memory is composed mainly of two subsystems:
+ * - Blocking Virutal Memory
+ * - Threaded Virtual Memory I/O
+ * The two parts are not fully decoupled, but functions are split among two
+ * different sections of the source code (delimited by comments) in order to
+ * make more clear what functionality is about the blocking VM and what about
+ * the threaded (not blocking) VM.
+ *
+ * Redis VM design:
+ *
+ * Redis VM is a blocking VM (one that blocks reading swapped values from
+ * disk into memory when a value swapped out is needed in memory) that is made
+ * unblocking by trying to examine the command argument vector in order to
+ * load in background values that will likely be needed in order to exec
+ * the command. The command is executed only once all the relevant keys
+ * are loaded into memory.
+ *
+ * This basically is almost as simple of a blocking VM, but almost as parallel
+ * as a fully non-blocking VM.
+ */
+
+void spawnIOThread(void);
+int cacheScheduleIOPushJobs(int onlyloads);
+
+/* =================== Virtual Memory - Blocking Side ====================== */
+
+void dsInit(void) {
+ int pipefds[2];
+ size_t stacksize;
+
+ zmalloc_enable_thread_safeness(); /* we need thread safe zmalloc() */
+
+ redisLog(REDIS_NOTICE,"Opening Disk Store: %s", server.ds_path);
+ /* Open Disk Store */
+ if (dsOpen() != REDIS_OK) {
+ redisLog(REDIS_WARNING,"Fatal error opening disk store. Exiting.");
+ exit(1);
+ };
+
+ /* Initialize threaded I/O for Object Cache */
+ server.io_newjobs = listCreate();
+ server.io_processing = listCreate();
+ server.io_processed = listCreate();
+ server.io_ready_clients = listCreate();
+ pthread_mutex_init(&server.io_mutex,NULL);
+ pthread_cond_init(&server.io_condvar,NULL);
+ server.io_active_threads = 0;
+ if (pipe(pipefds) == -1) {
+ redisLog(REDIS_WARNING,"Unable to intialized DS: pipe(2): %s. Exiting."
+ ,strerror(errno));
+ exit(1);
+ }
+ server.io_ready_pipe_read = pipefds[0];
+ server.io_ready_pipe_write = pipefds[1];
+ redisAssert(anetNonBlock(NULL,server.io_ready_pipe_read) != ANET_ERR);
+ /* LZF requires a lot of stack */
+ pthread_attr_init(&server.io_threads_attr);
+ pthread_attr_getstacksize(&server.io_threads_attr, &stacksize);
+
+ /* Solaris may report a stacksize of 0, let's set it to 1 otherwise
+ * multiplying it by 2 in the while loop later will not really help ;) */
+ if (!stacksize) stacksize = 1;
+
+ while (stacksize < REDIS_THREAD_STACK_SIZE) stacksize *= 2;
+ pthread_attr_setstacksize(&server.io_threads_attr, stacksize);
+ /* Listen for events in the threaded I/O pipe */
+ if (aeCreateFileEvent(server.el, server.io_ready_pipe_read, AE_READABLE,
+ vmThreadedIOCompletedJob, NULL) == AE_ERR)
+ oom("creating file event");
+
+ /* Spawn our I/O thread */
+ spawnIOThread();
+}
+
+/* Compute how good candidate the specified object is for eviction.
+ * An higher number means a better candidate. */
+double computeObjectSwappability(robj *o) {
+ /* actual age can be >= minage, but not < minage. As we use wrapping
+ * 21 bit clocks with minutes resolution for the LRU. */
+ return (double) estimateObjectIdleTime(o);
+}
+
+/* Try to free one entry from the diskstore object cache */
+int cacheFreeOneEntry(void) {
+ int j, i;
+ struct dictEntry *best = NULL;
+ double best_swappability = 0;
+ redisDb *best_db = NULL;
+ robj *val;
+ sds key;
+
+ for (j = 0; j < server.dbnum; j++) {
+ redisDb *db = server.db+j;
+ /* Why maxtries is set to 100?
+ * Because this way (usually) we'll find 1 object even if just 1% - 2%
+ * are swappable objects */
+ int maxtries = 100;
+
+ if (dictSize(db->dict) == 0) continue;
+ for (i = 0; i < 5; i++) {
+ dictEntry *de;
+ double swappability;
+ robj keyobj;
+ sds keystr;
+
+ if (maxtries) maxtries--;
+ de = dictGetRandomKey(db->dict);
+ keystr = dictGetEntryKey(de);
+ val = dictGetEntryVal(de);
+ initStaticStringObject(keyobj,keystr);
+
+ /* Don't remove objects that are currently target of a
+ * read or write operation. */
+ if (cacheScheduleIOGetFlags(db,&keyobj) != 0) {
+ if (maxtries) i--; /* don't count this try */
+ continue;
+ }
+ swappability = computeObjectSwappability(val);
+ if (!best || swappability > best_swappability) {
+ best = de;
+ best_swappability = swappability;
+ best_db = db;
+ }
+ }
+ }
+ if (best == NULL) {
+ /* Was not able to fix a single object... we should check if our
+ * IO queues have stuff in queue, and try to consume the queue
+ * otherwise we'll use an infinite amount of memory if changes to
+ * the dataset are faster than I/O */
+ if (listLength(server.cache_io_queue) > 0) {
+ cacheScheduleIOPushJobs(0);
+ waitEmptyIOJobsQueue();
+ processAllPendingIOJobs();
+ return REDIS_OK;
+ }
+ /* Nothing to free at all... */
+ return REDIS_ERR;
+ }
+ key = dictGetEntryKey(best);
+ val = dictGetEntryVal(best);
+
+ redisLog(REDIS_DEBUG,"Key selected for cache eviction: %s swappability:%f",
+ key, best_swappability);
+
+ /* Delete this key from memory */
+ {
+ robj *kobj = createStringObject(key,sdslen(key));
+ dbDelete(best_db,kobj);
+ decrRefCount(kobj);
+ }
+ return REDIS_OK;
+}
+
+/* Return true if it's safe to swap out objects in a given moment.
+ * Basically we don't want to swap objects out while there is a BGSAVE
+ * or a BGAEOREWRITE running in backgroud. */
+int dsCanTouchDiskStore(void) {
+ return (server.bgsavechildpid == -1 && server.bgrewritechildpid == -1);
+}
+
+/* ==================== Disk store negative caching ========================
+ *
+ * When disk store is enabled, we need negative caching, that is, to remember
+ * keys that are for sure *not* on the disk key-value store.
+ *
+ * This is usefuls because without negative caching cache misses will cost us
+ * a disk lookup, even if the same non existing key is accessed again and again.
+ *
+ * With negative caching we remember that the key is not on disk, so if it's
+ * not in memory and we have a negative cache entry, we don't try a disk
+ * access at all.
+ */
+
+/* Returns true if the specified key may exists on disk, that is, we don't
+ * have an entry in our negative cache for this key */
+int cacheKeyMayExist(redisDb *db, robj *key) {
+ return dictFind(db->io_negcache,key) == NULL;
+}
+
+/* Set the specified key as an entry that may possibily exist on disk, that is,
+ * remove the negative cache entry for this key if any. */
+void cacheSetKeyMayExist(redisDb *db, robj *key) {
+ dictDelete(db->io_negcache,key);
+}
+
+/* Set the specified key as non existing on disk, that is, create a negative
+ * cache entry for this key. */
+void cacheSetKeyDoesNotExist(redisDb *db, robj *key) {
+ if (dictReplace(db->io_negcache,key,(void*)time(NULL))) {
+ incrRefCount(key);
+ }
+}
+
+/* Remove one entry from negative cache using approximated LRU. */
+int negativeCacheEvictOneEntry(void) {
+ struct dictEntry *de;
+ robj *best = NULL;
+ redisDb *best_db = NULL;
+ time_t time, best_time = 0;
+ int j;
+
+ for (j = 0; j < server.dbnum; j++) {
+ redisDb *db = server.db+j;
+ int i;
+
+ if (dictSize(db->io_negcache) == 0) continue;
+ for (i = 0; i < 3; i++) {
+ de = dictGetRandomKey(db->io_negcache);
+ time = (time_t) dictGetEntryVal(de);
+
+ if (best == NULL || time < best_time) {
+ best = dictGetEntryKey(de);
+ best_db = db;
+ best_time = time;
+ }
+ }
+ }
+ if (best) {
+ dictDelete(best_db->io_negcache,best);
+ return REDIS_OK;
+ } else {
+ return REDIS_ERR;
+ }
+}
+
+/* ================== Disk store cache - Threaded I/O ====================== */
+
+void freeIOJob(iojob *j) {
+ decrRefCount(j->key);
+ /* j->val can be NULL if the job is about deleting the key from disk. */
+ if (j->val) decrRefCount(j->val);
+ zfree(j);
+}
+
+/* Every time a thread finished a Job, it writes a byte into the write side
+ * of an unix pipe in order to "awake" the main thread, and this function
+ * is called. */
+void vmThreadedIOCompletedJob(aeEventLoop *el, int fd, void *privdata,
+ int mask)
+{
+ char buf[1];
+ int retval, processed = 0, toprocess = -1;
+ REDIS_NOTUSED(el);
+ REDIS_NOTUSED(mask);
+ REDIS_NOTUSED(privdata);
+
+ /* For every byte we read in the read side of the pipe, there is one
+ * I/O job completed to process. */
+ while((retval = read(fd,buf,1)) == 1) {
+ iojob *j;
+ listNode *ln;
+
+ redisLog(REDIS_DEBUG,"Processing I/O completed job");
+
+ /* Get the processed element (the oldest one) */
+ lockThreadedIO();
+ redisAssert(listLength(server.io_processed) != 0);
+ if (toprocess == -1) {
+ toprocess = (listLength(server.io_processed)*REDIS_MAX_COMPLETED_JOBS_PROCESSED)/100;
+ if (toprocess <= 0) toprocess = 1;
+ }
+ ln = listFirst(server.io_processed);
+ j = ln->value;
+ listDelNode(server.io_processed,ln);
+ unlockThreadedIO();
+
+ /* Post process it in the main thread, as there are things we
+ * can do just here to avoid race conditions and/or invasive locks */
+ redisLog(REDIS_DEBUG,"COMPLETED Job type %s, key: %s",
+ (j->type == REDIS_IOJOB_LOAD) ? "load" : "save",
+ (unsigned char*)j->key->ptr);
+ if (j->type == REDIS_IOJOB_LOAD) {
+ /* Create the key-value pair in the in-memory database */
+ if (j->val != NULL) {
+ /* Note: it's possible that the key is already in memory
+ * due to a blocking load operation. */
+ if (dbAdd(j->db,j->key,j->val) == REDIS_OK) {
+ incrRefCount(j->val);
+ if (j->expire != -1) setExpire(j->db,j->key,j->expire);
+ }
+ } else {
+ /* Key not found on disk. If it is also not in memory
+ * as a cached object, nor there is a job writing it
+ * in background, we are sure the key does not exist
+ * currently.
+ *
+ * So we set a negative cache entry avoiding that the
+ * resumed client will block load what does not exist... */
+ if (dictFind(j->db->dict,j->key->ptr) == NULL &&
+ (cacheScheduleIOGetFlags(j->db,j->key) &
+ (REDIS_IO_SAVE|REDIS_IO_SAVEINPROG)) == 0)
+ {
+ cacheSetKeyDoesNotExist(j->db,j->key);
+ }
+ }
+ cacheScheduleIODelFlag(j->db,j->key,REDIS_IO_LOADINPROG);
+ handleClientsBlockedOnSwappedKey(j->db,j->key);
+ freeIOJob(j);
+ } else if (j->type == REDIS_IOJOB_SAVE) {
+ cacheScheduleIODelFlag(j->db,j->key,REDIS_IO_SAVEINPROG);
+ freeIOJob(j);
+ }
+ processed++;
+ if (processed == toprocess) return;
+ }
+ if (retval < 0 && errno != EAGAIN) {
+ redisLog(REDIS_WARNING,
+ "WARNING: read(2) error in vmThreadedIOCompletedJob() %s",
+ strerror(errno));
+ }
+}
+
+void lockThreadedIO(void) {
+ pthread_mutex_lock(&server.io_mutex);
+}
+
+void unlockThreadedIO(void) {
+ pthread_mutex_unlock(&server.io_mutex);
+}
+
+void *IOThreadEntryPoint(void *arg) {
+ iojob *j;
+ listNode *ln;
+ REDIS_NOTUSED(arg);
+
+ pthread_detach(pthread_self());
+ lockThreadedIO();
+ while(1) {
+ /* Get a new job to process */
+ if (listLength(server.io_newjobs) == 0) {
+ /* Wait for more work to do */
+ pthread_cond_wait(&server.io_condvar,&server.io_mutex);
+ continue;
+ }
+ redisLog(REDIS_DEBUG,"%ld IO jobs to process",
+ listLength(server.io_newjobs));
+ ln = listFirst(server.io_newjobs);
+ j = ln->value;
+ listDelNode(server.io_newjobs,ln);
+ /* Add the job in the processing queue */
+ listAddNodeTail(server.io_processing,j);
+ ln = listLast(server.io_processing); /* We use ln later to remove it */
+ unlockThreadedIO();
+
+ redisLog(REDIS_DEBUG,"Thread %ld: new job type %s: %p about key '%s'",
+ (long) pthread_self(),
+ (j->type == REDIS_IOJOB_LOAD) ? "load" : "save",
+ (void*)j, (char*)j->key->ptr);
+
+ /* Process the Job */
+ if (j->type == REDIS_IOJOB_LOAD) {
+ time_t expire;
+
+ j->val = dsGet(j->db,j->key,&expire);
+ if (j->val) j->expire = expire;
+ } else if (j->type == REDIS_IOJOB_SAVE) {
+ if (j->val) {
+ dsSet(j->db,j->key,j->val);
+ } else {
+ dsDel(j->db,j->key);
+ }
+ }
+
+ /* Done: insert the job into the processed queue */
+ redisLog(REDIS_DEBUG,"Thread %ld completed the job: %p (key %s)",
+ (long) pthread_self(), (void*)j, (char*)j->key->ptr);
+
+ lockThreadedIO();
+ listDelNode(server.io_processing,ln);
+ listAddNodeTail(server.io_processed,j);
+
+ /* Signal the main thread there is new stuff to process */
+ redisAssert(write(server.io_ready_pipe_write,"x",1) == 1);
+ }
+ /* never reached, but that's the full pattern... */
+ unlockThreadedIO();
+ return NULL;
+}
+
+void spawnIOThread(void) {
+ pthread_t thread;
+ sigset_t mask, omask;
+ int err;
+
+ sigemptyset(&mask);
+ sigaddset(&mask,SIGCHLD);
+ sigaddset(&mask,SIGHUP);
+ sigaddset(&mask,SIGPIPE);
+ pthread_sigmask(SIG_SETMASK, &mask, &omask);
+ while ((err = pthread_create(&thread,&server.io_threads_attr,IOThreadEntryPoint,NULL)) != 0) {
+ redisLog(REDIS_WARNING,"Unable to spawn an I/O thread: %s",
+ strerror(err));
+ usleep(1000000);
+ }
+ pthread_sigmask(SIG_SETMASK, &omask, NULL);
+ server.io_active_threads++;
+}
+
+/* Wait that all the pending IO Jobs are processed */
+void waitEmptyIOJobsQueue(void) {
+ while(1) {
+ int io_processed_len;
+
+ lockThreadedIO();
+ if (listLength(server.io_newjobs) == 0 &&
+ listLength(server.io_processing) == 0)
+ {
+ unlockThreadedIO();
+ return;
+ }
+ /* If there are new jobs we need to signal the thread to
+ * process the next one. */
+ redisLog(REDIS_DEBUG,"waitEmptyIOJobsQueue: new %d, processing %d",
+ listLength(server.io_newjobs),
+ listLength(server.io_processing));
+
+ /* FIXME: signal or not?
+ if (listLength(server.io_newjobs)) {
+ pthread_cond_signal(&server.io_condvar);
+ }
+ */
+ /* While waiting for empty jobs queue condition we post-process some
+ * finshed job, as I/O threads may be hanging trying to write against
+ * the io_ready_pipe_write FD but there are so much pending jobs that
+ * it's blocking. */
+ io_processed_len = listLength(server.io_processed);
+ unlockThreadedIO();
+ if (io_processed_len) {
+ vmThreadedIOCompletedJob(NULL,server.io_ready_pipe_read,
+ (void*)0xdeadbeef,0);
+ /* FIXME: probably wiser to drop this sleeps. Anyway
+ * the contention on the IO thread will avoid we to loop
+ * too fast here. */
+ usleep(1000); /* 1 millisecond */
+ } else {
+ /* FIXME: same as fixme above. */
+ usleep(10000); /* 10 milliseconds */
+ }
+ }
+}
+
+/* Process all the IO Jobs already completed by threads but still waiting
+ * processing from the main thread. */
+void processAllPendingIOJobs(void) {
+ while(1) {
+ int io_processed_len;
+
+ lockThreadedIO();
+ io_processed_len = listLength(server.io_processed);
+ unlockThreadedIO();
+ if (io_processed_len == 0) return;
+ vmThreadedIOCompletedJob(NULL,server.io_ready_pipe_read,
+ (void*)0xdeadbeef,0);
+ }
+}
+
+/* This function must be called while with threaded IO locked */
+void queueIOJob(iojob *j) {
+ redisLog(REDIS_DEBUG,"Queued IO Job %p type %d about key '%s'\n",
+ (void*)j, j->type, (char*)j->key->ptr);
+ listAddNodeTail(server.io_newjobs,j);
+ if (server.io_active_threads < server.vm_max_threads)
+ spawnIOThread();
+}
+
+void dsCreateIOJob(int type, redisDb *db, robj *key, robj *val) {
+ iojob *j;
+
+ j = zmalloc(sizeof(*j));
+ j->type = type;
+ j->db = db;
+ j->key = key;
+ incrRefCount(key);
+ j->val = val;
+ if (val) incrRefCount(val);
+
+ lockThreadedIO();
+ queueIOJob(j);
+ pthread_cond_signal(&server.io_condvar);
+ unlockThreadedIO();
+}
+
+/* ============= Disk store cache - Scheduling of IO operations =============
+ *
+ * We use a queue and an hash table to hold the state of IO operations
+ * so that's fast to lookup if there is already an IO operation in queue
+ * for a given key.
+ *
+ * There are two types of IO operations for a given key:
+ * REDIS_IO_LOAD and REDIS_IO_SAVE.
+ *
+ * The function cacheScheduleIO() function pushes the specified IO operation
+ * in the queue, but avoid adding the same key for the same operation
+ * multiple times, thanks to the associated hash table.
+ *
+ * We take a set of flags per every key, so when the scheduled IO operation
+ * gets moved from the scheduled queue to the actual IO Jobs queue that
+ * is processed by the IO thread, we flag it as IO_LOADINPROG or
+ * IO_SAVEINPROG.
+ *
+ * So for every given key we always know if there is some IO operation
+ * scheduled, or in progress, for this key.
+ *
+ * NOTE: all this is very important in order to guarantee correctness of
+ * the Disk Store Cache. Jobs are always queued here. Load jobs are
+ * queued at the head for faster execution only in the case there is not
+ * already a write operation of some kind for this job.
+ *
+ * So we have ordering, but can do exceptions when there are no already
+ * operations for a given key. Also when we need to block load a given
+ * key, for an immediate lookup operation, we can check if the key can
+ * be accessed synchronously without race conditions (no IN PROGRESS
+ * operations for this key), otherwise we blocking wait for completion. */
+
+#define REDIS_IO_LOAD 1
+#define REDIS_IO_SAVE 2
+#define REDIS_IO_LOADINPROG 4
+#define REDIS_IO_SAVEINPROG 8
+
+void cacheScheduleIOAddFlag(redisDb *db, robj *key, long flag) {
+ struct dictEntry *de = dictFind(db->io_queued,key);
+
+ if (!de) {
+ dictAdd(db->io_queued,key,(void*)flag);
+ incrRefCount(key);
+ return;
+ } else {
+ long flags = (long) dictGetEntryVal(de);
+
+ if (flags & flag) {
+ redisLog(REDIS_WARNING,"Adding the same flag again: was: %ld, addede: %ld",flags,flag);
+ redisAssert(!(flags & flag));
+ }
+ flags |= flag;
+ dictGetEntryVal(de) = (void*) flags;
+ }
+}
+
+void cacheScheduleIODelFlag(redisDb *db, robj *key, long flag) {
+ struct dictEntry *de = dictFind(db->io_queued,key);
+ long flags;
+
+ redisAssert(de != NULL);
+ flags = (long) dictGetEntryVal(de);
+ redisAssert(flags & flag);
+ flags &= ~flag;
+ if (flags == 0) {
+ dictDelete(db->io_queued,key);
+ } else {
+ dictGetEntryVal(de) = (void*) flags;
+ }
+}
+
+int cacheScheduleIOGetFlags(redisDb *db, robj *key) {
+ struct dictEntry *de = dictFind(db->io_queued,key);
+
+ return (de == NULL) ? 0 : ((long) dictGetEntryVal(de));
+}
+
+void cacheScheduleIO(redisDb *db, robj *key, int type) {
+ ioop *op;
+ long flags;
+
+ if ((flags = cacheScheduleIOGetFlags(db,key)) & type) return;
+
+ redisLog(REDIS_DEBUG,"Scheduling key %s for %s",
+ key->ptr, type == REDIS_IO_LOAD ? "loading" : "saving");
+ cacheScheduleIOAddFlag(db,key,type);
+ op = zmalloc(sizeof(*op));
+ op->type = type;
+ op->db = db;
+ op->key = key;
+ incrRefCount(key);
+ op->ctime = time(NULL);
+
+ /* Give priority to load operations if there are no save already
+ * in queue for the same key. */
+ if (type == REDIS_IO_LOAD && !(flags & REDIS_IO_SAVE)) {
+ listAddNodeHead(server.cache_io_queue, op);
+ cacheScheduleIOPushJobs(1);
+ } else {
+ /* FIXME: probably when this happens we want to at least move
+ * the write job about this queue on top, and set the creation time
+ * to a value that will force processing ASAP. */
+ listAddNodeTail(server.cache_io_queue, op);
+ }
+}
+
+/* Push scheduled IO operations into IO Jobs that the IO thread can process.
+ * If 'onlyloads' is true only IO_LOAD jobs are processed: this is useful
+ * since it's save to push LOAD IO jobs from any place of the code, while
+ * SAVE io jobs should never be pushed while we are processing a command
+ * (not protected by lookupKey() that will block on keys in IO_SAVEINPROG
+ * state. */
+#define MAX_IO_JOBS_QUEUE 100
+int cacheScheduleIOPushJobs(int onlyloads) {
+ time_t now = time(NULL);
+ listNode *ln;
+ int jobs, topush = 0, pushed = 0;
+
+ /* Sync stuff on disk, but only if we have less
+ * than MAX_IO_JOBS_QUEUE IO jobs. */
+ lockThreadedIO();
+ jobs = listLength(server.io_newjobs);
+ unlockThreadedIO();
+
+ topush = MAX_IO_JOBS_QUEUE-jobs;
+ if (topush < 0) topush = 0;
+ if (topush > (signed)listLength(server.cache_io_queue))
+ topush = listLength(server.cache_io_queue);
+
+ while((ln = listFirst(server.cache_io_queue)) != NULL) {
+ ioop *op = ln->value;
+
+ if (!topush) break;
+ topush--;
+
+ if (op->type == REDIS_IO_LOAD ||
+ (!onlyloads && (now - op->ctime) >= server.cache_flush_delay))
+ {
+ struct dictEntry *de;
+ robj *val;
+
+ /* Don't add a SAVE job in queue if there is already
+ * a save in progress for the same key. */
+ if (op->type == REDIS_IO_SAVE &&
+ cacheScheduleIOGetFlags(op->db,op->key) & REDIS_IO_SAVEINPROG)
+ {
+ /* Move the operation at the end of the list of there
+ * are other operations. Otherwise break, nothing to do
+ * here. */
+ if (listLength(server.cache_io_queue) > 1) {
+ listDelNode(server.cache_io_queue,ln);
+ listAddNodeTail(server.cache_io_queue,op);
+ continue;
+ } else {
+ break;
+ }
+ }
+
+ redisLog(REDIS_DEBUG,"Creating IO %s Job for key %s",
+ op->type == REDIS_IO_LOAD ? "load" : "save", op->key->ptr);
+
+ if (op->type == REDIS_IO_LOAD) {
+ dsCreateIOJob(REDIS_IOJOB_LOAD,op->db,op->key,NULL);
+ } else {
+ /* Lookup the key, in order to put the current value in the IO
+ * Job. Otherwise if the key does not exists we schedule a disk
+ * store delete operation, setting the value to NULL. */
+ de = dictFind(op->db->dict,op->key->ptr);
+ if (de) {
+ val = dictGetEntryVal(de);
+ } else {
+ /* Setting the value to NULL tells the IO thread to delete
+ * the key on disk. */
+ val = NULL;
+ }
+ dsCreateIOJob(REDIS_IOJOB_SAVE,op->db,op->key,val);
+ }
+ /* Mark the operation as in progress. */
+ cacheScheduleIODelFlag(op->db,op->key,op->type);
+ cacheScheduleIOAddFlag(op->db,op->key,
+ (op->type == REDIS_IO_LOAD) ? REDIS_IO_LOADINPROG :
+ REDIS_IO_SAVEINPROG);
+ /* Finally remove the operation from the queue.
+ * But we'll have trace of it in the hash table. */
+ listDelNode(server.cache_io_queue,ln);
+ decrRefCount(op->key);
+ zfree(op);
+ pushed++;
+ } else {
+ break; /* too early */
+ }
+ }
+ return pushed;
+}
+
+void cacheCron(void) {
+ /* Push jobs */
+ cacheScheduleIOPushJobs(0);
+
+ /* Reclaim memory from the object cache */
+ while (server.ds_enabled && zmalloc_used_memory() >
+ server.cache_max_memory)
+ {
+ int done = 0;
+
+ if (cacheFreeOneEntry() == REDIS_OK) done++;
+ if (negativeCacheEvictOneEntry() == REDIS_OK) done++;
+ if (done == 0) break; /* nothing more to free */
+ }
+}
+
+/* ========== Disk store cache - Blocking clients on missing keys =========== */
+
+/* This function makes the clinet 'c' waiting for the key 'key' to be loaded.
+ * If the key is already in memory we don't need to block.
+ *
+ * FIXME: we should try if it's actually better to suspend the client
+ * accessing an object that is being saved, and awake it only when
+ * the saving was completed.
+ *
+ * Otherwise if the key is not in memory, we block the client and start
+ * an IO Job to load it:
+ *
+ * the key is added to the io_keys list in the client structure, and also
+ * in the hash table mapping swapped keys to waiting clients, that is,
+ * server.io_waited_keys. */
+int waitForSwappedKey(redisClient *c, robj *key) {
+ struct dictEntry *de;
+ list *l;
+
+ /* Return ASAP if the key is in memory */
+ de = dictFind(c->db->dict,key->ptr);
+ if (de != NULL) return 0;
+
+ /* Don't wait for keys we are sure are not on disk either */
+ if (!cacheKeyMayExist(c->db,key)) return 0;
+
+ /* Add the key to the list of keys this client is waiting for.
+ * This maps clients to keys they are waiting for. */
+ listAddNodeTail(c->io_keys,key);
+ incrRefCount(key);
+
+ /* Add the client to the swapped keys => clients waiting map. */
+ de = dictFind(c->db->io_keys,key);
+ if (de == NULL) {
+ int retval;
+
+ /* For every key we take a list of clients blocked for it */
+ l = listCreate();
+ retval = dictAdd(c->db->io_keys,key,l);
+ incrRefCount(key);
+ redisAssert(retval == DICT_OK);
+ } else {
+ l = dictGetEntryVal(de);
+ }
+ listAddNodeTail(l,c);
+
+ /* Are we already loading the key from disk? If not create a job */
+ if (de == NULL)
+ cacheScheduleIO(c->db,key,REDIS_IO_LOAD);
+ return 1;
+}
+
+/* Preload keys for any command with first, last and step values for
+ * the command keys prototype, as defined in the command table. */
+void waitForMultipleSwappedKeys(redisClient *c, struct redisCommand *cmd, int argc, robj **argv) {
+ int j, last;
+ if (cmd->vm_firstkey == 0) return;
+ last = cmd->vm_lastkey;
+ if (last < 0) last = argc+last;
+ for (j = cmd->vm_firstkey; j <= last; j += cmd->vm_keystep) {
+ redisAssert(j < argc);
+ waitForSwappedKey(c,argv[j]);
+ }
+}
+
+/* Preload keys needed for the ZUNIONSTORE and ZINTERSTORE commands.
+ * Note that the number of keys to preload is user-defined, so we need to
+ * apply a sanity check against argc. */
+void zunionInterBlockClientOnSwappedKeys(redisClient *c, struct redisCommand *cmd, int argc, robj **argv) {
+ int i, num;
+ REDIS_NOTUSED(cmd);
+
+ num = atoi(argv[2]->ptr);
+ if (num > (argc-3)) return;
+ for (i = 0; i < num; i++) {
+ waitForSwappedKey(c,argv[3+i]);
+ }
+}
+
+/* Preload keys needed to execute the entire MULTI/EXEC block.
+ *
+ * This function is called by blockClientOnSwappedKeys when EXEC is issued,
+ * and will block the client when any command requires a swapped out value. */
+void execBlockClientOnSwappedKeys(redisClient *c, struct redisCommand *cmd, int argc, robj **argv) {
+ int i, margc;
+ struct redisCommand *mcmd;
+ robj **margv;
+ REDIS_NOTUSED(cmd);
+ REDIS_NOTUSED(argc);
+ REDIS_NOTUSED(argv);
+
+ if (!(c->flags & REDIS_MULTI)) return;
+ for (i = 0; i < c->mstate.count; i++) {
+ mcmd = c->mstate.commands[i].cmd;
+ margc = c->mstate.commands[i].argc;
+ margv = c->mstate.commands[i].argv;
+
+ if (mcmd->vm_preload_proc != NULL) {
+ mcmd->vm_preload_proc(c,mcmd,margc,margv);
+ } else {
+ waitForMultipleSwappedKeys(c,mcmd,margc,margv);
+ }
+ }
+}
+
+/* Is this client attempting to run a command against swapped keys?
+ * If so, block it ASAP, load the keys in background, then resume it.
+ *
+ * The important idea about this function is that it can fail! If keys will
+ * still be swapped when the client is resumed, this key lookups will
+ * just block loading keys from disk. In practical terms this should only
+ * happen with SORT BY command or if there is a bug in this function.
+ *
+ * Return 1 if the client is marked as blocked, 0 if the client can
+ * continue as the keys it is going to access appear to be in memory. */
+int blockClientOnSwappedKeys(redisClient *c, struct redisCommand *cmd) {
+ if (cmd->vm_preload_proc != NULL) {
+ cmd->vm_preload_proc(c,cmd,c->argc,c->argv);
+ } else {
+ waitForMultipleSwappedKeys(c,cmd,c->argc,c->argv);
+ }
+
+ /* If the client was blocked for at least one key, mark it as blocked. */
+ if (listLength(c->io_keys)) {
+ c->flags |= REDIS_IO_WAIT;
+ aeDeleteFileEvent(server.el,c->fd,AE_READABLE);
+ server.cache_blocked_clients++;
+ return 1;
+ } else {
+ return 0;
+ }
+}
+
+/* Remove the 'key' from the list of blocked keys for a given client.
+ *
+ * The function returns 1 when there are no longer blocking keys after
+ * the current one was removed (and the client can be unblocked). */
+int dontWaitForSwappedKey(redisClient *c, robj *key) {
+ list *l;
+ listNode *ln;
+ listIter li;
+ struct dictEntry *de;
+
+ /* The key object might be destroyed when deleted from the c->io_keys
+ * list (and the "key" argument is physically the same object as the
+ * object inside the list), so we need to protect it. */
+ incrRefCount(key);
+
+ /* Remove the key from the list of keys this client is waiting for. */
+ listRewind(c->io_keys,&li);
+ while ((ln = listNext(&li)) != NULL) {
+ if (equalStringObjects(ln->value,key)) {
+ listDelNode(c->io_keys,ln);
+ break;
+ }
+ }
+ redisAssert(ln != NULL);
+
+ /* Remove the client form the key => waiting clients map. */
+ de = dictFind(c->db->io_keys,key);
+ redisAssert(de != NULL);
+ l = dictGetEntryVal(de);
+ ln = listSearchKey(l,c);
+ redisAssert(ln != NULL);
+ listDelNode(l,ln);
+ if (listLength(l) == 0)
+ dictDelete(c->db->io_keys,key);
+
+ decrRefCount(key);
+ return listLength(c->io_keys) == 0;
+}
+
+/* Every time we now a key was loaded back in memory, we handle clients
+ * waiting for this key if any. */
+void handleClientsBlockedOnSwappedKey(redisDb *db, robj *key) {
+ struct dictEntry *de;
+ list *l;
+ listNode *ln;
+ int len;
+
+ de = dictFind(db->io_keys,key);
+ if (!de) return;
+
+ l = dictGetEntryVal(de);
+ len = listLength(l);
+ /* Note: we can't use something like while(listLength(l)) as the list
+ * can be freed by the calling function when we remove the last element. */
+ while (len--) {
+ ln = listFirst(l);
+ redisClient *c = ln->value;
+
+ if (dontWaitForSwappedKey(c,key)) {
+ /* Put the client in the list of clients ready to go as we
+ * loaded all the keys about it. */
+ listAddNodeTail(server.io_ready_clients,c);
+ }
+ }
+}
void addReply(redisClient *c, robj *obj) {
if (_installWriteEvent(c) != REDIS_OK) return;
- redisAssert(!server.vm_enabled || obj->storage == REDIS_VM_MEMORY);
/* This is an important place where we can avoid copy-on-write
* when there is a saving child running, avoiding touching the
/* Remove from the list of clients waiting for swapped keys, or ready
* to be restarted, but not yet woken up again. */
if (c->flags & REDIS_IO_WAIT) {
- redisAssert(server.vm_enabled);
+ redisAssert(server.ds_enabled);
if (listLength(c->io_keys) == 0) {
ln = listSearchKey(server.io_ready_clients,c);
dontWaitForSwappedKey(c,ln->value);
}
}
- server.vm_blocked_clients--;
+ server.cache_blocked_clients--;
}
listRelease(c->io_keys);
/* Master/slave cleanup.
/* The following is only needed if VM is active, but since the conditional
* is probably more costly than initializing the field it's better to
* have every field properly initialized anyway. */
- o->storage = REDIS_VM_MEMORY;
return o;
}
robj *createStringObjectFromLongLong(long long value) {
robj *o;
if (value >= 0 && value < REDIS_SHARED_INTEGERS &&
+ !server.ds_enabled &&
pthread_equal(pthread_self(),server.mainthread)) {
incrRefCount(shared.integers[value]);
o = shared.integers[value];
void decrRefCount(void *obj) {
robj *o = obj;
- /* Object is a swapped out value, or in the process of being loaded. */
- if (server.vm_enabled &&
- (o->storage == REDIS_VM_SWAPPED || o->storage == REDIS_VM_LOADING))
- {
- vmpointer *vp = obj;
- if (o->storage == REDIS_VM_LOADING) vmCancelThreadedIOJob(o);
- vmMarkPagesFree(vp->page,vp->usedpages);
- server.vm_stats_swapped_objects--;
- zfree(vp);
- return;
- }
-
if (o->refcount <= 0) redisPanic("decrRefCount against refcount <= 0");
- /* Object is in memory, or in the process of being swapped out.
- *
- * If the object is being swapped out, abort the operation on
- * decrRefCount even if the refcount does not drop to 0: the object
- * is referenced at least two times, as value of the key AND as
- * job->val in the iojob. So if we don't invalidate the iojob, when it is
- * done but the relevant key was removed in the meantime, the
- * complete jobs handler will not find the key about the job and the
- * assert will fail. */
- if (server.vm_enabled && o->storage == REDIS_VM_SWAPPING)
- vmCancelThreadedIOJob(o);
if (--(o->refcount) == 0) {
switch(o->type) {
case REDIS_STRING: freeStringObject(o); break;
/* Ok, this object can be encoded...
*
* Can I use a shared object? Only if the object is inside a given
- * range and if this is the main thread, since when VM is enabled we
- * have the constraint that I/O thread should only handle non-shared
- * objects, in order to avoid race conditions (we don't have per-object
- * locking).
+ * range and if the back end in use is in-memory. For disk store every
+ * object in memory used as value should be independent.
*
* Note that we also avoid using shared integers when maxmemory is used
- * because very object needs to have a private LRU field for the LRU
+ * because every object needs to have a private LRU field for the LRU
* algorithm to work well. */
- if (server.maxmemory == 0 && value >= 0 && value < REDIS_SHARED_INTEGERS &&
- pthread_equal(pthread_self(),server.mainthread)) {
+ if (!server.ds_enabled &&
+ server.maxmemory == 0 && value >= 0 && value < REDIS_SHARED_INTEGERS &&
+ pthread_equal(pthread_self(),server.mainthread))
+ {
decrRefCount(o);
incrRefCount(shared.integers[value]);
return shared.integers[value];
return len;
}
-/* Return the number of pages required to save this object in the swap file */
-off_t rdbSavedObjectPages(robj *o) {
- off_t bytes = rdbSavedObjectLen(o);
- return (bytes+(server.vm_page_size-1))/server.vm_page_size;
+/* Save a key-value pair, with expire time, type, key, value.
+ * On error -1 is returned.
+ * On success if the key was actaully saved 1 is returned, otherwise 0
+ * is returned (the key was already expired). */
+int rdbSaveKeyValuePair(FILE *fp, redisDb *db, robj *key, robj *val,
+ time_t now)
+{
+ time_t expiretime;
+
+ expiretime = getExpire(db,key);
+
+ /* Save the expire time */
+ if (expiretime != -1) {
+ /* If this key is already expired skip it */
+ if (expiretime < now) return 0;
+ if (rdbSaveType(fp,REDIS_EXPIRETIME) == -1) return -1;
+ if (rdbSaveTime(fp,expiretime) == -1) return -1;
+ }
+ /* Save type, key, value */
+ if (rdbSaveType(fp,val->type) == -1) return -1;
+ if (rdbSaveStringObject(fp,key) == -1) return -1;
+ if (rdbSaveObject(fp,val) == -1) return -1;
+ return 1;
}
/* Save the DB on disk. Return REDIS_ERR on error, REDIS_OK on success */
int j;
time_t now = time(NULL);
- /* Wait for I/O therads to terminate, just in case this is a
- * foreground-saving, to avoid seeking the swap file descriptor at the
- * same time. */
- if (server.vm_enabled)
- waitEmptyIOJobsQueue();
+ /* FIXME: implement .rdb save for disk store properly */
+ redisAssert(server.ds_enabled == 0);
snprintf(tmpfile,256,"temp-%d.rdb", (int) getpid());
fp = fopen(tmpfile,"w");
while((de = dictNext(di)) != NULL) {
sds keystr = dictGetEntryKey(de);
robj key, *o = dictGetEntryVal(de);
- time_t expiretime;
initStaticStringObject(key,keystr);
- expiretime = getExpire(db,&key);
-
- /* Save the expire time */
- if (expiretime != -1) {
- /* If this key is already expired skip it */
- if (expiretime < now) continue;
- if (rdbSaveType(fp,REDIS_EXPIRETIME) == -1) goto werr;
- if (rdbSaveTime(fp,expiretime) == -1) goto werr;
- }
- /* Save the key and associated value. This requires special
- * handling if the value is swapped out. */
- if (!server.vm_enabled || o->storage == REDIS_VM_MEMORY ||
- o->storage == REDIS_VM_SWAPPING) {
- /* Save type, key, value */
- if (rdbSaveType(fp,o->type) == -1) goto werr;
- if (rdbSaveStringObject(fp,&key) == -1) goto werr;
- if (rdbSaveObject(fp,o) == -1) goto werr;
- } else {
- /* REDIS_VM_SWAPPED or REDIS_VM_LOADING */
- robj *po;
- /* Get a preview of the object in memory */
- po = vmPreviewObject(o);
- /* Save type, key, value */
- if (rdbSaveType(fp,po->type) == -1) goto werr;
- if (rdbSaveStringObject(fp,&key) == -1) goto werr;
- if (rdbSaveObject(fp,po) == -1) goto werr;
- /* Remove the loaded object from memory */
- decrRefCount(po);
- }
+ if (rdbSaveKeyValuePair(fp,db,&key,o,now) == -1) goto werr;
}
dictReleaseIterator(di);
}
pid_t childpid;
if (server.bgsavechildpid != -1) return REDIS_ERR;
- if (server.vm_enabled) waitEmptyIOJobsQueue();
+ redisAssert(server.ds_enabled == 0);
server.dirty_before_bgsave = server.dirty;
if ((childpid = fork()) == 0) {
/* Child */
- if (server.vm_enabled) vmReopenSwapFile();
if (server.ipfd > 0) close(server.ipfd);
if (server.sofd > 0) close(server.sofd);
if (rdbSave(filename) == REDIS_OK) {
FILE *fp;
uint32_t dbid;
int type, retval, rdbver;
- int swap_all_values = 0;
redisDb *db = server.db+0;
char buf[1024];
time_t expiretime, now = time(NULL);
startLoading(fp);
while(1) {
robj *key, *val;
- int force_swapout;
-
expiretime = -1;
/* Serve the clients from time to time */
/* Set the expire time if needed */
if (expiretime != -1) setExpire(db,key,expiretime);
- /* Handle swapping while loading big datasets when VM is on */
-
- /* If we detecter we are hopeless about fitting something in memory
- * we just swap every new key on disk. Directly...
- * Note that's important to check for this condition before resorting
- * to random sampling, otherwise we may try to swap already
- * swapped keys. */
- if (swap_all_values) {
- dictEntry *de = dictFind(db->dict,key->ptr);
-
- /* de may be NULL since the key already expired */
- if (de) {
- vmpointer *vp;
- val = dictGetEntryVal(de);
-
- if (val->refcount == 1 &&
- (vp = vmSwapObjectBlocking(val)) != NULL)
- dictGetEntryVal(de) = vp;
- }
- decrRefCount(key);
- continue;
- }
decrRefCount(key);
-
- /* Flush data on disk once 32 MB of additional RAM are used... */
- force_swapout = 0;
- if ((zmalloc_used_memory() - server.vm_max_memory) > 1024*1024*32)
- force_swapout = 1;
-
- /* If we have still some hope of having some value fitting memory
- * then we try random sampling. */
- if (!swap_all_values && server.vm_enabled && force_swapout) {
- while (zmalloc_used_memory() > server.vm_max_memory) {
- if (vmSwapOneObjectBlocking() == REDIS_ERR) break;
- }
- if (zmalloc_used_memory() > server.vm_max_memory)
- swap_all_values = 1; /* We are already using too much mem */
- }
}
fclose(fp);
stopLoading();
}
}
-/* Sets type */
+/* Sets type and diskstore negative caching hash table */
dictType setDictType = {
dictEncObjHash, /* hash function */
NULL, /* key dup */
}
updateDictResizePolicy();
}
- } else {
+ } else if (!server.ds_enabled) {
/* If there is not a background saving in progress check if
* we have to save now */
time_t now = time(NULL);
* in order to guarantee a strict consistency. */
if (server.masterhost == NULL) activeExpireCycle();
- /* Swap a few keys on disk if we are over the memory limit and VM
- * is enbled. Try to free objects from the free list first. */
- if (vmCanSwapOut()) {
- while (server.vm_enabled && zmalloc_used_memory() >
- server.vm_max_memory)
- {
- int retval = (server.vm_max_threads == 0) ?
- vmSwapOneObjectBlocking() :
- vmSwapOneObjectThreaded();
- if (retval == REDIS_ERR && !(loops % 300) &&
- zmalloc_used_memory() >
- (server.vm_max_memory+server.vm_max_memory/10))
- {
- redisLog(REDIS_WARNING,"WARNING: vm-max-memory limit exceeded by more than 10%% but unable to swap more objects out!");
- }
- /* Note that when using threade I/O we free just one object,
- * because anyway when the I/O thread in charge to swap this
- * object out will finish, the handler of completed jobs
- * will try to swap more objects if we are still out of memory. */
- if (retval == REDIS_ERR || server.vm_max_threads > 0) break;
- }
- }
+ /* Remove a few cached objects from memory if we are over the
+ * configured memory limit */
+ if (server.ds_enabled) cacheCron();
/* Replication cron function -- used to reconnect to master and
* to detect transfer failures. */
listNode *ln;
redisClient *c;
- /* Awake clients that got all the swapped keys they requested */
- if (server.vm_enabled && listLength(server.io_ready_clients)) {
+ /* Awake clients that got all the on disk keys they requested */
+ if (server.ds_enabled && listLength(server.io_ready_clients)) {
listIter li;
listRewind(server.io_ready_clients,&li);
/* Resume the client. */
listDelNode(server.io_ready_clients,ln);
c->flags &= (~REDIS_IO_WAIT);
- server.vm_blocked_clients--;
+ server.cache_blocked_clients--;
aeCreateFileEvent(server.el, c->fd, AE_READABLE,
readQueryFromClient, c);
cmd = lookupCommand(c->argv[0]->ptr);
server.maxmemory = 0;
server.maxmemory_policy = REDIS_MAXMEMORY_VOLATILE_LRU;
server.maxmemory_samples = 3;
- server.vm_enabled = 0;
- server.vm_swap_file = zstrdup("/tmp/redis-%p.vm");
- server.vm_page_size = 256; /* 256 bytes per page */
- server.vm_pages = 1024*1024*100; /* 104 millions of pages */
- server.vm_max_memory = 1024LL*1024*1024*1; /* 1 GB of RAM */
- server.vm_max_threads = 4;
- server.vm_blocked_clients = 0;
+ server.ds_enabled = 0;
+ server.ds_path = sdsnew("/tmp/redis.ds");
+ server.cache_max_memory = 64LL*1024*1024; /* 64 MB of RAM */
+ server.cache_blocked_clients = 0;
server.hash_max_zipmap_entries = REDIS_HASH_MAX_ZIPMAP_ENTRIES;
server.hash_max_zipmap_value = REDIS_HASH_MAX_ZIPMAP_VALUE;
server.list_max_ziplist_entries = REDIS_LIST_MAX_ZIPLIST_ENTRIES;
server.list_max_ziplist_value = REDIS_LIST_MAX_ZIPLIST_VALUE;
server.set_max_intset_entries = REDIS_SET_MAX_INTSET_ENTRIES;
server.shutdown_asap = 0;
+ server.cache_flush_delay = 0;
updateLRUClock();
resetServerSaveParams();
server.slaves = listCreate();
server.monitors = listCreate();
server.unblocked_clients = listCreate();
+ server.cache_io_queue = listCreate();
+
createSharedObjects();
server.el = aeCreateEventLoop();
server.db = zmalloc(sizeof(redisDb)*server.dbnum);
server.db[j].expires = dictCreate(&keyptrDictType,NULL);
server.db[j].blocking_keys = dictCreate(&keylistDictType,NULL);
server.db[j].watched_keys = dictCreate(&keylistDictType,NULL);
- if (server.vm_enabled)
+ if (server.ds_enabled) {
server.db[j].io_keys = dictCreate(&keylistDictType,NULL);
+ server.db[j].io_negcache = dictCreate(&setDictType,NULL);
+ server.db[j].io_queued = dictCreate(&setDictType,NULL);
+ }
server.db[j].id = j;
}
server.pubsub_channels = dictCreate(&keylistDictType,NULL);
}
}
- if (server.vm_enabled) vmInit();
+ if (server.ds_enabled) dsInit();
}
/* Populates the Redis Command Table starting from the hard coded list
queueMultiCommand(c,cmd);
addReply(c,shared.queued);
} else {
- if (server.vm_enabled && server.vm_max_threads > 0 &&
- blockClientOnSwappedKeys(c,cmd)) return REDIS_ERR;
+ if (server.ds_enabled && blockClientOnSwappedKeys(c,cmd))
+ return REDIS_ERR;
call(c,cmd);
}
return REDIS_OK;
kill(server.bgsavechildpid,SIGKILL);
rdbRemoveTempFile(server.bgsavechildpid);
}
- if (server.appendonly) {
+ if (server.ds_enabled) {
+ /* FIXME: flush all objects on disk */
+ } else if (server.appendonly) {
/* Append only file: fsync() the AOF and exit */
aof_fsync(server.appendfd);
- if (server.vm_enabled) unlink(server.vm_swap_file);
} else if (server.saveparamslen > 0) {
/* Snapshotting. Perform a SYNC SAVE and exit */
if (rdbSave(server.dbfilename) != REDIS_OK) {
"hash_max_zipmap_value:%zu\r\n"
"pubsub_channels:%ld\r\n"
"pubsub_patterns:%u\r\n"
- "vm_enabled:%d\r\n"
+ "ds_enabled:%d\r\n"
"role:%s\r\n"
,REDIS_VERSION,
redisGitSHA1(),
server.hash_max_zipmap_value,
dictSize(server.pubsub_channels),
listLength(server.pubsub_patterns),
- server.vm_enabled != 0,
+ server.ds_enabled != 0,
server.masterhost == NULL ? "master" : "slave"
);
if (server.masterhost) {
);
}
}
- if (server.vm_enabled) {
+ if (server.ds_enabled) {
lockThreadedIO();
info = sdscatprintf(info,
- "vm_conf_max_memory:%llu\r\n"
- "vm_conf_page_size:%llu\r\n"
- "vm_conf_pages:%llu\r\n"
- "vm_stats_used_pages:%llu\r\n"
- "vm_stats_swapped_objects:%llu\r\n"
- "vm_stats_swappin_count:%llu\r\n"
- "vm_stats_swappout_count:%llu\r\n"
- "vm_stats_io_newjobs_len:%lu\r\n"
- "vm_stats_io_processing_len:%lu\r\n"
- "vm_stats_io_processed_len:%lu\r\n"
- "vm_stats_io_active_threads:%lu\r\n"
- "vm_stats_blocked_clients:%lu\r\n"
- ,(unsigned long long) server.vm_max_memory,
- (unsigned long long) server.vm_page_size,
- (unsigned long long) server.vm_pages,
- (unsigned long long) server.vm_stats_used_pages,
- (unsigned long long) server.vm_stats_swapped_objects,
- (unsigned long long) server.vm_stats_swapins,
- (unsigned long long) server.vm_stats_swapouts,
- (unsigned long) listLength(server.io_newjobs),
- (unsigned long) listLength(server.io_processing),
- (unsigned long) listLength(server.io_processed),
- (unsigned long) server.io_active_threads,
- (unsigned long) server.vm_blocked_clients
+ "cache_max_memory:%llu\r\n"
+ "cache_blocked_clients:%lu\r\n"
+ ,(unsigned long long) server.cache_max_memory,
+ (unsigned long) server.cache_blocked_clients
);
unlockThreadedIO();
}
linuxOvercommitMemoryWarning();
#endif
start = time(NULL);
- if (server.appendonly) {
+ if (server.ds_enabled) {
+ redisLog(REDIS_NOTICE,"DB not loaded (running with disk back end)");
+ } else if (server.appendonly) {
if (loadAppendOnlyFile(server.appendfilename) == REDIS_OK)
redisLog(REDIS_NOTICE,"DB loaded from append only file: %ld seconds",time(NULL)-start);
} else {
#define REDIS_RDB_ENC_INT32 2 /* 32 bit signed integer */
#define REDIS_RDB_ENC_LZF 3 /* string compressed with FASTLZ */
-/* Virtual memory object->where field. */
-#define REDIS_VM_MEMORY 0 /* The object is on memory */
-#define REDIS_VM_SWAPPED 1 /* The object is on disk */
-#define REDIS_VM_SWAPPING 2 /* Redis is swapping this object on disk */
-#define REDIS_VM_LOADING 3 /* Redis is loading this object from disk */
-
-/* Virtual memory static configuration stuff.
- * Check vmFindContiguousPages() to know more about this magic numbers. */
-#define REDIS_VM_MAX_NEAR_PAGES 65536
-#define REDIS_VM_MAX_RANDOM_JUMP 4096
-#define REDIS_VM_MAX_THREADS 32
-#define REDIS_THREAD_STACK_SIZE (1024*1024*4)
-/* The following is the *percentage* of completed I/O jobs to process when the
- * handelr is called. While Virtual Memory I/O operations are performed by
- * threads, this operations must be processed by the main thread when completed
- * in order to take effect. */
+/* Scheduled IO opeations flags. */
+#define REDIS_IO_LOAD 1
+#define REDIS_IO_SAVE 2
+#define REDIS_IO_LOADINPROG 4
+#define REDIS_IO_SAVEINPROG 8
+
#define REDIS_MAX_COMPLETED_JOBS_PROCESSED 1
+#define REDIS_THREAD_STACK_SIZE (1024*1024*4)
/* Client flags */
#define REDIS_SLAVE 1 /* This client is a slave server */
#define REDIS_LRU_CLOCK_RESOLUTION 10 /* LRU clock resolution in seconds */
typedef struct redisObject {
unsigned type:4;
- unsigned storage:2; /* REDIS_VM_MEMORY or REDIS_VM_SWAPPING */
+ unsigned notused:2; /* Not used */
unsigned encoding:4;
unsigned lru:22; /* lru time (relative to server.lruclock) */
int refcount;
_var.type = REDIS_STRING; \
_var.encoding = REDIS_ENCODING_RAW; \
_var.ptr = _ptr; \
- _var.storage = REDIS_VM_MEMORY; \
} while(0);
typedef struct redisDb {
dict *dict; /* The keyspace for this DB */
dict *expires; /* Timeout of keys with a timeout set */
dict *blocking_keys; /* Keys with clients waiting for data (BLPOP) */
- dict *io_keys; /* Keys with clients waiting for VM I/O */
+ dict *io_keys; /* Keys with clients waiting for DS I/O */
+ dict *io_negcache; /* Negative caching for disk store */
+ dict *io_queued; /* Queued IO operations hash table */
dict *watched_keys; /* WATCHED keys for MULTI/EXEC CAS */
int id;
} redisDb;
int maxmemory_samples;
/* Blocked clients */
unsigned int bpop_blocked_clients;
- unsigned int vm_blocked_clients;
- list *unblocked_clients;
+ unsigned int cache_blocked_clients;
+ list *unblocked_clients; /* list of clients to unblock before next loop */
+ list *cache_io_queue; /* IO operations queue */
+ int cache_flush_delay; /* seconds to wait before flushing keys */
/* Sort parameters - qsort_r() is only available under BSD so we
* have to take this state global, in order to pass it to sortCompare() */
int sort_desc;
int sort_alpha;
int sort_bypattern;
/* Virtual memory configuration */
- int vm_enabled;
- char *vm_swap_file;
- off_t vm_page_size;
- off_t vm_pages;
- unsigned long long vm_max_memory;
+ int ds_enabled; /* backend disk in redis.conf */
+ char *ds_path; /* location of the disk store on disk */
+ unsigned long long cache_max_memory;
/* Zip structure config */
size_t hash_max_zipmap_entries;
size_t hash_max_zipmap_value;
size_t list_max_ziplist_entries;
size_t list_max_ziplist_value;
size_t set_max_intset_entries;
- /* Virtual memory state */
- FILE *vm_fp;
- int vm_fd;
- off_t vm_next_page; /* Next probably empty page */
- off_t vm_near_pages; /* Number of pages allocated sequentially */
- unsigned char *vm_bitmap; /* Bitmap of free/used pages */
time_t unixtime; /* Unix time sampled every second. */
/* Virtual memory I/O threads stuff */
/* An I/O thread process an element taken from the io_jobs queue and
list *io_processed; /* List of VM I/O jobs already processed */
list *io_ready_clients; /* Clients ready to be unblocked. All keys loaded */
pthread_mutex_t io_mutex; /* lock to access io_jobs/io_done/io_thread_job */
- pthread_mutex_t io_swapfile_mutex; /* So we can lseek + write */
+ pthread_cond_t io_condvar; /* I/O threads conditional variable */
pthread_attr_t io_threads_attr; /* attributes for threads creation */
int io_active_threads; /* Number of running I/O threads */
int vm_max_threads; /* Max number of I/O threads running at the same time */
zskiplist *zsl;
} zset;
-/* VM threaded I/O request message */
-#define REDIS_IOJOB_LOAD 0 /* Load from disk to memory */
-#define REDIS_IOJOB_PREPARE_SWAP 1 /* Compute needed pages */
-#define REDIS_IOJOB_DO_SWAP 2 /* Swap from memory to disk */
+/* DIsk store threaded I/O request message */
+#define REDIS_IOJOB_LOAD 0
+#define REDIS_IOJOB_SAVE 1
+
typedef struct iojob {
int type; /* Request type, REDIS_IOJOB_* */
redisDb *db;/* Redis database */
- robj *key; /* This I/O request is about swapping this key */
- robj *id; /* Unique identifier of this job:
- this is the object to swap for REDIS_IOREQ_*_SWAP, or the
- vmpointer objct for REDIS_IOREQ_LOAD. */
- robj *val; /* the value to swap for REDIS_IOREQ_*_SWAP, otherwise this
- * field is populated by the I/O thread for REDIS_IOREQ_LOAD. */
- off_t page; /* Swap page where to read/write the object */
- off_t pages; /* Swap pages needed to save object. PREPARE_SWAP return val */
- int canceled; /* True if this command was canceled by blocking side of VM */
- pthread_t thread; /* ID of the thread processing this entry */
+ robj *key; /* This I/O request is about this key */
+ robj *val; /* the value to swap for REDIS_IOJOB_SAVE, otherwise this
+ * field is populated by the I/O thread for REDIS_IOJOB_LOAD. */
+ time_t expire; /* Expire time for this key on REDIS_IOJOB_LOAD */
} iojob;
+/* IO operations scheduled -- check dscache.c for more info */
+typedef struct ioop {
+ int type;
+ redisDb *db;
+ robj *key;
+ time_t ctime; /* This is the creation time of the entry. */
+} ioop;
+
/* Structure to hold list iteration abstraction. */
typedef struct {
robj *subject;
off_t rdbSavedObjectPages(robj *o);
robj *rdbLoadObject(int type, FILE *fp);
void backgroundSaveDoneHandler(int statloc);
+int rdbSaveKeyValuePair(FILE *fp, redisDb *db, robj *key, robj *val, time_t now);
+int rdbLoadType(FILE *fp);
+time_t rdbLoadTime(FILE *fp);
+robj *rdbLoadStringObject(FILE *fp);
/* AOF persistence */
void flushAppendOnlyFile(void);
void oom(const char *msg);
void populateCommandTable(void);
-/* Virtual Memory */
-void vmInit(void);
-void vmMarkPagesFree(off_t page, off_t count);
-robj *vmLoadObject(robj *o);
-robj *vmPreviewObject(robj *o);
-int vmSwapOneObjectBlocking(void);
-int vmSwapOneObjectThreaded(void);
-int vmCanSwapOut(void);
+/* Disk store */
+int dsOpen(void);
+int dsClose(void);
+int dsSet(redisDb *db, robj *key, robj *val);
+robj *dsGet(redisDb *db, robj *key, time_t *expire);
+int dsDel(redisDb *db, robj *key);
+int dsExists(redisDb *db, robj *key);
+void dsFlushDb(int dbid);
+
+/* Disk Store Cache */
+void dsInit(void);
void vmThreadedIOCompletedJob(aeEventLoop *el, int fd, void *privdata, int mask);
-void vmCancelThreadedIOJob(robj *o);
void lockThreadedIO(void);
void unlockThreadedIO(void);
-int vmSwapObjectThreaded(robj *key, robj *val, redisDb *db);
void freeIOJob(iojob *j);
void queueIOJob(iojob *j);
-int vmWriteObjectOnSwap(robj *o, off_t page);
-robj *vmReadObjectFromSwap(off_t page, int type);
void waitEmptyIOJobsQueue(void);
-void vmReopenSwapFile(void);
-int vmFreePage(off_t page);
+void processAllPendingIOJobs(void);
void zunionInterBlockClientOnSwappedKeys(redisClient *c, struct redisCommand *cmd, int argc, robj **argv);
void execBlockClientOnSwappedKeys(redisClient *c, struct redisCommand *cmd, int argc, robj **argv);
int blockClientOnSwappedKeys(redisClient *c, struct redisCommand *cmd);
int dontWaitForSwappedKey(redisClient *c, robj *key);
void handleClientsBlockedOnSwappedKey(redisDb *db, robj *key);
-vmpointer *vmSwapObjectBlocking(robj *val);
+int cacheFreeOneEntry(void);
+void cacheScheduleIOAddFlag(redisDb *db, robj *key, long flag);
+void cacheScheduleIODelFlag(redisDb *db, robj *key, long flag);
+int cacheScheduleIOGetFlags(redisDb *db, robj *key);
+void cacheScheduleIO(redisDb *db, robj *key, int type);
+void cacheCron(void);
+int cacheKeyMayExist(redisDb *db, robj *key);
+void cacheSetKeyMayExist(redisDb *db, robj *key);
+void cacheSetKeyDoesNotExist(redisDb *db, robj *key);
/* Set data type */
robj *setTypeCreate(robj *value);
int dbDelete(redisDb *db, robj *key);
long long emptyDb();
int selectDb(redisClient *c, int id);
+void signalModifiedKey(redisDb *db, robj *key);
+void signalFlushedDb(int dbid);
/* Git SHA1 */
char *redisGitSHA1(void);
* SORT result is empty a new key is set and maybe the old content
* replaced. */
server.dirty += 1+outputlen;
- touchWatchedKey(c->db,storekey);
+ signalModifiedKey(c->db,storekey);
addReplyLongLong(c,outputlen);
}
hashTypeTryObjectEncoding(o,&c->argv[2], &c->argv[3]);
update = hashTypeSet(o,c->argv[2],c->argv[3]);
addReply(c, update ? shared.czero : shared.cone);
- touchWatchedKey(c->db,c->argv[1]);
+ signalModifiedKey(c->db,c->argv[1]);
server.dirty++;
}
hashTypeTryObjectEncoding(o,&c->argv[2], &c->argv[3]);
hashTypeSet(o,c->argv[2],c->argv[3]);
addReply(c, shared.cone);
- touchWatchedKey(c->db,c->argv[1]);
+ signalModifiedKey(c->db,c->argv[1]);
server.dirty++;
}
}
hashTypeSet(o,c->argv[i],c->argv[i+1]);
}
addReply(c, shared.ok);
- touchWatchedKey(c->db,c->argv[1]);
+ signalModifiedKey(c->db,c->argv[1]);
server.dirty++;
}
hashTypeSet(o,c->argv[2],new);
decrRefCount(new);
addReplyLongLong(c,value);
- touchWatchedKey(c->db,c->argv[1]);
+ signalModifiedKey(c->db,c->argv[1]);
server.dirty++;
}
if (hashTypeDelete(o,c->argv[2])) {
if (hashTypeLength(o) == 0) dbDelete(c->db,c->argv[1]);
addReply(c,shared.cone);
- touchWatchedKey(c->db,c->argv[1]);
+ signalModifiedKey(c->db,c->argv[1]);
server.dirty++;
} else {
addReply(c,shared.czero);
return;
}
if (handleClientsWaitingListPush(c,c->argv[1],c->argv[2])) {
- touchWatchedKey(c->db,c->argv[1]);
+ signalModifiedKey(c->db,c->argv[1]);
addReply(c,shared.cone);
return;
}
}
listTypePush(lobj,c->argv[2],where);
addReplyLongLong(c,listTypeLength(lobj));
- touchWatchedKey(c->db,c->argv[1]);
+ signalModifiedKey(c->db,c->argv[1]);
server.dirty++;
}
if (subject->encoding == REDIS_ENCODING_ZIPLIST &&
ziplistLen(subject->ptr) > server.list_max_ziplist_entries)
listTypeConvert(subject,REDIS_ENCODING_LINKEDLIST);
- touchWatchedKey(c->db,c->argv[1]);
+ signalModifiedKey(c->db,c->argv[1]);
server.dirty++;
} else {
/* Notify client of a failed insert */
}
} else {
listTypePush(subject,val,where);
- touchWatchedKey(c->db,c->argv[1]);
+ signalModifiedKey(c->db,c->argv[1]);
server.dirty++;
}
o->ptr = ziplistInsert(o->ptr,p,value->ptr,sdslen(value->ptr));
decrRefCount(value);
addReply(c,shared.ok);
- touchWatchedKey(c->db,c->argv[1]);
+ signalModifiedKey(c->db,c->argv[1]);
server.dirty++;
}
} else if (o->encoding == REDIS_ENCODING_LINKEDLIST) {
listNodeValue(ln) = value;
incrRefCount(value);
addReply(c,shared.ok);
- touchWatchedKey(c->db,c->argv[1]);
+ signalModifiedKey(c->db,c->argv[1]);
server.dirty++;
}
} else {
addReplyBulk(c,value);
decrRefCount(value);
if (listTypeLength(o) == 0) dbDelete(c->db,c->argv[1]);
- touchWatchedKey(c->db,c->argv[1]);
+ signalModifiedKey(c->db,c->argv[1]);
server.dirty++;
}
}
redisPanic("Unknown list encoding");
}
if (listTypeLength(o) == 0) dbDelete(c->db,c->argv[1]);
- touchWatchedKey(c->db,c->argv[1]);
+ signalModifiedKey(c->db,c->argv[1]);
server.dirty++;
addReply(c,shared.ok);
}
if (listTypeLength(subject) == 0) dbDelete(c->db,c->argv[1]);
addReplyLongLong(c,removed);
- if (removed) touchWatchedKey(c->db,c->argv[1]);
+ if (removed) signalModifiedKey(c->db,c->argv[1]);
}
/* This is the semantic of this command:
dstobj = createZiplistObject();
dbAdd(c->db,dstkey,dstobj);
} else {
- touchWatchedKey(c->db,dstkey);
+ signalModifiedKey(c->db,dstkey);
server.dirty++;
}
listTypePush(dstobj,value,REDIS_HEAD);
/* Delete the source list when it is empty */
if (listTypeLength(sobj) == 0) dbDelete(c->db,c->argv[1]);
- touchWatchedKey(c->db,c->argv[1]);
+ signalModifiedKey(c->db,c->argv[1]);
server.dirty++;
}
}
}
}
if (setTypeAdd(set,c->argv[2])) {
- touchWatchedKey(c->db,c->argv[1]);
+ signalModifiedKey(c->db,c->argv[1]);
server.dirty++;
addReply(c,shared.cone);
} else {
c->argv[2] = tryObjectEncoding(c->argv[2]);
if (setTypeRemove(set,c->argv[2])) {
if (setTypeSize(set) == 0) dbDelete(c->db,c->argv[1]);
- touchWatchedKey(c->db,c->argv[1]);
+ signalModifiedKey(c->db,c->argv[1]);
server.dirty++;
addReply(c,shared.cone);
} else {
/* Remove the src set from the database when empty */
if (setTypeSize(srcset) == 0) dbDelete(c->db,c->argv[1]);
- touchWatchedKey(c->db,c->argv[1]);
- touchWatchedKey(c->db,c->argv[2]);
+ signalModifiedKey(c->db,c->argv[1]);
+ signalModifiedKey(c->db,c->argv[2]);
server.dirty++;
/* Create the destination set when it doesn't exist */
setTypeRemove(set,ele);
}
if (setTypeSize(set) == 0) dbDelete(c->db,c->argv[1]);
- touchWatchedKey(c->db,c->argv[1]);
+ signalModifiedKey(c->db,c->argv[1]);
server.dirty++;
}
zfree(sets);
if (dstkey) {
if (dbDelete(c->db,dstkey)) {
- touchWatchedKey(c->db,dstkey);
+ signalModifiedKey(c->db,dstkey);
server.dirty++;
}
addReply(c,shared.czero);
decrRefCount(dstset);
addReply(c,shared.czero);
}
- touchWatchedKey(c->db,dstkey);
+ signalModifiedKey(c->db,dstkey);
server.dirty++;
} else {
setDeferredMultiBulkLength(c,replylen,cardinality);
decrRefCount(dstset);
addReply(c,shared.czero);
}
- touchWatchedKey(c->db,dstkey);
+ signalModifiedKey(c->db,dstkey);
server.dirty++;
}
zfree(sets);
} else {
incrRefCount(val);
}
- touchWatchedKey(c->db,key);
+ signalModifiedKey(c->db,key);
server.dirty++;
removeExpire(c->db,key);
if (expire) setExpire(c->db,key,time(NULL)+seconds);
c->argv[2] = tryObjectEncoding(c->argv[2]);
dbReplace(c->db,c->argv[1],c->argv[2]);
incrRefCount(c->argv[2]);
- touchWatchedKey(c->db,c->argv[1]);
+ signalModifiedKey(c->db,c->argv[1]);
server.dirty++;
removeExpire(c->db,c->argv[1]);
}
byteval &= ~(1 << bit);
byteval |= ((on & 0x1) << bit);
((char*)o->ptr)[byte] = byteval;
- touchWatchedKey(c->db,c->argv[1]);
+ signalModifiedKey(c->db,c->argv[1]);
server.dirty++;
addReply(c, bitval ? shared.cone : shared.czero);
}
if (sdslen(value) > 0) {
o->ptr = sdsgrowzero(o->ptr,offset+sdslen(value));
memcpy((char*)o->ptr+offset,value,sdslen(value));
- touchWatchedKey(c->db,c->argv[1]);
+ signalModifiedKey(c->db,c->argv[1]);
server.dirty++;
}
addReplyLongLong(c,sdslen(o->ptr));
dbReplace(c->db,c->argv[j],c->argv[j+1]);
incrRefCount(c->argv[j+1]);
removeExpire(c->db,c->argv[j]);
- touchWatchedKey(c->db,c->argv[j]);
+ signalModifiedKey(c->db,c->argv[j]);
}
server.dirty += (c->argc-1)/2;
addReply(c, nx ? shared.cone : shared.ok);
}
o = createStringObjectFromLongLong(value);
dbReplace(c->db,c->argv[1],o);
- touchWatchedKey(c->db,c->argv[1]);
+ signalModifiedKey(c->db,c->argv[1]);
server.dirty++;
addReply(c,shared.colon);
addReply(c,o);
o->ptr = sdscatlen(o->ptr,append->ptr,sdslen(append->ptr));
totlen = sdslen(o->ptr);
}
- touchWatchedKey(c->db,c->argv[1]);
+ signalModifiedKey(c->db,c->argv[1]);
server.dirty++;
addReplyLongLong(c,totlen);
}
de = dictFind(zs->dict,ele);
redisAssert(de != NULL);
dictGetEntryVal(de) = &znode->score;
- touchWatchedKey(c->db,c->argv[1]);
+ signalModifiedKey(c->db,c->argv[1]);
server.dirty++;
if (incr)
addReplyDouble(c,score);
/* Update the score in the current dict entry */
dictGetEntryVal(de) = &znode->score;
- touchWatchedKey(c->db,c->argv[1]);
+ signalModifiedKey(c->db,c->argv[1]);
server.dirty++;
}
if (incr)
dictDelete(zs->dict,c->argv[2]);
if (htNeedsResize(zs->dict)) dictResize(zs->dict);
if (dictSize(zs->dict) == 0) dbDelete(c->db,c->argv[1]);
- touchWatchedKey(c->db,c->argv[1]);
+ signalModifiedKey(c->db,c->argv[1]);
server.dirty++;
addReply(c,shared.cone);
}
deleted = zslDeleteRangeByScore(zs->zsl,range,zs->dict);
if (htNeedsResize(zs->dict)) dictResize(zs->dict);
if (dictSize(zs->dict) == 0) dbDelete(c->db,c->argv[1]);
- if (deleted) touchWatchedKey(c->db,c->argv[1]);
+ if (deleted) signalModifiedKey(c->db,c->argv[1]);
server.dirty += deleted;
addReplyLongLong(c,deleted);
}
deleted = zslDeleteRangeByRank(zs->zsl,start+1,end+1,zs->dict);
if (htNeedsResize(zs->dict)) dictResize(zs->dict);
if (dictSize(zs->dict) == 0) dbDelete(c->db,c->argv[1]);
- if (deleted) touchWatchedKey(c->db,c->argv[1]);
+ if (deleted) signalModifiedKey(c->db,c->argv[1]);
server.dirty += deleted;
addReplyLongLong(c, deleted);
}
}
if (dbDelete(c->db,dstkey)) {
- touchWatchedKey(c->db,dstkey);
+ signalModifiedKey(c->db,dstkey);
touched = 1;
server.dirty++;
}
if (dstzset->zsl->length) {
dbAdd(c->db,dstkey,dstobj);
addReplyLongLong(c, dstzset->zsl->length);
- if (!touched) touchWatchedKey(c->db,dstkey);
+ if (!touched) signalModifiedKey(c->db,dstkey);
server.dirty++;
} else {
decrRefCount(dstobj);
-#define REDIS_VERSION "2.1.8"
+#define REDIS_VERSION "2.3.0"
+++ /dev/null
-#include "redis.h"
-
-#include <fcntl.h>
-#include <pthread.h>
-#include <math.h>
-#include <signal.h>
-
-/* Virtual Memory is composed mainly of two subsystems:
- * - Blocking Virutal Memory
- * - Threaded Virtual Memory I/O
- * The two parts are not fully decoupled, but functions are split among two
- * different sections of the source code (delimited by comments) in order to
- * make more clear what functionality is about the blocking VM and what about
- * the threaded (not blocking) VM.
- *
- * Redis VM design:
- *
- * Redis VM is a blocking VM (one that blocks reading swapped values from
- * disk into memory when a value swapped out is needed in memory) that is made
- * unblocking by trying to examine the command argument vector in order to
- * load in background values that will likely be needed in order to exec
- * the command. The command is executed only once all the relevant keys
- * are loaded into memory.
- *
- * This basically is almost as simple of a blocking VM, but almost as parallel
- * as a fully non-blocking VM.
- */
-
-/* =================== Virtual Memory - Blocking Side ====================== */
-
-/* Create a VM pointer object. This kind of objects are used in place of
- * values in the key -> value hash table, for swapped out objects. */
-vmpointer *createVmPointer(int vtype) {
- vmpointer *vp = zmalloc(sizeof(vmpointer));
-
- vp->type = REDIS_VMPOINTER;
- vp->storage = REDIS_VM_SWAPPED;
- vp->vtype = vtype;
- return vp;
-}
-
-void vmInit(void) {
- off_t totsize;
- int pipefds[2];
- size_t stacksize;
- struct flock fl;
-
- if (server.vm_max_threads != 0)
- zmalloc_enable_thread_safeness(); /* we need thread safe zmalloc() */
-
- redisLog(REDIS_NOTICE,"Using '%s' as swap file",server.vm_swap_file);
- /* Try to open the old swap file, otherwise create it */
- if ((server.vm_fp = fopen(server.vm_swap_file,"r+b")) == NULL) {
- server.vm_fp = fopen(server.vm_swap_file,"w+b");
- }
- if (server.vm_fp == NULL) {
- redisLog(REDIS_WARNING,
- "Can't open the swap file: %s. Exiting.",
- strerror(errno));
- exit(1);
- }
- server.vm_fd = fileno(server.vm_fp);
- /* Lock the swap file for writing, this is useful in order to avoid
- * another instance to use the same swap file for a config error. */
- fl.l_type = F_WRLCK;
- fl.l_whence = SEEK_SET;
- fl.l_start = fl.l_len = 0;
- if (fcntl(server.vm_fd,F_SETLK,&fl) == -1) {
- redisLog(REDIS_WARNING,
- "Can't lock the swap file at '%s': %s. Make sure it is not used by another Redis instance.", server.vm_swap_file, strerror(errno));
- exit(1);
- }
- /* Initialize */
- server.vm_next_page = 0;
- server.vm_near_pages = 0;
- server.vm_stats_used_pages = 0;
- server.vm_stats_swapped_objects = 0;
- server.vm_stats_swapouts = 0;
- server.vm_stats_swapins = 0;
- totsize = server.vm_pages*server.vm_page_size;
- redisLog(REDIS_NOTICE,"Allocating %lld bytes of swap file",totsize);
- if (ftruncate(server.vm_fd,totsize) == -1) {
- redisLog(REDIS_WARNING,"Can't ftruncate swap file: %s. Exiting.",
- strerror(errno));
- exit(1);
- } else {
- redisLog(REDIS_NOTICE,"Swap file allocated with success");
- }
- server.vm_bitmap = zcalloc((server.vm_pages+7)/8);
- redisLog(REDIS_VERBOSE,"Allocated %lld bytes page table for %lld pages",
- (long long) (server.vm_pages+7)/8, server.vm_pages);
-
- /* Initialize threaded I/O (used by Virtual Memory) */
- server.io_newjobs = listCreate();
- server.io_processing = listCreate();
- server.io_processed = listCreate();
- server.io_ready_clients = listCreate();
- pthread_mutex_init(&server.io_mutex,NULL);
- pthread_mutex_init(&server.io_swapfile_mutex,NULL);
- server.io_active_threads = 0;
- if (pipe(pipefds) == -1) {
- redisLog(REDIS_WARNING,"Unable to intialized VM: pipe(2): %s. Exiting."
- ,strerror(errno));
- exit(1);
- }
- server.io_ready_pipe_read = pipefds[0];
- server.io_ready_pipe_write = pipefds[1];
- redisAssert(anetNonBlock(NULL,server.io_ready_pipe_read) != ANET_ERR);
- /* LZF requires a lot of stack */
- pthread_attr_init(&server.io_threads_attr);
- pthread_attr_getstacksize(&server.io_threads_attr, &stacksize);
-
- /* Solaris may report a stacksize of 0, let's set it to 1 otherwise
- * multiplying it by 2 in the while loop later will not really help ;) */
- if (!stacksize) stacksize = 1;
-
- while (stacksize < REDIS_THREAD_STACK_SIZE) stacksize *= 2;
- pthread_attr_setstacksize(&server.io_threads_attr, stacksize);
- /* Listen for events in the threaded I/O pipe */
- if (aeCreateFileEvent(server.el, server.io_ready_pipe_read, AE_READABLE,
- vmThreadedIOCompletedJob, NULL) == AE_ERR)
- oom("creating file event");
-}
-
-/* Mark the page as used */
-void vmMarkPageUsed(off_t page) {
- off_t byte = page/8;
- int bit = page&7;
- redisAssert(vmFreePage(page) == 1);
- server.vm_bitmap[byte] |= 1<<bit;
-}
-
-/* Mark N contiguous pages as used, with 'page' being the first. */
-void vmMarkPagesUsed(off_t page, off_t count) {
- off_t j;
-
- for (j = 0; j < count; j++)
- vmMarkPageUsed(page+j);
- server.vm_stats_used_pages += count;
- redisLog(REDIS_DEBUG,"Mark USED pages: %lld pages at %lld\n",
- (long long)count, (long long)page);
-}
-
-/* Mark the page as free */
-void vmMarkPageFree(off_t page) {
- off_t byte = page/8;
- int bit = page&7;
- redisAssert(vmFreePage(page) == 0);
- server.vm_bitmap[byte] &= ~(1<<bit);
-}
-
-/* Mark N contiguous pages as free, with 'page' being the first. */
-void vmMarkPagesFree(off_t page, off_t count) {
- off_t j;
-
- for (j = 0; j < count; j++)
- vmMarkPageFree(page+j);
- server.vm_stats_used_pages -= count;
- redisLog(REDIS_DEBUG,"Mark FREE pages: %lld pages at %lld\n",
- (long long)count, (long long)page);
-}
-
-/* Test if the page is free */
-int vmFreePage(off_t page) {
- off_t byte = page/8;
- int bit = page&7;
- return (server.vm_bitmap[byte] & (1<<bit)) == 0;
-}
-
-/* Find N contiguous free pages storing the first page of the cluster in *first.
- * Returns REDIS_OK if it was able to find N contiguous pages, otherwise
- * REDIS_ERR is returned.
- *
- * This function uses a simple algorithm: we try to allocate
- * REDIS_VM_MAX_NEAR_PAGES sequentially, when we reach this limit we start
- * again from the start of the swap file searching for free spaces.
- *
- * If it looks pretty clear that there are no free pages near our offset
- * we try to find less populated places doing a forward jump of
- * REDIS_VM_MAX_RANDOM_JUMP, then we start scanning again a few pages
- * without hurry, and then we jump again and so forth...
- *
- * This function can be improved using a free list to avoid to guess
- * too much, since we could collect data about freed pages.
- *
- * note: I implemented this function just after watching an episode of
- * Battlestar Galactica, where the hybrid was continuing to say "JUMP!"
- */
-int vmFindContiguousPages(off_t *first, off_t n) {
- off_t base, offset = 0, since_jump = 0, numfree = 0;
-
- if (server.vm_near_pages == REDIS_VM_MAX_NEAR_PAGES) {
- server.vm_near_pages = 0;
- server.vm_next_page = 0;
- }
- server.vm_near_pages++; /* Yet another try for pages near to the old ones */
- base = server.vm_next_page;
-
- while(offset < server.vm_pages) {
- off_t this = base+offset;
-
- /* If we overflow, restart from page zero */
- if (this >= server.vm_pages) {
- this -= server.vm_pages;
- if (this == 0) {
- /* Just overflowed, what we found on tail is no longer
- * interesting, as it's no longer contiguous. */
- numfree = 0;
- }
- }
- if (vmFreePage(this)) {
- /* This is a free page */
- numfree++;
- /* Already got N free pages? Return to the caller, with success */
- if (numfree == n) {
- *first = this-(n-1);
- server.vm_next_page = this+1;
- redisLog(REDIS_DEBUG, "FOUND CONTIGUOUS PAGES: %lld pages at %lld\n", (long long) n, (long long) *first);
- return REDIS_OK;
- }
- } else {
- /* The current one is not a free page */
- numfree = 0;
- }
-
- /* Fast-forward if the current page is not free and we already
- * searched enough near this place. */
- since_jump++;
- if (!numfree && since_jump >= REDIS_VM_MAX_RANDOM_JUMP/4) {
- offset += random() % REDIS_VM_MAX_RANDOM_JUMP;
- since_jump = 0;
- /* Note that even if we rewind after the jump, we are don't need
- * to make sure numfree is set to zero as we only jump *if* it
- * is set to zero. */
- } else {
- /* Otherwise just check the next page */
- offset++;
- }
- }
- return REDIS_ERR;
-}
-
-/* Write the specified object at the specified page of the swap file */
-int vmWriteObjectOnSwap(robj *o, off_t page) {
- if (server.vm_enabled) pthread_mutex_lock(&server.io_swapfile_mutex);
- if (fseeko(server.vm_fp,page*server.vm_page_size,SEEK_SET) == -1) {
- if (server.vm_enabled) pthread_mutex_unlock(&server.io_swapfile_mutex);
- redisLog(REDIS_WARNING,
- "Critical VM problem in vmWriteObjectOnSwap(): can't seek: %s",
- strerror(errno));
- return REDIS_ERR;
- }
- rdbSaveObject(server.vm_fp,o);
- fflush(server.vm_fp);
- if (server.vm_enabled) pthread_mutex_unlock(&server.io_swapfile_mutex);
- return REDIS_OK;
-}
-
-/* Transfers the 'val' object to disk. Store all the information
- * a 'vmpointer' object containing all the information needed to load the
- * object back later is returned.
- *
- * If we can't find enough contiguous empty pages to swap the object on disk
- * NULL is returned. */
-vmpointer *vmSwapObjectBlocking(robj *val) {
- off_t pages = rdbSavedObjectPages(val);
- off_t page;
- vmpointer *vp;
-
- redisAssert(val->storage == REDIS_VM_MEMORY);
- redisAssert(val->refcount == 1);
- if (vmFindContiguousPages(&page,pages) == REDIS_ERR) return NULL;
- if (vmWriteObjectOnSwap(val,page) == REDIS_ERR) return NULL;
-
- vp = createVmPointer(val->type);
- vp->page = page;
- vp->usedpages = pages;
- decrRefCount(val); /* Deallocate the object from memory. */
- vmMarkPagesUsed(page,pages);
- redisLog(REDIS_DEBUG,"VM: object %p swapped out at %lld (%lld pages)",
- (void*) val,
- (unsigned long long) page, (unsigned long long) pages);
- server.vm_stats_swapped_objects++;
- server.vm_stats_swapouts++;
- return vp;
-}
-
-robj *vmReadObjectFromSwap(off_t page, int type) {
- robj *o;
-
- if (server.vm_enabled) pthread_mutex_lock(&server.io_swapfile_mutex);
- if (fseeko(server.vm_fp,page*server.vm_page_size,SEEK_SET) == -1) {
- redisLog(REDIS_WARNING,
- "Unrecoverable VM problem in vmReadObjectFromSwap(): can't seek: %s",
- strerror(errno));
- _exit(1);
- }
- o = rdbLoadObject(type,server.vm_fp);
- if (o == NULL) {
- redisLog(REDIS_WARNING, "Unrecoverable VM problem in vmReadObjectFromSwap(): can't load object from swap file: %s", strerror(errno));
- _exit(1);
- }
- if (server.vm_enabled) pthread_mutex_unlock(&server.io_swapfile_mutex);
- return o;
-}
-
-/* Load the specified object from swap to memory.
- * The newly allocated object is returned.
- *
- * If preview is true the unserialized object is returned to the caller but
- * the pages are not marked as freed, nor the vp object is freed. */
-robj *vmGenericLoadObject(vmpointer *vp, int preview) {
- robj *val;
-
- redisAssert(vp->type == REDIS_VMPOINTER &&
- (vp->storage == REDIS_VM_SWAPPED || vp->storage == REDIS_VM_LOADING));
- val = vmReadObjectFromSwap(vp->page,vp->vtype);
- if (!preview) {
- redisLog(REDIS_DEBUG, "VM: object %p loaded from disk", (void*)vp);
- vmMarkPagesFree(vp->page,vp->usedpages);
- zfree(vp);
- server.vm_stats_swapped_objects--;
- } else {
- redisLog(REDIS_DEBUG, "VM: object %p previewed from disk", (void*)vp);
- }
- server.vm_stats_swapins++;
- return val;
-}
-
-/* Plain object loading, from swap to memory.
- *
- * 'o' is actually a redisVmPointer structure that will be freed by the call.
- * The return value is the loaded object. */
-robj *vmLoadObject(robj *o) {
- /* If we are loading the object in background, stop it, we
- * need to load this object synchronously ASAP. */
- if (o->storage == REDIS_VM_LOADING)
- vmCancelThreadedIOJob(o);
- return vmGenericLoadObject((vmpointer*)o,0);
-}
-
-/* Just load the value on disk, without to modify the key.
- * This is useful when we want to perform some operation on the value
- * without to really bring it from swap to memory, like while saving the
- * dataset or rewriting the append only log. */
-robj *vmPreviewObject(robj *o) {
- return vmGenericLoadObject((vmpointer*)o,1);
-}
-
-/* How a good candidate is this object for swapping?
- * The better candidate it is, the greater the returned value.
- *
- * Currently we try to perform a fast estimation of the object size in
- * memory, and combine it with aging informations.
- *
- * Basically swappability = idle-time * log(estimated size)
- *
- * Bigger objects are preferred over smaller objects, but not
- * proportionally, this is why we use the logarithm. This algorithm is
- * just a first try and will probably be tuned later. */
-double computeObjectSwappability(robj *o) {
- /* actual age can be >= minage, but not < minage. As we use wrapping
- * 21 bit clocks with minutes resolution for the LRU. */
- time_t minage = estimateObjectIdleTime(o);
- long asize = 0, elesize;
- robj *ele;
- list *l;
- listNode *ln;
- dict *d;
- struct dictEntry *de;
- int z;
-
- if (minage <= 0) return 0;
- switch(o->type) {
- case REDIS_STRING:
- if (o->encoding != REDIS_ENCODING_RAW) {
- asize = sizeof(*o);
- } else {
- asize = sdslen(o->ptr)+sizeof(*o)+sizeof(long)*2;
- }
- break;
- case REDIS_LIST:
- if (o->encoding == REDIS_ENCODING_ZIPLIST) {
- asize = sizeof(*o)+ziplistSize(o->ptr);
- } else {
- l = o->ptr;
- ln = listFirst(l);
- asize = sizeof(list);
- if (ln) {
- ele = ln->value;
- elesize = (ele->encoding == REDIS_ENCODING_RAW) ?
- (sizeof(*o)+sdslen(ele->ptr)) : sizeof(*o);
- asize += (sizeof(listNode)+elesize)*listLength(l);
- }
- }
- break;
- case REDIS_SET:
- case REDIS_ZSET:
- z = (o->type == REDIS_ZSET);
- d = z ? ((zset*)o->ptr)->dict : o->ptr;
-
- if (!z && o->encoding == REDIS_ENCODING_INTSET) {
- intset *is = o->ptr;
- asize = sizeof(*is)+is->encoding*is->length;
- } else {
- asize = sizeof(dict)+(sizeof(struct dictEntry*)*dictSlots(d));
- if (z) asize += sizeof(zset)-sizeof(dict);
- if (dictSize(d)) {
- de = dictGetRandomKey(d);
- ele = dictGetEntryKey(de);
- elesize = (ele->encoding == REDIS_ENCODING_RAW) ?
- (sizeof(*o)+sdslen(ele->ptr)) : sizeof(*o);
- asize += (sizeof(struct dictEntry)+elesize)*dictSize(d);
- if (z) asize += sizeof(zskiplistNode)*dictSize(d);
- }
- }
- break;
- case REDIS_HASH:
- if (o->encoding == REDIS_ENCODING_ZIPMAP) {
- unsigned char *p = zipmapRewind((unsigned char*)o->ptr);
- unsigned int len = zipmapLen((unsigned char*)o->ptr);
- unsigned int klen, vlen;
- unsigned char *key, *val;
-
- if ((p = zipmapNext(p,&key,&klen,&val,&vlen)) == NULL) {
- klen = 0;
- vlen = 0;
- }
- asize = len*(klen+vlen+3);
- } else if (o->encoding == REDIS_ENCODING_HT) {
- d = o->ptr;
- asize = sizeof(dict)+(sizeof(struct dictEntry*)*dictSlots(d));
- if (dictSize(d)) {
- de = dictGetRandomKey(d);
- ele = dictGetEntryKey(de);
- elesize = (ele->encoding == REDIS_ENCODING_RAW) ?
- (sizeof(*o)+sdslen(ele->ptr)) : sizeof(*o);
- ele = dictGetEntryVal(de);
- elesize = (ele->encoding == REDIS_ENCODING_RAW) ?
- (sizeof(*o)+sdslen(ele->ptr)) : sizeof(*o);
- asize += (sizeof(struct dictEntry)+elesize)*dictSize(d);
- }
- }
- break;
- }
- return (double)minage*log(1+asize);
-}
-
-/* Try to swap an object that's a good candidate for swapping.
- * Returns REDIS_OK if the object was swapped, REDIS_ERR if it's not possible
- * to swap any object at all.
- *
- * If 'usethreaded' is true, Redis will try to swap the object in background
- * using I/O threads. */
-int vmSwapOneObject(int usethreads) {
- int j, i;
- struct dictEntry *best = NULL;
- double best_swappability = 0;
- redisDb *best_db = NULL;
- robj *val;
- sds key;
-
- for (j = 0; j < server.dbnum; j++) {
- redisDb *db = server.db+j;
- /* Why maxtries is set to 100?
- * Because this way (usually) we'll find 1 object even if just 1% - 2%
- * are swappable objects */
- int maxtries = 100;
-
- if (dictSize(db->dict) == 0) continue;
- for (i = 0; i < 5; i++) {
- dictEntry *de;
- double swappability;
-
- if (maxtries) maxtries--;
- de = dictGetRandomKey(db->dict);
- val = dictGetEntryVal(de);
- /* Only swap objects that are currently in memory.
- *
- * Also don't swap shared objects: not a good idea in general and
- * we need to ensure that the main thread does not touch the
- * object while the I/O thread is using it, but we can't
- * control other keys without adding additional mutex. */
- if (val->storage != REDIS_VM_MEMORY || val->refcount != 1) {
- if (maxtries) i--; /* don't count this try */
- continue;
- }
- swappability = computeObjectSwappability(val);
- if (!best || swappability > best_swappability) {
- best = de;
- best_swappability = swappability;
- best_db = db;
- }
- }
- }
- if (best == NULL) return REDIS_ERR;
- key = dictGetEntryKey(best);
- val = dictGetEntryVal(best);
-
- redisLog(REDIS_DEBUG,"Key with best swappability: %s, %f",
- key, best_swappability);
-
- /* Swap it */
- if (usethreads) {
- robj *keyobj = createStringObject(key,sdslen(key));
- vmSwapObjectThreaded(keyobj,val,best_db);
- decrRefCount(keyobj);
- return REDIS_OK;
- } else {
- vmpointer *vp;
-
- if ((vp = vmSwapObjectBlocking(val)) != NULL) {
- dictGetEntryVal(best) = vp;
- return REDIS_OK;
- } else {
- return REDIS_ERR;
- }
- }
-}
-
-int vmSwapOneObjectBlocking() {
- return vmSwapOneObject(0);
-}
-
-int vmSwapOneObjectThreaded() {
- return vmSwapOneObject(1);
-}
-
-/* Return true if it's safe to swap out objects in a given moment.
- * Basically we don't want to swap objects out while there is a BGSAVE
- * or a BGAEOREWRITE running in backgroud. */
-int vmCanSwapOut(void) {
- return (server.bgsavechildpid == -1 && server.bgrewritechildpid == -1);
-}
-
-/* =================== Virtual Memory - Threaded I/O ======================= */
-
-void freeIOJob(iojob *j) {
- if ((j->type == REDIS_IOJOB_PREPARE_SWAP ||
- j->type == REDIS_IOJOB_DO_SWAP ||
- j->type == REDIS_IOJOB_LOAD) && j->val != NULL)
- {
- /* we fix the storage type, otherwise decrRefCount() will try to
- * kill the I/O thread Job (that does no longer exists). */
- if (j->val->storage == REDIS_VM_SWAPPING)
- j->val->storage = REDIS_VM_MEMORY;
- decrRefCount(j->val);
- }
- decrRefCount(j->key);
- zfree(j);
-}
-
-/* Every time a thread finished a Job, it writes a byte into the write side
- * of an unix pipe in order to "awake" the main thread, and this function
- * is called.
- *
- * Note that this is called both by the event loop, when a I/O thread
- * sends a byte in the notification pipe, and is also directly called from
- * waitEmptyIOJobsQueue().
- *
- * In the latter case we don't want to swap more, so we use the
- * "privdata" argument setting it to a not NULL value to signal this
- * condition. */
-void vmThreadedIOCompletedJob(aeEventLoop *el, int fd, void *privdata,
- int mask)
-{
- char buf[1];
- int retval, processed = 0, toprocess = -1, trytoswap = 1;
- REDIS_NOTUSED(el);
- REDIS_NOTUSED(mask);
- REDIS_NOTUSED(privdata);
-
- if (privdata != NULL) trytoswap = 0; /* check the comments above... */
-
- /* For every byte we read in the read side of the pipe, there is one
- * I/O job completed to process. */
- while((retval = read(fd,buf,1)) == 1) {
- iojob *j;
- listNode *ln;
- struct dictEntry *de;
-
- redisLog(REDIS_DEBUG,"Processing I/O completed job");
-
- /* Get the processed element (the oldest one) */
- lockThreadedIO();
- redisAssert(listLength(server.io_processed) != 0);
- if (toprocess == -1) {
- toprocess = (listLength(server.io_processed)*REDIS_MAX_COMPLETED_JOBS_PROCESSED)/100;
- if (toprocess <= 0) toprocess = 1;
- }
- ln = listFirst(server.io_processed);
- j = ln->value;
- listDelNode(server.io_processed,ln);
- unlockThreadedIO();
- /* If this job is marked as canceled, just ignore it */
- if (j->canceled) {
- freeIOJob(j);
- continue;
- }
- /* Post process it in the main thread, as there are things we
- * can do just here to avoid race conditions and/or invasive locks */
- redisLog(REDIS_DEBUG,"COMPLETED Job type: %d, ID %p, key: %s", j->type, (void*)j->id, (unsigned char*)j->key->ptr);
- de = dictFind(j->db->dict,j->key->ptr);
- redisAssert(de != NULL);
- if (j->type == REDIS_IOJOB_LOAD) {
- redisDb *db;
- vmpointer *vp = dictGetEntryVal(de);
-
- /* Key loaded, bring it at home */
- vmMarkPagesFree(vp->page,vp->usedpages);
- redisLog(REDIS_DEBUG, "VM: object %s loaded from disk (threaded)",
- (unsigned char*) j->key->ptr);
- server.vm_stats_swapped_objects--;
- server.vm_stats_swapins++;
- dictGetEntryVal(de) = j->val;
- incrRefCount(j->val);
- db = j->db;
- /* Handle clients waiting for this key to be loaded. */
- handleClientsBlockedOnSwappedKey(db,j->key);
- freeIOJob(j);
- zfree(vp);
- } else if (j->type == REDIS_IOJOB_PREPARE_SWAP) {
- /* Now we know the amount of pages required to swap this object.
- * Let's find some space for it, and queue this task again
- * rebranded as REDIS_IOJOB_DO_SWAP. */
- if (!vmCanSwapOut() ||
- vmFindContiguousPages(&j->page,j->pages) == REDIS_ERR)
- {
- /* Ooops... no space or we can't swap as there is
- * a fork()ed Redis trying to save stuff on disk. */
- j->val->storage = REDIS_VM_MEMORY; /* undo operation */
- freeIOJob(j);
- } else {
- /* Note that we need to mark this pages as used now,
- * if the job will be canceled, we'll mark them as freed
- * again. */
- vmMarkPagesUsed(j->page,j->pages);
- j->type = REDIS_IOJOB_DO_SWAP;
- lockThreadedIO();
- queueIOJob(j);
- unlockThreadedIO();
- }
- } else if (j->type == REDIS_IOJOB_DO_SWAP) {
- vmpointer *vp;
-
- /* Key swapped. We can finally free some memory. */
- if (j->val->storage != REDIS_VM_SWAPPING) {
- vmpointer *vp = (vmpointer*) j->id;
- printf("storage: %d\n",vp->storage);
- printf("key->name: %s\n",(char*)j->key->ptr);
- printf("val: %p\n",(void*)j->val);
- printf("val->type: %d\n",j->val->type);
- printf("val->ptr: %s\n",(char*)j->val->ptr);
- }
- redisAssert(j->val->storage == REDIS_VM_SWAPPING);
- vp = createVmPointer(j->val->type);
- vp->page = j->page;
- vp->usedpages = j->pages;
- dictGetEntryVal(de) = vp;
- /* Fix the storage otherwise decrRefCount will attempt to
- * remove the associated I/O job */
- j->val->storage = REDIS_VM_MEMORY;
- decrRefCount(j->val);
- redisLog(REDIS_DEBUG,
- "VM: object %s swapped out at %lld (%lld pages) (threaded)",
- (unsigned char*) j->key->ptr,
- (unsigned long long) j->page, (unsigned long long) j->pages);
- server.vm_stats_swapped_objects++;
- server.vm_stats_swapouts++;
- freeIOJob(j);
- /* Put a few more swap requests in queue if we are still
- * out of memory */
- if (trytoswap && vmCanSwapOut() &&
- zmalloc_used_memory() > server.vm_max_memory)
- {
- int more = 1;
- while(more) {
- lockThreadedIO();
- more = listLength(server.io_newjobs) <
- (unsigned) server.vm_max_threads;
- unlockThreadedIO();
- /* Don't waste CPU time if swappable objects are rare. */
- if (vmSwapOneObjectThreaded() == REDIS_ERR) {
- trytoswap = 0;
- break;
- }
- }
- }
- }
- processed++;
- if (processed == toprocess) return;
- }
- if (retval < 0 && errno != EAGAIN) {
- redisLog(REDIS_WARNING,
- "WARNING: read(2) error in vmThreadedIOCompletedJob() %s",
- strerror(errno));
- }
-}
-
-void lockThreadedIO(void) {
- pthread_mutex_lock(&server.io_mutex);
-}
-
-void unlockThreadedIO(void) {
- pthread_mutex_unlock(&server.io_mutex);
-}
-
-/* Remove the specified object from the threaded I/O queue if still not
- * processed, otherwise make sure to flag it as canceled. */
-void vmCancelThreadedIOJob(robj *o) {
- list *lists[3] = {
- server.io_newjobs, /* 0 */
- server.io_processing, /* 1 */
- server.io_processed /* 2 */
- };
- int i;
-
- redisAssert(o->storage == REDIS_VM_LOADING || o->storage == REDIS_VM_SWAPPING);
-again:
- lockThreadedIO();
- /* Search for a matching object in one of the queues */
- for (i = 0; i < 3; i++) {
- listNode *ln;
- listIter li;
-
- listRewind(lists[i],&li);
- while ((ln = listNext(&li)) != NULL) {
- iojob *job = ln->value;
-
- if (job->canceled) continue; /* Skip this, already canceled. */
- if (job->id == o) {
- redisLog(REDIS_DEBUG,"*** CANCELED %p (key %s) (type %d) (LIST ID %d)\n",
- (void*)job, (char*)job->key->ptr, job->type, i);
- /* Mark the pages as free since the swap didn't happened
- * or happened but is now discarded. */
- if (i != 1 && job->type == REDIS_IOJOB_DO_SWAP)
- vmMarkPagesFree(job->page,job->pages);
- /* Cancel the job. It depends on the list the job is
- * living in. */
- switch(i) {
- case 0: /* io_newjobs */
- /* If the job was yet not processed the best thing to do
- * is to remove it from the queue at all */
- freeIOJob(job);
- listDelNode(lists[i],ln);
- break;
- case 1: /* io_processing */
- /* Oh Shi- the thread is messing with the Job:
- *
- * Probably it's accessing the object if this is a
- * PREPARE_SWAP or DO_SWAP job.
- * If it's a LOAD job it may be reading from disk and
- * if we don't wait for the job to terminate before to
- * cancel it, maybe in a few microseconds data can be
- * corrupted in this pages. So the short story is:
- *
- * Better to wait for the job to move into the
- * next queue (processed)... */
-
- /* We try again and again until the job is completed. */
- unlockThreadedIO();
- /* But let's wait some time for the I/O thread
- * to finish with this job. After all this condition
- * should be very rare. */
- usleep(1);
- goto again;
- case 2: /* io_processed */
- /* The job was already processed, that's easy...
- * just mark it as canceled so that we'll ignore it
- * when processing completed jobs. */
- job->canceled = 1;
- break;
- }
- /* Finally we have to adjust the storage type of the object
- * in order to "UNDO" the operaiton. */
- if (o->storage == REDIS_VM_LOADING)
- o->storage = REDIS_VM_SWAPPED;
- else if (o->storage == REDIS_VM_SWAPPING)
- o->storage = REDIS_VM_MEMORY;
- unlockThreadedIO();
- redisLog(REDIS_DEBUG,"*** DONE");
- return;
- }
- }
- }
- unlockThreadedIO();
- printf("Not found: %p\n", (void*)o);
- redisAssert(1 != 1); /* We should never reach this */
-}
-
-void *IOThreadEntryPoint(void *arg) {
- iojob *j;
- listNode *ln;
- REDIS_NOTUSED(arg);
-
- pthread_detach(pthread_self());
- while(1) {
- /* Get a new job to process */
- lockThreadedIO();
- if (listLength(server.io_newjobs) == 0) {
- /* No new jobs in queue, exit. */
- redisLog(REDIS_DEBUG,"Thread %ld exiting, nothing to do",
- (long) pthread_self());
- server.io_active_threads--;
- unlockThreadedIO();
- return NULL;
- }
- ln = listFirst(server.io_newjobs);
- j = ln->value;
- listDelNode(server.io_newjobs,ln);
- /* Add the job in the processing queue */
- j->thread = pthread_self();
- listAddNodeTail(server.io_processing,j);
- ln = listLast(server.io_processing); /* We use ln later to remove it */
- unlockThreadedIO();
- redisLog(REDIS_DEBUG,"Thread %ld got a new job (type %d): %p about key '%s'",
- (long) pthread_self(), j->type, (void*)j, (char*)j->key->ptr);
-
- /* Process the Job */
- if (j->type == REDIS_IOJOB_LOAD) {
- vmpointer *vp = (vmpointer*)j->id;
- j->val = vmReadObjectFromSwap(j->page,vp->vtype);
- } else if (j->type == REDIS_IOJOB_PREPARE_SWAP) {
- j->pages = rdbSavedObjectPages(j->val);
- } else if (j->type == REDIS_IOJOB_DO_SWAP) {
- if (vmWriteObjectOnSwap(j->val,j->page) == REDIS_ERR)
- j->canceled = 1;
- }
-
- /* Done: insert the job into the processed queue */
- redisLog(REDIS_DEBUG,"Thread %ld completed the job: %p (key %s)",
- (long) pthread_self(), (void*)j, (char*)j->key->ptr);
- lockThreadedIO();
- listDelNode(server.io_processing,ln);
- listAddNodeTail(server.io_processed,j);
- unlockThreadedIO();
-
- /* Signal the main thread there is new stuff to process */
- redisAssert(write(server.io_ready_pipe_write,"x",1) == 1);
- }
- return NULL; /* never reached */
-}
-
-void spawnIOThread(void) {
- pthread_t thread;
- sigset_t mask, omask;
- int err;
-
- sigemptyset(&mask);
- sigaddset(&mask,SIGCHLD);
- sigaddset(&mask,SIGHUP);
- sigaddset(&mask,SIGPIPE);
- pthread_sigmask(SIG_SETMASK, &mask, &omask);
- while ((err = pthread_create(&thread,&server.io_threads_attr,IOThreadEntryPoint,NULL)) != 0) {
- redisLog(REDIS_WARNING,"Unable to spawn an I/O thread: %s",
- strerror(err));
- usleep(1000000);
- }
- pthread_sigmask(SIG_SETMASK, &omask, NULL);
- server.io_active_threads++;
-}
-
-/* We need to wait for the last thread to exit before we are able to
- * fork() in order to BGSAVE or BGREWRITEAOF. */
-void waitEmptyIOJobsQueue(void) {
- while(1) {
- int io_processed_len;
-
- lockThreadedIO();
- if (listLength(server.io_newjobs) == 0 &&
- listLength(server.io_processing) == 0 &&
- server.io_active_threads == 0)
- {
- unlockThreadedIO();
- return;
- }
- /* While waiting for empty jobs queue condition we post-process some
- * finshed job, as I/O threads may be hanging trying to write against
- * the io_ready_pipe_write FD but there are so much pending jobs that
- * it's blocking. */
- io_processed_len = listLength(server.io_processed);
- unlockThreadedIO();
- if (io_processed_len) {
- vmThreadedIOCompletedJob(NULL,server.io_ready_pipe_read,
- (void*)0xdeadbeef,0);
- usleep(1000); /* 1 millisecond */
- } else {
- usleep(10000); /* 10 milliseconds */
- }
- }
-}
-
-void vmReopenSwapFile(void) {
- /* Note: we don't close the old one as we are in the child process
- * and don't want to mess at all with the original file object. */
- server.vm_fp = fopen(server.vm_swap_file,"r+b");
- if (server.vm_fp == NULL) {
- redisLog(REDIS_WARNING,"Can't re-open the VM swap file: %s. Exiting.",
- server.vm_swap_file);
- _exit(1);
- }
- server.vm_fd = fileno(server.vm_fp);
-}
-
-/* This function must be called while with threaded IO locked */
-void queueIOJob(iojob *j) {
- redisLog(REDIS_DEBUG,"Queued IO Job %p type %d about key '%s'\n",
- (void*)j, j->type, (char*)j->key->ptr);
- listAddNodeTail(server.io_newjobs,j);
- if (server.io_active_threads < server.vm_max_threads)
- spawnIOThread();
-}
-
-int vmSwapObjectThreaded(robj *key, robj *val, redisDb *db) {
- iojob *j;
-
- j = zmalloc(sizeof(*j));
- j->type = REDIS_IOJOB_PREPARE_SWAP;
- j->db = db;
- j->key = key;
- incrRefCount(key);
- j->id = j->val = val;
- incrRefCount(val);
- j->canceled = 0;
- j->thread = (pthread_t) -1;
- val->storage = REDIS_VM_SWAPPING;
-
- lockThreadedIO();
- queueIOJob(j);
- unlockThreadedIO();
- return REDIS_OK;
-}
-
-/* ============ Virtual Memory - Blocking clients on missing keys =========== */
-
-/* This function makes the clinet 'c' waiting for the key 'key' to be loaded.
- * If there is not already a job loading the key, it is craeted.
- * The key is added to the io_keys list in the client structure, and also
- * in the hash table mapping swapped keys to waiting clients, that is,
- * server.io_waited_keys. */
-int waitForSwappedKey(redisClient *c, robj *key) {
- struct dictEntry *de;
- robj *o;
- list *l;
-
- /* If the key does not exist or is already in RAM we don't need to
- * block the client at all. */
- de = dictFind(c->db->dict,key->ptr);
- if (de == NULL) return 0;
- o = dictGetEntryVal(de);
- if (o->storage == REDIS_VM_MEMORY) {
- return 0;
- } else if (o->storage == REDIS_VM_SWAPPING) {
- /* We were swapping the key, undo it! */
- vmCancelThreadedIOJob(o);
- return 0;
- }
-
- /* OK: the key is either swapped, or being loaded just now. */
-
- /* Add the key to the list of keys this client is waiting for.
- * This maps clients to keys they are waiting for. */
- listAddNodeTail(c->io_keys,key);
- incrRefCount(key);
-
- /* Add the client to the swapped keys => clients waiting map. */
- de = dictFind(c->db->io_keys,key);
- if (de == NULL) {
- int retval;
-
- /* For every key we take a list of clients blocked for it */
- l = listCreate();
- retval = dictAdd(c->db->io_keys,key,l);
- incrRefCount(key);
- redisAssert(retval == DICT_OK);
- } else {
- l = dictGetEntryVal(de);
- }
- listAddNodeTail(l,c);
-
- /* Are we already loading the key from disk? If not create a job */
- if (o->storage == REDIS_VM_SWAPPED) {
- iojob *j;
- vmpointer *vp = (vmpointer*)o;
-
- o->storage = REDIS_VM_LOADING;
- j = zmalloc(sizeof(*j));
- j->type = REDIS_IOJOB_LOAD;
- j->db = c->db;
- j->id = (robj*)vp;
- j->key = key;
- incrRefCount(key);
- j->page = vp->page;
- j->val = NULL;
- j->canceled = 0;
- j->thread = (pthread_t) -1;
- lockThreadedIO();
- queueIOJob(j);
- unlockThreadedIO();
- }
- return 1;
-}
-
-/* Preload keys for any command with first, last and step values for
- * the command keys prototype, as defined in the command table. */
-void waitForMultipleSwappedKeys(redisClient *c, struct redisCommand *cmd, int argc, robj **argv) {
- int j, last;
- if (cmd->vm_firstkey == 0) return;
- last = cmd->vm_lastkey;
- if (last < 0) last = argc+last;
- for (j = cmd->vm_firstkey; j <= last; j += cmd->vm_keystep) {
- redisAssert(j < argc);
- waitForSwappedKey(c,argv[j]);
- }
-}
-
-/* Preload keys needed for the ZUNIONSTORE and ZINTERSTORE commands.
- * Note that the number of keys to preload is user-defined, so we need to
- * apply a sanity check against argc. */
-void zunionInterBlockClientOnSwappedKeys(redisClient *c, struct redisCommand *cmd, int argc, robj **argv) {
- int i, num;
- REDIS_NOTUSED(cmd);
-
- num = atoi(argv[2]->ptr);
- if (num > (argc-3)) return;
- for (i = 0; i < num; i++) {
- waitForSwappedKey(c,argv[3+i]);
- }
-}
-
-/* Preload keys needed to execute the entire MULTI/EXEC block.
- *
- * This function is called by blockClientOnSwappedKeys when EXEC is issued,
- * and will block the client when any command requires a swapped out value. */
-void execBlockClientOnSwappedKeys(redisClient *c, struct redisCommand *cmd, int argc, robj **argv) {
- int i, margc;
- struct redisCommand *mcmd;
- robj **margv;
- REDIS_NOTUSED(cmd);
- REDIS_NOTUSED(argc);
- REDIS_NOTUSED(argv);
-
- if (!(c->flags & REDIS_MULTI)) return;
- for (i = 0; i < c->mstate.count; i++) {
- mcmd = c->mstate.commands[i].cmd;
- margc = c->mstate.commands[i].argc;
- margv = c->mstate.commands[i].argv;
-
- if (mcmd->vm_preload_proc != NULL) {
- mcmd->vm_preload_proc(c,mcmd,margc,margv);
- } else {
- waitForMultipleSwappedKeys(c,mcmd,margc,margv);
- }
- }
-}
-
-/* Is this client attempting to run a command against swapped keys?
- * If so, block it ASAP, load the keys in background, then resume it.
- *
- * The important idea about this function is that it can fail! If keys will
- * still be swapped when the client is resumed, this key lookups will
- * just block loading keys from disk. In practical terms this should only
- * happen with SORT BY command or if there is a bug in this function.
- *
- * Return 1 if the client is marked as blocked, 0 if the client can
- * continue as the keys it is going to access appear to be in memory. */
-int blockClientOnSwappedKeys(redisClient *c, struct redisCommand *cmd) {
- if (cmd->vm_preload_proc != NULL) {
- cmd->vm_preload_proc(c,cmd,c->argc,c->argv);
- } else {
- waitForMultipleSwappedKeys(c,cmd,c->argc,c->argv);
- }
-
- /* If the client was blocked for at least one key, mark it as blocked. */
- if (listLength(c->io_keys)) {
- c->flags |= REDIS_IO_WAIT;
- aeDeleteFileEvent(server.el,c->fd,AE_READABLE);
- server.vm_blocked_clients++;
- return 1;
- } else {
- return 0;
- }
-}
-
-/* Remove the 'key' from the list of blocked keys for a given client.
- *
- * The function returns 1 when there are no longer blocking keys after
- * the current one was removed (and the client can be unblocked). */
-int dontWaitForSwappedKey(redisClient *c, robj *key) {
- list *l;
- listNode *ln;
- listIter li;
- struct dictEntry *de;
-
- /* The key object might be destroyed when deleted from the c->io_keys
- * list (and the "key" argument is physically the same object as the
- * object inside the list), so we need to protect it. */
- incrRefCount(key);
-
- /* Remove the key from the list of keys this client is waiting for. */
- listRewind(c->io_keys,&li);
- while ((ln = listNext(&li)) != NULL) {
- if (equalStringObjects(ln->value,key)) {
- listDelNode(c->io_keys,ln);
- break;
- }
- }
- redisAssert(ln != NULL);
-
- /* Remove the client form the key => waiting clients map. */
- de = dictFind(c->db->io_keys,key);
- redisAssert(de != NULL);
- l = dictGetEntryVal(de);
- ln = listSearchKey(l,c);
- redisAssert(ln != NULL);
- listDelNode(l,ln);
- if (listLength(l) == 0)
- dictDelete(c->db->io_keys,key);
-
- decrRefCount(key);
- return listLength(c->io_keys) == 0;
-}
-
-/* Every time we now a key was loaded back in memory, we handle clients
- * waiting for this key if any. */
-void handleClientsBlockedOnSwappedKey(redisDb *db, robj *key) {
- struct dictEntry *de;
- list *l;
- listNode *ln;
- int len;
-
- de = dictFind(db->io_keys,key);
- if (!de) return;
-
- l = dictGetEntryVal(de);
- len = listLength(l);
- /* Note: we can't use something like while(listLength(l)) as the list
- * can be freed by the calling function when we remove the last element. */
- while (len--) {
- ln = listFirst(l);
- redisClient *c = ln->value;
-
- if (dontWaitForSwappedKey(c,key)) {
- /* Put the client in the list of clients ready to go as we
- * loaded all the keys about it. */
- listAddNodeTail(server.io_ready_clients,c);
- }
- }
-}
# Also the Append Only File will be created inside this directory.
#
# Note that you must specify a directory here, not a file name.
-dir ./test/tmp
+dir /tmp
################################# REPLICATION #################################
# To enable VM just set 'vm-enabled' to yes, and set the following three
# VM parameters accordingly to your needs.
-vm-enabled no
-# vm-enabled yes
-
-# This is the path of the Redis swap file. As you can guess, swap files
-# can't be shared by different Redis instances, so make sure to use a swap
-# file for every redis process you are running. Redis will complain if the
-# swap file is already in use.
-#
-# The best kind of storage for the Redis swap file (that's accessed at random)
-# is a Solid State Disk (SSD).
-#
-# *** WARNING *** if you are using a shared hosting the default of putting
-# the swap file under /tmp is not secure. Create a dir with access granted
-# only to Redis user and configure Redis to create the swap file there.
-vm-swap-file redis.swap
-
-# vm-max-memory configures the VM to use at max the specified amount of
-# RAM. Everything that deos not fit will be swapped on disk *if* possible, that
-# is, if there is still enough contiguous space in the swap file.
-#
-# With vm-max-memory 0 the system will swap everything it can. Not a good
-# default, just specify the max amount of RAM you can in bytes, but it's
-# better to leave some margin. For instance specify an amount of RAM
-# that's more or less between 60 and 80% of your free RAM.
-vm-max-memory 0
-
-# Redis swap files is split into pages. An object can be saved using multiple
-# contiguous pages, but pages can't be shared between different objects.
-# So if your page is too big, small objects swapped out on disk will waste
-# a lot of space. If you page is too small, there is less space in the swap
-# file (assuming you configured the same number of total swap file pages).
-#
-# If you use a lot of small objects, use a page size of 64 or 32 bytes.
-# If you use a lot of big objects, use a bigger page size.
-# If unsure, use the default :)
-vm-page-size 32
-
-# Number of total memory pages in the swap file.
-# Given that the page table (a bitmap of free/used pages) is taken in memory,
-# every 8 pages on disk will consume 1 byte of RAM.
-#
-# The total swap size is vm-page-size * vm-pages
-#
-# 32M swap should be enough for testing.
-vm-pages 1048576
-
-# Max number of VM I/O threads running at the same time.
-# This threads are used to read/write data from/to swap file, since they
-# also encode and decode objects from disk to memory or the reverse, a bigger
-# number of threads can help with big objects even if they can't help with
-# I/O itself as the physical device may not be able to couple with many
-# reads/writes operations at the same time.
-#
-# The special value of 0 turn off threaded I/O and enables the blocking
-# Virtual Memory implementation.
-vm-max-threads 4
+diskstore-enabled yes
+diskstore-path redis.ds
+cache-max-memory 0
+cache-flush-delay 0
############################### ADVANCED CONFIG ###############################
if {$::valgrind} {
exec valgrind --suppressions=src/valgrind.sup src/redis-server $config_file > $stdout 2> $stderr &
} else {
+ if {$::verbose} {
+ puts "Logging on $stdout / $stderr"
+ }
exec src/redis-server $config_file > $stdout 2> $stderr &
}
set ::port 16379
set ::traceleaks 0
set ::valgrind 0
-set ::verbose 0
+set ::verbose 1
set ::denytags {}
set ::allowtags {}
set ::external 0; # If "1" this means, we are running against external instance