Re: Hot standby, recovery infra - Mailing list pgsql-hackers

From Heikki Linnakangas
Subject Re: Hot standby, recovery infra
Date
Msg-id 498314F2.8030005@enterprisedb.com
Whole thread Raw
In response to Re: Hot standby, recovery infra  (Simon Riggs <simon@2ndQuadrant.com>)
Responses Re: Hot standby, recovery infra  (Simon Riggs <simon@2ndQuadrant.com>)
Re: Hot standby, recovery infra  (Fujii Masao <masao.fujii@gmail.com>)
List pgsql-hackers
Ok, here's an attempt to make shutdown work gracefully.

Startup process now signals postmaster three times during startup: first
when it has done all the initialization, and starts redo. At that point.
postmaster launches bgwriter, which starts to perform restartpoints when
it deems appropriate. The 2nd time signals when we've reached consistent
recovery state. As the patch stands, that's not significant, but it will
be with all the rest of the hot standby stuff. The 3rd signal is sent
when startup process has finished recovery. Postmaster used to wait for
the startup process to exit, and check the return code to determine
that, but now that we support shutdown, startup process also returns
with 0 exit code when it has been requested to terminate.

The startup process now catches SIGTERM, and calls proc_exit() at the
next WAL record. That's what will happen in a fast shutdown. Unexpected
death of the startup process is treated the same as a backend/auxiliary
process crash.

InitXLogAccess is now called in IsRecoeryProcessingMode() as you suggested.

Simon Riggs wrote:
> On Thu, 2009-01-29 at 19:20 +0200, Heikki Linnakangas wrote:
>> Heikki Linnakangas wrote:
>>> It looks like if you issue a fast shutdown during recovery, postmaster
>>> doesn't kill bgwriter.
>> Hmm, seems like we haven't thought through how shutdown during
>> consistent recovery is supposed to behave in general. Right now, smart
>> shutdown doesn't do anything during consistent recovery, because the
>> startup process will just keep going. And fast shutdown will simply
>> ExitPostmaster(1), which is clearly not right.
>
> That whole area was something I was leaving until last, since immediate
> shutdown doesn't work either, even in HEAD. (Fujii-san and I discussed
> this before Christmas, briefly).
>
>> I'm thinking that in both smart and fast shutdown, the startup process
>> should exit in a controlled way as soon as it's finished with the
>> current WAL record, and set minSafeStartPoint to the current point in
>> the replay.
>
> That makes sense, though isn't required.
>
>> I wonder if bgwriter should perform a restartpoint before exiting?
>> You'll have to start with recovery on the next startup anyway, but at
>> least we could minimize the amount of WAL that needs to be replayed.
>
> That seems like extra work for no additional benefit.
>
> I think we're beginning to blur the lines between review and you just
> adding some additional stuff in this area. There's nothing to stop you
> doing further changes after this has been committed. We can also commit
> what we have with some caveats also, i.e. commit in pieces.
>


--
   Heikki Linnakangas
   EnterpriseDB   http://www.enterprisedb.com
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index bd6035d..50be1d5 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -36,6 +36,7 @@
 #include "catalog/pg_control.h"
 #include "catalog/pg_type.h"
 #include "funcapi.h"
+#include "libpq/pqsignal.h"
 #include "miscadmin.h"
 #include "pgstat.h"
 #include "postmaster/bgwriter.h"
@@ -47,6 +48,7 @@
 #include "storage/smgr.h"
 #include "storage/spin.h"
 #include "utils/builtins.h"
+#include "utils/flatfiles.h"
 #include "utils/guc.h"
 #include "utils/ps_status.h"
 #include "pg_trace.h"
@@ -119,12 +121,26 @@ CheckpointStatsData CheckpointStats;
  */
 TimeLineID    ThisTimeLineID = 0;

-/* Are we doing recovery from XLOG? */
+/*
+ * Are we doing recovery from XLOG?
+ *
+ * This is only ever true in the startup process, when it's replaying WAL.
+ * It's used in functions that need to act differently when called from a
+ * redo function (e.g skip WAL logging).  To check whether the system is in
+ * recovery regardless of what process you're running in, use
+ * IsRecoveryProcessingMode().
+ */
 bool        InRecovery = false;

 /* Are we recovering using offline XLOG archives? */
 static bool InArchiveRecovery = false;

+/*
+ * Local copy of shared RecoveryProcessingMode variable. True actually
+ * means "not known, need to check the shared state"
+ */
+static bool LocalRecoveryProcessingMode = true;
+
 /* Was the last xlog file restored from archive, or local? */
 static bool restoredFromArchive = false;

@@ -133,7 +149,6 @@ static char *recoveryRestoreCommand = NULL;
 static bool recoveryTarget = false;
 static bool recoveryTargetExact = false;
 static bool recoveryTargetInclusive = true;
-static bool recoveryLogRestartpoints = false;
 static TransactionId recoveryTargetXid;
 static TimestampTz recoveryTargetTime;
 static TimestampTz recoveryLastXTime = 0;
@@ -313,6 +328,22 @@ typedef struct XLogCtlData
     int            XLogCacheBlck;    /* highest allocated xlog buffer index */
     TimeLineID    ThisTimeLineID;

+    /*
+     * SharedRecoveryProcessingMode indicates if we're still in crash or
+     * archive recovery. It's checked by IsRecoveryProcessingMode()
+     */
+    bool        SharedRecoveryProcessingMode;
+
+    /*
+     * During recovery, we keep a copy of the latest checkpoint record
+     * here. It's used by the background writer when it wants to create
+     * a restartpoint.
+     *
+     * is info_lck spinlock a bit too light-weight to protect this?
+     */
+    XLogRecPtr    lastCheckPointRecPtr;
+    CheckPoint    lastCheckPoint;
+
     slock_t        info_lck;        /* locks shared variables shown above */
 } XLogCtlData;

@@ -390,6 +421,11 @@ static TimeLineID lastPageTLI = 0;

 static bool InRedo = false;

+/*
+ * Flag set by interrupt handlers for later service in the redo loop.
+ */
+static volatile sig_atomic_t shutdown_requested = false;
+

 static void XLogArchiveNotify(const char *xlog);
 static void XLogArchiveNotifySeg(uint32 log, uint32 seg);
@@ -399,6 +435,7 @@ static void XLogArchiveCleanup(const char *xlog);
 static void readRecoveryCommandFile(void);
 static void exitArchiveRecovery(TimeLineID endTLI,
                     uint32 endLogId, uint32 endLogSeg);
+static void exitRecovery(void);
 static bool recoveryStopsHere(XLogRecord *record, bool *includeThis);
 static void CheckPointGuts(XLogRecPtr checkPointRedo, int flags);

@@ -483,6 +520,11 @@ XLogInsert(RmgrId rmid, uint8 info, XLogRecData *rdata)
     bool        updrqst;
     bool        doPageWrites;
     bool        isLogSwitch = (rmid == RM_XLOG_ID && info == XLOG_SWITCH);
+    bool        isRecoveryEnd = (rmid == RM_XLOG_ID && info == XLOG_RECOVERY_END);
+
+    /* cross-check on whether we should be here or not */
+    if (IsRecoveryProcessingMode() && !isRecoveryEnd)
+        elog(FATAL, "cannot make new WAL entries during recovery");

     /* info's high bits are reserved for use by me */
     if (info & XLR_INFO_MASK)
@@ -1730,7 +1772,7 @@ XLogFlush(XLogRecPtr record)
     XLogwrtRqst WriteRqst;

     /* Disabled during REDO */
-    if (InRedo)
+    if (IsRecoveryProcessingMode())
         return;

     /* Quick exit if already known flushed */
@@ -1818,9 +1860,9 @@ XLogFlush(XLogRecPtr record)
      * the bad page is encountered again during recovery then we would be
      * unable to restart the database at all!  (This scenario has actually
      * happened in the field several times with 7.1 releases. Note that we
-     * cannot get here while InRedo is true, but if the bad page is brought in
-     * and marked dirty during recovery then CreateCheckPoint will try to
-     * flush it at the end of recovery.)
+     * cannot get here while IsRecoveryProcessingMode(), but if the bad page is
+     * brought in and marked dirty during recovery then if a checkpoint were
+     * performed at the end of recovery it will try to flush it.
      *
      * The current approach is to ERROR under normal conditions, but only
      * WARNING during recovery, so that the system can be brought up even if
@@ -1830,7 +1872,7 @@ XLogFlush(XLogRecPtr record)
      * and so we will not force a restart for a bad LSN on a data page.
      */
     if (XLByteLT(LogwrtResult.Flush, record))
-        elog(InRecovery ? WARNING : ERROR,
+        elog(ERROR,
         "xlog flush request %X/%X is not satisfied --- flushed only to %X/%X",
              record.xlogid, record.xrecoff,
              LogwrtResult.Flush.xlogid, LogwrtResult.Flush.xrecoff);
@@ -2409,6 +2451,33 @@ XLogFileRead(uint32 log, uint32 seg, int emode)
                      xlogfname);
             set_ps_display(activitymsg, false);

