*** a/src/backend/access/transam/Makefile
--- b/src/backend/access/transam/Makefile
***************
*** 13,19 **** top_builddir = ../../../..
  include $(top_builddir)/src/Makefile.global
  
  OBJS = clog.o transam.o varsup.o xact.o rmgr.o slru.o subtrans.o multixact.o \
! 	twophase.o twophase_rmgr.o xlog.o xlogfuncs.o xlogutils.o
  
  include $(top_srcdir)/src/backend/common.mk
  
--- 13,19 ----
  include $(top_builddir)/src/Makefile.global
  
  OBJS = clog.o transam.o varsup.o xact.o rmgr.o slru.o subtrans.o multixact.o \
! 	twophase.o twophase_rmgr.o xlog.o xlogfuncs.o xlogutils.o groupcommit.o
  
  include $(top_srcdir)/src/backend/common.mk
  
*** /dev/null
--- b/src/backend/access/transam/groupcommit.c
***************
*** 0 ****
--- 1,354 ----
+ /*-------------------------------------------------------------------------
+  *
+  * groupcommit.c
+  *
+  * This code for Group Commit is new as of PostgreSQL 9.2.
+  *
+  * Transactions requesting XlogFlush() wait until the WALWriter has
+  * performed the write for them, then will be woken to continue.
+  * This module contains the code for waiting and release of backends.
+  *
+  * The best performing way to manage the waiting backends is to have a
+  * single ordered queue of waiting backends, so that we can avoid
+  * searching the through all waiters each time WALWrites writes/fsyncs.
+  *
+  * Portions Copyright (c) 2010-2011, PostgreSQL Global Development Group
+  *
+  * IDENTIFICATION
+  *	  src/backend/access/groupcommit.c
+  *
+  *-------------------------------------------------------------------------
+  */
+ #include "postgres.h"
+ 
+ #include <unistd.h>
+ 
+ #include "access/xact.h"
+ #include "access/xlog.h"
+ #include "miscadmin.h"
+ #include "access/groupcommit.h"
+ #include "storage/pmsignal.h"
+ #include "storage/proc.h"
+ #include "tcop/tcopprot.h"
+ #include "utils/builtins.h"
+ 
+ bool	UseGroupCommit = true;
+ 
+ static void GroupCommitQueueInsert(void);
+ static void GroupCommitCancelWait(void);
+ 
+ #ifdef USE_ASSERT_CHECKING
+ static bool GroupCommitQueueIsOrderedByLSN(void);
+ #endif
+ 
+ /*
+  * ================================================
+  * Group Commit functions for normal user backends
+  * ================================================
+  */
+ 
+ /*
+  * Wait for group commit.
+  *
+  * Initially backends start in state GROUP_COMMIT_NOT_WAITING and then
+  * change that state to GROUP_COMMIT_WAITING before adding ourselves
+  * to the wait queue. During GroupCommitWakeQueue() a WALWriter changes
+  * the state to GROUP_COMMIT_WAIT_COMPLETE once write/fsync is confirmed.
+  * This backend then resets its state to GROUP_COMMIT_NOT_WAITING.
+  */
+ bool
+ GroupCommitWaitForLSN(XLogRecPtr XactCommitLSN)
+ {
+ 	Assert(SHMQueueIsDetached(&(MyProc->waitLSNLinks)));
+ 
+ 	LWLockAcquire(GroupCommitLock, LW_EXCLUSIVE);
+ 	Assert(MyProc->grpCommitState == GROUP_COMMIT_NOT_WAITING);
+ 
+ 	/*
+ 	 * Set our waitLSN so WALWriter will know when to wake us, and add
+ 	 * ourselves to the queue.
+ 	 */
+ 	MyProc->waitLSN = XactCommitLSN;
+ 	MyProc->grpCommitState = GROUP_COMMIT_WAITING;
+ 	GroupCommitQueueInsert();
+ 	Assert(GroupCommitQueueIsOrderedByLSN());
+ 	LWLockRelease(GroupCommitLock);
+ 
+ #ifdef GROUP_COMMIT_DEBUG
+ 	elog(LOG, "waiting for LSN %X/%X",
+ 					XactCommitLSN.xlogid,
+ 					XactCommitLSN.xrecoff);
+ #endif
+ 
+ 	/*
+ 	 * Wait for specified LSN to be confirmed.
+ 	 *
+ 	 * Each proc has its own wait latch, so we perform a normal latch
+ 	 * check/wait loop here.
+ 	 */
+ 	for (;;)
+ 	{
+ 		int			grpCommitState;
+ 
+ 		/* Must reset the latch before testing state. */
+ 		ResetLatch(&MyProc->procLatch);
+ 
+ 		/*
+ 		 * Try checking the state without the lock first.  There's no
+ 		 * guarantee that we'll read the most up-to-date value, so if it looks
+ 		 * like we're still waiting, recheck while holding the lock.  But if
+ 		 * it looks like we're done, we must really be done, because once
+ 		 * walwriter changes the state to GROUP_COMMIT_WAIT_COMPLETE, it will
+ 		 * never update it again, so we can't be seeing a stale value in that
+ 		 * case.
+ 		 *
+ 		 * Note: on machines with weak memory ordering, the acquisition of
+ 		 * the lock is essential to avoid race conditions: we cannot be sure
+ 		 * the writer's state update has reached main memory until we acquire
+ 		 * the lock.  We could get rid of this dance if SetLatch/ResetLatch
+ 		 * contained memory barriers.
+ 		 */
+ 		grpCommitState = MyProc->grpCommitState;
+ 		if (grpCommitState == GROUP_COMMIT_WAITING)
+ 		{
+ 			LWLockAcquire(GroupCommitLock, LW_SHARED);
+ 			grpCommitState = MyProc->grpCommitState;
+ 			LWLockRelease(GroupCommitLock);
+ 		}
+ 		if (grpCommitState == GROUP_COMMIT_WAIT_COMPLETE)
+ 			break;
+ 
+ 		/*
+ 		 * If a wait for synchronous replication is pending, we can neither
+ 		 * acknowledge the commit nor raise ERROR or FATAL.  The latter would
+ 		 * lead the client to believe that that the transaction aborted, which
+ 		 * is not true: it's already committed locally. The former is no good
+ 		 * either: the client has requested synchronous replication, and is
+ 		 * entitled to assume that an acknowledged commit is also replicated,
+ 		 * which might not be true. So in this case we issue a WARNING (which
+ 		 * some clients may be able to interpret) and shut off further output.
+ 		 * We do NOT reset ProcDiePending, so that the process will die after
+ 		 * the commit is cleaned up.
+ 		 */
+ 		if (ProcDiePending)
+ 		{
+ 			ereport(WARNING,
+ 					(errcode(ERRCODE_ADMIN_SHUTDOWN),
+ 					 errmsg("canceling the wait for group commit and terminating connection due to administrator command")));
+ 			whereToSendOutput = DestNone;
+ 			GroupCommitCancelWait();
+ 			break;
+ 		}
+ 
+ 		/*
+ 		 * It's unclear what to do if a query cancel interrupt arrives.  We
+ 		 * can't actually abort at this point, but ignoring the interrupt
+ 		 * altogether is not helpful, so we just terminate the wait with a
+ 		 * suitable warning.
+ 		 */
+ 		if (QueryCancelPending)
+ 		{
+ 			QueryCancelPending = false;
+ 			ereport(WARNING,
+ 					(errmsg("canceling wait for group commit due to user request")));
+ 			GroupCommitCancelWait();
+ 			break;
+ 		}
+ 
+ 		/*
+ 		 * If the postmaster dies, we'll probably never get woken after write
+ 		 * because the wal writer processes will exit. So just bail out.
+ 		 */
+ 		if (!PostmasterIsAlive())
+ 		{
+ 			ProcDiePending = true;
+ 			whereToSendOutput = DestNone;
+ 			GroupCommitCancelWait();
+ 			break;
+ 		}
+ 
+ 		/*
+ 		 * Wait on latch.  Any condition that should wake us up will set
+ 		 * the latch, so no need for timeout.
+ 		 */
+ 		WaitLatch(&MyProc->procLatch, WL_LATCH_SET | WL_POSTMASTER_DEATH, -1);
+ 	}
+ 
+ 	/*
+ 	 * WalWriter has checked our LSN and has removed us from queue. Clean up
+ 	 * state and leave.  It's OK to reset these shared memory fields without
+ 	 * holding GroupCommitLock, because any WALwriter will ignore us anyway when
+ 	 * we're not on the queue.
+ 	 */
+ 	Assert(SHMQueueIsDetached(&(MyProc->waitLSNLinks)));
+ 	MyProc->grpCommitState = GROUP_COMMIT_NOT_WAITING;
+ 	MyProc->waitLSN.xlogid = 0;
+ 	MyProc->waitLSN.xrecoff = 0;
+ 
+ 	return true;
+ }
+ 
+ /*
+  * Insert MyProc into GroupCommitQueue, maintaining sorted invariant.
+  *
+  * Usually we will go at tail of queue, though it's possible that we arrive
+  * here out of order, so start at tail and work back to insertion point.
+  */
+ static void
+ GroupCommitQueueInsert(void)
+ {
+ 	PGPROC	   *proc;
+ 
+ 	proc = (PGPROC *) SHMQueuePrev(GroupCommitQueue(),
+ 								   GroupCommitQueue(),
+ 								   offsetof(PGPROC, waitLSNLinks));
+ 
+ 	while (proc)
+ 	{
+ 		/*
+ 		 * Stop at the queue element that we should after to ensure the queue
+ 		 * is ordered by LSN.
+ 		 */
+ 		if (XLByteLT(proc->waitLSN, MyProc->waitLSN))
+ 			break;
+ 
+ 		proc = (PGPROC *) SHMQueuePrev(GroupCommitQueue(),
+ 									   &(proc->waitLSNLinks),
+ 									   offsetof(PGPROC, waitLSNLinks));
+ 	}
+ 
+ 	if (proc)
+ 		SHMQueueInsertAfter(&(proc->waitLSNLinks), &(MyProc->waitLSNLinks));
+ 	else
+ 		SHMQueueInsertAfter(GroupCommitQueue(), &(MyProc->waitLSNLinks));
+ }
+ 
+ /*
+  * Acquire GroupCommitLock and cancel any wait currently in progress.
+  */
+ static void
+ GroupCommitCancelWait(void)
+ {
+ 	LWLockAcquire(GroupCommitLock, LW_EXCLUSIVE);
+ 	if (!SHMQueueIsDetached(&(MyProc->waitLSNLinks)))
+ 		SHMQueueDelete(&(MyProc->waitLSNLinks));
+ 	MyProc->grpCommitState = GROUP_COMMIT_NOT_WAITING;
+ 	LWLockRelease(GroupCommitLock);
+ }
+ 
+ void
+ GroupCommitCleanupAtProcExit(void)
+ {
+ 	if (!SHMQueueIsDetached(&(MyProc->waitLSNLinks)) &&
+ 		MyProc->grpCommitState != GROUP_COMMIT_NOT_WAITING)
+ 	{
+ 		LWLockAcquire(GroupCommitLock, LW_EXCLUSIVE);
+ 		SHMQueueDelete(&(MyProc->waitLSNLinks));
+ 		LWLockRelease(GroupCommitLock);
+ 	}
+ }
+ 
+ /*
+  * ==============================================
+  * Group Commit functions for WAL writer process
+  * ==============================================
+  */
+ 
+ /*
+  * Walk queue from head.  Set the state of any backends that need to be woken,
+  * remove them from the queue, and then wake them up to the requested LSN.
+  */
+ void
+ GroupCommitReleaseWaiters(XLogRecPtr flushLSN)
+ {
+ 	PGPROC	   *proc = NULL;
+ 	PGPROC	   *thisproc = NULL;
+ 	int			numprocs = 0;
+ 
+ 	LWLockAcquire(GroupCommitLock, LW_EXCLUSIVE);
+ 
+ 	Assert(GroupCommitQueueIsOrderedByLSN());
+ 
+ 	proc = (PGPROC *) SHMQueueNext(GroupCommitQueue(),
+ 								   GroupCommitQueue(),
+ 								   offsetof(PGPROC, waitLSNLinks));
+ 
+ 	while (proc)
+ 	{
+ 		/*
+ 		 * Assume the queue is ordered by LSN
+ 		 */
+ 		if (XLByteLT(flushLSN, proc->waitLSN))
+ 			break;
+ 
+ 		/*
+ 		 * Move to next proc, so we can delete thisproc from the queue.
+ 		 * thisproc is valid, proc may be NULL after this.
+ 		 */
+ 		thisproc = proc;
+ 		proc = (PGPROC *) SHMQueueNext(GroupCommitQueue(),
+ 									   &(proc->waitLSNLinks),
+ 									   offsetof(PGPROC, waitLSNLinks));
+ 
+ 		/*
+ 		 * Set state to complete; see GroupCommitWaitForLSN() for discussion of
+ 		 * the various states.
+ 		 */
+ 		thisproc->grpCommitState = GROUP_COMMIT_WAIT_COMPLETE;
+ 
+ 		/*
+ 		 * Remove thisproc from queue.
+ 		 */
+ 		SHMQueueDelete(&(thisproc->waitLSNLinks));
+ 
+ 		/*
+ 		 * Wake only when we have set state and removed from queue.
+ 		 */
+ 		SetLatch(&(thisproc->procLatch));
+ 
+ 		numprocs++;
+ 	}
+ 
+ 	LWLockRelease(GroupCommitLock);
+ 
+ #ifdef GROUP_COMMIT_DEBUG
+ 	elog(LOG, "released %d procs up to %X/%X",
+ 		 numprocs,
+ 		 flushLSN.xlogid,
+ 		 flushLSN.xrecoff);
+ #endif
+ }
+ 
+ #ifdef USE_ASSERT_CHECKING
+ static bool
+ GroupCommitQueueIsOrderedByLSN(void)
+ {
+ 	PGPROC	   *proc = NULL;
+ 	XLogRecPtr	lastLSN;
+ 
+ 	lastLSN.xlogid = 0;
+ 	lastLSN.xrecoff = 0;
+ 
+ 	proc = (PGPROC *) SHMQueueNext(GroupCommitQueue(),
+ 								   GroupCommitQueue(),
+ 								   offsetof(PGPROC, waitLSNLinks));
+ 
+ 	while (proc)
+ 	{
+ 		/*
+ 		 * Check the queue is ordered by LSN and that multiple procs don't
+ 		 * have matching LSNs
+ 		 */
+ 		if (XLByteLE(proc->waitLSN, lastLSN))
+ 			return false;
+ 
+ 		lastLSN = proc->waitLSN;
+ 
+ 		proc = (PGPROC *) SHMQueueNext(GroupCommitQueue(),
+ 									   &(proc->waitLSNLinks),
+ 									   offsetof(PGPROC, waitLSNLinks));
+ 	}
+ 
+ 	return true;
+ }
+ #endif
*** a/src/backend/access/transam/xact.c
--- b/src/backend/access/transam/xact.c
***************
*** 67,75 **** bool		XactDeferrable;
  
  int			synchronous_commit = SYNCHRONOUS_COMMIT_ON;
  
