]> granicus.if.org Git - postgresql/commitdiff
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;
 
        /*