diff options
author | Simon Riggs <simon@2ndQuadrant.com> | 2010-01-23 16:37:12 +0000 |
---|---|---|
committer | Simon Riggs <simon@2ndQuadrant.com> | 2010-01-23 16:37:12 +0000 |
commit | 959ac58c04130d467fb05e63a3ceb8e2ded404c7 (patch) | |
tree | 314eeeea7c6c8afa7cbe35bfe5ecde04eff35f71 /src/backend/storage | |
parent | 4fa69e566cf1b836ae8aa9bee24ab0c556cfe94e (diff) | |
download | postgresql-959ac58c04130d467fb05e63a3ceb8e2ded404c7.tar.gz postgresql-959ac58c04130d467fb05e63a3ceb8e2ded404c7.zip |
In HS, Startup process sets SIGALRM when waiting for buffer pin. If
woken by alarm we send SIGUSR1 to all backends requesting that they
check to see if they are blocking Startup process. If so, they throw
ERROR/FATAL as for other conflict resolutions. Deadlock stop gap
removed. max_standby_delay = -1 option removed to prevent deadlock.
Diffstat (limited to 'src/backend/storage')
-rw-r--r-- | src/backend/storage/buffer/bufmgr.c | 40 | ||||
-rw-r--r-- | src/backend/storage/ipc/procarray.c | 20 | ||||
-rw-r--r-- | src/backend/storage/ipc/procsignal.c | 5 | ||||
-rw-r--r-- | src/backend/storage/ipc/standby.c | 124 | ||||
-rw-r--r-- | src/backend/storage/lmgr/lock.c | 21 | ||||
-rw-r--r-- | src/backend/storage/lmgr/proc.c | 150 |
6 files changed, 310 insertions, 50 deletions
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; +} |