+            /*
+             * Calculate and write out a new safeStartPoint. This defines
+             * the latest LSN that might appear on-disk while we apply
+             * the WAL records in this file. If we crash during recovery
+             * we must reach this point again before we can prove
+             * database consistency. Not a restartpoint! Restart points
+             * define where we should start recovery from, if we crash.
+             */
+            if (InArchiveRecovery)
+            {
+                XLogRecPtr    nextSegRecPtr;
+                uint32        nextLog = log;
+                uint32        nextSeg = seg;
+
+                NextLogSeg(nextLog, nextSeg);
+                nextSegRecPtr.xlogid = nextLog;
+                nextSegRecPtr.xrecoff = nextSeg * XLogSegSize;
+
+                LWLockAcquire(ControlFileLock, LW_EXCLUSIVE);
+                if (XLByteLT(ControlFile->minSafeStartPoint, nextSegRecPtr))
+                {
+                    ControlFile->minSafeStartPoint = nextSegRecPtr;
+                    UpdateControlFile();
+                }
+                LWLockRelease(ControlFileLock);
+            }
+
             return fd;
         }
         if (errno != ENOENT)    /* unexpected failure? */
@@ -2677,11 +2746,22 @@ RestoreArchivedFile(char *path, const char *xlogfname,
      * those it's a good bet we should have gotten it too.  Aborting on other
      * signals such as SIGTERM seems a good idea as well.
      *
+     * However, if we were requested to terminate, we don't really care what
+     * happened to the restore command, so we just exit cleanly. In fact,
+     * the restore command most likely received the SIGTERM too, and we don't
+     * want to complain about that.
+     *
      * Per the Single Unix Spec, shells report exit status > 128 when a called
      * command died on a signal.  Also, 126 and 127 are used to report
      * problems such as an unfindable command; treat those as fatal errors
      * too.
      */
+    if (shutdown_requested && InRedo)
+    {
+        /* XXX: We should update minSafeStartPoint to the exact value here */
+        proc_exit(0);
+    }
+
     signaled = WIFSIGNALED(rc) || WEXITSTATUS(rc) > 125;

     ereport(signaled ? FATAL : DEBUG2,
@@ -4587,18 +4667,6 @@ readRecoveryCommandFile(void)
             ereport(LOG,
                     (errmsg("recovery_target_inclusive = %s", tok2)));
         }
-        else if (strcmp(tok1, "log_restartpoints") == 0)
-        {
-            /*
-             * does nothing if a recovery_target is not also set
-             */
-            if (!parse_bool(tok2, &recoveryLogRestartpoints))
-                  ereport(ERROR,
-                            (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
-                      errmsg("parameter \"log_restartpoints\" requires a Boolean value")));
-            ereport(LOG,
-                    (errmsg("log_restartpoints = %s", tok2)));
-        }
         else
             ereport(FATAL,
                     (errmsg("unrecognized recovery parameter \"%s\"",
@@ -4734,7 +4802,10 @@ exitArchiveRecovery(TimeLineID endTLI, uint32 endLogId, uint32 endLogSeg)

     /*
      * Rename the config file out of the way, so that we don't accidentally
-     * re-enter archive recovery mode in a subsequent crash.
+     * re-enter archive recovery mode in a subsequent crash. We have already
+     * restored all the WAL segments we need from the archive, and we trust
+     * that they are not going to go away even if we crash. (XXX: should
+     * we fsync() them all to ensure that?)
      */
     unlink(RECOVERY_COMMAND_DONE);
     if (rename(RECOVERY_COMMAND_FILE, RECOVERY_COMMAND_DONE) != 0)
@@ -4876,6 +4947,8 @@ StartupXLOG(void)
     CheckPoint    checkPoint;
     bool        wasShutdown;
     bool        reachedStopPoint = false;
+    bool        reachedSafeStartPoint = false;
+    bool        performedRecovery = false;
     bool        haveBackupLabel = false;
     XLogRecPtr    RecPtr,
                 LastRec,
@@ -4888,6 +4961,8 @@ StartupXLOG(void)
     uint32        freespace;
     TransactionId oldestActiveXID;

+    XLogCtl->SharedRecoveryProcessingMode = true;
+
     /*
      * Read control file and check XLOG status looks valid.
      *
@@ -5108,9 +5183,15 @@ StartupXLOG(void)
         if (minRecoveryLoc.xlogid != 0 || minRecoveryLoc.xrecoff != 0)
             ControlFile->minRecoveryPoint = minRecoveryLoc;
         ControlFile->time = (pg_time_t) time(NULL);
+        /* No need to hold ControlFileLock yet, we aren't up far enough */
         UpdateControlFile();

         /*
+         * Reset pgstat data, because it may be invalid after recovery.
+         */
+        pgstat_reset_all();
+
+        /*
          * If there was a backup label file, it's done its job and the info
          * has now been propagated into pg_control.  We must get rid of the
          * label file so that if we crash during recovery, we'll pick up at
@@ -5155,6 +5236,7 @@ StartupXLOG(void)
             bool        recoveryContinue = true;
             bool        recoveryApply = true;
             ErrorContextCallback errcontext;
+            XLogRecPtr    minSafeStartPoint;

             InRedo = true;
             ereport(LOG,
@@ -5162,6 +5244,16 @@ StartupXLOG(void)
                             ReadRecPtr.xlogid, ReadRecPtr.xrecoff)));

             /*
+             * Take a local copy of minSafeStartPoint at the beginning of
+             * recovery, because it's updated as we go.
+             */
+            minSafeStartPoint = ControlFile->minSafeStartPoint;
+
+            /* Let postmaster know we've started redo now */
+            if (InArchiveRecovery && IsUnderPostmaster)
+                SendPostmasterSignal(PMSIGNAL_RECOVERY_STARTED);
+
+            /*
              * main redo apply loop
              */
             do
@@ -5186,6 +5278,46 @@ StartupXLOG(void)
 #endif

                 /*
+                 * Process any requests or signals received recently.
+                 */
+                if (shutdown_requested)
+                {
+                    /*
+                     * We were requested to exit without finishing recovery.
+                     *
+                     * XXX: We should update minSafeStartPoint to the exact
+                     * value here.
+                     */
+                    proc_exit(0);
+                }
+
+                /*
+                 * Have we reached our safe starting point? If so, we can
+                 * signal postmaster to enter consistent recovery mode.
+                 *
+                 * There are two points in the log we must pass. The first is
+                 * the minRecoveryPoint, which is the LSN at the time the
+                 * base backup was taken that we are about to rollfoward from.
+                 * If recovery has ever crashed or was stopped there is
+                 * another point also: minSafeStartPoint, which is the
+                 * latest LSN that recovery could have reached prior to crash.
+                 */
+                if (!reachedSafeStartPoint &&
+                     XLByteLE(minSafeStartPoint, EndRecPtr) &&
+                     XLByteLE(ControlFile->minRecoveryPoint, EndRecPtr))
+                {
+                    reachedSafeStartPoint = true;
+                    if (InArchiveRecovery)
+                    {
+                        ereport(LOG,
+                            (errmsg("consistent recovery state reached at %X/%X",
+                                EndRecPtr.xlogid, EndRecPtr.xrecoff)));
+                        if (IsUnderPostmaster)
+                            SendPostmasterSignal(PMSIGNAL_RECOVERY_CONSISTENT);
+                    }
+                }
+
+                /*
                  * Have we reached our recovery target?
                  */
                 if (recoveryStopsHere(record, &recoveryApply))
@@ -5238,6 +5370,7 @@ StartupXLOG(void)
             /* there are no WAL records following the checkpoint */
             ereport(LOG,
                     (errmsg("redo is not required")));
+            reachedSafeStartPoint = true;
         }
     }

@@ -5253,7 +5386,7 @@ StartupXLOG(void)
      * Complain if we did not roll forward far enough to render the backup
      * dump consistent.
      */
-    if (XLByteLT(EndOfLog, ControlFile->minRecoveryPoint))
+    if (InRecovery && !reachedSafeStartPoint)
     {
         if (reachedStopPoint)    /* stopped because of stop request */
             ereport(FATAL,
@@ -5375,38 +5508,16 @@ StartupXLOG(void)
         XLogCheckInvalidPages();

         /*
-         * Reset pgstat data, because it may be invalid after recovery.
+         * Finally exit recovery and mark that in WAL. Pre-8.4 we wrote
+         * a shutdown checkpoint here, but we ask bgwriter to do that now.
          */
-        pgstat_reset_all();
+        exitRecovery();

-        /*
-         * Perform a checkpoint to update all our recovery activity to disk.
-         *
-         * Note that we write a shutdown checkpoint rather than an on-line
-         * one. This is not particularly critical, but since we may be
-         * assigning a new TLI, using a shutdown checkpoint allows us to have
-         * the rule that TLI only changes in shutdown checkpoints, which
-         * allows some extra error checking in xlog_redo.
-         */
-        CreateCheckPoint(CHECKPOINT_IS_SHUTDOWN | CHECKPOINT_IMMEDIATE);
+        performedRecovery = true;
     }

-    /*
-     * Preallocate additional log files, if wanted.
-     */
-    PreallocXlogFiles(EndOfLog);
-
-    /*
-     * Okay, we're officially UP.
-     */
-    InRecovery = false;
-
-    ControlFile->state = DB_IN_PRODUCTION;
-    ControlFile->time = (pg_time_t) time(NULL);
-    UpdateControlFile();
-
     /* start the archive_timeout timer running */
-    XLogCtl->Write.lastSegSwitchTime = ControlFile->time;
+    XLogCtl->Write.lastSegSwitchTime = (pg_time_t) time(NULL);

     /* initialize shared-memory copy of latest checkpoint XID/epoch */
     XLogCtl->ckptXidEpoch = ControlFile->checkPointCopy.nextXidEpoch;
@@ -5441,6 +5552,74 @@ StartupXLOG(void)
         readRecordBuf = NULL;
         readRecordBufSize = 0;
     }
+
+    /*
+     * If we had to replay any WAL records, request a checkpoint. This isn't
+     * strictly necessary: if we crash now, the recovery will simply restart
+     * from the same point as this time (or from the last restartpoint). The
+     * control file is left in DB_IN_*_RECOVERY state; the first checkpoint
+     * will change that to DB_IN_PRODUCTION.
+     */
+    if (performedRecovery)
+    {
+        /*
+         * Okay, we can come up now. Allow others to write WAL.
+         */
+        XLogCtl->SharedRecoveryProcessingMode = false;
+
+        RequestCheckpoint(CHECKPOINT_FORCE | CHECKPOINT_IMMEDIATE |
+                          CHECKPOINT_STARTUP);
+    }
+    else
+    {
+        /*
+         * No recovery, so let's just get on with it.
+         */
+        LWLockAcquire(ControlFileLock, LW_EXCLUSIVE);
+        ControlFile->state = DB_IN_PRODUCTION;
+        ControlFile->time = (pg_time_t) time(NULL);
+        UpdateControlFile();
+        LWLockRelease(ControlFileLock);
+
+        /*
+         * Okay, we're officially UP.
+         */
+        XLogCtl->SharedRecoveryProcessingMode = false;
+    }
+}
+
+/*
+ * Is the system still in recovery?
+ *
+ * As a side-effect, we initialize the local TimeLineID and RedoRecPtr
+ * variables the first time we see that recovery is finished.
+ */
+bool
+IsRecoveryProcessingMode(void)
+{
+    /*
+     * We check shared state each time only until we leave recovery mode.
+     * We can't re-enter recovery, so we rely on the local state variable
+     * after that.
+     */
+    if (!LocalRecoveryProcessingMode)
+        return false;
+    else
+    {
+        /* use volatile pointer to prevent code rearrangement */
+        volatile XLogCtlData *xlogctl = XLogCtl;
+
+        LocalRecoveryProcessingMode = xlogctl->SharedRecoveryProcessingMode;
+
+        /*
+         * Initialize TimeLineID and RedoRecPtr the first time we see that
+         * recovery is finished.
+         */
+        if (!LocalRecoveryProcessingMode)
+            InitXLOGAccess();
+
+        return LocalRecoveryProcessingMode;
+    }
 }

 /*
@@ -5572,6 +5751,8 @@ InitXLOGAccess(void)
 {
     /* ThisTimeLineID doesn't change so we need no lock to copy it */
     ThisTimeLineID = XLogCtl->ThisTimeLineID;
+    Assert(ThisTimeLineID != 0);
+
     /* Use GetRedoRecPtr to copy the RedoRecPtr safely */
     (void) GetRedoRecPtr();
 }
