Commit ea9df812 authored by Robert Haas's avatar Robert Haas

Relax the requirement that all lwlocks be stored in a single array.

This makes it possible to store lwlocks as part of some other data
structure in the main shared memory segment, or in a dynamic shared
memory segment.  There is still a main LWLock array and this patch does
not move anything out of it, but it provides necessary infrastructure
for doing that in the future.

This change is likely to increase the size of LWLockPadded on some
platforms, especially 32-bit platforms where it was previously only
16 bytes.

Patch by me.  Review by Andres Freund and KaiGai Kohei.
parent f62eba20
......@@ -116,7 +116,7 @@ pg_buffercache_pages(PG_FUNCTION_ARGS)
* possible deadlocks.
*/
for (i = 0; i < NUM_BUFFER_PARTITIONS; i++)
LWLockAcquire(FirstBufMappingLock + i, LW_SHARED);
LWLockAcquire(BufMappingPartitionLockByIndex(i), LW_SHARED);
/*
* Scan though all the buffers, saving the relevant fields in the
......@@ -157,7 +157,7 @@ pg_buffercache_pages(PG_FUNCTION_ARGS)
* avoids O(N^2) behavior inside LWLockRelease.
*/
for (i = NUM_BUFFER_PARTITIONS; --i >= 0;)
LWLockRelease(FirstBufMappingLock + i);
LWLockRelease(BufMappingPartitionLockByIndex(i));
}
funcctx = SRF_PERCALL_SETUP();
......
......@@ -150,7 +150,7 @@ typedef struct pgssEntry
*/
typedef struct pgssSharedState
{
LWLockId lock; /* protects hashtable search/modification */
LWLock *lock; /* protects hashtable search/modification */
int query_size; /* max query length in bytes */
double cur_median_usage; /* current median usage in hashtable */
} pgssSharedState;
......
......@@ -2212,49 +2212,55 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
</row>
<row>
<entry>lwlock-acquire</entry>
<entry>(LWLockId, LWLockMode)</entry>
<entry>(char *, int, LWLockMode)</entry>
<entry>Probe that fires when an LWLock has been acquired.
arg0 is the LWLock's ID.
arg1 is the requested lock mode, either exclusive or shared.</entry>
arg0 is the LWLock's tranche.
arg1 is the LWLock's offset within its trance.
arg2 is the requested lock mode, either exclusive or shared.</entry>
</row>
<row>
<entry>lwlock-release</entry>
<entry>(LWLockId)</entry>
<entry>(char *, int)</entry>
<entry>Probe that fires when an LWLock has been released (but note
that any released waiters have not yet been awakened).
arg0 is the LWLock's ID.</entry>
arg0 is the LWLock's tranche.
arg1 is the LWLock's offset within its trance.</entry>
</row>
<row>
<entry>lwlock-wait-start</entry>
<entry>(LWLockId, LWLockMode)</entry>
<entry>(char *, int, LWLockMode)</entry>
<entry>Probe that fires when an LWLock was not immediately available and
a server process has begun to wait for the lock to become available.
arg0 is the LWLock's ID.
arg1 is the requested lock mode, either exclusive or shared.</entry>
arg0 is the LWLock's tranche.
arg1 is the LWLock's offset within its trance.
arg2 is the requested lock mode, either exclusive or shared.</entry>
</row>
<row>
<entry>lwlock-wait-done</entry>
<entry>(LWLockId, LWLockMode)</entry>
<entry>(char *, int, LWLockMode)</entry>
<entry>Probe that fires when a server process has been released from its
wait for an LWLock (it does not actually have the lock yet).
arg0 is the LWLock's ID.
arg1 is the requested lock mode, either exclusive or shared.</entry>
arg0 is the LWLock's tranche.
arg1 is the LWLock's offset within its trance.
arg2 is the requested lock mode, either exclusive or shared.</entry>
</row>
<row>
<entry>lwlock-condacquire</entry>
<entry>(LWLockId, LWLockMode)</entry>
<entry>(char *, int, LWLockMode)</entry>
<entry>Probe that fires when an LWLock was successfully acquired when the
caller specified no waiting.
arg0 is the LWLock's ID.
arg1 is the requested lock mode, either exclusive or shared.</entry>
arg0 is the LWLock's tranche.
arg1 is the LWLock's offset within its trance.
arg2 is the requested lock mode, either exclusive or shared.</entry>
</row>
<row>
<entry>lwlock-condacquire-fail</entry>
<entry>(LWLockId, LWLockMode)</entry>
<entry>(char *, int, LWLockMode)</entry>
<entry>Probe that fires when an LWLock was not successfully acquired when
the caller specified no waiting.
arg0 is the LWLock's ID.
arg1 is the requested lock mode, either exclusive or shared.</entry>
arg0 is the LWLock's tranche.
arg1 is the LWLock's offset within its trance.
arg2 is the requested lock mode, either exclusive or shared.</entry>
</row>
<row>
<entry>lock-wait-start</entry>
......@@ -2299,10 +2305,6 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
<entry>LocalTransactionId</entry>
<entry>unsigned int</entry>
</row>
<row>
<entry>LWLockId</entry>
<entry>int</entry>
</row>
<row>
<entry>LWLockMode</entry>
<entry>int</entry>
......
......@@ -151,7 +151,7 @@ SimpleLruShmemSize(int nslots, int nlsns)
sz += MAXALIGN(nslots * sizeof(bool)); /* page_dirty[] */
sz += MAXALIGN(nslots * sizeof(int)); /* page_number[] */
sz += MAXALIGN(nslots * sizeof(int)); /* page_lru_count[] */
sz += MAXALIGN(nslots * sizeof(LWLockId)); /* buffer_locks[] */
sz += MAXALIGN(nslots * sizeof(LWLock *)); /* buffer_locks[] */
if (nlsns > 0)
sz += MAXALIGN(nslots * nlsns * sizeof(XLogRecPtr)); /* group_lsn[] */
......@@ -161,7 +161,7 @@ SimpleLruShmemSize(int nslots, int nlsns)
void
SimpleLruInit(SlruCtl ctl, const char *name, int nslots, int nlsns,
LWLockId ctllock, const char *subdir)
LWLock *ctllock, const char *subdir)
{
SlruShared shared;
bool found;
......@@ -202,8 +202,8 @@ SimpleLruInit(SlruCtl ctl, const char *name, int nslots, int nlsns,
offset += MAXALIGN(nslots * sizeof(int));
shared->page_lru_count = (int *) (ptr + offset);
offset += MAXALIGN(nslots * sizeof(int));
shared->buffer_locks = (LWLockId *) (ptr + offset);
offset += MAXALIGN(nslots * sizeof(LWLockId));
shared->buffer_locks = (LWLock **) (ptr + offset);
offset += MAXALIGN(nslots * sizeof(LWLock *));
if (nlsns > 0)
{
......
......@@ -448,8 +448,6 @@ typedef struct
typedef int InheritableSocket;
#endif
typedef struct LWLock LWLock; /* ugly kluge */
/*
* Structure contains all variables passed to exec:ed backends
*/
......@@ -473,7 +471,7 @@ typedef struct
#ifndef HAVE_SPINLOCKS
PGSemaphore SpinlockSemaArray;
#endif
LWLock *LWLockArray;
LWLock *MainLWLockArray;
slock_t *ProcStructLock;
PROC_HDR *ProcGlobal;
PGPROC *AuxiliaryProcs;
......@@ -5576,7 +5574,6 @@ PostmasterMarkPIDForWorkerNotify(int pid)
* functions. They are marked NON_EXEC_STATIC in their home modules.
*/
extern slock_t *ShmemLock;
extern LWLock *LWLockArray;
extern slock_t *ProcStructLock;
extern PGPROC *AuxiliaryProcs;
extern PMSignalData *PMSignalState;
......@@ -5625,7 +5622,7 @@ save_backend_variables(BackendParameters *param, Port *port,
#ifndef HAVE_SPINLOCKS
param->SpinlockSemaArray = SpinlockSemaArray;
#endif
param->LWLockArray = LWLockArray;
param->MainLWLockArray = MainLWLockArray;
param->ProcStructLock = ProcStructLock;
param->ProcGlobal = ProcGlobal;
param->AuxiliaryProcs = AuxiliaryProcs;
......@@ -5856,7 +5853,7 @@ restore_backend_variables(BackendParameters *param, Port *port)
#ifndef HAVE_SPINLOCKS
SpinlockSemaArray = param->SpinlockSemaArray;
#endif
LWLockArray = param->LWLockArray;
MainLWLockArray = param->MainLWLockArray;
ProcStructLock = param->ProcStructLock;
ProcGlobal = param->ProcGlobal;
AuxiliaryProcs = param->AuxiliaryProcs;
......
......@@ -146,7 +146,7 @@ PrefetchBuffer(Relation reln, ForkNumber forkNum, BlockNumber blockNum)
{
BufferTag newTag; /* identity of requested block */
uint32 newHash; /* hash value for newTag */
LWLockId newPartitionLock; /* buffer partition lock for it */
LWLock *newPartitionLock; /* buffer partition lock for it */
int buf_id;
/* create a tag so we can lookup the buffer */
......@@ -539,10 +539,10 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
{
BufferTag newTag; /* identity of requested block */
uint32 newHash; /* hash value for newTag */
LWLockId newPartitionLock; /* buffer partition lock for it */
LWLock *newPartitionLock; /* buffer partition lock for it */
BufferTag oldTag; /* previous identity of selected buffer */
uint32 oldHash; /* hash value for oldTag */
LWLockId oldPartitionLock; /* buffer partition lock for it */
LWLock *oldPartitionLock; /* buffer partition lock for it */
BufFlags oldFlags;
int buf_id;
volatile BufferDesc *buf;
......@@ -891,7 +891,7 @@ InvalidateBuffer(volatile BufferDesc *buf)
{
BufferTag oldTag;
uint32 oldHash; /* hash value for oldTag */
LWLockId oldPartitionLock; /* buffer partition lock for it */
LWLock *oldPartitionLock; /* buffer partition lock for it */
BufFlags oldFlags;
/* Save the original buffer tag before dropping the spinlock */
......
......@@ -182,8 +182,7 @@ CreateSharedMemoryAndSemaphores(bool makePrivate, int port)
* Now initialize LWLocks, which do shared memory allocation and are
* needed for InitShmemIndex.
*/
if (!IsUnderPostmaster)
CreateLWLocks();
CreateLWLocks();
/*
* Set up shmem.c index hashtable
......
......@@ -565,7 +565,7 @@ LockHasWaiters(const LOCKTAG *locktag, LOCKMODE lockmode, bool sessionLock)
LOCALLOCK *locallock;
LOCK *lock;
PROCLOCK *proclock;
LWLockId partitionLock;
LWLock *partitionLock;
bool hasWaiters = false;
if (lockmethodid <= 0 || lockmethodid >= lengthof(LockMethods))
......@@ -702,7 +702,7 @@ LockAcquireExtended(const LOCKTAG *locktag,
bool found;
ResourceOwner owner;
uint32 hashcode;
LWLockId partitionLock;
LWLock *partitionLock;
int status;
bool log_lock = false;
......@@ -1744,7 +1744,7 @@ LockRelease(const LOCKTAG *locktag, LOCKMODE lockmode, bool sessionLock)
LOCALLOCK *locallock;
LOCK *lock;
PROCLOCK *proclock;
LWLockId partitionLock;
LWLock *partitionLock;
bool wakeupNeeded;
if (lockmethodid <= 0 || lockmethodid >= lengthof(LockMethods))
......@@ -2096,10 +2096,12 @@ LockReleaseAll(LOCKMETHODID lockmethodid, bool allLocks)
*/
for (partition = 0; partition < NUM_LOCK_PARTITIONS; partition++)
{
LWLockId partitionLock = FirstLockMgrLock + partition;
LWLock *partitionLock;
SHM_QUEUE *procLocks = &(MyProc->myProcLocks[partition]);
PROCLOCK *nextplock;
partitionLock = LockHashPartitionLockByIndex(partition);
/*
* If the proclock list for this partition is empty, we can skip
* acquiring the partition lock. This optimization is trickier than
......@@ -2475,7 +2477,7 @@ static bool
FastPathTransferRelationLocks(LockMethod lockMethodTable, const LOCKTAG *locktag,
uint32 hashcode)
{
LWLockId partitionLock = LockHashPartitionLock(hashcode);
LWLock *partitionLock = LockHashPartitionLock(hashcode);
Oid relid = locktag->locktag_field2;
uint32 i;
......@@ -2565,7 +2567,7 @@ FastPathGetRelationLockEntry(LOCALLOCK *locallock)
LockMethod lockMethodTable = LockMethods[DEFAULT_LOCKMETHOD];
LOCKTAG *locktag = &locallock->tag.lock;
PROCLOCK *proclock = NULL;
LWLockId partitionLock = LockHashPartitionLock(locallock->hashcode);
LWLock *partitionLock = LockHashPartitionLock(locallock->hashcode);
Oid relid = locktag->locktag_field2;
uint32 f;
......@@ -2671,7 +2673,7 @@ GetLockConflicts(const LOCKTAG *locktag, LOCKMODE lockmode)
SHM_QUEUE *procLocks;
PROCLOCK *proclock;
uint32 hashcode;
LWLockId partitionLock;
LWLock *partitionLock;
int count = 0;
int fast_count = 0;
......@@ -2883,7 +2885,7 @@ LockRefindAndRelease(LockMethod lockMethodTable, PGPROC *proc,
PROCLOCKTAG proclocktag;
uint32 hashcode;
uint32 proclock_hashcode;
LWLockId partitionLock;
LWLock *partitionLock;
bool wakeupNeeded;
hashcode = LockTagHashCode(locktag);
......@@ -3159,10 +3161,12 @@ PostPrepare_Locks(TransactionId xid)
*/
for (partition = 0; partition < NUM_LOCK_PARTITIONS; partition++)
{
LWLockId partitionLock = FirstLockMgrLock + partition;
LWLock *partitionLock;
SHM_QUEUE *procLocks = &(MyProc->myProcLocks[partition]);
PROCLOCK *nextplock;
partitionLock = LockHashPartitionLockByIndex(partition);
/*
* If the proclock list for this partition is empty, we can skip
* acquiring the partition lock. This optimization is safer than the
......@@ -3400,7 +3404,7 @@ GetLockStatusData(void)
* Must grab LWLocks in partition-number order to avoid LWLock deadlock.
*/
for (i = 0; i < NUM_LOCK_PARTITIONS; i++)
LWLockAcquire(FirstLockMgrLock + i, LW_SHARED);
LWLockAcquire(LockHashPartitionLockByIndex(i), LW_SHARED);
/* Now we can safely count the number of proclocks */
data->nelements = el + hash_get_num_entries(LockMethodProcLockHash);
......@@ -3442,7 +3446,7 @@ GetLockStatusData(void)
* behavior inside LWLockRelease.
*/
for (i = NUM_LOCK_PARTITIONS; --i >= 0;)
LWLockRelease(FirstLockMgrLock + i);
LWLockRelease(LockHashPartitionLockByIndex(i));
Assert(el == data->nelements);
......@@ -3477,7 +3481,7 @@ GetRunningTransactionLocks(int *nlocks)
* Must grab LWLocks in partition-number order to avoid LWLock deadlock.
*/
for (i = 0; i < NUM_LOCK_PARTITIONS; i++)
LWLockAcquire(FirstLockMgrLock + i, LW_SHARED);
LWLockAcquire(LockHashPartitionLockByIndex(i), LW_SHARED);
/* Now we can safely count the number of proclocks */
els = hash_get_num_entries(LockMethodProcLockHash);
......@@ -3537,7 +3541,7 @@ GetRunningTransactionLocks(int *nlocks)
* behavior inside LWLockRelease.
*/
for (i = NUM_LOCK_PARTITIONS; --i >= 0;)
LWLockRelease(FirstLockMgrLock + i);
LWLockRelease(LockHashPartitionLockByIndex(i));
*nlocks = index;
return accessExclusiveLocks;
......@@ -3673,7 +3677,7 @@ lock_twophase_recover(TransactionId xid, uint16 info,
uint32 hashcode;
uint32 proclock_hashcode;
int partition;
LWLockId partitionLock;
LWLock *partitionLock;
LockMethod lockMethodTable;
Assert(len == sizeof(TwoPhaseLockRecord));
......@@ -4044,7 +4048,7 @@ VirtualXactLock(VirtualTransactionId vxid, bool wait)
{
PROCLOCK *proclock;
uint32 hashcode;
LWLockId partitionLock;
LWLock *partitionLock;
hashcode = LockTagHashCode(&tag);
......
This diff is collapsed.
......@@ -241,7 +241,10 @@
#define PredicateLockHashPartition(hashcode) \
((hashcode) % NUM_PREDICATELOCK_PARTITIONS)
#define PredicateLockHashPartitionLock(hashcode) \
((LWLockId) (FirstPredicateLockMgrLock + PredicateLockHashPartition(hashcode)))
(&MainLWLockArray[PREDICATELOCK_MANAGER_LWLOCK_OFFSET + \
PredicateLockHashPartition(hashcode)].lock)
#define PredicateLockHashPartitionLockByIndex(i) \
(&MainLWLockArray[PREDICATELOCK_MANAGER_LWLOCK_OFFSET + (i)].lock)
#define NPREDICATELOCKTARGETENTS() \
mul_size(max_predicate_locks_per_xact, add_size(MaxBackends, max_prepared_xacts))
......@@ -383,7 +386,7 @@ static SHM_QUEUE *FinishedSerializableTransactions;
*/
static const PREDICATELOCKTARGETTAG ScratchTargetTag = {0, 0, 0, 0};
static uint32 ScratchTargetTagHash;
static int ScratchPartitionLock;
static LWLock *ScratchPartitionLock;
/*
* The local hash table used to determine when to combine multiple fine-
......@@ -1398,7 +1401,7 @@ GetPredicateLockStatusData(void)
* in ascending order, then SerializableXactHashLock.
*/
for (i = 0; i < NUM_PREDICATELOCK_PARTITIONS; i++)
LWLockAcquire(FirstPredicateLockMgrLock + i, LW_SHARED);
LWLockAcquire(PredicateLockHashPartitionLockByIndex(i), LW_SHARED);
LWLockAcquire(SerializableXactHashLock, LW_SHARED);
/* Get number of locks and allocate appropriately-sized arrays. */
......@@ -1427,7 +1430,7 @@ GetPredicateLockStatusData(void)
/* Release locks in reverse order */
LWLockRelease(SerializableXactHashLock);
for (i = NUM_PREDICATELOCK_PARTITIONS - 1; i >= 0; i--)
LWLockRelease(FirstPredicateLockMgrLock + i);
LWLockRelease(PredicateLockHashPartitionLockByIndex(i));
return data;
}
......@@ -1856,7 +1859,7 @@ PageIsPredicateLocked(Relation relation, BlockNumber blkno)
{
PREDICATELOCKTARGETTAG targettag;
uint32 targettaghash;
LWLockId partitionLock;
LWLock *partitionLock;
PREDICATELOCKTARGET *target;
SET_PREDICATELOCKTARGETTAG_PAGE(targettag,
......@@ -2089,7 +2092,7 @@ DeleteChildTargetLocks(const PREDICATELOCKTARGETTAG *newtargettag)
if (TargetTagIsCoveredBy(oldtargettag, *newtargettag))
{
uint32 oldtargettaghash;
LWLockId partitionLock;
LWLock *partitionLock;
PREDICATELOCK *rmpredlock PG_USED_FOR_ASSERTS_ONLY;
oldtargettaghash = PredicateLockTargetTagHashCode(&oldtargettag);
......@@ -2301,7 +2304,7 @@ CreatePredicateLock(const PREDICATELOCKTARGETTAG *targettag,
PREDICATELOCKTARGET *target;
PREDICATELOCKTAG locktag;
PREDICATELOCK *lock;
LWLockId partitionLock;
LWLock *partitionLock;
bool found;
partitionLock = PredicateLockHashPartitionLock(targettaghash);
......@@ -2599,10 +2602,10 @@ TransferPredicateLocksToNewTarget(PREDICATELOCKTARGETTAG oldtargettag,
bool removeOld)
{
uint32 oldtargettaghash;
LWLockId oldpartitionLock;
LWLock *oldpartitionLock;
PREDICATELOCKTARGET *oldtarget;
uint32 newtargettaghash;
LWLockId newpartitionLock;
LWLock *newpartitionLock;
bool found;
bool outOfShmem = false;
......@@ -2858,7 +2861,7 @@ DropAllPredicateLocksFromTable(Relation relation, bool transfer)
/* Acquire locks on all lock partitions */
LWLockAcquire(SerializablePredicateLockListLock, LW_EXCLUSIVE);
for (i = 0; i < NUM_PREDICATELOCK_PARTITIONS; i++)
LWLockAcquire(FirstPredicateLockMgrLock + i, LW_EXCLUSIVE);
LWLockAcquire(PredicateLockHashPartitionLockByIndex(i), LW_EXCLUSIVE);
LWLockAcquire(SerializableXactHashLock, LW_EXCLUSIVE);
/*
......@@ -2996,7 +2999,7 @@ DropAllPredicateLocksFromTable(Relation relation, bool transfer)
/* Release locks in reverse order */
LWLockRelease(SerializableXactHashLock);
for (i = NUM_PREDICATELOCK_PARTITIONS - 1; i >= 0; i--)
LWLockRelease(FirstPredicateLockMgrLock + i);
LWLockRelease(PredicateLockHashPartitionLockByIndex(i));
LWLockRelease(SerializablePredicateLockListLock);
}
......@@ -3611,7 +3614,7 @@ ClearOldPredicateLocks(void)
PREDICATELOCKTARGET *target;
PREDICATELOCKTARGETTAG targettag;
uint32 targettaghash;
LWLockId partitionLock;
LWLock *partitionLock;
tag = predlock->tag;
target = tag.myTarget;
......@@ -3690,7 +3693,7 @@ ReleaseOneSerializableXact(SERIALIZABLEXACT *sxact, bool partial,
PREDICATELOCKTARGET *target;
PREDICATELOCKTARGETTAG targettag;
uint32 targettaghash;
LWLockId partitionLock;
LWLock *partitionLock;
nextpredlock = (PREDICATELOCK *)
SHMQueueNext(&(sxact->predicateLocks),
......@@ -4068,7 +4071,7 @@ static void
CheckTargetForConflictsIn(PREDICATELOCKTARGETTAG *targettag)
{
uint32 targettaghash;
LWLockId partitionLock;
LWLock *partitionLock;
PREDICATELOCKTARGET *target;
PREDICATELOCK *predlock;
PREDICATELOCK *mypredlock = NULL;
......@@ -4360,7 +4363,7 @@ CheckTableForSerializableConflictIn(Relation relation)
LWLockAcquire(SerializablePredicateLockListLock, LW_EXCLUSIVE);
for (i = 0; i < NUM_PREDICATELOCK_PARTITIONS; i++)
LWLockAcquire(FirstPredicateLockMgrLock + i, LW_SHARED);
LWLockAcquire(PredicateLockHashPartitionLockByIndex(i), LW_SHARED);
LWLockAcquire(SerializableXactHashLock, LW_SHARED);
/* Scan through target list */
......@@ -4407,7 +4410,7 @@ CheckTableForSerializableConflictIn(Relation relation)
/* Release locks in reverse order */
LWLockRelease(SerializableXactHashLock);
for (i = NUM_PREDICATELOCK_PARTITIONS - 1; i >= 0; i--)
LWLockRelease(FirstPredicateLockMgrLock + i);
LWLockRelease(PredicateLockHashPartitionLockByIndex(i));
LWLockRelease(SerializablePredicateLockListLock);
}
......
......@@ -189,7 +189,8 @@ InitProcGlobal(void)
*/
procs = (PGPROC *) ShmemAlloc(TotalProcs * sizeof(PGPROC));
ProcGlobal->allProcs = procs;
ProcGlobal->allProcCount = TotalProcs;
/* XXX allProcCount isn't really all of them; it excludes prepared xacts */
ProcGlobal->allProcCount = MaxBackends + NUM_AUXILIARY_PROCS;
if (!procs)
ereport(FATAL,
(errcode(ERRCODE_OUT_OF_MEMORY),
......@@ -663,7 +664,7 @@ IsWaitingForLock(void)
void
LockErrorCleanup(void)
{
LWLockId partitionLock;
LWLock *partitionLock;
DisableTimeoutParams timeouts[2];
AbortStrongLockAcquire();
......@@ -942,7 +943,7 @@ ProcSleep(LOCALLOCK *locallock, LockMethod lockMethodTable)
LOCK *lock = locallock->lock;
PROCLOCK *proclock = locallock->proclock;
uint32 hashcode = locallock->hashcode;
LWLockId partitionLock = LockHashPartitionLock(hashcode);
LWLock *partitionLock = LockHashPartitionLock(hashcode);
PROC_QUEUE *waitQueue = &(lock->waitProcs);
LOCKMASK myHeldLocks = MyProc->heldLocks;
bool early_deadlock = false;
......@@ -1440,7 +1441,7 @@ CheckDeadLock(void)
* interrupts.
*/
for (i = 0; i < NUM_LOCK_PARTITIONS; i++)
LWLockAcquire(FirstLockMgrLock + i, LW_EXCLUSIVE);
LWLockAcquire(LockHashPartitionLockByIndex(i), LW_EXCLUSIVE);
/*
* Check to see if we've been awoken by anyone in the interim.
......@@ -1522,7 +1523,7 @@ CheckDeadLock(void)
*/
check_done:
for (i = NUM_LOCK_PARTITIONS; --i >= 0;)
LWLockRelease(FirstLockMgrLock + i);
LWLockRelease(LockHashPartitionLockByIndex(i));
}
......
......@@ -15,7 +15,6 @@
* in probe definitions, as they cause compilation errors on Mac OS X 10.5.
*/
#define LocalTransactionId unsigned int
#define LWLockId int
#define LWLockMode int
#define LOCKMODE int
#define BlockNumber unsigned int
......@@ -29,14 +28,14 @@ provider postgresql {
probe transaction__commit(LocalTransactionId);
probe transaction__abort(LocalTransactionId);
probe lwlock__acquire(LWLockId, LWLockMode);
probe lwlock__release(LWLockId);
probe lwlock__wait__start(LWLockId, LWLockMode);
probe lwlock__wait__done(LWLockId, LWLockMode);
probe lwlock__condacquire(LWLockId, LWLockMode);
probe lwlock__condacquire__fail(LWLockId, LWLockMode);
probe lwlock__wait__until__free(LWLockId, LWLockMode);
probe lwlock__wait__until__free__fail(LWLockId, LWLockMode);
probe lwlock__acquire(const char *, int, LWLockMode);
probe lwlock__release(const char *, int);
probe lwlock__wait__start(const char *, int, LWLockMode);
probe lwlock__wait__done(const char *, int, LWLockMode);
probe lwlock__condacquire(const char *, int, LWLockMode);
probe lwlock__condacquire__fail(const char *, int, LWLockMode);
probe lwlock__wait__until__free(const char *, int, LWLockMode);
probe lwlock__wait__until__free__fail(const char *, int, LWLockMode);
probe lock__wait__start(unsigned int, unsigned int, unsigned int, unsigned int, unsigned int, LOCKMODE);
probe lock__wait__done(unsigned int, unsigned int, unsigned int, unsigned int, unsigned int, LOCKMODE);
......
......@@ -55,7 +55,7 @@ typedef enum
*/
typedef struct SlruSharedData
{
LWLockId ControlLock;
LWLock *ControlLock;
/* Number of buffers managed by this SLRU structure */
int num_slots;
......@@ -69,7 +69,7 @@ typedef struct SlruSharedData
bool *page_dirty;
int *page_number;
int *page_lru_count;
LWLockId *buffer_locks;
LWLock **buffer_locks;
/*
* Optional array of WAL flush LSNs associated with entries in the SLRU
......@@ -136,7 +136,7 @@ typedef SlruCtlData *SlruCtl;
extern Size SimpleLruShmemSize(int nslots, int nlsns);
extern void SimpleLruInit(SlruCtl ctl, const char *name, int nslots, int nlsns,
LWLockId ctllock, const char *subdir);
LWLock *ctllock, const char *subdir);
extern int SimpleLruZeroPage(SlruCtl ctl, int pageno);
extern int SimpleLruReadPage(SlruCtl ctl, int pageno, bool write_ok,
TransactionId xid);
......
......@@ -104,7 +104,10 @@ typedef struct buftag
#define BufTableHashPartition(hashcode) \
((hashcode) % NUM_BUFFER_PARTITIONS)
#define BufMappingPartitionLock(hashcode) \
((LWLockId) (FirstBufMappingLock + BufTableHashPartition(hashcode)))
(&MainLWLockArray[BUFFER_MAPPING_LWLOCK_OFFSET + \
BufTableHashPartition(hashcode)].lock)
#define BufMappingPartitionLockByIndex(i) \
(&MainLWLockArray[BUFFER_MAPPING_LWLOCK_OFFSET + (i)].lock)
/*
* BufferDesc -- shared descriptor/state data for a single shared buffer.
......@@ -144,8 +147,8 @@ typedef struct sbufdesc
int buf_id; /* buffer's index number (from 0) */
int freeNext; /* link in freelist chain */
LWLockId io_in_progress_lock; /* to wait for I/O to complete */
LWLockId content_lock; /* to lock access to buffer contents */
LWLock *io_in_progress_lock; /* to wait for I/O to complete */
LWLock *content_lock; /* to lock access to buffer contents */
} BufferDesc;
#define BufferDescriptorGetBuffer(bdesc) ((bdesc)->buf_id + 1)
......
......@@ -483,8 +483,10 @@ typedef enum
#define LockHashPartition(hashcode) \
((hashcode) % NUM_LOCK_PARTITIONS)
#define LockHashPartitionLock(hashcode) \
((LWLockId) (FirstLockMgrLock + LockHashPartition(hashcode)))
(&MainLWLockArray[LOCK_MANAGER_LWLOCK_OFFSET + \
LockHashPartition(hashcode)].lock)
#define LockHashPartitionLockByIndex(i) \
(&MainLWLockArray[LOCK_MANAGER_LWLOCK_OFFSET + (i)].lock)
/*
* function prototypes
......
This diff is collapsed.
......@@ -131,7 +131,7 @@ struct PGPROC
struct XidCache subxids; /* cache for subtransaction XIDs */
/* Per-backend LWLock. Protects fields below. */
LWLockId backendLock; /* protects the fields below */
LWLock *backendLock; /* protects the fields below */
/* Lock manager data, recording fast-path locks taken by this backend. */
uint64 fpLockBits; /* lock modes held for each fast-path slot */
......
......@@ -896,7 +896,6 @@ LPWSTR
LSEG
LVRelStats
LWLock
LWLockId
LWLockMode
LWLockPadded
LabelProvider
......
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment