Make recovery from WAL be restartable, by executing a checkpoint-like
authorTom Lane <tgl@sss.pgh.pa.us>
Mon, 7 Aug 2006 16:57:57 +0000 (16:57 +0000)
committerTom Lane <tgl@sss.pgh.pa.us>
Mon, 7 Aug 2006 16:57:57 +0000 (16:57 +0000)
operation every so often.  This improves the usefulness of PITR log
shipping for hot standby: formerly, if the standby server crashed, it
was necessary to restart it from the last base backup and replay all
the WAL since then.  Now it will only need to reread about the same
amount of WAL as the master server would.  The behavior might also
come in handy during a long PITR replay sequence.  Simon Riggs,
with some editorialization by Tom Lane.

12 files changed:
src/backend/access/gin/ginxlog.c
src/backend/access/gist/gistxlog.c
src/backend/access/nbtree/nbtxlog.c
src/backend/access/transam/rmgr.c
src/backend/access/transam/xlog.c
src/bin/pg_controldata/pg_controldata.c
src/bin/pg_resetxlog/pg_resetxlog.c
src/include/access/gin.h
src/include/access/gist_private.h
src/include/access/nbtree.h
src/include/access/xlog_internal.h
src/include/catalog/pg_control.h

index eedf02085e113311a78368a960f3bc749f73c56f..c24d09235f86089dfb84bbc17fd2836dfea06e20 100644 (file)
@@ -528,7 +528,8 @@ gin_xlog_cleanup(void) {
 
        topCtx = MemoryContextSwitchTo(opCtx);
 
-       foreach(l, incomplete_splits) {
+       foreach(l, incomplete_splits)
+       {
                ginIncompleteSplit *split = (ginIncompleteSplit *) lfirst(l);
                ginContinueSplit( split );
                MemoryContextReset( opCtx );
@@ -538,3 +539,10 @@ gin_xlog_cleanup(void) {
        MemoryContextDelete(opCtx);
 }
 
+bool
+gin_safe_restartpoint(void)
+{
+       if (incomplete_splits)
+               return false;
+       return true;
+}
index a7f508defd946860269225e974a1ac967b52b561..cbc588b85f6e9ae0419cc0b1364129d07c8b5803 100644 (file)
@@ -818,6 +818,14 @@ gist_xlog_cleanup(void)
        MemoryContextDelete(insertCtx);
 }
 
+bool
+gist_safe_restartpoint(void)
+{
+       if (incomplete_inserts)
+               return false;
+       return true;
+}
+
 
 XLogRecData *
 formSplitRdata(RelFileNode node, BlockNumber blkno, bool page_is_leaf,
index 346da547891b0b1ea56063df20823311f9128662..18081c375f27d0a6901a45472aa7dd5527093dd8 100644 (file)
@@ -794,3 +794,11 @@ btree_xlog_cleanup(void)
        }
        incomplete_splits = NIL;
 }
+
+bool
+btree_safe_restartpoint(void)
+{
+       if (incomplete_splits)
+               return false;
+       return true;
+}
index bc2155d5322aefb8eb91c82d1cda019a41882f03..e1af81b6c4c9eb5287617e1a61de885e73d06552 100644 (file)
 
 
 const RmgrData RmgrTable[RM_MAX_ID + 1] = {
-       {"XLOG", xlog_redo, xlog_desc, NULL, NULL},
-       {"Transaction", xact_redo, xact_desc, NULL, NULL},
-       {"Storage", smgr_redo, smgr_desc, NULL, NULL},
-       {"CLOG", clog_redo, clog_desc, NULL, NULL},
-       {"Database", dbase_redo, dbase_desc, NULL, NULL},
-       {"Tablespace", tblspc_redo, tblspc_desc, NULL, NULL},
-       {"MultiXact", multixact_redo, multixact_desc, NULL, NULL},
-       {"Reserved 7", NULL, NULL, NULL, NULL},
-       {"Reserved 8", NULL, NULL, NULL, NULL},
-       {"Reserved 9", NULL, NULL, NULL, NULL},
-       {"Heap", heap_redo, heap_desc, NULL, NULL},
-       {"Btree", btree_redo, btree_desc, btree_xlog_startup, btree_xlog_cleanup},
-       {"Hash", hash_redo, hash_desc, NULL, NULL},
-       {"Gin", gin_redo, gin_desc, gin_xlog_startup, gin_xlog_cleanup},
-       {"Gist", gist_redo, gist_desc, gist_xlog_startup, gist_xlog_cleanup},
-       {"Sequence", seq_redo, seq_desc, NULL, NULL}
+       {"XLOG", xlog_redo, xlog_desc, NULL, NULL, NULL},
+       {"Transaction", xact_redo, xact_desc, NULL, NULL, NULL},
+       {"Storage", smgr_redo, smgr_desc, NULL, NULL, NULL},
+       {"CLOG", clog_redo, clog_desc, NULL, NULL, NULL},
+       {"Database", dbase_redo, dbase_desc, NULL, NULL, NULL},
+       {"Tablespace", tblspc_redo, tblspc_desc, NULL, NULL, NULL},
+       {"MultiXact", multixact_redo, multixact_desc, NULL, NULL, NULL},
+       {"Reserved 7", NULL, NULL, NULL, NULL, NULL},
+       {"Reserved 8", NULL, NULL, NULL, NULL, NULL},
+       {"Reserved 9", NULL, NULL, NULL, NULL, NULL},
+       {"Heap", heap_redo, heap_desc, NULL, NULL, NULL},
+       {"Btree", btree_redo, btree_desc, btree_xlog_startup, btree_xlog_cleanup, btree_safe_restartpoint},
+       {"Hash", hash_redo, hash_desc, NULL, NULL, NULL},
+       {"Gin", gin_redo, gin_desc, gin_xlog_startup, gin_xlog_cleanup, gin_safe_restartpoint},
+       {"Gist", gist_redo, gist_desc, gist_xlog_startup, gist_xlog_cleanup, gist_safe_restartpoint},
+       {"Sequence", seq_redo, seq_desc, NULL, NULL, NULL}
 };
index 2df92d64a7623ddd6f5408786b1d31b8112cd45b..4617d0afea4b81a60569bfae0e5f6125bec12cc9 100644 (file)
 
 /* File path names (all relative to $PGDATA) */
 #define BACKUP_LABEL_FILE              "backup_label"
+#define BACKUP_LABEL_OLD               "backup_label.old"
 #define RECOVERY_COMMAND_FILE  "recovery.conf"
 #define RECOVERY_COMMAND_DONE  "recovery.done"
 
@@ -185,9 +186,6 @@ static TransactionId recoveryStopXid;
 static time_t recoveryStopTime;
 static bool recoveryStopAfter;
 
-/* constraint set by read_backup_label */
-static XLogRecPtr recoveryMinXlogOffset = {0, 0};
-
 /*
  * During normal operation, the only timeline we care about is ThisTimeLineID.
  * During recovery, however, things are more complicated.  To simplify life
@@ -463,6 +461,7 @@ static void readRecoveryCommandFile(void);
 static void exitArchiveRecovery(TimeLineID endTLI,
                                        uint32 endLogId, uint32 endLogSeg);
 static bool recoveryStopsHere(XLogRecord *record, bool *includeThis);
+static void CheckPointGuts(XLogRecPtr checkPointRedo);
 
 static bool XLogCheckBuffer(XLogRecData *rdata, bool doPageWrites,
                                XLogRecPtr *lsn, BkpBlock *bkpb);
@@ -499,8 +498,8 @@ static void issue_xlog_fsync(void);
 #ifdef WAL_DEBUG
 static void xlog_outrec(StringInfo buf, XLogRecord *record);
 #endif
-static bool read_backup_label(XLogRecPtr *checkPointLoc);
-static void remove_backup_label(void);
+static bool read_backup_label(XLogRecPtr *checkPointLoc,
+                                                         XLogRecPtr *minRecoveryLoc);
 static void rm_redo_error_callback(void *arg);
 
 
@@ -4594,9 +4593,11 @@ StartupXLOG(void)
        CheckPoint      checkPoint;
        bool            wasShutdown;
        bool            needNewTimeLine = false;
+       bool            haveBackupLabel = false;
        XLogRecPtr      RecPtr,
                                LastRec,
                                checkPointLoc,
+                               minRecoveryLoc,
                                EndOfLog;
        uint32          endLogId;
        uint32          endLogSeg;
@@ -4629,12 +4630,18 @@ StartupXLOG(void)
                ereport(LOG,
                                (errmsg("database system shutdown was interrupted at %s",
                                                str_time(ControlFile->time))));
-       else if (ControlFile->state == DB_IN_RECOVERY)
+       else if (ControlFile->state == DB_IN_CRASH_RECOVERY)
                ereport(LOG,
                   (errmsg("database system was interrupted while in recovery at %s",
                                   str_time(ControlFile->time)),
                        errhint("This probably means that some data is corrupted and"
                                        " you will have to use the last backup for recovery.")));
+       else if (ControlFile->state == DB_IN_ARCHIVE_RECOVERY)
+               ereport(LOG,
+                  (errmsg("database system was interrupted while in recovery at log time %s",
+                                  str_time(ControlFile->checkPointCopy.time)),
+                       errhint("If this has occurred more than once some data may be corrupted"
+                                       " and you may need to choose an earlier recovery target.")));
        else if (ControlFile->state == DB_IN_PRODUCTION)
                ereport(LOG,
                                (errmsg("database system was interrupted at %s",
@@ -4673,7 +4680,7 @@ StartupXLOG(void)
                                                recoveryTargetTLI,
                                                ControlFile->checkPointCopy.ThisTimeLineID)));
 
-       if (read_backup_label(&checkPointLoc))
+       if (read_backup_label(&checkPointLoc, &minRecoveryLoc))
        {
                /*
                 * When a backup_label file is present, we want to roll forward from
@@ -4693,6 +4700,8 @@ StartupXLOG(void)
                                        (errmsg("could not locate required checkpoint record"),
                                         errhint("If you are not restoring from a backup, try removing the file \"%s/backup_label\".", DataDir)));
                }
+               /* set flag to delete it later */
+               haveBackupLabel = true;
        }
        else
        {
@@ -4790,17 +4799,51 @@ StartupXLOG(void)
        {
                int                     rmid;
 
+               /*
+                * Update pg_control to show that we are recovering and to show
+                * the selected checkpoint as the place we are starting from.
+                * We also mark pg_control with any minimum recovery stop point
+                * obtained from a backup history file.
+                */
                if (InArchiveRecovery)
+               {
                        ereport(LOG,
                                        (errmsg("automatic recovery in progress")));
+                       ControlFile->state = DB_IN_ARCHIVE_RECOVERY;
+               }
                else
+               {
                        ereport(LOG,
                                        (errmsg("database system was not properly shut down; "
                                                        "automatic recovery in progress")));
-               ControlFile->state = DB_IN_RECOVERY;
+                       ControlFile->state = DB_IN_CRASH_RECOVERY;
+               }
+               ControlFile->prevCheckPoint = ControlFile->checkPoint;
+               ControlFile->checkPoint = checkPointLoc;
+               ControlFile->checkPointCopy = checkPoint;
+               if (minRecoveryLoc.xlogid != 0 || minRecoveryLoc.xrecoff != 0)
+                       ControlFile->minRecoveryPoint = minRecoveryLoc;
                ControlFile->time = time(NULL);
                UpdateControlFile();
 
+               /*
+                * 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 the latest recovery restartpoint instead of going all the way
+                * back to the backup start point.  It seems prudent though to just
+                * rename the file out of the way rather than delete it completely.
+                */
+               if (haveBackupLabel)
+               {
+                       unlink(BACKUP_LABEL_OLD);
+                       if (rename(BACKUP_LABEL_FILE, BACKUP_LABEL_OLD) != 0)
+                               ereport(FATAL,
+                                               (errcode_for_file_access(),
+                                                errmsg("could not rename file \"%s\" to \"%s\": %m",
+                                                               BACKUP_LABEL_FILE, BACKUP_LABEL_OLD)));
+               }
+
                /* Start up the recovery environment */
                XLogInitRelationCache();
 
@@ -4927,7 +4970,7 @@ StartupXLOG(void)
         * Complain if we did not roll forward far enough to render the backup
         * dump consistent.
         */
-       if (XLByteLT(EndOfLog, recoveryMinXlogOffset))
+       if (XLByteLT(EndOfLog, ControlFile->minRecoveryPoint))
        {
                if (needNewTimeLine)    /* stopped because of stop request */
                        ereport(FATAL,
@@ -5051,34 +5094,20 @@ StartupXLOG(void)
                pgstat_reset_all();
 
                /*
-                * Perform a new checkpoint to update our recovery activity to disk.
+                * 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.
-                *
-                * In case we had to use the secondary checkpoint, make sure that it
-                * will still be shown as the secondary checkpoint after this
-                * CreateCheckPoint operation; we don't want the broken primary
-                * checkpoint to become prevCheckPoint...
                 */
-               if (XLByteEQ(checkPointLoc, ControlFile->prevCheckPoint))
-                       ControlFile->checkPoint = checkPointLoc;
-
                CreateCheckPoint(true, true);
 
                /*
                 * Close down recovery environment
                 */
                XLogCloseRelationCache();
-
-               /*
-                * Now that we've checkpointed the recovery, it's safe to flush old
-                * backup_label, if present.
-                */
-               remove_backup_label();
        }
 
        /*
@@ -5464,6 +5493,10 @@ CreateCheckPoint(bool shutdown, bool force)
 
        LWLockRelease(CheckpointStartLock);
 
+       if (!shutdown)
+               ereport(DEBUG2,
+                               (errmsg("checkpoint starting")));
+
        /*
         * Get the other info we need for the checkpoint record.
         */
@@ -5494,16 +5527,7 @@ CreateCheckPoint(bool shutdown, bool force)
         */
        END_CRIT_SECTION();
 
-       if (!shutdown)
-               ereport(DEBUG2,
-                               (errmsg("checkpoint starting")));
-
-       CheckPointCLOG();
-       CheckPointSUBTRANS();
-       CheckPointMultiXact();
-       FlushBufferPool();
-       /* We deliberately delay 2PC checkpointing as long as possible */
-       CheckPointTwoPhase(checkPoint.redo);
+       CheckPointGuts(checkPoint.redo);
 
        START_CRIT_SECTION();
 
@@ -5591,6 +5615,85 @@ CreateCheckPoint(bool shutdown, bool force)
        LWLockRelease(CheckpointLock);
 }
 
+/*
+ * Flush all data in shared memory to disk, and fsync
+ *
+ * This is the common code shared between regular checkpoints and
+ * recovery restartpoints.
+ */
+static void
+CheckPointGuts(XLogRecPtr checkPointRedo)
+{
+       CheckPointCLOG();
+       CheckPointSUBTRANS();
+       CheckPointMultiXact();
+       FlushBufferPool();     /* performs all required fsyncs */
+       /* We deliberately delay 2PC checkpointing as long as possible */
+       CheckPointTwoPhase(checkPointRedo);
+}
+
+/*
+ * 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.
+ */
+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 = time(NULL) - ControlFile->time;
+       if (elapsed_secs < CheckPointTimeout / 2)
+               return;
+
+       /*
+        * Is it safe to checkpoint?  We must ask each of the resource managers
+        * whether they have any partial state information that might prevent a
+        * correct restart from this point.  If so, we skip this opportunity, but
+        * return at the next checkpoint record for another try.
+        */
+       for (rmid = 0; rmid <= RM_MAX_ID; rmid++)
+       {
+               if (RmgrTable[rmid].rm_safe_restartpoint != NULL)
+                       if (!(RmgrTable[rmid].rm_safe_restartpoint()))
+                               return;
+       }
+
+       /*
+        * OK, force data out to disk
+        */
+       CheckPointGuts(checkPoint->redo);
+
+       /*
+        * Update pg_control so that any subsequent crash will restart from
+        * this checkpoint.  Note: ReadRecPtr gives the XLOG address of the
+        * checkpoint record itself.
+        */
+       ControlFile->prevCheckPoint = ControlFile->checkPoint;
+       ControlFile->checkPoint = ReadRecPtr;
+       ControlFile->checkPointCopy = *checkPoint;
+       ControlFile->time = time(NULL);
+       UpdateControlFile();
+
+       ereport(DEBUG2,
+                       (errmsg("recovery restart point at %X/%X",
+                                       checkPoint->redo.xlogid, checkPoint->redo.xrecoff)));
+}
+
 /*
  * Write a NEXTOID log record
  */
@@ -5687,6 +5790,8 @@ xlog_redo(XLogRecPtr lsn, XLogRecord *record)
                        /* Following WAL records should be run with new TLI */
                        ThisTimeLineID = checkPoint.ThisTimeLineID;
                }