- int			CommitDelay = 0;	/* precommit delay in microseconds */
- int			CommitSiblings = 5; /* # concurrent xacts needed to sleep */
- 
  /*
   * MyXactAccessedTempRel is set when a temporary relation is accessed.
   * We don't allow PREPARE TRANSACTION in that case.  (This is global
--- 67,72 ----
***************
*** 1094,1115 **** RecordTransactionCommit(void)
  	if ((wrote_xlog && synchronous_commit > SYNCHRONOUS_COMMIT_OFF) ||
  		forceSyncCommit || nrels > 0)
  	{
- 		/*
- 		 * Synchronous commit case:
- 		 *
- 		 * Sleep before flush! So we can flush more than one commit records
- 		 * per single fsync.  (The idea is some other backend may do the
- 		 * XLogFlush while we're sleeping.  This needs work still, because on
- 		 * most Unixen, the minimum select() delay is 10msec or more, which is
- 		 * way too long.)
- 		 *
- 		 * We do not sleep if enableFsync is not turned on, nor if there are
- 		 * fewer than CommitSiblings other backends with active transactions.
- 		 */
- 		if (CommitDelay > 0 && enableFsync &&
- 			MinimumActiveBackends(CommitSiblings))
- 			pg_usleep(CommitDelay);
- 
  		XLogFlush(XactLastRecEnd);
  
  		/*
--- 1091,1096 ----
*** a/src/backend/access/transam/xlog.c
--- b/src/backend/access/transam/xlog.c
***************
*** 24,29 ****
--- 24,30 ----
  #include <unistd.h>
  
  #include "access/clog.h"
+ #include "access/groupcommit.h"
  #include "access/multixact.h"
  #include "access/subtrans.h"
  #include "access/transam.h"
***************
*** 39,44 ****
--- 40,46 ----
  #include "miscadmin.h"
  #include "pgstat.h"
  #include "postmaster/bgwriter.h"
+ #include "postmaster/walwriter.h"
  #include "postmaster/startup.h"
  #include "replication/walreceiver.h"
  #include "replication/walsender.h"
***************
*** 51,56 ****
--- 53,59 ----
  #include "storage/proc.h"
  #include "storage/procarray.h"
  #include "storage/reinit.h"
+ #include "storage/shmem.h"
  #include "storage/smgr.h"
  #include "storage/spin.h"
  #include "utils/builtins.h"
***************
*** 438,443 **** typedef struct XLogCtlData
--- 441,451 ----
  	Latch		WALWriterLatch;
  
  	/*
+ 	 * Group Commit queue. Protected by GroupCommitLock
+ 	 */
+ 	SHM_QUEUE	GroupCommitQueue;
+ 
+ 	/*
  	 * During recovery, we keep a copy of the latest checkpoint record here.
  	 * Used by the background writer when it wants to create a restartpoint.
  	 *
***************
*** 2036,2041 **** XLogFlush(XLogRecPtr record)
--- 2044,2053 ----
  	XLogRecPtr	WriteRqstPtr;
  	XLogwrtRqst WriteRqst;
  
+ 	/* Quick exit if already known flushed */
+ 	if (XLByteLE(record, LogwrtResult.Flush))
+ 		return;
+ 
  	/*
  	 * During REDO, we are reading not writing WAL.  Therefore, instead of
  	 * trying to flush the WAL, we should update minRecoveryPoint instead. We
***************
*** 2049,2058 **** XLogFlush(XLogRecPtr record)
  		return;
  	}
  
- 	/* Quick exit if already known flushed */
- 	if (XLByteLE(record, LogwrtResult.Flush))
- 		return;
- 
  #ifdef WAL_DEBUG
  	if (XLOG_DEBUG)
  		elog(LOG, "xlog flush request %X/%X; write %X/%X; flush %X/%X",
--- 2061,2066 ----
***************
*** 2071,2124 **** XLogFlush(XLogRecPtr record)
  	 * gives us some chance of avoiding another fsync immediately after.
  	 */
  
! 	/* initialize to given target; may increase below */
! 	WriteRqstPtr = record;
! 
! 	/* read LogwrtResult and update local state */
  	{
! 		/* use volatile pointer to prevent code rearrangement */
! 		volatile XLogCtlData *xlogctl = XLogCtl;
  
! 		SpinLockAcquire(&xlogctl->info_lck);
! 		if (XLByteLT(WriteRqstPtr, xlogctl->LogwrtRqst.Write))
! 			WriteRqstPtr = xlogctl->LogwrtRqst.Write;
! 		LogwrtResult = xlogctl->LogwrtResult;
! 		SpinLockRelease(&xlogctl->info_lck);
! 	}
  
! 	/* done already? */
! 	if (!XLByteLE(record, LogwrtResult.Flush))
! 	{
! 		/* now wait for the write lock */
! 		LWLockAcquire(WALWriteLock, LW_EXCLUSIVE);
! 		LogwrtResult = XLogCtl->Write.LogwrtResult;
  		if (!XLByteLE(record, LogwrtResult.Flush))
  		{
! 			/* try to write/flush later additions to XLOG as well */
! 			if (LWLockConditionalAcquire(WALInsertLock, LW_EXCLUSIVE))
! 			{
! 				XLogCtlInsert *Insert = &XLogCtl->Insert;
! 				uint32		freespace = INSERT_FREESPACE(Insert);
  
! 				if (freespace < SizeOfXLogRecord)		/* buffer is full */
! 					WriteRqstPtr = XLogCtl->xlblocks[Insert->curridx];
! 				else
! 				{
! 					WriteRqstPtr = XLogCtl->xlblocks[Insert->curridx];
! 					WriteRqstPtr.xrecoff -= freespace;
! 				}
! 				LWLockRelease(WALInsertLock);
! 				WriteRqst.Write = WriteRqstPtr;
! 				WriteRqst.Flush = WriteRqstPtr;
! 			}
! 			else
  			{
  				WriteRqst.Write = WriteRqstPtr;
  				WriteRqst.Flush = record;
  			}
! 			XLogWrite(WriteRqst, false, false);
  		}
- 		LWLockRelease(WALWriteLock);
  	}
  
  	END_CRIT_SECTION();
--- 2079,2149 ----
  	 * gives us some chance of avoiding another fsync immediately after.
  	 */
  
! 	if (IsUnderPostmaster && UseGroupCommit)
  	{
! 		/* read LogwrtResult and update local state */
! 		{
! 			/* use volatile pointer to prevent code rearrangement */
! 			volatile XLogCtlData *xlogctl = XLogCtl;
  
! 			SpinLockAcquire(&xlogctl->info_lck);
! 			if (XLByteLT(xlogctl->LogwrtRqst.Write, record))
! 				xlogctl->LogwrtRqst.Write = record;
! 			if (XLByteLT(xlogctl->LogwrtRqst.Flush, record))
! 				xlogctl->LogwrtRqst.Flush = record;
! 			LogwrtResult = xlogctl->LogwrtResult;
! 			SpinLockRelease(&xlogctl->info_lck);
! 		}
  
! 		/* done already? */
  		if (!XLByteLE(record, LogwrtResult.Flush))
  		{
! 			/*
! 			 * Kick the WAL Writer into action before we wait.
! 			 */
! 			SetLatch(WALWriterLatch());
  
! 			/*
! 			 * XXX This next bit requires special handling in group commit.
! 			 * What if we are asked to flush an LSN well ahead of actual pointer.
! 			 * Must be able to wake up process with a state of "bad request"
! 			 * and then return false to the main group commit request above.
! 			 */
! 			if (GroupCommitWaitForLSN(record))
! 				LogwrtResult.Flush = record;
! 		}
! 	}
! 	else
! 	{
! 		/* initialize to given target; may increase below */
! 		WriteRqstPtr = record;
! 
! 		/* read LogwrtResult and update local state */
! 		{
! 			/* use volatile pointer to prevent code rearrangement */
! 			volatile XLogCtlData *xlogctl = XLogCtl;
! 
! 			SpinLockAcquire(&xlogctl->info_lck);
! 			if (XLByteLT(WriteRqstPtr, xlogctl->LogwrtRqst.Write))
! 				WriteRqstPtr = xlogctl->LogwrtRqst.Write;
! 			LogwrtResult = xlogctl->LogwrtResult;
! 			SpinLockRelease(&xlogctl->info_lck);
! 		}
! 
! 		/* done already? */
! 		if (!XLByteLE(record, LogwrtResult.Flush))
! 		{
! 			/* now wait for the write lock */
! 			LWLockAcquire(WALWriteLock, LW_EXCLUSIVE);
! 			LogwrtResult = XLogCtl->Write.LogwrtResult;
! 			if (!XLByteLE(record, LogwrtResult.Flush))
  			{
  				WriteRqst.Write = WriteRqstPtr;
  				WriteRqst.Flush = record;
+ 				XLogWrite(WriteRqst, false, false);
  			}
! 			LWLockRelease(WALWriteLock);
  		}
  	}
  
  	END_CRIT_SECTION();
***************
*** 2165,2180 **** XLogFlush(XLogRecPtr record)
   * case for async commits.)
   *
   * This routine is invoked periodically by the background walwriter process.
   */