@@ -5683,7 +5864,10 @@ ShutdownXLOG(int code, Datum arg)
     ereport(LOG,
             (errmsg("shutting down")));

-    CreateCheckPoint(CHECKPOINT_IS_SHUTDOWN | CHECKPOINT_IMMEDIATE);
+    if (IsRecoveryProcessingMode())
+        CreateRestartPoint(CHECKPOINT_IS_SHUTDOWN | CHECKPOINT_IMMEDIATE);
+    else
+        CreateCheckPoint(CHECKPOINT_IS_SHUTDOWN | CHECKPOINT_IMMEDIATE);
     ShutdownCLOG();
     ShutdownSUBTRANS();
     ShutdownMultiXact();
@@ -5696,10 +5880,22 @@ ShutdownXLOG(int code, Datum arg)
  * Log start of a checkpoint.
  */
 static void
-LogCheckpointStart(int flags)
+LogCheckpointStart(int flags, bool restartpoint)
 {
-    elog(LOG, "checkpoint starting:%s%s%s%s%s%s",
+    char *msg;
+
+    /*
+     * XXX: This is hopelessly untranslatable. We could call gettext_noop
+     * for the main message, but what about all the flags?
+     */
+    if (restartpoint)
+        msg = "restartpoint starting:%s%s%s%s%s%s%s";
+    else
+        msg = "checkpoint starting:%s%s%s%s%s%s%s";
+
+    elog(LOG, msg,
          (flags & CHECKPOINT_IS_SHUTDOWN) ? " shutdown" : "",
+         (flags & CHECKPOINT_STARTUP) ? " startup" : "",
          (flags & CHECKPOINT_IMMEDIATE) ? " immediate" : "",
          (flags & CHECKPOINT_FORCE) ? " force" : "",
          (flags & CHECKPOINT_WAIT) ? " wait" : "",
@@ -5711,7 +5907,7 @@ LogCheckpointStart(int flags)
  * Log end of a checkpoint.
  */
 static void
-LogCheckpointEnd(void)
+LogCheckpointEnd(bool restartpoint)
 {
     long        write_secs,
                 sync_secs,
@@ -5734,17 +5930,26 @@ LogCheckpointEnd(void)
                         CheckpointStats.ckpt_sync_end_t,
                         &sync_secs, &sync_usecs);

-    elog(LOG, "checkpoint complete: wrote %d buffers (%.1f%%); "
-         "%d transaction log file(s) added, %d removed, %d recycled; "
-         "write=%ld.%03d s, sync=%ld.%03d s, total=%ld.%03d s",
-         CheckpointStats.ckpt_bufs_written,
-         (double) CheckpointStats.ckpt_bufs_written * 100 / NBuffers,
-         CheckpointStats.ckpt_segs_added,
-         CheckpointStats.ckpt_segs_removed,
-         CheckpointStats.ckpt_segs_recycled,
-         write_secs, write_usecs / 1000,
-         sync_secs, sync_usecs / 1000,
-         total_secs, total_usecs / 1000);
+    if (restartpoint)
+        elog(LOG, "restartpoint complete: wrote %d buffers (%.1f%%); "
+             "write=%ld.%03d s, sync=%ld.%03d s, total=%ld.%03d s",
+             CheckpointStats.ckpt_bufs_written,
+             (double) CheckpointStats.ckpt_bufs_written * 100 / NBuffers,
+             write_secs, write_usecs / 1000,
+             sync_secs, sync_usecs / 1000,
+             total_secs, total_usecs / 1000);
+    else
+        elog(LOG, "checkpoint complete: wrote %d buffers (%.1f%%); "
+             "%d transaction log file(s) added, %d removed, %d recycled; "
+             "write=%ld.%03d s, sync=%ld.%03d s, total=%ld.%03d s",
+             CheckpointStats.ckpt_bufs_written,
+             (double) CheckpointStats.ckpt_bufs_written * 100 / NBuffers,
+             CheckpointStats.ckpt_segs_added,
+             CheckpointStats.ckpt_segs_removed,
+             CheckpointStats.ckpt_segs_recycled,
+             write_secs, write_usecs / 1000,
+             sync_secs, sync_usecs / 1000,
+             total_secs, total_usecs / 1000);
 }

 /*
@@ -5775,6 +5980,10 @@ CreateCheckPoint(int flags)
     TransactionId *inCommitXids;
     int            nInCommit;

+    /* shouldn't happen */
+    if (IsRecoveryProcessingMode())
+        elog(ERROR, "can't create a checkpoint during recovery");
+
     /*
      * Acquire CheckpointLock to ensure only one checkpoint happens at a time.
      * (This is just pro forma, since in the present system structure there is
@@ -5800,9 +6009,11 @@ CreateCheckPoint(int flags)

     if (shutdown)
     {
+        LWLockAcquire(ControlFileLock, LW_EXCLUSIVE);
         ControlFile->state = DB_SHUTDOWNING;
         ControlFile->time = (pg_time_t) time(NULL);
         UpdateControlFile();
+        LWLockRelease(ControlFileLock);
     }

     /*
@@ -5906,7 +6117,7 @@ CreateCheckPoint(int flags)
      * to log anything if we decided to skip the checkpoint.
      */
     if (log_checkpoints)
-        LogCheckpointStart(flags);
+        LogCheckpointStart(flags, false);

     TRACE_POSTGRESQL_CHECKPOINT_START(flags);

@@ -6010,11 +6221,14 @@ CreateCheckPoint(int flags)
     XLByteToSeg(ControlFile->checkPointCopy.redo, _logId, _logSeg);

     /*
-     * Update the control file.
+     * Update the control file. This also sets state to IN_DB_PRODUCTION
+     * if this is the first checkpoint after recovery.
      */
     LWLockAcquire(ControlFileLock, LW_EXCLUSIVE);
     if (shutdown)
         ControlFile->state = DB_SHUTDOWNED;
+    else
+        ControlFile->state = DB_IN_PRODUCTION;
     ControlFile->prevCheckPoint = ControlFile->checkPoint;
     ControlFile->checkPoint = ProcLastRecPtr;
     ControlFile->checkPointCopy = checkPoint;
@@ -6068,12 +6282,11 @@ CreateCheckPoint(int flags)
      * in subtrans.c).    During recovery, though, we mustn't do this because
      * StartupSUBTRANS hasn't been called yet.
      */
-    if (!InRecovery)
-        TruncateSUBTRANS(GetOldestXmin(true, false));
+    TruncateSUBTRANS(GetOldestXmin(true, false));

     /* All real work is done, but log before releasing lock. */
     if (log_checkpoints)
-        LogCheckpointEnd();
+        LogCheckpointEnd(false);

         TRACE_POSTGRESQL_CHECKPOINT_DONE(CheckpointStats.ckpt_bufs_written,
                                 NBuffers, CheckpointStats.ckpt_segs_added,
@@ -6101,32 +6314,17 @@ CheckPointGuts(XLogRecPtr checkPointRedo, int flags)
 }

 /*
- * Set a recovery restart point if appropriate
- *
- * This is similar to CreateCheckPoint, but is used during WAL recovery
- * to establish a point from which recovery can roll forward without
- * replaying the entire recovery log.  This function is called each time
- * a checkpoint record is read from XLOG; it must determine whether a
- * restartpoint is needed or not.
+ * This is used during WAL recovery to establish a point from which recovery
+ * can roll forward without replaying the entire recovery log.  This function
+ * is called each time a checkpoint record is read from XLOG. It is stored
+ * in shared memory, so that it can be used as a restartpoint later on.
  */
 static void
 RecoveryRestartPoint(const CheckPoint *checkPoint)
 {
-    int            elapsed_secs;
     int            rmid;
-
-    /*
-     * Do nothing if the elapsed time since the last restartpoint is less than
-     * half of checkpoint_timeout.    (We use a value less than
-     * checkpoint_timeout so that variations in the timing of checkpoints on
-     * the master, or speed of transmission of WAL segments to a slave, won't
-     * make the slave skip a restartpoint once it's synced with the master.)
-     * Checking true elapsed time keeps us from doing restartpoints too often
-     * while rapidly scanning large amounts of WAL.
-     */
-    elapsed_secs = (pg_time_t) time(NULL) - ControlFile->time;
-    if (elapsed_secs < CheckPointTimeout / 2)
-        return;
+    /* use volatile pointer to prevent code rearrangement */
+    volatile XLogCtlData *xlogctl = XLogCtl;

     /*
      * Is it safe to checkpoint?  We must ask each of the resource managers
@@ -6148,28 +6346,111 @@ RecoveryRestartPoint(const CheckPoint *checkPoint)
     }

     /*
-     * OK, force data out to disk
+     * Copy the checkpoint record to shared memory, so that bgwriter can
+     * use it the next time it wants to perform a restartpoint.
      */
-    CheckPointGuts(checkPoint->redo, CHECKPOINT_IMMEDIATE);
+    SpinLockAcquire(&xlogctl->info_lck);
+    XLogCtl->lastCheckPointRecPtr = ReadRecPtr;
+    memcpy(&XLogCtl->lastCheckPoint, checkPoint, sizeof(CheckPoint));
+    SpinLockRelease(&xlogctl->info_lck);
+}
+
+/*
+ * This is similar to CreateCheckPoint, but is used during WAL recovery
+ * to establish a point from which recovery can roll forward without
+ * replaying the entire recovery log.
+ */
+void
+CreateRestartPoint(int flags)
+{
+    XLogRecPtr lastCheckPointRecPtr;
+    CheckPoint lastCheckPoint;
+    /* use volatile pointer to prevent code rearrangement */
+    volatile XLogCtlData *xlogctl = XLogCtl;
+
+    /*
+     * Acquire CheckpointLock to ensure only one restartpoint happens at a
+     * time. (This is just pro forma, since in the present system structure
+     * there is only one process that is allowed to issue checkpoints or
+     * restart points at any given time.)
+     */
+    LWLockAcquire(CheckpointLock, LW_EXCLUSIVE);
+
+    /* Get the a local copy of the last checkpoint record. */
+    SpinLockAcquire(&xlogctl->info_lck);
+    lastCheckPointRecPtr = xlogctl->lastCheckPointRecPtr;
+    memcpy(&lastCheckPoint, &XLogCtl->lastCheckPoint, sizeof(CheckPoint));
+    SpinLockRelease(&xlogctl->info_lck);

     /*
-     * Update pg_control so that any subsequent crash will restart from this
-     * checkpoint.    Note: ReadRecPtr gives the XLOG address of the checkpoint
-     * record itself.
+     * If the last checkpoint record we've replayed is already our last
+     * restartpoint, we're done.
      */
+    if (XLByteLE(lastCheckPoint.redo, ControlFile->checkPointCopy.redo))
+    {
+        ereport(DEBUG2,
+                (errmsg("skipping restartpoint, already performed at %X/%X",
+                        lastCheckPoint.redo.xlogid, lastCheckPoint.redo.xrecoff)));
+        LWLockRelease(CheckpointLock);
+        return;
+    }
+
+    /*
+     * Check that we're still in recovery mode. It's ok if we exit recovery
+     * mode after this check, the restart point is valid anyway.
+     */
+    if (!IsRecoveryProcessingMode())
+    {
+        ereport(DEBUG2,
+                (errmsg("skipping restartpoint, recovery has already ended")));
+        LWLockRelease(CheckpointLock);
+        return;
+    }
+
+    if (log_checkpoints)
+    {
+        /*
+         * Prepare to accumulate statistics.
+         */
+        MemSet(&CheckpointStats, 0, sizeof(CheckpointStats));
+        CheckpointStats.ckpt_start_t = GetCurrentTimestamp();
+
+        LogCheckpointStart(flags, true);
+    }
+
+    CheckPointGuts(lastCheckPoint.redo, flags);
+
+    /*
+     * Update pg_control, using current time
+     */
+    LWLockAcquire(ControlFileLock, LW_EXCLUSIVE);
     ControlFile->prevCheckPoint = ControlFile->checkPoint;
-    ControlFile->checkPoint = ReadRecPtr;
-    ControlFile->checkPointCopy = *checkPoint;
+    ControlFile->checkPoint = lastCheckPointRecPtr;
+    ControlFile->checkPointCopy = lastCheckPoint;
     ControlFile->time = (pg_time_t) time(NULL);
     UpdateControlFile();
+    LWLockRelease(ControlFileLock);
+
+    /*
+     * Currently, there is no need to truncate pg_subtrans during recovery.
+     * If we did do that, we will need to have called StartupSUBTRANS()
+     * already and then TruncateSUBTRANS() would go here.
+     */

-    ereport((recoveryLogRestartpoints ? LOG : DEBUG2),
+    /* All real work is done, but log before releasing lock. */
+    if (log_checkpoints)
+        LogCheckpointEnd(true);
+
+    ereport((log_checkpoints ? LOG : DEBUG2),
             (errmsg("recovery restart point at %X/%X",
-                    checkPoint->redo.xlogid, checkPoint->redo.xrecoff)));
+                    lastCheckPoint.redo.xlogid, lastCheckPoint.redo.xrecoff)));
+
     if (recoveryLastXTime)
