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 (pgsql-patches@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-patches

Reply via email to