*/
#include "redis.h"
+#include "slowlog.h"
#ifdef HAVE_BACKTRACE
#include <execinfo.h>
#include <limits.h>
#include <float.h>
#include <math.h>
-#include <pthread.h>
#include <sys/resource.h>
/* Our shared "common" objects */
{"sdiff",sdiffCommand,-2,REDIS_CMD_DENYOOM,NULL,1,-1,1,0,0},
{"sdiffstore",sdiffstoreCommand,-3,REDIS_CMD_DENYOOM,NULL,2,-1,1,0,0},
{"smembers",sinterCommand,2,0,NULL,1,1,1,0,0},
- {"zadd",zaddCommand,4,REDIS_CMD_DENYOOM,NULL,1,1,1,0,0},
+ {"zadd",zaddCommand,-4,REDIS_CMD_DENYOOM,NULL,1,1,1,0,0},
{"zincrby",zincrbyCommand,4,REDIS_CMD_DENYOOM,NULL,1,1,1,0,0},
- {"zrem",zremCommand,3,0,NULL,1,1,1,0,0},
+ {"zrem",zremCommand,-3,0,NULL,1,1,1,0,0},
{"zremrangebyscore",zremrangebyscoreCommand,4,0,NULL,1,1,1,0,0},
{"zremrangebyrank",zremrangebyrankCommand,4,0,NULL,1,1,1,0,0},
{"zunionstore",zunionstoreCommand,-4,REDIS_CMD_DENYOOM,zunionInterGetKeys,0,0,0,0,0},
{"migrate",migrateCommand,6,0,NULL,0,0,0,0,0},
{"dump",dumpCommand,2,0,NULL,0,0,0,0,0},
{"object",objectCommand,-2,0,NULL,0,0,0,0,0},
- {"client",clientCommand,-2,0,NULL,0,0,0,0,0}
+ {"client",clientCommand,-2,0,NULL,0,0,0,0,0},
+ {"slowlog",slowlogCommand,-2,0,NULL,0,0,0,0,0}
};
/*============================ Utility functions ============================ */
if ((server.maxidletime && !(loops % 100)) || server.bpop_blocked_clients)
closeTimedoutClients();
+ /* Start a scheduled AOF rewrite if this was requested by the user while
+ * a BGSAVE was in progress. */
+ if (server.bgsavechildpid == -1 && server.bgrewritechildpid == -1 &&
+ server.aofrewrite_scheduled)
+ {
+ rewriteAppendOnlyFileBackground();
+ }
+
/* Check if a background saving or AOF rewrite in progress terminated. */
if (server.bgsavechildpid != -1 || server.bgrewritechildpid != -1) {
int statloc;
}
updateDictResizePolicy();
}
- } else if (server.bgsavethread != (pthread_t) -1) {
- if (server.bgsavethread != (pthread_t) -1) {
- int state;
-
- pthread_mutex_lock(&server.bgsavethread_mutex);
- state = server.bgsavethread_state;
- pthread_mutex_unlock(&server.bgsavethread_mutex);
-
- if (state == REDIS_BGSAVE_THREAD_DONE_OK ||
- state == REDIS_BGSAVE_THREAD_DONE_ERR)
- {
- backgroundSaveDoneHandler(
- (state == REDIS_BGSAVE_THREAD_DONE_OK) ? 0 : 1, 0);
- }
- }
- } else if (!server.ds_enabled) {
- /* If there is not a background saving in progress check if
- * we have to save now */
+ } else {
time_t now = time(NULL);
+
+ /* If there is not a background saving/rewrite in progress check if
+ * we have to save/rewrite now */
for (j = 0; j < server.saveparamslen; j++) {
struct saveparam *sp = server.saveparams+j;
break;
}
}
+
+ /* Trigger an AOF rewrite if needed */
+ if (server.bgsavechildpid == -1 &&
+ server.bgrewritechildpid == -1 &&
+ server.auto_aofrewrite_perc &&
+ server.appendonly_current_size > server.auto_aofrewrite_min_size)
+ {
+ int base = server.auto_aofrewrite_base_size ?
+ server.auto_aofrewrite_base_size : 1;
+ long long growth = (server.appendonly_current_size*100/base) - 100;
+ if (growth >= server.auto_aofrewrite_perc) {
+ redisLog(REDIS_NOTICE,"Starting automatic rewriting of AOF on %lld%% growth",growth);
+ rewriteAppendOnlyFileBackground();
+ }
+ }
}
/* Expire a few keys per cycle, only if this is a master.
* in order to guarantee a strict consistency. */
if (server.masterhost == NULL) activeExpireCycle();
- /* 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. */
if (!(loops % 10)) replicationCron();
listNode *ln;
redisClient *c;
- /* 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);
- while((ln = listNext(&li))) {
- c = ln->value;
- struct redisCommand *cmd;
-
- /* Resume the client. */
- listDelNode(server.io_ready_clients,ln);
- c->flags &= (~REDIS_IO_WAIT);
- server.cache_blocked_clients--;
- aeCreateFileEvent(server.el, c->fd, AE_READABLE,
- readQueryFromClient, c);
- cmd = lookupCommand(c->argv[0]->ptr);
- redisAssert(cmd != NULL);
- call(c,cmd);
- resetClient(c);
- /* There may be more data to process in the input buffer. */
- if (c->querybuf && sdslen(c->querybuf) > 0)
- processInputBuffer(c);
- }
- }
-
/* Try to process pending commands for clients that were just unblocked. */
while (listLength(server.unblocked_clients)) {
ln = listFirst(server.unblocked_clients);
server.appendonly = 0;
server.appendfsync = APPENDFSYNC_EVERYSEC;
server.no_appendfsync_on_rewrite = 0;
+ server.auto_aofrewrite_perc = REDIS_AUTO_AOFREWRITE_PERC;
+ server.auto_aofrewrite_min_size = REDIS_AUTO_AOFREWRITE_MIN_SIZE;
+ server.auto_aofrewrite_base_size = 0;
+ server.aofrewrite_scheduled = 0;
server.lastfsync = time(NULL);
server.appendfd = -1;
server.appendseldb = -1; /* Make sure the first time will not match */
server.maxmemory = 0;
server.maxmemory_policy = REDIS_MAXMEMORY_VOLATILE_LRU;
server.maxmemory_samples = 3;
- 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.zset_max_ziplist_entries = REDIS_ZSET_MAX_ZIPLIST_ENTRIES;
server.zset_max_ziplist_value = REDIS_ZSET_MAX_ZIPLIST_VALUE;
server.shutdown_asap = 0;
- server.cache_flush_delay = 0;
server.cluster_enabled = 0;
server.cluster.configfile = zstrdup("nodes.conf");
server.masterport = 6379;
server.master = NULL;
server.replstate = REDIS_REPL_NONE;
+ server.repl_syncio_timeout = REDIS_REPL_SYNCIO_TIMEOUT;
server.repl_serve_stale_data = 1;
+ server.repl_down_since = -1;
/* Double constants initialization */
R_Zero = 0.0;
populateCommandTable();
server.delCommand = lookupCommandByCString("del");
server.multiCommand = lookupCommandByCString("multi");
+
+ /* Slow log */
+ server.slowlog_log_slower_than = REDIS_SLOWLOG_LOG_SLOWER_THAN;
+ server.slowlog_max_len = REDIS_SLOWLOG_MAX_LEN;
}
void initServer() {
server.syslog_facility);
}
- server.mainthread = pthread_self();
server.clients = listCreate();
server.slaves = listCreate();
server.monitors = listCreate();
server.unblocked_clients = listCreate();
- server.cache_io_queue = listCreate();
createSharedObjects();
server.el = aeCreateEventLoop();
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.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);
server.cronloops = 0;
server.bgsavechildpid = -1;
server.bgrewritechildpid = -1;
- server.bgsavethread_state = REDIS_BGSAVE_THREAD_UNACTIVE;
- server.bgsavethread = (pthread_t) -1;
server.bgrewritebuf = sdsempty();
server.aofbuf = sdsempty();
server.lastsave = time(NULL);
server.stat_keyspace_misses = 0;
server.stat_keyspace_hits = 0;
server.stat_peak_memory = 0;
+ server.stat_fork_time = 0;
server.unixtime = time(NULL);
aeCreateTimeEvent(server.el, 1, serverCron, NULL, NULL);
if (server.ipfd > 0 && aeCreateFileEvent(server.el,server.ipfd,AE_READABLE,
}
}
- if (server.ds_enabled) dsInit();
if (server.cluster_enabled) clusterInit();
+ slowlogInit();
srand(time(NULL)^getpid());
}
}
/* Call() is the core of Redis execution of a command */
-void call(redisClient *c, struct redisCommand *cmd) {
- long long dirty, start = ustime();
+void call(redisClient *c) {
+ long long dirty, start = ustime(), duration;
dirty = server.dirty;
- cmd->proc(c);
+ c->cmd->proc(c);
dirty = server.dirty-dirty;
- cmd->microseconds += ustime()-start;
- cmd->calls++;
+ duration = ustime()-start;
+ c->cmd->microseconds += duration;
+ slowlogPushEntryIfNeeded(c->argv,c->argc,duration);
+ c->cmd->calls++;
if (server.appendonly && dirty)
- feedAppendOnlyFile(cmd,c->db->id,c->argv,c->argc);
- if ((dirty || cmd->flags & REDIS_CMD_FORCE_REPLICATION) &&
+ feedAppendOnlyFile(c->cmd,c->db->id,c->argv,c->argc);
+ if ((dirty || c->cmd->flags & REDIS_CMD_FORCE_REPLICATION) &&
listLength(server.slaves))
replicationFeedSlaves(server.slaves,c->db->id,c->argv,c->argc);
if (listLength(server.monitors))
* and other operations can be performed by the caller. Otherwise
* if 0 is returned the client was destroied (i.e. after QUIT). */
int processCommand(redisClient *c) {
- struct redisCommand *cmd;
-
/* The QUIT command is handled separately. Normal command procs will
* go through checking for replication and QUIT will cause trouble
* when FORCE_REPLICATION is enabled and would be implemented in
}
/* Now lookup the command and check ASAP about trivial error conditions
- * such wrong arity, bad command name and so forth. */
- cmd = lookupCommand(c->argv[0]->ptr);
- if (!cmd) {
+ * such as wrong arity, bad command name and so forth. */
+ c->cmd = lookupCommand(c->argv[0]->ptr);
+ if (!c->cmd) {
addReplyErrorFormat(c,"unknown command '%s'",
(char*)c->argv[0]->ptr);
return REDIS_OK;
- } else if ((cmd->arity > 0 && cmd->arity != c->argc) ||
- (c->argc < -cmd->arity)) {
+ } else if ((c->cmd->arity > 0 && c->cmd->arity != c->argc) ||
+ (c->argc < -c->cmd->arity)) {
addReplyErrorFormat(c,"wrong number of arguments for '%s' command",
- cmd->name);
+ c->cmd->name);
return REDIS_OK;
}
/* Check if the user is authenticated */
- if (server.requirepass && !c->authenticated && cmd->proc != authCommand) {
+ if (server.requirepass && !c->authenticated && c->cmd->proc != authCommand)
+ {
addReplyError(c,"operation not permitted");
return REDIS_OK;
}
/* If cluster is enabled, redirect here */
if (server.cluster_enabled &&
- !(cmd->getkeys_proc == NULL && cmd->firstkey == 0)) {
+ !(c->cmd->getkeys_proc == NULL && c->cmd->firstkey == 0)) {
int hashslot;
if (server.cluster.state != REDIS_CLUSTER_OK) {
return REDIS_OK;
} else {
int ask;
- clusterNode *n = getNodeByQuery(c,cmd,c->argv,c->argc,&hashslot,&ask);
+ clusterNode *n = getNodeByQuery(c,c->cmd,c->argv,c->argc,&hashslot,&ask);
if (n == NULL) {
addReplyError(c,"Multi keys request invalid in cluster");
return REDIS_OK;
* keys in the dataset). If there are not the only thing we can do
* is returning an error. */
if (server.maxmemory) freeMemoryIfNeeded();
- if (server.maxmemory && (cmd->flags & REDIS_CMD_DENYOOM) &&
+ if (server.maxmemory && (c->cmd->flags & REDIS_CMD_DENYOOM) &&
zmalloc_used_memory() > server.maxmemory)
{
addReplyError(c,"command not allowed when used memory > 'maxmemory'");
/* Only allow SUBSCRIBE and UNSUBSCRIBE in the context of Pub/Sub */
if ((dictSize(c->pubsub_channels) > 0 || listLength(c->pubsub_patterns) > 0)
&&
- cmd->proc != subscribeCommand && cmd->proc != unsubscribeCommand &&
- cmd->proc != psubscribeCommand && cmd->proc != punsubscribeCommand) {
+ c->cmd->proc != subscribeCommand &&
+ c->cmd->proc != unsubscribeCommand &&
+ c->cmd->proc != psubscribeCommand &&
+ c->cmd->proc != punsubscribeCommand) {
addReplyError(c,"only (P)SUBSCRIBE / (P)UNSUBSCRIBE / QUIT allowed in this context");
return REDIS_OK;
}
* we are a slave with a broken link with master. */
if (server.masterhost && server.replstate != REDIS_REPL_CONNECTED &&
server.repl_serve_stale_data == 0 &&
- cmd->proc != infoCommand && cmd->proc != slaveofCommand)
+ c->cmd->proc != infoCommand && c->cmd->proc != slaveofCommand)
{
addReplyError(c,
"link with MASTER is down and slave-serve-stale-data is set to no");
}
/* Loading DB? Return an error if the command is not INFO */
- if (server.loading && cmd->proc != infoCommand) {
+ if (server.loading && c->cmd->proc != infoCommand) {
addReply(c, shared.loadingerr);
return REDIS_OK;
}
/* Exec the command */
if (c->flags & REDIS_MULTI &&
- cmd->proc != execCommand && cmd->proc != discardCommand &&
- cmd->proc != multiCommand && cmd->proc != watchCommand)
+ c->cmd->proc != execCommand && c->cmd->proc != discardCommand &&
+ c->cmd->proc != multiCommand && c->cmd->proc != watchCommand)
{
- queueMultiCommand(c,cmd);
+ queueMultiCommand(c);
addReply(c,shared.queued);
} else {
- if (server.ds_enabled && blockClientOnSwappedKeys(c,cmd))
- return REDIS_ERR;
- call(c,cmd);
+ call(c);
}
return REDIS_OK;
}
kill(server.bgsavechildpid,SIGKILL);
rdbRemoveTempFile(server.bgsavechildpid);
}
- if (server.ds_enabled) {
- /* FIXME: flush all objects on disk */
- } else if (server.appendonly) {
+ if (server.appendonly) {
/* Append only file: fsync() the AOF and exit */
aof_fsync(server.appendfd);
} else if (server.saveparamslen > 0) {
server.stat_peak_memory,
peak_hmem,
zmalloc_get_fragmentation_ratio(),
- REDIS_MALLOC
+ ZMALLOC_LIB
);
}
- /* Allocation statistics */
- if (allsections || !strcasecmp(section,"allocstats")) {
- if (sections++) info = sdscat(info,"\r\n");
- info = sdscat(info, "# Allocstats\r\nallocation_stats:");
- for (j = 0; j <= ZMALLOC_MAX_ALLOC_STAT; j++) {
- size_t count = zmalloc_allocations_for_size(j);
- if (count) {
- if (info[sdslen(info)-1] != ':') info = sdscatlen(info,",",1);
- info = sdscatprintf(info,"%s%d=%zu",
- (j == ZMALLOC_MAX_ALLOC_STAT) ? ">=" : "",
- j,count);
- }
- }
- info = sdscat(info,"\r\n");
- }
-
/* Persistence */
if (allsections || defsections || !strcasecmp(section,"persistence")) {
if (sections++) info = sdscat(info,"\r\n");
server.loading,
server.appendonly,
server.dirty,
- server.bgsavechildpid != -1 ||
- server.bgsavethread != (pthread_t) -1,
+ server.bgsavechildpid != -1,
server.lastsave,
server.bgrewritechildpid != -1);
+ if (server.appendonly) {
+ info = sdscatprintf(info,
+ "aof_current_size:%lld\r\n"
+ "aof_base_size:%lld\r\n"
+ "aof_pending_rewrite:%d\r\n",
+ (long long) server.appendonly_current_size,
+ (long long) server.auto_aofrewrite_base_size,
+ server.aofrewrite_scheduled);
+ }
+
if (server.loading) {
double perc;
time_t eta, elapsed;
}
}
- /* Diskstore */
- if (allsections || defsections || !strcasecmp(section,"diskstore")) {
- if (sections++) info = sdscat(info,"\r\n");
- info = sdscatprintf(info,
- "# Diskstore\r\n"
- "ds_enabled:%d\r\n",
- server.ds_enabled != 0);
- if (server.ds_enabled) {
- lockThreadedIO();
- info = sdscatprintf(info,
- "cache_max_memory:%llu\r\n"
- "cache_blocked_clients:%lu\r\n"
- "cache_io_queue_len:%lu\r\n"
- "cache_io_jobs_new:%lu\r\n"
- "cache_io_jobs_processing:%lu\r\n"
- "cache_io_jobs_processed:%lu\r\n"
- "cache_io_ready_clients:%lu\r\n"
- ,(unsigned long long) server.cache_max_memory,
- (unsigned long) server.cache_blocked_clients,
- (unsigned long) listLength(server.cache_io_queue),
- (unsigned long) listLength(server.io_newjobs),
- (unsigned long) listLength(server.io_processing),
- (unsigned long) listLength(server.io_processed),
- (unsigned long) listLength(server.io_ready_clients)
- );
- unlockThreadedIO();
- }
- }
-
/* Stats */
if (allsections || defsections || !strcasecmp(section,"stats")) {
if (sections++) info = sdscat(info,"\r\n");
"keyspace_hits:%lld\r\n"
"keyspace_misses:%lld\r\n"
"pubsub_channels:%ld\r\n"
- "pubsub_patterns:%u\r\n",
+ "pubsub_patterns:%u\r\n"
+ "latest_fork_usec:%lld\r\n",
server.stat_numconnections,
server.stat_numcommands,
server.stat_expiredkeys,
server.stat_keyspace_hits,
server.stat_keyspace_misses,
dictSize(server.pubsub_channels),
- listLength(server.pubsub_patterns));
+ listLength(server.pubsub_patterns),
+ server.stat_fork_time);
}
/* Replication */
(int)(time(NULL)-server.repl_transfer_lastio)
);
}
+
+ if (server.replstate != REDIS_REPL_CONNECTED) {
+ info = sdscatprintf(info,
+ "master_link_down_since_seconds:%ld\r\n",
+ (long)time(NULL)-server.repl_down_since);
+ }
}
info = sdscatprintf(info,
"connected_slaves:%d\r\n",
"# CPU\r\n"
"used_cpu_sys:%.2f\r\n"
"used_cpu_user:%.2f\r\n"
- "used_cpu_sys_childrens:%.2f\r\n"
- "used_cpu_user_childrens:%.2f\r\n",
+ "used_cpu_sys_children:%.2f\r\n"
+ "used_cpu_user_children:%.2f\r\n",
(float)self_ru.ru_utime.tv_sec+(float)self_ru.ru_utime.tv_usec/1000000,
(float)self_ru.ru_stime.tv_sec+(float)self_ru.ru_stime.tv_usec/1000000,
(float)c_ru.ru_utime.tv_sec+(float)c_ru.ru_utime.tv_usec/1000000,
linuxOvercommitMemoryWarning();
#endif
start = ustime();
- if (server.ds_enabled) {
- redisLog(REDIS_NOTICE,"DB not loaded (running with disk back end)");
- } else if (server.appendonly) {
+ if (server.appendonly) {
if (loadAppendOnlyFile(server.appendfilename) == REDIS_OK)
redisLog(REDIS_NOTICE,"DB loaded from append only file: %.3f seconds",(float)(ustime()-start)/1000000);
} else {