! void
  XLogBackgroundFlush(void)
  {
  	XLogRecPtr	WriteRqstPtr;
! 	bool		flexible = true;
  
  	/* XLOG doesn't need flushing during recovery */
  	if (RecoveryInProgress())
! 		return;
  
  	/* read LogwrtResult and update local state */
  	{
--- 2190,2209 ----
   * case for async commits.)
   *
   * This routine is invoked periodically by the background walwriter process.
+  *
+  * Returns true if we attempted to write WAL.
   */
! int
  XLogBackgroundFlush(void)
  {
  	XLogRecPtr	WriteRqstPtr;
! 	XLogRecPtr	AsyncRqstPtr;
! 	bool		flexible = false;
! 	int			request;
  
  	/* XLOG doesn't need flushing during recovery */
  	if (RecoveryInProgress())
! 		return false;
  
  	/* read LogwrtResult and update local state */
  	{
***************
*** 2184,2205 **** XLogBackgroundFlush(void)
  		SpinLockAcquire(&xlogctl->info_lck);
  		LogwrtResult = xlogctl->LogwrtResult;
  		WriteRqstPtr = xlogctl->LogwrtRqst.Write;
  		SpinLockRelease(&xlogctl->info_lck);
  	}
  
! 	/* back off to last completed page boundary */
! 	WriteRqstPtr.xrecoff -= WriteRqstPtr.xrecoff % XLOG_BLCKSZ;
  
! 	/* if we have already flushed that far, consider async commit records */
! 	if (XLByteLE(WriteRqstPtr, LogwrtResult.Flush))
  	{
! 		/* use volatile pointer to prevent code rearrangement */
! 		volatile XLogCtlData *xlogctl = XLogCtl;
! 
! 		SpinLockAcquire(&xlogctl->info_lck);
! 		WriteRqstPtr = xlogctl->asyncXactLSN;
! 		SpinLockRelease(&xlogctl->info_lck);
! 		flexible = false;		/* ensure it all gets written */
  	}
  
  	/*
--- 2213,2250 ----
  		SpinLockAcquire(&xlogctl->info_lck);
  		LogwrtResult = xlogctl->LogwrtResult;
  		WriteRqstPtr = xlogctl->LogwrtRqst.Write;
+ 		AsyncRqstPtr = xlogctl->asyncXactLSN;
  		SpinLockRelease(&xlogctl->info_lck);
  	}
  
! #ifdef GROUP_COMMIT_DEBUG
! 	elog(LOG, "write %X/%X async %X/%X flush %X/%X",
! 					WriteRqstPtr.xlogid,
! 					WriteRqstPtr.xrecoff,
! 					AsyncRqstPtr.xlogid,
! 					AsyncRqstPtr.xrecoff,
! 					LogwrtResult.Flush.xlogid,
! 					LogwrtResult.Flush.xrecoff);
! #endif
  
! 	if (XLByteLT(LogwrtResult.Flush, WriteRqstPtr))
! 		request = WAL_WRITE_SYNC;
! 	else
  	{
! 		/*
! 		 * We get here either because we hit end of a WAL block during
! 		 * XLogSetAsyncXactLSN() or because we hit an wakeup from inaction.
! 		 * Either way we want to flush up to the current pointer, so there is
! 		 * no need to do anything here to ensure efficient writes at page
! 		 * boundaries.
! 		 */
! 		if (XLByteLT(LogwrtResult.Flush, AsyncRqstPtr))
! 		{
! 			WriteRqstPtr = AsyncRqstPtr;
! 			request = WAL_WRITE_ASYNC;
! 		}
! 		else
! 			request = WAL_WRITE_NONE;
  	}
  
  	/*
***************
*** 2216,2222 **** XLogBackgroundFlush(void)
  				XLogFileClose();
  			}
  		}
! 		return;
  	}
  
  #ifdef WAL_DEBUG
--- 2261,2267 ----
  				XLogFileClose();
  			}
  		}
! 		return false;
  	}
  
  #ifdef WAL_DEBUG
***************
*** 2243,2248 **** XLogBackgroundFlush(void)
--- 2288,2297 ----
  	LWLockRelease(WALWriteLock);
  
  	END_CRIT_SECTION();
+ 
+ 	GroupCommitReleaseWaiters(WriteRqstPtr);
+ 
+ 	return request;
  }
  
  /*
***************
*** 5094,5099 **** XLOGShmemInit(void)
--- 5143,5149 ----
  	SpinLockInit(&XLogCtl->info_lck);
  	InitSharedLatch(&XLogCtl->recoveryWakeupLatch);
  	InitSharedLatch(&XLogCtl->WALWriterLatch);
+ 	SHMQueueInit(&(XLogCtl->GroupCommitQueue));
  
  	/*
  	 * If we are not in bootstrap mode, pg_control should already exist. Read
***************
*** 10044,10046 **** WALWriterLatch(void)
--- 10094,10105 ----
  {
  	return &XLogCtl->WALWriterLatch;
  }
+ 
+ /*
+  * Manage the group commit queue. Use this only when GroupCommitLock held.
+  */
+ SHM_QUEUE	*
+ GroupCommitQueue(void)
+ {
+ 	return &XLogCtl->GroupCommitQueue;
+ }
*** a/src/backend/postmaster/checkpointer.c
--- b/src/backend/postmaster/checkpointer.c
***************
*** 41,46 ****
--- 41,47 ----
  #include <time.h>
  #include <unistd.h>
  
+ #include "access/groupcommit.h"
  #include "access/xlog_internal.h"
  #include "libpq/pqsignal.h"
  #include "miscadmin.h"
*** a/src/backend/postmaster/walwriter.c
--- b/src/backend/postmaster/walwriter.c
***************
*** 62,70 ****
  
  
  /*
!  * GUC parameters
   */
! int			WalWriterDelay = 200;
  
  /*
   * Flags set by interrupt handlers for later service in the main loop.
--- 62,72 ----
  
  
  /*
!  * Timing parameters
   */
! #define		WALWRITER_NORMAL_TIMEOUT		200
! #define		WALWRITER_LOOPS_UNTIL_POWERSAVE	300
! #define 	WALWRITER_POWERSAVE_TIMEOUT		60000
  
  /*
   * Flags set by interrupt handlers for later service in the main loop.
***************
*** 88,93 **** WalWriterMain(void)
--- 90,98 ----
  {
  	sigjmp_buf	local_sigjmp_buf;
  	MemoryContext walwriter_context;
+ 	int			loops_until_powersave = WALWRITER_LOOPS_UNTIL_POWERSAVE;
+ 	int			delay = WALWRITER_NORMAL_TIMEOUT;
+ 	int			flush_result;
  
  	InitLatch(WALWriterLatch()); /* initialize latch used in main loop */
  
***************
*** 246,258 **** WalWriterMain(void)
  		}
  
  		/*
! 		 * Do what we're here for...
  		 */
