diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index f3bf527d5b4b..a9669c3ec8e5 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -493,6 +493,15 @@ postgres 27093 0.0 0.0 30096 2752 ? Ss 11:34 0:00 postgres: ser
+
+ pg_stat_lock pg_stat_lock
+
+ One row for each lock type, containing cluster-wide locks statistics.
+ See
+ pg_stat_lock for details.
+
+
+
pg_stat_replication_slots pg_stat_replication_slots
One row per replication slot, showing statistics about the
@@ -3053,6 +3062,142 @@ description | Waiting for a newly initialized WAL file to reach durable storage
+
+
+ pg_stat_lock
+
+
+ pg_stat_lock
+
+
+
+ The pg_stat_lock view will contain one row for each
+ lock type, showing cluster-wide locks statistics.
+
+
+
+ pg_stat_lock View
+
+
+
+
+
+ Column Type
+
+
+ Description
+
+
+
+
+
+
+
+
+ locktype text
+
+
+ Type of the lockable object:
+ relation ,
+ extend ,
+ frozenid ,
+ page ,
+ tuple ,
+ transactionid ,
+ virtualxid ,
+ spectoken ,
+ object ,
+ userlock ,
+ advisory , or
+ applytransaction .
+ (See also .)
+
+
+
+
+
+
+
+ requests bigint
+
+
+ Number of requests for this lock type.
+
+
+
+
+
+
+
+ waits bigint
+
+
+ Number of times requests for this lock type had to wait.
+
+
+
+
+
+
+
+ timeouts bigint
+
+
+ Number of times requests for this lock type had to wait longer
+ than lock_timeout .
+
+
+
+
+
+
+
+ deadlock_timeouts bigint
+
+
+ Number of times requests for this lock type had to wait longer
+ than deadlock_timeout .
+
+
+
+
+
+
+
+ deadlocks bigint
+
+
+ Number of times a deadlock occurred on this lock type.
+
+
+
+
+
+
+
+ fastpath bigint
+
+
+ Number of times this lock type was taken via fast path.
+
+
+
+
+
+
+
+ stats_reset timestamp with time zone
+
+
+ Time at which these statistics were last reset.
+
+
+
+
+
+
+
+
pg_stat_bgwriter
@@ -5124,6 +5269,12 @@ description | Waiting for a newly initialized WAL file to reach durable storage
pg_stat_io view.
+
+
+ lock : Reset all the counters shown in the
+ pg_stat_lock view.
+
+
recovery_prefetch : Reset all the counters shown in
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index dec8df4f8ee6..636411e3f11a 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -964,6 +964,18 @@ CREATE VIEW pg_stat_slru AS
s.stats_reset
FROM pg_stat_get_slru() s;
+CREATE VIEW pg_stat_lock AS
+ SELECT
+ l.locktype,
+ l.requests,
+ l.waits,
+ l.timeouts,
+ l.deadlock_timeouts,
+ l.deadlocks,
+ l.fastpath,
+ l.stats_reset
+ FROM pg_stat_get_lock() l;
+
CREATE VIEW pg_stat_wal_receiver AS
SELECT
s.pid,
diff --git a/src/backend/storage/lmgr/lock.c b/src/backend/storage/lmgr/lock.c
index 4cc7f645c317..ce3137337398 100644
--- a/src/backend/storage/lmgr/lock.c
+++ b/src/backend/storage/lmgr/lock.c
@@ -45,6 +45,7 @@
#include "storage/spin.h"
#include "storage/standby.h"
#include "utils/memutils.h"
+#include "utils/pgstat_internal.h"
#include "utils/ps_status.h"
#include "utils/resowner.h"
@@ -871,6 +872,9 @@ LockAcquireExtended(const LOCKTAG *locktag,
lockMethodTable->lockModeNames[lockmode]),
errhint("Only RowExclusiveLock or less can be acquired on database objects during recovery.")));
+ /* Increment the lock statistics requests counter */
+ pgstat_count_lock_requests(locktag->locktag_type);
+
#ifdef LOCK_DEBUG
if (LOCK_DEBUG_ENABLED(locktag))
elog(LOG, "LockAcquire: lock [%u,%u] %s",
@@ -1202,6 +1206,8 @@ LockAcquireExtended(const LOCKTAG *locktag,
}
else
{
+ /* Increment the lock statistics deadlocks counter */
+ pgstat_count_lock_deadlocks(locallock->tag.lock.locktag_type);
DeadLockReport();
/* DeadLockReport() will not return */
}
@@ -1213,6 +1219,8 @@ LockAcquireExtended(const LOCKTAG *locktag,
*/
if (waitResult == PROC_WAIT_STATUS_WAITING)
{
+ /* Increment the lock statistics waits counter */
+ pgstat_count_lock_waits(locktag->locktag_type);
Assert(!dontWait);
PROCLOCK_PRINT("LockAcquire: sleeping on lock", proclock);
LOCK_PRINT("LockAcquire: sleeping on lock", lock, lockmode);
@@ -1233,6 +1241,8 @@ LockAcquireExtended(const LOCKTAG *locktag,
* now.
*/
Assert(!dontWait);
+ /* Increment the lock statistics deadlocks counter */
+ pgstat_count_lock_deadlocks(locallock->tag.lock.locktag_type);
DeadLockReport();
/* DeadLockReport() will not return */
}
@@ -2799,6 +2809,8 @@ FastPathGrantRelationLock(Oid relid, LOCKMODE lockmode)
{
Assert(!FAST_PATH_CHECK_LOCKMODE(MyProc, f, lockmode));
FAST_PATH_SET_LOCKMODE(MyProc, f, lockmode);
+ /* Increment the lock statistics fastpath counter */
+ pgstat_count_lock_fastpath(LOCKTAG_RELATION);
return true;
}
}
@@ -2808,6 +2820,8 @@ FastPathGrantRelationLock(Oid relid, LOCKMODE lockmode)
{
MyProc->fpRelId[unused_slot] = relid;
FAST_PATH_SET_LOCKMODE(MyProc, unused_slot, lockmode);
+ /* Increment the lock statistics fastpath counter */
+ pgstat_count_lock_fastpath(LOCKTAG_RELATION);
++FastPathLocalUseCounts[group];
return true;
}
@@ -4632,6 +4646,10 @@ VirtualXactLockTableInsert(VirtualTransactionId vxid)
MyProc->fpVXIDLock = true;
MyProc->fpLocalTransactionId = vxid.localTransactionId;
+ /* Increment the lock statistics requests and fastpath counters */
+ pgstat_count_lock_requests(LOCKTAG_VIRTUALTRANSACTION);
+ pgstat_count_lock_fastpath(LOCKTAG_VIRTUALTRANSACTION);
+
LWLockRelease(&MyProc->fpInfoLock);
}
diff --git a/src/backend/storage/lmgr/proc.c b/src/backend/storage/lmgr/proc.c
index 96f29aafc391..75e606bfed51 100644
--- a/src/backend/storage/lmgr/proc.c
+++ b/src/backend/storage/lmgr/proc.c
@@ -1456,9 +1456,12 @@ ProcSleep(LOCALLOCK *locallock)
/* check for deadlocks first, as that's probably log-worthy */
if (got_deadlock_timeout)
{
+ /* Increment the lock statistics deadlock_timeouts counter */
+ pgstat_count_lock_deadlock_timeouts(locallock->tag.lock.locktag_type);
CheckDeadLock();
got_deadlock_timeout = false;
}
+
CHECK_FOR_INTERRUPTS();
}
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index 2bd89102686e..02d29e43c24e 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -3448,6 +3448,14 @@ ProcessInterrupts(void)
if (lock_timeout_occurred)
{
+ LOCALLOCK *lockAwaited;
+
+ lockAwaited = GetAwaitedLock();
+
+ /* Increment the lock statistics timeouts counter */
+ if (lockAwaited)
+ pgstat_count_lock_timeouts(lockAwaited->tag.lock.locktag_type);
+
LockErrorCleanup();
ereport(ERROR,
(errcode(ERRCODE_LOCK_NOT_AVAILABLE),
diff --git a/src/backend/utils/activity/Makefile b/src/backend/utils/activity/Makefile
index 9c2443e1ecd3..4293d0f9efda 100644
--- a/src/backend/utils/activity/Makefile
+++ b/src/backend/utils/activity/Makefile
@@ -26,6 +26,7 @@ OBJS = \
pgstat_database.o \
pgstat_function.o \
pgstat_io.o \
+ pgstat_lock.o \
pgstat_relation.o \
pgstat_replslot.o \
pgstat_shmem.o \
diff --git a/src/backend/utils/activity/meson.build b/src/backend/utils/activity/meson.build
index d8e56b49c247..3d3d7b67b5f6 100644
--- a/src/backend/utils/activity/meson.build
+++ b/src/backend/utils/activity/meson.build
@@ -11,6 +11,7 @@ backend_sources += files(
'pgstat_database.c',
'pgstat_function.c',
'pgstat_io.c',
+ 'pgstat_lock.c',
'pgstat_relation.c',
'pgstat_replslot.c',
'pgstat_shmem.c',
diff --git a/src/backend/utils/activity/pgstat.c b/src/backend/utils/activity/pgstat.c
index 7ef06150df7f..0b51b02a5156 100644
--- a/src/backend/utils/activity/pgstat.c
+++ b/src/backend/utils/activity/pgstat.c
@@ -83,6 +83,7 @@
* - pgstat_database.c
* - pgstat_function.c
* - pgstat_io.c
+ * - pgstat_lock.c
* - pgstat_relation.c
* - pgstat_replslot.c
* - pgstat_slru.c
@@ -448,6 +449,23 @@ static const PgStat_KindInfo pgstat_kind_builtin_infos[PGSTAT_KIND_BUILTIN_SIZE]
.snapshot_cb = pgstat_io_snapshot_cb,
},
+ [PGSTAT_KIND_LOCK] = {
+ .name = "lock",
+
+ .fixed_amount = true,
+ .write_to_file = true,
+
+ .snapshot_ctl_off = offsetof(PgStat_Snapshot, lock),
+ .shared_ctl_off = offsetof(PgStat_ShmemControl, lock),
+ .shared_data_off = offsetof(PgStatShared_Lock, stats),
+ .shared_data_len = sizeof(((PgStatShared_Lock *) 0)->stats),
+
+ .flush_static_cb = pgstat_lock_flush_cb,
+ .init_shmem_cb = pgstat_lock_init_shmem_cb,
+ .reset_all_cb = pgstat_lock_reset_all_cb,
+ .snapshot_cb = pgstat_lock_snapshot_cb,
+ },
+
[PGSTAT_KIND_SLRU] = {
.name = "slru",
diff --git a/src/backend/utils/activity/pgstat_lock.c b/src/backend/utils/activity/pgstat_lock.c
new file mode 100644
index 000000000000..73bdeb5ff66f
--- /dev/null
+++ b/src/backend/utils/activity/pgstat_lock.c
@@ -0,0 +1,176 @@
+/* -------------------------------------------------------------------------
+ *
+ * pgstat_lock.c
+ * Implementation of lock statistics.
+ *
+ * This file contains the implementation of lock statistics. It is kept separate
+ * from pgstat.c to enforce the line between the statistics access / storage
+ * implementation and the details about individual types of statistics.
+ *
+ * Copyright (c) 2021-2025, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ * src/backend/utils/activity/pgstat_lock.c
+ * -------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "utils/pgstat_internal.h"
+
+static PgStat_PendingLock PendingLockStats;
+static bool have_lockstats = false;
+
+PgStat_Lock *
+pgstat_fetch_stat_lock(void)
+{
+ pgstat_snapshot_fixed(PGSTAT_KIND_LOCK);
+
+ return &pgStatLocal.snapshot.lock;
+}
+
+/*
+ * Simpler wrapper of pgstat_lock_flush_cb()
+ */
+void
+pgstat_lock_flush(bool nowait)
+{
+ (void) pgstat_lock_flush_cb(nowait);
+}
+
+/*
+ * Flush out locally pending lock statistics
+ *
+ * If no stats have been recorded, this function returns false.
+ *
+ * If nowait is true, this function returns true if the lock could not be
+ * acquired. Otherwise, return false.
+ */
+bool
+pgstat_lock_flush_cb(bool nowait)
+{
+ LWLock *lcktype_lock;
+ PgStat_LockEntry *lck_shstats;
+ bool lock_not_acquired = false;
+
+ if (!have_lockstats)
+ return false;
+
+ for (int i = 0; i <= LOCKTAG_LAST_TYPE; i++)
+ {
+ lcktype_lock = &pgStatLocal.shmem->lock.locks[i];
+ lck_shstats =
+ &pgStatLocal.shmem->lock.stats.stats[i];
+
+ if (!nowait)
+ LWLockAcquire(lcktype_lock, LW_EXCLUSIVE);
+ else if (!LWLockConditionalAcquire(lcktype_lock, LW_EXCLUSIVE))
+ {
+ lock_not_acquired = true;
+ continue;
+ }
+
+#define LOCKSTAT_ACC(fld) \
+ (lck_shstats->fld += PendingLockStats.stats[i].fld)
+ LOCKSTAT_ACC(requests);
+ LOCKSTAT_ACC(waits);
+ LOCKSTAT_ACC(timeouts);
+ LOCKSTAT_ACC(deadlock_timeouts);
+ LOCKSTAT_ACC(deadlocks);
+ LOCKSTAT_ACC(fastpath);
+#undef LOCKSTAT_ACC
+
+ LWLockRelease(lcktype_lock);
+ }
+
+ memset(&PendingLockStats, 0, sizeof(PendingLockStats));
+
+ have_lockstats = false;
+
+ return lock_not_acquired;
+}
+
+
+void
+pgstat_lock_init_shmem_cb(void *stats)
+{
+ PgStatShared_Lock *stat_shmem = (PgStatShared_Lock *) stats;
+
+ for (int i = 0; i <= LOCKTAG_LAST_TYPE; i++)
+ LWLockInitialize(&stat_shmem->locks[i], LWTRANCHE_PGSTATS_DATA);
+}
+
+void
+pgstat_lock_reset_all_cb(TimestampTz ts)
+{
+ for (int i = 0; i <= LOCKTAG_LAST_TYPE; i++)
+ {
+ LWLock *lcktype_lock = &pgStatLocal.shmem->lock.locks[i];
+ PgStat_LockEntry *lck_shstats = &pgStatLocal.shmem->lock.stats.stats[i];
+
+ LWLockAcquire(lcktype_lock, LW_EXCLUSIVE);
+
+ /*
+ * Use the lock in the first lock type PgStat_LockEntry to protect the
+ * reset timestamp as well.
+ */
+ if (i == 0)
+ pgStatLocal.shmem->lock.stats.stat_reset_timestamp = ts;
+
+ memset(lck_shstats, 0, sizeof(*lck_shstats));
+ LWLockRelease(lcktype_lock);
+ }
+}
+
+void
+pgstat_lock_snapshot_cb(void)
+{
+ for (int i = 0; i <= LOCKTAG_LAST_TYPE; i++)
+ {
+ LWLock *lcktype_lock = &pgStatLocal.shmem->lock.locks[i];
+ PgStat_LockEntry *lck_shstats = &pgStatLocal.shmem->lock.stats.stats[i];
+ PgStat_LockEntry *lck_snap = &pgStatLocal.snapshot.lock.stats[i];
+
+ LWLockAcquire(lcktype_lock, LW_SHARED);
+
+ /*
+ * Use the lock in the first lock type PgStat_LockEntry to protect the
+ * reset timestamp as well.
+ */
+ if (i == 0)
+ pgStatLocal.snapshot.lock.stat_reset_timestamp =
+ pgStatLocal.shmem->lock.stats.stat_reset_timestamp;
+
+ /* using struct assignment due to better type safety */
+ *lck_snap = *lck_shstats;
+ LWLockRelease(lcktype_lock);
+ }
+}
+
+#define PGSTAT_COUNT_LOCK_FUNC(stat) \
+void \
+CppConcat(pgstat_count_lock_,stat)(uint8 locktag_type) \
+{ \
+ Assert(locktag_type <= LOCKTAG_LAST_TYPE); \
+ PendingLockStats.stats[locktag_type].stat++; \
+ have_lockstats = true; \
+ pgstat_report_fixed = true; \
+}
+
+/* pgstat_count_lock_requests */
+PGSTAT_COUNT_LOCK_FUNC(requests)
+
+/* pgstat_count_lock_waits */
+PGSTAT_COUNT_LOCK_FUNC(waits)
+
+/* pgstat_count_lock_timeouts */
+PGSTAT_COUNT_LOCK_FUNC(timeouts)
+
+/* pgstat_count_lock_deadlock_timeouts */
+PGSTAT_COUNT_LOCK_FUNC(deadlock_timeouts)
+
+/* pgstat_count_lock_deadlocks */
+PGSTAT_COUNT_LOCK_FUNC(deadlocks)
+
+/* pgstat_count_lock_fastpath */
+PGSTAT_COUNT_LOCK_FUNC(fastpath)
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index a710508979e4..85a65071ac4c 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1725,6 +1725,42 @@ pg_stat_get_wal(PG_FUNCTION_ARGS)
wal_stats->stat_reset_timestamp));
}
+Datum
+pg_stat_get_lock(PG_FUNCTION_ARGS)
+{
+#define PG_STAT_LOCK_COLS 8
+ ReturnSetInfo *rsinfo;
+ PgStat_Lock *lock_stats;
+
+ InitMaterializedSRF(fcinfo, 0);
+ rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+
+ lock_stats = pgstat_fetch_stat_lock();
+
+ for (int lcktype = 0; lcktype <= LOCKTAG_LAST_TYPE; lcktype++)
+ {
+ const char *locktypename;
+ Datum values[PG_STAT_LOCK_COLS] = {0};
+ bool nulls[PG_STAT_LOCK_COLS] = {0};
+ PgStat_LockEntry *lck_stats = &lock_stats->stats[lcktype];
+
+ locktypename = LockTagTypeNames[lcktype];
+
+ values[0] = CStringGetTextDatum(locktypename);
+ values[1] = Int64GetDatum(lck_stats->requests);
+ values[2] = Int64GetDatum(lck_stats->waits);
+ values[3] = Int64GetDatum(lck_stats->timeouts);
+ values[4] = Int64GetDatum(lck_stats->deadlock_timeouts);
+ values[5] = Int64GetDatum(lck_stats->deadlocks);
+ values[6] = Int64GetDatum(lck_stats->fastpath);
+ values[7] = TimestampTzGetDatum(lock_stats->stat_reset_timestamp);
+
+ tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc, values, nulls);
+ }
+
+ return (Datum) 0;
+}
+
/*
* Returns statistics of SLRU caches.
*/
@@ -1909,6 +1945,7 @@ pg_stat_reset_shared(PG_FUNCTION_ARGS)
pgstat_reset_of_kind(PGSTAT_KIND_BGWRITER);
pgstat_reset_of_kind(PGSTAT_KIND_CHECKPOINTER);
pgstat_reset_of_kind(PGSTAT_KIND_IO);
+ pgstat_reset_of_kind(PGSTAT_KIND_LOCK);
XLogPrefetchResetStats();
pgstat_reset_of_kind(PGSTAT_KIND_SLRU);
pgstat_reset_of_kind(PGSTAT_KIND_WAL);
@@ -1926,6 +1963,8 @@ pg_stat_reset_shared(PG_FUNCTION_ARGS)
pgstat_reset_of_kind(PGSTAT_KIND_CHECKPOINTER);
else if (strcmp(target, "io") == 0)
pgstat_reset_of_kind(PGSTAT_KIND_IO);
+ else if (strcmp(target, "lock") == 0)
+ pgstat_reset_of_kind(PGSTAT_KIND_LOCK);
else if (strcmp(target, "recovery_prefetch") == 0)
XLogPrefetchResetStats();
else if (strcmp(target, "slru") == 0)
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 34b7fddb0e7a..96a339629fbe 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -6017,6 +6017,15 @@
proargnames => '{backend_type,object,context,reads,read_bytes,read_time,writes,write_bytes,write_time,writebacks,writeback_time,extends,extend_bytes,extend_time,hits,evictions,reuses,fsyncs,fsync_time,stats_reset}',
prosrc => 'pg_stat_get_io' },
+{ oid => '9375', descr => 'statistics: per lock type statistics',
+ proname => 'pg_stat_get_lock', prorows => '10', proretset => 't',
+ provolatile => 'v', proparallel => 'r', prorettype => 'record',
+ proargtypes => '',
+ proallargtypes => '{text,int8,int8,int8,int8,int8,int8,timestamptz}',
+ proargmodes => '{o,o,o,o,o,o,o,o}',
+ proargnames => '{locktype,requests,waits,timeouts,deadlock_timeouts,deadlocks,fastpath,stats_reset}',
+ prosrc => 'pg_stat_get_lock' },
+
{ oid => '6386', descr => 'statistics: backend IO statistics',
proname => 'pg_stat_get_backend_io', prorows => '5', proretset => 't',
provolatile => 'v', proparallel => 'r', prorettype => 'record',
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 7ae503e71a27..6c509e2cf031 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -16,6 +16,7 @@
#include "portability/instr_time.h"
#include "postmaster/pgarch.h" /* for MAX_XFN_CHARS */
#include "replication/conflict.h"
+#include "storage/lock.h"
#include "utils/backend_progress.h" /* for backward compatibility */ /* IWYU pragma: export */
#include "utils/backend_status.h" /* for backward compatibility */ /* IWYU pragma: export */
#include "utils/pgstat_kind.h"
@@ -340,6 +341,27 @@ typedef struct PgStat_IO
PgStat_BktypeIO stats[BACKEND_NUM_TYPES];
} PgStat_IO;
+typedef struct PgStat_LockEntry
+{
+ PgStat_Counter requests;
+ PgStat_Counter waits;
+ PgStat_Counter timeouts;
+ PgStat_Counter deadlock_timeouts;
+ PgStat_Counter deadlocks;
+ PgStat_Counter fastpath;
+} PgStat_LockEntry;
+
+typedef struct PgStat_PendingLock
+{
+ PgStat_LockEntry stats[LOCKTAG_LAST_TYPE + 1];
+} PgStat_PendingLock;
+
+typedef struct PgStat_Lock
+{
+ TimestampTz stat_reset_timestamp;
+ PgStat_LockEntry stats[LOCKTAG_LAST_TYPE + 1];
+} PgStat_Lock;
+
typedef struct PgStat_StatDBEntry
{
PgStat_Counter xact_commit;
@@ -609,6 +631,19 @@ extern bool pgstat_tracks_io_op(BackendType bktype, IOObject io_object,
IOContext io_context, IOOp io_op);
+/*
+ * Functions in pgstat_lock.c
+ */
+
+extern void pgstat_lock_flush(bool nowait);
+extern void pgstat_count_lock_requests(uint8 locktag_type);
+extern void pgstat_count_lock_waits(uint8 locktag_type);
+extern void pgstat_count_lock_timeouts(uint8 locktag_type);
+extern void pgstat_count_lock_deadlock_timeouts(uint8 locktag_type);
+extern void pgstat_count_lock_deadlocks(uint8 locktag_type);
+extern void pgstat_count_lock_fastpath(uint8 locktag_type);
+extern PgStat_Lock *pgstat_fetch_stat_lock(void);
+
/*
* Functions in pgstat_database.c
*/
diff --git a/src/include/utils/pgstat_internal.h b/src/include/utils/pgstat_internal.h
index 4d2b8aa6081a..30c0cff9cc3f 100644
--- a/src/include/utils/pgstat_internal.h
+++ b/src/include/utils/pgstat_internal.h
@@ -417,6 +417,16 @@ typedef struct PgStatShared_IO
PgStat_IO stats;
} PgStatShared_IO;
+typedef struct PgStatShared_Lock
+{
+ /*
+ * locks[i] protects stats.stats[i]. locks[0] also protects
+ * stats.stat_reset_timestamp.
+ */
+ LWLock locks[LOCKTAG_LAST_TYPE + 1];
+ PgStat_Lock stats;
+} PgStatShared_Lock;
+
typedef struct PgStatShared_SLRU
{
/* lock protects ->stats */
@@ -523,6 +533,7 @@ typedef struct PgStat_ShmemControl
PgStatShared_BgWriter bgwriter;
PgStatShared_Checkpointer checkpointer;
PgStatShared_IO io;
+ PgStatShared_Lock lock;
PgStatShared_SLRU slru;
PgStatShared_Wal wal;
@@ -555,6 +566,8 @@ typedef struct PgStat_Snapshot
PgStat_IO io;
+ PgStat_Lock lock;
+
PgStat_SLRUStats slru[SLRU_NUM_ELEMENTS];
PgStat_WalStats wal;
@@ -705,6 +718,14 @@ extern void pgstat_io_init_shmem_cb(void *stats);
extern void pgstat_io_reset_all_cb(TimestampTz ts);
extern void pgstat_io_snapshot_cb(void);
+/*
+ * Functions in pgstat_lock.c
+ */
+
+extern bool pgstat_lock_flush_cb(bool nowait);
+extern void pgstat_lock_init_shmem_cb(void *stats);
+extern void pgstat_lock_reset_all_cb(TimestampTz ts);
+extern void pgstat_lock_snapshot_cb(void);
/*
* Functions in pgstat_relation.c
diff --git a/src/include/utils/pgstat_kind.h b/src/include/utils/pgstat_kind.h
index eb5f0b3ae6db..f8e95f555331 100644
--- a/src/include/utils/pgstat_kind.h
+++ b/src/include/utils/pgstat_kind.h
@@ -36,8 +36,9 @@
#define PGSTAT_KIND_BGWRITER 8
#define PGSTAT_KIND_CHECKPOINTER 9
#define PGSTAT_KIND_IO 10
-#define PGSTAT_KIND_SLRU 11
-#define PGSTAT_KIND_WAL 12
+#define PGSTAT_KIND_LOCK 11
+#define PGSTAT_KIND_SLRU 12
+#define PGSTAT_KIND_WAL 13
#define PGSTAT_KIND_BUILTIN_MIN PGSTAT_KIND_DATABASE
#define PGSTAT_KIND_BUILTIN_MAX PGSTAT_KIND_WAL
diff --git a/src/test/isolation/expected/deadlock-hard.out b/src/test/isolation/expected/deadlock-hard.out
index 460653f2b86a..ff448da52991 100644
--- a/src/test/isolation/expected/deadlock-hard.out
+++ b/src/test/isolation/expected/deadlock-hard.out
@@ -1,6 +1,12 @@
Parsed test spec with 8 sessions
-starting permutation: s1a1 s2a2 s3a3 s4a4 s5a5 s6a6 s7a7 s8a8 s1a2 s2a3 s3a4 s4a5 s5a6 s6a7 s7a8 s8a1 s8c s7c s6c s5c s4c s3c s2c s1c
+starting permutation: s1rl s1a1 s2a2 s3a3 s4a4 s5a5 s6a6 s7a7 s8a8 s1a2 s2a3 s3a4 s4a5 s5a6 s6a7 s7a8 s8a1 s8c s8f s7c s6c s5c s4c s3c s2c s1c s1sl
+step s1rl: SELECT pg_stat_reset_shared('lock');
+pg_stat_reset_shared
+--------------------
+
+(1 row)
+
step s1a1: LOCK TABLE a1;
step s2a2: LOCK TABLE a2;
step s3a3: LOCK TABLE a3;
@@ -21,6 +27,12 @@ step s8a1: <... completed>
ERROR: deadlock detected
step s7a8: <... completed>
step s8c: COMMIT;
+step s8f: SELECT pg_stat_force_next_flush();
+pg_stat_force_next_flush
+------------------------
+
+(1 row)
+
step s7c: COMMIT;
step s6a7: <... completed>
step s6c: COMMIT;
@@ -34,3 +46,9 @@ step s2a3: <... completed>
step s2c: COMMIT;
step s1a2: <... completed>
step s1c: COMMIT;
+step s1sl: SELECT deadlocks > 0, deadlock_timeouts > 0 FROM pg_stat_lock WHERE locktype = 'relation';
+?column?|?column?
+--------+--------
+t |t
+(1 row)
+
diff --git a/src/test/isolation/specs/deadlock-hard.spec b/src/test/isolation/specs/deadlock-hard.spec
index 60bedca237a4..6e8330662b48 100644
--- a/src/test/isolation/specs/deadlock-hard.spec
+++ b/src/test/isolation/specs/deadlock-hard.spec
@@ -25,6 +25,8 @@ setup { BEGIN; SET deadlock_timeout = '100s'; }
step s1a1 { LOCK TABLE a1; }
step s1a2 { LOCK TABLE a2; }
step s1c { COMMIT; }
+step s1sl { SELECT deadlocks > 0, deadlock_timeouts > 0 FROM pg_stat_lock WHERE locktype = 'relation'; }
+step s1rl { SELECT pg_stat_reset_shared('lock'); }
session s2
setup { BEGIN; SET deadlock_timeout = '100s'; }
@@ -67,6 +69,7 @@ setup { BEGIN; SET deadlock_timeout = '10ms'; }
step s8a8 { LOCK TABLE a8; }
step s8a1 { LOCK TABLE a1; }
step s8c { COMMIT; }
+step s8f { SELECT pg_stat_force_next_flush(); }
# Note: when s8a1 detects the deadlock and fails, s7a8 is released, making
# it timing-dependent which query completion is received first by the tester.
@@ -76,4 +79,4 @@ step s8c { COMMIT; }
# dummy blocking mark to s8a1 to ensure it will be reported as "waiting"
# regardless of that.
-permutation s1a1 s2a2 s3a3 s4a4 s5a5 s6a6 s7a7 s8a8 s1a2 s2a3 s3a4 s4a5 s5a6 s6a7 s7a8(s8a1) s8a1(*) s8c s7c s6c s5c s4c s3c s2c s1c
+permutation s1rl s1a1 s2a2 s3a3 s4a4 s5a5 s6a6 s7a7 s8a8 s1a2 s2a3 s3a4 s4a5 s5a6 s6a7 s7a8(s8a1) s8a1(*) s8c s8f s7c s6c s5c s4c s3c s2c s1c s1sl
diff --git a/src/test/regress/expected/advisory_lock.out b/src/test/regress/expected/advisory_lock.out
index 02e07765ac2e..fdaa1756ba41 100644
--- a/src/test/regress/expected/advisory_lock.out
+++ b/src/test/regress/expected/advisory_lock.out
@@ -2,6 +2,12 @@
-- ADVISORY LOCKS
--
SELECT oid AS datoid FROM pg_database WHERE datname = current_database() \gset
+SELECT pg_stat_reset_shared('lock');
+ pg_stat_reset_shared
+----------------------
+
+(1 row)
+
BEGIN;
SELECT
pg_advisory_xact_lock(1), pg_advisory_xact_lock_shared(2),
@@ -48,6 +54,12 @@ WARNING: you don't own a lock of type ShareLock
f | f | f | f
(1 row)
+SELECT pg_stat_force_next_flush();
+ pg_stat_force_next_flush
+--------------------------
+
+(1 row)
+
-- automatically release xact locks at commit
COMMIT;
SELECT count(*) FROM pg_locks WHERE locktype = 'advisory' AND database = :datoid;
@@ -56,6 +68,12 @@ SELECT count(*) FROM pg_locks WHERE locktype = 'advisory' AND database = :datoid
0
(1 row)
+SELECT requests FROM pg_stat_lock WHERE locktype = 'advisory';
+ requests
+----------
+ 4
+(1 row)
+
BEGIN;
-- holding both session and xact locks on the same objects, xact first
SELECT
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 2bf968ae3d37..2a79735c865e 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1948,6 +1948,15 @@ pg_stat_io| SELECT backend_type,
fsync_time,
stats_reset
FROM pg_stat_get_io() b(backend_type, object, context, reads, read_bytes, read_time, writes, write_bytes, write_time, writebacks, writeback_time, extends, extend_bytes, extend_time, hits, evictions, reuses, fsyncs, fsync_time, stats_reset);
+pg_stat_lock| SELECT locktype,
+ requests,
+ waits,
+ timeouts,
+ deadlock_timeouts,
+ deadlocks,
+ fastpath,
+ stats_reset
+ FROM pg_stat_get_lock() l(locktype, requests, waits, timeouts, deadlock_timeouts, deadlocks, fastpath, stats_reset);
pg_stat_progress_analyze| SELECT s.pid,
s.datid,
d.datname,
diff --git a/src/test/regress/sql/advisory_lock.sql b/src/test/regress/sql/advisory_lock.sql
index 8513ab8e98fe..f1bff60fd37d 100644
--- a/src/test/regress/sql/advisory_lock.sql
+++ b/src/test/regress/sql/advisory_lock.sql
@@ -4,6 +4,8 @@
SELECT oid AS datoid FROM pg_database WHERE datname = current_database() \gset
+SELECT pg_stat_reset_shared('lock');
+
BEGIN;
SELECT
@@ -26,12 +28,14 @@ SELECT
pg_advisory_unlock(1), pg_advisory_unlock_shared(2),
pg_advisory_unlock(1, 1), pg_advisory_unlock_shared(2, 2);
+SELECT pg_stat_force_next_flush();
-- automatically release xact locks at commit
COMMIT;
SELECT count(*) FROM pg_locks WHERE locktype = 'advisory' AND database = :datoid;
+SELECT requests FROM pg_stat_lock WHERE locktype = 'advisory';
BEGIN;
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 2ca7b75af579..7e9c520678a2 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2224,6 +2224,7 @@ PgStatShared_HashEntry
PgStatShared_IO
PgStatShared_InjectionPoint
PgStatShared_InjectionPointFixed
+PgStatShared_Lock
PgStatShared_Relation
PgStatShared_ReplSlot
PgStatShared_SLRU
@@ -2246,8 +2247,11 @@ PgStat_HashKey
PgStat_IO
PgStat_KindInfo
PgStat_LocalState
+PgStat_Lock
+PgStat_LockEntry
PgStat_PendingDroppedStatsItem
PgStat_PendingIO
+PgStat_PendingLock
PgStat_SLRUStats
PgStat_ShmemControl
PgStat_Snapshot