#include <ctype.h>
#include <arpa/inet.h>
#include <sys/socket.h>
+#include <sys/wait.h>
extern char **environ;
#define SRI_RECONF_SENT (1<<11) /* SLAVEOF <newmaster> sent. */
#define SRI_RECONF_INPROG (1<<12) /* Slave synchronization in progress. */
#define SRI_RECONF_DONE (1<<13) /* Slave synchronized with new master. */
+#define SRI_FORCE_FAILOVER (1<<14) /* Force failover with master up. */
+#define SRI_SCRIPT_KILL_SENT (1<<15) /* SCRIPT KILL already sent on -BUSY */
#define SENTINEL_INFO_PERIOD 10000
#define SENTINEL_PING_PERIOD 1000
/* Generic flags that can be used with different functions. */
#define SENTINEL_NO_FLAGS 0
#define SENTINEL_GENERATE_EVENT 1
+#define SENTINEL_LEADER 2
+#define SENTINEL_OBSERVER 4
+
+/* Script execution flags and limits. */
+#define SENTINEL_SCRIPT_NONE 0
+#define SENTINEL_SCRIPT_RUNNING 1
+#define SENTINEL_SCRIPT_MAX_QUEUE 256
+#define SENTINEL_SCRIPT_MAX_RUNNING 16
+#define SENTINEL_SCRIPT_MAX_RUNTIME 60000 /* 60 seconds max exec time. */
+#define SENTINEL_SCRIPT_MAX_RETRY 10
+#define SENTINEL_SCRIPT_RETRY_DELAY 30000 /* 30 seconds between retries. */
typedef struct sentinelRedisInstance {
int flags; /* See SRI_... defines */
Key is the instance name, value is the
sentinelRedisInstance structure pointer. */
int tilt; /* Are we in TILT mode? */
+ int running_scripts; /* Number of scripts in execution right now. */
mstime_t tilt_start_time; /* When TITL started. */
mstime_t previous_time; /* Time last time we ran the time handler. */
+ list *scripts_queue; /* Queue of user scripts to execute. */
} sentinel;
+/* A script execution job. */
+typedef struct sentinelScriptJob {
+ int flags; /* Script job flags: SENTINEL_SCRIPT_* */
+ int retry_num; /* Number of times we tried to execute it. */
+ char **argv; /* Arguments to call the script. */
+ mstime_t start_time; /* Script execution time if the script is running,
+ otherwise 0 if we are allowed to retry the
+ execution at any time. If the script is not
+ running and it's not 0, it means: do not run
+ before the specified time. */
+ pid_t pid; /* Script execution pid. */
+} sentinelScriptJob;
+
/* ======================= hiredis ae.c adapters =============================
* Note: this implementation is taken from hiredis/adapters/ae.h, however
* we have our modified copy for Sentinel in order to use our allocator
void sentinelAbortFailover(sentinelRedisInstance *ri);
void sentinelEvent(int level, char *type, sentinelRedisInstance *ri, const char *fmt, ...);
sentinelRedisInstance *sentinelSelectSlave(sentinelRedisInstance *master);
+void sentinelScheduleScriptExecution(char *path, ...);
+void sentinelStartFailover(sentinelRedisInstance *master, int state);
/* ========================= Dictionary types =============================== */
/* =========================== Initialization =============================== */
void sentinelCommand(redisClient *c);
+void sentinelInfoCommand(redisClient *c);
struct redisCommand sentinelcmds[] = {
{"ping",pingCommand,1,"",0,NULL,0,0,0,0,0},
{"subscribe",subscribeCommand,-2,"",0,NULL,0,0,0,0,0},
{"unsubscribe",unsubscribeCommand,-1,"",0,NULL,0,0,0,0,0},
{"psubscribe",psubscribeCommand,-2,"",0,NULL,0,0,0,0,0},
- {"punsubscribe",punsubscribeCommand,-1,"",0,NULL,0,0,0,0,0}
+ {"punsubscribe",punsubscribeCommand,-1,"",0,NULL,0,0,0,0,0},
+ {"info",sentinelInfoCommand,-1,"",0,NULL,0,0,0,0,0}
};
/* This function overwrites a few normal Redis config default with Sentinel
sentinel.tilt = 0;
sentinel.tilt_start_time = mstime();
sentinel.previous_time = mstime();
+ sentinel.running_scripts = 0;
+ sentinel.scripts_queue = listCreate();
}
/* ============================== sentinelAddr ============================== */
/* =========================== Events notification ========================== */
-void sentinelCallNotificationScript(char *scriptpath, char *type, char *msg) {
- pid_t pid = fork();
-
- if (pid == -1) {
- /* Parent on error. */
- sentinelEvent(REDIS_WARNING,"-notification-script-error",NULL,
- "#can't fork: %s",strerror(errno));
- return;
- } else if (pid == 0) {
- /* Child */
- char *argv[4];
-
- argv[0] = scriptpath;
- argv[1] = type;
- argv[2] = msg;
- argv[3] = NULL;
- execve(scriptpath,argv,environ);
- /* If we are here an error occurred. */
- sentinelEvent(REDIS_WARNING,"-notification-script-error",NULL,
- "#execve(2): %s",strerror(errno));
- _exit(1);
- } else {
- sentinelEvent(REDIS_DEBUG,"+child",NULL,"%ld",(long)pid);
- }
-}
-
/* Send an event to log, pub/sub, user notification script.
*
* 'level' is the log level for logging. Only REDIS_WARNING events will trigger
sentinelRedisInstance *master = (ri->flags & SRI_MASTER) ?
ri : ri->master;
if (master->notification_script) {
- sentinelCallNotificationScript(master->notification_script,
- type,msg);
+ sentinelScheduleScriptExecution(master->notification_script,
+ type,msg,NULL);
+ }
+ }
+}
+
+/* ============================ script execution ============================ */
+
+/* Release a script job structure and all the associated data. */
+void sentinelReleaseScriptJob(sentinelScriptJob *sj) {
+ int j = 0;
+
+ while(sj->argv[j]) sdsfree(sj->argv[j++]);
+ zfree(sj->argv);
+ zfree(sj);
+}
+
+#define SENTINEL_SCRIPT_MAX_ARGS 16
+void sentinelScheduleScriptExecution(char *path, ...) {
+ va_list ap;
+ char *argv[SENTINEL_SCRIPT_MAX_ARGS+1];
+ int argc = 1;
+ sentinelScriptJob *sj;
+
+ va_start(ap, path);
+ while(argc < SENTINEL_SCRIPT_MAX_ARGS) {
+ argv[argc] = va_arg(ap,char*);
+ if (!argv[argc]) break;
+ argv[argc] = sdsnew(argv[argc]); /* Copy the string. */
+ argc++;
+ }
+ va_end(ap);
+ argv[0] = sdsnew(path);
+
+ sj = zmalloc(sizeof(*sj));
+ sj->flags = SENTINEL_SCRIPT_NONE;
+ sj->retry_num = 0;
+ sj->argv = zmalloc(sizeof(char*)*(argc+1));
+ sj->start_time = 0;
+ sj->pid = 0;
+ memcpy(sj->argv,argv,sizeof(char*)*(argc+1));
+
+ listAddNodeTail(sentinel.scripts_queue,sj);
+
+ /* Remove the oldest non running script if we already hit the limit. */
+ if (listLength(sentinel.scripts_queue) > SENTINEL_SCRIPT_MAX_QUEUE) {
+ listNode *ln;
+ listIter li;
+
+ listRewind(sentinel.scripts_queue,&li);
+ while ((ln = listNext(&li)) != NULL) {
+ sj = ln->value;
+
+ if (sj->flags & SENTINEL_SCRIPT_RUNNING) continue;
+ /* The first node is the oldest as we add on tail. */
+ listDelNode(sentinel.scripts_queue,ln);
+ sentinelReleaseScriptJob(sj);
+ break;
+ }
+ redisAssert(listLength(sentinel.scripts_queue) <=
+ SENTINEL_SCRIPT_MAX_QUEUE);
+ }
+}
+
+/* Lookup a script in the scripts queue via pid, and returns the list node
+ * (so that we can easily remove it from the queue if needed). */
+listNode *sentinelGetScriptListNodeByPid(pid_t pid) {
+ listNode *ln;
+ listIter li;
+
+ listRewind(sentinel.scripts_queue,&li);
+ while ((ln = listNext(&li)) != NULL) {
+ sentinelScriptJob *sj = ln->value;
+
+ if ((sj->flags & SENTINEL_SCRIPT_RUNNING) && sj->pid == pid)
+ return ln;
+ }
+ return NULL;
+}
+
+/* Run pending scripts if we are not already at max number of running
+ * scripts. */
+void sentinelRunPendingScripts(void) {
+ listNode *ln;
+ listIter li;
+ mstime_t now = mstime();
+
+ /* Find jobs that are not running and run them, from the top to the
+ * tail of the queue, so we run older jobs first. */
+ listRewind(sentinel.scripts_queue,&li);
+ while (sentinel.running_scripts < SENTINEL_SCRIPT_MAX_RUNNING &&
+ (ln = listNext(&li)) != NULL)
+ {
+ sentinelScriptJob *sj = ln->value;
+ pid_t pid;
+
+ /* Skip if already running. */
+ if (sj->flags & SENTINEL_SCRIPT_RUNNING) continue;
+
+ /* Skip if it's a retry, but not enough time has elapsed. */
+ if (sj->start_time && sj->start_time > now) continue;
+
+ sj->flags |= SENTINEL_SCRIPT_RUNNING;
+ sj->start_time = mstime();
+ sj->retry_num++;
+ pid = fork();
+
+ if (pid == -1) {
+ /* Parent (fork error).
+ * We report fork errors as signal 99, in order to unify the
+ * reporting with other kind of errors. */
+ sentinelEvent(REDIS_WARNING,"-script-error",NULL,
+ "%s %d %d", sj->argv[0], 99, 0);
+ sj->flags &= ~SENTINEL_SCRIPT_RUNNING;
+ sj->pid = 0;
+ } else if (pid == 0) {
+ /* Child */
+ execve(sj->argv[0],sj->argv,environ);
+ /* If we are here an error occurred. */
+ _exit(2); /* Don't retry execution. */
+ } else {
+ sentinel.running_scripts++;
+ sj->pid = pid;
+ sentinelEvent(REDIS_DEBUG,"+script-child",NULL,"%ld",(long)pid);
+ }
+ }
+}
+
+/* How much to delay the execution of a script that we need to retry after
+ * an error?
+ *
+ * We double the retry delay for every further retry we do. So for instance
+ * if RETRY_DELAY is set to 30 seconds and the max number of retries is 10
+ * starting from the second attempt to execute the script the delays are:
+ * 30 sec, 60 sec, 2 min, 4 min, 8 min, 16 min, 32 min, 64 min, 128 min. */
+mstime_t sentinelScriptRetryDelay(int retry_num) {
+ mstime_t delay = SENTINEL_SCRIPT_RETRY_DELAY;
+
+ while (retry_num-- > 1) delay *= 2;
+ return delay;
+}
+
+/* Check for scripts that terminated, and remove them from the queue if the
+ * script terminated successfully. If instead the script was terminated by
+ * a signal, or returned exit code "1", it is scheduled to run again if
+ * the max number of retries did not already elapsed. */
+void sentinelCollectTerminatedScripts(void) {
+ int statloc;
+ pid_t pid;
+
+ while ((pid = wait3(&statloc,WNOHANG,NULL)) > 0) {
+ int exitcode = WEXITSTATUS(statloc);
+ int bysignal = 0;
+ listNode *ln;
+ sentinelScriptJob *sj;
+
+ if (WIFSIGNALED(statloc)) bysignal = WTERMSIG(statloc);
+ sentinelEvent(REDIS_DEBUG,"-script-child",NULL,"%ld %d %d",
+ (long)pid, exitcode, bysignal);
+
+ ln = sentinelGetScriptListNodeByPid(pid);
+ if (ln == NULL) {
+ redisLog(REDIS_WARNING,"wait3() returned a pid (%ld) we can't find in our scripts execution queue!", (long)pid);
+ continue;
+ }
+ sj = ln->value;
+
+ /* If the script was terminated by a signal or returns an
+ * exit code of "1" (that means: please retry), we reschedule it
+ * if the max number of retries is not already reached. */
+ if ((bysignal || exitcode == 1) &&
+ sj->retry_num != SENTINEL_SCRIPT_MAX_RETRY)
+ {
+ sj->flags &= ~SENTINEL_SCRIPT_RUNNING;
+ sj->pid = 0;
+ sj->start_time = mstime() +
+ sentinelScriptRetryDelay(sj->retry_num);
+ } else {
+ /* Otherwise let's remove the script, but log the event if the
+ * execution did not terminated in the best of the ways. */
+ if (bysignal || exitcode != 0) {
+ sentinelEvent(REDIS_WARNING,"-script-error",NULL,
+ "%s %d %d", sj->argv[0], bysignal, exitcode);
+ }
+ listDelNode(sentinel.scripts_queue,ln);
+ sentinelReleaseScriptJob(sj);
+ sentinel.running_scripts--;
+ }
+ }
+}
+
+/* Kill scripts in timeout, they'll be collected by the
+ * sentinelCollectTerminatedScripts() function. */
+void sentinelKillTimedoutScripts(void) {
+ listNode *ln;
+ listIter li;
+ mstime_t now = mstime();
+
+ listRewind(sentinel.scripts_queue,&li);
+ while ((ln = listNext(&li)) != NULL) {
+ sentinelScriptJob *sj = ln->value;
+
+ if (sj->flags & SENTINEL_SCRIPT_RUNNING &&
+ (now - sj->start_time) > SENTINEL_SCRIPT_MAX_RUNTIME)
+ {
+ sentinelEvent(REDIS_WARNING,"-script-timeout",NULL,"%s %ld",
+ sj->argv[0], (long)sj->pid);
+ kill(sj->pid,SIGKILL);
+ }
+ }
+}
+
+/* Implements SENTINEL PENDING-SCRIPTS command. */
+void sentinelPendingScriptsCommand(redisClient *c) {
+ listNode *ln;
+ listIter li;
+
+ addReplyMultiBulkLen(c,listLength(sentinel.scripts_queue));
+ listRewind(sentinel.scripts_queue,&li);
+ while ((ln = listNext(&li)) != NULL) {
+ sentinelScriptJob *sj = ln->value;
+ int j = 0;
+
+ addReplyMultiBulkLen(c,10);
+
+ addReplyBulkCString(c,"argv");
+ while (sj->argv[j]) j++;
+ addReplyMultiBulkLen(c,j);
+ j = 0;
+ while (sj->argv[j]) addReplyBulkCString(c,sj->argv[j++]);
+
+ addReplyBulkCString(c,"flags");
+ addReplyBulkCString(c,
+ (sj->flags & SENTINEL_SCRIPT_RUNNING) ? "running" : "scheduled");
+
+ addReplyBulkCString(c,"pid");
+ addReplyBulkLongLong(c,sj->pid);
+
+ if (sj->flags & SENTINEL_SCRIPT_RUNNING) {
+ addReplyBulkCString(c,"run-time");
+ addReplyBulkLongLong(c,mstime() - sj->start_time);
+ } else {
+ mstime_t delay = sj->start_time ? (sj->start_time-mstime()) : 0;
+ if (delay < 0) delay = 0;
+ addReplyBulkCString(c,"run-delay");
+ addReplyBulkLongLong(c,delay);
}
+
+ addReplyBulkCString(c,"retry-num");
+ addReplyBulkLongLong(c,sj->retry_num);
}
}
+/* This function calls, if any, the client reconfiguration script with the
+ * following parameters:
+ *
+ * <master-name> <role> <state> <from-ip> <from-port> <to-ip> <to-port>
+ *
+ * It is called every time a failover starts, ends, or is aborted.
+ *
+ * <state> is "start", "end" or "abort".
+ * <role> is either "leader" or "observer".
+ *
+ * from/to fields are respectively master -> promoted slave addresses for
+ * "start" and "end", or the reverse (promoted slave -> master) in case of
+ * "abort".
+ */
+void sentinelCallClientReconfScript(sentinelRedisInstance *master, int role, char *state, sentinelAddr *from, sentinelAddr *to) {
+ char fromport[32], toport[32];
+
+ if (master->client_reconfig_script == NULL) return;
+ ll2string(fromport,sizeof(fromport),from->port);
+ ll2string(toport,sizeof(toport),to->port);
+ sentinelScheduleScriptExecution(master->client_reconfig_script,
+ master->name,
+ (role == SENTINEL_LEADER) ? "leader" : "observer",
+ state, from->ip, fromport, to->ip, toport, NULL);
+}
+
/* ========================== sentinelRedisInstance ========================= */
/* Create a redis instance, the following fields must be populated by the
sentinelRedisInstance *createSentinelRedisInstance(char *name, int flags, char *hostname, int port, int quorum, sentinelRedisInstance *master) {
sentinelRedisInstance *ri;
sentinelAddr *addr;
- dict *table;
+ dict *table = NULL;
char slavename[128], *sdsname;
redisAssert(flags & (SRI_MASTER|SRI_SLAVE|SRI_SENTINEL));
SENTINEL_MASTER_LINK_STATUS_UP :
SENTINEL_MASTER_LINK_STATUS_DOWN;
}
+
+ /* slave_priority:<priority> */
+ if (sdslen(l) >= 15 && !memcmp(l,"slave_priority:",15))
+ ri->slave_priority = atoi(l+15);
}
}
ri->info_refresh = mstime();
sdsfreesplitres(lines,numlines);
+ /* ---------------------------- Acting half ----------------------------- */
if (sentinel.tilt) return;
/* Act if a master turned into a slave. */
if ((ri->flags & SRI_MASTER) && role == SRI_SLAVE) {
- if (first_runid && ri->slave_master_host) {
+ if ((first_runid || runid_changed) && ri->slave_master_host) {
/* If it is the first time we receive INFO from it, but it's
* a slave while it was configured as a master, we want to monitor
* its master instead. */
if (!(ri->master->flags & SRI_FAILOVER_IN_PROGRESS) &&
(runid_changed || first_runid))
{
- /* If a slave turned into a master, but at the same time the
- * runid has changed, or it is simply the first time we see and
- * INFO output from this instance, this is a reboot with a wrong
- * configuration.
+ /* If a slave turned into master but:
*
+ * 1) Failover not in progress.
+ * 2) RunID hs changed, or its the first time we see an INFO output.
+ *
+ * We assume this is a reboot with a wrong configuration.
* Log the event and remove the slave. */
int retval;
} else if (ri->flags & SRI_PROMOTED) {
/* If this is a promoted slave we can change state to the
* failover state machine. */
- if (ri->master &&
- (ri->master->flags & SRI_FAILOVER_IN_PROGRESS) &&
+ if ((ri->master->flags & SRI_FAILOVER_IN_PROGRESS) &&
(ri->master->flags & SRI_I_AM_THE_LEADER) &&
(ri->master->failover_state ==
SENTINEL_FAILOVER_STATE_WAIT_PROMOTION))
sentinelEvent(REDIS_WARNING,"+promoted-slave",ri,"%@");
sentinelEvent(REDIS_WARNING,"+failover-state-reconf-slaves",
ri->master,"%@");
+ sentinelCallClientReconfScript(ri->master,SENTINEL_LEADER,
+ "start",ri->master->addr,ri->addr);
}
- } else {
- /* Otherwise we interpret this as the start of the failover. */
- if (ri->master &&
- (ri->master->flags & SRI_FAILOVER_IN_PROGRESS) == 0)
- {
- ri->master->flags |= SRI_FAILOVER_IN_PROGRESS;
- sentinelEvent(REDIS_WARNING,"failover-detected",ri->master,"%@");
- ri->master->failover_state = SENTINEL_FAILOVER_STATE_DETECT_END;
- ri->master->failover_state_change_time = mstime();
- ri->master->promoted_slave = ri;
- ri->flags |= SRI_PROMOTED;
- /* We are an observer, so we can only assume that the leader
- * is reconfiguring the slave instances. For this reason we
- * set all the instances as RECONF_SENT waiting for progresses
- * on this side. */
- sentinelAddFlagsToDictOfRedisInstances(ri->master->slaves,
- SRI_RECONF_SENT);
+ } else if (!(ri->master->flags & SRI_FAILOVER_IN_PROGRESS) ||
+ ((ri->master->flags & SRI_FAILOVER_IN_PROGRESS) &&
+ (ri->master->flags & SRI_I_AM_THE_LEADER) &&
+ ri->master->failover_state ==
+ SENTINEL_FAILOVER_STATE_WAIT_START))
+ {
+ /* No failover in progress? Then it is the start of a failover
+ * and we are an observer.
+ *
+ * We also do that if we are a leader doing a failover, in wait
+ * start, but well, somebody else started before us. */
+
+ if (ri->master->flags & SRI_FAILOVER_IN_PROGRESS) {
+ sentinelEvent(REDIS_WARNING,"-failover-abort-race",
+ ri->master, "%@");
+ sentinelAbortFailover(ri->master);
}
+
+ ri->master->flags |= SRI_FAILOVER_IN_PROGRESS;
+ sentinelEvent(REDIS_WARNING,"+failover-detected",ri->master,"%@");
+ ri->master->failover_state = SENTINEL_FAILOVER_STATE_DETECT_END;
+ ri->master->failover_state_change_time = mstime();
+ ri->master->promoted_slave = ri;
+ ri->flags |= SRI_PROMOTED;
+ sentinelCallClientReconfScript(ri->master,SENTINEL_OBSERVER,
+ "start", ri->master->addr,ri->addr);
+ /* We are an observer, so we can only assume that the leader
+ * is reconfiguring the slave instances. For this reason we
+ * set all the instances as RECONF_SENT waiting for progresses
+ * on this side. */
+ sentinelAddFlagsToDictOfRedisInstances(ri->master->slaves,
+ SRI_RECONF_SENT);
}
}
strncmp(r->str,"MASTERDOWN",10) == 0)
{
ri->last_avail_time = mstime();
+ } else {
+ /* Send a SCRIPT KILL command if the instance appears to be
+ * down because of a busy script. */
+ if (strncmp(r->str,"BUSY",4) == 0 &&
+ (ri->flags & SRI_S_DOWN) &&
+ !(ri->flags & SRI_SCRIPT_KILL_SENT))
+ {
+ redisAsyncCommand(ri->cc,
+ sentinelDiscardReplyCallback, NULL, "SCRIPT KILL");
+ ri->flags |= SRI_SCRIPT_KILL_SENT;
+ }
}
}
ri->last_pong_time = mstime();
addReplyBulkCString(c,"master-port");
addReplyBulkLongLong(c,ri->slave_master_port);
fields++;
+
+ addReplyBulkCString(c,"slave-priority");
+ addReplyBulkLongLong(c,ri->slave_priority);
+ fields++;
}
/* Only sentinels */
addReplyBulkCString(c,addr->ip);
addReplyBulkLongLong(c,addr->port);
}
+ } else if (!strcasecmp(c->argv[1]->ptr,"failover")) {
+ /* SENTINEL FAILOVER <master-name> */
+ sentinelRedisInstance *ri;
+
+ if (c->argc != 3) goto numargserr;
+ if ((ri = sentinelGetMasterByNameOrReplyError(c,c->argv[2])) == NULL)
+ return;
+ if (ri->flags & SRI_FAILOVER_IN_PROGRESS) {
+ addReplySds(c,sdsnew("-INPROG Failover already in progress\r\n"));
+ return;
+ }
+ if (sentinelSelectSlave(ri) == NULL) {
+ addReplySds(c,sdsnew("-NOGOODSLAVE No suitable slave to promote\r\n"));
+ return;
+ }
+ sentinelStartFailover(ri,SENTINEL_FAILOVER_STATE_WAIT_START);
+ ri->flags |= SRI_FORCE_FAILOVER;
+ addReply(c,shared.ok);
+ } else if (!strcasecmp(c->argv[1]->ptr,"pending-scripts")) {
+ /* SENTINEL PENDING-SCRIPTS */
+
+ if (c->argc != 2) goto numargserr;
+ sentinelPendingScriptsCommand(c);
} else {
addReplyErrorFormat(c,"Unknown sentinel subcommand '%s'",
(char*)c->argv[1]->ptr);
(char*)c->argv[1]->ptr);
}
+void sentinelInfoCommand(redisClient *c) {
+ char *section = c->argc == 2 ? c->argv[1]->ptr : "default";
+ sds info = sdsempty();
+ int defsections = !strcasecmp(section,"default");
+ int sections = 0;
+
+ if (c->argc > 2) {
+ addReply(c,shared.syntaxerr);
+ return;
+ }
+
+ if (!strcasecmp(section,"server") || defsections) {
+ if (sections++) info = sdscat(info,"\r\n");
+ sds serversection = genRedisInfoString("server");
+ info = sdscatlen(info,serversection,sdslen(serversection));
+ sdsfree(serversection);
+ }
+
+ if (!strcasecmp(section,"sentinel") || defsections) {
+ dictIterator *di;
+ dictEntry *de;
+ int master_id = 0;
+
+ if (sections++) info = sdscat(info,"\r\n");
+ info = sdscatprintf(info,
+ "# Sentinel\r\n"
+ "sentinel_masters:%lu\r\n"
+ "sentinel_tilt:%d\r\n"
+ "sentinel_running_scripts:%d\r\n"
+ "sentinel_scripts_queue_length:%ld\r\n",
+ dictSize(sentinel.masters),
+ sentinel.tilt,
+ sentinel.running_scripts,
+ listLength(sentinel.scripts_queue));
+
+ di = dictGetIterator(sentinel.masters);
+ while((de = dictNext(di)) != NULL) {
+ sentinelRedisInstance *ri = dictGetVal(de);
+ char *status = "ok";
+
+ if (ri->flags & SRI_O_DOWN) status = "odown";
+ else if (ri->flags & SRI_S_DOWN) status = "sdown";
+ info = sdscatprintf(info,
+ "master%d:name=%s,status=%s,address=%s:%d,"
+ "slaves=%lu,sentinels=%lu\r\n",
+ master_id++, ri->name, status,
+ ri->addr->ip, ri->addr->port,
+ dictSize(ri->slaves),
+ dictSize(ri->sentinels)+1);
+ }
+ dictReleaseIterator(di);
+ }
+
+ addReplySds(c,sdscatprintf(sdsempty(),"$%lu\r\n",
+ (unsigned long)sdslen(info)));
+ addReplySds(c,info);
+ addReply(c,shared.crlf);
+}
+
/* ===================== SENTINEL availability checks ======================= */
/* Is this instance down from our point of view? */
/* Is subjectively up */
if (ri->flags & SRI_S_DOWN) {
sentinelEvent(REDIS_WARNING,"-sdown",ri,"%@");
- ri->flags &= ~SRI_S_DOWN;
+ ri->flags &= ~(SRI_S_DOWN|SRI_SCRIPT_KILL_SENT);
}
}
}
return winner;
}
+/* Setup the master state to start a failover as a leader.
+ *
+ * State can be either:
+ *
+ * SENTINEL_FAILOVER_STATE_WAIT_START: starts a failover from scratch.
+ * SENTINEL_FAILOVER_STATE_RECONF_SLAVES: takedown a failed failover.
+ */
+void sentinelStartFailover(sentinelRedisInstance *master, int state) {
+ redisAssert(master->flags & SRI_MASTER);
+ redisAssert(state == SENTINEL_FAILOVER_STATE_WAIT_START ||
+ state == SENTINEL_FAILOVER_STATE_RECONF_SLAVES);
+
+ master->failover_state = state;
+ master->flags |= SRI_FAILOVER_IN_PROGRESS|SRI_I_AM_THE_LEADER;
+ sentinelEvent(REDIS_WARNING,"+failover-triggered",master,"%@");
+
+ /* Pick a random delay if it's a fresh failover (WAIT_START), and not
+ * a recovery of a failover started by another sentinel. */
+ if (master->failover_state == SENTINEL_FAILOVER_STATE_WAIT_START) {
+ master->failover_start_time = mstime() +
+ SENTINEL_FAILOVER_FIXED_DELAY +
+ (rand() % SENTINEL_FAILOVER_MAX_RANDOM_DELAY);
+ sentinelEvent(REDIS_WARNING,"+failover-state-wait-start",master,
+ "%@ #starting in %lld milliseconds",
+ master->failover_start_time-mstime());
+ }
+ master->failover_state_change_time = mstime();
+}
+
/* This function checks if there are the conditions to start the failover,
* that is:
*
* If the conditions are met we flag the master as SRI_FAILOVER_IN_PROGRESS
* and SRI_I_AM_THE_LEADER.
*/
-void sentinelStartFailover(sentinelRedisInstance *master) {
+void sentinelStartFailoverIfNeeded(sentinelRedisInstance *master) {
char *leader;
int isleader;
/* We have already an elected slave if we are in
* FAILOVER_IN_PROGRESS state, that is, the slave that we
* observed turning into a master. */
- master->failover_state = SENTINEL_FAILOVER_STATE_RECONF_SLAVES;
+ sentinelStartFailover(master,SENTINEL_FAILOVER_STATE_RECONF_SLAVES);
/* As an observer we flagged all the slaves as RECONF_SENT but
* now we are in charge of actually sending the reconfiguration
* command so let's clear this flag for all the instances. */
* Do we have a slave to promote? Otherwise don't start a failover
* at all. */
if (sentinelSelectSlave(master) == NULL) return;
- master->failover_state = SENTINEL_FAILOVER_STATE_WAIT_START;
- }
-
- master->flags |= SRI_FAILOVER_IN_PROGRESS|SRI_I_AM_THE_LEADER;
- sentinelEvent(REDIS_WARNING,"+failover-triggered",master,"%@");
-
- /* Pick a random delay if it's a fresh failover (WAIT_START), and not
- * a recovery of a failover started by another sentinel. */
- if (master->failover_state == SENTINEL_FAILOVER_STATE_WAIT_START) {
- master->failover_start_time = mstime() +
- SENTINEL_FAILOVER_FIXED_DELAY +
- (rand() % SENTINEL_FAILOVER_MAX_RANDOM_DELAY);
- sentinelEvent(REDIS_WARNING,"+failover-state-wait-start",master,
- "%@ #starting in %lld milliseconds",
- master->failover_start_time-mstime());
+ sentinelStartFailover(master,SENTINEL_FAILOVER_STATE_WAIT_START);
}
- master->failover_state_change_time = mstime();
}
/* Select a suitable slave to promote. The current algorithm only uses
* 3) info_refresh more recent than SENTINEL_INFO_VALIDITY_TIME.
* 4) master_link_down_time no more than:
* (now - master->s_down_since_time) + (master->down_after_period * 10).
+ * 5) Slave priority can't be zero, otherwise the slave is discareded.
*
* Among all the slaves matching the above conditions we select the slave
* with lower slave_priority. If priority is the same we select the slave
int compareSlavesForPromotion(const void *a, const void *b) {
sentinelRedisInstance **sa = (sentinelRedisInstance **)a,
**sb = (sentinelRedisInstance **)b;
+ char *sa_runid, *sb_runid;
+
if ((*sa)->slave_priority != (*sb)->slave_priority)
return (*sa)->slave_priority - (*sb)->slave_priority;
- return strcasecmp((*sa)->runid,(*sb)->runid);
+
+ /* If priority is the same, select the slave with that has the
+ * lexicographically smaller runid. Note that we try to handle runid
+ * == NULL as there are old Redis versions that don't publish runid in
+ * INFO. A NULL runid is considered bigger than any other runid. */
+ sa_runid = (*sa)->runid;
+ sb_runid = (*sb)->runid;
+ if (sa_runid == NULL && sb_runid == NULL) return 0;
+ else if (sa_runid == NULL) return 1; /* a > b */
+ else if (sb_runid == NULL) return -1; /* a < b */
+ return strcasecmp(sa_runid, sb_runid);
}
sentinelRedisInstance *sentinelSelectSlave(sentinelRedisInstance *master) {
int instances = 0;
dictIterator *di;
dictEntry *de;
- mstime_t max_master_down_time;
+ mstime_t max_master_down_time = 0;
- max_master_down_time = (mstime() - master->s_down_since_time) +
- (master->down_after_period * 10);
+ if (master->flags & SRI_S_DOWN)
+ max_master_down_time += mstime() - master->s_down_since_time;
+ max_master_down_time += master->down_after_period * 10;
di = dictGetIterator(master->slaves);
while((de = dictNext(di)) != NULL) {
if (slave->flags & (SRI_S_DOWN|SRI_O_DOWN|SRI_DISCONNECTED)) continue;
if (slave->last_avail_time < info_validity_time) continue;
+ if (slave->slave_priority == 0) continue;
+
+ /* If the master is in SDOWN state we get INFO for slaves every second.
+ * Otherwise we get it with the usual period so we need to account for
+ * a larger delay. */
+ if ((master->flags & SRI_S_DOWN) == 0)
+ info_validity_time -= SENTINEL_INFO_PERIOD;
if (slave->info_refresh < info_validity_time) continue;
if (slave->master_link_down_time > max_master_down_time) continue;
instance[instances++] = slave;
/* ---------------- Failover state machine implementation ------------------- */
void sentinelFailoverWaitStart(sentinelRedisInstance *ri) {
+ /* If we in "wait start" but the master is no longer in ODOWN nor in
+ * SDOWN condition we abort the failover. This is important as it
+ * prevents a useless failover in a a notable case of netsplit, where
+ * the senitnels are split from the redis instances. In this case
+ * the failover will not start while there is the split because no
+ * good slave can be reached. However when the split is resolved, we
+ * can go to waitstart if the slave is back rechable a few milliseconds
+ * before the master is. In that case when the master is back online
+ * we cancel the failover. */
+ if ((ri->flags & (SRI_S_DOWN|SRI_O_DOWN|SRI_FORCE_FAILOVER)) == 0) {
+ sentinelEvent(REDIS_WARNING,"-failover-abort-master-is-back",
+ ri,"%@");
+ sentinelAbortFailover(ri);
+ return;
+ }
+
+ /* Start the failover going to the next state if enough time has
+ * elapsed. */
if (mstime() >= ri->failover_start_time) {
ri->failover_state = SENTINEL_FAILOVER_STATE_SELECT_SLAVE;
ri->failover_state_change_time = mstime();
}
if (not_reconfigured == 0) {
+ int role = (master->flags & SRI_I_AM_THE_LEADER) ? SENTINEL_LEADER :
+ SENTINEL_OBSERVER;
+
sentinelEvent(REDIS_WARNING,"+failover-end",master,"%@");
master->failover_state = SENTINEL_FAILOVER_STATE_UPDATE_CONFIG;
master->failover_state_change_time = mstime();
+ sentinelCallClientReconfScript(master,role,"end",master->addr,
+ master->promoted_slave->addr);
}
/* If I'm the leader it is a good idea to send a best effort SLAVEOF
char master_port[32];
dictIterator *di;
dictEntry *de;
+ int sentinel_role;
redisAssert(ri->flags & SRI_FAILOVER_IN_PROGRESS);
ll2string(master_port,sizeof(master_port),ri->addr->port);
}
dictReleaseIterator(di);
- ri->flags &= ~(SRI_FAILOVER_IN_PROGRESS|SRI_I_AM_THE_LEADER);
+ sentinel_role = (ri->flags & SRI_I_AM_THE_LEADER) ? SENTINEL_LEADER :
+ SENTINEL_OBSERVER;
+ ri->flags &= ~(SRI_FAILOVER_IN_PROGRESS|SRI_I_AM_THE_LEADER|SRI_FORCE_FAILOVER);
ri->failover_state = SENTINEL_FAILOVER_STATE_NONE;
ri->failover_state_change_time = mstime();
if (ri->promoted_slave) {
+ sentinelCallClientReconfScript(ri,sentinel_role,"abort",
+ ri->promoted_slave->addr,ri->addr);
ri->promoted_slave->flags &= ~SRI_PROMOTED;
ri->promoted_slave = NULL;
}
/* Only masters */
if (ri->flags & SRI_MASTER) {
sentinelCheckObjectivelyDown(ri);
- sentinelStartFailover(ri);
+ sentinelStartFailoverIfNeeded(ri);
sentinelFailoverStateMachine(ri);
sentinelAbortFailoverIfNeeded(ri);
}
sentinel.previous_time = mstime();
}
-/* Handle terminated childs resulting from calls to notifications and client
- * reconfigurations scripts. */
-void sentinelHandleChildren(void) {
- int statloc;
- pid_t pid;
-
- if ((pid = wait3(&statloc,WNOHANG,NULL)) != 0) {
- int exitcode = WEXITSTATUS(statloc);
- int bysignal = 0;
-
- if (WIFSIGNALED(statloc)) bysignal = WTERMSIG(statloc);
- sentinelEvent(REDIS_DEBUG,"-child",NULL,"%ld %d %d",
- (long)pid, exitcode, bysignal);
-
- /* TODO: remove client reconfiguration scripts from the queue. */
- }
-}
-
void sentinelTimer(void) {
sentinelCheckTiltCondition();
sentinelHandleDictOfRedisInstances(sentinel.masters);
- sentinelHandleChildren();
+ sentinelRunPendingScripts();
+ sentinelCollectTerminatedScripts();
+ sentinelKillTimedoutScripts();
}