mirror of
https://github.com/postgres/postgres.git
synced 2025-08-24 09:27:52 +03:00
Allow read only connections during recovery, known as Hot Standby.
Enabled by recovery_connections = on (default) and forcing archive recovery using a recovery.conf. Recovery processing now emulates the original transactions as they are replayed, providing full locking and MVCC behaviour for read only queries. Recovery must enter consistent state before connections are allowed, so there is a delay, typically short, before connections succeed. Replay of recovering transactions can conflict and in some cases deadlock with queries during recovery; these result in query cancellation after max_standby_delay seconds have expired. Infrastructure changes have minor effects on normal running, though introduce four new types of WAL record. New test mode "make standbycheck" allows regression tests of static command behaviour on a standby server while in recovery. Typical and extreme dynamic behaviours have been checked via code inspection and manual testing. Few port specific behaviours have been utilised, though primary testing has been on Linux only so far. This commit is the basic patch. Additional changes will follow in this release to enhance some aspects of behaviour, notably improved handling of conflicts, deadlock detection and query cancellation. Changes to VACUUM FULL are also required. Simon Riggs, with significant and lengthy review by Heikki Linnakangas, including streamlined redesign of snapshot creation and two-phase commit. Important contributions from Florian Pflug, Mark Kirkwood, Merlin Moncure, Greg Stark, Gianni Ciolli, Gabriele Bartolini, Hannu Krosing, Robert Haas, Tatsuo Ishii, Hiroyuki Yamada plus support and feedback from many other community members.
This commit is contained in:
@@ -1,7 +1,7 @@
|
||||
#
|
||||
# Makefile for storage/ipc
|
||||
#
|
||||
# $PostgreSQL: pgsql/src/backend/storage/ipc/Makefile,v 1.22 2009/07/31 20:26:23 tgl Exp $
|
||||
# $PostgreSQL: pgsql/src/backend/storage/ipc/Makefile,v 1.23 2009/12/19 01:32:35 sriggs Exp $
|
||||
#
|
||||
|
||||
subdir = src/backend/storage/ipc
|
||||
@@ -16,6 +16,6 @@ endif
|
||||
endif
|
||||
|
||||
OBJS = ipc.o ipci.o pmsignal.o procarray.o procsignal.o shmem.o shmqueue.o \
|
||||
sinval.o sinvaladt.o
|
||||
sinval.o sinvaladt.o standby.o
|
||||
|
||||
include $(top_srcdir)/src/backend/common.mk
|
||||
|
File diff suppressed because it is too large
Load Diff
@@ -8,7 +8,7 @@
|
||||
*
|
||||
*
|
||||
* IDENTIFICATION
|
||||
* $PostgreSQL: pgsql/src/backend/storage/ipc/sinvaladt.c,v 1.79 2009/07/31 20:26:23 tgl Exp $
|
||||
* $PostgreSQL: pgsql/src/backend/storage/ipc/sinvaladt.c,v 1.80 2009/12/19 01:32:35 sriggs Exp $
|
||||
*
|
||||
*-------------------------------------------------------------------------
|
||||
*/
|
||||
@@ -144,6 +144,13 @@ typedef struct ProcState
|
||||
bool resetState; /* backend needs to reset its state */
|
||||
bool signaled; /* backend has been sent catchup signal */
|
||||
|
||||
/*
|
||||
* Backend only sends invalidations, never receives them. This only makes sense
|
||||
* for Startup process during recovery because it doesn't maintain a relcache,
|
||||
* yet it fires inval messages to allow query backends to see schema changes.
|
||||
*/
|
||||
bool sendOnly; /* backend only sends, never receives */
|
||||
|
||||
/*
|
||||
* Next LocalTransactionId to use for each idle backend slot. We keep
|
||||
* this here because it is indexed by BackendId and it is convenient to
|
||||
@@ -249,7 +256,7 @@ CreateSharedInvalidationState(void)
|
||||
* Initialize a new backend to operate on the sinval buffer
|
||||
*/
|
||||
void
|
||||
SharedInvalBackendInit(void)
|
||||
SharedInvalBackendInit(bool sendOnly)
|
||||
{
|
||||
int index;
|
||||
ProcState *stateP = NULL;
|
||||
@@ -308,6 +315,7 @@ SharedInvalBackendInit(void)
|
||||
stateP->nextMsgNum = segP->maxMsgNum;
|
||||
stateP->resetState = false;
|
||||
stateP->signaled = false;
|
||||
stateP->sendOnly = sendOnly;
|
||||
|
||||
LWLockRelease(SInvalWriteLock);
|
||||
|
||||
@@ -579,7 +587,9 @@ SICleanupQueue(bool callerHasWriteLock, int minFree)
|
||||
/*
|
||||
* Recompute minMsgNum = minimum of all backends' nextMsgNum, identify the
|
||||
* furthest-back backend that needs signaling (if any), and reset any
|
||||
* backends that are too far back.
|
||||
* backends that are too far back. Note that because we ignore sendOnly
|
||||
* backends here it is possible for them to keep sending messages without
|
||||
* a problem even when they are the only active backend.
|
||||
*/
|
||||
min = segP->maxMsgNum;
|
||||
minsig = min - SIG_THRESHOLD;
|
||||
@@ -591,7 +601,7 @@ SICleanupQueue(bool callerHasWriteLock, int minFree)
|
||||
int n = stateP->nextMsgNum;
|
||||
|
||||
/* Ignore if inactive or already in reset state */
|
||||
if (stateP->procPid == 0 || stateP->resetState)
|
||||
if (stateP->procPid == 0 || stateP->resetState || stateP->sendOnly)
|
||||
continue;
|
||||
|
||||
/*
|
||||
|
717
src/backend/storage/ipc/standby.c
Normal file
717
src/backend/storage/ipc/standby.c
Normal file
@@ -0,0 +1,717 @@
|
||||
/*-------------------------------------------------------------------------
|
||||
*
|
||||
* standby.c
|
||||
* Misc functions used in Hot Standby mode.
|
||||
*
|
||||
* InitRecoveryTransactionEnvironment()
|
||||
* ShutdownRecoveryTransactionEnvironment()
|
||||
*
|
||||
* ResolveRecoveryConflictWithVirtualXIDs()
|
||||
*
|
||||
* All functions for handling RM_STANDBY_ID, which relate to
|
||||
* AccessExclusiveLocks and starting snapshots for Hot Standby mode.
|
||||
*
|
||||
* Portions Copyright (c) 1996-2009, PostgreSQL Global Development Group
|
||||
* Portions Copyright (c) 1994, Regents of the University of California
|
||||
*
|
||||
* IDENTIFICATION
|
||||
* $PostgreSQL: pgsql/src/backend/storage/ipc/standby.c,v 1.1 2009/12/19 01:32:35 sriggs Exp $
|
||||
*
|
||||
*-------------------------------------------------------------------------
|
||||
*/
|
||||
#include "postgres.h"
|
||||
#include "access/transam.h"
|
||||
#include "access/twophase.h"
|
||||
#include "access/xact.h"
|
||||
#include "access/xlog.h"
|
||||
#include "miscadmin.h"
|
||||
#include "pgstat.h"
|
||||
#include "storage/lmgr.h"
|
||||
#include "storage/proc.h"
|
||||
#include "storage/procarray.h"
|
||||
#include "storage/sinvaladt.h"
|
||||
#include "storage/standby.h"
|
||||
#include "utils/ps_status.h"
|
||||
|
||||
int vacuum_defer_cleanup_age;
|
||||
|
||||
static List *RecoveryLockList;
|
||||
|
||||
static void LogCurrentRunningXacts(RunningTransactions CurrRunningXacts);
|
||||
static void LogAccessExclusiveLocks(int nlocks, xl_standby_lock *locks);
|
||||
|
||||
/*
|
||||
* InitRecoveryTransactionEnvironment
|
||||
* Initiallize tracking of in-progress transactions in master
|
||||
*
|
||||
* We need to issue shared invalidations and hold locks. Holding locks
|
||||
* means others may want to wait on us, so we need to make lock table
|
||||
* inserts to appear like a transaction. We could create and delete
|
||||
* lock table entries for each transaction but its simpler just to create
|
||||
* one permanent entry and leave it there all the time. Locks are then
|
||||
* acquired and released as needed. Yes, this means you can see the
|
||||
* Startup process in pg_locks once we have run this.
|
||||
*/
|
||||
void
|
||||
InitRecoveryTransactionEnvironment(void)
|
||||
{
|
||||
VirtualTransactionId vxid;
|
||||
|
||||
/*
|
||||
* Initialise shared invalidation management for Startup process,
|
||||
* being careful to register ourselves as a sendOnly process so
|
||||
* we don't need to read messages, nor will we get signalled
|
||||
* when the queue starts filling up.
|
||||
*/
|
||||
SharedInvalBackendInit(true);
|
||||
|
||||
/*
|
||||
* Record the PID and PGPROC structure of the startup process.
|
||||
*/
|
||||
PublishStartupProcessInformation();
|
||||
|
||||
/*
|
||||
* Lock a virtual transaction id for Startup process.
|
||||
*
|
||||
* We need to do GetNextLocalTransactionId() because
|
||||
* SharedInvalBackendInit() leaves localTransactionid invalid and
|
||||
* the lock manager doesn't like that at all.
|
||||
*
|
||||
* Note that we don't need to run XactLockTableInsert() because nobody
|
||||
* needs to wait on xids. That sounds a little strange, but table locks
|
||||
* are held by vxids and row level locks are held by xids. All queries
|
||||
* hold AccessShareLocks so never block while we write or lock new rows.
|
||||
*/
|
||||
vxid.backendId = MyBackendId;
|
||||
vxid.localTransactionId = GetNextLocalTransactionId();
|
||||
VirtualXactLockTableInsert(vxid);
|
||||
|
||||
standbyState = STANDBY_INITIALIZED;
|
||||
}
|
||||
|
||||
/*
|
||||
* ShutdownRecoveryTransactionEnvironment
|
||||
* Shut down transaction tracking
|
||||
*
|
||||
* Prepare to switch from hot standby mode to normal operation. Shut down
|
||||
* recovery-time transaction tracking.
|
||||
*/
|
||||
void
|
||||
ShutdownRecoveryTransactionEnvironment(void)
|
||||
{
|
||||
/* Mark all tracked in-progress transactions as finished. */
|
||||
ExpireAllKnownAssignedTransactionIds();
|
||||
|
||||
/* Release all locks the tracked transactions were holding */
|
||||
StandbyReleaseAllLocks();
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* -----------------------------------------------------
|
||||
* Standby wait timers and backend cancel logic
|
||||
* -----------------------------------------------------
|
||||
*/
|
||||
|
||||
#define STANDBY_INITIAL_WAIT_US 1000
|
||||
static int standbyWait_us = STANDBY_INITIAL_WAIT_US;
|
||||
|
||||
/*
|
||||
* Standby wait logic for ResolveRecoveryConflictWithVirtualXIDs.
|
||||
* We wait here for a while then return. If we decide we can't wait any
|
||||
* more then we return true, if we can wait some more return false.
|
||||
*/
|
||||
static bool
|
||||
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);
|
||||
if (delay_secs > MaxStandbyDelay)
|
||||
return true;
|
||||
|
||||
/*
|
||||
* Sleep, then do bookkeeping.
|
||||
*/
|
||||
pg_usleep(standbyWait_us);
|
||||
|
||||
/*
|
||||
* Progressively increase the sleep times.
|
||||
*/
|
||||
standbyWait_us *= 2;
|
||||
if (standbyWait_us > 1000000)
|
||||
standbyWait_us = 1000000;
|
||||
if (standbyWait_us > MaxStandbyDelay * 1000000 / 4)
|
||||
standbyWait_us = MaxStandbyDelay * 1000000 / 4;
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
/*
|
||||
* This is the main executioner for any query backend that conflicts with
|
||||
* recovery processing. Judgement has already been passed on it within
|
||||
* a specific rmgr. Here we just issue the orders to the procs. The procs
|
||||
* then throw the required error as instructed.
|
||||
*
|
||||
* We may ask for a specific cancel_mode, typically ERROR or FATAL.
|
||||
*/
|
||||
void
|
||||
ResolveRecoveryConflictWithVirtualXIDs(VirtualTransactionId *waitlist,
|
||||
char *reason, int cancel_mode)
|
||||
{
|
||||
char waitactivitymsg[100];
|
||||
|
||||
Assert(cancel_mode > 0);
|
||||
|
||||
while (VirtualTransactionIdIsValid(*waitlist))
|
||||
{
|
||||
long wait_s;
|
||||
int wait_us; /* wait in microseconds (us) */
|
||||
TimestampTz waitStart;
|
||||
bool logged;
|
||||
|
||||
waitStart = GetCurrentTimestamp();
|
||||
standbyWait_us = STANDBY_INITIAL_WAIT_US;
|
||||
logged = false;
|
||||
|
||||
/* wait until the virtual xid is gone */
|
||||
while(!ConditionalVirtualXactLockTableWait(*waitlist))
|
||||
{
|
||||
/*
|
||||
* Report if we have been waiting for a while now...
|
||||
*/
|
||||
TimestampTz now = GetCurrentTimestamp();
|
||||
TimestampDifference(waitStart, now, &wait_s, &wait_us);
|
||||
if (!logged && (wait_s > 0 || wait_us > 500000))
|
||||
{
|
||||
const char *oldactivitymsg;
|
||||
int len;
|
||||
|
||||
oldactivitymsg = get_ps_display(&len);
|
||||
snprintf(waitactivitymsg, sizeof(waitactivitymsg),
|
||||
"waiting for max_standby_delay (%u ms)",
|
||||
MaxStandbyDelay);
|
||||
set_ps_display(waitactivitymsg, false);
|
||||
if (len > 100)
|
||||
len = 100;
|
||||
memcpy(waitactivitymsg, oldactivitymsg, len);
|
||||
|
||||
ereport(trace_recovery(DEBUG5),
|
||||
(errmsg("virtual transaction %u/%u is blocking %s",
|
||||
waitlist->backendId,
|
||||
waitlist->localTransactionId,
|
||||
reason)));
|
||||
|
||||
pgstat_report_waiting(true);
|
||||
|
||||
logged = true;
|
||||
}
|
||||
|
||||
/* Is it time to kill it? */
|
||||
if (WaitExceedsMaxStandbyDelay())
|
||||
{
|
||||
pid_t pid;
|
||||
|
||||
/*
|
||||
* Now find out who to throw out of the balloon.
|
||||
*/
|
||||
Assert(VirtualTransactionIdIsValid(*waitlist));
|
||||
pid = CancelVirtualTransaction(*waitlist, cancel_mode);
|
||||
|
||||
if (pid != 0)
|
||||
{
|
||||
/*
|
||||
* Startup process debug messages
|
||||
*/
|
||||
switch (cancel_mode)
|
||||
{
|
||||
case CONFLICT_MODE_FATAL:
|
||||
elog(trace_recovery(DEBUG1),
|
||||
"recovery disconnects session with pid %d because of conflict with %s",
|
||||
pid,
|
||||
reason);
|
||||
break;
|
||||
case CONFLICT_MODE_ERROR:
|
||||
elog(trace_recovery(DEBUG1),
|
||||
"recovery cancels virtual transaction %u/%u pid %d because of conflict with %s",
|
||||
waitlist->backendId,
|
||||
waitlist->localTransactionId,
|
||||
pid,
|
||||
reason);
|
||||
break;
|
||||
default:
|
||||
/* No conflict pending, so fall through */
|
||||
break;
|
||||
}
|
||||
|
||||
/*
|
||||
* Wait awhile for it to die so that we avoid flooding an
|
||||
* unresponsive backend when system is heavily loaded.
|
||||
*/
|
||||
pg_usleep(5000);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/* Reset ps display */
|
||||
if (logged)
|
||||
{
|
||||
set_ps_display(waitactivitymsg, false);
|
||||
pgstat_report_waiting(false);
|
||||
}
|
||||
|
||||
/* The virtual transaction is gone now, wait for the next one */
|
||||
waitlist++;
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* -----------------------------------------------------
|
||||
* Locking in Recovery Mode
|
||||
* -----------------------------------------------------
|
||||
*
|
||||
* All locks are held by the Startup process using a single virtual
|
||||
* transaction. This implementation is both simpler and in some senses,
|
||||
* more correct. The locks held mean "some original transaction held
|
||||
* this lock, so query access is not allowed at this time". So the Startup
|
||||
* process is the proxy by which the original locks are implemented.
|
||||
*
|
||||
* We only keep track of AccessExclusiveLocks, which are only ever held by
|
||||
* one transaction on one relation, and don't worry about lock queuing.
|
||||
*
|
||||
* We keep a single dynamically expandible list of locks in local memory,
|
||||
* RelationLockList, so we can keep track of the various entried made by
|
||||
* the Startup process's virtual xid in the shared lock table.
|
||||
*
|
||||
* List elements use type xl_rel_lock, since the WAL record type exactly
|
||||
* matches the information that we need to keep track of.
|
||||
*
|
||||
* We use session locks rather than normal locks so we don't need
|
||||
* ResourceOwners.
|
||||
*/
|
||||
|
||||
|
||||
void
|
||||
StandbyAcquireAccessExclusiveLock(TransactionId xid, Oid dbOid, Oid relOid)
|
||||
{
|
||||
xl_standby_lock *newlock;
|
||||
LOCKTAG locktag;
|
||||
bool report_memory_error = false;
|
||||
int num_attempts = 0;
|
||||
|
||||
/* Already processed? */
|
||||
if (TransactionIdDidCommit(xid) || TransactionIdDidAbort(xid))
|
||||
return;
|
||||
|
||||
elog(trace_recovery(DEBUG4),
|
||||
"adding recovery lock: db %d rel %d", dbOid, relOid);
|
||||
|
||||
/* dbOid is InvalidOid when we are locking a shared relation. */
|
||||
Assert(OidIsValid(relOid));
|
||||
|
||||
newlock = palloc(sizeof(xl_standby_lock));
|
||||
newlock->xid = xid;
|
||||
newlock->dbOid = dbOid;
|
||||
newlock->relOid = relOid;
|
||||
RecoveryLockList = lappend(RecoveryLockList, newlock);
|
||||
|
||||
/*
|
||||
* Attempt to acquire the lock as requested.
|
||||
*/
|
||||
SET_LOCKTAG_RELATION(locktag, newlock->dbOid, newlock->relOid);
|
||||
|
||||
/*
|
||||
* Wait for lock to clear or kill anyone in our way.
|
||||
*/
|
||||
while (LockAcquireExtended(&locktag, AccessExclusiveLock,
|
||||
true, true, report_memory_error)
|
||||
== LOCKACQUIRE_NOT_AVAIL)
|
||||
{
|
||||
VirtualTransactionId *backends;
|
||||
|
||||
/*
|
||||
* If blowing away everybody with conflicting locks doesn't work,
|
||||
* after the first two attempts then we just start blowing everybody
|
||||
* away until it does work. We do this because its likely that we
|
||||
* either have too many locks and we just can't get one at all,
|
||||
* or that there are many people crowding for the same table.
|
||||
* Recovery must win; the end justifies the means.
|
||||
*/
|
||||
if (++num_attempts < 3)
|
||||
backends = GetLockConflicts(&locktag, AccessExclusiveLock);
|
||||
else
|
||||
{
|
||||
backends = GetConflictingVirtualXIDs(InvalidTransactionId,
|
||||
InvalidOid,
|
||||
true);
|
||||
report_memory_error = true;
|
||||
}
|
||||
|
||||
ResolveRecoveryConflictWithVirtualXIDs(backends,
|
||||
"exclusive lock",
|
||||
CONFLICT_MODE_ERROR);
|
||||
}
|
||||
}
|
||||
|
||||
static void
|
||||
StandbyReleaseLocks(TransactionId xid)
|
||||
{
|
||||
ListCell *cell,
|
||||
*prev,
|
||||
*next;
|
||||
|
||||
/*
|
||||
* Release all matching locks and remove them from list
|
||||
*/
|
||||
prev = NULL;
|
||||
for (cell = list_head(RecoveryLockList); cell; cell = next)
|
||||
{
|
||||
xl_standby_lock *lock = (xl_standby_lock *) lfirst(cell);
|
||||
next = lnext(cell);
|
||||
|
||||
if (!TransactionIdIsValid(xid) || lock->xid == xid)
|
||||
{
|
||||
LOCKTAG locktag;
|
||||
|
||||
elog(trace_recovery(DEBUG4),
|
||||
"releasing recovery lock: xid %u db %d rel %d",
|
||||
lock->xid, lock->dbOid, lock->relOid);
|
||||
SET_LOCKTAG_RELATION(locktag, lock->dbOid, lock->relOid);
|
||||
if (!LockRelease(&locktag, AccessExclusiveLock, true))
|
||||
elog(trace_recovery(LOG),
|
||||
"RecoveryLockList contains entry for lock "
|
||||
"no longer recorded by lock manager "
|
||||
"xid %u database %d relation %d",
|
||||
lock->xid, lock->dbOid, lock->relOid);
|
||||
|
||||
RecoveryLockList = list_delete_cell(RecoveryLockList, cell, prev);
|
||||
pfree(lock);
|
||||
}
|
||||
else
|
||||
prev = cell;
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* Release locks for a transaction tree, starting at xid down, from
|
||||
* RecoveryLockList.
|
||||
*
|
||||
* Called during WAL replay of COMMIT/ROLLBACK when in hot standby mode,
|
||||
* to remove any AccessExclusiveLocks requested by a transaction.
|
||||
*/
|
||||
void
|
||||
StandbyReleaseLockTree(TransactionId xid, int nsubxids, TransactionId *subxids)
|
||||
{
|
||||
int i;
|
||||
|
||||
StandbyReleaseLocks(xid);
|
||||
|
||||
for (i = 0; i < nsubxids; i++)
|
||||
StandbyReleaseLocks(subxids[i]);
|
||||
}
|
||||
|
||||
/*
|
||||
* StandbyReleaseOldLocks
|
||||
* Release standby locks held by XIDs < removeXid
|
||||
* In some cases, keep prepared transactions.
|
||||
*/
|
||||
static void
|
||||
StandbyReleaseLocksMany(TransactionId removeXid, bool keepPreparedXacts)
|
||||
{
|
||||
ListCell *cell,
|
||||
*prev,
|
||||
*next;
|
||||
LOCKTAG locktag;
|
||||
|
||||
/*
|
||||
* Release all matching locks.
|
||||
*/
|
||||
prev = NULL;
|
||||
for (cell = list_head(RecoveryLockList); cell; cell = next)
|
||||
{
|
||||
xl_standby_lock *lock = (xl_standby_lock *) lfirst(cell);
|
||||
next = lnext(cell);
|
||||
|
||||
if (!TransactionIdIsValid(removeXid) || TransactionIdPrecedes(lock->xid, removeXid))
|
||||
{
|
||||
if (keepPreparedXacts && StandbyTransactionIdIsPrepared(lock->xid))
|
||||
continue;
|
||||
elog(trace_recovery(DEBUG4),
|
||||
"releasing recovery lock: xid %u db %d rel %d",
|
||||
lock->xid, lock->dbOid, lock->relOid);
|
||||
SET_LOCKTAG_RELATION(locktag, lock->dbOid, lock->relOid);
|
||||
if (!LockRelease(&locktag, AccessExclusiveLock, true))
|
||||
elog(trace_recovery(LOG),
|
||||
"RecoveryLockList contains entry for lock "
|
||||
"no longer recorded by lock manager "
|
||||
"xid %u database %d relation %d",
|
||||
lock->xid, lock->dbOid, lock->relOid);
|
||||
RecoveryLockList = list_delete_cell(RecoveryLockList, cell, prev);
|
||||
pfree(lock);
|
||||
}
|
||||
else
|
||||
prev = cell;
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* Called at end of recovery and when we see a shutdown checkpoint.
|
||||
*/
|
||||
void
|
||||
StandbyReleaseAllLocks(void)
|
||||
{
|
||||
elog(trace_recovery(DEBUG2), "release all standby locks");
|
||||
StandbyReleaseLocksMany(InvalidTransactionId, false);
|
||||
}
|
||||
|
||||
/*
|
||||
* StandbyReleaseOldLocks
|
||||
* Release standby locks held by XIDs < removeXid, as long
|
||||
* as their not prepared transactions.
|
||||
*/
|
||||
void
|
||||
StandbyReleaseOldLocks(TransactionId removeXid)
|
||||
{
|
||||
StandbyReleaseLocksMany(removeXid, true);
|
||||
}
|
||||
|
||||
/*
|
||||
* --------------------------------------------------------------------
|
||||
* Recovery handling for Rmgr RM_STANDBY_ID
|
||||
*
|
||||
* These record types will only be created if XLogStandbyInfoActive()
|
||||
* --------------------------------------------------------------------
|
||||
*/
|
||||
|
||||
void
|
||||
standby_redo(XLogRecPtr lsn, XLogRecord *record)
|
||||
{
|
||||
uint8 info = record->xl_info & ~XLR_INFO_MASK;
|
||||
|
||||
/* Do nothing if we're not in standby mode */
|
||||
if (standbyState == STANDBY_DISABLED)
|
||||
return;
|
||||
|
||||
if (info == XLOG_STANDBY_LOCK)
|
||||
{
|
||||
xl_standby_locks *xlrec = (xl_standby_locks *) XLogRecGetData(record);
|
||||
int i;
|
||||
|
||||
for (i = 0; i < xlrec->nlocks; i++)
|
||||
StandbyAcquireAccessExclusiveLock(xlrec->locks[i].xid,
|
||||
xlrec->locks[i].dbOid,
|
||||
xlrec->locks[i].relOid);
|
||||
}
|
||||
else if (info == XLOG_RUNNING_XACTS)
|
||||
{
|
||||
xl_running_xacts *xlrec = (xl_running_xacts *) XLogRecGetData(record);
|
||||
RunningTransactionsData running;
|
||||
|
||||
running.xcnt = xlrec->xcnt;
|
||||
running.subxid_overflow = xlrec->subxid_overflow;
|
||||
running.nextXid = xlrec->nextXid;
|
||||
running.oldestRunningXid = xlrec->oldestRunningXid;
|
||||
running.xids = xlrec->xids;
|
||||
|
||||
ProcArrayApplyRecoveryInfo(&running);
|
||||
}
|
||||
else
|
||||
elog(PANIC, "relation_redo: unknown op code %u", info);
|
||||
}
|
||||
|
||||
static void
|
||||
standby_desc_running_xacts(StringInfo buf, xl_running_xacts *xlrec)
|
||||
{
|
||||
int i;
|
||||
|
||||
appendStringInfo(buf,
|
||||
" nextXid %u oldestRunningXid %u",
|
||||
xlrec->nextXid,
|
||||
xlrec->oldestRunningXid);
|
||||
if (xlrec->xcnt > 0)
|
||||
{
|
||||
appendStringInfo(buf, "; %d xacts:", xlrec->xcnt);
|
||||
for (i = 0; i < xlrec->xcnt; i++)
|
||||
appendStringInfo(buf, " %u", xlrec->xids[i]);
|
||||
}
|
||||
|
||||
if (xlrec->subxid_overflow)
|
||||
appendStringInfo(buf, "; subxid ovf");
|
||||
}
|
||||
|
||||
void
|
||||
standby_desc(StringInfo buf, uint8 xl_info, char *rec)
|
||||
{
|
||||
uint8 info = xl_info & ~XLR_INFO_MASK;
|
||||
|
||||
if (info == XLOG_STANDBY_LOCK)
|
||||
{
|
||||
xl_standby_locks *xlrec = (xl_standby_locks *) rec;
|
||||
int i;
|
||||
|
||||
appendStringInfo(buf, "AccessExclusive locks:");
|
||||
|
||||
for (i = 0; i < xlrec->nlocks; i++)
|
||||
appendStringInfo(buf, " xid %u db %d rel %d",
|
||||
xlrec->locks[i].xid, xlrec->locks[i].dbOid,
|
||||
xlrec->locks[i].relOid);
|
||||
}
|
||||
else if (info == XLOG_RUNNING_XACTS)
|
||||
{
|
||||
xl_running_xacts *xlrec = (xl_running_xacts *) rec;
|
||||
|
||||
appendStringInfo(buf, " running xacts:");
|
||||
standby_desc_running_xacts(buf, xlrec);
|
||||
}
|
||||
else
|
||||
appendStringInfo(buf, "UNKNOWN");
|
||||
}
|
||||
|
||||
/*
|
||||
* Log details of the current snapshot to WAL. This allows the snapshot state
|
||||
* to be reconstructed on the standby.
|
||||
*/
|
||||
void
|
||||
LogStandbySnapshot(TransactionId *oldestActiveXid, TransactionId *nextXid)
|
||||
{
|
||||
RunningTransactions running;
|
||||
xl_standby_lock *locks;
|
||||
int nlocks;
|
||||
|
||||
Assert(XLogStandbyInfoActive());
|
||||
|
||||
/*
|
||||
* Get details of any AccessExclusiveLocks being held at the moment.
|
||||
*/
|
||||
locks = GetRunningTransactionLocks(&nlocks);
|
||||
if (nlocks > 0)
|
||||
LogAccessExclusiveLocks(nlocks, locks);
|
||||
|
||||
/*
|
||||
* Log details of all in-progress transactions. This should be the last
|
||||
* record we write, because standby will open up when it sees this.
|
||||
*/
|
||||
running = GetRunningTransactionData();
|
||||
LogCurrentRunningXacts(running);
|
||||
|
||||
*oldestActiveXid = running->oldestRunningXid;
|
||||
*nextXid = running->nextXid;
|
||||
}
|
||||
|
||||
/*
|
||||
* Record an enhanced snapshot of running transactions into WAL.
|
||||
*
|
||||
* The definitions of RunningTransactionData and xl_xact_running_xacts
|
||||
* are similar. We keep them separate because xl_xact_running_xacts
|
||||
* is a contiguous chunk of memory and never exists fully until it is
|
||||
* assembled in WAL.
|
||||
*/
|
||||
static void
|
||||
LogCurrentRunningXacts(RunningTransactions CurrRunningXacts)
|
||||
{
|
||||
xl_running_xacts xlrec;
|
||||
XLogRecData rdata[2];
|
||||
int lastrdata = 0;
|
||||
XLogRecPtr recptr;
|
||||
|
||||
xlrec.xcnt = CurrRunningXacts->xcnt;
|
||||
xlrec.subxid_overflow = CurrRunningXacts->subxid_overflow;
|
||||
xlrec.nextXid = CurrRunningXacts->nextXid;
|
||||
xlrec.oldestRunningXid = CurrRunningXacts->oldestRunningXid;
|
||||
|
||||
/* Header */
|
||||
rdata[0].data = (char *) (&xlrec);
|
||||
rdata[0].len = MinSizeOfXactRunningXacts;
|
||||
rdata[0].buffer = InvalidBuffer;
|
||||
|
||||
/* array of TransactionIds */
|
||||
if (xlrec.xcnt > 0)
|
||||
{
|
||||
rdata[0].next = &(rdata[1]);
|
||||
rdata[1].data = (char *) CurrRunningXacts->xids;
|
||||
rdata[1].len = xlrec.xcnt * sizeof(TransactionId);
|
||||
rdata[1].buffer = InvalidBuffer;
|
||||
lastrdata = 1;
|
||||
}
|
||||
|
||||
rdata[lastrdata].next = NULL;
|
||||
|
||||
recptr = XLogInsert(RM_STANDBY_ID, XLOG_RUNNING_XACTS, rdata);
|
||||
|
||||
if (CurrRunningXacts->subxid_overflow)
|
||||
ereport(trace_recovery(DEBUG2),
|
||||
(errmsg("snapshot of %u running transactions overflowed (lsn %X/%X oldest xid %u next xid %u)",
|
||||
CurrRunningXacts->xcnt,
|
||||
recptr.xlogid, recptr.xrecoff,
|
||||
CurrRunningXacts->oldestRunningXid,
|
||||
CurrRunningXacts->nextXid)));
|
||||
else
|
||||
ereport(trace_recovery(DEBUG2),
|
||||
(errmsg("snapshot of %u running transaction ids (lsn %X/%X oldest xid %u next xid %u)",
|
||||
CurrRunningXacts->xcnt,
|
||||
recptr.xlogid, recptr.xrecoff,
|
||||
CurrRunningXacts->oldestRunningXid,
|
||||
CurrRunningXacts->nextXid)));
|
||||
|
||||
}
|
||||
|
||||
/*
|
||||
* Wholesale logging of AccessExclusiveLocks. Other lock types need not be
|
||||
* logged, as described in backend/storage/lmgr/README.
|
||||
*/
|
||||
static void
|
||||
LogAccessExclusiveLocks(int nlocks, xl_standby_lock *locks)
|
||||
{
|
||||
XLogRecData rdata[2];
|
||||
xl_standby_locks xlrec;
|
||||
|
||||
xlrec.nlocks = nlocks;
|
||||
|
||||
rdata[0].data = (char *) &xlrec;
|
||||
rdata[0].len = offsetof(xl_standby_locks, locks);
|
||||
rdata[0].buffer = InvalidBuffer;
|
||||
rdata[0].next = &rdata[1];
|
||||
|
||||
rdata[1].data = (char *) locks;
|
||||
rdata[1].len = nlocks * sizeof(xl_standby_lock);
|
||||
rdata[1].buffer = InvalidBuffer;
|
||||
rdata[1].next = NULL;
|
||||
|
||||
(void) XLogInsert(RM_STANDBY_ID, XLOG_STANDBY_LOCK, rdata);
|
||||
}
|
||||
|
||||
/*
|
||||
* Individual logging of AccessExclusiveLocks for use during LockAcquire()
|
||||
*/
|
||||
void
|
||||
LogAccessExclusiveLock(Oid dbOid, Oid relOid)
|
||||
{
|
||||
xl_standby_lock xlrec;
|
||||
|
||||
/*
|
||||
* Ensure that a TransactionId has been assigned to this transaction.
|
||||
* We don't actually need the xid yet but if we don't do this then
|
||||
* RecordTransactionCommit() and RecordTransactionAbort() will optimise
|
||||
* away the transaction completion record which recovery relies upon to
|
||||
* release locks. It's a hack, but for a corner case not worth adding
|
||||
* code for into the main commit path.
|
||||
*/
|
||||
xlrec.xid = GetTopTransactionId();
|
||||
|
||||
/*
|
||||
* Decode the locktag back to the original values, to avoid
|
||||
* sending lots of empty bytes with every message. See
|
||||
* lock.h to check how a locktag is defined for LOCKTAG_RELATION
|
||||
*/
|
||||
xlrec.dbOid = dbOid;
|
||||
xlrec.relOid = relOid;
|
||||
|
||||
LogAccessExclusiveLocks(1, &xlrec);
|
||||
}
|
@@ -1,4 +1,4 @@
|
||||
$PostgreSQL: pgsql/src/backend/storage/lmgr/README,v 1.24 2008/03/21 13:23:28 momjian Exp $
|
||||
$PostgreSQL: pgsql/src/backend/storage/lmgr/README,v 1.25 2009/12/19 01:32:35 sriggs Exp $
|
||||
|
||||
Locking Overview
|
||||
================
|
||||
@@ -517,3 +517,27 @@ interfere with each other.
|
||||
User locks are always held as session locks, so that they are not released at
|
||||
transaction end. They must be released explicitly by the application --- but
|
||||
they are released automatically when a backend terminates.
|
||||
|
||||
Locking during Hot Standby
|
||||
--------------------------
|
||||
|
||||
The Startup process is the only backend that can make changes during
|
||||
recovery, all other backends are read only. As a result the Startup
|
||||
process does not acquire locks on relations or objects except when the lock
|
||||
level is AccessExclusiveLock.
|
||||
|
||||
Regular backends are only allowed to take locks on relations or objects
|
||||
at RowExclusiveLock or lower. This ensures that they do not conflict with
|
||||
each other or with the Startup process, unless AccessExclusiveLocks are
|
||||
requested by one of the backends.
|
||||
|
||||
Deadlocks involving AccessExclusiveLocks are not possible, so we need
|
||||
not be concerned that a user initiated deadlock can prevent recovery from
|
||||
progressing.
|
||||
|
||||
AccessExclusiveLocks on the primary or master node generate WAL records
|
||||
that are then applied by the Startup process. Locks are released at end
|
||||
of transaction just as they are in normal processing. These locks are
|
||||
held by the Startup process, acting as a proxy for the backends that
|
||||
originally acquired these locks. Again, these locks cannot conflict with
|
||||
one another, so the Startup process cannot deadlock itself either.
|
||||
|
@@ -8,7 +8,7 @@
|
||||
*
|
||||
*
|
||||
* IDENTIFICATION
|
||||
* $PostgreSQL: pgsql/src/backend/storage/lmgr/lock.c,v 1.188 2009/06/11 14:49:02 momjian Exp $
|
||||
* $PostgreSQL: pgsql/src/backend/storage/lmgr/lock.c,v 1.189 2009/12/19 01:32:35 sriggs Exp $
|
||||
*
|
||||
* NOTES
|
||||
* A lock table is a shared memory hash table. When
|
||||
@@ -38,6 +38,7 @@
|
||||
#include "miscadmin.h"
|
||||
#include "pg_trace.h"
|
||||
#include "pgstat.h"
|
||||
#include "storage/standby.h"
|
||||
#include "utils/memutils.h"
|
||||
#include "utils/ps_status.h"
|
||||
#include "utils/resowner.h"
|
||||
@@ -468,6 +469,25 @@ LockAcquire(const LOCKTAG *locktag,
|
||||
LOCKMODE lockmode,
|
||||
bool sessionLock,
|
||||
bool dontWait)
|
||||
{
|
||||
return LockAcquireExtended(locktag, lockmode, sessionLock, dontWait, true);
|
||||
}
|
||||
|
||||
/*
|
||||
* LockAcquireExtended - allows us to specify additional options
|
||||
*
|
||||
* reportMemoryError specifies whether a lock request that fills the
|
||||
* lock table should generate an ERROR or not. This allows a priority
|
||||
* caller to note that the lock table is full and then begin taking
|
||||
* extreme action to reduce the number of other lock holders before
|
||||
* retrying the action.
|
||||
*/
|
||||
LockAcquireResult
|
||||
LockAcquireExtended(const LOCKTAG *locktag,
|
||||
LOCKMODE lockmode,
|
||||
bool sessionLock,
|
||||
bool dontWait,
|
||||
bool reportMemoryError)
|
||||
{
|
||||
LOCKMETHODID lockmethodid = locktag->locktag_lockmethodid;
|
||||
LockMethod lockMethodTable;
|
||||
@@ -490,6 +510,16 @@ LockAcquire(const LOCKTAG *locktag,
|
||||
if (lockmode <= 0 || lockmode > lockMethodTable->numLockModes)
|
||||
elog(ERROR, "unrecognized lock mode: %d", lockmode);
|
||||
|
||||
if (RecoveryInProgress() && !InRecovery &&
|
||||
(locktag->locktag_type == LOCKTAG_OBJECT ||
|
||||
locktag->locktag_type == LOCKTAG_RELATION ) &&
|
||||
lockmode > RowExclusiveLock)
|
||||
ereport(ERROR,
|
||||
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||
errmsg("cannot acquire lockmode %s on database objects while recovery is in progress",
|
||||
lockMethodTable->lockModeNames[lockmode]),
|
||||
errhint("Only RowExclusiveLock or less can be acquired on database objects during recovery.")));
|
||||
|
||||
#ifdef LOCK_DEBUG
|
||||
if (LOCK_DEBUG_ENABLED(locktag))
|
||||
elog(LOG, "LockAcquire: lock [%u,%u] %s",
|
||||
@@ -578,10 +608,13 @@ LockAcquire(const LOCKTAG *locktag,
|
||||
if (!lock)
|
||||
{
|
||||
LWLockRelease(partitionLock);
|
||||
ereport(ERROR,
|
||||
(errcode(ERRCODE_OUT_OF_MEMORY),
|
||||
errmsg("out of shared memory"),
|
||||
errhint("You might need to increase max_locks_per_transaction.")));
|
||||
if (reportMemoryError)
|
||||
ereport(ERROR,
|
||||
(errcode(ERRCODE_OUT_OF_MEMORY),
|
||||
errmsg("out of shared memory"),
|
||||
errhint("You might need to increase max_locks_per_transaction.")));
|
||||
else
|
||||
return LOCKACQUIRE_NOT_AVAIL;
|
||||
}
|
||||
locallock->lock = lock;
|
||||
|
||||
@@ -644,10 +677,13 @@ LockAcquire(const LOCKTAG *locktag,
|
||||
elog(PANIC, "lock table corrupted");
|
||||
}
|
||||
LWLockRelease(partitionLock);
|
||||
ereport(ERROR,
|
||||
(errcode(ERRCODE_OUT_OF_MEMORY),
|
||||
errmsg("out of shared memory"),
|
||||
errhint("You might need to increase max_locks_per_transaction.")));
|
||||
if (reportMemoryError)
|
||||
ereport(ERROR,
|
||||
(errcode(ERRCODE_OUT_OF_MEMORY),
|
||||
errmsg("out of shared memory"),
|
||||
errhint("You might need to increase max_locks_per_transaction.")));
|
||||
else
|
||||
return LOCKACQUIRE_NOT_AVAIL;
|
||||
}
|
||||
locallock->proclock = proclock;
|
||||
|
||||
@@ -778,6 +814,25 @@ LockAcquire(const LOCKTAG *locktag,
|
||||
return LOCKACQUIRE_NOT_AVAIL;
|
||||
}
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
@@ -827,6 +882,27 @@ LockAcquire(const LOCKTAG *locktag,
|
||||
|
||||
LWLockRelease(partitionLock);
|
||||
|
||||
/*
|
||||
* Emit a WAL record if acquisition of this lock need to be replayed in
|
||||
* a standby server. Only AccessExclusiveLocks can conflict with lock
|
||||
* types that read-only transactions can acquire in a standby server.
|
||||
*
|
||||
* Make sure this definition matches the one GetRunningTransactionLocks().
|
||||
*/
|
||||
if (lockmode >= AccessExclusiveLock &&
|
||||
locktag->locktag_type == LOCKTAG_RELATION &&
|
||||
!RecoveryInProgress() &&
|
||||
XLogStandbyInfoActive())
|
||||
{
|
||||
/*
|
||||
* Decode the locktag back to the original values, to avoid
|
||||
* sending lots of empty bytes with every message. See
|
||||
* lock.h to check how a locktag is defined for LOCKTAG_RELATION
|
||||
*/
|
||||
LogAccessExclusiveLock(locktag->locktag_field1,
|
||||
locktag->locktag_field2);
|
||||
}
|
||||
|
||||
return LOCKACQUIRE_OK;
|
||||
}
|
||||
|
||||
@@ -2193,6 +2269,79 @@ GetLockStatusData(void)
|
||||
return data;
|
||||
}
|
||||
|
||||
/*
|
||||
* Returns a list of currently held AccessExclusiveLocks, for use
|
||||
* by GetRunningTransactionData().
|
||||
*/
|
||||
xl_standby_lock *
|
||||
GetRunningTransactionLocks(int *nlocks)
|
||||
{
|
||||
PROCLOCK *proclock;
|
||||
HASH_SEQ_STATUS seqstat;
|
||||
int i;
|
||||
int index;
|
||||
int els;
|
||||
xl_standby_lock *accessExclusiveLocks;
|
||||
|
||||
/*
|
||||
* Acquire lock on the entire shared lock data structure.
|
||||
*
|
||||
* Must grab LWLocks in partition-number order to avoid LWLock deadlock.
|
||||
*/
|
||||
for (i = 0; i < NUM_LOCK_PARTITIONS; i++)
|
||||
LWLockAcquire(FirstLockMgrLock + i, LW_SHARED);
|
||||
|
||||
/* Now scan the tables to copy the data */
|
||||
hash_seq_init(&seqstat, LockMethodProcLockHash);
|
||||
|
||||
/* Now we can safely count the number of proclocks */
|
||||
els = hash_get_num_entries(LockMethodProcLockHash);
|
||||
|
||||
/*
|
||||
* Allocating enough space for all locks in the lock table is overkill,
|
||||
* but it's more convenient and faster than having to enlarge the array.
|
||||
*/
|
||||
accessExclusiveLocks = palloc(els * sizeof(xl_standby_lock));
|
||||
|
||||
/*
|
||||
* If lock is a currently granted AccessExclusiveLock then
|
||||
* it will have just one proclock holder, so locks are never
|
||||
* accessed twice in this particular case. Don't copy this code
|
||||
* for use elsewhere because in the general case this will
|
||||
* give you duplicate locks when looking at non-exclusive lock types.
|
||||
*/
|
||||
index = 0;
|
||||
while ((proclock = (PROCLOCK *) hash_seq_search(&seqstat)))
|
||||
{
|
||||
/* make sure this definition matches the one used in LockAcquire */
|
||||
if ((proclock->holdMask & LOCKBIT_ON(AccessExclusiveLock)) &&
|
||||
proclock->tag.myLock->tag.locktag_type == LOCKTAG_RELATION)
|
||||
{
|
||||
PGPROC *proc = proclock->tag.myProc;
|
||||
LOCK *lock = proclock->tag.myLock;
|
||||
|
||||
accessExclusiveLocks[index].xid = proc->xid;
|
||||
accessExclusiveLocks[index].dbOid = lock->tag.locktag_field1;
|
||||
accessExclusiveLocks[index].relOid = lock->tag.locktag_field2;
|
||||
|
||||
index++;
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* And release locks. We do this in reverse order for two reasons: (1)
|
||||
* Anyone else who needs more than one of the locks will be trying to lock
|
||||
* them in increasing order; we don't want to release the other process
|
||||
* until it can get all the locks it needs. (2) This avoids O(N^2)
|
||||
* behavior inside LWLockRelease.
|
||||
*/
|
||||
for (i = NUM_LOCK_PARTITIONS; --i >= 0;)
|
||||
LWLockRelease(FirstLockMgrLock + i);
|
||||
|
||||
*nlocks = index;
|
||||
return accessExclusiveLocks;
|
||||
}
|
||||
|
||||
/* Provide the textual name of any lock mode */
|
||||
const char *
|
||||
GetLockmodeName(LOCKMETHODID lockmethodid, LOCKMODE mode)
|
||||
@@ -2288,6 +2437,24 @@ DumpAllLocks(void)
|
||||
* Because this function is run at db startup, re-acquiring the locks should
|
||||
* never conflict with running transactions because there are none. We
|
||||
* assume that the lock state represented by the stored 2PC files is legal.
|
||||
*
|
||||
* When switching from Hot Standby mode to normal operation, the locks will
|
||||
* be already held by the startup process. The locks are acquired for the new
|
||||
* procs without checking for conflicts, so we don'get a conflict between the
|
||||
* startup process and the dummy procs, even though we will momentarily have
|
||||
* a situation where two procs are holding the same AccessExclusiveLock,
|
||||
* which isn't normally possible because the conflict. If we're in standby
|
||||
* mode, but a recovery snapshot hasn't been established yet, it's possible
|
||||
* that some but not all of the locks are already held by the startup process.
|
||||
*
|
||||
* This approach is simple, but also a bit dangerous, because if there isn't
|
||||
* enough shared memory to acquire the locks, an error will be thrown, which
|
||||
* is promoted to FATAL and recovery will abort, bringing down postmaster.
|
||||
* A safer approach would be to transfer the locks like we do in
|
||||
* AtPrepare_Locks, but then again, in hot standby mode it's possible for
|
||||
* read-only backends to use up all the shared lock memory anyway, so that
|
||||
* replaying the WAL record that needs to acquire a lock will throw an error
|
||||
* and PANIC anyway.
|
||||
*/
|
||||
void
|
||||
lock_twophase_recover(TransactionId xid, uint16 info,
|
||||
@@ -2443,12 +2610,45 @@ lock_twophase_recover(TransactionId xid, uint16 info,
|
||||
|
||||
/*
|
||||
* We ignore any possible conflicts and just grant ourselves the lock.
|
||||
* Not only because we don't bother, but also to avoid deadlocks when
|
||||
* switching from standby to normal mode. See function comment.
|
||||
*/
|
||||
GrantLock(lock, proclock, lockmode);
|
||||
|
||||
LWLockRelease(partitionLock);
|
||||
}
|
||||
|
||||
/*
|
||||
* Re-acquire a lock belonging to a transaction that was prepared, when
|
||||
* when starting up into hot standby mode.
|
||||
*/
|
||||
void
|
||||
lock_twophase_standby_recover(TransactionId xid, uint16 info,
|
||||
void *recdata, uint32 len)
|
||||
{
|
||||
TwoPhaseLockRecord *rec = (TwoPhaseLockRecord *) recdata;
|
||||
LOCKTAG *locktag;
|
||||
LOCKMODE lockmode;
|
||||
LOCKMETHODID lockmethodid;
|
||||
|
||||
Assert(len == sizeof(TwoPhaseLockRecord));
|
||||
locktag = &rec->locktag;
|
||||
lockmode = rec->lockmode;
|
||||
lockmethodid = locktag->locktag_lockmethodid;
|
||||
|
||||
if (lockmethodid <= 0 || lockmethodid >= lengthof(LockMethods))
|
||||
elog(ERROR, "unrecognized lock method: %d", lockmethodid);
|
||||
|
||||
if (lockmode == AccessExclusiveLock &&
|
||||
locktag->locktag_type == LOCKTAG_RELATION)
|
||||
{
|
||||
StandbyAcquireAccessExclusiveLock(xid,
|
||||
locktag->locktag_field1 /* dboid */,
|
||||
locktag->locktag_field2 /* reloid */);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* 2PC processing routine for COMMIT PREPARED case.
|
||||
*
|
||||
|
@@ -8,7 +8,7 @@
|
||||
*
|
||||
*
|
||||
* IDENTIFICATION
|
||||
* $PostgreSQL: pgsql/src/backend/storage/lmgr/proc.c,v 1.209 2009/08/31 19:41:00 tgl Exp $
|
||||
* $PostgreSQL: pgsql/src/backend/storage/lmgr/proc.c,v 1.210 2009/12/19 01:32:36 sriggs Exp $
|
||||
*
|
||||
*-------------------------------------------------------------------------
|
||||
*/
|
||||
@@ -318,6 +318,7 @@ InitProcess(void)
|
||||
MyProc->waitProcLock = NULL;
|
||||
for (i = 0; i < NUM_LOCK_PARTITIONS; i++)
|
||||
SHMQueueInit(&(MyProc->myProcLocks[i]));
|
||||
MyProc->recoveryConflictMode = 0;
|
||||
|
||||
/*
|
||||
* We might be reusing a semaphore that belonged to a failed process. So
|
||||
@@ -374,6 +375,11 @@ InitProcessPhase2(void)
|
||||
* to the ProcArray or the sinval messaging mechanism, either. They also
|
||||
* don't get a VXID assigned, since this is only useful when we actually
|
||||
* hold lockmgr locks.
|
||||
*
|
||||
* Startup process however uses locks but never waits for them in the
|
||||
* normal backend sense. Startup process also takes part in sinval messaging
|
||||
* as a sendOnly process, so never reads messages from sinval queue. So
|
||||
* Startup process does have a VXID and does show up in pg_locks.
|
||||
*/
|
||||
void
|
||||
InitAuxiliaryProcess(void)
|
||||
@@ -461,6 +467,24 @@ InitAuxiliaryProcess(void)
|
||||
on_shmem_exit(AuxiliaryProcKill, Int32GetDatum(proctype));
|
||||
}
|
||||
|
||||
/*
|
||||
* Record the PID and PGPROC structures for the Startup process, for use in
|
||||
* ProcSendSignal(). See comments there for further explanation.
|
||||
*/
|
||||
void
|
||||
PublishStartupProcessInformation(void)
|
||||
{
|
||||
/* use volatile pointer to prevent code rearrangement */
|
||||
volatile PROC_HDR *procglobal = ProcGlobal;
|
||||
|
||||
SpinLockAcquire(ProcStructLock);
|
||||
|
||||
procglobal->startupProc = MyProc;
|
||||
procglobal->startupProcPid = MyProcPid;
|
||||
|
||||
SpinLockRelease(ProcStructLock);
|
||||
}
|
||||
|
||||
/*
|
||||
* Check whether there are at least N free PGPROC objects.
|
||||
*
|
||||
@@ -1289,7 +1313,31 @@ ProcWaitForSignal(void)
|
||||
void
|
||||
ProcSendSignal(int pid)
|
||||
{
|
||||
PGPROC *proc = BackendPidGetProc(pid);
|
||||
PGPROC *proc = NULL;
|
||||
|
||||
if (RecoveryInProgress())
|
||||
{
|
||||
/* use volatile pointer to prevent code rearrangement */
|
||||
volatile PROC_HDR *procglobal = ProcGlobal;
|
||||
|
||||
SpinLockAcquire(ProcStructLock);
|
||||
|
||||
/*
|
||||
* Check to see whether it is the Startup process we wish to signal.
|
||||
* This call is made by the buffer manager when it wishes to wake
|
||||
* up a process that has been waiting for a pin in so it can obtain a
|
||||
* cleanup lock using LockBufferForCleanup(). Startup is not a normal
|
||||
* backend, so BackendPidGetProc() will not return any pid at all.
|
||||
* So we remember the information for this special case.
|
||||
*/
|
||||
if (pid == procglobal->startupProcPid)
|
||||
proc = procglobal->startupProc;
|
||||
|
||||
SpinLockRelease(ProcStructLock);
|
||||
}
|
||||
|
||||
if (proc == NULL)
|
||||
proc = BackendPidGetProc(pid);
|
||||
|
||||
if (proc != NULL)
|
||||
PGSemaphoreUnlock(&proc->sem);
|
||||
|
Reference in New Issue
Block a user