<itemizedlist>
<listitem>
<para>
- <literal>LWLockNamed</>: The backend is waiting for a specific named
- lightweight lock. Each such lock protects a particular data
- structure in shared memory. <literal>wait_event</> will contain
- the name of the lightweight lock.
- </para>
- </listitem>
- <listitem>
- <para>
- <literal>LWLockTranche</>: The backend is waiting for one of a
- group of related lightweight locks. All locks in the group perform
- a similar function; <literal>wait_event</> will identify the general
- purpose of locks in that group.
+ <literal>LWLock</>: The backend is waiting for a lightweight lock.
+ Each such lock protects a particular data structure in shared memory.
+ <literal>wait_event</> will contain a name identifying the purpose
+ of the lightweight lock. (Some locks have specific names; others
+ are part of a group of locks each with a similar purpose.)
</para>
</listitem>
<listitem>
<tbody>
<row>
- <entry morerows="41"><literal>LWLockNamed</></entry>
+ <entry morerows="57"><literal>LWLock</></entry>
<entry><literal>ShmemIndexLock</></entry>
<entry>Waiting to find or allocate space in shared memory.</entry>
</row>
<entry>Waiting to read or update old snapshot control information.</entry>
</row>
<row>
- <entry morerows="15"><literal>LWLockTranche</></entry>
<entry><literal>clog</></entry>
<entry>Waiting for I/O on a clog (transaction status) buffer.</entry>
</row>
pid | wait_event_type | wait_event
------+-----------------+---------------
2540 | Lock | relation
- 6644 | LWLockNamed | ProcArrayLock
+ 6644 | LWLock | ProcArrayLock
(2 rows)
</programlisting>
</para>
</row>
<row>
<entry><literal>lwlock-acquire</literal></entry>
- <entry><literal>(char *, int, LWLockMode)</literal></entry>
+ <entry><literal>(char *, LWLockMode)</literal></entry>
<entry>Probe that fires when an LWLock has been acquired.
arg0 is the LWLock's tranche.
- arg1 is the LWLock's offset within its tranche.
- arg2 is the requested lock mode, either exclusive or shared.</entry>
+ arg1 is the requested lock mode, either exclusive or shared.</entry>
</row>
<row>
<entry><literal>lwlock-release</literal></entry>
- <entry><literal>(char *, int)</literal></entry>
+ <entry><literal>(char *)</literal></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 tranche.
- arg1 is the LWLock's offset within its tranche.</entry>
+ arg0 is the LWLock's tranche.</entry>
</row>
<row>
<entry><literal>lwlock-wait-start</literal></entry>
- <entry><literal>(char *, int, LWLockMode)</literal></entry>
+ <entry><literal>(char *, LWLockMode)</literal></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 tranche.
- arg1 is the LWLock's offset within its tranche.
- arg2 is the requested lock mode, either exclusive or shared.</entry>
+ arg1 is the requested lock mode, either exclusive or shared.</entry>
</row>
<row>
<entry><literal>lwlock-wait-done</literal></entry>
- <entry><literal>(char *, int, LWLockMode)</literal></entry>
+ <entry><literal>(char *, LWLockMode)</literal></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 tranche.
- arg1 is the LWLock's offset within its tranche.
- arg2 is the requested lock mode, either exclusive or shared.</entry>
+ arg1 is the requested lock mode, either exclusive or shared.</entry>
</row>
<row>
<entry><literal>lwlock-condacquire</literal></entry>
- <entry><literal>(char *, int, LWLockMode)</literal></entry>
+ <entry><literal>(char *, LWLockMode)</literal></entry>
<entry>Probe that fires when an LWLock was successfully acquired when the
caller specified no waiting.
arg0 is the LWLock's tranche.
- arg1 is the LWLock's offset within its tranche.
- arg2 is the requested lock mode, either exclusive or shared.</entry>
+ arg1 is the requested lock mode, either exclusive or shared.</entry>
</row>
<row>
<entry><literal>lwlock-condacquire-fail</literal></entry>
- <entry><literal>(char *, int, LWLockMode)</literal></entry>
+ <entry><literal>(char *, LWLockMode)</literal></entry>
<entry>Probe that fires when an LWLock was not successfully acquired when
the caller specified no waiting.
arg0 is the LWLock's tranche.
- arg1 is the LWLock's offset within its tranche.
- arg2 is the requested lock mode, either exclusive or shared.</entry>
+ arg1 is the requested lock mode, either exclusive or shared.</entry>
</row>
<row>
<entry><literal>lock-wait-start</literal></entry>
Assert(strlen(name) + 1 < SLRU_MAX_NAME_LENGTH);
strlcpy(shared->lwlock_tranche_name, name, SLRU_MAX_NAME_LENGTH);
shared->lwlock_tranche_id = tranche_id;
- shared->lwlock_tranche.name = shared->lwlock_tranche_name;
- shared->lwlock_tranche.array_base = shared->buffer_locks;
- shared->lwlock_tranche.array_stride = sizeof(LWLockPadded);
ptr += BUFFERALIGN(offset);
for (slotno = 0; slotno < nslots; slotno++)
Assert(found);
/* Register SLRU tranche in the main tranches array */
- LWLockRegisterTranche(shared->lwlock_tranche_id, &shared->lwlock_tranche);
+ LWLockRegisterTranche(shared->lwlock_tranche_id,
+ shared->lwlock_tranche_name);
/*
* Initialize the unshared control struct, including directory path. We
* WAL insertion locks.
*/
WALInsertLockPadded *WALInsertLocks;
- LWLockTranche WALInsertLockTranche;
} XLogCtlInsert;
/*
/* Initialize local copy of WALInsertLocks and register the tranche */
WALInsertLocks = XLogCtl->Insert.WALInsertLocks;
LWLockRegisterTranche(LWTRANCHE_WAL_INSERT,
- &XLogCtl->Insert.WALInsertLockTranche);
+ "wal_insert");
return;
}
memset(XLogCtl, 0, sizeof(XLogCtlData));
(WALInsertLockPadded *) allocptr;
allocptr += sizeof(WALInsertLockPadded) * NUM_XLOGINSERT_LOCKS;
- XLogCtl->Insert.WALInsertLockTranche.name = "wal_insert";
- XLogCtl->Insert.WALInsertLockTranche.array_base = WALInsertLocks;
- XLogCtl->Insert.WALInsertLockTranche.array_stride = sizeof(WALInsertLockPadded);
-
- LWLockRegisterTranche(LWTRANCHE_WAL_INSERT, &XLogCtl->Insert.WALInsertLockTranche);
+ LWLockRegisterTranche(LWTRANCHE_WAL_INSERT, "wal_insert");
for (i = 0; i < NUM_XLOGINSERT_LOCKS; i++)
{
LWLockInitialize(&WALInsertLocks[i].l.lock, LWTRANCHE_WAL_INSERT);
switch (classId)
{
- case PG_WAIT_LWLOCK_NAMED:
- event_type = "LWLockNamed";
- break;
- case PG_WAIT_LWLOCK_TRANCHE:
- event_type = "LWLockTranche";
+ case PG_WAIT_LWLOCK:
+ event_type = "LWLock";
break;
case PG_WAIT_LOCK:
event_type = "Lock";
switch (classId)
{
- case PG_WAIT_LWLOCK_NAMED:
- case PG_WAIT_LWLOCK_TRANCHE:
+ case PG_WAIT_LWLOCK:
event_name = GetLWLockIdentifier(classId, eventId);
break;
case PG_WAIT_LOCK:
typedef struct ReplicationStateCtl
{
int tranche_id;
- LWLockTranche tranche;
ReplicationState states[FLEXIBLE_ARRAY_MEMBER];
} ReplicationStateCtl;
int i;
replication_states_ctl->tranche_id = LWTRANCHE_REPLICATION_ORIGIN;
- replication_states_ctl->tranche.name = "replication_origin";
- replication_states_ctl->tranche.array_base =
- &replication_states[0].lock;
- replication_states_ctl->tranche.array_stride =
- sizeof(ReplicationState);
MemSet(replication_states, 0, ReplicationOriginShmemSize());
}
LWLockRegisterTranche(replication_states_ctl->tranche_id,
- &replication_states_ctl->tranche);
+ "replication_origin");
}
/* ---------------------------------------------------------------------------
int max_replication_slots = 0; /* the maximum number of replication
* slots */
-static LWLockTranche ReplSlotIOLWLockTranche;
-
static void ReplicationSlotDropAcquired(void);
static void ReplicationSlotDropPtr(ReplicationSlot *slot);
ShmemInitStruct("ReplicationSlot Ctl", ReplicationSlotsShmemSize(),
&found);
- ReplSlotIOLWLockTranche.name = "replication_slot_io";
- ReplSlotIOLWLockTranche.array_base =
- ((char *) ReplicationSlotCtl) + offsetof(ReplicationSlotCtlData, replication_slots) +offsetof(ReplicationSlot, io_in_progress_lock);
- ReplSlotIOLWLockTranche.array_stride = sizeof(ReplicationSlot);
LWLockRegisterTranche(LWTRANCHE_REPLICATION_SLOT_IO_IN_PROGRESS,
- &ReplSlotIOLWLockTranche);
+ "replication_slot_io");
if (!found)
{
BufferDescPadded *BufferDescriptors;
char *BufferBlocks;
LWLockMinimallyPadded *BufferIOLWLockArray = NULL;
-LWLockTranche BufferIOLWLockTranche;
-LWLockTranche BufferContentLWLockTranche;
WritebackContext BackendWritebackContext;
CkptSortItem *CkptBufferIds;
NBuffers * (Size) sizeof(LWLockMinimallyPadded),
&foundIOLocks);
- BufferIOLWLockTranche.name = "buffer_io";
- BufferIOLWLockTranche.array_base = BufferIOLWLockArray;
- BufferIOLWLockTranche.array_stride = sizeof(LWLockMinimallyPadded);
- LWLockRegisterTranche(LWTRANCHE_BUFFER_IO_IN_PROGRESS,
- &BufferIOLWLockTranche);
-
- BufferContentLWLockTranche.name = "buffer_content";
- BufferContentLWLockTranche.array_base =
- ((char *) BufferDescriptors) + offsetof(BufferDesc, content_lock);
- BufferContentLWLockTranche.array_stride = sizeof(BufferDescPadded);
- LWLockRegisterTranche(LWTRANCHE_BUFFER_CONTENT,
- &BufferContentLWLockTranche);
+ LWLockRegisterTranche(LWTRANCHE_BUFFER_IO_IN_PROGRESS, "buffer_io");
+ LWLockRegisterTranche(LWTRANCHE_BUFFER_CONTENT, "buffer_content");
/*
* The array used to sort to-be-checkpointed buffer ids is located in
static bool *KnownAssignedXidsValid;
static TransactionId latestObservedXid = InvalidTransactionId;
-/* LWLock tranche for backend locks */
-static LWLockTranche ProcLWLockTranche;
-
/*
* If we're in STANDBY_SNAPSHOT_PENDING state, standbySnapshotPendingXmin is
* the highest xid that might still be running that we don't have in
}
/* Register and initialize fields of ProcLWLockTranche */
- ProcLWLockTranche.name = "proc";
- ProcLWLockTranche.array_base = (char *) (ProcGlobal->allProcs) +
- offsetof(PGPROC, backendLock);
- ProcLWLockTranche.array_stride = sizeof(PGPROC);
- LWLockRegisterTranche(LWTRANCHE_PROC, &ProcLWLockTranche);
+ LWLockRegisterTranche(LWTRANCHE_PROC, "proc");
}
/*
#define LW_SHARED_MASK ((uint32) ((1 << 24)-1))
/*
- * This is indexed by tranche ID and stores metadata for all tranches known
+ * This is indexed by tranche ID and stores the names of all tranches known
* to the current backend.
*/
-static LWLockTranche **LWLockTrancheArray = NULL;
+static char **LWLockTrancheArray = NULL;
static int LWLockTranchesAllocated = 0;
#define T_NAME(lock) \
- (LWLockTrancheArray[(lock)->tranche]->name)
-#define T_ID(lock) \
- ((int) ((((char *) lock) - \
- ((char *) LWLockTrancheArray[(lock)->tranche]->array_base)) / \
- LWLockTrancheArray[(lock)->tranche]->array_stride))
+ (LWLockTrancheArray[(lock)->tranche])
/*
* This points to the main array of LWLocks in shared memory. Backends inherit
* where we have special measures to pass it down).
*/
LWLockPadded *MainLWLockArray = NULL;
-static LWLockTranche MainLWLockTranche;
-static LWLockTranche BufMappingLWLockTranche;
-static LWLockTranche LockManagerLWLockTranche;
-static LWLockTranche PredicateLockManagerLWLockTranche;
/*
* We use this structure to keep track of locked LWLocks for release
typedef struct lwlock_stats_key
{
int tranche;
- int instance;
+ void *instance;
} lwlock_stats_key;
typedef struct lwlock_stats
if (Trace_lwlocks)
{
uint32 state = pg_atomic_read_u32(&lock->state);
- int id = T_ID(lock);
-
- if (lock->tranche == 0 && id < NUM_INDIVIDUAL_LWLOCKS)
- ereport(LOG,
- (errhidestmt(true),
- errhidecontext(true),
- errmsg_internal("%d: %s(%s): excl %u shared %u haswaiters %u waiters %u rOK %d",
- MyProcPid,
- where, MainLWLockNames[id],
- (state & LW_VAL_EXCLUSIVE) != 0,
- state & LW_SHARED_MASK,
- (state & LW_FLAG_HAS_WAITERS) != 0,
- pg_atomic_read_u32(&lock->nwaiters),
- (state & LW_FLAG_RELEASE_OK) != 0)));
- else
- ereport(LOG,
- (errhidestmt(true),
- errhidecontext(true),
- errmsg_internal("%d: %s(%s %d): excl %u shared %u haswaiters %u waiters %u rOK %d",
- MyProcPid,
- where, T_NAME(lock), id,
- (state & LW_VAL_EXCLUSIVE) != 0,
- state & LW_SHARED_MASK,
- (state & LW_FLAG_HAS_WAITERS) != 0,
- pg_atomic_read_u32(&lock->nwaiters),
- (state & LW_FLAG_RELEASE_OK) != 0)));
+
+ ereport(LOG,
+ (errhidestmt(true),
+ errhidecontext(true),
+ errmsg_internal("%d: %s(%s %p): excl %u shared %u haswaiters %u waiters %u rOK %d",
+ MyProcPid,
+ where, T_NAME(lock), lock,
+ (state & LW_VAL_EXCLUSIVE) != 0,
+ state & LW_SHARED_MASK,
+ (state & LW_FLAG_HAS_WAITERS) != 0,
+ pg_atomic_read_u32(&lock->nwaiters),
+ (state & LW_FLAG_RELEASE_OK) != 0)));
}
}
/* hide statement & context here, otherwise the log is just too verbose */
if (Trace_lwlocks)
{
- int id = T_ID(lock);
-
- if (lock->tranche == 0 && id < NUM_INDIVIDUAL_LWLOCKS)
- ereport(LOG,
- (errhidestmt(true),
- errhidecontext(true),
- errmsg_internal("%s(%s): %s", where,
- MainLWLockNames[id], msg)));
- else
- ereport(LOG,
- (errhidestmt(true),
- errhidecontext(true),
- errmsg_internal("%s(%s %d): %s", where,
- T_NAME(lock), id, msg)));
+ ereport(LOG,
+ (errhidestmt(true),
+ errhidecontext(true),
+ errmsg_internal("%s(%s %p): %s", where,
+ T_NAME(lock), lock, msg)));
}
}
while ((lwstats = (lwlock_stats *) hash_seq_search(&scan)) != NULL)
{
fprintf(stderr,
- "PID %d lwlock %s %d: shacq %u exacq %u blk %u spindelay %u dequeue self %u\n",
- MyProcPid, LWLockTrancheArray[lwstats->key.tranche]->name,
+ "PID %d lwlock %s %p: shacq %u exacq %u blk %u spindelay %u dequeue self %u\n",
+ MyProcPid, LWLockTrancheArray[lwstats->key.tranche],
lwstats->key.instance, lwstats->sh_acquire_count,
lwstats->ex_acquire_count, lwstats->block_count,
lwstats->spin_delay_count, lwstats->dequeue_self_count);
/* Fetch or create the entry. */
key.tranche = lock->tranche;
- key.instance = T_ID(lock);
+ key.instance = lock;
lwstats = hash_search(lwlock_stats_htab, &key, HASH_ENTER, &found);
if (!found)
{
/* Initialize all individual LWLocks in main array */
for (id = 0, lock = MainLWLockArray; id < NUM_INDIVIDUAL_LWLOCKS; id++, lock++)
- LWLockInitialize(&lock->lock, LWTRANCHE_MAIN);
+ LWLockInitialize(&lock->lock, id);
/* Initialize buffer mapping LWLocks in main array */
lock = MainLWLockArray + NUM_INDIVIDUAL_LWLOCKS;
name = trancheNames;
trancheNames += strlen(request->tranche_name) + 1;
strcpy(name, request->tranche_name);
- tranche->lwLockTranche.name = name;
tranche->trancheId = LWLockNewTrancheId();
- tranche->lwLockTranche.array_base = lock;
- tranche->lwLockTranche.array_stride = sizeof(LWLockPadded);
+ tranche->trancheName = name;
for (j = 0; j < request->num_lwlocks; j++, lock++)
LWLockInitialize(&lock->lock, tranche->trancheId);
if (LWLockTrancheArray == NULL)
{
- LWLockTranchesAllocated = 32;
- LWLockTrancheArray = (LWLockTranche **)
+ LWLockTranchesAllocated = 64;
+ LWLockTrancheArray = (char **)
MemoryContextAllocZero(TopMemoryContext,
- LWLockTranchesAllocated * sizeof(LWLockTranche *));
+ LWLockTranchesAllocated * sizeof(char *));
Assert(LWLockTranchesAllocated >= LWTRANCHE_FIRST_USER_DEFINED);
}
- MainLWLockTranche.name = "main";
- MainLWLockTranche.array_base = MainLWLockArray;
- MainLWLockTranche.array_stride = sizeof(LWLockPadded);
- LWLockRegisterTranche(LWTRANCHE_MAIN, &MainLWLockTranche);
+ for (i = 0; i < NUM_INDIVIDUAL_LWLOCKS; ++i)
+ LWLockRegisterTranche(i, MainLWLockNames[i]);
- BufMappingLWLockTranche.name = "buffer_mapping";
- BufMappingLWLockTranche.array_base = MainLWLockArray + NUM_INDIVIDUAL_LWLOCKS;
- BufMappingLWLockTranche.array_stride = sizeof(LWLockPadded);
- LWLockRegisterTranche(LWTRANCHE_BUFFER_MAPPING, &BufMappingLWLockTranche);
-
- LockManagerLWLockTranche.name = "lock_manager";
- LockManagerLWLockTranche.array_base = MainLWLockArray + NUM_INDIVIDUAL_LWLOCKS +
- NUM_BUFFER_PARTITIONS;
- LockManagerLWLockTranche.array_stride = sizeof(LWLockPadded);
- LWLockRegisterTranche(LWTRANCHE_LOCK_MANAGER, &LockManagerLWLockTranche);
-
- PredicateLockManagerLWLockTranche.name = "predicate_lock_manager";
- PredicateLockManagerLWLockTranche.array_base = MainLWLockArray + NUM_INDIVIDUAL_LWLOCKS +
- NUM_BUFFER_PARTITIONS + NUM_LOCK_PARTITIONS;
- PredicateLockManagerLWLockTranche.array_stride = sizeof(LWLockPadded);
- LWLockRegisterTranche(LWTRANCHE_PREDICATE_LOCK_MANAGER, &PredicateLockManagerLWLockTranche);
+ LWLockRegisterTranche(LWTRANCHE_BUFFER_MAPPING, "buffer_mapping");
+ LWLockRegisterTranche(LWTRANCHE_LOCK_MANAGER, "lock_manager");
+ LWLockRegisterTranche(LWTRANCHE_PREDICATE_LOCK_MANAGER,
+ "predicate_lock_manager");
/* Register named tranches. */
for (i = 0; i < NamedLWLockTrancheRequests; i++)
LWLockRegisterTranche(NamedLWLockTrancheArray[i].trancheId,
- &NamedLWLockTrancheArray[i].lwLockTranche);
+ NamedLWLockTrancheArray[i].trancheName);
}
/*
* (TopMemoryContext, static variable, or similar).
*/
void
-LWLockRegisterTranche(int tranche_id, LWLockTranche *tranche)
+LWLockRegisterTranche(int tranche_id, char *tranche_name)
{
Assert(LWLockTrancheArray != NULL);
while (i <= tranche_id)
i *= 2;
- LWLockTrancheArray = (LWLockTranche **)
- repalloc(LWLockTrancheArray,
- i * sizeof(LWLockTranche *));
+ LWLockTrancheArray = (char **)
+ repalloc(LWLockTrancheArray, i * sizeof(char *));
LWLockTranchesAllocated = i;
while (j < LWLockTranchesAllocated)
LWLockTrancheArray[j++] = NULL;
}
- LWLockTrancheArray[tranche_id] = tranche;
+ LWLockTrancheArray[tranche_id] = tranche_name;
}
/*
static inline void
LWLockReportWaitStart(LWLock *lock)
{
- int lockId = T_ID(lock);
-
- if (lock->tranche == 0)
- pgstat_report_wait_start(PG_WAIT_LWLOCK_NAMED | (uint16) lockId);
- else
- pgstat_report_wait_start(PG_WAIT_LWLOCK_TRANCHE | lock->tranche);
+ pgstat_report_wait_start(PG_WAIT_LWLOCK | lock->tranche);
}
/*
const char *
GetLWLockIdentifier(uint32 classId, uint16 eventId)
{
- if (classId == PG_WAIT_LWLOCK_NAMED)
- return MainLWLockNames[eventId];
-
- Assert(classId == PG_WAIT_LWLOCK_TRANCHE);
+ Assert(classId == PG_WAIT_LWLOCK);
/*
* It is quite possible that user has registered tranche in one of the
* all of them, so we can't assume the tranche is registered here.
*/
if (eventId >= LWLockTranchesAllocated ||
- LWLockTrancheArray[eventId]->name == NULL)
+ LWLockTrancheArray[eventId] == NULL)
return "extension";
- return LWLockTrancheArray[eventId]->name;
+ return LWLockTrancheArray[eventId];
}
/*
#endif
LWLockReportWaitStart(lock);
- TRACE_POSTGRESQL_LWLOCK_WAIT_START(T_NAME(lock), T_ID(lock), mode);
+ TRACE_POSTGRESQL_LWLOCK_WAIT_START(T_NAME(lock), mode);
for (;;)
{
}
#endif
- TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(T_NAME(lock), T_ID(lock), mode);
+ TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(T_NAME(lock), mode);
LWLockReportWaitEnd();
LOG_LWDEBUG("LWLockAcquire", lock, "awakened");
result = false;
}
- TRACE_POSTGRESQL_LWLOCK_ACQUIRE(T_NAME(lock), T_ID(lock), mode);
+ TRACE_POSTGRESQL_LWLOCK_ACQUIRE(T_NAME(lock), mode);
/* Add lock to list of locks held by this backend */
held_lwlocks[num_held_lwlocks].lock = lock;
RESUME_INTERRUPTS();
LOG_LWDEBUG("LWLockConditionalAcquire", lock, "failed");
- TRACE_POSTGRESQL_LWLOCK_CONDACQUIRE_FAIL(T_NAME(lock), T_ID(lock), mode);
+ TRACE_POSTGRESQL_LWLOCK_CONDACQUIRE_FAIL(T_NAME(lock), mode);
}
else
{
/* Add lock to list of locks held by this backend */
held_lwlocks[num_held_lwlocks].lock = lock;
held_lwlocks[num_held_lwlocks++].mode = mode;
- TRACE_POSTGRESQL_LWLOCK_CONDACQUIRE(T_NAME(lock), T_ID(lock), mode);
+ TRACE_POSTGRESQL_LWLOCK_CONDACQUIRE(T_NAME(lock), mode);
}
return !mustwait;
}
#endif
LWLockReportWaitStart(lock);
- TRACE_POSTGRESQL_LWLOCK_WAIT_START(T_NAME(lock), T_ID(lock), mode);
+ TRACE_POSTGRESQL_LWLOCK_WAIT_START(T_NAME(lock), mode);
for (;;)
{
Assert(nwaiters < MAX_BACKENDS);
}
#endif
- TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(T_NAME(lock), T_ID(lock), mode);
+ TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(T_NAME(lock), mode);
LWLockReportWaitEnd();
LOG_LWDEBUG("LWLockAcquireOrWait", lock, "awakened");
/* Failed to get lock, so release interrupt holdoff */
RESUME_INTERRUPTS();
LOG_LWDEBUG("LWLockAcquireOrWait", lock, "failed");
- TRACE_POSTGRESQL_LWLOCK_ACQUIRE_OR_WAIT_FAIL(T_NAME(lock), T_ID(lock),
- mode);
+ TRACE_POSTGRESQL_LWLOCK_ACQUIRE_OR_WAIT_FAIL(T_NAME(lock), mode);
}
else
{
/* Add lock to list of locks held by this backend */
held_lwlocks[num_held_lwlocks].lock = lock;
held_lwlocks[num_held_lwlocks++].mode = mode;
- TRACE_POSTGRESQL_LWLOCK_ACQUIRE_OR_WAIT(T_NAME(lock), T_ID(lock), mode);
+ TRACE_POSTGRESQL_LWLOCK_ACQUIRE_OR_WAIT(T_NAME(lock), mode);
}
return !mustwait;
#endif
LWLockReportWaitStart(lock);
- TRACE_POSTGRESQL_LWLOCK_WAIT_START(T_NAME(lock), T_ID(lock),
- LW_EXCLUSIVE);
+ TRACE_POSTGRESQL_LWLOCK_WAIT_START(T_NAME(lock), LW_EXCLUSIVE);
for (;;)
{
}
#endif
- TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(T_NAME(lock), T_ID(lock),
- LW_EXCLUSIVE);
+ TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(T_NAME(lock), LW_EXCLUSIVE);
LWLockReportWaitEnd();
LOG_LWDEBUG("LWLockWaitForVar", lock, "awakened");
/* Now loop back and check the status of the lock again. */
}
- TRACE_POSTGRESQL_LWLOCK_ACQUIRE(T_NAME(lock), T_ID(lock), LW_EXCLUSIVE);
+ TRACE_POSTGRESQL_LWLOCK_ACQUIRE(T_NAME(lock), LW_EXCLUSIVE);
/*
* Fix the process wait semaphore's count for any absorbed wakeups.
break;
if (i < 0)
- elog(ERROR, "lock %s %d is not held", T_NAME(lock), T_ID(lock));
+ elog(ERROR, "lock %s is not held", T_NAME(lock));
mode = held_lwlocks[i].mode;
LWLockWakeup(lock);
}
- TRACE_POSTGRESQL_LWLOCK_RELEASE(T_NAME(lock), T_ID(lock));
+ TRACE_POSTGRESQL_LWLOCK_RELEASE(T_NAME(lock));
/*
* Now okay to allow cancel/die interrupts.
/* Pointer to the control object in shared memory. */
dsa_area_control *control;
- /* The lock tranche for this process. */
- LWLockTranche lwlock_tranche;
-
/* Has the mapping been pinned? */
bool mapping_pinned;
area->mapping_pinned = false;
memset(area->segment_maps, 0, sizeof(dsa_segment_map) * DSA_MAX_SEGMENTS);
area->high_segment_index = 0;
- area->lwlock_tranche.array_base = &area->control->pools[0];
- area->lwlock_tranche.array_stride = sizeof(dsa_area_pool);
- area->lwlock_tranche.name = control->lwlock_tranche_name;
- LWLockRegisterTranche(control->lwlock_tranche_id, &area->lwlock_tranche);
+ LWLockRegisterTranche(control->lwlock_tranche_id,
+ control->lwlock_tranche_name);
LWLockInitialize(&control->lock, control->lwlock_tranche_id);
for (i = 0; i < DSA_NUM_SIZE_CLASSES; ++i)
LWLockInitialize(DSA_SCLASS_LOCK(area, i),
memset(&area->segment_maps[0], 0,
sizeof(dsa_segment_map) * DSA_MAX_SEGMENTS);
area->high_segment_index = 0;
- area->lwlock_tranche.array_base = &area->control->pools[0];
- area->lwlock_tranche.array_stride = sizeof(dsa_area_pool);
- area->lwlock_tranche.name = control->lwlock_tranche_name;
- LWLockRegisterTranche(control->lwlock_tranche_id, &area->lwlock_tranche);
+ LWLockRegisterTranche(control->lwlock_tranche_id,
+ control->lwlock_tranche_name);
/* Set up the segment map for this process's mapping. */
segment_map = &area->segment_maps[0];
probe transaction__commit(LocalTransactionId);
probe transaction__abort(LocalTransactionId);
- 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__acquire__or__wait(const char *, int, LWLockMode);
- probe lwlock__acquire__or__wait__fail(const char *, int, LWLockMode);
+ probe lwlock__acquire(const char *, LWLockMode);
+ probe lwlock__release(const char *);
+ probe lwlock__wait__start(const char *, LWLockMode);
+ probe lwlock__wait__done(const char *, LWLockMode);
+ probe lwlock__condacquire(const char *, LWLockMode);
+ probe lwlock__condacquire__fail(const char *, LWLockMode);
+ probe lwlock__acquire__or__wait(const char *, LWLockMode);
+ probe lwlock__acquire__or__wait__fail(const char *, 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);
/* LWLocks */
int lwlock_tranche_id;
- LWLockTranche lwlock_tranche;
char lwlock_tranche_name[SLRU_MAX_NAME_LENGTH];
LWLockPadded *buffer_locks;
} SlruSharedData;
* Wait Classes
* ----------
*/
-#define PG_WAIT_LWLOCK_NAMED 0x01000000U
-#define PG_WAIT_LWLOCK_TRANCHE 0x02000000U
+#define PG_WAIT_LWLOCK 0x01000000U
#define PG_WAIT_LOCK 0x03000000U
#define PG_WAIT_BUFFER_PIN 0x04000000U
#define PG_WAIT_ACTIVITY 0x05000000U
struct PGPROC;
-/*
- * Prior to PostgreSQL 9.4, every lightweight lock in the system was stored
- * in a single array. For convenience and for compatibility with past
- * releases, we still have a main array, but it's now also permissible to
- * store LWLocks elsewhere in the main shared memory segment or in a dynamic
- * shared memory segment. Each array of lwlocks forms a separate "tranche".
- *
- * It's occasionally necessary to identify a particular LWLock "by name"; e.g.
- * because we wish to report the lock to dtrace. We could store a name or
- * other identifying information in the lock itself, but since it's common
- * to have many nearly-identical locks (e.g. one per buffer) this would end
- * up wasting significant amounts of memory. Instead, each lwlock stores a
- * tranche ID which tells us which array it's part of. Based on that, we can
- * figure out where the lwlock lies within the array using the data structure
- * shown below; the lock is then identified based on the tranche name and
- * computed array index. We need the array stride because the array might not
- * be an array of lwlocks, but rather some larger data structure that includes
- * one or more lwlocks per element.
- */
-typedef struct LWLockTranche
-{
- const char *name;
- void *array_base;
- Size array_stride;
-} LWLockTranche;
-
/*
* Code outside of lwlock.c should not manipulate the contents of this
* structure directly, but we have to declare it here to allow LWLocks to be
/* struct for storing named tranche information */
typedef struct NamedLWLockTranche
{
- LWLockTranche lwLockTranche;
int trancheId;
+ char *trancheName;
} NamedLWLockTranche;
extern PGDLLIMPORT NamedLWLockTranche *NamedLWLockTrancheArray;
* There is another, more flexible method of obtaining lwlocks. First, call
* LWLockNewTrancheId just once to obtain a tranche ID; this allocates from
* a shared counter. Next, each individual process using the tranche should
- * call LWLockRegisterTranche() to associate that tranche ID with appropriate
- * metadata. Finally, LWLockInitialize should be called just once per lwlock,
- * passing the tranche ID as an argument.
+ * call LWLockRegisterTranche() to associate that tranche ID with a name.
+ * Finally, LWLockInitialize should be called just once per lwlock, passing
+ * the tranche ID as an argument.
*
* It may seem strange that each process using the tranche must register it
* separately, but dynamic shared memory segments aren't guaranteed to be
* registration in the main shared memory segment wouldn't work for that case.
*/
extern int LWLockNewTrancheId(void);
-extern void LWLockRegisterTranche(int tranche_id, LWLockTranche *tranche);
+extern void LWLockRegisterTranche(int tranche_id, char *tranche_name);
extern void LWLockInitialize(LWLock *lock, int tranche_id);
/*
- * We reserve a few predefined tranche IDs. A call to LWLockNewTrancheId
- * will never return a value less than LWTRANCHE_FIRST_USER_DEFINED.
+ * Every tranche ID less than NUM_INDIVIDUAL_LWLOCKS is reserved; also,
+ * we reserve additional tranche IDs for builtin tranches not included in
+ * the set of individual LWLocks. A call to LWLockNewTrancheId will never
+ * return a value less than LWTRANCHE_FIRST_USER_DEFINED.
*/
typedef enum BuiltinTrancheIds
{
- LWTRANCHE_MAIN,
- LWTRANCHE_CLOG_BUFFERS,
+ LWTRANCHE_CLOG_BUFFERS = NUM_INDIVIDUAL_LWLOCKS,
LWTRANCHE_COMMITTS_BUFFERS,
LWTRANCHE_SUBTRANS_BUFFERS,
LWTRANCHE_MXACTOFFSET_BUFFERS,