! 		XLogBackgroundFlush();
  
  		(void) WaitLatch(WALWriterLatch(),
  							   WL_LATCH_SET | WL_TIMEOUT | WL_POSTMASTER_DEATH,
! 							   WalWriterDelay /* ms */);
  	}
  }
  
--- 251,295 ----
  		}
  
  		/*
! 		 * Do what we're here for.
  		 */
! 		flush_result = XLogBackgroundFlush();
! 
! 		switch (flush_result)
! 		{
! 			case WAL_WRITE_SYNC:
! 					/*
! 					 * If we wrote a commit, repeat again quickly.
! 					 * We musn't wait here because additional wakeups
! 					 * may have occurred while we were busy.
! 					 */
! 					loops_until_powersave = WALWRITER_LOOPS_UNTIL_POWERSAVE;
! 					continue;
! 
! 			case WAL_WRITE_ASYNC:
! 					/*
! 					 * If we're working with async commits, sleep until
! 					 * we need to write out whole blocks or wake to avoid
! 					 * too long a wait.
! 					 */
! 					delay = WALWRITER_NORMAL_TIMEOUT;
! 					loops_until_powersave = WALWRITER_LOOPS_UNTIL_POWERSAVE;
! 					break;
! 
! 			case WAL_WRITE_NONE:
! 					if (--loops_until_powersave > 0)
! 						delay = WALWRITER_NORMAL_TIMEOUT;
! 					else
! 					{
! 						loops_until_powersave = 0;
! 						delay = WALWRITER_POWERSAVE_TIMEOUT;
! 					}
! 					break;
! 		}
  
  		(void) WaitLatch(WALWriterLatch(),
  							   WL_LATCH_SET | WL_TIMEOUT | WL_POSTMASTER_DEATH,
! 							   delay /* ms */);
  	}
  }
  
