Allow pg_basebackup from standby node with safety checking.
authorSimon Riggs <simon@2ndQuadrant.com>
Wed, 25 Jan 2012 18:02:04 +0000 (18:02 +0000)
committerSimon Riggs <simon@2ndQuadrant.com>
Wed, 25 Jan 2012 18:02:04 +0000 (18:02 +0000)
Base backup follows recommended procedure, plus goes to great
lengths to ensure that partial page writes are avoided.

Jun Ishizuka and Fujii Masao, with minor modifications

doc/src/sgml/ref/pg_basebackup.sgml
src/backend/access/transam/xlog.c
src/backend/postmaster/checkpointer.c
src/backend/postmaster/postmaster.c
src/backend/replication/basebackup.c
src/backend/utils/misc/guc.c
src/bin/pg_controldata/pg_controldata.c
src/bin/pg_resetxlog/pg_resetxlog.c
src/include/access/xlog.h
src/include/access/xlog_internal.h
src/include/catalog/pg_control.h

index 05d5bedef93c7dc4ae3758544da98d1d017a7e65..c654a364124c7554bd01656f02ee5d96acf778b4 100644 (file)
@@ -64,6 +64,48 @@ PostgreSQL documentation
    better from a performance point of view to take only one backup, and copy
    the result.
   </para>
+
+  <para>
+   <application>pg_basebackup</application> can make a base backup from
+   not only the master but also the standby. To take a backup from the standby,
+   set up the standby so that it can accept replication connections (that is, set
+   <varname>max_wal_senders</> and <xref linkend="guc-hot-standby">,
+   and configure <link linkend="auth-pg-hba-conf">host-based authentication</link>).
+   You will also need to enable <xref linkend="guc-full-page-writes"> on the master.
+  </para>
+
+  <para>
+   Note that there are some limitations in an online backup from the standby:
+
+   <itemizedlist>
+    <listitem>
+     <para>
+      The backup history file is not created in the database cluster backed up.
+     </para>
+    </listitem>
+    <listitem>
+     <para>
+      There is no guarantee that all WAL files required for the backup are archived
+      at the end of backup. If you are planning to use the backup for an archive
+      recovery and want to ensure that all required files are available at that moment,
+      you need to include them into the backup by using <literal>-x</> option.
+     </para>
+    </listitem>
+    <listitem>
+     <para>
+      If the standby is promoted to the master during online backup, the backup fails.
+     </para>
+    </listitem>
+    <listitem>
+     <para>
+      All WAL records required for the backup must contain sufficient full-page writes,
+      which requires you to enable <varname>full_page_writes</> on the master and
+      not to use the tool like <application>pg_compresslog</> as
+      <varname>archive_command</> to remove full-page writes from WAL files.
+     </para>
+    </listitem>
+   </itemizedlist>
+  </para>
  </refsect1>
 
  <refsect1>
index ce659ec63d920bd29755aa393c55c600fe052d71..4b273a8318f44324d911ed16a52907052fdeb56a 100644 (file)
@@ -156,6 +156,14 @@ HotStandbyState standbyState = STANDBY_DISABLED;
 
 static XLogRecPtr LastRec;
 
+/*
+ * During recovery, lastFullPageWrites keeps track of full_page_writes that
+ * the replayed WAL records indicate. It's initialized with full_page_writes
+ * that the recovery starting checkpoint record indicates, and then updated
+ * each time XLOG_FPW_CHANGE record is replayed.
+ */
+static bool lastFullPageWrites;
+
 /*
  * Local copy of SharedRecoveryInProgress variable. True actually means "not
  * known, need to check the shared state".
@@ -354,6 +362,15 @@ typedef struct XLogCtlInsert
    XLogRecPtr  RedoRecPtr;     /* current redo point for insertions */
    bool        forcePageWrites;    /* forcing full-page writes for PITR? */
 
+   /*
+    * fullPageWrites is the master copy used by all backends to determine
+    * whether to write full-page to WAL, instead of using process-local
+    * one. This is required because, when full_page_writes is changed
+    * by SIGHUP, we must WAL-log it before it actually affects
+    * WAL-logging by backends. Checkpointer sets at startup or after SIGHUP.
+    */
+   bool        fullPageWrites;
+
    /*
     * exclusiveBackup is true if a backup started with pg_start_backup() is
     * in progress, and nonExclusiveBackups is a counter indicating the number
@@ -460,6 +477,12 @@ typedef struct XLogCtlData
    /* Are we requested to pause recovery? */
    bool        recoveryPause;
 
+   /*
+    * lastFpwDisableRecPtr points to the start of the last replayed
+    * XLOG_FPW_CHANGE record that instructs full_page_writes is disabled.
+    */
+   XLogRecPtr  lastFpwDisableRecPtr;
+
    slock_t     info_lck;       /* locks shared variables shown above */
 } XLogCtlData;
 
@@ -663,7 +686,7 @@ static void xlog_outrec(StringInfo buf, XLogRecord *record);
 #endif
 static void pg_start_backup_callback(int code, Datum arg);
 static bool read_backup_label(XLogRecPtr *checkPointLoc,
-                 bool *backupEndRequired);
+                 bool *backupEndRequired, bool *backupFromStandby);
 static void rm_redo_error_callback(void *arg);
 static int get_sync_bit(int method);
 
@@ -708,7 +731,8 @@ XLogInsert(RmgrId rmid, uint8 info, XLogRecData *rdata)
    unsigned    i;
    bool        updrqst;
    bool        doPageWrites;
-   bool        isLogSwitch = (rmid == RM_XLOG_ID && info == XLOG_SWITCH);
+   bool        isLogSwitch = false;
+   bool        fpwChange = false;
    uint8       info_orig = info;
 
    /* cross-check on whether we should be here or not */
