Commit 53be0b1a authored by Robert Haas's avatar Robert Haas

Provide much better wait information in pg_stat_activity.

When a process is waiting for a heavyweight lock, we will now indicate
the type of heavyweight lock for which it is waiting.  Also, you can
now see when a process is waiting for a lightweight lock - in which
case we will indicate the individual lock name or the tranche, as
appropriate - or for a buffer pin.

Amit Kapila, Ildus Kurbangaliev, reviewed by me.  Lots of helpful
discussion and suggestions by many others, including Alexander
Korotkov, Vladimir Borodin, and many others.
parent a3a8309d
This diff is collapsed.
......@@ -2447,13 +2447,14 @@ AbortTransaction(void)
*/
LWLockReleaseAll();
/* Clear wait information and command progress indicator */
pgstat_report_wait_end();
pgstat_progress_end_command();
/* Clean up buffer I/O and buffer context locks, too */
AbortBufferIO();
UnlockBuffers();
/* Clear command progress indicator */
pgstat_progress_end_command();
/* Reset WAL record construction state */
XLogResetInsertion();
......@@ -4541,9 +4542,10 @@ AbortSubTransaction(void)
*/
LWLockReleaseAll();
pgstat_report_wait_end();
pgstat_progress_end_command();
AbortBufferIO();
UnlockBuffers();
pgstat_progress_end_command();
/* Reset WAL record construction state */
XLogResetInsertion();
......@@ -4653,6 +4655,9 @@ AbortSubTransaction(void)
*/
XactReadOnly = s->prevXactReadOnly;
/* Report wait end here, when there is no further possibility of wait */
pgstat_report_wait_end();
RESUME_INTERRUPTS();
}
......
......@@ -26,6 +26,7 @@
#include "miscadmin.h"
#include "nodes/makefuncs.h"
#include "pg_getopt.h"
#include "pgstat.h"
#include "postmaster/bgwriter.h"
#include "postmaster/startup.h"
#include "postmaster/walwriter.h"
......@@ -534,6 +535,7 @@ static void
ShutdownAuxiliaryProcess(int code, Datum arg)
{
LWLockReleaseAll();
pgstat_report_wait_end();
}
/* ----------------------------------------------------------------
......
......@@ -636,7 +636,8 @@ CREATE VIEW pg_stat_activity AS
S.xact_start,
S.query_start,
S.state_change,
S.waiting,
S.wait_event_type,
S.wait_event,
S.state,
S.backend_xid,
s.backend_xmin,
......
......@@ -224,6 +224,9 @@ BackgroundWriterMain(void)
* It's not clear we need it elsewhere, but shouldn't hurt.
*/
smgrcloseall();
/* Report wait end here, when there is no further possibility of wait */
pgstat_report_wait_end();
}
/* We can now handle ereport(ERROR) */
......
......@@ -273,6 +273,7 @@ CheckpointerMain(void)
* files.
*/
LWLockReleaseAll();
pgstat_report_wait_end();
AbortBufferIO();
UnlockBuffers();
/* buffer pins are released here: */
......
......@@ -48,12 +48,12 @@
#include "postmaster/autovacuum.h"
#include "postmaster/fork_process.h"
#include "postmaster/postmaster.h"
#include "storage/proc.h"
#include "storage/backendid.h"
#include "storage/dsm.h"
#include "storage/fd.h"
#include "storage/ipc.h"
#include "storage/latch.h"
#include "storage/lmgr.h"
#include "storage/pg_shmem.h"
#include "storage/procsignal.h"
#include "storage/sinvaladt.h"
......@@ -2723,7 +2723,6 @@ pgstat_bestart(void)
#else
beentry->st_ssl = false;
#endif
beentry->st_waiting = false;
beentry->st_state = STATE_UNDEFINED;
beentry->st_appname[0] = '\0';
beentry->st_activity[0] = '\0';
......@@ -2810,6 +2809,8 @@ pgstat_report_activity(BackendState state, const char *cmd_str)
{
if (beentry->st_state != STATE_DISABLED)
{
volatile PGPROC *proc = MyProc;
/*
* track_activities is disabled, but we last reported a
* non-disabled state. As our final update, change the state and
......@@ -2820,9 +2821,9 @@ pgstat_report_activity(BackendState state, const char *cmd_str)
beentry->st_state_start_timestamp = 0;
beentry->st_activity[0] = '\0';
beentry->st_activity_start_timestamp = 0;
/* st_xact_start_timestamp and st_waiting are also disabled */
/* st_xact_start_timestamp and wait_event_info are also disabled */
beentry->st_xact_start_timestamp = 0;
beentry->st_waiting = false;
proc->wait_event_info = 0;
pgstat_increment_changecount_after(beentry);
}
return;
......@@ -2978,32 +2979,6 @@ pgstat_report_xact_timestamp(TimestampTz tstamp)
pgstat_increment_changecount_after(beentry);
}
/* ----------
* pgstat_report_waiting() -
*
* Called from lock manager to report beginning or end of a lock wait.
*
* NB: this *must* be able to survive being called before MyBEEntry has been
* initialized.
* ----------
*/
void
pgstat_report_waiting(bool waiting)
{
volatile PgBackendStatus *beentry = MyBEEntry;
if (!pgstat_track_activities || !beentry)
return;
/*
* Since this is a single-byte 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_waiting = waiting;
}
/* ----------
* pgstat_read_current_status() -
*
......@@ -3119,6 +3094,87 @@ pgstat_read_current_status(void)
localBackendStatusTable = localtable;
}
/* ----------
* pgstat_get_wait_event_type() -
*
* Return a string representing the current wait event type, backend is
* waiting on.
*/
const char *
pgstat_get_wait_event_type(uint32 wait_event_info)
{
uint8 classId;
const char *event_type;
/* report process as not waiting. */
if (wait_event_info == 0)
return NULL;
wait_event_info = wait_event_info >> 24;
classId = wait_event_info & 0XFF;
switch (classId)
{
case WAIT_LWLOCK_NAMED:
event_type = "LWLockNamed";
break;
case WAIT_LWLOCK_TRANCHE:
event_type = "LWLockTranche";
break;
case WAIT_LOCK:
event_type = "Lock";
break;
case WAIT_BUFFER_PIN:
event_type = "BufferPin";
break;
default:
event_type = "???";
break;
}
return event_type;
}
/* ----------
* pgstat_get_wait_event() -
*
* Return a string representing the current wait event, backend is
* waiting on.
*/
const char *
pgstat_get_wait_event(uint32 wait_event_info)
{
uint8 classId;
uint16 eventId;
const char *event_name;
/* report process as not waiting. */
if (wait_event_info == 0)
return NULL;
eventId = wait_event_info & ((1 << 24) - 1);
wait_event_info = wait_event_info >> 24;
classId = wait_event_info & 0XFF;
switch (classId)
{
case WAIT_LWLOCK_NAMED:
case WAIT_LWLOCK_TRANCHE:
event_name = GetLWLockIdentifier(classId, eventId);
break;
case WAIT_LOCK:
event_name = GetLockNameFromTagType(eventId);
break;
case WAIT_BUFFER_PIN:
event_name = "BufferPin";
break;
default:
event_name = "unknown wait event";
break;
}
return event_name;
}
/* ----------
* pgstat_get_backend_current_activity() -
......
......@@ -47,6 +47,7 @@
#include "access/xlog.h"
#include "libpq/pqsignal.h"
#include "miscadmin.h"
#include "pgstat.h"
#include "postmaster/walwriter.h"
#include "storage/bufmgr.h"
#include "storage/fd.h"
......@@ -168,6 +169,7 @@ WalWriterMain(void)
* about in walwriter, but we do have LWLocks, and perhaps buffers?
*/
LWLockReleaseAll();
pgstat_report_wait_end();
AbortBufferIO();
UnlockBuffers();
/* buffer pins are released here: */
......
......@@ -55,6 +55,7 @@
#include "libpq/pqformat.h"
#include "miscadmin.h"
#include "nodes/replnodes.h"
#include "pgstat.h"
#include "replication/basebackup.h"
#include "replication/decode.h"
#include "replication/logical.h"
......@@ -252,6 +253,7 @@ void
WalSndErrorCleanup(void)
{
LWLockReleaseAll();
pgstat_report_wait_end();
if (sendFile >= 0)
{
......
......@@ -3351,6 +3351,9 @@ LockBufferForCleanup(Buffer buffer)
UnlockBufHdr(bufHdr);
LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
/* Report the wait */
pgstat_report_wait_start(WAIT_BUFFER_PIN, 0);
/* Wait to be signaled by UnpinBuffer() */
if (InHotStandby)
{
......@@ -3364,6 +3367,8 @@ LockBufferForCleanup(Buffer buffer)
else
ProcWaitForSignal();
pgstat_report_wait_end();
/*
* Remove flag marking us as waiter. Normally this will not be set
* anymore, but ProcWaitForSignal() can return for other signals as
......
......@@ -994,3 +994,26 @@ DescribeLockTag(StringInfo buf, const LOCKTAG *tag)
break;
}
}
/*
* GetLockNameFromTagType
*
* Given locktag type, return the corresponding lock name.
*/
const char *
GetLockNameFromTagType(uint16 locktag_type)
{
const char *locktypename;
char tnbuf[32];
if (locktag_type <= LOCKTAG_LAST_TYPE)
locktypename = LockTagTypeNames[locktag_type];
else
{
snprintf(tnbuf, sizeof(tnbuf), "unknown %d",
(int) locktag_type);
locktypename = tnbuf;
}
return locktypename;
}
......@@ -1676,7 +1676,7 @@ WaitOnLock(LOCALLOCK *locallock, ResourceOwner owner)
set_ps_display(new_status, false);
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;
......@@ -1724,7 +1724,7 @@ WaitOnLock(LOCALLOCK *locallock, ResourceOwner owner)
/* In this path, awaitedLock remains set until LockErrorCleanup */
/* Report change to non-waiting status */
pgstat_report_waiting(false);
pgstat_report_wait_end();
if (update_process_title)
{
set_ps_display(new_status, false);
......@@ -1739,7 +1739,7 @@ WaitOnLock(LOCALLOCK *locallock, ResourceOwner owner)
awaitedLock = NULL;
/* Report change to non-waiting status */
pgstat_report_waiting(false);
pgstat_report_wait_end();
if (update_process_title)
{
set_ps_display(new_status, false);
......
......@@ -77,6 +77,7 @@
#include "postgres.h"
#include "miscadmin.h"
#include "pgstat.h"
#include "pg_trace.h"
#include "postmaster/postmaster.h"
#include "replication/slot.h"
......@@ -165,6 +166,9 @@ static bool lock_named_request_allowed = true;
static void InitializeLWLocks(void);
static void RegisterLWLockTranches(void);
static inline void LWLockReportWaitStart(LWLock *lock);
static inline void LWLockReportWaitEnd();
#ifdef LWLOCK_STATS
typedef struct lwlock_stats_key
{
......@@ -525,7 +529,7 @@ RegisterLWLockTranches(void)
{
LWLockTranchesAllocated = 32;
LWLockTrancheArray = (LWLockTranche **)
MemoryContextAlloc(TopMemoryContext,
MemoryContextAllocZero(TopMemoryContext,
LWLockTranchesAllocated * sizeof(LWLockTranche *));
Assert(LWLockTranchesAllocated >= LWTRANCHE_FIRST_USER_DEFINED);
}
......@@ -636,6 +640,7 @@ LWLockRegisterTranche(int tranche_id, LWLockTranche *tranche)
if (tranche_id >= LWLockTranchesAllocated)
{
int i = LWLockTranchesAllocated;
int j = LWLockTranchesAllocated;
while (i <= tranche_id)
i *= 2;
......@@ -644,6 +649,8 @@ LWLockRegisterTranche(int tranche_id, LWLockTranche *tranche)
repalloc(LWLockTrancheArray,
i * sizeof(LWLockTranche *));
LWLockTranchesAllocated = i;
while (j < LWLockTranchesAllocated)
LWLockTrancheArray[j++] = NULL;
}
LWLockTrancheArray[tranche_id] = tranche;
......@@ -713,6 +720,57 @@ LWLockInitialize(LWLock *lock, int tranche_id)
dlist_init(&lock->waiters);
}
/*
* Report start of 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 tranche and lock id.
*/
static inline void
LWLockReportWaitStart(LWLock *lock)
{
int lockId = T_ID(lock);
if (lock->tranche == 0)
pgstat_report_wait_start(WAIT_LWLOCK_NAMED, (uint16) lockId);
else
pgstat_report_wait_start(WAIT_LWLOCK_TRANCHE, lock->tranche);
}
/*
* Report end of wait event for light-weight locks.
*/
static inline void
LWLockReportWaitEnd()
{
pgstat_report_wait_end();
}
/*
* Return an identifier for an LWLock based on the wait class and event.
*/
const char *
GetLWLockIdentifier(uint8 classId, uint16 eventId)
{
if (classId == WAIT_LWLOCK_NAMED)
return MainLWLockNames[eventId];
Assert(classId == WAIT_LWLOCK_TRANCHE);
/*
* It is quite possible that user has registered tranche in one of the
* backends (e.g. by allocation lwlocks in dynamic shared memory) but not
* all of them, so we can't assume the tranche is registered here.
* extension for such cases.
*/
if (eventId >= LWLockTranchesAllocated ||
LWLockTrancheArray[eventId]->name == NULL)
return "extension";
return LWLockTrancheArray[eventId]->name;
}
/*
* Internal function that tries to atomically acquire the lwlock in the passed
* in mode.
......@@ -1162,6 +1220,7 @@ LWLockAcquire(LWLock *lock, LWLockMode mode)
lwstats->block_count++;
#endif
LWLockReportWaitStart(lock);
TRACE_POSTGRESQL_LWLOCK_WAIT_START(T_NAME(lock), T_ID(lock), mode);
for (;;)
......@@ -1185,6 +1244,7 @@ LWLockAcquire(LWLock *lock, LWLockMode mode)
#endif
TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(T_NAME(lock), T_ID(lock), mode);
LWLockReportWaitEnd();
LOG_LWDEBUG("LWLockAcquire", lock, "awakened");
......@@ -1320,6 +1380,8 @@ LWLockAcquireOrWait(LWLock *lock, LWLockMode mode)
#ifdef LWLOCK_STATS
lwstats->block_count++;
#endif
LWLockReportWaitStart(lock);
TRACE_POSTGRESQL_LWLOCK_WAIT_START(T_NAME(lock), T_ID(lock), mode);
for (;;)
......@@ -1339,6 +1401,7 @@ LWLockAcquireOrWait(LWLock *lock, LWLockMode mode)
}
#endif
TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(T_NAME(lock), T_ID(lock), mode);
LWLockReportWaitEnd();
LOG_LWDEBUG("LWLockAcquireOrWait", lock, "awakened");
}
......@@ -1544,6 +1607,7 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
lwstats->block_count++;
#endif
LWLockReportWaitStart(lock);
TRACE_POSTGRESQL_LWLOCK_WAIT_START(T_NAME(lock), T_ID(lock),
LW_EXCLUSIVE);
......@@ -1566,6 +1630,7 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(T_NAME(lock), T_ID(lock),
LW_EXCLUSIVE);
LWLockReportWaitEnd();
LOG_LWDEBUG("LWLockWaitForVar", lock, "awakened");
......
......@@ -404,6 +404,9 @@ InitProcess(void)
Assert(MyProc->lockGroupLeader == NULL);
Assert(dlist_is_empty(&MyProc->lockGroupMembers));
/* Initialize wait event information. */
MyProc->wait_event_info = 0;
/*
* Acquire ownership of the PGPROC's latch, so that we can use WaitLatch
* on it. That allows us to repoint the process latch, which so far
......
......@@ -23,7 +23,7 @@
/* This must match enum LockTagType! */
static const char *const LockTagTypeNames[] = {
const char *const LockTagTypeNames[] = {
"relation",
"extend",
"page",
......
......@@ -20,6 +20,8 @@
#include "libpq/ip.h"
#include "miscadmin.h"
#include "pgstat.h"
#include "storage/proc.h"
#include "storage/procarray.h"
#include "utils/acl.h"
#include "utils/builtins.h"
#include "utils/inet.h"
......@@ -58,7 +60,8 @@ extern Datum pg_stat_get_backend_pid(PG_FUNCTION_ARGS);
extern Datum pg_stat_get_backend_dbid(PG_FUNCTION_ARGS);
extern Datum pg_stat_get_backend_userid(PG_FUNCTION_ARGS);
extern Datum pg_stat_get_backend_activity(PG_FUNCTION_ARGS);
extern Datum pg_stat_get_backend_waiting(PG_FUNCTION_ARGS);
extern Datum pg_stat_get_backend_wait_event_type(PG_FUNCTION_ARGS);
extern Datum pg_stat_get_backend_wait_event(PG_FUNCTION_ARGS);
extern Datum pg_stat_get_backend_activity_start(PG_FUNCTION_ARGS);
extern Datum pg_stat_get_backend_xact_start(PG_FUNCTION_ARGS);
extern Datum pg_stat_get_backend_start(PG_FUNCTION_ARGS);
......@@ -633,7 +636,7 @@ pg_stat_get_progress_info(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);
......@@ -676,6 +679,9 @@ pg_stat_get_activity(PG_FUNCTION_ARGS)
bool nulls[PG_STAT_GET_ACTIVITY_COLS];
LocalPgBackendStatus *local_beentry;
PgBackendStatus *beentry;
PGPROC *proc;
const char *wait_event_type;
const char *wait_event;
MemSet(values, 0, sizeof(values));
MemSet(nulls, 0, sizeof(nulls));
......@@ -720,28 +726,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 */
......@@ -775,36 +781,48 @@ 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);
proc = BackendPidGetProc(beentry->st_procpid);
wait_event_type = pgstat_get_wait_event_type(proc->wait_event_info);
if (wait_event_type)
values[6] = CStringGetTextDatum(wait_event_type);
else
nulls[6] = true;
wait_event = pgstat_get_wait_event(proc->wait_event_info);
if (wait_event)
values[7] = CStringGetTextDatum(wait_event);
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
{
......@@ -828,20 +846,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)
......@@ -852,16 +870,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;
}
}
}
......@@ -878,6 +896,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);
......@@ -959,23 +978,52 @@ pg_stat_get_backend_activity(PG_FUNCTION_ARGS)
PG_RETURN_TEXT_P(cstring_to_text(activity));
}
Datum
pg_stat_get_backend_waiting(PG_FUNCTION_ARGS)
pg_stat_get_backend_wait_event_type(PG_FUNCTION_ARGS)
{
int32 beid = PG_GETARG_INT32(0);
bool result;
PgBackendStatus *beentry;
PGPROC *proc;
const char *wait_event_type;
if ((beentry = pgstat_fetch_stat_beentry(beid)) == NULL)
PG_RETURN_NULL();
wait_event_type = "<backend information not available>";
else if (!has_privs_of_role(GetUserId(), beentry->st_userid))
wait_event_type = "<insufficient privilege>";
else
{
proc = BackendPidGetProc(beentry->st_procpid);
wait_event_type = pgstat_get_wait_event_type(proc->wait_event_info);
}
if (!has_privs_of_role(GetUserId(), beentry->st_userid))
if (!wait_event_type)
PG_RETURN_NULL();
result = beentry->st_waiting;
PG_RETURN_TEXT_P(cstring_to_text(wait_event_type));
}
Datum
pg_stat_get_backend_wait_event(PG_FUNCTION_ARGS)
{
int32 beid = PG_GETARG_INT32(0);
PgBackendStatus *beentry;
PGPROC *proc;
const char *wait_event;
if ((beentry = pgstat_fetch_stat_beentry(beid)) == NULL)
wait_event = "<backend information not available>";
else if (!has_privs_of_role(GetUserId(), beentry->st_userid))
wait_event = "<insufficient privilege>";
else
{
proc = BackendPidGetProc(beentry->st_procpid);
wait_event = pgstat_get_wait_event(proc->wait_event_info);
}
if (!wait_event)
PG_RETURN_NULL();
PG_RETURN_BOOL(result);
PG_RETURN_TEXT_P(cstring_to_text(wait_event));
}
......
......@@ -53,6 +53,6 @@
*/
/* yyyymmddN */
#define CATALOG_VERSION_NO 201603091
#define CATALOG_VERSION_NO 201603101
#endif
......@@ -2708,7 +2708,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 r 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 r 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 r 1 0 2249 "23" "{23,26,23,26,25,25,25,25,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,wait_event_type,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 = 3318 ( pg_stat_get_progress_info PGNSP PGUID 12 1 100 0 0 f f f f f t s r 1 0 2249 "25" "{25,23,26,26,20,20,20,20,20,20,20,20,20,20}" "{i,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{cmdtype,pid,datid,relid,param1,param2,param3,param4,param5,param6,param7,param8,param9,param10}" _null_ _null_ pg_stat_get_progress_info _null_ _null_ _null_ ));
DESCR("statistics: information about progress of backends running maintenance command");
......@@ -2726,8 +2726,10 @@ DATA(insert OID = 1939 ( pg_stat_get_backend_userid PGNSP PGUID 12 1 0 0 0 f f
DESCR("statistics: user ID of backend");
DATA(insert OID = 1940 ( pg_stat_get_backend_activity PGNSP PGUID 12 1 0 0 0 f f f f t f s r 1 0 25 "23" _null_ _null_ _null_ _null_ _null_ pg_stat_get_backend_activity _null_ _null_ _null_ ));
DESCR("statistics: current query of backend");
DATA(insert OID = 2853 ( pg_stat_get_backend_waiting PGNSP PGUID 12 1 0 0 0 f f f f t f s r 1 0 16 "23" _null_ _null_ _null_ _null_ _null_ pg_stat_get_backend_waiting _null_ _null_ _null_ ));
DESCR("statistics: is backend currently waiting for a lock");
DATA(insert OID = 2788 ( pg_stat_get_backend_wait_event_type PGNSP PGUID 12 1 0 0 0 f f f f t f s r 1 0 25 "23" _null_ _null_ _null_ _null_ _null_ pg_stat_get_backend_wait_event_type _null_ _null_ _null_ ));
DESCR("statistics: wait event type on which backend is currently waiting");
DATA(insert OID = 2853 ( pg_stat_get_backend_wait_event PGNSP PGUID 12 1 0 0 0 f f f f t f s r 1 0 25 "23" _null_ _null_ _null_ _null_ _null_ pg_stat_get_backend_wait_event _null_ _null_ _null_ ));
DESCR("statistics: wait event on which backend is currently waiting");
DATA(insert OID = 2094 ( pg_stat_get_backend_activity_start PGNSP PGUID 12 1 0 0 0 f f f f t f s r 1 0 1184 "23" _null_ _null_ _null_ _null_ _null_ pg_stat_get_backend_activity_start _null_ _null_ _null_ ));
DESCR("statistics: start time for current query of backend");
DATA(insert OID = 2857 ( pg_stat_get_backend_xact_start PGNSP PGUID 12 1 0 0 0 f f f f t f s r 1 0 1184 "23" _null_ _null_ _null_ _null_ _null_ pg_stat_get_backend_xact_start _null_ _null_ _null_ ));
......
......@@ -17,6 +17,7 @@
#include "portability/instr_time.h"
#include "postmaster/pgarch.h"
#include "storage/barrier.h"
#include "storage/proc.h"
#include "utils/hsearch.h"
#include "utils/relcache.h"
......@@ -695,6 +696,21 @@ typedef enum BackendState
STATE_DISABLED
} BackendState;
/* ----------
* Wait Classes
* ----------
*/
typedef enum WaitClass
{
WAIT_UNDEFINED,
WAIT_LWLOCK_NAMED,
WAIT_LWLOCK_TRANCHE,
WAIT_LOCK,
WAIT_BUFFER_PIN
} WaitClass;
/* ----------
* Command type for progress reporting purposes
* ----------
......@@ -777,9 +793,6 @@ typedef struct PgBackendStatus
bool st_ssl;
PgBackendSSLStatus *st_sslstatus;
/* Is backend currently waiting on an lmgr lock? */
bool st_waiting;
/* current state */
BackendState st_state;
......@@ -956,7 +969,8 @@ extern void pgstat_report_activity(BackendState state, const char *cmd_str);
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 const char *pgstat_get_wait_event(uint32 wait_event_info);
extern const char *pgstat_get_wait_event_type(uint32 wait_event_info);
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);
......@@ -971,6 +985,65 @@ extern PgStat_BackendFunctionEntry *find_funcstat_entry(Oid func_id);
extern void pgstat_initstats(Relation rel);
/* ----------
* pgstat_report_wait_start() -
*
* Called from places where server process needs to wait. This is called
* to report wait event information. The wait information is stored
* as 4-bytes where first byte repersents the wait event class (type of
* wait, for different types of wait, refer WaitClass) and the next
* 3-bytes repersent the actual wait event. Currently 2-bytes are used
* for wait event which is sufficient for current usage, 1-byte is
* reserved for future usage.
*
* NB: this *must* be able to survive being called before MyProc has been
* initialized.
* ----------
*/
static inline void
pgstat_report_wait_start(uint8 classId, uint16 eventId)
{
volatile PGPROC *proc = MyProc;
uint32 wait_event_val;
if (!pgstat_track_activities || !proc)
return;
wait_event_val = classId;
wait_event_val <<= 24;
wait_event_val |= eventId;
/*
* Since this is a four-byte field which is always read and written as
* four-bytes, updates are atomic.
*/
proc->wait_event_info = wait_event_val;
}
/* ----------
* pgstat_report_wait_end() -
*
* Called to report end of a wait.
*
* NB: this *must* be able to survive being called before MyProc has been
* initialized.
* ----------
*/
static inline void
pgstat_report_wait_end(void)
{
volatile PGPROC *proc = MyProc;
if (!pgstat_track_activities || !proc)
return;
/*
* Since this is a four-byte field which is always read and written as
* four-bytes, updates are atomic.
*/
proc->wait_event_info = 0;
}
/* nontransactional event counts are simple enough to inline */
#define pgstat_count_heap_scan(rel) \
......
......@@ -101,4 +101,6 @@ extern void UnlockSharedObjectForSession(Oid classid, Oid objid, uint16 objsubid
/* Describe a locktag for error messages */
extern void DescribeLockTag(StringInfo buf, const LOCKTAG *tag);
extern const char *GetLockNameFromTagType(uint16 locktag_type);
#endif /* LMGR_H */
......@@ -166,6 +166,8 @@ typedef enum LockTagType
#define LOCKTAG_LAST_TYPE LOCKTAG_ADVISORY
extern const char *const LockTagTypeNames[];
/*
* The LOCKTAG struct is defined with malice aforethought to fit into 16
* bytes with no padding. Note that this would need adjustment if we were
......
......@@ -187,6 +187,8 @@ extern Size LWLockShmemSize(void);
extern void CreateLWLocks(void);
extern void InitLWLockAccess(void);
extern const char *GetLWLockIdentifier(uint8 classId, uint16 eventId);
/*
* Extensions (or core code) can obtain an LWLocks by calling
* RequestNamedLWLockTranche() during postmaster startup. Subsequently,
......
......@@ -152,6 +152,8 @@ struct PGPROC
*/
TransactionId procArrayGroupMemberXid;
uint32 wait_event_info; /* proc's wait information */
/* Per-backend LWLock. Protects fields below (but not group fields). */
LWLock backendLock;
......
......@@ -1650,13 +1650,14 @@ pg_stat_activity| SELECT s.datid,
s.xact_start,
s.query_start,
s.state_change,
s.waiting,
s.wait_event_type,
s.wait_event,
s.state,
s.backend_xid,
s.backend_xmin,
s.query
FROM pg_database d,
pg_stat_get_activity(NULL::integer) s(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),
pg_stat_get_activity(NULL::integer) s(datid, pid, usesysid, application_name, state, query, wait_event_type, 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),
pg_authid u
WHERE ((s.datid = d.oid) AND (s.usesysid = u.oid));
pg_stat_all_indexes| SELECT c.oid AS relid,
......@@ -1762,7 +1763,7 @@ pg_stat_replication| SELECT s.pid,
w.replay_location,
w.sync_priority,
w.sync_state
FROM pg_stat_get_activity(NULL::integer) s(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),
FROM pg_stat_get_activity(NULL::integer) s(datid, pid, usesysid, application_name, state, query, wait_event_type, 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),
pg_authid u,
pg_stat_get_wal_senders() w(pid, state, sent_location, write_location, flush_location, replay_location, sync_priority, sync_state)
WHERE ((s.usesysid = u.oid) AND (s.pid = w.pid));
......@@ -1773,7 +1774,7 @@ pg_stat_ssl| SELECT s.pid,
s.sslbits AS bits,
s.sslcompression AS compression,
s.sslclientdn AS clientdn
FROM pg_stat_get_activity(NULL::integer) s(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);
FROM pg_stat_get_activity(NULL::integer) s(datid, pid, usesysid, application_name, state, query, wait_event_type, 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);
pg_stat_sys_indexes| SELECT pg_stat_all_indexes.relid,
pg_stat_all_indexes.indexrelid,
pg_stat_all_indexes.schemaname,
......
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