-        ereport((recoveryLogRestartpoints ? LOG : DEBUG2),
-                (errmsg("last completed transaction was at log time %s",
-                        timestamptz_to_str(recoveryLastXTime))));
+        ereport((log_checkpoints ? LOG : DEBUG2),
+            (errmsg("last completed transaction was at log time %s",
+                    timestamptz_to_str(recoveryLastXTime))));
+
+    LWLockRelease(CheckpointLock);
 }

 /*
@@ -6234,7 +6515,43 @@ RequestXLogSwitch(void)
 }

 /*
+ * exitRecovery()
+ *
+ * Exit recovery state and write a XLOG_RECOVERY_END record. This is the
+ * only record type that can record a change of timelineID. We assume
+ * caller has already set ThisTimeLineID, if appropriate.
+ */
+static void
+exitRecovery(void)
+{
+    XLogRecData rdata;
+
+    rdata.buffer = InvalidBuffer;
+    rdata.data = (char *) (&ThisTimeLineID);
+    rdata.len = sizeof(TimeLineID);
+    rdata.next = NULL;
+
+    /*
+     * This is the only type of WAL message that can be inserted during
+     * recovery. This ensures that we don't allow others to get access
+     * until after we have changed state.
+     */
+    (void) XLogInsert(RM_XLOG_ID, XLOG_RECOVERY_END, &rdata);
+
+    /*
+     * We don't XLogFlush() here otherwise we'll end up zeroing the WAL
+     * file ourselves. So just let bgwriter's forthcoming checkpoint do
+     * that for us.
+     */
+
+    InRecovery = false;
+}
+
+/*
  * XLOG resource manager's routines
+ *
+ * Definitions of message info are in include/catalog/pg_control.h,
+ * though not all messages relate to control file processing.
  */
 void
 xlog_redo(XLogRecPtr lsn, XLogRecord *record)
@@ -6272,21 +6589,38 @@ xlog_redo(XLogRecPtr lsn, XLogRecord *record)
         ControlFile->checkPointCopy.nextXid = checkPoint.nextXid;

         /*
-         * TLI may change in a shutdown checkpoint, but it shouldn't decrease
+         * TLI no longer changes at shutdown checkpoint, since as of 8.4,
+         * shutdown checkpoints only occur at shutdown. Much less confusing.
          */