@@ -722,11 +746,30 @@ XLogInsert(RmgrId rmid, uint8 info, XLogRecData *rdata)
    TRACE_POSTGRESQL_XLOG_INSERT(rmid, info);
 
    /*
-    * In bootstrap mode, we don't actually log anything but XLOG resources;
-    * return a phony record pointer.
+    * Handle special cases/records.
     */
-   if (IsBootstrapProcessingMode() && rmid != RM_XLOG_ID)
+   if (rmid == RM_XLOG_ID)
+   {
+       switch (info)
+       {
+           case XLOG_SWITCH:
+               isLogSwitch = true;
+               break;
+
+           case XLOG_FPW_CHANGE:
+               fpwChange = true;
+               break;
+
+           default:
+               break;
+       }
+   }
+   else if (IsBootstrapProcessingMode())
    {
+       /*
+        * In bootstrap mode, we don't actually log anything but XLOG resources;
+        * return a phony record pointer.
+        */
        RecPtr.xlogid = 0;
        RecPtr.xrecoff = SizeOfXLogLongPHD;     /* start of 1st chkpt record */
        return RecPtr;
@@ -756,10 +799,10 @@ begin:;
    /*
     * Decide if we need to do full-page writes in this XLOG record: true if
     * full_page_writes is on or we have a PITR request for it.  Since we
-    * don't yet have the insert lock, forcePageWrites could change under us,
-    * but we'll recheck it once we have the lock.
+    * don't yet have the insert lock, fullPageWrites and forcePageWrites
+    * could change under us, but we'll recheck them once we have the lock.
     */
-   doPageWrites = fullPageWrites || Insert->forcePageWrites;
+   doPageWrites = Insert->fullPageWrites || Insert->forcePageWrites;
 
    len = 0;
    for (rdt = rdata;;)
@@ -939,12 +982,12 @@ begin:;
    }
 
    /*
-    * Also check to see if forcePageWrites was just turned on; if we weren't
-    * already doing full-page writes then go back and recompute. (If it was
-    * just turned off, we could recompute the record without full pages, but
-    * we choose not to bother.)
+    * Also check to see if fullPageWrites or forcePageWrites was just turned on;
+    * if we weren't already doing full-page writes then go back and recompute.
+    * (If it was just turned off, we could recompute the record without full pages,
+    * but we choose not to bother.)
     */
-   if (Insert->forcePageWrites && !doPageWrites)
+   if ((Insert->fullPageWrites || Insert->forcePageWrites) && !doPageWrites)
    {
        /* Oops, must redo it with full-page data. */
        LWLockRelease(WALInsertLock);
@@ -1189,6 +1232,15 @@ begin:;
        WriteRqst = XLogCtl->xlblocks[curridx];
    }
 
+   /*
+    * If the record is an XLOG_FPW_CHANGE, we update full_page_writes
+    * in shared memory before releasing WALInsertLock. This ensures that
+    * an XLOG_FPW_CHANGE record precedes any WAL record affected
+    * by this change of full_page_writes.
+    */
+   if (fpwChange)
+       Insert->fullPageWrites = fullPageWrites;
+
    LWLockRelease(WALInsertLock);
 
    if (updrqst)
@@ -5147,6 +5199,7 @@ BootStrapXLOG(void)
    checkPoint.redo.xlogid = 0;
    checkPoint.redo.xrecoff = XLogSegSize + SizeOfXLogLongPHD;
    checkPoint.ThisTimeLineID = ThisTimeLineID;
+   checkPoint.fullPageWrites = fullPageWrites;
    checkPoint.nextXidEpoch = 0;
    checkPoint.nextXid = FirstNormalTransactionId;
    checkPoint.nextOid = FirstBootstrapObjectId;
@@ -5961,6 +6014,8 @@ StartupXLOG(void)
    uint32      freespace;
    TransactionId oldestActiveXID;
    bool        backupEndRequired = false;
+   bool        backupFromStandby = false;
+   DBState     dbstate_at_startup;
 
    /*
     * Read control file and check XLOG status looks valid.
@@ -6094,7 +6149,8 @@ StartupXLOG(void)
    if (StandbyMode)
        OwnLatch(&XLogCtl->recoveryWakeupLatch);
 
-   if (read_backup_label(&checkPointLoc, &backupEndRequired))
+   if (read_backup_label(&checkPointLoc, &backupEndRequired,
+                         &backupFromStandby))
    {
        /*
         * When a backup_label file is present, we want to roll forward from
@@ -6210,6 +6266,8 @@ StartupXLOG(void)
     */
    ThisTimeLineID = checkPoint.ThisTimeLineID;
 
+   lastFullPageWrites = checkPoint.fullPageWrites;
+
    RedoRecPtr = XLogCtl->Insert.RedoRecPtr = checkPoint.redo;
 
    if (XLByteLT(RecPtr, checkPoint.redo))
@@ -6250,6 +6308,7 @@ StartupXLOG(void)
         * pg_control with any minimum recovery stop point obtained from a
         * backup history file.
         */
+       dbstate_at_startup = ControlFile->state;
        if (InArchiveRecovery)
            ControlFile->state = DB_IN_ARCHIVE_RECOVERY;
        else