*** a/src/backend/replication/syncrep.c
--- b/src/backend/replication/syncrep.c
***************
*** 105,111 **** SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
  	if (!SyncRepRequested() || !SyncStandbysDefined())
  		return;
  
! 	Assert(SHMQueueIsDetached(&(MyProc->syncRepLinks)));
  	Assert(WalSndCtl != NULL);
  
  	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
--- 105,111 ----
  	if (!SyncRepRequested() || !SyncStandbysDefined())
  		return;
  
! 	Assert(SHMQueueIsDetached(&(MyProc->waitLSNLinks)));
  	Assert(WalSndCtl != NULL);
  
  	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
***************
*** 253,259 **** SyncRepWaitForLSN(XLogRecPtr XactCommitLSN)
  	 * holding SyncRepLock, because any walsenders will ignore us anyway when
  	 * we're not on the queue.
  	 */
! 	Assert(SHMQueueIsDetached(&(MyProc->syncRepLinks)));
  	MyProc->syncRepState = SYNC_REP_NOT_WAITING;
  	MyProc->waitLSN.xlogid = 0;
  	MyProc->waitLSN.xrecoff = 0;
--- 253,259 ----
  	 * holding SyncRepLock, because any walsenders will ignore us anyway when
  	 * we're not on the queue.
  	 */
