aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--doc/src/sgml/backup.sgml16
-rw-r--r--doc/src/sgml/config.sgml15
-rw-r--r--src/backend/access/transam/xlog.c11
-rw-r--r--src/backend/storage/buffer/bufmgr.c40
-rw-r--r--src/backend/storage/ipc/procarray.c20
-rw-r--r--src/backend/storage/ipc/procsignal.c5
-rw-r--r--src/backend/storage/ipc/standby.c124
-rw-r--r--src/backend/storage/lmgr/lock.c21
-rw-r--r--src/backend/storage/lmgr/proc.c150
-rw-r--r--src/backend/tcop/postgres.c14
-rw-r--r--src/backend/utils/misc/guc.c4
-rw-r--r--src/include/storage/bufmgr.h3
-rw-r--r--src/include/storage/proc.h12
-rw-r--r--src/include/storage/procarray.h7
-rw-r--r--src/include/storage/procsignal.h3
-rw-r--r--src/include/storage/standby.h9
16 files changed, 367 insertions, 87 deletions
diff --git a/doc/src/sgml/backup.sgml b/doc/src/sgml/backup.sgml
index da52f42af69..a2c1375431a 100644
--- a/doc/src/sgml/backup.sgml
+++ b/doc/src/sgml/backup.sgml
@@ -1,4 +1,4 @@
-<!-- $PostgreSQL: pgsql/doc/src/sgml/backup.sgml,v 2.136 2010/01/15 09:18:56 heikki Exp $ -->
+<!-- $PostgreSQL: pgsql/doc/src/sgml/backup.sgml,v 2.137 2010/01/23 16:37:12 sriggs Exp $ -->
<chapter id="backup">
<title>Backup and Restore</title>
@@ -2399,7 +2399,7 @@ primary_conninfo = 'host=192.168.1.50 port=5432 user=foo password=foopass'
</listitem>
<listitem>
<para>
- Waiting to acquire buffer cleanup locks (for which there is no time out)
+ Waiting to acquire buffer cleanup locks
</para>
</listitem>
<listitem>
@@ -2536,11 +2536,7 @@ primary_conninfo = 'host=192.168.1.50 port=5432 user=foo password=foopass'
Three-way deadlocks are possible between AccessExclusiveLocks arriving from
the primary, cleanup WAL records that require buffer cleanup locks and
user requests that are waiting behind replayed AccessExclusiveLocks. Deadlocks
- are currently resolved by the cancellation of user processes that would
- need to wait on a lock. This is heavy-handed and generates more query
- cancellations than we need to, though does remove the possibility of deadlock.
- This behaviour is expected to improve substantially for the main release
- version of 8.5.
+ are resolved by time-out when we exceed <varname>max_standby_delay</>.
</para>
<para>
@@ -2630,11 +2626,7 @@ LOG: database system is ready to accept read only connections
<varname>max_standby_delay</> or even set it to zero, though that is a
very aggressive setting. If the standby server is tasked as an additional
server for decision support queries then it may be acceptable to set this
- to a value of many hours (in seconds). It is also possible to set
- <varname>max_standby_delay</> to -1 which means wait forever for queries
- to complete, if there are conflicts; this will be useful when performing
- an archive recovery from a backup.
- </para>
+ to a value of many hours (in seconds).
<para>
Transaction status "hint bits" written on primary are not WAL-logged,
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 03667754b85..4a681991954 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -1,4 +1,4 @@
-<!-- $PostgreSQL: pgsql/doc/src/sgml/config.sgml,v 1.244 2010/01/15 09:18:58 heikki Exp $ -->
+<!-- $PostgreSQL: pgsql/doc/src/sgml/config.sgml,v 1.245 2010/01/23 16:37:12 sriggs Exp $ -->
<chapter Id="runtime-config">
<title>Server Configuration</title>
@@ -1825,14 +1825,15 @@ archive_command = 'copy "%p" "C:\\server\\archivedir\\%f"' # Windows
<listitem>
<para>
When server acts as a standby, this parameter specifies a wait policy
- for queries that conflict with incoming data changes. Valid settings
- are -1, meaning wait forever, or a wait time of 0 or more seconds.
- If a conflict should occur the server will delay up to this
- amount before it begins trying to resolve things less amicably, as
+ for queries that conflict with data changes being replayed by recovery.
+ If a conflict should occur the server will delay up to this number
+ of seconds before it begins trying to resolve things less amicably, as
described in <xref linkend="hot-standby-conflict">. Typically,
this parameter makes sense only during replication, so when
- performing an archive recovery to recover from data loss a
- parameter setting of 0 is recommended. The default is 30 seconds.
+ performing an archive recovery to recover from data loss a very high
+ parameter setting is recommended. The default is 30 seconds.
+ There is no wait-forever setting because of the potential for deadlock
+ which that setting would introduce.
This parameter can only be set in the <filename>postgresql.conf</>
file or on the server command line.
</para>
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index a7ff66a6b86..004ea0a46bf 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $PostgreSQL: pgsql/src/backend/access/transam/xlog.c,v 1.359 2010/01/20 19:43:40 heikki Exp $
+ * $PostgreSQL: pgsql/src/backend/access/transam/xlog.c,v 1.360 2010/01/23 16:37:12 sriggs Exp $
*
*-------------------------------------------------------------------------
*/
@@ -8803,9 +8803,12 @@ StartupProcessMain(void)
*/
pqsignal(SIGHUP, StartupProcSigHupHandler); /* reload config file */
pqsignal(SIGINT, SIG_IGN); /* ignore query cancel */
- pqsignal(SIGTERM, StartupProcShutdownHandler); /* request shutdown */
- pqsignal(SIGQUIT, startupproc_quickdie); /* hard crash time */
- pqsignal(SIGALRM, SIG_IGN);
+ pqsignal(SIGTERM, StartupProcShutdownHandler); /* request shutdown */
+ pqsignal(SIGQUIT, startupproc_quickdie); /* hard crash time */
+ if (XLogRequestRecoveryConnections)
+ pqsignal(SIGALRM, handle_standby_sig_alarm); /* ignored unless InHotStandby */
+ else
+ pqsignal(SIGALRM, SIG_IGN);
pqsignal(SIGPIPE, SIG_IGN);
pqsignal(SIGUSR1, SIG_IGN);
pqsignal(SIGUSR2, SIG_IGN);
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 77f35fd361d..470800d5f47 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/storage/buffer/bufmgr.c,v 1.254 2010/01/02 16:57:51 momjian Exp $
+ * $PostgreSQL: pgsql/src/backend/storage/buffer/bufmgr.c,v 1.255 2010/01/23 16:37:12 sriggs Exp $
*
*-------------------------------------------------------------------------
*/
@@ -44,6 +44,7 @@
#include "storage/ipc.h"
#include "storage/proc.h"
#include "storage/smgr.h"
+#include "storage/standby.h"
#include "utils/rel.h"
#include "utils/resowner.h"
@@ -2417,14 +2418,49 @@ LockBufferForCleanup(Buffer buffer)
PinCountWaitBuf = bufHdr;
UnlockBufHdr(bufHdr);
LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
+
/* Wait to be signaled by UnpinBuffer() */
- ProcWaitForSignal();
+ if (InHotStandby)
+ {
+ /* Share the bufid that Startup process waits on */
+ SetStartupBufferPinWaitBufId(buffer - 1);
+ /* Set alarm and then wait to be signaled by UnpinBuffer() */
+ ResolveRecoveryConflictWithBufferPin();
+ SetStartupBufferPinWaitBufId(-1);
+ }
+ else
+ ProcWaitForSignal();
+
PinCountWaitBuf = NULL;
/* Loop back and try again */
}
}
/*
+ * Check called from RecoveryConflictInterrupt handler when Startup
+ * process requests cancelation of all pin holders that are blocking it.
+ */
+bool
+HoldingBufferPinThatDelaysRecovery(void)
+{
+ int bufid = GetStartupBufferPinWaitBufId();
+
+ /*
+ * If we get woken slowly then it's possible that the Startup process
+ * was already woken by other backends before we got here. Also possible
+ * that we get here by multiple interrupts or interrupts at inappropriate
+ * times, so make sure we do nothing if the bufid is not set.
+ */
+ if (bufid < 0)
+ return false;
+
+ if (PrivateRefCount[bufid] > 0)
+ return true;
+
+ return false;
+}
+
+/*
* ConditionalLockBufferForCleanup - as above, but don't wait to get the lock
*
* We won't loop, but just check once to see if the pin count is OK. If
diff --git a/src/backend/storage/ipc/procarray.c b/src/backend/storage/ipc/procarray.c
index 1793783cab9..7cd57f31405 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -37,7 +37,7 @@
*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/storage/ipc/procarray.c,v 1.58 2010/01/21 00:53:58 sriggs Exp $
+ * $PostgreSQL: pgsql/src/backend/storage/ipc/procarray.c,v 1.59 2010/01/23 16:37:12 sriggs Exp $
*
*-------------------------------------------------------------------------
*/
@@ -1680,15 +1680,13 @@ GetCurrentVirtualXIDs(TransactionId limitXmin, bool excludeXmin0,
* latestCompletedXid since doing so would be a performance issue during
* normal running, so we check it essentially for free on the standby.
*
- * If dbOid is valid we skip backends attached to other databases. Some
- * callers choose to skipExistingConflicts.
+ * If dbOid is valid we skip backends attached to other databases.
*
* Be careful to *not* pfree the result from this function. We reuse
* this array sufficiently often that we use malloc for the result.
*/
VirtualTransactionId *
-GetConflictingVirtualXIDs(TransactionId limitXmin, Oid dbOid,
- bool skipExistingConflicts)
+GetConflictingVirtualXIDs(TransactionId limitXmin, Oid dbOid)
{
static VirtualTransactionId *vxids;
ProcArrayStruct *arrayP = procArray;
@@ -1727,9 +1725,6 @@ GetConflictingVirtualXIDs(TransactionId limitXmin, Oid dbOid,
if (proc->pid == 0)
continue;
- if (skipExistingConflicts && proc->recoveryConflictPending)
- continue;
-
if (!OidIsValid(dbOid) ||
proc->databaseId == dbOid)
{
@@ -1886,7 +1881,7 @@ CountDBBackends(Oid databaseid)
* CancelDBBackends --- cancel backends that are using specified database
*/
void
-CancelDBBackends(Oid databaseid)
+CancelDBBackends(Oid databaseid, ProcSignalReason sigmode, bool conflictPending)
{
ProcArrayStruct *arrayP = procArray;
int index;
@@ -1899,13 +1894,13 @@ CancelDBBackends(Oid databaseid)
{
volatile PGPROC *proc = arrayP->procs[index];
- if (proc->databaseId == databaseid)
+ if (databaseid == InvalidOid || proc->databaseId == databaseid)
{
VirtualTransactionId procvxid;
GET_VXID_FROM_PGPROC(procvxid, *proc);
- proc->recoveryConflictPending = true;
+ proc->recoveryConflictPending = conflictPending;
pid = proc->pid;
if (pid != 0)
{
@@ -1913,8 +1908,7 @@ CancelDBBackends(Oid databaseid)
* Kill the pid if it's still here. If not, that's what we wanted
* so ignore any errors.
*/
- (void) SendProcSignal(pid, PROCSIG_RECOVERY_CONFLICT_DATABASE,
- procvxid.backendId);
+ (void) SendProcSignal(pid, sigmode, procvxid.backendId);
}
}
}
diff --git a/src/backend/storage/ipc/procsignal.c b/src/backend/storage/ipc/procsignal.c
index 453d94aea5b..6c38d423f23 100644
--- a/src/backend/storage/ipc/procsignal.c
+++ b/src/backend/storage/ipc/procsignal.c
@@ -8,7 +8,7 @@
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/storage/ipc/procsignal.c,v 1.3 2010/01/16 10:05:50 sriggs Exp $
+ * $PostgreSQL: pgsql/src/backend/storage/ipc/procsignal.c,v 1.4 2010/01/23 16:37:12 sriggs Exp $
*
*-------------------------------------------------------------------------
*/
@@ -272,5 +272,8 @@ procsignal_sigusr1_handler(SIGNAL_ARGS)
if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_SNAPSHOT))
RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_SNAPSHOT);
+ if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_BUFFERPIN))
+ RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_BUFFERPIN);
+
errno = save_errno;
}
diff --git a/src/backend/storage/ipc/standby.c b/src/backend/storage/ipc/standby.c
index dcead94b27c..f079dba8dcf 100644
--- a/src/backend/storage/ipc/standby.c
+++ b/src/backend/storage/ipc/standby.c
@@ -11,7 +11,7 @@
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/storage/ipc/standby.c,v 1.6 2010/01/16 10:13:04 sriggs Exp $
+ * $PostgreSQL: pgsql/src/backend/storage/ipc/standby.c,v 1.7 2010/01/23 16:37:12 sriggs Exp $
*
*-------------------------------------------------------------------------
*/
@@ -126,10 +126,6 @@ WaitExceedsMaxStandbyDelay(void)
long delay_secs;
int delay_usecs;
- /* max_standby_delay = -1 means wait forever, if necessary */
- if (MaxStandbyDelay < 0)
- return false;
-
/* Are we past max_standby_delay? */
TimestampDifference(GetLatestXLogTime(), GetCurrentTimestamp(),
&delay_secs, &delay_usecs);
@@ -241,8 +237,7 @@ ResolveRecoveryConflictWithSnapshot(TransactionId latestRemovedXid)
VirtualTransactionId *backends;
backends = GetConflictingVirtualXIDs(latestRemovedXid,
- InvalidOid,
- true);
+ InvalidOid);
ResolveRecoveryConflictWithVirtualXIDs(backends,
PROCSIG_RECOVERY_CONFLICT_SNAPSHOT);
@@ -273,8 +268,7 @@ ResolveRecoveryConflictWithTablespace(Oid tsid)
* non-transactional.
*/
temp_file_users = GetConflictingVirtualXIDs(InvalidTransactionId,
- InvalidOid,
- false);
+ InvalidOid);
ResolveRecoveryConflictWithVirtualXIDs(temp_file_users,
PROCSIG_RECOVERY_CONFLICT_TABLESPACE);
}
@@ -295,7 +289,7 @@ ResolveRecoveryConflictWithDatabase(Oid dbid)
*/
while (CountDBBackends(dbid) > 0)
{
- CancelDBBackends(dbid);
+ CancelDBBackends(dbid, PROCSIG_RECOVERY_CONFLICT_TABLESPACE, true);
/*
* Wait awhile for them to die so that we avoid flooding an
@@ -331,8 +325,7 @@ ResolveRecoveryConflictWithLock(Oid dbOid, Oid relOid)
else
{
backends = GetConflictingVirtualXIDs(InvalidTransactionId,
- InvalidOid,
- true);
+ InvalidOid);
report_memory_error = true;
}
@@ -346,6 +339,113 @@ ResolveRecoveryConflictWithLock(Oid dbOid, Oid relOid)
}
/*
+ * ResolveRecoveryConflictWithBufferPin is called from LockBufferForCleanup()
+ * to resolve conflicts with other backends holding buffer pins.
+ *
+ * We either resolve conflicts immediately or set a SIGALRM to wake us at
+ * the limit of our patience. The sleep in LockBufferForCleanup() is
+ * performed here, for code clarity.
+ *
+ * Resolve conflict by sending a SIGUSR1 reason to all backends to check if
+ * they hold one of the buffer pins that is blocking Startup process. If so,
+ * backends will take an appropriate error action, ERROR or FATAL.
+ *
+ * A secondary purpose of this is to avoid deadlocks that might occur between
+ * the Startup process and lock waiters. Deadlocks occur because if queries
+ * wait on a lock, that must be behind an AccessExclusiveLock, which can only
+ * be clared if the Startup process replays a transaction completion record.
+ * If Startup process is waiting then that is a deadlock. If we allowed a
+ * setting of max_standby_delay that meant "wait forever" we would then need
+ * special code to protect against deadlock. Such deadlocks are rare, so the
+ * code would be almost certainly buggy, so we avoid both long waits and
+ * deadlocks using the same mechanism.
+ */
+void
+ResolveRecoveryConflictWithBufferPin(void)
+{
+ bool sig_alarm_enabled = false;
+
+ Assert(InHotStandby);
+
+ /*
+ * Signal immediately or set alarm for later.
+ */
+ if (MaxStandbyDelay == 0)
+ SendRecoveryConflictWithBufferPin();
+ else
+ {
+ TimestampTz now;
+ long standby_delay_secs; /* How far Startup process is lagging */
+ int standby_delay_usecs;
+
+ now = GetCurrentTimestamp();
+
+ /* Are we past max_standby_delay? */
+ TimestampDifference(GetLatestXLogTime(), now,
+ &standby_delay_secs, &standby_delay_usecs);
+
+ if (standby_delay_secs >= (long) MaxStandbyDelay)
+ SendRecoveryConflictWithBufferPin();
+ else
+ {
+ TimestampTz fin_time; /* Expected wake-up time by timer */
+ long timer_delay_secs; /* Amount of time we set timer for */
+ int timer_delay_usecs = 0;
+
+ /*
+ * How much longer we should wait?
+ */
+ timer_delay_secs = MaxStandbyDelay - standby_delay_secs;
+ if (standby_delay_usecs > 0)
+ {
+ timer_delay_secs -= 1;
+ timer_delay_usecs = 1000000 - standby_delay_usecs;
+ }
+
+ /*
+ * It's possible that the difference is less than a microsecond;
+ * ensure we don't cancel, rather than set, the interrupt.
+ */
+ if (timer_delay_secs == 0 && timer_delay_usecs == 0)
+ timer_delay_usecs = 1;
+
+ /*
+ * When is the finish time? We recheck this if we are woken early.
+ */
+ fin_time = TimestampTzPlusMilliseconds(now,
+ (timer_delay_secs * 1000) +
+ (timer_delay_usecs / 1000));
+
+ if (enable_standby_sig_alarm(timer_delay_secs, timer_delay_usecs, fin_time))
+ sig_alarm_enabled = true;
+ else
+ elog(FATAL, "could not set timer for process wakeup");
+ }
+ }
+
+ /* Wait to be signaled by UnpinBuffer() */
+ ProcWaitForSignal();
+
+ if (sig_alarm_enabled)
+ {
+ if (!disable_standby_sig_alarm())
+ elog(FATAL, "could not disable timer for process wakeup");
+ }
+}
+
+void
+SendRecoveryConflictWithBufferPin(void)
+{
+ /*
+ * We send signal to all backends to ask them if they are holding
+ * the buffer pin which is delaying the Startup process. We must
+ * not set the conflict flag yet, since most backends will be innocent.
+ * Let the SIGUSR1 handling in each backend decide their own fate.
+ */
+ CancelDBBackends(InvalidOid, PROCSIG_RECOVERY_CONFLICT_BUFFERPIN, false);
+}
+
+/*
* -----------------------------------------------------
* Locking in Recovery Mode
* -----------------------------------------------------
diff --git a/src/backend/storage/lmgr/lock.c b/src/backend/storage/lmgr/lock.c
index 74fbcbd18ec..ea781a8b29c 100644
--- a/src/backend/storage/lmgr/lock.c
+++ b/src/backend/storage/lmgr/lock.c
@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/storage/lmgr/lock.c,v 1.190 2010/01/02 16:57:52 momjian Exp $
+ * $PostgreSQL: pgsql/src/backend/storage/lmgr/lock.c,v 1.191 2010/01/23 16:37:12 sriggs Exp $
*
* NOTES
* A lock table is a shared memory hash table. When
@@ -815,25 +815,6 @@ LockAcquireExtended(const LOCKTAG *locktag,
}
/*
- * In Hot Standby we abort the lock wait if Startup process is waiting
- * since this would result in a deadlock. The deadlock occurs because
- * if we are waiting it must be behind an AccessExclusiveLock, which
- * can only clear when a transaction completion record is replayed.
- * If Startup process is waiting we never will clear that lock, so to
- * wait for it just causes a deadlock.
- */
- if (RecoveryInProgress() && !InRecovery &&
- locktag->locktag_type == LOCKTAG_RELATION)
- {
- LWLockRelease(partitionLock);
- ereport(ERROR,
- (errcode(ERRCODE_T_R_DEADLOCK_DETECTED),
- errmsg("possible deadlock detected"),
- errdetail("process conflicts with recovery - please resubmit query later"),
- errdetail_log("process conflicts with recovery")));
- }
-
- /*
* Set bitmask of locks this process already holds on this object.
*/
MyProc->heldLocks = proclock->holdMask;
diff --git a/src/backend/storage/lmgr/proc.c b/src/backend/storage/lmgr/proc.c
index ac891827ec1..da64e1953a3 100644
--- a/src/backend/storage/lmgr/proc.c
+++ b/src/backend/storage/lmgr/proc.c
@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/storage/lmgr/proc.c,v 1.213 2010/01/16 10:05:50 sriggs Exp $
+ * $PostgreSQL: pgsql/src/backend/storage/lmgr/proc.c,v 1.214 2010/01/23 16:37:12 sriggs Exp $
*
*-------------------------------------------------------------------------
*/
@@ -73,6 +73,7 @@ NON_EXEC_STATIC PGPROC *AuxiliaryProcs = NULL;
static LOCALLOCK *lockAwaited = NULL;
/* Mark these volatile because they can be changed by signal handler */
+static volatile bool standby_timeout_active = false;
static volatile bool statement_timeout_active = false;
static volatile bool deadlock_timeout_active = false;
static volatile DeadLockState deadlock_state = DS_NOT_YET_CHECKED;
@@ -89,6 +90,7 @@ static void RemoveProcFromArray(int code, Datum arg);
static void ProcKill(int code, Datum arg);
static void AuxiliaryProcKill(int code, Datum arg);
static bool CheckStatementTimeout(void);
+static bool CheckStandbyTimeout(void);
/*
@@ -107,6 +109,8 @@ ProcGlobalShmemSize(void)
size = add_size(size, mul_size(MaxBackends, sizeof(PGPROC)));
/* ProcStructLock */
size = add_size(size, sizeof(slock_t));
+ /* startupBufferPinWaitBufId */
+ size = add_size(size, sizeof(NBuffers));
return size;
}
@@ -487,11 +491,44 @@ PublishStartupProcessInformation(void)
procglobal->startupProc = MyProc;
procglobal->startupProcPid = MyProcPid;
+ procglobal->startupBufferPinWaitBufId = 0;
SpinLockRelease(ProcStructLock);
}
/*
+ * Used from bufgr to share the value of the buffer that Startup waits on,
+ * or to reset the value to "not waiting" (-1). This allows processing
+ * of recovery conflicts for buffer pins. Set is made before backends look
+ * at this value, so locking not required, especially since the set is
+ * an atomic integer set operation.
+ */
+void
+SetStartupBufferPinWaitBufId(int bufid)
+{
+ /* use volatile pointer to prevent code rearrangement */
+ volatile PROC_HDR *procglobal = ProcGlobal;
+
+ procglobal->startupBufferPinWaitBufId = bufid;
+}
+
+/*
+ * Used by backends when they receive a request to check for buffer pin waits.
+ */
+int
+GetStartupBufferPinWaitBufId(void)
+{
+ int bufid;
+
+ /* use volatile pointer to prevent code rearrangement */
+ volatile PROC_HDR *procglobal = ProcGlobal;
+
+ bufid = procglobal->startupBufferPinWaitBufId;
+
+ return bufid;
+}
+
+/*
* Check whether there are at least N free PGPROC objects.
*
* Note: this is designed on the assumption that N will generally be small.
@@ -1542,7 +1579,7 @@ CheckStatementTimeout(void)
/*
- * Signal handler for SIGALRM
+ * Signal handler for SIGALRM for normal user backends
*
* Process deadlock check and/or statement timeout check, as needed.
* To avoid various edge cases, we must be careful to do nothing
@@ -1565,3 +1602,112 @@ handle_sig_alarm(SIGNAL_ARGS)
errno = save_errno;
}
+
+/*
+ * Signal handler for SIGALRM in Startup process
+ *
+ * To avoid various edge cases, we must be careful to do nothing
+ * when there is nothing to be done. We also need to be able to
+ * reschedule the timer interrupt if called before end of statement.
+ */
+bool
+enable_standby_sig_alarm(long delay_s, int delay_us, TimestampTz fin_time)
+{
+ struct itimerval timeval;
+
+ Assert(delay_s >= 0 && delay_us >= 0);
+
+ statement_fin_time = fin_time;
+
+ standby_timeout_active = true;
+
+ MemSet(&timeval, 0, sizeof(struct itimerval));
+ timeval.it_value.tv_sec = delay_s;
+ timeval.it_value.tv_usec = delay_us;
+ if (setitimer(ITIMER_REAL, &timeval, NULL))
+ return false;
+ return true;
+}
+
+bool
+disable_standby_sig_alarm(void)
+{
+ /*
+ * Always disable the interrupt if it is active; this avoids being
+ * interrupted by the signal handler and thereby possibly getting
+ * confused.
+ *
+ * We will re-enable the interrupt if necessary in CheckStandbyTimeout.
+ */
+ if (standby_timeout_active)
+ {
+ struct itimerval timeval;
+
+ MemSet(&timeval, 0, sizeof(struct itimerval));
+ if (setitimer(ITIMER_REAL, &timeval, NULL))
+ {
+ standby_timeout_active = false;
+ return false;
+ }
+ }
+
+ standby_timeout_active = false;
+
+ return true;
+}
+
+/*
+ * CheckStandbyTimeout() runs unconditionally in the Startup process
+ * SIGALRM handler. Timers will only be set when InHotStandby.
+ * We simply ignore any signals unless the timer has been set.
+ */
+static bool
+CheckStandbyTimeout(void)
+{
+ TimestampTz now;
+
+ standby_timeout_active = false;
+
+ now = GetCurrentTimestamp();
+
+ if (now >= statement_fin_time)
+ SendRecoveryConflictWithBufferPin();
+ else
+ {
+ /* Not time yet, so (re)schedule the interrupt */
+ long secs;
+ int usecs;
+ struct itimerval timeval;
+
+ TimestampDifference(now, statement_fin_time,
+ &secs, &usecs);
+
+ /*
+ * It's possible that the difference is less than a microsecond;
+ * ensure we don't cancel, rather than set, the interrupt.
+ */
+ if (secs == 0 && usecs == 0)
+ usecs = 1;
+
+ standby_timeout_active = true;
+
+ MemSet(&timeval, 0, sizeof(struct itimerval));
+ timeval.it_value.tv_sec = secs;
+ timeval.it_value.tv_usec = usecs;
+ if (setitimer(ITIMER_REAL, &timeval, NULL))
+ return false;
+ }
+
+ return true;
+}
+
+void
+handle_standby_sig_alarm(SIGNAL_ARGS)
+{
+ int save_errno = errno;
+
+ if (standby_timeout_active)
+ (void) CheckStandbyTimeout();
+
+ errno = save_errno;
+}
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index e034a4e6501..86d9e3c9847 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/tcop/postgres.c,v 1.585 2010/01/21 09:30:36 sriggs Exp $
+ * $PostgreSQL: pgsql/src/backend/tcop/postgres.c,v 1.586 2010/01/23 16:37:12 sriggs Exp $
*
* NOTES
* this is the "main" module of the postgres backend and
@@ -2718,6 +2718,18 @@ RecoveryConflictInterrupt(ProcSignalReason reason)
{
switch (reason)
{
+ case PROCSIG_RECOVERY_CONFLICT_BUFFERPIN:
+ /*
+ * If we aren't blocking the Startup process there is
+ * nothing more to do.
+ */
+ if (!HoldingBufferPinThatDelaysRecovery())
+ return;
+
+ MyProc->recoveryConflictPending = true;
+
+ /* Intentional drop through to error handling */
+
case PROCSIG_RECOVERY_CONFLICT_LOCK:
case PROCSIG_RECOVERY_CONFLICT_TABLESPACE:
case PROCSIG_RECOVERY_CONFLICT_SNAPSHOT:
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 274030195fb..6846d13a9c7 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -10,7 +10,7 @@
* Written by Peter Eisentraut <peter_e@gmx.net>.
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/utils/misc/guc.c,v 1.533 2010/01/15 09:19:04 heikki Exp $
+ * $PostgreSQL: pgsql/src/backend/utils/misc/guc.c,v 1.534 2010/01/23 16:37:12 sriggs Exp $
*
*--------------------------------------------------------------------
*/
@@ -1383,7 +1383,7 @@ static struct config_int ConfigureNamesInt[] =
NULL
},
&MaxStandbyDelay,
- 30, -1, INT_MAX, NULL, NULL
+ 30, 0, INT_MAX, NULL, NULL
},
{
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index e8e2abae1ea..dc4376ee9a5 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $PostgreSQL: pgsql/src/include/storage/bufmgr.h,v 1.123 2010/01/02 16:58:08 momjian Exp $
+ * $PostgreSQL: pgsql/src/include/storage/bufmgr.h,v 1.124 2010/01/23 16:37:12 sriggs Exp $
*
*-------------------------------------------------------------------------
*/
@@ -198,6 +198,7 @@ extern void LockBuffer(Buffer buffer, int mode);
extern bool ConditionalLockBuffer(Buffer buffer);
extern void LockBufferForCleanup(Buffer buffer);
extern bool ConditionalLockBufferForCleanup(Buffer buffer);
+extern bool HoldingBufferPinThatDelaysRecovery(void);
extern void AbortBufferIO(void);
diff --git a/src/include/storage/proc.h b/src/include/storage/proc.h
index 50500903245..5a4421bfa5c 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $PostgreSQL: pgsql/src/include/storage/proc.h,v 1.118 2010/01/16 10:05:50 sriggs Exp $
+ * $PostgreSQL: pgsql/src/include/storage/proc.h,v 1.119 2010/01/23 16:37:12 sriggs Exp $
*
*-------------------------------------------------------------------------
*/
@@ -16,7 +16,7 @@
#include "storage/lock.h"
#include "storage/pg_sema.h"
-
+#include "utils/timestamp.h"
/*
* Each backend advertises up to PGPROC_MAX_CACHED_SUBXIDS TransactionIds
@@ -145,6 +145,8 @@ typedef struct PROC_HDR
/* The proc of the Startup process, since not in ProcArray */
PGPROC *startupProc;
int startupProcPid;
+ /* Buffer id of the buffer that Startup process waits for pin on */
+ int startupBufferPinWaitBufId;
} PROC_HDR;
/*
@@ -177,6 +179,8 @@ extern void InitProcessPhase2(void);
extern void InitAuxiliaryProcess(void);
extern void PublishStartupProcessInformation(void);
+extern void SetStartupBufferPinWaitBufId(int bufid);
+extern int GetStartupBufferPinWaitBufId(void);
extern bool HaveNFreeProcs(int n);
extern void ProcReleaseLocks(bool isCommit);
@@ -194,4 +198,8 @@ extern bool enable_sig_alarm(int delayms, bool is_statement_timeout);
extern bool disable_sig_alarm(bool is_statement_timeout);
extern void handle_sig_alarm(SIGNAL_ARGS);
+extern bool enable_standby_sig_alarm(long delay_s, int delay_us, TimestampTz fin_time);
+extern bool disable_standby_sig_alarm(void);
+extern void handle_standby_sig_alarm(SIGNAL_ARGS);
+
#endif /* PROC_H */
diff --git a/src/include/storage/procarray.h b/src/include/storage/procarray.h
index 42953b0843c..ed68be6f271 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $PostgreSQL: pgsql/src/include/storage/procarray.h,v 1.30 2010/01/16 10:05:56 sriggs Exp $
+ * $PostgreSQL: pgsql/src/include/storage/procarray.h,v 1.31 2010/01/23 16:37:12 sriggs Exp $
*
*-------------------------------------------------------------------------
*/
@@ -57,13 +57,12 @@ extern bool IsBackendPid(int pid);
extern VirtualTransactionId *GetCurrentVirtualXIDs(TransactionId limitXmin,
bool excludeXmin0, bool allDbs, int excludeVacuum,
int *nvxids);
-extern VirtualTransactionId *GetConflictingVirtualXIDs(TransactionId limitXmin,
- Oid dbOid, bool skipExistingConflicts);
+extern VirtualTransactionId *GetConflictingVirtualXIDs(TransactionId limitXmin, Oid dbOid);
extern pid_t CancelVirtualTransaction(VirtualTransactionId vxid, ProcSignalReason sigmode);
extern int CountActiveBackends(void);
extern int CountDBBackends(Oid databaseid);
-extern void CancelDBBackends(Oid databaseid);
+extern void CancelDBBackends(Oid databaseid, ProcSignalReason sigmode, bool conflictPending);
extern int CountUserBackends(Oid roleid);
extern bool CountOtherDBBackends(Oid databaseId,
int *nbackends, int *nprepared);
diff --git a/src/include/storage/procsignal.h b/src/include/storage/procsignal.h
index cbe0e24c672..2a67a62a922 100644
--- a/src/include/storage/procsignal.h
+++ b/src/include/storage/procsignal.h
@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $PostgreSQL: pgsql/src/include/storage/procsignal.h,v 1.3 2010/01/16 10:05:57 sriggs Exp $
+ * $PostgreSQL: pgsql/src/include/storage/procsignal.h,v 1.4 2010/01/23 16:37:12 sriggs Exp $
*
*-------------------------------------------------------------------------
*/
@@ -37,6 +37,7 @@ typedef enum
PROCSIG_RECOVERY_CONFLICT_TABLESPACE,
PROCSIG_RECOVERY_CONFLICT_LOCK,
PROCSIG_RECOVERY_CONFLICT_SNAPSHOT,
+ PROCSIG_RECOVERY_CONFLICT_BUFFERPIN,
NUM_PROCSIGNALS /* Must be last! */
} ProcSignalReason;
diff --git a/src/include/storage/standby.h b/src/include/storage/standby.h
index 8c982cffb63..e12798045ae 100644
--- a/src/include/storage/standby.h
+++ b/src/include/storage/standby.h
@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $PostgreSQL: pgsql/src/include/storage/standby.h,v 1.4 2010/01/16 10:05:57 sriggs Exp $
+ * $PostgreSQL: pgsql/src/include/storage/standby.h,v 1.5 2010/01/23 16:37:12 sriggs Exp $
*
*-------------------------------------------------------------------------
*/
@@ -19,12 +19,15 @@
extern int vacuum_defer_cleanup_age;
+extern void InitRecoveryTransactionEnvironment(void);
+extern void ShutdownRecoveryTransactionEnvironment(void);
+
extern void ResolveRecoveryConflictWithSnapshot(TransactionId latestRemovedXid);
extern void ResolveRecoveryConflictWithTablespace(Oid tsid);
extern void ResolveRecoveryConflictWithDatabase(Oid dbid);
-extern void InitRecoveryTransactionEnvironment(void);
-extern void ShutdownRecoveryTransactionEnvironment(void);
+extern void ResolveRecoveryConflictWithBufferPin(void);
+extern void SendRecoveryConflictWithBufferPin(void);
/*
* Standby Rmgr (RM_STANDBY_ID)