@@ -6270,12 +6329,28 @@ StartupXLOG(void)
        }
 
        /*
-        * set backupStartPoint if we're starting recovery from a base backup
+        * Set backupStartPoint if we're starting recovery from a base backup.
+        *
+        * Set backupEndPoint and use minRecoveryPoint as the backup end location
+        * if we're starting recovery from a base backup which was taken from
+        * the standby. In this case, the database system status in pg_control must
+        * indicate DB_IN_ARCHIVE_RECOVERY. If not, which means that backup
+        * is corrupted, so we cancel recovery.
         */
        if (haveBackupLabel)
        {
            ControlFile->backupStartPoint = checkPoint.redo;
            ControlFile->backupEndRequired = backupEndRequired;
+
+           if (backupFromStandby)
+           {
+               if (dbstate_at_startup != DB_IN_ARCHIVE_RECOVERY)
+                   ereport(FATAL,
+                           (errmsg("backup_label contains inconsistent data with control file"),
+                            errhint("This means that the backup is corrupted and you will "
+                                    "have to use another backup for recovery.")));
+               ControlFile->backupEndPoint = ControlFile->minRecoveryPoint;
+           }
        }
        ControlFile->time = (pg_time_t) time(NULL);
        /* No need to hold ControlFileLock yet, we aren't up far enough */
@@ -6564,6 +6639,27 @@ StartupXLOG(void)
                /* Pop the error context stack */
                error_context_stack = errcontext.previous;
 
+               if (!XLogRecPtrIsInvalid(ControlFile->backupStartPoint) &&
+                   XLByteLE(ControlFile->backupEndPoint, EndRecPtr))
+               {
+                   /*
+                    * We have reached the end of base backup, the point where
+                    * the minimum recovery point in pg_control indicates.
+                    * The data on disk is now consistent. Reset backupStartPoint
+                    * and backupEndPoint.
+                    */
+                   elog(DEBUG1, "end of backup reached");
+
+                   LWLockAcquire(ControlFileLock, LW_EXCLUSIVE);
+
+                   MemSet(&ControlFile->backupStartPoint, 0, sizeof(XLogRecPtr));
+                   MemSet(&ControlFile->backupEndPoint, 0, sizeof(XLogRecPtr));
+                   ControlFile->backupEndRequired = false;
+                   UpdateControlFile();
+
+                   LWLockRelease(ControlFileLock);
+               }
+
                /*
                 * Update shared recoveryLastRecPtr after this record has been
                 * replayed.
@@ -6763,6 +6859,16 @@ StartupXLOG(void)
    /* Pre-scan prepared transactions to find out the range of XIDs present */
    oldestActiveXID = PrescanPreparedTransactions(NULL, NULL);
 
+   /*
+    * Update full_page_writes in shared memory and write an
+    * XLOG_FPW_CHANGE record before resource manager writes cleanup
+    * WAL records or checkpoint record is written.
+    */
+   Insert->fullPageWrites = lastFullPageWrites;
+   LocalSetXLogInsertAllowed();
+   UpdateFullPageWrites();
+   LocalXLogInsertAllowed = -1;
+
    if (InRecovery)
    {
        int         rmid;
@@ -7644,6 +7750,7 @@ CreateCheckPoint(int flags)
        LocalSetXLogInsertAllowed();
 
    checkPoint.ThisTimeLineID = ThisTimeLineID;
+   checkPoint.fullPageWrites = Insert->fullPageWrites;
 
    /*
     * Compute new REDO record ptr = location of next XLOG record.
@@ -8358,6 +8465,48 @@ XLogReportParameters(void)
    }
 }
 
+/*
+ * Update full_page_writes in shared memory, and write an
+ * XLOG_FPW_CHANGE record if necessary.
+ */
+void
+UpdateFullPageWrites(void)
+{
+   XLogCtlInsert *Insert = &XLogCtl->Insert;
+
+   /*
+    * Do nothing if full_page_writes has not been changed.
+    *
+    * It's safe to check the shared full_page_writes without the lock,
+    * because we can guarantee that there is no concurrently running
+    * process which can update it.
+    */
+   if (fullPageWrites == Insert->fullPageWrites)
+       return;
+
+   /*
+    * Write an XLOG_FPW_CHANGE record. This allows us to keep
+    * track of full_page_writes during archive recovery, if required.
+    */
+   if (XLogStandbyInfoActive() && !RecoveryInProgress())
+   {
+       XLogRecData rdata;
+
+       rdata.data = (char *) (&fullPageWrites);
+       rdata.len = sizeof(bool);
+       rdata.buffer = InvalidBuffer;
+       rdata.next = NULL;
+
+       XLogInsert(RM_XLOG_ID, XLOG_FPW_CHANGE, &rdata);
+   }
+   else
+   {
+       LWLockAcquire(WALInsertLock, LW_EXCLUSIVE);
+       Insert->fullPageWrites = fullPageWrites;
+       LWLockRelease(WALInsertLock);
+   }
+}
+
 /*
  * XLOG resource manager's routines
  *
@@ -8402,7 +8551,8 @@ xlog_redo(XLogRecPtr lsn, XLogRecord *record)
         * never arrive.
         */
        if (InArchiveRecovery &&
-           !XLogRecPtrIsInvalid(ControlFile->backupStartPoint))
+           !XLogRecPtrIsInvalid(ControlFile->backupStartPoint) &&
+           XLogRecPtrIsInvalid(ControlFile->backupEndPoint))
            ereport(ERROR,
                    (errmsg("online backup was canceled, recovery cannot continue")));
 
@@ -8571,6 +8721,30 @@ xlog_redo(XLogRecPtr lsn, XLogRecord *record)
        /* Check to see if any changes to max_connections give problems */
        CheckRequiredParameterValues();
    }
+   else if (info == XLOG_FPW_CHANGE)
+   {
+       /* use volatile pointer to prevent code rearrangement */
+       volatile XLogCtlData *xlogctl = XLogCtl;
+       bool        fpw;
+
+       memcpy(&fpw, XLogRecGetData(record), sizeof(bool));
+
+       /*
+        * Update the LSN of the last replayed XLOG_FPW_CHANGE record
+        * so that do_pg_start_backup() and do_pg_stop_backup() can check
+        * whether full_page_writes has been disabled during online backup.
+        */
+       if (!fpw)
+       {
+           SpinLockAcquire(&xlogctl->info_lck);
+           if (XLByteLT(xlogctl->lastFpwDisableRecPtr, ReadRecPtr))
+               xlogctl->lastFpwDisableRecPtr = ReadRecPtr;
+           SpinLockRelease(&xlogctl->info_lck);
+       }
+
+       /* Keep track of full_page_writes */
+       lastFullPageWrites = fpw;
+   }
 }
 
 void
