On Mon, 2008-09-08 at 13:34 -0400, Tom Lane wrote:
> ISTM that it would probably be better if there were exactly one InRedo
> flag in shared memory, probably in xlog.c's shared state, with the
> postmaster not being responsible for setting or clearing it; rather
> the startup process should do those things.
Done
> > * bgwriter and stats process starts in consistent recovery mode.
> > bgwriter changes mode when startup process completes.
>
> I'm not sure about the interaction of this. In particular, what about
> recovery restart points before we have reached the safe stop point?
> I don't think we want to give up the capability of having those.
>
> Also, it seems pretty bogus to update the in-memory ControlFile
> checkpoint values before the restart point is actually done. It looks
> to me like what you have done is to try to use those fields as signaling
> for the restart request in addition to their existing purposes, which
> I think is confusing and probably dangerous. I'd rather there were a
> different signaling path and ControlFile maintains its currrent
> definition.
Done
Testing takes a while on this, I probably won't complete it until
Friday. So enclosed patch is for eyeballs only at this stage.
I added in the XLogCtl padding we've discussed before, while I'm there.
--
Simon Riggs www.2ndQuadrant.com
PostgreSQL Training, Services and Support
Index: src/backend/access/transam/xlog.c
===================================================================
RCS file: /home/sriggs/pg/REPOSITORY/pgsql/src/backend/access/transam/xlog.c,v
retrieving revision 1.317
diff -c -r1.317 xlog.c
*** src/backend/access/transam/xlog.c 11 Aug 2008 11:05:10 -0000 1.317
--- src/backend/access/transam/xlog.c 10 Sep 2008 20:48:52 -0000
***************
*** 119,124 ****
--- 119,125 ----
/* Are we doing recovery from XLOG? */
bool InRecovery = false;
+ bool reachedSafeStopPoint = false;
/* Are we recovering using offline XLOG archives? */
static bool InArchiveRecovery = false;
***************
*** 131,137 ****
static bool recoveryTarget = false;
static bool recoveryTargetExact = false;
static bool recoveryTargetInclusive = true;
- static bool recoveryLogRestartpoints = false;
static TransactionId recoveryTargetXid;
static TimestampTz recoveryTargetTime;
static TimestampTz recoveryLastXTime = 0;
--- 132,137 ----
***************
*** 286,295 ****
--- 286,297 ----
/*
* Total shared-memory state for XLOG.
*/
+ #define XLOGCTL_BUFFER_SPACING 128
typedef struct XLogCtlData
{
/* Protected by WALInsertLock: */
XLogCtlInsert Insert;
+ char InsertPadding[XLOGCTL_BUFFER_SPACING - sizeof(XLogCtlInsert)];
/* Protected by info_lck: */
XLogwrtRqst LogwrtRqst;
***************
*** 297,305 ****
--- 299,314 ----
uint32 ckptXidEpoch; /* nextXID & epoch of latest checkpoint */
TransactionId ckptXid;
XLogRecPtr asyncCommitLSN; /* LSN of newest async commit */
+ /* add data structure padding for above info_lck declarations */
+ char InfoPadding[XLOGCTL_BUFFER_SPACING - sizeof(XLogwrtRqst)
+ - sizeof(XLogwrtResult)
+ - sizeof(uint32)
+ - sizeof(TransactionId)
+ - sizeof(XLogRecPtr)];
/* Protected by WALWriteLock: */
XLogCtlWrite Write;
+ char WritePadding[XLOGCTL_BUFFER_SPACING - sizeof(XLogCtlWrite)];
/*
* These values do not change after startup, although the pointed-to pages
***************
*** 311,316 ****
--- 320,336 ----
int XLogCacheBlck; /* highest allocated xlog buffer index */
TimeLineID ThisTimeLineID;
+ /*
+ * InStartupXLog changes at the end of recovery only and is only ever set
+ * by the Startup process. We assume that changes to it are atomic,
+ * so accesses to it is never locked. When it does change bgwriter
+ * must immediately begin using it, since this helps it decide whether
+ * to flush WAL or not when it writes dirty blocks. If bgwriter does
+ * it too soon, we will write invalid WAL records and if it reflects the
+ * change too late it could skip flushing WAL for a data block change.
+ */
+ bool InStartupXLog;
+
slock_t info_lck; /* locks shared variables shown above */
} XLogCtlData;
***************
*** 480,485 ****
--- 500,509 ----
bool doPageWrites;
bool isLogSwitch = (rmid == RM_XLOG_ID && info == XLOG_SWITCH);
+ /* cross-check on whether we should be here or not */
+ if (InStartupXLog())
+ elog(FATAL, "cannot write new WAL data during recovery mode");
+
/* info's high bits are reserved for use by me */
if (info & XLR_INFO_MASK)
elog(PANIC, "invalid xlog info mask %02X", info);
***************
*** 1677,1684 ****
XLogRecPtr WriteRqstPtr;
XLogwrtRqst WriteRqst;
! /* Disabled during REDO */
! if (InRedo)
return;
/* Quick exit if already known flushed */
--- 1701,1708 ----
XLogRecPtr WriteRqstPtr;
XLogwrtRqst WriteRqst;
! /* Disabled during StartupXLog */
! if (InStartupXLog())
return;
/* Quick exit if already known flushed */
***************
*** 1766,1774 ****
* the bad page is encountered again during recovery then we would be
* unable to restart the database at all! (This scenario has actually
* happened in the field several times with 7.1 releases. Note that we
! * cannot get here while InRedo is true, but if the bad page is brought in
! * and marked dirty during recovery then CreateCheckPoint will try to
! * flush it at the end of recovery.)
*
* The current approach is to ERROR under normal conditions, but only
* WARNING during recovery, so that the system can be brought up even if
--- 1790,1798 ----
* the bad page is encountered again during recovery then we would be
* unable to restart the database at all! (This scenario has actually
* happened in the field several times with 7.1 releases. Note that we
! * cannot get here while InStartupXLog() is true, but if the bad page is
! * brought in and marked dirty during recovery then CreateCheckPoint will
! * try to flush it at the end of recovery.)
*
* The current approach is to ERROR under normal conditions, but only
* WARNING during recovery, so that the system can be brought up even if
***************
*** 2051,2057 ****
unlink(tmppath);
}
! elog(DEBUG2, "done creating and filling new WAL file");
/* Set flag to tell caller there was no existent file */
*use_existent = false;
--- 2075,2082 ----
unlink(tmppath);
}
! XLogFileName(tmppath, ThisTimeLineID, log, seg);
! elog(DEBUG2, "done creating and filling new WAL file %s", tmppath);
/* Set flag to tell caller there was no existent file */
*use_existent = false;
***************
*** 4532,4546 ****
}
else if (strcmp(tok1, "log_restartpoints") == 0)
{
- /*
- * does nothing if a recovery_target is not also set
- */
- if (!parse_bool(tok2, &recoveryLogRestartpoints))
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
- errmsg("parameter \"log_restartpoints\" requires a Boolean value")));
ereport(LOG,
! (errmsg("log_restartpoints = %s", tok2)));
}
else
ereport(FATAL,
--- 4557,4565 ----
}
else if (strcmp(tok1, "log_restartpoints") == 0)
{
ereport(LOG,
! (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
! errmsg("parameter \"log_restartpoints\" has been deprecated")));
}
else
ereport(FATAL,
***************
*** 4823,4828 ****
--- 4842,4849 ----
uint32 freespace;
TransactionId oldestActiveXID;
+ XLogCtl->InStartupXLog = true;
+
/*
* Read control file and check XLOG status looks valid.
*
***************
*** 5039,5044 ****
--- 5060,5070 ----
UpdateControlFile();
/*
+ * Reset pgstat data, because it may be invalid after recovery.
+ */
+ pgstat_reset_all();
+
+ /*
* If there was a backup label file, it's done its job and the info
* has now been propagated into pg_control. We must get rid of the
* label file so that if we crash during recovery, we'll pick up at
***************
*** 5148,5153 ****
--- 5174,5194 ----
LastRec = ReadRecPtr;
+ /*
+ * Have we reached our safe stopping point? If so, we can
+ * signal Postmaster to enter consistent recovery mode
+ */
+ if (!reachedSafeStopPoint &&
+ XLByteLE(ControlFile->minRecoveryPoint, EndRecPtr))
+ {
+ reachedSafeStopPoint = true;
+ ereport(LOG,
+ (errmsg("consistent recovery state reached at %X/%X",
+ EndRecPtr.xlogid, EndRecPtr.xrecoff)));
+ if (IsUnderPostmaster)
+ SendPostmasterSignal(PMSIGNAL_RECOVERY_START);
+ }
+
record = ReadRecord(NULL, LOG);
} while (record != NULL && recoveryContinue);
***************
*** 5169,5174 ****
--- 5210,5216 ----
/* there are no WAL records following the checkpoint */
ereport(LOG,
(errmsg("redo is not required")));
+ reachedSafeStopPoint = true;
}
}
***************
*** 5184,5190 ****
* Complain if we did not roll forward far enough to render the backup
* dump consistent.
*/
! if (XLByteLT(EndOfLog, ControlFile->minRecoveryPoint))
{
if (reachedStopPoint) /* stopped because of stop request */
ereport(FATAL,
--- 5226,5232 ----
* Complain if we did not roll forward far enough to render the backup
* dump consistent.
*/
! if (InRecovery && !reachedSafeStopPoint)
{
if (reachedStopPoint) /* stopped because of stop request */
ereport(FATAL,
***************
*** 5305,5314 ****
*/
XLogCheckInvalidPages();
! /*
! * Reset pgstat data, because it may be invalid after recovery.
! */
! pgstat_reset_all();
/*
* Perform a checkpoint to update all our recovery activity to disk.
--- 5347,5354 ----
*/
XLogCheckInvalidPages();
! if (IsUnderPostmaster)
! BgWriterCompleteRestartPointImmediately();
/*
* Perform a checkpoint to update all our recovery activity to disk.
***************
*** 5318,5323 ****
--- 5358,5366 ----
* assigning a new TLI, using a shutdown checkpoint allows us to have
* the rule that TLI only changes in shutdown checkpoints, which
* allows some extra error checking in xlog_redo.
+ *
+ * Note that this will wait behind any restartpoint that the bgwriter
+ * is currently performing, though will be much faster as a result.
*/
CreateCheckPoint(CHECKPOINT_IS_SHUTDOWN | CHECKPOINT_IMMEDIATE);
}
***************
*** 5372,5377 ****
--- 5415,5428 ----
readRecordBuf = NULL;
readRecordBufSize = 0;
}
+
+ XLogCtl->InStartupXLog = false;
+ }
+
+ bool
+ InStartupXLog(void)
+ {
+ return XLogCtl->InStartupXLog;
}
/*
***************
*** 5642,5648 ****
* Log end of a checkpoint.
*/
static void
! LogCheckpointEnd(void)
{
long write_secs,
sync_secs,
--- 5693,5699 ----
* Log end of a checkpoint.
*/
static void
! LogCheckpointEnd(bool checkpoint)
{
long write_secs,
sync_secs,
***************
*** 5665,5673 ****
CheckpointStats.ckpt_sync_end_t,
&sync_secs, &sync_usecs);
! elog(LOG, "checkpoint complete: wrote %d buffers (%.1f%%); "
"%d transaction log file(s) added, %d removed, %d recycled; "
"write=%ld.%03d s, sync=%ld.%03d s, total=%ld.%03d s",
CheckpointStats.ckpt_bufs_written,
(double) CheckpointStats.ckpt_bufs_written * 100 / NBuffers,
CheckpointStats.ckpt_segs_added,
--- 5716,5725 ----
CheckpointStats.ckpt_sync_end_t,
&sync_secs, &sync_usecs);
! elog(LOG, "%s complete: wrote %d buffers (%.1f%%); "
"%d transaction log file(s) added, %d removed, %d recycled; "
"write=%ld.%03d s, sync=%ld.%03d s, total=%ld.%03d s",
+ (checkpoint ? " checkpoint" : "restartpoint"),
CheckpointStats.ckpt_bufs_written,
(double) CheckpointStats.ckpt_bufs_written * 100 / NBuffers,
CheckpointStats.ckpt_segs_added,
***************
*** 6002,6008 ****
/* All real work is done, but log before releasing lock. */
if (log_checkpoints)
! LogCheckpointEnd();
LWLockRelease(CheckpointLock);
}
--- 6054,6060 ----
/* All real work is done, but log before releasing lock. */
if (log_checkpoints)
! LogCheckpointEnd(true);
LWLockRelease(CheckpointLock);
}
***************
*** 6071,6099 ****
}
}
/*
! * OK, force data out to disk
*/
! CheckPointGuts(checkPoint->redo, CHECKPOINT_IMMEDIATE);
/*
! * Update pg_control so that any subsequent crash will restart from this
! * checkpoint. Note: ReadRecPtr gives the XLOG address of the checkpoint
! * record itself.
*/
ControlFile->prevCheckPoint = ControlFile->checkPoint;
! ControlFile->checkPoint = ReadRecPtr;
! ControlFile->checkPointCopy = *checkPoint;
ControlFile->time = (pg_time_t) time(NULL);
UpdateControlFile();
! ereport((recoveryLogRestartpoints ? LOG : DEBUG2),
(errmsg("recovery restart point at %X/%X",
! checkPoint->redo.xlogid, checkPoint->redo.xrecoff)));
! if (recoveryLastXTime)
! ereport((recoveryLogRestartpoints ? LOG : DEBUG2),
! (errmsg("last completed transaction was at log time %s",
! timestamptz_to_str(recoveryLastXTime))));
}
/*
--- 6123,6184 ----
}
}
+ if (recoveryLastXTime)
+ ereport((log_checkpoints ? LOG : DEBUG2),
+ (errmsg("last completed transaction was at log time %s",
+ timestamptz_to_str(recoveryLastXTime))));
+
+ RequestRestartPoint(ReadRecPtr, checkPoint, reachedSafeStopPoint);
+ }
+
+ /*
+ * As of 8.4, RestartPoints are always created by the bgwriter
+ * once we have reachedSafeStopPoint. We use bgwriter's shared memory
+ * area wherever we call it from, to keep better code structure.
+ */
+ void
+ CreateRestartPoint(const XLogRecPtr ReadPtr, const CheckPoint *restartPoint)
+ {
+ if (log_checkpoints)
+ {
+ /*
+ * Prepare to accumulate statistics.
+ */
+
+ MemSet(&CheckpointStats, 0, sizeof(CheckpointStats));
+ CheckpointStats.ckpt_start_t = GetCurrentTimestamp();
+
+ elog(LOG, "restartpoint starting:");
+ }
+
+ LWLockAcquire(CheckpointLock, LW_EXCLUSIVE);
+
/*
! * OK, write out dirty blocks smoothly
*/
! CheckPointGuts(restartPoint->redo, 0);
/*
! * Update pg_control, using current time
*/
+ LWLockAcquire(ControlFileLock, LW_EXCLUSIVE);
ControlFile->prevCheckPoint = ControlFile->checkPoint;
! ControlFile->checkPoint = ReadPtr;
! ControlFile->checkPointCopy = *restartPoint;
ControlFile->time = (pg_time_t) time(NULL);
UpdateControlFile();
+ LWLockRelease(ControlFileLock);
! /* All real work is done, but log before releasing lock. */
! if (log_checkpoints)
! LogCheckpointEnd(true);
!
! ereport((log_checkpoints ? LOG : DEBUG2),
(errmsg("recovery restart point at %X/%X",
! restartPoint->redo.xlogid, restartPoint->redo.xrecoff)));
!
! LWLockRelease(CheckpointLock);
!
}
/*
Index: src/backend/postmaster/bgwriter.c
===================================================================
RCS file: /home/sriggs/pg/REPOSITORY/pgsql/src/backend/postmaster/bgwriter.c,v
retrieving revision 1.51
diff -c -r1.51 bgwriter.c
*** src/backend/postmaster/bgwriter.c 11 Aug 2008 11:05:11 -0000 1.51
--- src/backend/postmaster/bgwriter.c 10 Sep 2008 20:53:07 -0000
***************
*** 49,54 ****
--- 49,55 ----
#include <unistd.h>
#include "access/xlog_internal.h"
+ #include "catalog/pg_control.h"
#include "libpq/pqsignal.h"
#include "miscadmin.h"
#include "pgstat.h"
***************
*** 130,135 ****
--- 131,144 ----
int ckpt_flags; /* checkpoint flags, as defined in xlog.h */
+ /*
+ * When the Startup process wants a restartpoint, it sets these fields
+ * so that whoever performs the restartpoint can update the control file,
+ * allowing the caller to continue, if it is running in another process.
+ */
+ XLogRecPtr ReadPtr; /* ReadRecPtr for RestartPoint request */
+ CheckPoint *restartPoint; /* restartPoint data for ControlFile */
+
uint32 num_backend_writes; /* counts non-bgwriter buffer writes */
int num_requests; /* current # of requests */
***************
*** 164,172 ****
static bool ckpt_active = false;
/* these values are valid when ckpt_active is true: */
static pg_time_t ckpt_start_time;
! static XLogRecPtr ckpt_start_recptr;
static double ckpt_cached_elapsed;
static pg_time_t last_checkpoint_time;
--- 173,184 ----
static bool ckpt_active = false;
+ static bool BgWriterInStartup;
+
+
/* these values are valid when ckpt_active is true: */
static pg_time_t ckpt_start_time;
! static XLogRecPtr ckpt_start_recptr; /* not used if InStartupXLog */
static double ckpt_cached_elapsed;
static pg_time_t last_checkpoint_time;
***************
*** 186,191 ****
--- 198,212 ----
static void ReqCheckpointHandler(SIGNAL_ARGS);
static void ReqShutdownHandler(SIGNAL_ARGS);
+ /* ask bgwriter to complete any restartpoint, if any, with zero delay */
+ void
+ BgWriterCompleteRestartPointImmediately(void)
+ {
+ SpinLockAcquire(&BgWriterShmem->ckpt_lck);
+ BgWriterShmem->ckpt_flags = CHECKPOINT_IMMEDIATE;
+ SpinLockRelease(&BgWriterShmem->ckpt_lck);
+ elog(DEBUG2, "asking bgwriter to complete any restartpoint with zero delay");
+ }
/*
* Main entry point for bgwriter process
***************
*** 202,207 ****
--- 223,230 ----
BgWriterShmem->bgwriter_pid = MyProcPid;
am_bg_writer = true;
+ BgWriterInStartup = InStartupXLog();
+
/*
* If possible, make this process a group leader, so that the postmaster
* can signal any child processes too. (bgwriter probably never has any
***************
*** 356,371 ****
*/
PG_SETMASK(&UnBlockSig);
/*
* Loop forever
*/
for (;;)
{
- bool do_checkpoint = false;
- int flags = 0;
- pg_time_t now;
- int elapsed_secs;
-
/*
* Emergency bailout if postmaster has died. This is to avoid the
* necessity for manual cleanup of all postmaster children.
--- 379,393 ----
*/
PG_SETMASK(&UnBlockSig);
+ if (BgWriterInStartup)
+ elog(DEBUG1, "bgwriter starting in recovery mode, pid = %u",
+ BgWriterShmem->bgwriter_pid);
+
/*
* Loop forever
*/
for (;;)
{
/*
* Emergency bailout if postmaster has died. This is to avoid the
* necessity for manual cleanup of all postmaster children.
***************
*** 383,501 ****
got_SIGHUP = false;
ProcessConfigFile(PGC_SIGHUP);
}
- if (checkpoint_requested)
- {
- checkpoint_requested = false;
- do_checkpoint = true;
- BgWriterStats.m_requested_checkpoints++;
- }
- if (shutdown_requested)
- {
- /*
- * From here on, elog(ERROR) should end with exit(1), not send
- * control back to the sigsetjmp block above
- */
- ExitOnAnyError = true;
- /* Close down the database */
- ShutdownXLOG(0, 0);
- DumpFreeSpaceMap(0, 0);
- /* Normal exit from the bgwriter is here */
- proc_exit(0); /* done */
- }
-
- /*
- * Force a checkpoint if too much time has elapsed since the last one.
- * Note that we count a timed checkpoint in stats only when this
- * occurs without an external request, but we set the CAUSE_TIME flag
- * bit even if there is also an external request.
- */
- now = (pg_time_t) time(NULL);
- elapsed_secs = now - last_checkpoint_time;
- if (elapsed_secs >= CheckPointTimeout)
- {
- if (!do_checkpoint)
- BgWriterStats.m_timed_checkpoints++;
- do_checkpoint = true;
- flags |= CHECKPOINT_CAUSE_TIME;
- }
! /*
! * Do a checkpoint if requested, otherwise do one cycle of
! * dirty-buffer writing.
! */
! if (do_checkpoint)
{
! /* use volatile pointer to prevent code rearrangement */
! volatile BgWriterShmemStruct *bgs = BgWriterShmem;
!
! /*
! * Atomically fetch the request flags to figure out what kind of a
! * checkpoint we should perform, and increase the started-counter
! * to acknowledge that we've started a new checkpoint.
! */
! SpinLockAcquire(&bgs->ckpt_lck);
! flags |= bgs->ckpt_flags;
! bgs->ckpt_flags = 0;
! bgs->ckpt_started++;
! SpinLockRelease(&bgs->ckpt_lck);
/*
! * We will warn if (a) too soon since last checkpoint (whatever
! * caused it) and (b) somebody set the CHECKPOINT_CAUSE_XLOG flag
! * since the last checkpoint start. Note in particular that this
! * implementation will not generate warnings caused by
! * CheckPointTimeout < CheckPointWarning.
*/
! if ((flags & CHECKPOINT_CAUSE_XLOG) &&
! elapsed_secs < CheckPointWarning)
! ereport(LOG,
! (errmsg("checkpoints are occurring too frequently (%d seconds apart)",
! elapsed_secs),
! errhint("Consider increasing the configuration parameter \"checkpoint_segments\".")));
!
! /*
! * Initialize bgwriter-private variables used during checkpoint.
! */
! ckpt_active = true;
! ckpt_start_recptr = GetInsertRecPtr();
! ckpt_start_time = now;
! ckpt_cached_elapsed = 0;
/*
! * Do the checkpoint.
*/
! CreateCheckPoint(flags);
/*
! * After any checkpoint, close all smgr files. This is so we
! * won't hang onto smgr references to deleted files indefinitely.
*/
! smgrcloseall();
! /*
! * Indicate checkpoint completion to any waiting backends.
! */
! SpinLockAcquire(&bgs->ckpt_lck);
! bgs->ckpt_done = bgs->ckpt_started;
! SpinLockRelease(&bgs->ckpt_lck);
! ckpt_active = false;
!
! /*
! * Note we record the checkpoint start time not end time as
! * last_checkpoint_time. This is so that time-driven checkpoints
! * happen at a predictable spacing.
! */
! last_checkpoint_time = now;
}
- else
- BgBufferSync();
-
- /* Check for archive_timeout and switch xlog files if necessary. */
- CheckArchiveTimeout();
-
- /* Nap for the configured time. */
- BgWriterNap();
}
}
--- 405,607 ----
got_SIGHUP = false;
ProcessConfigFile(PGC_SIGHUP);
}
! if (BgWriterInStartup)
{
! if (checkpoint_requested)
! {
! XLogRecPtr ReadPtr;
! CheckPoint restartPoint;
!
! /*
! * Initialize bgwriter-private variables used during checkpoint.
! */
! ckpt_active = true;
! ckpt_start_time = (pg_time_t) time(NULL);
! ckpt_cached_elapsed = 0;
!
! /*
! * Get the requested values from shared memory that the
! * Startup process has put there for us
! */
! SpinLockAcquire(&BgWriterShmem->ckpt_lck);
! ReadPtr = BgWriterShmem->ReadPtr;
! memcpy(&restartPoint, &BgWriterShmem->restartPoint, sizeof(CheckPoint));
! SpinLockRelease(&BgWriterShmem->ckpt_lck);
!
! CreateRestartPoint(ReadPtr, &restartPoint);
!
! /*
! * Reset any flags if we requested immediate completion part
! * way through the restart point
! */
! SpinLockAcquire(&BgWriterShmem->ckpt_lck);
! BgWriterShmem->ckpt_flags = 0;
! SpinLockRelease(&BgWriterShmem->ckpt_lck);
!
! ckpt_active = false;
! checkpoint_requested = false;
! }
! else
! {
! /* Clean buffers dirtied by recovery */
! BgBufferSync(true);
!
! /* Nap for the configured time. */
! BgWriterNap();
! }
!
! if (shutdown_requested)
! {
! /*
! * From here on, elog(ERROR) should end with exit(1), not send
! * control back to the sigsetjmp block above
! */
! ExitOnAnyError = true;
! /* Normal exit from the bgwriter is here */
! proc_exit(0); /* done */
! }
/*
! * Check to see whether startup process has completed redo.
! * If so, we can permanently change out of recovery mode.
*/
! if (InStartupXLog())
! {
! elog(DEBUG2, "changing from recovery to normal mode");
!
! InitXLOGAccess();
! BgWriterInStartup = false;
!
! /*
! * Start time-driven events from now
! */
! last_checkpoint_time = last_xlog_switch_time = (pg_time_t) time(NULL);
! }
! }
! else /* Normal processing */
! {
! bool do_checkpoint = false;
! int flags = 0;
! pg_time_t now;
! int elapsed_secs;
!
! Assert(!InStartupXLog());
!
! if (checkpoint_requested)
! {
! checkpoint_requested = false;
! do_checkpoint = true;
! BgWriterStats.m_requested_checkpoints++;
! }
! if (shutdown_requested)
! {
! /*
! * From here on, elog(ERROR) should end with exit(1), not send
! * control back to the sigsetjmp block above
! */
! ExitOnAnyError = true;
! /* Close down the database */
! ShutdownXLOG(0, 0);
! DumpFreeSpaceMap(0, 0);
! /* Normal exit from the bgwriter is here */
! proc_exit(0); /* done */
! }
/*
! * Force a checkpoint if too much time has elapsed since the last one.
! * Note that we count a timed checkpoint in stats only when this
! * occurs without an external request, but we set the CAUSE_TIME flag
! * bit even if there is also an external request.
*/
! now = (pg_time_t) time(NULL);
! elapsed_secs = now - last_checkpoint_time;
! if (elapsed_secs >= CheckPointTimeout)
! {
! if (!do_checkpoint)
! BgWriterStats.m_timed_checkpoints++;
! do_checkpoint = true;
! flags |= CHECKPOINT_CAUSE_TIME;
! }
/*
! * Do a checkpoint if requested, otherwise do one cycle of
! * dirty-buffer writing.
*/
! if (do_checkpoint)
! {
! /* use volatile pointer to prevent code rearrangement */
! volatile BgWriterShmemStruct *bgs = BgWriterShmem;
!
! /*
! * Atomically fetch the request flags to figure out what kind of a
! * checkpoint we should perform, and increase the started-counter
! * to acknowledge that we've started a new checkpoint.
! */
! SpinLockAcquire(&bgs->ckpt_lck);
! flags |= bgs->ckpt_flags;
! bgs->ckpt_flags = 0;
! bgs->ckpt_started++;
! SpinLockRelease(&bgs->ckpt_lck);
!
! /*
! * We will warn if (a) too soon since last checkpoint (whatever
! * caused it) and (b) somebody set the CHECKPOINT_CAUSE_XLOG flag
! * since the last checkpoint start. Note in particular that this
! * implementation will not generate warnings caused by
! * CheckPointTimeout < CheckPointWarning.
! */
! if ((flags & CHECKPOINT_CAUSE_XLOG) &&
! elapsed_secs < CheckPointWarning)
! ereport(LOG,
! (errmsg("checkpoints are occurring too frequently (%d seconds apart)",
! elapsed_secs),
! errhint("Consider increasing the configuration parameter \"checkpoint_segments\".")));
!
! /*
! * Initialize bgwriter-private variables used during checkpoint.
! */
! ckpt_active = true;
! ckpt_start_recptr = GetInsertRecPtr();
! ckpt_start_time = now;
! ckpt_cached_elapsed = 0;
!
! /*
! * Do the checkpoint.
! */
! CreateCheckPoint(flags);
!
! /*
! * After any checkpoint, close all smgr files. This is so we
! * won't hang onto smgr references to deleted files indefinitely.
! */
! smgrcloseall();
!
! /*
! * Indicate checkpoint completion to any waiting backends.
! */
! SpinLockAcquire(&bgs->ckpt_lck);
! bgs->ckpt_done = bgs->ckpt_started;
! SpinLockRelease(&bgs->ckpt_lck);
!
! ckpt_active = false;
!
! /*
! * Note we record the checkpoint start time not end time as
! * last_checkpoint_time. This is so that time-driven checkpoints
! * happen at a predictable spacing.
! */
! last_checkpoint_time = now;
! }
! else
! BgBufferSync(false);
! /* Check for archive_timeout and switch xlog files if necessary. */
! CheckArchiveTimeout();
! /* Nap for the configured time. */
! BgWriterNap();
}
}
}
***************
*** 588,594 ****
(ckpt_active ? ImmediateCheckpointRequested() : checkpoint_requested))
break;
pg_usleep(1000000L);
! AbsorbFsyncRequests();
udelay -= 1000000L;
}
--- 694,701 ----
(ckpt_active ? ImmediateCheckpointRequested() : checkpoint_requested))
break;
pg_usleep(1000000L);
! if (!InStartupXLog())
! AbsorbFsyncRequests();
udelay -= 1000000L;
}
***************
*** 642,647 ****
--- 749,767 ----
if (!am_bg_writer)
return;
+ /* Perform minimal duties during recovery and skip wait if requested */
+ if (InStartupXLog())
+ {
+ BgBufferSync(true);
+
+ if (!ImmediateCheckpointRequested() &&
+ !shutdown_requested &&
+ IsCheckpointOnSchedule(progress))
+ BgWriterNap();
+
+ return;
+ }
+
/*
* Perform the usual bgwriter duties and take a nap, unless we're behind
* schedule, in which case we just try to catch up as quickly as possible.
***************
*** 660,666 ****
AbsorbFsyncRequests();
absorb_counter = WRITES_PER_ABSORB;
! BgBufferSync();
CheckArchiveTimeout();
BgWriterNap();
}
--- 780,786 ----
AbsorbFsyncRequests();
absorb_counter = WRITES_PER_ABSORB;
! BgBufferSync(false);
CheckArchiveTimeout();
BgWriterNap();
}
***************
*** 716,731 ****
* However, it's good enough for our purposes, we're only calculating an
* estimate anyway.
*/
! recptr = GetInsertRecPtr();
! elapsed_xlogs =
! (((double) (int32) (recptr.xlogid - ckpt_start_recptr.xlogid)) * XLogSegsPerFile +
! ((double) recptr.xrecoff - (double) ckpt_start_recptr.xrecoff) / XLogSegSize) /
! CheckPointSegments;
!
! if (progress < elapsed_xlogs)
{
! ckpt_cached_elapsed = elapsed_xlogs;
! return false;
}
/*
--- 836,854 ----
* However, it's good enough for our purposes, we're only calculating an
* estimate anyway.
*/
! if (!InStartupXLog())
{
! recptr = GetInsertRecPtr();
! elapsed_xlogs =
! (((double) (int32) (recptr.xlogid - ckpt_start_recptr.xlogid)) * XLogSegsPerFile +
! ((double) recptr.xrecoff - (double) ckpt_start_recptr.xrecoff) / XLogSegSize) /
! CheckPointSegments;
!
! if (progress < elapsed_xlogs)
! {
! ckpt_cached_elapsed = elapsed_xlogs;
! return false;
! }
}
/*
***************
*** 967,972 ****
--- 1090,1126 ----
}
/*
+ * Always runs in Startup process (see xlog.c)
+ */
+ void
+ RequestRestartPoint(const XLogRecPtr ReadPtr, const CheckPoint *restartPoint, bool sendToBGWriter)
+ {
+ /*
+ * Should we just do it ourselves?
+ */
+ if (!IsPostmasterEnvironment || !sendToBGWriter)
+ {
+ CreateRestartPoint(ReadPtr, restartPoint);
+ return;
+ }
+
+ /*
+ * Push requested values into shared memory, then signal to request restartpoint.
+ */
+ if (BgWriterShmem->bgwriter_pid == 0)
+ elog(LOG, "could not request restartpoint because bgwriter not running");
+
+ SpinLockAcquire(&BgWriterShmem->ckpt_lck);
+ BgWriterShmem->ReadPtr = ReadPtr;
+ memcpy(&BgWriterShmem->restartPoint, restartPoint, sizeof(CheckPoint));
+ SpinLockRelease(&BgWriterShmem->ckpt_lck);
+
+
+ if (kill(BgWriterShmem->bgwriter_pid, SIGINT) != 0)
+ elog(LOG, "could not signal for restartpoint: %m");
+ }
+
+ /*
* ForwardFsyncRequest
* Forward a file-fsync request from a backend to the bgwriter
*
Index: src/backend/postmaster/postmaster.c
===================================================================
RCS file: /home/sriggs/pg/REPOSITORY/pgsql/src/backend/postmaster/postmaster.c,v
retrieving revision 1.561
diff -c -r1.561 postmaster.c
*** src/backend/postmaster/postmaster.c 26 Jun 2008 02:47:19 -0000 1.561
--- src/backend/postmaster/postmaster.c 10 Sep 2008 20:46:59 -0000
***************
*** 254,259 ****
--- 254,264 ----
{
PM_INIT, /* postmaster starting */
PM_STARTUP, /* waiting for startup subprocess */
+ PM_RECOVERY, /* consistent recovery mode; state only
+ * entered for archive and streaming recovery,
+ * and only after the point where the
+ * all data is in consistent state.
+ */
PM_RUN, /* normal "database is alive" state */
PM_WAIT_BACKUP, /* waiting for online backup mode to end */
PM_WAIT_BACKENDS, /* waiting for live backends to exit */
***************
*** 2104,2110 ****
if (pid == StartupPID)
{
StartupPID = 0;
! Assert(pmState == PM_STARTUP);
/* FATAL exit of startup is treated as catastrophic */
if (!EXIT_STATUS_0(exitstatus))
--- 2109,2115 ----
if (pid == StartupPID)
{
StartupPID = 0;
! Assert(pmState == PM_STARTUP || pmState == PM_RECOVERY);
/* FATAL exit of startup is treated as catastrophic */
if (!EXIT_STATUS_0(exitstatus))
***************
*** 2148,2155 ****
* Crank up the background writer. It doesn't matter if this
* fails, we'll just try again later.
*/
! Assert(BgWriterPID == 0);
! BgWriterPID = StartBackgroundWriter();
/*
* Likewise, start other special children as needed. In a restart
--- 2153,2160 ----
* Crank up the background writer. It doesn't matter if this
* fails, we'll just try again later.
*/
! if (BgWriterPID == 0)
! BgWriterPID = StartBackgroundWriter();
/*
* Likewise, start other special children as needed. In a restart
***************
*** 3821,3826 ****
--- 3826,3876 ----
PG_SETMASK(&BlockSig);
+ if (CheckPostmasterSignal(PMSIGNAL_RECOVERY_START))
+ {
+ Assert(pmState == PM_STARTUP);
+
+ /*
+ * Go to shutdown mode if a shutdown request was pending.
+ */
+ if (Shutdown > NoShutdown)
+ {
+ pmState = PM_WAIT_BACKENDS;
+ /* PostmasterStateMachine logic does the rest */
+ }
+ else
+ {
+ /*
+ * Startup process has entered recovery
+ */
+ pmState = PM_RECOVERY;
+
+ /*
+ * Load the flat authorization file into postmaster's cache. The
+ * startup process won't have recomputed this from the database yet,
+ * so we it may change following recovery.
+ */
+ load_role();
+
+ /*
+ * Crank up the background writer. It doesn't matter if this
+ * fails, we'll just try again later.
+ */
+ Assert(BgWriterPID == 0);
+ BgWriterPID = StartBackgroundWriter();
+
+ /*
+ * Likewise, start other special children as needed.
+ */
+ Assert(PgStatPID == 0);
+ PgStatPID = pgstat_start();
+
+ /* XXX at this point we could accept read-only connections */
+ ereport(DEBUG1,
+ (errmsg("database system is in consistent recovery mode")));
+ }
+ }
+
if (CheckPostmasterSignal(PMSIGNAL_PASSWORD_CHANGE))
{
/*
Index: src/backend/storage/buffer/bufmgr.c
===================================================================
RCS file: /home/sriggs/pg/REPOSITORY/pgsql/src/backend/storage/buffer/bufmgr.c,v
retrieving revision 1.237
diff -c -r1.237 bufmgr.c
*** src/backend/storage/buffer/bufmgr.c 11 Aug 2008 11:05:11 -0000 1.237
--- src/backend/storage/buffer/bufmgr.c 10 Sep 2008 20:42:10 -0000
***************
*** 1211,1217 ****
* This is called periodically by the background writer process.
*/
void
! BgBufferSync(void)
{
/* info obtained from freelist.c */
int strategy_buf_id;
--- 1211,1217 ----
* This is called periodically by the background writer process.
*/
void
! BgBufferSync(bool InStartup)
{
/* info obtained from freelist.c */
int strategy_buf_id;
***************
*** 1423,1428 ****
--- 1423,1434 ----
{
int buffer_state = SyncOneBuffer(next_to_clean, true);
+ /*
+ * Quit scanning if bgwriter state changes while we're here
+ */
+ if (InStartup && !InStartupXLog())
+ break;
+
if (++next_to_clean >= NBuffers)
{
next_to_clean = 0;
Index: src/include/access/xlog.h
===================================================================
RCS file: /home/sriggs/pg/REPOSITORY/pgsql/src/include/access/xlog.h,v
retrieving revision 1.88
diff -c -r1.88 xlog.h
*** src/include/access/xlog.h 12 May 2008 08:35:05 -0000 1.88
--- src/include/access/xlog.h 10 Sep 2008 20:42:55 -0000
***************
*** 197,202 ****
--- 197,203 ----
extern void xlog_redo(XLogRecPtr lsn, XLogRecord *record);
extern void xlog_desc(StringInfo buf, uint8 xl_info, char *rec);
+ extern bool InStartupXLog(void);
extern void UpdateControlFile(void);
extern Size XLOGShmemSize(void);
extern void XLOGShmemInit(void);
Index: src/include/access/xlog_internal.h
===================================================================
RCS file: /home/sriggs/pg/REPOSITORY/pgsql/src/include/access/xlog_internal.h,v
retrieving revision 1.24
diff -c -r1.24 xlog_internal.h
*** src/include/access/xlog_internal.h 11 Aug 2008 11:05:11 -0000 1.24
--- src/include/access/xlog_internal.h 10 Sep 2008 20:42:43 -0000
***************
*** 17,22 ****
--- 17,23 ----
#define XLOG_INTERNAL_H
#include "access/xlog.h"
+ #include "catalog/pg_control.h"
#include "fmgr.h"
#include "pgtime.h"
#include "storage/block.h"
***************
*** 245,250 ****
--- 246,254 ----
extern pg_time_t GetLastSegSwitchTime(void);
extern XLogRecPtr RequestXLogSwitch(void);
+
+ extern void CreateRestartPoint(const XLogRecPtr ReadPtr, const CheckPoint *restartPoint);
+
/*
* These aren't in xlog.h because I'd rather not include fmgr.h there.
*/
Index: src/include/postmaster/bgwriter.h
===================================================================
RCS file: /home/sriggs/pg/REPOSITORY/pgsql/src/include/postmaster/bgwriter.h,v
retrieving revision 1.12
diff -c -r1.12 bgwriter.h
*** src/include/postmaster/bgwriter.h 11 Aug 2008 11:05:11 -0000 1.12
--- src/include/postmaster/bgwriter.h 10 Sep 2008 20:51:50 -0000
***************
*** 12,17 ****
--- 12,18 ----
#ifndef _BGWRITER_H
#define _BGWRITER_H
+ #include "catalog/pg_control.h"
#include "storage/block.h"
#include "storage/relfilenode.h"
***************
*** 25,36 ****
--- 26,40 ----
extern void BackgroundWriterMain(void);
extern void RequestCheckpoint(int flags);
+ extern void RequestRestartPoint(const XLogRecPtr ReadPtr, const CheckPoint *restartPoint, bool sendToBGWriter);
extern void CheckpointWriteDelay(int flags, double progress);
extern bool ForwardFsyncRequest(RelFileNode rnode, ForkNumber forknum,
BlockNumber segno);
extern void AbsorbFsyncRequests(void);
+ extern void BgWriterCompleteRestartPointImmediately(void);
+
extern Size BgWriterShmemSize(void);
extern void BgWriterShmemInit(void);
Index: src/include/storage/bufmgr.h
===================================================================
RCS file: /home/sriggs/pg/REPOSITORY/pgsql/src/include/storage/bufmgr.h,v
retrieving revision 1.115
diff -c -r1.115 bufmgr.h
*** src/include/storage/bufmgr.h 11 Aug 2008 11:05:11 -0000 1.115
--- src/include/storage/bufmgr.h 10 Sep 2008 19:38:38 -0000
***************
*** 193,199 ****
extern void AbortBufferIO(void);
extern void BufmgrCommit(void);
! extern void BgBufferSync(void);
extern void AtProcExit_LocalBuffers(void);
--- 193,199 ----
extern void AbortBufferIO(void);
extern void BufmgrCommit(void);
! extern void BgBufferSync(bool InStartup);
extern void AtProcExit_LocalBuffers(void);
Index: src/include/storage/pmsignal.h
===================================================================
RCS file: /home/sriggs/pg/REPOSITORY/pgsql/src/include/storage/pmsignal.h,v
retrieving revision 1.20
diff -c -r1.20 pmsignal.h
*** src/include/storage/pmsignal.h 19 Jun 2008 21:32:56 -0000 1.20
--- src/include/storage/pmsignal.h 31 Aug 2008 19:54:58 -0000
***************
*** 22,27 ****
--- 22,28 ----
*/
typedef enum
{
+ PMSIGNAL_RECOVERY_START, /* move to PM_RECOVERY state */
PMSIGNAL_PASSWORD_CHANGE, /* pg_auth file has changed */
PMSIGNAL_WAKEN_ARCHIVER, /* send a NOTIFY signal to xlog archiver */
PMSIGNAL_ROTATE_LOGFILE, /* send SIGUSR1 to syslogger to rotate logfile */
--
Sent via pgsql-patches mailing list ([email protected])
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-patches