! 	Assert(SHMQueueIsDetached(&(MyProc->waitLSNLinks)));
  	MyProc->syncRepState = SYNC_REP_NOT_WAITING;
  	MyProc->waitLSN.xlogid = 0;
  	MyProc->waitLSN.xrecoff = 0;
***************
*** 279,285 **** SyncRepQueueInsert(void)
  
  	proc = (PGPROC *) SHMQueuePrev(&(WalSndCtl->SyncRepQueue),
  								   &(WalSndCtl->SyncRepQueue),
! 								   offsetof(PGPROC, syncRepLinks));
  
  	while (proc)
  	{
--- 279,285 ----
  
  	proc = (PGPROC *) SHMQueuePrev(&(WalSndCtl->SyncRepQueue),
  								   &(WalSndCtl->SyncRepQueue),
! 								   offsetof(PGPROC, waitLSNLinks));
  
  	while (proc)
  	{
***************
*** 291,304 **** SyncRepQueueInsert(void)
  			break;
  
  		proc = (PGPROC *) SHMQueuePrev(&(WalSndCtl->SyncRepQueue),
! 									   &(proc->syncRepLinks),
! 									   offsetof(PGPROC, syncRepLinks));
  	}
  
  	if (proc)
! 		SHMQueueInsertAfter(&(proc->syncRepLinks), &(MyProc->syncRepLinks));
  	else
! 		SHMQueueInsertAfter(&(WalSndCtl->SyncRepQueue), &(MyProc->syncRepLinks));
  }
  
  /*
--- 291,304 ----
  			break;
  
  		proc = (PGPROC *) SHMQueuePrev(&(WalSndCtl->SyncRepQueue),
! 									   &(proc->waitLSNLinks),
! 									   offsetof(PGPROC, waitLSNLinks));
  	}
  
  	if (proc)
! 		SHMQueueInsertAfter(&(proc->waitLSNLinks), &(MyProc->waitLSNLinks));
  	else
! 		SHMQueueInsertAfter(&(WalSndCtl->SyncRepQueue), &(MyProc->waitLSNLinks));
  }
  
  /*
***************
*** 308,315 **** static void
  SyncRepCancelWait(void)
  {
  	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
! 	if (!SHMQueueIsDetached(&(MyProc->syncRepLinks)))
! 		SHMQueueDelete(&(MyProc->syncRepLinks));
  	MyProc->syncRepState = SYNC_REP_NOT_WAITING;
  	LWLockRelease(SyncRepLock);
  }
--- 308,315 ----
  SyncRepCancelWait(void)
  {
  	LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
! 	if (!SHMQueueIsDetached(&(MyProc->waitLSNLinks)))
! 		SHMQueueDelete(&(MyProc->waitLSNLinks));
  	MyProc->syncRepState = SYNC_REP_NOT_WAITING;
  	LWLockRelease(SyncRepLock);
  }
***************
*** 317,326 **** SyncRepCancelWait(void)
  void
  SyncRepCleanupAtProcExit(void)
  {
! 	if (!SHMQueueIsDetached(&(MyProc->syncRepLinks)))
  	{
  		LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
! 		SHMQueueDelete(&(MyProc->syncRepLinks));
  		LWLockRelease(SyncRepLock);
  	}
  }
--- 317,327 ----
  void
  SyncRepCleanupAtProcExit(void)
  {
! 	if (!SHMQueueIsDetached(&(MyProc->waitLSNLinks)) &&
! 		MyProc->syncRepState != SYNC_REP_NOT_WAITING)
  	{
  		LWLockAcquire(SyncRepLock, LW_EXCLUSIVE);
! 		SHMQueueDelete(&(MyProc->waitLSNLinks));
  		LWLockRelease(SyncRepLock);
  	}
  }
***************
*** 525,531 **** SyncRepWakeQueue(bool all)
  
  	proc = (PGPROC *) SHMQueueNext(&(WalSndCtl->SyncRepQueue),
  								   &(WalSndCtl->SyncRepQueue),
! 								   offsetof(PGPROC, syncRepLinks));
  
  	while (proc)
  	{
--- 526,532 ----
  
  	proc = (PGPROC *) SHMQueueNext(&(WalSndCtl->SyncRepQueue),
  								   &(WalSndCtl->SyncRepQueue),
! 								   offsetof(PGPROC, waitLSNLinks));
  
  	while (proc)
  	{
***************
*** 541,548 **** SyncRepWakeQueue(bool all)
  		 */
  		thisproc = proc;
  		proc = (PGPROC *) SHMQueueNext(&(WalSndCtl->SyncRepQueue),
! 									   &(proc->syncRepLinks),
! 									   offsetof(PGPROC, syncRepLinks));
  
  		/*
  		 * Set state to complete; see SyncRepWaitForLSN() for discussion of
--- 542,549 ----
  		 */
  		thisproc = proc;
  		proc = (PGPROC *) SHMQueueNext(&(WalSndCtl->SyncRepQueue),
! 									   &(proc->waitLSNLinks),
! 									   offsetof(PGPROC, waitLSNLinks));
  
  		/*
  		 * Set state to complete; see SyncRepWaitForLSN() for discussion of
***************
*** 553,559 **** SyncRepWakeQueue(bool all)
  		/*
  		 * Remove thisproc from queue.
  		 */