@@ -8584,10 +8758,11 @@ xlog_desc(StringInfo buf, uint8 xl_info, char *rec)
        CheckPoint *checkpoint = (CheckPoint *) rec;
 
        appendStringInfo(buf, "checkpoint: redo %X/%X; "
-                        "tli %u; xid %u/%u; oid %u; multi %u; offset %u; "
+                        "tli %u; fpw %s; xid %u/%u; oid %u; multi %u; offset %u; "
                         "oldest xid %u in DB %u; oldest running xid %u; %s",
                         checkpoint->redo.xlogid, checkpoint->redo.xrecoff,
                         checkpoint->ThisTimeLineID,
+                        checkpoint->fullPageWrites ? "true" : "false",
                         checkpoint->nextXidEpoch, checkpoint->nextXid,
                         checkpoint->nextOid,
                         checkpoint->nextMulti,
@@ -8652,6 +8827,13 @@ xlog_desc(StringInfo buf, uint8 xl_info, char *rec)
                         xlrec.max_locks_per_xact,
                         wal_level_str);
    }
+   else if (info == XLOG_FPW_CHANGE)
+   {
+       bool        fpw;
+
+       memcpy(&fpw, rec, sizeof(bool));
+       appendStringInfo(buf, "full_page_writes: %s", fpw ? "true" : "false");
+   }
    else
        appendStringInfo(buf, "UNKNOWN");
 }
