Hello.
I did some refactoring to previous patch. Improvements:
1) Wait is determined by class and event without affecting to atomic
usage of it.
They are still stored in one variable. This improvement gives an
opportunity to make
more detailed views later (waits can be grouped by class).
2) Only active wait of each backend is visible. pg_report_wait_end()
function called
on the end of wait and clears it.
3) Wait name determination was optimized (last version used cycles for
each of them,
and was very heavy). I added lazy `group` field to LWLock, which used as
index in
lwlock names array.
4) New wait can be added by more simpler way. For example an individual
lwlock
requires only specifying its name in LWLock names arrayb
5) Added new types of waits: Storage, Network, Latch
This patch is more informative and it'll be easier to extend.
Sample:
b1=# select pid, wait_event from pg_stat_activity;
pid | wait_event
-------+------------------------------
17099 | LWLocks: BufferCleanupLock
17100 | Locks: Transaction
17101 | LWLocks: BufferPartitionLock
17102 |
17103 | Network: READ
17086 |
(6 rows)
--
Ildus Kurbangaliev
Postgres Professional: http://www.postgrespro.com
The Russian Postgres Company
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 1dd31b3..91dca5c 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -4661,8 +4661,10 @@ XLOGShmemInit(void)
LWLockRegisterTranche(XLogCtl->Insert.WALInsertLockTrancheId, &XLogCtl->Insert.WALInsertLockTranche);
for (i = 0; i < NUM_XLOGINSERT_LOCKS; i++)
{
- LWLockInitialize(&WALInsertLocks[i].l.lock,
+ LWLock *lock = &WALInsertLocks[i].l.lock;
+ LWLockInitialize(lock,
XLogCtl->Insert.WALInsertLockTrancheId);
+ lock->group = LWLOCK_WAL_INSERT;
WALInsertLocks[i].l.insertingAt = InvalidXLogRecPtr;
}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index e82a53a..fd04258 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -621,6 +621,7 @@ CREATE VIEW pg_stat_activity AS
S.query_start,
S.state_change,
S.waiting,
+ S.wait_event,
S.state,
S.backend_xid,
s.backend_xmin,
diff --git a/src/backend/libpq/be-secure.c b/src/backend/libpq/be-secure.c
index 4a650cc..e5c023f 100644
--- a/src/backend/libpq/be-secure.c
+++ b/src/backend/libpq/be-secure.c
@@ -36,6 +36,7 @@
#include "tcop/tcopprot.h"
#include "utils/memutils.h"
#include "storage/proc.h"
+#include "pgstat.h"
char *ssl_cert_file;
@@ -129,6 +130,8 @@ secure_read(Port *port, void *ptr, size_t len)
ssize_t n;
int waitfor;
+ pgstat_report_wait_start(WAIT_NETWORK, WAIT_NETWORK_READ);
+
retry:
#ifdef USE_SSL
waitfor = 0;
@@ -175,6 +178,7 @@ retry:
* interrupts from being processed.
*/
ProcessClientReadInterrupt(false);
+ pgstat_report_wait_end();
return n;
}
@@ -209,6 +213,8 @@ secure_write(Port *port, void *ptr, size_t len)
ssize_t n;
int waitfor;
+ pgstat_report_wait_start(WAIT_NETWORK, WAIT_NETWORK_WRITE);
+
retry:
waitfor = 0;
#ifdef USE_SSL
@@ -254,6 +260,7 @@ retry:
* interrupts from being processed.
*/
ProcessClientWriteInterrupt(false);
+ pgstat_report_wait_end();
return n;
}
diff --git a/src/backend/port/unix_latch.c b/src/backend/port/unix_latch.c
index 90ec4f8..dcebfe4 100644
--- a/src/backend/port/unix_latch.c
+++ b/src/backend/port/unix_latch.c
@@ -55,6 +55,7 @@
#include "storage/latch.h"
#include "storage/pmsignal.h"
#include "storage/shmem.h"
+#include "pgstat.h"
/* Are we currently in WaitLatch? The signal handler would like to know. */
static volatile sig_atomic_t waiting = false;
@@ -262,6 +263,8 @@ WaitLatchOrSocket(volatile Latch *latch, int wakeEvents, pgsocket sock,
#endif
}
+ pgstat_report_wait_start(WAIT_LATCH, 0);
+
waiting = true;
do
{
@@ -500,6 +503,7 @@ WaitLatchOrSocket(volatile Latch *latch, int wakeEvents, pgsocket sock,
}
} while (result == 0);
waiting = false;
+ pgstat_report_wait_end();
return result;
}
diff --git a/src/backend/port/win32_latch.c b/src/backend/port/win32_latch.c
index 0e3aaee..92bb5a0 100644
--- a/src/backend/port/win32_latch.c
+++ b/src/backend/port/win32_latch.c
@@ -177,6 +177,8 @@ WaitLatchOrSocket(volatile Latch *latch, int wakeEvents, pgsocket sock,
/* Ensure that signals are serviced even if latch is already set */
pgwin32_dispatch_queued_signals();
+ pgstat_report_wait_start(WAIT_LATCH, 0);
+
do
{
/*
@@ -278,6 +280,8 @@ WaitLatchOrSocket(volatile Latch *latch, int wakeEvents, pgsocket sock,
}
} while (result == 0);
+ pgstat_report_wait_end();
+
/* Clean up the event object we created for the socket */
if (sockevent != WSA_INVALID_EVENT)
{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index e9fbc38..b379f7f 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -54,6 +54,7 @@
#include "storage/fd.h"
#include "storage/ipc.h"
#include "storage/latch.h"
+#include "storage/lwlock.h"
#include "storage/pg_shmem.h"
#include "storage/procsignal.h"
#include "storage/sinvaladt.h"
@@ -99,7 +100,6 @@
#define PGSTAT_TAB_HASH_SIZE 512
#define PGSTAT_FUNCTION_HASH_SIZE 512
-
/* ----------
* GUC parameters
* ----------
@@ -242,6 +242,16 @@ static volatile bool got_SIGHUP = false;
*/
static instr_time total_func_time;
+/* Names of WAIT_CLASSES */
+static const char *WAIT_CLASS_NAMES[] =
+{
+ "",
+ "LWLocks",
+ "Locks",
+ "Storage",
+ "Latch",
+ "Network"
+};
/* ----------
* Local function forward declarations
@@ -2918,6 +2928,89 @@ pgstat_report_waiting(bool waiting)
beentry->st_waiting = waiting;
}
+/*
+ * pgstat_get_wait_class_name() -
+ *
+ * Return wait class name for given class
+ */
+
+const char *
+pgstat_get_wait_class_name(uint8 classId)
+{
+ return WAIT_CLASS_NAMES[classId];
+}
+
+/*
+ * pgstat_get_wait_event_name() -
+ *
+ * Return wait event name for the given class and event
+ */
+const char *
+pgstat_get_wait_event_name(uint8 classId, uint8 eventId)
+{
+ static const char *eventsIO[] = {"READ", "WRITE"};
+ static const char *empty = "";
+
+ switch (classId)
+ {
+ case WAIT_LOCK: return LOCK_NAMES[eventId];
+ case WAIT_LWLOCK: return LWLOCK_GROUPS[eventId];
+ case WAIT_IO: /* fallthrough */;
+ case WAIT_NETWORK: return eventsIO[eventId];
+ case WAIT_LATCH: return WAIT_CLASS_NAMES[WAIT_LATCH];
+ };
+ return empty;
+}
+
+/* ----------
+ * pgstat_report_wait_start() -
+ *
+ * Called from backends to report wait event type information.
+ *
+ * NB: this *must* be able to survive being called before MyBEEntry has been
+ * initialized.
+ * ----------
+ */
+void
+pgstat_report_wait_start(uint8 classId, uint8 eventId)
+{
+ volatile PgBackendStatus *beentry = MyBEEntry;
+
+ if (!pgstat_track_activities || !beentry)
+ return;
+
+ /* prevent nested waits */
+ if (beentry->st_wait_nested++ > 0)
+ return;
+
+ /*
+ * Since this is a uint32 field in a struct that only this process
+ * may modify, there seems no need to bother with the st_changecount
+ * protocol. The update must appear atomic in any case.
+ */
+ beentry->st_wait_data = ((uint32)classId << 8) + eventId;
+}
+
+/* ----------
+ * pgstat_report_wait_end() -
+ *
+ * Called from backends, indicates that wait was ended
+ * ---------
+ */
+void
+pgstat_report_wait_end()
+{
+ volatile PgBackendStatus *beentry = MyBEEntry;
+
+ if (!pgstat_track_activities || !beentry)
+ return;
+
+ /* prevent nested waits */
+ if ((--beentry->st_wait_nested) > 0)
+ return;
+
+ beentry->st_wait_data = 0;
+}
/* ----------
* pgstat_read_current_status() -
diff --git a/src/backend/replication/logical/origin.c b/src/backend/replication/logical/origin.c
index f4ba86e..dc1c3ee 100644
--- a/src/backend/replication/logical/origin.c
+++ b/src/backend/replication/logical/origin.c
@@ -479,8 +479,11 @@ ReplicationOriginShmemInit(void)
MemSet(replication_states, 0, ReplicationOriginShmemSize());
for (i = 0; i < max_replication_slots; i++)
- LWLockInitialize(&replication_states[i].lock,
- replication_states_ctl->tranche_id);
+ {
+ LWLock *lock = &replication_states[i].lock;
+ LWLockInitialize(lock, replication_states_ctl->tranche_id);
+ lock->group = LWLOCK_REPLICATIONORIGIN;
+ }
}
LWLockRegisterTranche(replication_states_ctl->tranche_id,
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index e4b25587..26452aa 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -3239,6 +3239,9 @@ LockBufferForCleanup(Buffer buffer)
bufHdr = GetBufferDescriptor(buffer - 1);
+ /* Report the wait */
+ pgstat_report_wait_start(WAIT_LWLOCK, LWLOCK_BUFFER_CLEANUP);
+
for (;;)
{
/* Try to acquire lock */
@@ -3264,6 +3267,7 @@ LockBufferForCleanup(Buffer buffer)
UnlockBufHdr(bufHdr);
LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
+
/* Wait to be signaled by UnpinBuffer() */
if (InHotStandby)
{
@@ -3294,6 +3298,8 @@ LockBufferForCleanup(Buffer buffer)
PinCountWaitBuf = NULL;
/* Loop back and try again */
}
+
+ pgstat_report_wait_end();
}
/*
diff --git a/src/backend/storage/lmgr/lock.c b/src/backend/storage/lmgr/lock.c
index 1eb2d4b..c3a6621 100644
--- a/src/backend/storage/lmgr/lock.c
+++ b/src/backend/storage/lmgr/lock.c
@@ -54,6 +54,20 @@ int max_locks_per_xact; /* set by guc.c */
#define NLOCKENTS() \
mul_size(max_locks_per_xact, add_size(MaxBackends, max_prepared_xacts))
+/* Lock names. For monitoring purposes */
+const char *LOCK_NAMES[] =
+{
+ "Relation",
+ "RelationExtend",
+ "Page",
+ "Tuple",
+ "Transaction",
+ "VirtualTransaction",
+ "SpeculativeToken",
+ "Object",
+ "Userlock",
+ "Advisory"
+};
/*
* Data structures defining the semantics of the standard lock methods.
@@ -1591,6 +1605,7 @@ WaitOnLock(LOCALLOCK *locallock, ResourceOwner owner)
new_status[len] = '\0'; /* truncate off " waiting" */
}
pgstat_report_waiting(true);
+ pgstat_report_wait_start(WAIT_LOCK, locallock->tag.lock.locktag_type);
awaitedLock = locallock;
awaitedOwner = owner;
@@ -1639,6 +1654,7 @@ WaitOnLock(LOCALLOCK *locallock, ResourceOwner owner)
/* Report change to non-waiting status */
pgstat_report_waiting(false);
+ pgstat_report_wait_end();
if (update_process_title)
{
set_ps_display(new_status, false);
@@ -1654,6 +1670,7 @@ WaitOnLock(LOCALLOCK *locallock, ResourceOwner owner)
/* Report change to non-waiting status */
pgstat_report_waiting(false);
+ pgstat_report_wait_end();
if (update_process_title)
{
set_ps_display(new_status, false);
diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index 46cab49..6b20bd3 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -80,6 +80,7 @@
#include "access/subtrans.h"
#include "commands/async.h"
#include "miscadmin.h"
+#include "pgstat.h"
#include "pg_trace.h"
#include "postmaster/postmaster.h"
#include "replication/slot.h"
@@ -129,6 +130,10 @@ static int LWLockTranchesAllocated = 0;
LWLockPadded *MainLWLockArray = NULL;
static LWLockTranche MainLWLockTranche;
+/* This points to array of LWLock offsets */
+int *main_lwlock_groups = NULL;
+static bool lwlock_group_notified = false;
+
/*
* We use this structure to keep track of locked LWLocks for release
* during error recovery. Normally, only a few will be held at once, but
@@ -152,6 +157,81 @@ static bool lock_addin_request_allowed = true;
static inline bool LWLockAcquireCommon(LWLock *l, LWLockMode mode,
uint64 *valptr, uint64 val);
+static void LWLockReportStat(LWLock *lock);
+
+/* We keep all LWLock group names here. Order is very important.
+ * Index is equal to lwlock group field
+ */
+const char *LWLOCK_GROUPS[] =
+{
+ /* Individual LWLocks */
+ "", /* formely was BufFreelistLock */
+ "ShmemIndexLock",
+ "OidGenLock",
+ "XidGenLock",
+ "ProcArrayLock",
+ "SInvalReadLock",
+ "SInvalWriteLock",
+ "WALBufMappingLock",
+ "WALWriteLock",
+ "ControlFileLock",
+ "CheckpointLock",
+ "CLogControlLock",
+ "SubtransControlLock",
+ "MultiXactGenLock",
+ "MultiXactOffsetControlLock",
+ "MultiXactMemberControlLock",
+ "RelCacheInitLock",
+ "CheckpointerCommLock",
+ "TwoPhaseStateLock",
+ "TablespaceCreateLock",
+ "BtreeVacuumLock",
+ "AddinShmemInitLock",
+ "AutovacuumLock",
+ "AutovacuumScheduleLock",
+ "SyncScanLock",
+ "RelationMappingLock",
+ "AsyncCtlLock",
+ "AsyncQueueLock",
+ "SerializableXactHashLock",
+ "SerializableFinishedListLock",
+ "SerializablePredicateLockListLock",
+ "OldSerXidLock",
+ "SyncRepLock",
+ "BackgroundWorkerLock",
+ "DynamicSharedMemoryControlLock",
+ "AutoFileLock",
+ "ReplicationSlotAllocationLock",
+ "ReplicationSlotControlLock",
+ "CommitTsControlLock",
+ "CommitTsLock",
+ "ReplicationOriginLock",
+
+ /* LWLocks defined by offset */
+ "BufferPartitionLock",
+ "LockManagerPartitionLock",
+ "PredicatePartitionLock",
+
+ /* Dynamic LWLocks */
+ "SharedBufferLocks",
+ "ProcessLock",
+ "CLogBufferLock",
+ "CommitTsBufferLock",
+ "SubtransBufferLock",
+ "MultixactBufferLock",
+ "AsyncBufferLock",
+ "OldSerializalbeXidBufferLock",
+ "ReplicationSlotLock",
+ "UserDefinedLock",
+ /*
+ * LWLocks in new tranches must be added to end of this array
+ * New lwlocks in main tranche must be added
+ * before this comment
+ */
+ "BufferCleanupLock",
+ "WALInsertLocks",
+ "ReplicationOriginLocks",
+};
#ifdef LWLOCK_STATS
typedef struct lwlock_stats_key
@@ -317,64 +397,141 @@ get_lwlock_stats_entry(LWLock *lock)
}
#endif /* LWLOCK_STATS */
+/*
+ * Initiate 2-dimensional array with lwlock groups.
+ * First dimension contains count of lwlocks contained in this group
+ * Second dimension contains wait type (index of group)
+ */
+static int
+init_main_lwlock_groups()
+{
+ int idx = 0;
+ if (main_lwlock_groups == NULL)
+ {
+ main_lwlock_groups = MemoryContextAllocZero(TopMemoryContext,
+ sizeof(int) * NUM_ADD_LWLOCK_GROUPS * 2);
+ }
+
+ /* bufmap lwlocks */
+ main_lwlock_groups[0] = LOCK_MANAGER_LWLOCK_OFFSET;
+ main_lwlock_groups[1] = NUM_INDIVIDUAL_LWLOCKS;
+
+ /* lock manager lwlocks */
+ idx++;
+ main_lwlock_groups[idx * 2] = PREDICATELOCK_MANAGER_LWLOCK_OFFSET;
+ main_lwlock_groups[idx * 2 + 1] = NUM_INDIVIDUAL_LWLOCKS + idx;
+
+ /* predicate lwlocks */
+ idx++;
+ main_lwlock_groups[idx * 2] = NUM_FIXED_LWLOCKS;
+ main_lwlock_groups[idx * 2 + 1] = NUM_INDIVIDUAL_LWLOCKS + idx;
+ return idx;
+}
/*
- * Compute number of LWLocks to allocate in the main array.
+ * Gets lwlock group. If this lwlock from main tranche, returns index
+ * from `main_lwlock_groups`, for other tranches that group must be
+ * defined manually (by defining it in lwlock.h, and then assigning group in
+ * tranche lwlocks creation)
*/
static int
-NumLWLocks(void)
+get_lwlock_group(volatile LWLock *lock)
{
- int numLocks;
+ int group = -1;
+ if (lock->tranche == 0)
+ {
+ int numLocks, i;
+ int idx = (union LWLockPadded *)lock - MainLWLockArray;
- /*
- * Possibly this logic should be spread out among the affected modules,
- * the same way that shmem space estimation is done. But for now, there
- * are few enough users of LWLocks that we can get away with just keeping
- * the knowledge here.
- */
+ if (idx < NUM_INDIVIDUAL_LWLOCKS)
+ group = idx;
- /* Predefined LWLocks */
- numLocks = NUM_FIXED_LWLOCKS;
+ numLocks = NUM_INDIVIDUAL_LWLOCKS;
+ for (i=0; i < NUM_ADD_LWLOCK_GROUPS; i++)
+ {
+ numLocks += main_lwlock_groups[i*2];
+ if (idx < numLocks)
+ {
+ group = main_lwlock_groups[i*2+1];
+ break;
+ }
+ }
+ }
+ else if (!lwlock_group_notified)
+ {
+ elog(INFO, "LWLock with tranche id #%d not monitored", lock->tranche);
+ lwlock_group_notified = true;
+ }
- /* bufmgr.c needs two for each shared buffer */
- numLocks += 2 * NBuffers;
+ return group;
+}
- /* proc.c needs one for each backend or auxiliary process */
- numLocks += MaxBackends + NUM_AUXILIARY_PROCS;
+/*
+ * Compute number of LWLocks to allocate in the main array.
+ */
+static int
+NumLWLocks(void)
+{
+ int numLocks, i;
+ int sizes[NUM_DYN_LWLOCK_GROUPS] = {
+ /* bufmgr.c needs two for each shared buffer */
+ 2 * NBuffers,
+
+ /* proc.c needs one for each backend or auxiliary process */
+ MaxBackends + NUM_AUXILIARY_PROCS,
- /* clog.c needs one per CLOG buffer */
- numLocks += CLOGShmemBuffers();
+ /* clog.c needs one per CLOG buffer */
+ CLOGShmemBuffers(),
- /* commit_ts.c needs one per CommitTs buffer */
- numLocks += CommitTsShmemBuffers();
+ /* commit_ts.c needs one per CommitTs buffer */
+ CommitTsShmemBuffers(),
- /* subtrans.c needs one per SubTrans buffer */
- numLocks += NUM_SUBTRANS_BUFFERS;
+ /* subtrans.c needs one per SubTrans buffer */
+ NUM_SUBTRANS_BUFFERS,
- /* multixact.c needs two SLRU areas */
- numLocks += NUM_MXACTOFFSET_BUFFERS + NUM_MXACTMEMBER_BUFFERS;
+ /* multixact.c needs two SLRU areas */
+ NUM_MXACTOFFSET_BUFFERS + NUM_MXACTMEMBER_BUFFERS,
- /* async.c needs one per Async buffer */
- numLocks += NUM_ASYNC_BUFFERS;
+ /* async.c needs one per Async buffer */
+ NUM_ASYNC_BUFFERS,
- /* predicate.c needs one per old serializable xid buffer */
- numLocks += NUM_OLDSERXID_BUFFERS;
+ /* predicate.c needs one per old serializable xid buffer */
+ NUM_OLDSERXID_BUFFERS,
- /* slot.c needs one for each slot */
- numLocks += max_replication_slots;
+ /* slot.c needs one for each slot */
+ max_replication_slots,
+
+ /*
+ * Add any requested by loadable modules; for backwards-compatibility
+ * reasons, allocate at least NUM_USER_DEFINED_LWLOCKS of them even if
+ * there are no explicit requests.
+ */
+ Max(lock_addin_request, NUM_USER_DEFINED_LWLOCKS)
+ };
+ int idx = init_main_lwlock_groups();
/*
- * Add any requested by loadable modules; for backwards-compatibility
- * reasons, allocate at least NUM_USER_DEFINED_LWLOCKS of them even if
- * there are no explicit requests.
+ * Possibly this logic should be spread out among the affected modules,
+ * the same way that shmem space estimation is done. But for now, there
+ * are few enough users of LWLocks that we can get away with just keeping
+ * the knowledge here.
*/
- lock_addin_request_allowed = false;
- numLocks += Max(lock_addin_request, NUM_USER_DEFINED_LWLOCKS);
+
+ /* Predefined LWLocks */
+ numLocks = NUM_FIXED_LWLOCKS;
+ for (i=0; i < NUM_DYN_LWLOCK_GROUPS; ++i)
+ {
+ numLocks += sizes[i];
+ idx++;
+ main_lwlock_groups[idx*2] = numLocks;
+ main_lwlock_groups[idx*2+1] = NUM_INDIVIDUAL_LWLOCKS + idx;
+ }
+
+ lock_addin_request_allowed = false;
return numLocks;
}
-
/*
* RequestAddinLWLocks
* Request that extra LWLocks be allocated for use by
@@ -568,10 +725,29 @@ LWLockInitialize(LWLock *lock, int tranche_id)
pg_atomic_init_u32(&lock->nwaiters, 0);
#endif
lock->tranche = tranche_id;
+ lock->group = -1;
dlist_init(&lock->waiters);
}
/*
+ * Report wait event for light-weight locks.
+ *
+ * This function will be used by all the light-weight lock calls which
+ * needs to wait to acquire the lock. This function distinguishes wait
+ * event based on trance name and lock id.
+ */
+static void
+LWLockReportStat(LWLock *lock)
+{
+ if (lock->group == -1)
+ lock->group = get_lwlock_group(lock);
+
+ /* Assert here will work in case of new tranches */
+ Assert(lock->group >= 0);
+ pgstat_report_wait_start(WAIT_LWLOCK, lock->group);
+}
+
+/*
* Internal function that tries to atomically acquire the lwlock in the passed
* in mode.
*
@@ -1037,6 +1213,9 @@ LWLockAcquireCommon(LWLock *lock, LWLockMode mode, uint64 *valptr, uint64 val)
TRACE_POSTGRESQL_LWLOCK_WAIT_START(T_NAME(lock), T_ID(lock), mode);
+ /* Report the wait */
+ LWLockReportStat(lock);
+
for (;;)
{
PGSemaphoreLock(&proc->sem);
@@ -1057,6 +1236,7 @@ LWLockAcquireCommon(LWLock *lock, LWLockMode mode, uint64 *valptr, uint64 val)
}
#endif
+ pgstat_report_wait_end();
TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(T_NAME(lock), T_ID(lock), mode);
LOG_LWDEBUG("LWLockAcquire", lock, "awakened");
@@ -1199,6 +1379,9 @@ LWLockAcquireOrWait(LWLock *lock, LWLockMode mode)
#endif
TRACE_POSTGRESQL_LWLOCK_WAIT_START(T_NAME(lock), T_ID(lock), mode);
+ /* Report the wait */
+ LWLockReportStat(lock);
+
for (;;)
{
PGSemaphoreLock(&proc->sem);
@@ -1215,6 +1398,8 @@ LWLockAcquireOrWait(LWLock *lock, LWLockMode mode)
Assert(nwaiters < MAX_BACKENDS);
}
#endif
+
+ pgstat_report_wait_end();
TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(T_NAME(lock), T_ID(lock), mode);
LOG_LWDEBUG("LWLockAcquireOrWait", lock, "awakened");
@@ -1401,6 +1586,9 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
TRACE_POSTGRESQL_LWLOCK_WAIT_START(T_NAME(lock), T_ID(lock),
LW_EXCLUSIVE);
+ /* Report the wait */
+ LWLockReportStat(lock);
+
for (;;)
{
PGSemaphoreLock(&proc->sem);
@@ -1418,6 +1606,7 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
}
#endif
+ pgstat_report_wait_end();
TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(T_NAME(lock), T_ID(lock),
LW_EXCLUSIVE);
diff --git a/src/backend/storage/smgr/md.c b/src/backend/storage/smgr/md.c
index 42a43bb..71661d3 100644
--- a/src/backend/storage/smgr/md.c
+++ b/src/backend/storage/smgr/md.c
@@ -37,6 +37,7 @@
#include "utils/hsearch.h"
#include "utils/memutils.h"
#include "pg_trace.h"
+#include "pgstat.h"
/* intervals for calling AbsorbFsyncRequests in mdsync and mdpostckpt */
@@ -674,6 +675,8 @@ mdread(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
int nbytes;
MdfdVec *v;
+ pgstat_report_wait_start(WAIT_IO, WAIT_IO_READ);
+
TRACE_POSTGRESQL_SMGR_MD_READ_START(forknum, blocknum,
reln->smgr_rnode.node.spcNode,
reln->smgr_rnode.node.dbNode,
@@ -702,6 +705,8 @@ mdread(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
nbytes,
BLCKSZ);
+ pgstat_report_wait_end();
+
if (nbytes != BLCKSZ)
{
if (nbytes < 0)
@@ -749,6 +754,8 @@ mdwrite(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
Assert(blocknum < mdnblocks(reln, forknum));
#endif
+ pgstat_report_wait_start(WAIT_IO, WAIT_IO_WRITE);
+
TRACE_POSTGRESQL_SMGR_MD_WRITE_START(forknum, blocknum,
reln->smgr_rnode.node.spcNode,
reln->smgr_rnode.node.dbNode,
@@ -777,6 +784,8 @@ mdwrite(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
nbytes,
BLCKSZ);
+ pgstat_report_wait_end();
+
if (nbytes != BLCKSZ)
{
if (nbytes < 0)
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index f7c9bf6..f2813c9 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -530,7 +530,7 @@ pg_stat_get_backend_idset(PG_FUNCTION_ARGS)
Datum
pg_stat_get_activity(PG_FUNCTION_ARGS)
{
-#define PG_STAT_GET_ACTIVITY_COLS 22
+#define PG_STAT_GET_ACTIVITY_COLS 23
int num_backends = pgstat_fetch_stat_numbackends();
int curr_backend;
int pid = PG_ARGISNULL(0) ? -1 : PG_GETARG_INT32(0);
@@ -617,28 +617,28 @@ pg_stat_get_activity(PG_FUNCTION_ARGS)
nulls[3] = true;
if (TransactionIdIsValid(local_beentry->backend_xid))
- values[14] = TransactionIdGetDatum(local_beentry->backend_xid);
+ values[15] = TransactionIdGetDatum(local_beentry->backend_xid);
else
- nulls[14] = true;
+ nulls[15] = true;
if (TransactionIdIsValid(local_beentry->backend_xmin))
- values[15] = TransactionIdGetDatum(local_beentry->backend_xmin);
+ values[16] = TransactionIdGetDatum(local_beentry->backend_xmin);
else
- nulls[15] = true;
+ nulls[16] = true;
if (beentry->st_ssl)
{
- values[16] = BoolGetDatum(true); /* ssl */
- values[17] = CStringGetTextDatum(beentry->st_sslstatus->ssl_version);
- values[18] = CStringGetTextDatum(beentry->st_sslstatus->ssl_cipher);
- values[19] = Int32GetDatum(beentry->st_sslstatus->ssl_bits);
- values[20] = BoolGetDatum(beentry->st_sslstatus->ssl_compression);
- values[21] = CStringGetTextDatum(beentry->st_sslstatus->ssl_clientdn);
+ values[17] = BoolGetDatum(true); /* ssl */
+ values[18] = CStringGetTextDatum(beentry->st_sslstatus->ssl_version);
+ values[19] = CStringGetTextDatum(beentry->st_sslstatus->ssl_cipher);
+ values[20] = Int32GetDatum(beentry->st_sslstatus->ssl_bits);
+ values[21] = BoolGetDatum(beentry->st_sslstatus->ssl_compression);
+ values[22] = CStringGetTextDatum(beentry->st_sslstatus->ssl_clientdn);
}
else
{
- values[16] = BoolGetDatum(false); /* ssl */
- nulls[17] = nulls[18] = nulls[19] = nulls[20] = nulls[21] = true;
+ values[17] = BoolGetDatum(false); /* ssl */
+ nulls[18] = nulls[19] = nulls[20] = nulls[21] = nulls[22] = true;
}
/* Values only available to role member */
@@ -674,34 +674,58 @@ pg_stat_get_activity(PG_FUNCTION_ARGS)
values[5] = CStringGetTextDatum(beentry->st_activity);
values[6] = BoolGetDatum(beentry->st_waiting);
- if (beentry->st_xact_start_timestamp != 0)
- values[7] = TimestampTzGetDatum(beentry->st_xact_start_timestamp);
+ if (beentry->st_wait_data > 0)
+ {
+ /*
+ * Reassigning of value is important, because st_wait_data
+ * can change if we read class and event separately
+ */
+
+ uint32 wait_data = beentry->st_wait_data;
+ uint8 classId = (uint8)(wait_data >> 8);
+ uint8 eventId = (uint8)wait_data;
+ const char *class_name = pgstat_get_wait_class_name(classId);
+ const char *event_name = pgstat_get_wait_event_name(classId,
+ eventId);
+
+ int size = strlen(class_name) + strlen(event_name) + 3;
+ char *wait_name = (char *)palloc(size);
+ snprintf(wait_name, size, "%s: %s", class_name, event_name);
+
+ values[7] = CStringGetTextDatum(wait_name);
+ pfree(wait_name);
+ }
else
nulls[7] = true;
- if (beentry->st_activity_start_timestamp != 0)
- values[8] = TimestampTzGetDatum(beentry->st_activity_start_timestamp);
+ if (beentry->st_xact_start_timestamp != 0)
+ values[8] = TimestampTzGetDatum(beentry->st_xact_start_timestamp);
else
nulls[8] = true;
- if (beentry->st_proc_start_timestamp != 0)
- values[9] = TimestampTzGetDatum(beentry->st_proc_start_timestamp);
+ if (beentry->st_activity_start_timestamp != 0)
+ values[9] = TimestampTzGetDatum(beentry->st_activity_start_timestamp);
else
nulls[9] = true;
- if (beentry->st_state_start_timestamp != 0)
- values[10] = TimestampTzGetDatum(beentry->st_state_start_timestamp);
+ if (beentry->st_proc_start_timestamp != 0)
+ values[10] = TimestampTzGetDatum(beentry->st_proc_start_timestamp);
else
nulls[10] = true;
+ if (beentry->st_state_start_timestamp != 0)
+ values[11] = TimestampTzGetDatum(beentry->st_state_start_timestamp);
+ else
+ nulls[11] = true;
+
/* A zeroed client addr means we don't know */
memset(&zero_clientaddr, 0, sizeof(zero_clientaddr));
if (memcmp(&(beentry->st_clientaddr), &zero_clientaddr,
sizeof(zero_clientaddr)) == 0)
{
- nulls[11] = true;
nulls[12] = true;
nulls[13] = true;
+ nulls[14] = true;
}
else
{
@@ -725,20 +749,20 @@ pg_stat_get_activity(PG_FUNCTION_ARGS)
if (ret == 0)
{
clean_ipv6_addr(beentry->st_clientaddr.addr.ss_family, remote_host);
- values[11] = DirectFunctionCall1(inet_in,
+ values[12] = DirectFunctionCall1(inet_in,
CStringGetDatum(remote_host));
if (beentry->st_clienthostname &&
beentry->st_clienthostname[0])
- values[12] = CStringGetTextDatum(beentry->st_clienthostname);
+ values[13] = CStringGetTextDatum(beentry->st_clienthostname);
else
- nulls[12] = true;
- values[13] = Int32GetDatum(atoi(remote_port));
+ nulls[13] = true;
+ values[14] = Int32GetDatum(atoi(remote_port));
}
else
{
- nulls[11] = true;
nulls[12] = true;
nulls[13] = true;
+ nulls[14] = true;
}
}
else if (beentry->st_clientaddr.addr.ss_family == AF_UNIX)
@@ -749,16 +773,16 @@ pg_stat_get_activity(PG_FUNCTION_ARGS)
* connections we have no permissions to view, or with
* errors.
*/
- nulls[11] = true;
nulls[12] = true;
- values[13] = DatumGetInt32(-1);
+ nulls[13] = true;
+ values[14] = DatumGetInt32(-1);
}
else
{
/* Unknown address type, should never happen */
- nulls[11] = true;
nulls[12] = true;
nulls[13] = true;
+ nulls[14] = true;
}
}
}
@@ -775,6 +799,7 @@ pg_stat_get_activity(PG_FUNCTION_ARGS)
nulls[11] = true;
nulls[12] = true;
nulls[13] = true;
+ nulls[14] = true;
}
tuplestore_putvalues(tupstore, tupdesc, values, nulls);
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 1d68ad7..05bc0af 100644
--- a/src/include/catalog/pg_proc.h
+++ b/src/include/catalog/pg_proc.h
@@ -2777,7 +2777,7 @@ DATA(insert OID = 3057 ( pg_stat_get_autoanalyze_count PGNSP PGUID 12 1 0 0 0 f
DESCR("statistics: number of auto analyzes for a table");
DATA(insert OID = 1936 ( pg_stat_get_backend_idset PGNSP PGUID 12 1 100 0 0 f f f f t t s 0 0 23 "" _null_ _null_ _null_ _null_ _null_ pg_stat_get_backend_idset _null_ _null_ _null_ ));
DESCR("statistics: currently active backend IDs");
-DATA(insert OID = 2022 ( pg_stat_get_activity PGNSP PGUID 12 1 100 0 0 f f f f f t s 1 0 2249 "23" "{23,26,23,26,25,25,25,16,1184,1184,1184,1184,869,25,23,28,28,16,25,25,23,16,25}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{pid,datid,pid,usesysid,application_name,state,query,waiting,xact_start,query_start,backend_start,state_change,client_addr,client_hostname,client_port,backend_xid,backend_xmin,ssl,sslversion,sslcipher,sslbits,sslcompression,sslclientdn}" _null_ _null_ pg_stat_get_activity _null_ _null_ _null_ ));
+DATA(insert OID = 2022 ( pg_stat_get_activity PGNSP PGUID 12 1 100 0 0 f f f f f t s 1 0 2249 "23" "{23,26,23,26,25,25,25,16,25,1184,1184,1184,1184,869,25,23,28,28,16,25,25,23,16,25}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{pid,datid,pid,usesysid,application_name,state,query,waiting,wait_event,xact_start,query_start,backend_start,state_change,client_addr,client_hostname,client_port,backend_xid,backend_xmin,ssl,sslversion,sslcipher,sslbits,sslcompression,sslclientdn}" _null_ _null_ pg_stat_get_activity _null_ _null_ _null_ ));
DESCR("statistics: information about currently active backends");
DATA(insert OID = 3099 ( pg_stat_get_wal_senders PGNSP PGUID 12 1 10 0 0 f f f f f t s 0 0 2249 "" "{23,25,3220,3220,3220,3220,23,25}" "{o,o,o,o,o,o,o,o}" "{pid,state,sent_location,write_location,flush_location,replay_location,sync_priority,sync_state}" _null_ _null_ pg_stat_get_wal_senders _null_ _null_ _null_ ));
DESCR("statistics: information about currently active replication");
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 9ecc163..8f2a88f 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -67,6 +67,33 @@ typedef enum StatMsgType
} StatMsgType;
/* ----------
+ * Wait definitions
+ * ----------
+ */
+
+typedef enum WAIT_CLASSES
+{
+ WAIT_UNDEFINED,
+ WAIT_LWLOCK,
+ WAIT_LOCK,
+ WAIT_IO,
+ WAIT_LATCH,
+ WAIT_NETWORK
+} WAIT_CLASSES;
+
+typedef enum WAIT_EVENTS_IO
+{
+ WAIT_IO_READ,
+ WAIT_IO_WRITE
+} WAIT_EVENTS_IO;
+
+typedef enum WAIT_EVENTS_NETWORK
+{
+ WAIT_NETWORK_READ,
+ WAIT_NETWORK_WRITE
+} WAIT_EVENT_NETWORK;
+
+/* ----------
* The data type used for counters.
* ----------
*/
@@ -768,6 +795,14 @@ typedef struct PgBackendStatus
/* Is backend currently waiting on an lmgr lock? */
bool st_waiting;
+ /* Contains class end event of wait. It's in one
+ * variable because we need read it atomically
+ */
+ volatile uint32 st_wait_data;
+
+ /* keep track of nested waits, and skip them */
+ int st_wait_nested;
+
/* current state */
BackendState st_state;
@@ -932,6 +967,12 @@ extern void pgstat_report_tempfile(size_t filesize);
extern void pgstat_report_appname(const char *appname);
extern void pgstat_report_xact_timestamp(TimestampTz tstamp);
extern void pgstat_report_waiting(bool waiting);
+
+extern void pgstat_report_wait_start(uint8 classId, uint8 eventId);
+extern void pgstat_report_wait_end();
+extern const char *pgstat_get_wait_class_name(uint8 classId);
+extern const char *pgstat_get_wait_event_name(uint8 classId, uint8 eventId);
+
extern const char *pgstat_get_backend_current_activity(int pid, bool checkUser);
extern const char *pgstat_get_crashed_backend_activity(int pid, char *buffer,
int buflen);
diff --git a/src/include/storage/lock.h b/src/include/storage/lock.h
index 96fe3a6..e05bea4 100644
--- a/src/include/storage/lock.h
+++ b/src/include/storage/lock.h
@@ -39,6 +39,9 @@ extern int Trace_lock_table;
extern bool Debug_deadlocks;
#endif /* LOCK_DEBUG */
+/* Contains names of all heavyweight locks */
+extern PGDLLIMPORT const char *LOCK_NAMES[];
+
/*
* Top-level transactions are identified by VirtualTransactionIDs comprising
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index cff3b99..55b0687 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -58,6 +58,9 @@ typedef struct LWLock
#ifdef LOCK_DEBUG
struct PGPROC *owner; /* last exlusive owner of the lock */
#endif
+
+ /* LWLock group, initialized as -1, calculated in first acquire */
+ int group;
} LWLock;
/*
@@ -87,6 +90,9 @@ typedef union LWLockPadded
} LWLockPadded;
extern PGDLLIMPORT LWLockPadded *MainLWLockArray;
+/* LWLock group names */
+extern const char *LWLOCK_GROUPS[];
+
/*
* Some commonly-used locks have predefined positions within MainLWLockArray;
* defining macros here makes it much easier to keep track of these. If you
@@ -164,6 +170,33 @@ extern PGDLLIMPORT LWLockPadded *MainLWLockArray;
#define NUM_FIXED_LWLOCKS \
(PREDICATELOCK_MANAGER_LWLOCK_OFFSET + NUM_PREDICATELOCK_PARTITIONS)
+/*
+ * LWLocks grouped by type, these groups are used as LWLock id in
+ * waits monitoring. In case of new
+ * lwlocks they must be added in LWLOCK_GROUPS
+ */
+
+/* Number of groups with dynamic sizes in main tranche */
+#define NUM_DYN_LWLOCK_GROUPS 10
+
+/* Number of groups that are not individual. Includes lwlock groups
+ * defined above by offset and with dynamic size
+ */
+#define NUM_ADD_LWLOCK_GROUPS (3 + NUM_DYN_LWLOCK_GROUPS)
+#define NUM_LWLOCK_GROUPS (NUM_INDIVIDUAL_LWLOCKS + NUM_ADD_LWLOCK_GROUPS)
+
+/*
+ * Additional LWLock groups (mainly from other tranches),
+ * Group for LWLock from main tranche determined automatically
+ */
+
+#define LWLOCK_BUFFER_CLEANUP NUM_LWLOCK_GROUPS
+#define LWLOCK_WAL_INSERT (NUM_LWLOCK_GROUPS + 1)
+#define LWLOCK_REPLICATIONORIGIN (NUM_LWLOCK_GROUPS + 2)
+
+#define LWLOCK_LAST_GROUP WAIT_LWLOCK_REPLICATIONORIGIN
+
+
typedef enum LWLockMode
{
LW_EXCLUSIVE,
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers