]> git.saurik.com Git - redis.git/blobdiff - src/sentinel.c
Sentinel: more easy master redirection if master is a slave.
[redis.git] / src / sentinel.c
index a5ce31ee8ff56ef97bb4a2febc3f69a766c111fc..7fdc45e36bed6cd271913f4fa157307fdf354060 100644 (file)
@@ -73,6 +73,8 @@ typedef struct sentinelAddr {
 #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
@@ -116,6 +118,17 @@ typedef struct sentinelAddr {
 /* 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 */
@@ -182,10 +195,25 @@ struct sentinelState {
                            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
@@ -295,6 +323,9 @@ void sentinelKillLink(sentinelRedisInstance *ri, redisAsyncContext *c);
 const char *sentinelRedisInstanceTypeStr(sentinelRedisInstance *ri);
 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 =============================== */
 
@@ -335,6 +366,7 @@ dictType leaderVotesDictType = {
 /* =========================== Initialization =============================== */
 
 void sentinelCommand(redisClient *c);
+void sentinelInfoCommand(redisClient *c);
 
 struct redisCommand sentinelcmds[] = {
     {"ping",pingCommand,1,"",0,NULL,0,0,0,0,0},
@@ -342,7 +374,8 @@ struct redisCommand sentinelcmds[] = {
     {"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
@@ -371,6 +404,8 @@ void initSentinel(void) {
     sentinel.tilt = 0;
     sentinel.tilt_start_time = mstime();
     sentinel.previous_time = mstime();
+    sentinel.running_scripts = 0;
+    sentinel.scripts_queue = listCreate();
 }
 
 /* ============================== sentinelAddr ============================== */
@@ -406,32 +441,6 @@ void releaseSentinelAddr(sentinelAddr *sa) {
 
 /* =========================== 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
@@ -507,12 +516,285 @@ void sentinelEvent(int level, char *type, sentinelRedisInstance *ri,
         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
@@ -538,7 +820,7 @@ void sentinelEvent(int level, char *type, sentinelRedisInstance *ri,
 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));
@@ -1130,16 +1412,21 @@ void sentinelRefreshInstanceInfo(sentinelRedisInstance *ri, const char *info) {
                     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. */
@@ -1158,11 +1445,12 @@ void sentinelRefreshInstanceInfo(sentinelRedisInstance *ri, const char *info) {
         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;
 
@@ -1173,8 +1461,7 @@ void sentinelRefreshInstanceInfo(sentinelRedisInstance *ri, const char *info) {
         } 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))
@@ -1184,25 +1471,41 @@ void sentinelRefreshInstanceInfo(sentinelRedisInstance *ri, const char *info) {
                 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);
         }
     }
 
@@ -1276,6 +1579,17 @@ void sentinelPingReplyCallback(redisAsyncContext *c, void *reply, void *privdata
             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();
@@ -1574,6 +1888,10 @@ void addReplySentinelRedisInstance(redisClient *c, sentinelRedisInstance *ri) {
         addReplyBulkCString(c,"master-port");
         addReplyBulkLongLong(c,ri->slave_master_port);
         fields++;
+
+        addReplyBulkCString(c,"slave-priority");
+        addReplyBulkLongLong(c,ri->slave_priority);
+        fields++;
     }
 
     /* Only sentinels */
@@ -1696,6 +2014,29 @@ void sentinelCommand(redisClient *c) {
             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);
@@ -1707,6 +2048,65 @@ numargserr:
                           (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? */
@@ -1750,7 +2150,7 @@ void sentinelCheckSubjectivelyDown(sentinelRedisInstance *ri) {
         /* 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);
         }
     }
 }
@@ -2004,6 +2404,35 @@ char *sentinelGetObjectiveLeader(sentinelRedisInstance *master) {
     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:
  *
@@ -2014,7 +2443,7 @@ char *sentinelGetObjectiveLeader(sentinelRedisInstance *master) {
  * 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;
 
@@ -2054,7 +2483,7 @@ void sentinelStartFailover(sentinelRedisInstance *master) {
             /* 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. */
@@ -2062,24 +2491,13 @@ void sentinelStartFailover(sentinelRedisInstance *master) {
                 SRI_RECONF_SENT);
         }
     } else {
-        /* Brand new failover as SRI_FAILOVER_IN_PROGRESS was not set. */
-        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());
+        /* Brand new failover as SRI_FAILOVER_IN_PROGRESS was not set.
+         *
+         * Do we have a slave to promote? Otherwise don't start a failover
+         * at all. */
+        if (sentinelSelectSlave(master) == NULL) return;
+        sentinelStartFailover(master,SENTINEL_FAILOVER_STATE_WAIT_START);
     }
-    master->failover_state_change_time = mstime();
 }
 
 /* Select a suitable slave to promote. The current algorithm only uses
@@ -2090,6 +2508,7 @@ void sentinelStartFailover(sentinelRedisInstance *master) {
  * 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
@@ -2102,9 +2521,21 @@ void sentinelStartFailover(sentinelRedisInstance *master) {
 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) {
@@ -2114,10 +2545,11 @@ 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) {
@@ -2126,6 +2558,13 @@ sentinelRedisInstance *sentinelSelectSlave(sentinelRedisInstance *master) {
 
         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;
@@ -2142,6 +2581,24 @@ sentinelRedisInstance *sentinelSelectSlave(sentinelRedisInstance *master) {
 
 /* ---------------- 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();
@@ -2232,9 +2689,14 @@ void sentinelFailoverDetectEnd(sentinelRedisInstance *master) {
     }
 
     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
@@ -2389,6 +2851,7 @@ void sentinelAbortFailover(sentinelRedisInstance *ri) {
     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);
@@ -2418,10 +2881,14 @@ void sentinelAbortFailover(sentinelRedisInstance *ri) {
     }
     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;
     }
@@ -2490,7 +2957,7 @@ void sentinelHandleRedisInstance(sentinelRedisInstance *ri) {
     /* Only masters */
     if (ri->flags & SRI_MASTER) {
         sentinelCheckObjectivelyDown(ri);
-        sentinelStartFailover(ri);
+        sentinelStartFailoverIfNeeded(ri);
         sentinelFailoverStateMachine(ri);
         sentinelAbortFailoverIfNeeded(ri);
     }
@@ -2553,27 +3020,11 @@ void sentinelCheckTiltCondition(void) {
     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();
 }