@@ -8837,6 +9019,7 @@ XLogRecPtr
 do_pg_start_backup(const char *backupidstr, bool fast, char **labelfile)
 {
    bool        exclusive = (labelfile == NULL);
+   bool        backup_started_in_recovery = false;
    XLogRecPtr  checkpointloc;
    XLogRecPtr  startpoint;
    pg_time_t   stamp_time;
@@ -8848,18 +9031,27 @@ do_pg_start_backup(const char *backupidstr, bool fast, char **labelfile)
    FILE       *fp;
    StringInfoData labelfbuf;
 
+   backup_started_in_recovery = RecoveryInProgress();
+
    if (!superuser() && !is_authenticated_user_replication_role())
        ereport(ERROR,
                (errcode(ERRCODE_INSUFFICIENT_PRIVILEGE),
           errmsg("must be superuser or replication role to run a backup")));
 
-   if (RecoveryInProgress())
+   /*
+    * Currently only non-exclusive backup can be taken during recovery.
+    */
+   if (backup_started_in_recovery && exclusive)
        ereport(ERROR,
                (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
                 errmsg("recovery is in progress"),
                 errhint("WAL control functions cannot be executed during recovery.")));
 
-   if (!XLogIsNeeded())
+   /*
+    * During recovery, we don't need to check WAL level. Because, if WAL level
+    * is not sufficient, it's impossible to get here during recovery.
+    */
+   if (!backup_started_in_recovery && !XLogIsNeeded())
        ereport(ERROR,
                (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
              errmsg("WAL level not sufficient for making an online backup"),
@@ -8885,6 +9077,9 @@ do_pg_start_backup(const char *backupidstr, bool fast, char **labelfile)
     * since we expect that any pages not modified during the backup interval
     * must have been correctly captured by the backup.)
     *
+    * Note that forcePageWrites has no effect during an online backup from
+    * the standby.
+    *
     * We must hold WALInsertLock to change the value of forcePageWrites, to
     * ensure adequate interlocking against XLogInsert().
     */
@@ -8927,17 +9122,32 @@ do_pg_start_backup(const char *backupidstr, bool fast, char **labelfile)
         * Therefore, if a WAL archiver (such as pglesslog) is trying to
         * compress out removable backup blocks, it won't remove any that
         * occur after this point.
+        *
+        * During recovery, we skip forcing XLOG file switch, which means that
+        * the backup taken during recovery is not available for the special
+        * recovery case described above.
         */
-       RequestXLogSwitch();
+       if (!backup_started_in_recovery)
+           RequestXLogSwitch();
 
        do
        {
+           bool        checkpointfpw;
+
            /*
-            * Force a CHECKPOINT.  Aside from being necessary to prevent torn
+            * Force a CHECKPOINT.  Aside from being necessary to prevent torn
             * page problems, this guarantees that two successive backup runs
             * will have different checkpoint positions and hence different
             * history file names, even if nothing happened in between.
             *
+            * During recovery, establish a restartpoint if possible. We use the last
+            * restartpoint as the backup starting checkpoint. This means that two
+            * successive backup runs can have same checkpoint positions.
+            *
+            * Since the fact that we are executing do_pg_start_backup() during
+            * recovery means that checkpointer is running, we can use
+            * RequestCheckpoint() to establish a restartpoint.
+            *
             * We use CHECKPOINT_IMMEDIATE only if requested by user (via
             * passing fast = true).  Otherwise this can take awhile.
             */
@@ -8953,8 +9163,44 @@ do_pg_start_backup(const char *backupidstr, bool fast, char **labelfile)
            LWLockAcquire(ControlFileLock, LW_SHARED);
            checkpointloc = ControlFile->checkPoint;
            startpoint = ControlFile->checkPointCopy.redo;
+           checkpointfpw = ControlFile->checkPointCopy.fullPageWrites;
            LWLockRelease(ControlFileLock);
 
+           if (backup_started_in_recovery)
+           {
+               /* use volatile pointer to prevent code rearrangement */
+               volatile XLogCtlData *xlogctl = XLogCtl;
+               XLogRecPtr      recptr;
+
+               /*
+                * Check to see if all WAL replayed during online backup (i.e.,
+                * since last restartpoint used as backup starting checkpoint)
+                * contain full-page writes.
+                */
+               SpinLockAcquire(&xlogctl->info_lck);
+               recptr = xlogctl->lastFpwDisableRecPtr;
+               SpinLockRelease(&xlogctl->info_lck);
+
+               if (!checkpointfpw || XLByteLE(startpoint, recptr))
+                   ereport(ERROR,
+                           (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+                            errmsg("WAL generated with full_page_writes=off was replayed "
+                                   "since last restartpoint"),
+                            errhint("This means that the backup being taken on standby "
+                                    "is corrupt and should not be used. "
+                                    "Enable full_page_writes and run CHECKPOINT on the master, "
+                                    "and then try an online backup again.")));
+
+               /*
+                * During recovery, since we don't use the end-of-backup WAL
+                * record and don't write the backup history file, the starting WAL
+                * location doesn't need to be unique. This means that two base
+                * backups started at the same time might use the same checkpoint
+                * as starting locations.
+                */
+               gotUniqueStartpoint = true;
+           }
+
            /*
             * If two base backups are started at the same time (in WAL sender
             * processes), we need to make sure that they use different
@@ -8994,6 +9240,8 @@ do_pg_start_backup(const char *backupidstr, bool fast, char **labelfile)
                         checkpointloc.xlogid, checkpointloc.xrecoff);
        appendStringInfo(&labelfbuf, "BACKUP METHOD: %s\n",
                         exclusive ? "pg_start_backup" : "streamed");
+       appendStringInfo(&labelfbuf, "BACKUP FROM: %s\n",
+                        backup_started_in_recovery ? "standby" : "master");
        appendStringInfo(&labelfbuf, "START TIME: %s\n", strfbuf);
        appendStringInfo(&labelfbuf, "LABEL: %s\n", backupidstr);
 
@@ -9088,6 +9336,7 @@ XLogRecPtr
 do_pg_stop_backup(char *labelfile, bool waitforarchive)
 {
    bool        exclusive = (labelfile == NULL);
+   bool        backup_started_in_recovery = false;
    XLogRecPtr  startpoint;
    XLogRecPtr  stoppoint;
    XLogRecData rdata;
@@ -9098,6 +9347,7 @@ do_pg_stop_backup(char *labelfile, bool waitforarchive)
    char        stopxlogfilename[MAXFNAMELEN];
    char        lastxlogfilename[MAXFNAMELEN];
    char        histfilename[MAXFNAMELEN];
+   char        backupfrom[20];
    uint32      _logId;
    uint32      _logSeg;
    FILE       *lfp;
@@ -9107,19 +9357,29 @@ do_pg_stop_backup(char *labelfile, bool waitforarchive)
    int         waits = 0;
    bool        reported_waiting = false;
    char       *remaining;
+   char       *ptr;
+
+   backup_started_in_recovery = RecoveryInProgress();
 
    if (!superuser() && !is_authenticated_user_replication_role())
        ereport(ERROR,
                (errcode(ERRCODE_INSUFFICIENT_PRIVILEGE),
         (errmsg("must be superuser or replication role to run a backup"))));
 
-   if (RecoveryInProgress())
+   /*
+    * Currently only non-exclusive backup can be taken during recovery.
+    */
+   if (backup_started_in_recovery && exclusive)
        ereport(ERROR,
                (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
                 errmsg("recovery is in progress"),
                 errhint("WAL control functions cannot be executed during recovery.")));
 
-   if (!XLogIsNeeded())
+   /*
+    * During recovery, we don't need to check WAL level. Because, if WAL level
+    * is not sufficient, it's impossible to get here during recovery.
+    */
+   if (!backup_started_in_recovery && !XLogIsNeeded())
        ereport(ERROR,
                (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
              errmsg("WAL level not sufficient for making an online backup"),
@@ -9209,6 +9469,82 @@ do_pg_stop_backup(char *labelfile, bool waitforarchive)
                 errmsg("invalid data in file \"%s\"", BACKUP_LABEL_FILE)));
    remaining = strchr(labelfile, '\n') + 1;    /* %n is not portable enough */
 
+   /*
+    * Parse the BACKUP FROM line. If we are taking an online backup from
+    * the standby, we confirm that the standby has not been promoted
+    * during the backup.
+    */
+   ptr = strstr(remaining, "BACKUP FROM:");
+   if (sscanf(ptr, "BACKUP FROM: %19s\n", backupfrom) != 1)
+       ereport(ERROR,
+               (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+                errmsg("invalid data in file \"%s\"", BACKUP_LABEL_FILE)));
+   if (strcmp(backupfrom, "standby") == 0 && !backup_started_in_recovery)
+       ereport(ERROR,
+               (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+                errmsg("the standby was promoted during online backup"),
+                errhint("This means that the backup being taken is corrupt "
+                        "and should not be used. "
+                        "Try taking another online backup.")));
+
+   /*
+    * During recovery, we don't write an end-of-backup record. We assume
+    * that pg_control was backed up last and its minimum recovery
+    * point can be available as the backup end location. Since we don't
+    * have an end-of-backup record, we use the pg_control value to check
+    * whether we've reached the end of backup when starting recovery from
+    * this backup. We have no way of checking if pg_control wasn't backed
+    * up last however.
+    *
+    * We don't force a switch to new WAL file and wait for all the required
+    * files to be archived. This is okay if we use the backup to start
+    * the standby. But, if it's for an archive recovery, to ensure all the
+    * required files are available, a user should wait for them to be archived,
+    * or include them into the backup.
+    *
+    * We return the current minimum recovery point as the backup end
+    * location. Note that it's would be bigger than the exact backup end
+    * location if the minimum recovery point is updated since the backup
+    * of pg_control. This is harmless for current uses.
+    *
+    * XXX currently a backup history file is for informational and debug
+    * purposes only. It's not essential for an online backup. Furthermore,
+    * even if it's created, it will not be archived during recovery because
+    * an archiver is not invoked. So it doesn't seem worthwhile to write
+    * a backup history file during recovery.
+    */
+   if (backup_started_in_recovery)
+   {
+       /* use volatile pointer to prevent code rearrangement */
+       volatile XLogCtlData *xlogctl = XLogCtl;
+       XLogRecPtr  recptr;
+
+       /*
+        * Check to see if all WAL replayed during online backup contain
+        * full-page writes.
+        */
+       SpinLockAcquire(&xlogctl->info_lck);
+       recptr = xlogctl->lastFpwDisableRecPtr;
+       SpinLockRelease(&xlogctl->info_lck);
+
+       if (XLByteLE(startpoint, recptr))
+           ereport(ERROR,
+                   (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+                    errmsg("WAL generated with full_page_writes=off was replayed "
+                           "during online backup"),
+                    errhint("This means that the backup being taken on standby "
+                            "is corrupt and should not be used. "
+                            "Enable full_page_writes and run CHECKPOINT on the master, "
+                            "and then try an online backup again.")));
+
+
+       LWLockAcquire(ControlFileLock, LW_SHARED);
+       stoppoint = ControlFile->minRecoveryPoint;
+       LWLockRelease(ControlFileLock);
+
+       return stoppoint;
+   }
+
    /*
     * Write the backup-end xlog record
     */
@@ -9454,18 +9790,22 @@ GetXLogWriteRecPtr(void)
  * Returns TRUE if a backup_label was found (and fills the checkpoint
  * location and its REDO location into *checkPointLoc and RedoStartLSN,
  * respectively); returns FALSE if not. If this backup_label came from a
- * streamed backup, *backupEndRequired is set to TRUE.
+ * streamed backup, *backupEndRequired is set to TRUE. If this backup_label
+ * was created during recovery, *backupFromStandby is set to TRUE.
  */
 static bool
-read_backup_label(XLogRecPtr *checkPointLoc, bool *backupEndRequired)
+read_backup_label(XLogRecPtr *checkPointLoc, bool *backupEndRequired,
+                 bool *backupFromStandby)
 {
    char        startxlogfilename[MAXFNAMELEN];
    TimeLineID  tli;
    FILE       *lfp;
    char        ch;
    char        backuptype[20];
+   char        backupfrom[20];
 
    *backupEndRequired = false;
+   *backupFromStandby = false;
 
    /*
     * See if label file is present
@@ -9499,16 +9839,22 @@ read_backup_label(XLogRecPtr *checkPointLoc, bool *backupEndRequired)
                (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
                 errmsg("invalid data in file \"%s\"", BACKUP_LABEL_FILE)));
    /*
-    * BACKUP METHOD line is new in 9.1. We can't restore from an older backup
-    * anyway, but since the information on it is not strictly required, don't
-    * error out if it's missing for some reason.
+    * BACKUP METHOD and BACKUP FROM lines are new in 9.2. We can't
+    * restore from an older backup anyway, but since the information on it
+    * is not strictly required, don't error out if it's missing for some reason.
     */
-   if (fscanf(lfp, "BACKUP METHOD: %19s", backuptype) == 1)
+   if (fscanf(lfp, "BACKUP METHOD: %19s\n", backuptype) == 1)
    {
        if (strcmp(backuptype, "streamed") == 0)
            *backupEndRequired = true;
    }
 
+   if (fscanf(lfp, "BACKUP FROM: %19s\n", backupfrom) == 1)
+   {
+       if (strcmp(backupfrom, "standby") == 0)
+           *backupFromStandby = true;
+   }
+
    if (ferror(lfp) || FreeFile(lfp))
        ereport(FATAL,
                (errcode_for_file_access(),
index 0b792d2b105a8b0b401481f82cc1fc397f744ce0..76cb25cd382a1de527007b36425cfa8b2437e916 100644 (file)
@@ -171,6 +171,7 @@ static void CheckArchiveTimeout(void);
 static bool IsCheckpointOnSchedule(double progress);
 static bool ImmediateCheckpointRequested(void);
 static bool CompactCheckpointerRequestQueue(void);
+static void UpdateSharedMemoryConfig(void);
 
 /* Signal handlers */
 
@@ -351,8 +352,12 @@ CheckpointerMain(void)
    if (RecoveryInProgress())
        ThisTimeLineID = GetRecoveryTargetTLI();
 
-   /* Do this once before starting the loop, then just at SIGHUP time. */
-   SyncRepUpdateSyncStandbysDefined();
+   /*
+    * Ensure all shared memory values are set correctly for the config.
+    * Doing this here ensures no race conditions from other concurrent
+    * updaters.
+    */
+   UpdateSharedMemoryConfig();
 
    /*
     * Loop forever
@@ -380,8 +385,19 @@ CheckpointerMain(void)
        {
            got_SIGHUP = false;
            ProcessConfigFile(PGC_SIGHUP);
-           /* update global shmem state for sync rep */
-           SyncRepUpdateSyncStandbysDefined();
+
+           /*
+            * Checkpointer is the last process to shutdown, so we ask
+            * it to hold the keys for a range of other tasks required
+            * most of which have nothing to do with checkpointing at all.
+            *
+            * For various reasons, some config values can change
+            * dynamically so are the primary copy of them is held in
+            * shared memory to make sure all backends see the same value.
+            * We make Checkpointer responsible for updating the shared
+            * memory copy if the parameter setting changes because of SIGHUP.
+            */
+           UpdateSharedMemoryConfig();
        }
        if (checkpoint_requested)
        {
@@ -1239,3 +1255,21 @@ AbsorbFsyncRequests(void)
 
    END_CRIT_SECTION();
 }
+
+/*
+ * Update any shared memory configurations based on config parameters
+ */
+static void
+UpdateSharedMemoryConfig(void)
+{
+   /* update global shmem state for sync rep */
+   SyncRepUpdateSyncStandbysDefined();
+
+   /*
+    * If full_page_writes has been changed by SIGHUP, we update it
+    * in shared memory and write an XLOG_FPW_CHANGE record.
+    */
+   UpdateFullPageWrites();
+
+   elog(DEBUG2, "checkpointer updated shared memory configuration values");
+}
index ad0c17ac7a88cfd074a67cf14ce6cb4f3845bee5..9d242cbfcb7c1a8dc7af571a7b29af7b601dcc41 100644 (file)
@@ -3067,8 +3067,8 @@ PostmasterStateMachine(void)
        else
        {
            /*
-            * Terminate backup mode to avoid recovery after a clean fast
-            * shutdown.  Since a backup can only be taken during normal
+            * Terminate exclusive backup mode to avoid recovery after a clean fast
+            * shutdown.  Since an exclusive backup can only be taken during normal
             * running (and not, for example, while running under Hot Standby)
             * it only makes sense to do this if we reached normal running. If
             * we're still in recovery, the backup file is one we're
index 2fa1f5461356a191559b93b591c9037c6c75b389..81203c9f5ac9dbf38da09e1ff55b29846c83f514 100644 (file)
@@ -180,6 +180,22 @@ perform_base_backup(basebackup_options *opt, DIR *tblspcdir)
                    ti->path == NULL ? 1 : strlen(ti->path),
                    false);
 
+           /* In the main tar, include pg_control last. */
+           if (ti->path == NULL)
+           {
+               struct stat statbuf;
+
+               if (lstat(XLOG_CONTROL_FILE, &statbuf) != 0)
+               {
+                   ereport(ERROR,
+                           (errcode_for_file_access(),
+                            errmsg("could not stat control file \"%s\": %m",
+                                   XLOG_CONTROL_FILE)));
+               }
+
+               sendFile(XLOG_CONTROL_FILE, XLOG_CONTROL_FILE, &statbuf);
+           }
+
            /*
             * If we're including WAL, and this is the main data directory we
             * don't terminate the tar stream here. Instead, we will append
@@ -361,11 +377,6 @@ SendBaseBackup(BaseBackupCmd *cmd)
    MemoryContext old_context;
    basebackup_options opt;
 
-   if (am_cascading_walsender)
-       ereport(FATAL,
-               (errcode(ERRCODE_CANNOT_CONNECT_NOW),
-                errmsg("recovery is still in progress, can't accept WAL streaming connections for backup")));
-
    parse_basebackup_options(cmd->options, &opt);
 
    backup_context = AllocSetContextCreate(CurrentMemoryContext,
@@ -609,6 +620,10 @@ sendDir(char *path, int basepathlen, bool sizeonly)
            strcmp(pathbuf, "./postmaster.opts") == 0)
            continue;
 
+       /* Skip pg_control here to back up it last */
+       if (strcmp(pathbuf, "./global/pg_control") == 0)
+           continue;
+
        if (lstat(pathbuf, &statbuf) != 0)
        {
            if (errno != ENOENT)
index ec8f2f2309b3168470d63b6d932b74afe24f0ca7..7df5292f95140a40e2cc6ff572757cca50602e6f 100644 (file)
@@ -130,7 +130,6 @@ extern int  CommitSiblings;
 extern char *default_tablespace;
 extern char *temp_tablespaces;
 extern bool synchronize_seqscans;
-extern bool fullPageWrites;
 extern int ssl_renegotiation_limit;
 extern char *SSLCipherSuites;
 
index 9fafb7e8e76b5d86a1676d516bd353fff4572c0c..c00183ab4feb41786f83764df5f2b1812aafe0bb 100644 (file)
@@ -209,6 +209,8 @@ main(int argc, char *argv[])
           ControlFile.checkPointCopy.redo.xrecoff);
    printf(_("Latest checkpoint's TimeLineID:       %u\n"),
           ControlFile.checkPointCopy.ThisTimeLineID);
+   printf(_("Latest checkpoint's full_page_writes: %s\n"),
+          ControlFile.checkPointCopy.fullPageWrites ? _("on") : _("off"));
    printf(_("Latest checkpoint's NextXID:          %u/%u\n"),
           ControlFile.checkPointCopy.nextXidEpoch,
           ControlFile.checkPointCopy.nextXid);
@@ -232,6 +234,9 @@ main(int argc, char *argv[])
    printf(_("Backup start location:                %X/%X\n"),
           ControlFile.backupStartPoint.xlogid,
           ControlFile.backupStartPoint.xrecoff);
+   printf(_("Backup end location:                  %X/%X\n"),
+          ControlFile.backupEndPoint.xlogid,
+          ControlFile.backupEndPoint.xrecoff);
    printf(_("End-of-backup record required:        %s\n"),
           ControlFile.backupEndRequired ? _("yes") : _("no"));
    printf(_("Current wal_level setting:            %s\n"),
index a14601ce7b40b01114cf84f6603412f95e81d14f..a3cd37e37fdc78291f54c4850231ff35a09740c4 100644 (file)
@@ -489,6 +489,7 @@ GuessControlValues(void)
    ControlFile.checkPointCopy.redo.xlogid = 0;
    ControlFile.checkPointCopy.redo.xrecoff = SizeOfXLogLongPHD;
    ControlFile.checkPointCopy.ThisTimeLineID = 1;
+   ControlFile.checkPointCopy.fullPageWrites = false;
    ControlFile.checkPointCopy.nextXidEpoch = 0;
    ControlFile.checkPointCopy.nextXid = FirstNormalTransactionId;
    ControlFile.checkPointCopy.nextOid = FirstBootstrapObjectId;
@@ -503,7 +504,7 @@ GuessControlValues(void)
    ControlFile.time = (pg_time_t) time(NULL);
    ControlFile.checkPoint = ControlFile.checkPointCopy.redo;
 
-   /* minRecoveryPoint and backupStartPoint can be left zero */
+   /* minRecoveryPoint, backupStartPoint and backupEndPoint can be left zero */
 
    ControlFile.wal_level = WAL_LEVEL_MINIMAL;
    ControlFile.MaxConnections = 100;
@@ -569,6 +570,8 @@ PrintControlValues(bool guessed)
           sysident_str);
    printf(_("Latest checkpoint's TimeLineID:       %u\n"),
           ControlFile.checkPointCopy.ThisTimeLineID);
+   printf(_("Latest checkpoint's full_page_writes:       %s\n"),
+          ControlFile.checkPointCopy.fullPageWrites ? _("on") : _("off"));
    printf(_("Latest checkpoint's NextXID:          %u/%u\n"),
           ControlFile.checkPointCopy.nextXidEpoch,
           ControlFile.checkPointCopy.nextXid);
@@ -637,6 +640,8 @@ RewriteControlFile(void)
    ControlFile.minRecoveryPoint.xrecoff = 0;
    ControlFile.backupStartPoint.xlogid = 0;
    ControlFile.backupStartPoint.xrecoff = 0;
+   ControlFile.backupEndPoint.xlogid = 0;
+   ControlFile.backupEndPoint.xrecoff = 0;
    ControlFile.backupEndRequired = false;
 
    /*
index 1ddf4bf15f9a0c879551078018f2db6c4e1639c0..f8aecef665b897d47b44510a05cd1587c5971501 100644 (file)
@@ -192,6 +192,7 @@ extern int  XLogArchiveTimeout;
 extern bool XLogArchiveMode;
 extern char *XLogArchiveCommand;
 extern bool EnableHotStandby;
+extern bool fullPageWrites;
 extern bool log_checkpoints;
 
 /* WAL levels */
@@ -307,6 +308,7 @@ extern void CreateCheckPoint(int flags);
 extern bool CreateRestartPoint(int flags);
 extern void XLogPutNextOid(Oid nextOid);
 extern XLogRecPtr XLogRestorePoint(const char *rpName);
+extern void UpdateFullPageWrites(void);
 extern XLogRecPtr GetRedoRecPtr(void);
 extern XLogRecPtr GetInsertRecPtr(void);
 extern XLogRecPtr GetFlushRecPtr(void);
index db6380f7de468df894b1b18b4075ceba25bcae76..b81c15688182baa1fe23e9be429bb981d0da129e 100644 (file)
@@ -71,7 +71,7 @@ typedef struct XLogContRecord
 /*
  * Each page of XLOG file has a header like this:
  */
-#define XLOG_PAGE_MAGIC 0xD069 /* can be used as WAL version indicator */
+#define XLOG_PAGE_MAGIC 0xD070 /* can be used as WAL version indicator */
 
 typedef struct XLogPageHeaderData
 {
index d0d2e9e39d5b1934306ce8e6dea4fe9b6d115a70..1031e5651274268b8433f0632022f8aa49ef9829 100644 (file)
@@ -21,7 +21,7 @@
 
 
 /* Version identifier for this pg_control format */
-#define PG_CONTROL_VERSION 921
+#define PG_CONTROL_VERSION 922
 
 /*
  * Body of CheckPoint XLOG records.  This is declared here because we keep
@@ -33,6 +33,7 @@ typedef struct CheckPoint
    XLogRecPtr  redo;           /* next RecPtr available when we began to
                                 * create CheckPoint (i.e. REDO start point) */
    TimeLineID  ThisTimeLineID; /* current TLI */
+   bool            fullPageWrites; /* current full_page_writes */
    uint32      nextXidEpoch;   /* higher-order bits of nextXid */
    TransactionId nextXid;      /* next free XID */
    Oid         nextOid;        /* next free OID */
@@ -60,6 +61,7 @@ typedef struct CheckPoint
 #define XLOG_BACKUP_END                    0x50
 #define XLOG_PARAMETER_CHANGE          0x60
 #define XLOG_RESTORE_POINT             0x70
+#define XLOG_FPW_CHANGE                0x80
 
 
 /*
@@ -138,6 +140,12 @@ typedef struct ControlFileData
     * record, to make sure the end-of-backup record corresponds the base
     * backup we're recovering from.
     *
+    * backupEndPoint is the backup end location, if we are recovering from
+    * an online backup which was taken from the standby and haven't reached
+    * the end of backup yet. It is initialized to the minimum recovery point
+    * in pg_control which was backed up last. It is reset to zero when
+    * the end of backup is reached, and we mustn't start up before that.
+    *
     * If backupEndRequired is true, we know for sure that we're restoring
     * from a backup, and must see a backup-end record before we can safely
     * start up. If it's false, but backupStartPoint is set, a backup_label
@@ -146,6 +154,7 @@ typedef struct ControlFileData
     */
    XLogRecPtr  minRecoveryPoint;
    XLogRecPtr  backupStartPoint;
+   XLogRecPtr  backupEndPoint;
    bool        backupEndRequired;
 
    /*