-        if (checkPoint.ThisTimeLineID != ThisTimeLineID)
+
+        RecoveryRestartPoint(&checkPoint);
+    }
+    else if (info == XLOG_RECOVERY_END)
+    {
+        TimeLineID    tli;
+
+        memcpy(&tli, XLogRecGetData(record), sizeof(TimeLineID));
+
+        /*
+         * TLI may change when recovery ends, but it shouldn't decrease.
+         *
+         * This is the only WAL record that can tell us to change timelineID
+         * while we process WAL records.
+         *
+         * We can *choose* to stop recovery at any point, generating a
+         * new timelineID which is recorded using this record type.
+         */
+        if (tli != ThisTimeLineID)
         {
-            if (checkPoint.ThisTimeLineID < ThisTimeLineID ||
+            if (tli < ThisTimeLineID ||
                 !list_member_int(expectedTLIs,
-                                 (int) checkPoint.ThisTimeLineID))
+                                 (int) tli))
                 ereport(PANIC,
-                        (errmsg("unexpected timeline ID %u (after %u) in checkpoint record",
-                                checkPoint.ThisTimeLineID, ThisTimeLineID)));
+                        (errmsg("unexpected timeline ID %u (after %u) at recovery end record",
+                                tli, ThisTimeLineID)));
             /* Following WAL records should be run with new TLI */
-            ThisTimeLineID = checkPoint.ThisTimeLineID;
+            ThisTimeLineID = tli;
         }