! 		SHMQueueDelete(&(thisproc->syncRepLinks));
  
  		/*
  		 * Wake only when we have set state and removed from queue.
--- 554,560 ----
  		/*
  		 * Remove thisproc from queue.
  		 */
! 		SHMQueueDelete(&(thisproc->waitLSNLinks));
  
  		/*
  		 * Wake only when we have set state and removed from queue.
***************
*** 615,621 **** SyncRepQueueIsOrderedByLSN(void)
  
  	proc = (PGPROC *) SHMQueueNext(&(WalSndCtl->SyncRepQueue),
  								   &(WalSndCtl->SyncRepQueue),
! 								   offsetof(PGPROC, syncRepLinks));
  
  	while (proc)
  	{
--- 616,622 ----
  
  	proc = (PGPROC *) SHMQueueNext(&(WalSndCtl->SyncRepQueue),
  								   &(WalSndCtl->SyncRepQueue),
! 								   offsetof(PGPROC, waitLSNLinks));
  
  	while (proc)
  	{
***************
*** 629,636 **** SyncRepQueueIsOrderedByLSN(void)
  		lastLSN = proc->waitLSN;
  
  		proc = (PGPROC *) SHMQueueNext(&(WalSndCtl->SyncRepQueue),
! 									   &(proc->syncRepLinks),
! 									   offsetof(PGPROC, syncRepLinks));
  	}
  
  	return true;
--- 630,637 ----
  		lastLSN = proc->waitLSN;
  
  		proc = (PGPROC *) SHMQueueNext(&(WalSndCtl->SyncRepQueue),
! 									   &(proc->waitLSNLinks),
! 									   offsetof(PGPROC, waitLSNLinks));
  	}
  
  	return true;
*** a/src/backend/storage/lmgr/proc.c
--- b/src/backend/storage/lmgr/proc.c
***************
*** 35,40 ****
--- 35,41 ----
  #include <unistd.h>
  #include <sys/time.h>
  
+ #include "access/groupcommit.h"
  #include "access/transam.h"
  #include "access/xact.h"
  #include "miscadmin.h"
***************
*** 342,352 **** InitProcess(void)
  #endif
  	MyProc->recoveryConflictPending = false;
  
! 	/* Initialize fields for sync rep */
  	MyProc->waitLSN.xlogid = 0;
  	MyProc->waitLSN.xrecoff = 0;
  	MyProc->syncRepState = SYNC_REP_NOT_WAITING;
! 	SHMQueueElemInit(&(MyProc->syncRepLinks));
  
  	/*
  	 * Acquire ownership of the PGPROC's latch, so that we can use WaitLatch.
--- 343,354 ----
  #endif
  	MyProc->recoveryConflictPending = false;
  
! 	/* Initialize fields for waiting on WAL LSNs */
  	MyProc->waitLSN.xlogid = 0;
  	MyProc->waitLSN.xrecoff = 0;
  	MyProc->syncRepState = SYNC_REP_NOT_WAITING;
! 	MyProc->grpCommitState = GROUP_COMMIT_NOT_WAITING;
! 	SHMQueueElemInit(&(MyProc->waitLSNLinks));
  
  	/*
  	 * Acquire ownership of the PGPROC's latch, so that we can use WaitLatch.
***************
*** 509,514 **** InitAuxiliaryProcess(void)
--- 511,522 ----
  	PGSemaphoreReset(&MyProc->sem);
  
  	/*
+ 	 * Don't use Group Commit for auxiliary processes, so we can shutdown without
+ 	 * creating a dependency on the WALWriter process.
+ 	 */
+ 	UseGroupCommit = false;
+ 
+ 	/*
  	 * Arrange to clean up at process exit.
  	 */
  	on_shmem_exit(AuxiliaryProcKill, Int32GetDatum(proctype));
***************
*** 703,710 **** ProcKill(int code, Datum arg)
  
  	Assert(MyProc != NULL);
  
! 	/* Make sure we're out of the sync rep lists */
  	SyncRepCleanupAtProcExit();
  
  #ifdef USE_ASSERT_CHECKING
  	if (assert_enabled)
--- 711,719 ----
  
  	Assert(MyProc != NULL);
  
! 	/* Make sure we're out of the wait for LSN lists */
  	SyncRepCleanupAtProcExit();
+ 	GroupCommitCleanupAtProcExit();
  
  #ifdef USE_ASSERT_CHECKING
  	if (assert_enabled)
*** a/src/backend/utils/misc/guc.c
--- b/src/backend/utils/misc/guc.c
***************
*** 125,132 ****
  
  /* XXX these should appear in other modules' header files */
  extern bool Log_disconnections;
- extern int	CommitDelay;
- extern int	CommitSiblings;
  extern char *default_tablespace;
  extern char *temp_tablespaces;
  extern bool synchronize_seqscans;