+
+               RecoveryRestartPoint(&checkPoint);
        }
        else if (info == XLOG_CHECKPOINT_ONLINE)
        {
@@ -5709,6 +5814,8 @@ xlog_redo(XLogRecPtr lsn, XLogRecord *record)
                        ereport(PANIC,
                                        (errmsg("unexpected timeline ID %u (should be %u) in checkpoint record",
                                                        checkPoint.ThisTimeLineID, ThisTimeLineID)));
+
+               RecoveryRestartPoint(&checkPoint);
        }
        else if (info == XLOG_SWITCH)
        {
@@ -6349,14 +6456,14 @@ pg_xlogfile_name(PG_FUNCTION_ARGS)
  * point, we will fail to restore a consistent database state.
  *
  * We also attempt to retrieve the corresponding backup history file.
- * If successful, set recoveryMinXlogOffset to constrain valid PITR stopping
+ * If successful, set *minRecoveryLoc to constrain valid PITR stopping
  * points.
  *
  * Returns TRUE if a backup_label was found (and fills the checkpoint
  * location into *checkPointLoc); returns FALSE if not.
  */
 static bool
-read_backup_label(XLogRecPtr *checkPointLoc)
+read_backup_label(XLogRecPtr *checkPointLoc, XLogRecPtr *minRecoveryLoc)
 {
        XLogRecPtr      startpoint;
        XLogRecPtr      stoppoint;
@@ -6371,6 +6478,10 @@ read_backup_label(XLogRecPtr *checkPointLoc)
        FILE       *fp;
        char            ch;
 
+       /* Default is to not constrain recovery stop point */
+       minRecoveryLoc->xlogid = 0;
+       minRecoveryLoc->xrecoff = 0;
+
        /*
         * See if label file is present
         */
@@ -6439,7 +6550,7 @@ read_backup_label(XLogRecPtr *checkPointLoc)
                        ereport(FATAL,
                                        (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
                                         errmsg("invalid data in file \"%s\"", histfilename)));
-               recoveryMinXlogOffset = stoppoint;
+               *minRecoveryLoc = stoppoint;
                if (ferror(fp) || FreeFile(fp))
                        ereport(FATAL,
                                        (errcode_for_file_access(),
@@ -6450,24 +6561,6 @@ read_backup_label(XLogRecPtr *checkPointLoc)
        return true;
 }
 
-/*
- * remove_backup_label: remove any extant backup_label after successful
- * recovery.  Once we have completed the end-of-recovery checkpoint there
- * is no reason to have to replay from the start point indicated by the
- * label (and indeed we'll probably have removed/recycled the needed WAL
- * segments), so remove the label to prevent trouble in later crash recoveries.
- */
-static void
-remove_backup_label(void)
-{
-       if (unlink(BACKUP_LABEL_FILE) != 0)
-               if (errno != ENOENT)
-                       ereport(FATAL,
-                                       (errcode_for_file_access(),
-                                        errmsg("could not remove file \"%s\": %m",
-                                                       BACKUP_LABEL_FILE)));
-}
-
 /*
  * Error context callback for errors occurring during rm_redo().
  */
index 2ae03816903466bb95c0509dfd36c1bb316fb403..13e988c776b96b1c8ecc4c4178dd5de345b49ea5 100644 (file)
@@ -50,8 +50,10 @@ dbState(DBState state)
                        return _("shut down");
                case DB_SHUTDOWNING:
                        return _("shutting down");
-               case DB_IN_RECOVERY:
-                       return _("in recovery");
+               case DB_IN_CRASH_RECOVERY:
+                       return _("in crash recovery");
+               case DB_IN_ARCHIVE_RECOVERY:
+                       return _("in archive recovery");
                case DB_IN_PRODUCTION:
                        return _("in production");
        }
@@ -147,40 +149,70 @@ main(int argc, char *argv[])
        snprintf(sysident_str, sizeof(sysident_str), UINT64_FORMAT,
                         ControlFile.system_identifier);
 
-       printf(_("pg_control version number:            %u\n"), ControlFile.pg_control_version);
-       printf(_("Catalog version number:               %u\n"), ControlFile.catalog_version_no);
-       printf(_("Database system identifier:           %s\n"), sysident_str);
-       printf(_("Database cluster state:               %s\n"), dbState(ControlFile.state));
-       printf(_("pg_control last modified:             %s\n"), pgctime_str);
-       printf(_("Current log file ID:                  %u\n"), ControlFile.logId);
-       printf(_("Next log file segment:                %u\n"), ControlFile.logSeg);
+       printf(_("pg_control version number:            %u\n"),
+                  ControlFile.pg_control_version);
+       printf(_("Catalog version number:               %u\n"),
+                  ControlFile.catalog_version_no);
+       printf(_("Database system identifier:           %s\n"),
+                  sysident_str);
+       printf(_("Database cluster state:               %s\n"),
+                  dbState(ControlFile.state));
+       printf(_("pg_control last modified:             %s\n"),
+                  pgctime_str);
+       printf(_("Current log file ID:                  %u\n"),
+                  ControlFile.logId);
+       printf(_("Next log file segment:                %u\n"),
+                  ControlFile.logSeg);
        printf(_("Latest checkpoint location:           %X/%X\n"),
-                  ControlFile.checkPoint.xlogid, ControlFile.checkPoint.xrecoff);
+                  ControlFile.checkPoint.xlogid,
+                  ControlFile.checkPoint.xrecoff);
        printf(_("Prior checkpoint location:            %X/%X\n"),
-         ControlFile.prevCheckPoint.xlogid, ControlFile.prevCheckPoint.xrecoff);
+                  ControlFile.prevCheckPoint.xlogid,
+                  ControlFile.prevCheckPoint.xrecoff);
        printf(_("Latest checkpoint's REDO location:    %X/%X\n"),
-                  ControlFile.checkPointCopy.redo.xlogid, ControlFile.checkPointCopy.redo.xrecoff);
+                  ControlFile.checkPointCopy.redo.xlogid,
+                  ControlFile.checkPointCopy.redo.xrecoff);
        printf(_("Latest checkpoint's UNDO location:    %X/%X\n"),
-                  ControlFile.checkPointCopy.undo.xlogid, ControlFile.checkPointCopy.undo.xrecoff);
-       printf(_("Latest checkpoint's TimeLineID:       %u\n"), ControlFile.checkPointCopy.ThisTimeLineID);
-       printf(_("Latest checkpoint's NextXID:          %u\n"), ControlFile.checkPointCopy.nextXid);
-       printf(_("Latest checkpoint's NextOID:          %u\n"), ControlFile.checkPointCopy.nextOid);
-       printf(_("Latest checkpoint's NextMultiXactId:  %u\n"), ControlFile.checkPointCopy.nextMulti);
-       printf(_("Latest checkpoint's NextMultiOffset:  %u\n"), ControlFile.checkPointCopy.nextMultiOffset);
-       printf(_("Time of latest checkpoint:            %s\n"), ckpttime_str);
-       printf(_("Maximum data alignment:               %u\n"), ControlFile.maxAlign);
+                  ControlFile.checkPointCopy.undo.xlogid,
+                  ControlFile.checkPointCopy.undo.xrecoff);
+       printf(_("Latest checkpoint's TimeLineID:       %u\n"),
+                  ControlFile.checkPointCopy.ThisTimeLineID);
+       printf(_("Latest checkpoint's NextXID:          %u\n"),
+                  ControlFile.checkPointCopy.nextXid);
+       printf(_("Latest checkpoint's NextOID:          %u\n"),
+                  ControlFile.checkPointCopy.nextOid);
+       printf(_("Latest checkpoint's NextMultiXactId:  %u\n"),
+                  ControlFile.checkPointCopy.nextMulti);
+       printf(_("Latest checkpoint's NextMultiOffset:  %u\n"),
+                  ControlFile.checkPointCopy.nextMultiOffset);
+       printf(_("Time of latest checkpoint:            %s\n"),
+                  ckpttime_str);
+       printf(_("Minimum recovery ending location:     %X/%X\n"),
+                  ControlFile.minRecoveryPoint.xlogid,
+                  ControlFile.minRecoveryPoint.xrecoff);
+       printf(_("Maximum data alignment:               %u\n"),
+                  ControlFile.maxAlign);
        /* we don't print floatFormat since can't say much useful about it */