-
-        RecoveryRestartPoint(&checkPoint);
     }
     else if (info == XLOG_CHECKPOINT_ONLINE)
     {
@@ -6309,7 +6643,7 @@ xlog_redo(XLogRecPtr lsn, XLogRecord *record)
         ControlFile->checkPointCopy.nextXidEpoch = checkPoint.nextXidEpoch;
         ControlFile->checkPointCopy.nextXid = checkPoint.nextXid;

-        /* TLI should not change in an on-line checkpoint */
+        /* TLI must not change at a checkpoint */
         if (checkPoint.ThisTimeLineID != ThisTimeLineID)
             ereport(PANIC,
                     (errmsg("unexpected timeline ID %u (should be %u) in checkpoint record",
@@ -7224,3 +7558,89 @@ CancelBackup(void)
     }
 }

+/* ------------------------------------------------------
+ *  Startup Process main entry point and signal handlers
+ * ------------------------------------------------------
+ */
+
+/*
+ * wal_quickdie() occurs when signalled SIGQUIT by the postmaster.
+ *
+ * Some backend has bought the farm,
+ * so we need to stop what we're doing and exit.
+ */
+static void
+startupproc_quickdie(SIGNAL_ARGS)
+{
+    PG_SETMASK(&BlockSig);
+
+    /*
+     * DO NOT proc_exit() -- we're here because shared memory may be
+     * corrupted, so we don't want to try to clean up our transaction. Just
+     * nail the windows shut and get out of town.
+     *
+     * Note we do exit(2) not exit(0).    This is to force the postmaster into a
+     * system reset cycle if some idiot DBA sends a manual SIGQUIT to a random
+     * backend.  This is necessary precisely because we don't clean up our
+     * shared memory state.
+     */
+    exit(2);
+}
+
+
+/* SIGTERM: set flag to abort redo and exit */
+static void
+StartupProcShutdownHandler(SIGNAL_ARGS)
+{
+    shutdown_requested = true;
+}
+
+/* Main entry point for startup process */
+void
+StartupProcessMain(void)
+{
+    /*
+     * If possible, make this process a group leader, so that the postmaster
+     * can signal any child processes too.
+     */
+#ifdef HAVE_SETSID
+    if (setsid() < 0)
+        elog(FATAL, "setsid() failed: %m");
+#endif
+
+    /*
+     * Properly accept or ignore signals the postmaster might send us
+     */
+    pqsignal(SIGHUP, SIG_IGN);    /* ignore config file updates */
+    pqsignal(SIGINT, SIG_IGN);        /* ignore query cancel */
+    pqsignal(SIGTERM, StartupProcShutdownHandler); /* request shutdown */
+    pqsignal(SIGQUIT, startupproc_quickdie);        /* hard crash time */
+    pqsignal(SIGALRM, SIG_IGN);
+    pqsignal(SIGPIPE, SIG_IGN);
+    pqsignal(SIGUSR1, SIG_IGN);
+    pqsignal(SIGUSR2, SIG_IGN);
+
+    /*
+     * Reset some signals that are accepted by postmaster but not here
+     */
+    pqsignal(SIGCHLD, SIG_DFL);
+    pqsignal(SIGTTIN, SIG_DFL);
+    pqsignal(SIGTTOU, SIG_DFL);
+    pqsignal(SIGCONT, SIG_DFL);
+    pqsignal(SIGWINCH, SIG_DFL);
+
+    /*
+     * Unblock signals (they were blocked when the postmaster forked us)
+     */
+    PG_SETMASK(&UnBlockSig);
+
+    StartupXLOG();
+
+    BuildFlatFiles(false);
+
+    /* Let postmaster know that startup is finished */
+    SendPostmasterSignal(PMSIGNAL_RECOVERY_COMPLETED);
+
+    /* exit normally */
+    proc_exit(0);
+}
diff --git a/src/backend/bootstrap/bootstrap.c b/src/backend/bootstrap/bootstrap.c
index 431a95f..13d5bcb 100644
--- a/src/backend/bootstrap/bootstrap.c
+++ b/src/backend/bootstrap/bootstrap.c
@@ -37,7 +37,6 @@
 #include "storage/proc.h"
 #include "tcop/tcopprot.h"
 #include "utils/builtins.h"
-#include "utils/flatfiles.h"
 #include "utils/fmgroids.h"
 #include "utils/memutils.h"
 #include "utils/ps_status.h"
@@ -416,14 +415,12 @@ AuxiliaryProcessMain(int argc, char *argv[])
             proc_exit(1);        /* should never return */

         case StartupProcess:
-            bootstrap_signals();
-            StartupXLOG();
-            BuildFlatFiles(false);
-            proc_exit(0);        /* startup done */
+            /* don't set signals, startup process has its own agenda */
+            StartupProcessMain();
+            proc_exit(1);        /* should never return */

         case BgWriterProcess:
             /* don't set signals, bgwriter has its own agenda */
-            InitXLOGAccess();
             BackgroundWriterMain();
             proc_exit(1);        /* should never return */

diff --git a/src/backend/postmaster/bgwriter.c b/src/backend/postmaster/bgwriter.c
index 6a0cd4e..4c8c54c 100644
--- a/src/backend/postmaster/bgwriter.c
+++ b/src/backend/postmaster/bgwriter.c
@@ -49,6 +49,7 @@
 #include <unistd.h>

 #include "access/xlog_internal.h"
+#include "catalog/pg_control.h"
 #include "libpq/pqsignal.h"
 #include "miscadmin.h"
 #include "pgstat.h"
@@ -197,6 +198,9 @@ BackgroundWriterMain(void)
 {
     sigjmp_buf    local_sigjmp_buf;
     MemoryContext bgwriter_context;
+    bool        BgWriterRecoveryMode = true;
+    /* use volatile pointer to prevent code rearrangement */
+    volatile BgWriterShmemStruct *bgs = BgWriterShmem;

     BgWriterShmem->bgwriter_pid = MyProcPid;
     am_bg_writer = true;
@@ -356,6 +360,20 @@ BackgroundWriterMain(void)
     PG_SETMASK(&UnBlockSig);

     /*
+     * If someone requested a checkpoint before we started up, process that.
+     *
+     * This check exists primarily for crash recovery: after the startup
+     * process is finished with WAL replay, it will request a checkpoint, but
+     * the background writer might not have started yet. This check will
+     * actually not notice a checkpoint that's been requested without any
+     * flags, but it's good enough for the startup checkpoint.
+     */
+    SpinLockAcquire(&bgs->ckpt_lck);
+    if (bgs->ckpt_flags)
+        checkpoint_requested = true;
+    SpinLockRelease(&bgs->ckpt_lck);
+
+    /*
      * Loop forever
      */
     for (;;)
@@ -397,6 +415,7 @@ BackgroundWriterMain(void)
             ExitOnAnyError = true;
             /* Close down the database */
             ShutdownXLOG(0, 0);
+
             /* Normal exit from the bgwriter is here */
             proc_exit(0);        /* done */
         }
@@ -418,14 +437,25 @@ BackgroundWriterMain(void)
         }

         /*
+         * Check if we've exited recovery. We do this after determining
+         * whether to perform a checkpoint or not, to be sure that we
+         * perform a real checkpoint and not a restartpoint, if someone
+         * (like the startup process!) requested a checkpoint immediately
+         * after exiting recovery. And we must have the right TimeLineID
+         * when we perform a checkpoint.
+         */
+         if (BgWriterRecoveryMode && !IsRecoveryProcessingMode())
+          {
+            elog(DEBUG1, "bgwriter changing from recovery to normal mode");
+            BgWriterRecoveryMode = false;
+        }
+
+        /*
          * Do a checkpoint if requested, otherwise do one cycle of
          * dirty-buffer writing.
          */
         if (do_checkpoint)
         {
-            /* use volatile pointer to prevent code rearrangement */
-            volatile BgWriterShmemStruct *bgs = BgWriterShmem;
-
             /*
              * Atomically fetch the request flags to figure out what kind of a
              * checkpoint we should perform, and increase the started-counter
@@ -444,7 +474,8 @@ BackgroundWriterMain(void)
              * implementation will not generate warnings caused by
              * CheckPointTimeout < CheckPointWarning.
              */
-            if ((flags & CHECKPOINT_CAUSE_XLOG) &&
+            if (!BgWriterRecoveryMode &&
+                (flags & CHECKPOINT_CAUSE_XLOG) &&
                 elapsed_secs < CheckPointWarning)
                 ereport(LOG,
                         (errmsg("checkpoints are occurring too frequently (%d seconds apart)",
@@ -455,14 +486,18 @@ BackgroundWriterMain(void)
              * Initialize bgwriter-private variables used during checkpoint.
              */
             ckpt_active = true;
-            ckpt_start_recptr = GetInsertRecPtr();
+            if (!BgWriterRecoveryMode)
+                ckpt_start_recptr = GetInsertRecPtr();
             ckpt_start_time = now;
             ckpt_cached_elapsed = 0;

             /*
              * Do the checkpoint.
              */
-            CreateCheckPoint(flags);
+            if (!BgWriterRecoveryMode)
+                CreateCheckPoint(flags);
+            else
+                CreateRestartPoint(flags);

             /*
              * After any checkpoint, close all smgr files.    This is so we
@@ -507,7 +542,7 @@ CheckArchiveTimeout(void)
     pg_time_t    now;
     pg_time_t    last_time;

-    if (XLogArchiveTimeout <= 0)
+    if (XLogArchiveTimeout <= 0 || IsRecoveryProcessingMode())
         return;

     now = (pg_time_t) time(NULL);
@@ -586,7 +621,8 @@ BgWriterNap(void)
         (ckpt_active ? ImmediateCheckpointRequested() : checkpoint_requested))
             break;
         pg_usleep(1000000L);
-        AbsorbFsyncRequests();
+        if (!IsRecoveryProcessingMode())
+            AbsorbFsyncRequests();
         udelay -= 1000000L;
     }

@@ -714,16 +750,19 @@ IsCheckpointOnSchedule(double progress)
      * However, it's good enough for our purposes, we're only calculating an
      * estimate anyway.
      */
-    recptr = GetInsertRecPtr();
-    elapsed_xlogs =
-        (((double) (int32) (recptr.xlogid - ckpt_start_recptr.xlogid)) * XLogSegsPerFile +
-         ((double) recptr.xrecoff - (double) ckpt_start_recptr.xrecoff) / XLogSegSize) /
-        CheckPointSegments;
-
-    if (progress < elapsed_xlogs)
+    if (!IsRecoveryProcessingMode())
     {
-        ckpt_cached_elapsed = elapsed_xlogs;
-        return false;
+        recptr = GetInsertRecPtr();
+        elapsed_xlogs =
+            (((double) (int32) (recptr.xlogid - ckpt_start_recptr.xlogid)) * XLogSegsPerFile +
+             ((double) recptr.xrecoff - (double) ckpt_start_recptr.xrecoff) / XLogSegSize) /
+            CheckPointSegments;
+
+        if (progress < elapsed_xlogs)
+        {
+            ckpt_cached_elapsed = elapsed_xlogs;
+            return false;
+        }
     }

     /*
@@ -850,6 +889,7 @@ BgWriterShmemInit(void)
  *
  * flags is a bitwise OR of the following:
  *    CHECKPOINT_IS_SHUTDOWN: checkpoint is for database shutdown.
+ *    CHECKPOINT_IS_STARTUP: checkpoint is for database startup.
  *    CHECKPOINT_IMMEDIATE: finish the checkpoint ASAP,
  *        ignoring checkpoint_completion_target parameter.
  *    CHECKPOINT_FORCE: force a checkpoint even if no XLOG activity has occured
@@ -916,6 +956,18 @@ RequestCheckpoint(int flags)
     {
         if (BgWriterShmem->bgwriter_pid == 0)
         {
+            /*
+             * The only difference between a startup checkpoint and a normal
+             * online checkpoint is that it's quite normal for the bgwriter
+             * to not be up yet when the startup checkpoint is requested.
+             * (it might be, though). That's ok, background writer will
+             * perform the checkpoint as soon as it starts up.
+             */
+            if (flags & CHECKPOINT_STARTUP)
+            {
+                Assert(!(flags & CHECKPOINT_WAIT));
+                break;
+            }
             if (ntries >= 20)        /* max wait 2.0 sec */
             {
                 elog((flags & CHECKPOINT_WAIT) ? ERROR : LOG,
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index 3380b80..15fc7ad 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -226,10 +226,36 @@ static int    Shutdown = NoShutdown;

 static bool FatalError = false; /* T if recovering from backend crash */

+/* State of WAL redo */
+#define            NoRecovery            0
+#define            RecoveryStarted        1
+#define            RecoveryConsistent    2
+#define            RecoveryCompleted    3
+
+static int    RecoveryStatus = NoRecovery;
+
 /*
  * We use a simple state machine to control startup, shutdown, and
  * crash recovery (which is rather like shutdown followed by startup).
  *
+ * After doing all the postmaster initialization work, we enter PM_STARTUP
+ * state and the startup process is launched. The startup process begins by
+ * reading the control file and other preliminary initialization steps. When
+ * it's ready to start WAL redo, it signals postmaster, and we switch to
+ * PM_RECOVERY phase. The background writer is launched, while the startup
+ * process continues applying WAL.
+ *
+ * After reaching a consistent point in WAL redo, startup process signals
+ * us again, and we switch to PM_RECOVERY_CONSISTENT phase. There's currently
+ * no difference between PM_RECOVERY and PM_RECOVERY_CONSISTENT, but we
+ * could start accepting connections to perform read-only queries at this
+ * point, if we had the infrastructure to do that.
+ *
+ * When the WAL redo is finished, the startup process signals us the third
+ * time, and we switch to PM_RUN state. The startup process can also skip the
+ * recovery and consistent recovery phases altogether, as it will during
+ * normal startup when there's no recovery to be done, for example.
+ *
  * Normal child backends can only be launched when we are in PM_RUN state.
  * (We also allow it in PM_WAIT_BACKUP state, but only for superusers.)
  * In other states we handle connection requests by launching "dead_end"
@@ -254,6 +280,8 @@ typedef enum
 {
     PM_INIT,                    /* postmaster starting */
     PM_STARTUP,                    /* waiting for startup subprocess */
+    PM_RECOVERY,                /* in recovery mode */
+    PM_RECOVERY_CONSISTENT,        /* consistent recovery mode */
     PM_RUN,                        /* normal "database is alive" state */
     PM_WAIT_BACKUP,                /* waiting for online backup mode to end */
     PM_WAIT_BACKENDS,            /* waiting for live backends to exit */
@@ -307,6 +335,7 @@ static void pmdie(SIGNAL_ARGS);
 static void reaper(SIGNAL_ARGS);
 static void sigusr1_handler(SIGNAL_ARGS);
 static void dummy_handler(SIGNAL_ARGS);
+static void CheckRecoverySignals(void);
 static void CleanupBackend(int pid, int exitstatus);
 static void HandleChildCrash(int pid, int exitstatus, const char *procname);
 static void LogChildExit(int lev, const char *procname,
@@ -1302,7 +1331,9 @@ ServerLoop(void)
          * state that prevents it, start one.  It doesn't matter if this
          * fails, we'll just try again later.
          */
-        if (BgWriterPID == 0 && pmState == PM_RUN)
+        if (BgWriterPID == 0 &&
+            (pmState == PM_RUN || pmState == PM_RECOVERY ||
+             pmState == PM_RECOVERY_CONSISTENT))
             BgWriterPID = StartBackgroundWriter();

         /*
@@ -1982,7 +2013,7 @@ pmdie(SIGNAL_ARGS)
             ereport(LOG,
                     (errmsg("received smart shutdown request")));

-            if (pmState == PM_RUN)
+            if (pmState == PM_RUN || pmState == PM_RECOVERY || pmState == PM_RECOVERY_CONSISTENT)
             {
                 /* autovacuum workers are told to shut down immediately */
                 SignalAutovacWorkers(SIGTERM);
@@ -2019,7 +2050,14 @@ pmdie(SIGNAL_ARGS)

             if (StartupPID != 0)
                 signal_child(StartupPID, SIGTERM);
-            if (pmState == PM_RUN || pmState == PM_WAIT_BACKUP)
+            if (pmState == PM_RECOVERY)
+            {
+                /* only bgwriter is active in this state */
+                pmState = PM_WAIT_BACKENDS;
+            }
+            if (pmState == PM_RUN ||
+                pmState == PM_WAIT_BACKUP ||
+                pmState == PM_RECOVERY_CONSISTENT)
             {
                 ereport(LOG,
                         (errmsg("aborting any active transactions")));
@@ -2116,10 +2154,22 @@ reaper(SIGNAL_ARGS)
         if (pid == StartupPID)
         {
             StartupPID = 0;
-            Assert(pmState == PM_STARTUP);

-            /* FATAL exit of startup is treated as catastrophic */
-            if (!EXIT_STATUS_0(exitstatus))
+            /*
+             * Check if we've received a signal from the startup process
+             * first. This can change pmState. If the startup process sends
+             * a signal, and exits immediately after that, we might not have
+             * processed the signal yet, and we need to know if it completed
+             * recovery before exiting.
+             */
+            CheckRecoverySignals();
+
+            /*
+             * Unexpected exit of startup process (including FATAL exit)
+             * during PM_STARTUP is treated as catastrophic. There is no
+             * other processes running yet.
+             */
+            if (pmState == PM_STARTUP)
             {
                 LogChildExit(LOG, _("startup process"),
                              pid, exitstatus);
@@ -2127,60 +2177,27 @@ reaper(SIGNAL_ARGS)
                 (errmsg("aborting startup due to startup process failure")));
                 ExitPostmaster(1);
             }
-
             /*
-             * Startup succeeded - we are done with system startup or
-             * recovery.
+             * Any unexpected exit (including FATAL exit) of the startup
+             * process is treated as a crash.
              */
-            FatalError = false;
-
-            /*
-             * Go to shutdown mode if a shutdown request was pending.
-             */
-            if (Shutdown > NoShutdown)
+            if (!EXIT_STATUS_0(exitstatus))
             {
-                pmState = PM_WAIT_BACKENDS;
-                /* PostmasterStateMachine logic does the rest */
+                HandleChildCrash(pid, exitstatus,
+                                 _("startup process"));
                 continue;
             }
-
-            /*
-             * Otherwise, commence normal operations.
-             */
-            pmState = PM_RUN;
-
-            /*
-             * Load the flat authorization file into postmaster's cache. The
-             * startup process has recomputed this from the database contents,
-             * so we wait till it finishes before loading it.
-             */
-            load_role();
-
             /*
-             * Crank up the background writer.    It doesn't matter if this
-             * fails, we'll just try again later.
-             */
-            Assert(BgWriterPID == 0);
-            BgWriterPID = StartBackgroundWriter();
-
-            /*
-             * Likewise, start other special children as needed.  In a restart
-             * situation, some of them may be alive already.
+             * Startup process exited normally, but didn't finish recovery.
+             * This can happen if someone else than postmaster kills the
+             * startup process with SIGTERM. Treat it like a crash.
              */
-            if (WalWriterPID == 0)
-                WalWriterPID = StartWalWriter();
-            if (AutoVacuumingActive() && AutoVacPID == 0)
-                AutoVacPID = StartAutoVacLauncher();
-            if (XLogArchivingActive() && PgArchPID == 0)
-                PgArchPID = pgarch_start();
-            if (PgStatPID == 0)
-                PgStatPID = pgstat_start();
-
-            /* at this point we are really open for business */
-            ereport(LOG,
-                 (errmsg("database system is ready to accept connections")));
-
-            continue;
+            if (pmState == PM_RECOVERY || pmState == PM_RECOVERY_CONSISTENT)
+            {
+                HandleChildCrash(pid, exitstatus,
+                                 _("startup process"));
+                continue;
+            }
         }

         /*
@@ -2443,6 +2460,18 @@ HandleChildCrash(int pid, int exitstatus, const char *procname)
         }
     }

+    /* Take care of the startup process too */
+    if (pid == StartupPID)
+        StartupPID = 0;
+    else if (StartupPID != 0 && !FatalError)
+    {
+        ereport(DEBUG2,
+                (errmsg_internal("sending %s to process %d",
+                                 (SendStop ? "SIGSTOP" : "SIGQUIT"),
+                                 (int) StartupPID)));
+        signal_child(BgWriterPID, (SendStop ? SIGSTOP : SIGQUIT));
+    }
+
     /* Take care of the bgwriter too */
     if (pid == BgWriterPID)
         BgWriterPID = 0;
@@ -2514,7 +2543,9 @@ HandleChildCrash(int pid, int exitstatus, const char *procname)

     FatalError = true;
     /* We now transit into a state of waiting for children to die */
-    if (pmState == PM_RUN ||
+    if (pmState == PM_RECOVERY ||
+        pmState == PM_RECOVERY_CONSISTENT ||
+        pmState == PM_RUN ||
         pmState == PM_WAIT_BACKUP ||
         pmState == PM_SHUTDOWN)
         pmState = PM_WAIT_BACKENDS;
@@ -2582,6 +2613,128 @@ LogChildExit(int lev, const char *procname, int pid, int exitstatus)
 static void
 PostmasterStateMachine(void)
 {
+    /* Startup states */
+
+    if (pmState == PM_STARTUP && RecoveryStatus > NoRecovery)
+    {
+        /* Recovery has started */
+
+        /*
+         * Go to shutdown mode if a shutdown request was pending.
+         */
+        if (Shutdown > NoShutdown)
+        {
+            pmState = PM_WAIT_BACKENDS;
+            /* PostmasterStateMachine logic does the rest */
+        }
+        else
+        {
+            /*
+             * Crank up the background writer.    It doesn't matter if this
+             * fails, we'll just try again later.
+             */
+            Assert(BgWriterPID == 0);
+            BgWriterPID = StartBackgroundWriter();
+
+            pmState = PM_RECOVERY;
+        }
+    }
+    if (pmState == PM_RECOVERY && RecoveryStatus >= RecoveryConsistent)
+    {
+        /*
+         * Go to shutdown mode if a shutdown request was pending.
+         */
+        if (Shutdown > NoShutdown)
+        {
+            pmState = PM_WAIT_BACKENDS;
+            /* PostmasterStateMachine logic does the rest */
+        }
+        else
+        {
+            /*
+             * Startup process has entered recovery. We consider that good
+             * enough to reset FatalError.
+             */
+            pmState = PM_RECOVERY_CONSISTENT;
+            FatalError = false;
+
+            /*
+             * Load the flat authorization file into postmaster's cache. The
+             * startup process won't have recomputed this from the database yet,
+             * so we it may change following recovery.
+             */
+            load_role();
+
+            /*
+             * Likewise, start other special children as needed.
+             */
+            Assert(PgStatPID == 0);
+            PgStatPID = pgstat_start();
+
+            /* XXX at this point we could accept read-only connections */
+            ereport(DEBUG1,
+                 (errmsg("database system is in consistent recovery mode")));
+        }
+    }
+    if ((pmState == PM_RECOVERY || pmState == PM_RECOVERY_CONSISTENT || pmState == PM_STARTUP) && RecoveryStatus ==
RecoveryCompleted)
+    {
+        /*
+         * Startup succeeded - we are done with system startup or
+         * recovery.
+         */
+        FatalError = false;
+
+        /*
+         * Go to shutdown mode if a shutdown request was pending.
+         */
+        if (Shutdown > NoShutdown)
+        {
+            pmState = PM_WAIT_BACKENDS;
+            /* PostmasterStateMachine logic does the rest */
+        }
+        else
+        {
+            /*
+             * Otherwise, commence normal operations.
+             */
+            pmState = PM_RUN;
+
+            /*
+             * Load the flat authorization file into postmaster's cache. The
+             * startup process has recomputed this from the database contents,
+             * so we wait till it finishes before loading it.
+             */
+            load_role();
+
+            /*
+             * Crank up the background writer, if we didn't do that already
+             * when we entered consistent recovery phase.  It doesn't matter
+             * if this fails, we'll just try again later.
+             */
+            if (BgWriterPID == 0)
+                BgWriterPID = StartBackgroundWriter();
+
+            /*
+             * Likewise, start other special children as needed.  In a restart
+             * situation, some of them may be alive already.
+             */
+            if (WalWriterPID == 0)
+                WalWriterPID = StartWalWriter();
+            if (AutoVacuumingActive() && AutoVacPID == 0)
+                AutoVacPID = StartAutoVacLauncher();
+            if (XLogArchivingActive() && PgArchPID == 0)
+                PgArchPID = pgarch_start();
+            if (PgStatPID == 0)
+                PgStatPID = pgstat_start();
+
+            /* at this point we are really open for business */
+            ereport(LOG,
+                (errmsg("database system is ready to accept connections")));
+        }
+    }
+
+    /* Shutdown states */
+
     if (pmState == PM_WAIT_BACKUP)
     {
         /*
@@ -2734,6 +2887,8 @@ PostmasterStateMachine(void)
         shmem_exit(1);
         reset_shared(PostPortNumber);

+        RecoveryStatus = NoRecovery;
+
         StartupPID = StartupDataBase();
         Assert(StartupPID != 0);
         pmState = PM_STARTUP;
@@ -3838,6 +3993,37 @@ ExitPostmaster(int status)
 }

 /*
+ * common code used in sigusr1_handler() and reaper() to handle
+ * recovery-related signals from startup process
+ */
+static void
+CheckRecoverySignals(void)
+{
+    bool changed = false;
+
+    if (CheckPostmasterSignal(PMSIGNAL_RECOVERY_STARTED))
+    {
+        Assert(pmState == PM_STARTUP);
+
+        RecoveryStatus = RecoveryStarted;
+        changed = true;
+    }
+    if (CheckPostmasterSignal(PMSIGNAL_RECOVERY_CONSISTENT))
+    {
+        RecoveryStatus = RecoveryConsistent;
+        changed = true;
+    }
+    if (CheckPostmasterSignal(PMSIGNAL_RECOVERY_COMPLETED))
+    {
+        RecoveryStatus = RecoveryCompleted;
+        changed = true;
+    }
+
+    if (changed)
+        PostmasterStateMachine();
+}
+
+/*
  * sigusr1_handler - handle signal conditions from child processes
  */
 static void
@@ -3847,6 +4033,8 @@ sigusr1_handler(SIGNAL_ARGS)

     PG_SETMASK(&BlockSig);

+    CheckRecoverySignals();
+
     if (CheckPostmasterSignal(PMSIGNAL_PASSWORD_CHANGE))
     {
         /*
diff --git a/src/backend/storage/buffer/README b/src/backend/storage/buffer/README
index 62b22bd..a7b81e3 100644
--- a/src/backend/storage/buffer/README
+++ b/src/backend/storage/buffer/README
@@ -268,3 +268,12 @@ out (and anyone else who flushes buffer contents to disk must do so too).
 This ensures that the page image transferred to disk is reasonably consistent.
 We might miss a hint-bit update or two but that isn't a problem, for the same
 reasons mentioned under buffer access rules.
+
+As of 8.4, background writer starts during recovery mode when there is
+some form of potentially extended recovery to perform. It performs an
+identical service to normal processing, except that checkpoints it
+writes are technically restartpoints. Flushing outstanding WAL for dirty
+buffers is also skipped, though there shouldn't ever be new WAL entries
+at that time in any case. We could choose to start background writer
+immediately but we hold off until we can prove the database is in a
+consistent state so that postmaster has a single, clean state change.
diff --git a/src/backend/utils/init/postinit.c b/src/backend/utils/init/postinit.c
index cf98323..b359395 100644
--- a/src/backend/utils/init/postinit.c
+++ b/src/backend/utils/init/postinit.c
@@ -324,7 +324,7 @@ InitCommunication(void)
  * If you're wondering why this is separate from InitPostgres at all:
  * the critical distinction is that this stuff has to happen before we can
  * run XLOG-related initialization, which is done before InitPostgres --- in
- * fact, for cases such as checkpoint creation processes, InitPostgres may
+ * fact, for cases such as the background writer process, InitPostgres may
  * never be done at all.
  */
 void
diff --git a/src/bin/pg_controldata/pg_controldata.c b/src/bin/pg_controldata/pg_controldata.c
index 4ea849d..3bba50a 100644
--- a/src/bin/pg_controldata/pg_controldata.c
+++ b/src/bin/pg_controldata/pg_controldata.c
@@ -197,6 +197,9 @@ main(int argc, char *argv[])
     printf(_("Minimum recovery ending location:     %X/%X\n"),
            ControlFile.minRecoveryPoint.xlogid,
            ControlFile.minRecoveryPoint.xrecoff);
+    printf(_("Minimum safe starting location:       %X/%X\n"),
+           ControlFile.minSafeStartPoint.xlogid,
+           ControlFile.minSafeStartPoint.xrecoff);
     printf(_("Maximum data alignment:               %u\n"),
            ControlFile.maxAlign);
     /* we don't print floatFormat since can't say much useful about it */
diff --git a/src/bin/pg_resetxlog/pg_resetxlog.c b/src/bin/pg_resetxlog/pg_resetxlog.c
index 51cdde1..b20d4bd 100644
--- a/src/bin/pg_resetxlog/pg_resetxlog.c
+++ b/src/bin/pg_resetxlog/pg_resetxlog.c
@@ -603,6 +603,8 @@ RewriteControlFile(void)
     ControlFile.prevCheckPoint.xrecoff = 0;
     ControlFile.minRecoveryPoint.xlogid = 0;
     ControlFile.minRecoveryPoint.xrecoff = 0;
+    ControlFile.minSafeStartPoint.xlogid = 0;
+    ControlFile.minSafeStartPoint.xrecoff = 0;

     /* Now we can force the recorded xlog seg size to the right thing. */
     ControlFile.xlog_seg_size = XLogSegSize;
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 6913f7c..c3b3ec7 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -133,7 +133,16 @@ typedef struct XLogRecData
 } XLogRecData;

 extern TimeLineID ThisTimeLineID;        /* current TLI */
-extern bool InRecovery;
+
+/*
+ * Prior to 8.4, all activity during recovery were carried out by Startup
+ * process. This local variable continues to be used in many parts of the
+ * code to indicate actions taken by RecoveryManagers. Other processes who
+ * potentially perform work during recovery should check
+ * IsRecoveryProcessingMode(), see XLogCtl notes in xlog.c
+ */
+extern bool InRecovery;
+
 extern XLogRecPtr XactLastRecEnd;

 /* these variables are GUC parameters related to XLOG */
@@ -161,11 +170,12 @@ extern bool XLOG_DEBUG;
 #define CHECKPOINT_IS_SHUTDOWN    0x0001    /* Checkpoint is for shutdown */
 #define CHECKPOINT_IMMEDIATE    0x0002    /* Do it without delays */
 #define CHECKPOINT_FORCE        0x0004    /* Force even if no activity */
+#define CHECKPOINT_STARTUP        0x0008    /* Startup checkpoint */
 /* These are important to RequestCheckpoint */
-#define CHECKPOINT_WAIT            0x0008    /* Wait for completion */
+#define CHECKPOINT_WAIT            0x0010    /* Wait for completion */
 /* These indicate the cause of a checkpoint request */
-#define CHECKPOINT_CAUSE_XLOG    0x0010    /* XLOG consumption */
-#define CHECKPOINT_CAUSE_TIME    0x0020    /* Elapsed time */
+#define CHECKPOINT_CAUSE_XLOG    0x0020    /* XLOG consumption */
+#define CHECKPOINT_CAUSE_TIME    0x0040    /* Elapsed time */

 /* Checkpoint statistics */
 typedef struct CheckpointStatsData
@@ -199,6 +209,8 @@ extern void RestoreBkpBlocks(XLogRecPtr lsn, XLogRecord *record, bool cleanup);
 extern void xlog_redo(XLogRecPtr lsn, XLogRecord *record);
 extern void xlog_desc(StringInfo buf, uint8 xl_info, char *rec);

+extern bool IsRecoveryProcessingMode(void);
+
 extern void UpdateControlFile(void);
 extern Size XLOGShmemSize(void);
 extern void XLOGShmemInit(void);
@@ -207,9 +219,12 @@ extern void StartupXLOG(void);
 extern void ShutdownXLOG(int code, Datum arg);
 extern void InitXLOGAccess(void);
 extern void CreateCheckPoint(int flags);
+extern void CreateRestartPoint(int flags);
 extern void XLogPutNextOid(Oid nextOid);
 extern XLogRecPtr GetRedoRecPtr(void);
 extern XLogRecPtr GetInsertRecPtr(void);
 extern void GetNextXidAndEpoch(TransactionId *xid, uint32 *epoch);

+extern void StartupProcessMain(void);
+
 #endif   /* XLOG_H */
diff --git a/src/include/catalog/pg_control.h b/src/include/catalog/pg_control.h
index 400f32c..e69c8ec 100644
--- a/src/include/catalog/pg_control.h
+++ b/src/include/catalog/pg_control.h
@@ -21,7 +21,7 @@


 /* Version identifier for this pg_control format */
-#define PG_CONTROL_VERSION    843
+#define PG_CONTROL_VERSION    847

 /*
  * Body of CheckPoint XLOG records.  This is declared here because we keep
@@ -46,7 +46,7 @@ typedef struct CheckPoint
 #define XLOG_NOOP                        0x20
 #define XLOG_NEXTOID                    0x30
 #define XLOG_SWITCH                        0x40
-
+#define XLOG_RECOVERY_END            0x50

 /* System status indicator */
 typedef enum DBState
@@ -102,6 +102,7 @@ typedef struct ControlFileData
     CheckPoint    checkPointCopy; /* copy of last check point record */

     XLogRecPtr    minRecoveryPoint;        /* must replay xlog to here */
+    XLogRecPtr    minSafeStartPoint;        /* safe point after recovery crashes */

     /*
      * This data is used to check for hardware-architecture compatibility of
diff --git a/src/include/storage/pmsignal.h b/src/include/storage/pmsignal.h
index 3101092..62dddfc 100644
--- a/src/include/storage/pmsignal.h
+++ b/src/include/storage/pmsignal.h
@@ -22,6 +22,9 @@
  */
 typedef enum
 {
+    PMSIGNAL_RECOVERY_STARTED,    /* recovery has started */
+    PMSIGNAL_RECOVERY_CONSISTENT, /* recovery has reached consistent state */
+    PMSIGNAL_RECOVERY_COMPLETED, /* recovery completed */
     PMSIGNAL_PASSWORD_CHANGE,    /* pg_auth file has changed */
     PMSIGNAL_WAKEN_ARCHIVER,    /* send a NOTIFY signal to xlog archiver */
     PMSIGNAL_ROTATE_LOGFILE,    /* send SIGUSR1 to syslogger to rotate logfile */

pgsql-hackers by date:

Previous
From: Andrew Dunstan
Date:
Subject: Re: mingw check hung
Next
From: Bruce Momjian
Date:
Subject: Re: [PATCH] Space reservation v02