--- 125,130 ----
***************
*** 1986,2002 **** static struct config_int ConfigureNamesInt[] =
  	},
  
  	{
- 		{"wal_writer_delay", PGC_SIGHUP, WAL_SETTINGS,
- 			gettext_noop("WAL writer sleep time between WAL flushes."),
- 			NULL,
- 			GUC_UNIT_MS
- 		},
- 		&WalWriterDelay,
- 		200, 1, 10000,
- 		NULL, NULL, NULL
- 	},
- 
- 	{
  		/* see max_connections */
  		{"max_wal_senders", PGC_POSTMASTER, REPLICATION_SENDING,
  			gettext_noop("Sets the maximum number of simultaneously running WAL sender processes."),
--- 1984,1989 ----
***************
*** 2019,2046 **** static struct config_int ConfigureNamesInt[] =
  	},
  
  	{
- 		{"commit_delay", PGC_USERSET, WAL_SETTINGS,
- 			gettext_noop("Sets the delay in microseconds between transaction commit and "
- 						 "flushing WAL to disk."),
- 			NULL
- 		},
- 		&CommitDelay,
- 		0, 0, 100000,
- 		NULL, NULL, NULL
- 	},
- 
- 	{
- 		{"commit_siblings", PGC_USERSET, WAL_SETTINGS,
- 			gettext_noop("Sets the minimum concurrent open transactions before performing "
- 						 "commit_delay."),
- 			NULL
- 		},
- 		&CommitSiblings,
- 		5, 0, 1000,
- 		NULL, NULL, NULL
- 	},
- 
- 	{
  		{"extra_float_digits", PGC_USERSET, CLIENT_CONN_LOCALE,
  			gettext_noop("Sets the number of digits displayed for floating-point values."),
  			gettext_noop("This affects real, double precision, and geometric data types. "
--- 2006,2011 ----
*** /dev/null
--- b/src/include/access/groupcommit.h
***************
*** 0 ****
--- 1,34 ----
+ /*-------------------------------------------------------------------------
+  *
+  * groupcommit.h
+  *	  Exports from access/transam/groupcommit.c.
+  *
+  * Portions Copyright (c) 2010-2011, PostgreSQL Global Development Group
+  *
+  * IDENTIFICATION
+  *		src/include/access/groupcommit.h
+  *
+  *-------------------------------------------------------------------------
+  */
+ #ifndef _GROUPCOMMIT_H
+ #define _GROUPCOMMIT_H
+ 
+ #include "access/xlogdefs.h"
+ 
+ /* grpCommitState */
+ #define GROUP_COMMIT_NOT_WAITING		0
+ #define GROUP_COMMIT_WAITING			1
+ #define GROUP_COMMIT_WAIT_COMPLETE		2
+ 
+ extern PGDLLIMPORT bool UseGroupCommit;
+ 
+ /* called by user backend */
+ extern bool GroupCommitWaitForLSN(XLogRecPtr XactCommitLSN);
+ 
+ /* called at backend exit */
+ extern void GroupCommitCleanupAtProcExit(void);
+ 
+ /* called by wal writer */
+ extern void GroupCommitReleaseWaiters(XLogRecPtr flushLSN);
+ 
+ #endif   /* _GROUPCOMMIT_H */
*** a/src/include/access/xlog.h
--- b/src/include/access/xlog.h
***************
*** 17,22 ****
--- 17,23 ----
  #include "lib/stringinfo.h"
  #include "storage/buf.h"
  #include "storage/latch.h"
+ #include "storage/shmem.h"
  #include "utils/pg_crc.h"
  
  /*
***************
*** 271,277 **** extern CheckpointStatsData CheckpointStats;
  
  extern XLogRecPtr XLogInsert(RmgrId rmid, uint8 info, XLogRecData *rdata);
  extern void XLogFlush(XLogRecPtr RecPtr);
! extern void XLogBackgroundFlush(void);
  extern bool XLogNeedsFlush(XLogRecPtr RecPtr);
  extern int XLogFileInit(uint32 log, uint32 seg,
  			 bool *use_existent, bool use_lock);
--- 272,278 ----
  
  extern XLogRecPtr XLogInsert(RmgrId rmid, uint8 info, XLogRecData *rdata);
  extern void XLogFlush(XLogRecPtr RecPtr);
! extern int XLogBackgroundFlush(void);
  extern bool XLogNeedsFlush(XLogRecPtr RecPtr);
  extern int XLogFileInit(uint32 log, uint32 seg,
  			 bool *use_existent, bool use_lock);
***************
*** 321,326 **** extern TimeLineID GetRecoveryTargetTLI(void);
--- 322,328 ----
  extern bool CheckPromoteSignal(void);
  extern void WakeupRecovery(void);
  extern Latch *WALWriterLatch(void);
+ extern SHM_QUEUE *GroupCommitQueue(void);
  
  /*
   * Starting/stopping a base backup
*** a/src/include/postmaster/walwriter.h
--- b/src/include/postmaster/walwriter.h
***************
*** 12,19 ****
  #ifndef _WALWRITER_H
  #define _WALWRITER_H
  
! /* GUC options */
! extern int	WalWriterDelay;
  
  extern void WalWriterMain(void);
  
--- 12,20 ----
  #ifndef _WALWRITER_H
  #define _WALWRITER_H
  
! #define WAL_WRITE_SYNC					0
! #define WAL_WRITE_ASYNC					1
! #define WAL_WRITE_NONE					2
  
  extern void WalWriterMain(void);
  
*** a/src/include/storage/lwlock.h
--- b/src/include/storage/lwlock.h
***************
*** 79,84 **** typedef enum LWLockId
--- 79,85 ----
  	SerializablePredicateLockListLock,
  	OldSerXidLock,
  	SyncRepLock,
+ 	GroupCommitLock,
  	/* Individual lock IDs end here */
  	FirstBufMappingLock,
  	FirstLockMgrLock = FirstBufMappingLock + NUM_BUFFER_PARTITIONS,
*** a/src/include/storage/proc.h
--- b/src/include/storage/proc.h
***************
*** 134,141 **** struct PGPROC
  	 * syncRepLinks used only while holding SyncRepLock.
  	 */
  	XLogRecPtr	waitLSN;		/* waiting for this LSN or higher */
  	int			syncRepState;	/* wait state for sync rep */
! 	SHM_QUEUE	syncRepLinks;	/* list link if process is in syncrep queue */
  
  	/*
  	 * All PROCLOCK objects for locks held or awaited by this backend are
--- 134,142 ----
  	 * syncRepLinks used only while holding SyncRepLock.
  	 */
  	XLogRecPtr	waitLSN;		/* waiting for this LSN or higher */
+ 	int			grpCommitState;	/* wait state for group commit */
  	int			syncRepState;	/* wait state for sync rep */
! 	SHM_QUEUE	waitLSNLinks;	/* list link if process is in WAL queue */
  
  	/*
  	 * All PROCLOCK objects for locks held or awaited by this backend are