-       printf(_("Database block size:                  %u\n"), ControlFile.blcksz);
-       printf(_("Blocks per segment of large relation: %u\n"), ControlFile.relseg_size);
-       printf(_("WAL block size:                       %u\n"), ControlFile.xlog_blcksz);
-       printf(_("Bytes per WAL segment:                %u\n"), ControlFile.xlog_seg_size);
-       printf(_("Maximum length of identifiers:        %u\n"), ControlFile.nameDataLen);
-       printf(_("Maximum columns in an index:          %u\n"), ControlFile.indexMaxKeys);
+       printf(_("Database block size:                  %u\n"),
+                  ControlFile.blcksz);
+       printf(_("Blocks per segment of large relation: %u\n"),
+                  ControlFile.relseg_size);
+       printf(_("WAL block size:                       %u\n"),
+                  ControlFile.xlog_blcksz);
+       printf(_("Bytes per WAL segment:                %u\n"),
+                  ControlFile.xlog_seg_size);
+       printf(_("Maximum length of identifiers:        %u\n"),
+                  ControlFile.nameDataLen);
+       printf(_("Maximum columns in an index:          %u\n"),
+                  ControlFile.indexMaxKeys);
        printf(_("Date/time type storage:               %s\n"),
                   (ControlFile.enableIntTimes ? _("64-bit integers") : _("floating-point numbers")));
-       printf(_("Maximum length of locale name:        %u\n"), ControlFile.localeBuflen);
-       printf(_("LC_COLLATE:                           %s\n"), ControlFile.lc_collate);
-       printf(_("LC_CTYPE:                             %s\n"), ControlFile.lc_ctype);
+       printf(_("Maximum length of locale name:        %u\n"),
+                  ControlFile.localeBuflen);
+       printf(_("LC_COLLATE:                           %s\n"),
+                  ControlFile.lc_collate);
+       printf(_("LC_CTYPE:                             %s\n"),
+                  ControlFile.lc_ctype);
 
        return 0;
 }
index 46ade3d974755c91bc8efb44089e945a6a600345..e50126a6b2ffaa44b65bf9960d2173f7d3ea5e81 100644 (file)
@@ -578,6 +578,8 @@ RewriteControlFile(void)
        ControlFile.checkPoint = ControlFile.checkPointCopy.redo;
        ControlFile.prevCheckPoint.xlogid = 0;
        ControlFile.prevCheckPoint.xrecoff = 0;
+       ControlFile.minRecoveryPoint.xlogid = 0;
+       ControlFile.minRecoveryPoint.xrecoff = 0;
 
        /* Contents are protected with a CRC */
        INIT_CRC32(ControlFile.crc);
index b14a155e03fddf9831406a9fe19f87d79c30b8c9..874eefd46131f246ebf4e550301b2728c3d1f1d9 100644 (file)
@@ -234,6 +234,7 @@ extern void gin_redo(XLogRecPtr lsn, XLogRecord *record);
 extern void gin_desc(StringInfo buf, uint8 xl_info, char *rec);
 extern void gin_xlog_startup(void);
 extern void gin_xlog_cleanup(void);
+extern bool gin_safe_restartpoint(void);
 
 /* ginbtree.c */
 
index 619b36dd07740865ccefedc527f40df3c383178f..48377c691cea1f8988233729e766b01f9c0c361c 100644 (file)
@@ -251,6 +251,7 @@ extern void gist_redo(XLogRecPtr lsn, XLogRecord *record);
 extern void gist_desc(StringInfo buf, uint8 xl_info, char *rec);
 extern void gist_xlog_startup(void);
 extern void gist_xlog_cleanup(void);
+extern bool gist_safe_restartpoint(void);
 extern IndexTuple gist_form_invalid_tuple(BlockNumber blkno);
 
 extern XLogRecData *formUpdateRdata(RelFileNode node, Buffer buffer,
index a824ed866d660f17357286431b95ded93bb19476..77a4ed130486fe265f017021159b35d74959bf5a 100644 (file)
@@ -545,5 +545,6 @@ extern void btree_redo(XLogRecPtr lsn, XLogRecord *record);
 extern void btree_desc(StringInfo buf, uint8 xl_info, char *rec);
 extern void btree_xlog_startup(void);
 extern void btree_xlog_cleanup(void);
+extern bool btree_safe_restartpoint(void);
 
 #endif   /* NBTREE_H */
index 7ab0b294214791f1d36ce8579e3fc360aa096c6f..7ebad26292351f403dc8893131ed9782fcbd4c99 100644 (file)
@@ -232,6 +232,7 @@ typedef struct RmgrData
        void            (*rm_desc) (StringInfo buf, uint8 xl_info, char *rec);
        void            (*rm_startup) (void);
        void            (*rm_cleanup) (void);
+       bool            (*rm_safe_restartpoint) (void);
 } RmgrData;
 
 extern const RmgrData RmgrTable[];
index a50a956d736a024406244e264ef0860410850572..23f72e1e2dfd671a2f049d784e09fb52715b8484 100644 (file)
@@ -22,7 +22,7 @@
 
 
 /* Version identifier for this pg_control format */
-#define PG_CONTROL_VERSION     820
+#define PG_CONTROL_VERSION     821
 
 /*
  * Body of CheckPoint XLOG records.  This is declared here because we keep
@@ -56,7 +56,8 @@ typedef enum DBState
        DB_STARTUP = 0,
        DB_SHUTDOWNED,
        DB_SHUTDOWNING,
-       DB_IN_RECOVERY,
+       DB_IN_CRASH_RECOVERY,
+       DB_IN_ARCHIVE_RECOVERY,
        DB_IN_PRODUCTION
 } DBState;
 
@@ -107,6 +108,8 @@ typedef struct ControlFileData
 
        CheckPoint      checkPointCopy; /* copy of last check point record */
 
+       XLogRecPtr      minRecoveryPoint;       /* must replay xlog to here */
+
        /*
         * This data is used to check for hardware-architecture compatibility of
         * the database and the backend executable.  We need not check endianness