Refactor PROCLOCK hash table into partitioned list allocator

Started by Andrey Borodin13 days ago5 messages
#1Andrey Borodin
x4mmm@yandex-team.ru
1 attachment(s)

Hi hackers,

Following up on the Discord discussion about the PROCLOCK hash table being
a "weird allocator" that we never actually use for lookups - I took a stab at
replacing it with a simpler partitioned free list approach as was suggested.
I was doing this mostly to educate myself on Lock Manager internals.

The current implementation uses LockMethodProcLockHash purely as an allocator.
We never do hash lookups by key; we only allocate entries, link them to the lock's
procLocks list, and free them later. Using a full hash table for this adds
unnecessary complexity and maybe even overhead (I did not measure this).

The attached patch replaces this with:
- ProcLockArray: A fixed-size array of all PROCLOCK structs (allocated at startup)
- ProcLockFreeList: Partitioned free lists, one per lock partition to reduce contention
- ProcLockAlloc/Free: Simple push/pop operations on the free lists
- PROCLOCK lookup: Linear traversal of lock->procLocks (see LockRefindAndRelease()
and FastPathGetRelationLockEntry())

The last point bothers me most. It seems like this traversals are expected to be short.
But I'm not 100% sure.

This patch removes the proclock_hash() function and ProcLockHashCode() entirely, and
eliminates all hash_search() calls for PROCLOCKs. The allocation/deallocation
is now just dlist operations instead of hash table management.

Would appreciate your thoughts on this approach. Is this the direction worth working on?

Best regards, Andrey Borodin.

Attachments:

v1-0001-Replace-PROCLOCK-hash-table-with-partitioned-free.patchapplication/octet-stream; name=v1-0001-Replace-PROCLOCK-hash-table-with-partitioned-free.patch; x-unix-mode=0644Download
From 3e927ff45fc1d72445e09842cab77d8d13c96c2b Mon Sep 17 00:00:00 2001
From: Andrey Borodin <amborodin@acm.org>
Date: Tue, 25 Nov 2025 22:15:49 +0500
Subject: [PATCH v1] Replace PROCLOCK hash table with partitioned free list
 allocator

The PROCLOCK hash table (LockMethodProcLockHash) was an unnecessarily complex
allocator. PROCLOCKs are never looked up by hash key - they're only allocated,
initialized, and freed. Using a hash table for this was too complex.

This patch replaces the hash table with a simpler and faster approach:
- ProcLockArray: Fixed array of all PROCLOCK structs
- ProcLockFreeList: Partitioned free lists (one per lock partition)
- ProcLockAlloc/Free: Simple free list pop/push operations
- PROCLOCK lookup: Linear search through lock->procLocks list (typically short)

Removed functions:
- proclock_hash() - specialized hash function, no longer needed
- ProcLockHashCode() - wrapper around proclock_hash()
---
 src/backend/storage/lmgr/README |  25 +-
 src/backend/storage/lmgr/lock.c | 601 +++++++++++++++++---------------
 src/include/storage/lock.h      |   9 +-
 3 files changed, 325 insertions(+), 310 deletions(-)

diff --git a/src/backend/storage/lmgr/README b/src/backend/storage/lmgr/README
index 45de0fd2bd6..49ed9cec95f 100644
--- a/src/backend/storage/lmgr/README
+++ b/src/backend/storage/lmgr/README
@@ -158,12 +158,9 @@ go to zero, the LOCK object is no longer needed and can be freed.
 The lock manager's PROCLOCK objects contain:
 
 tag -
-    The key fields that are used for hashing entries in the shared memory
-    PROCLOCK hash table.  This is declared as a separate struct to ensure that
-    we always zero out the correct number of bytes.  It is critical that any
-    alignment-padding bytes the compiler might insert in the struct be zeroed
-    out, else the hash computation will be random.  (Currently, we are careful
-    to define struct PROCLOCKTAG so that there are no padding bytes.)
+    Identifies which LOCK object and PGPROC this PROCLOCK belongs to.
+    This is declared as a separate struct to ensure consistency and
+    to facilitate searching through a lock's procLocks list.
 
     tag.myLock
         Pointer to the shared LOCK object this PROCLOCK is for.
@@ -215,15 +212,13 @@ Here are the details:
 its LOCKTAG value.  The partition's LWLock is considered to protect all the
 LOCK objects of that partition as well as their subsidiary PROCLOCKs.
 
-* The shared-memory hash tables for LOCKs and PROCLOCKs are organized
-so that different partitions use different hash chains, and thus there
-is no conflict in working with objects in different partitions.  This
-is supported directly by dynahash.c's "partitioned table" mechanism
-for the LOCK table: we need only ensure that the partition number is
-taken from the low-order bits of the dynahash hash value for the LOCKTAG.
-To make it work for PROCLOCKs, we have to ensure that a PROCLOCK's hash
-value has the same low-order bits as its associated LOCK.  This requires
-a specialized hash function (see proclock_hash).
+* The shared-memory LOCK hash table is organized so that different partitions
+use different hash chains, and thus there is no conflict in working with
+LOCK objects in different partitions.  This is supported directly by
+dynahash.c's "partitioned table" mechanism.  PROCLOCKs are allocated from
+partitioned free lists (one free list per partition) rather than using a hash
+table, which is more efficient since PROCLOCKs are never looked up by hash
+key - they are only allocated, initialized, and freed.
 
 * Formerly, each PGPROC had a single list of PROCLOCKs belonging to it.
 This has now been split into per-partition lists, so that access to a
diff --git a/src/backend/storage/lmgr/lock.c b/src/backend/storage/lmgr/lock.c
index 9015ba3caf7..bfdfa9c9a72 100644
--- a/src/backend/storage/lmgr/lock.c
+++ b/src/backend/storage/lmgr/lock.c
@@ -313,13 +313,17 @@ static volatile FastPathStrongRelationLockData *FastPathStrongRelationLocks;
 
 
 /*
- * Pointers to hash tables containing lock state
+ * Pointers to shared lock state
  *
- * The LockMethodLockHash and LockMethodProcLockHash hash tables are in
- * shared memory; LockMethodLocalHash is local to each backend.
+ * LockMethodLockHash is a hash table for LOCK objects.  ProcLockArray is
+ * an array of all PROCLOCK objects, managed via partitioned free lists in
+ * ProcLockFreeList.  These are all in shared memory; LockMethodLocalHash
+ * is local to each backend.
  */
 static HTAB *LockMethodLockHash;
-static HTAB *LockMethodProcLockHash;
+static PROCLOCK *ProcLockArray;		/* array of all PROCLOCK structs */
+static dlist_head *ProcLockFreeList;	/* partitioned free lists */
+static int	MaxProcLocks;			/* size of ProcLockArray */
 static HTAB *LockMethodLocalHash;
 
 
@@ -407,7 +411,6 @@ PROCLOCK_PRINT(const char *where, const PROCLOCK *proclockP)
 #endif							/* not LOCK_DEBUG */
 
 
-static uint32 proclock_hash(const void *key, Size keysize);
 static void RemoveLocalLock(LOCALLOCK *locallock);
 static PROCLOCK *SetupLockInTable(LockMethod lockMethodTable, PGPROC *proc,
 								  const LOCKTAG *locktag, uint32 hashcode, LOCKMODE lockmode);
@@ -434,11 +437,12 @@ static void GetSingleProcBlockerStatusData(PGPROC *blocked_proc,
  * Initialize the lock manager's shmem data structures.
  *
  * This is called from CreateSharedMemoryAndSemaphores(), which see for more
- * comments.  In the normal postmaster case, the shared hash tables are
- * created here, and backends inherit pointers to them via fork().  In the
- * EXEC_BACKEND case, each backend re-executes this code to obtain pointers to
- * the already existing shared hash tables.  In either case, each backend must
- * also call InitLockManagerAccess() to create the locallock hash table.
+ * comments.  In the normal postmaster case, the shared hash table for LOCKs
+ * and the PROCLOCK array/freelists are created here, and backends inherit
+ * pointers to them via fork().  In the EXEC_BACKEND case, each backend
+ * re-executes this code to obtain pointers to the already existing shared
+ * structures.  In either case, each backend must also call
+ * InitLockManagerAccess() to create the locallock hash table.
  */
 void
 LockManagerShmemInit(void)
@@ -471,22 +475,39 @@ LockManagerShmemInit(void)
 
 	/* Assume an average of 2 holders per lock */
 	max_table_size *= 2;
-	init_table_size *= 2;
 
 	/*
-	 * Allocate hash table for PROCLOCK structs.  This stores
+	 * Allocate array for PROCLOCK structs.  This stores
 	 * per-lock-per-holder information.
 	 */
-	info.keysize = sizeof(PROCLOCKTAG);
-	info.entrysize = sizeof(PROCLOCK);
-	info.hash = proclock_hash;
-	info.num_partitions = NUM_LOCK_PARTITIONS;
+	MaxProcLocks = max_table_size;
+	ProcLockArray = (PROCLOCK *)
+		ShmemInitStruct("PROCLOCK array",
+						mul_size(MaxProcLocks, sizeof(PROCLOCK)),
+						&found);
+
+	ProcLockFreeList = (dlist_head *)
+		ShmemInitStruct("PROCLOCK free lists",
+						mul_size(NUM_LOCK_PARTITIONS, sizeof(dlist_head)),
+						&found);
+
+	if (!found)
+	{
+		int			i;
 
-	LockMethodProcLockHash = ShmemInitHash("PROCLOCK hash",
-										   init_table_size,
-										   max_table_size,
-										   &info,
-										   HASH_ELEM | HASH_FUNCTION | HASH_PARTITION);
+		/* Initialize partitioned free lists */
+		for (i = 0; i < NUM_LOCK_PARTITIONS; i++)
+			dlist_init(&ProcLockFreeList[i]);
+
+		/* Link all PROCLOCK entries into the free lists */
+		for (i = 0; i < MaxProcLocks; i++)
+		{
+			int			partition = i % NUM_LOCK_PARTITIONS;
+
+			dlist_push_tail(&ProcLockFreeList[partition],
+							&ProcLockArray[i].lockLink);
+		}
+	}
 
 	/*
 	 * Allocate fast-path structures.
@@ -559,61 +580,37 @@ LockTagHashCode(const LOCKTAG *locktag)
 	return get_hash_value(LockMethodLockHash, locktag);
 }
 
+
 /*
- * Compute the hash code associated with a PROCLOCKTAG.
+ * Allocate a PROCLOCK from the free list for the given partition.
  *
- * Because we want to use just one set of partition locks for both the
- * LOCK and PROCLOCK hash tables, we have to make sure that PROCLOCKs
- * fall into the same partition number as their associated LOCKs.
- * dynahash.c expects the partition number to be the low-order bits of
- * the hash code, and therefore a PROCLOCKTAG's hash code must have the
- * same low-order bits as the associated LOCKTAG's hash code.  We achieve
- * this with this specialized hash function.
+ * Caller must hold the partition LWLock.
+ * Returns NULL if no free entries are available.
  */
-static uint32
-proclock_hash(const void *key, Size keysize)
+static PROCLOCK *
+ProcLockAlloc(int partition)
 {
-	const PROCLOCKTAG *proclocktag = (const PROCLOCKTAG *) key;
-	uint32		lockhash;
-	Datum		procptr;
-
-	Assert(keysize == sizeof(PROCLOCKTAG));
+	PROCLOCK   *proclock;
+	dlist_node *node;
 
-	/* Look into the associated LOCK object, and compute its hash code */
-	lockhash = LockTagHashCode(&proclocktag->myLock->tag);
+	if (dlist_is_empty(&ProcLockFreeList[partition]))
+		return NULL;
 
-	/*
-	 * To make the hash code also depend on the PGPROC, we xor the proc
-	 * struct's address into the hash code, left-shifted so that the
-	 * partition-number bits don't change.  Since this is only a hash, we
-	 * don't care if we lose high-order bits of the address; use an
-	 * intermediate variable to suppress cast-pointer-to-int warnings.
-	 */
-	procptr = PointerGetDatum(proclocktag->myProc);
-	lockhash ^= DatumGetUInt32(procptr) << LOG2_NUM_LOCK_PARTITIONS;
+	node = dlist_pop_head_node(&ProcLockFreeList[partition]);
+	proclock = dlist_container(PROCLOCK, lockLink, node);
 
-	return lockhash;
+	return proclock;
 }
 
 /*
- * Compute the hash code associated with a PROCLOCKTAG, given the hashcode
- * for its underlying LOCK.
+ * Return a PROCLOCK to the free list for the given partition.
  *
- * We use this just to avoid redundant calls of LockTagHashCode().
+ * Caller must hold the partition LWLock.
  */
-static inline uint32
-ProcLockHashCode(const PROCLOCKTAG *proclocktag, uint32 hashcode)
+static void
+ProcLockFree(PROCLOCK *proclock, int partition)
 {
-	uint32		lockhash = hashcode;
-	Datum		procptr;
-
-	/*
-	 * This must match proclock_hash()!
-	 */
-	procptr = PointerGetDatum(proclocktag->myProc);
-	lockhash ^= DatumGetUInt32(procptr) << LOG2_NUM_LOCK_PARTITIONS;
-
-	return lockhash;
+	dlist_push_head(&ProcLockFreeList[partition], &proclock->lockLink);
 }
 
 /*
@@ -1127,18 +1124,11 @@ LockAcquireExtended(const LOCKTAG *locktag,
 
 		if (proclock->holdMask == 0)
 		{
-			uint32		proclock_hashcode;
+			uint32		partition = LockHashPartition(hashcode);
 
-			proclock_hashcode = ProcLockHashCode(&proclock->tag,
-												 hashcode);
 			dlist_delete(&proclock->lockLink);
 			dlist_delete(&proclock->procLink);
-			if (!hash_search_with_hash_value(LockMethodProcLockHash,
-											 &(proclock->tag),
-											 proclock_hashcode,
-											 HASH_REMOVE,
-											 NULL))
-				elog(PANIC, "proclock table corrupted");
+			ProcLockFree(proclock, partition);
 		}
 		else
 			PROCLOCK_PRINT("LockAcquire: did not join wait queue", proclock);
@@ -1285,8 +1275,6 @@ SetupLockInTable(LockMethod lockMethodTable, PGPROC *proc,
 {
 	LOCK	   *lock;
 	PROCLOCK   *proclock;
-	PROCLOCKTAG proclocktag;
-	uint32		proclock_hashcode;
 	bool		found;
 
 	/*
@@ -1324,50 +1312,60 @@ SetupLockInTable(LockMethod lockMethodTable, PGPROC *proc,
 	}
 
 	/*
-	 * Create the hash key for the proclock table.
-	 */
-	proclocktag.myLock = lock;
-	proclocktag.myProc = proc;
-
-	proclock_hashcode = ProcLockHashCode(&proclocktag, hashcode);
-
-	/*
-	 * Find or create a proclock entry with this tag
+	 * Find or create a proclock entry for this lock and proc.
+	 * Search the lock's procLocks list to see if we already have one.
 	 */
-	proclock = (PROCLOCK *) hash_search_with_hash_value(LockMethodProcLockHash,
-														&proclocktag,
-														proclock_hashcode,
-														HASH_ENTER_NULL,
-														&found);
-	if (!proclock)
+	proclock = NULL;
+	found = false;
 	{
-		/* Oops, not enough shmem for the proclock */
-		if (lock->nRequested == 0)
+		dlist_iter	iter;
+
+		dlist_foreach(iter, &lock->procLocks)
 		{
-			/*
-			 * There are no other requestors of this lock, so garbage-collect
-			 * the lock object.  We *must* do this to avoid a permanent leak
-			 * of shared memory, because there won't be anything to cause
-			 * anyone to release the lock object later.
-			 */
-			Assert(dlist_is_empty(&(lock->procLocks)));
-			if (!hash_search_with_hash_value(LockMethodLockHash,
-											 &(lock->tag),
-											 hashcode,
-											 HASH_REMOVE,
-											 NULL))
-				elog(PANIC, "lock table corrupted");
+			PROCLOCK   *existing;
+
+			existing = dlist_container(PROCLOCK, lockLink, iter.cur);
+			if (existing->tag.myProc == proc)
+			{
+				proclock = existing;
+				found = true;
+				break;
+			}
 		}
-		return NULL;
 	}
 
-	/*
-	 * If new, initialize the new entry
-	 */
 	if (!found)
 	{
 		uint32		partition = LockHashPartition(hashcode);
 
+		/* Allocate a new PROCLOCK from the free list */
+		proclock = ProcLockAlloc(partition);
+		if (!proclock)
+		{
+			/* Oops, not enough shmem for the proclock */
+			if (lock->nRequested == 0)
+			{
+				/*
+				 * There are no other requestors of this lock, so garbage-collect
+				 * the lock object.  We *must* do this to avoid a permanent leak
+				 * of shared memory, because there won't be anything to cause
+				 * anyone to release the lock object later.
+				 */
+				Assert(dlist_is_empty(&(lock->procLocks)));
+				if (!hash_search_with_hash_value(LockMethodLockHash,
+												 &(lock->tag),
+												 hashcode,
+												 HASH_REMOVE,
+												 NULL))
+					elog(PANIC, "lock table corrupted");
+			}
+			return NULL;
+		}
+
+		/* Initialize the new entry */
+		proclock->tag.myLock = lock;
+		proclock->tag.myProc = proc;
+
 		/*
 		 * It might seem unsafe to access proclock->groupLeader without a
 		 * lock, but it's not really.  Either we are initializing a proclock
@@ -1745,18 +1743,12 @@ CleanUpLock(LOCK *lock, PROCLOCK *proclock,
 	 */
 	if (proclock->holdMask == 0)
 	{
-		uint32		proclock_hashcode;
+		uint32		partition = LockHashPartition(hashcode);
 
 		PROCLOCK_PRINT("CleanUpLock: deleting", proclock);
 		dlist_delete(&proclock->lockLink);
 		dlist_delete(&proclock->procLink);
-		proclock_hashcode = ProcLockHashCode(&proclock->tag, hashcode);
-		if (!hash_search_with_hash_value(LockMethodProcLockHash,
-										 &(proclock->tag),
-										 proclock_hashcode,
-										 HASH_REMOVE,
-										 NULL))
-			elog(PANIC, "proclock table corrupted");
+		ProcLockFree(proclock, partition);
 	}
 
 	if (lock->nRequested == 0)
@@ -2241,8 +2233,6 @@ LockRelease(const LOCKTAG *locktag, LOCKMODE lockmode, bool sessionLock)
 	lock = locallock->lock;
 	if (!lock)
 	{
-		PROCLOCKTAG proclocktag;
-
 		Assert(EligibleForRelationFastPath(locktag, lockmode));
 		lock = (LOCK *) hash_search_with_hash_value(LockMethodLockHash,
 													locktag,
@@ -2253,12 +2243,23 @@ LockRelease(const LOCKTAG *locktag, LOCKMODE lockmode, bool sessionLock)
 			elog(ERROR, "failed to re-find shared lock object");
 		locallock->lock = lock;
 
-		proclocktag.myLock = lock;
-		proclocktag.myProc = MyProc;
-		locallock->proclock = (PROCLOCK *) hash_search(LockMethodProcLockHash,
-													   &proclocktag,
-													   HASH_FIND,
-													   NULL);
+		/* Find the PROCLOCK for this lock and proc */
+		locallock->proclock = NULL;
+		{
+			dlist_iter	iter;
+
+			dlist_foreach(iter, &lock->procLocks)
+			{
+				PROCLOCK   *proclock_item;
+
+				proclock_item = dlist_container(PROCLOCK, lockLink, iter.cur);
+				if (proclock_item->tag.myProc == MyProc)
+				{
+					locallock->proclock = proclock_item;
+					break;
+				}
+			}
+		}
 		if (!locallock->proclock)
 			elog(ERROR, "failed to re-find shared proclock object");
 	}
@@ -3015,8 +3016,7 @@ FastPathGetRelationLockEntry(LOCALLOCK *locallock)
 	if (proclock == NULL)
 	{
 		LOCK	   *lock;
-		PROCLOCKTAG proclocktag;
-		uint32		proclock_hashcode;
+		dlist_iter	iter;
 
 		LWLockAcquire(partitionLock, LW_SHARED);
 
@@ -3028,16 +3028,19 @@ FastPathGetRelationLockEntry(LOCALLOCK *locallock)
 		if (!lock)
 			elog(ERROR, "failed to re-find shared lock object");
 
-		proclocktag.myLock = lock;
-		proclocktag.myProc = MyProc;
+		/* Find the PROCLOCK for this lock and proc */
+		proclock = NULL;
+		dlist_foreach(iter, &lock->procLocks)
+		{
+			PROCLOCK   *existing;
 
-		proclock_hashcode = ProcLockHashCode(&proclocktag, locallock->hashcode);
-		proclock = (PROCLOCK *)
-			hash_search_with_hash_value(LockMethodProcLockHash,
-										&proclocktag,
-										proclock_hashcode,
-										HASH_FIND,
-										NULL);
+			existing = dlist_container(PROCLOCK, lockLink, iter.cur);
+			if (existing->tag.myProc == MyProc)
+			{
+				proclock = existing;
+				break;
+			}
+		}
 		if (!proclock)
 			elog(ERROR, "failed to re-find shared proclock object");
 		LWLockRelease(partitionLock);
@@ -3287,9 +3290,7 @@ LockRefindAndRelease(LockMethod lockMethodTable, PGPROC *proc,
 {
 	LOCK	   *lock;
 	PROCLOCK   *proclock;
-	PROCLOCKTAG proclocktag;
 	uint32		hashcode;
-	uint32		proclock_hashcode;
 	LWLock	   *partitionLock;
 	bool		wakeupNeeded;
 
@@ -3312,16 +3313,22 @@ LockRefindAndRelease(LockMethod lockMethodTable, PGPROC *proc,
 	/*
 	 * Re-find the proclock object (ditto).
 	 */
-	proclocktag.myLock = lock;
-	proclocktag.myProc = proc;
+	proclock = NULL;
+	{
+		dlist_iter	iter;
 
-	proclock_hashcode = ProcLockHashCode(&proclocktag, hashcode);
+		dlist_foreach(iter, &lock->procLocks)
+		{
+			PROCLOCK   *existing;
 
-	proclock = (PROCLOCK *) hash_search_with_hash_value(LockMethodProcLockHash,
-														&proclocktag,
-														proclock_hashcode,
-														HASH_FIND,
-														NULL);
+			existing = dlist_container(PROCLOCK, lockLink, iter.cur);
+			if (existing->tag.myProc == proc)
+			{
+				proclock = existing;
+				break;
+			}
+		}
+	}
 	if (!proclock)
 		elog(PANIC, "failed to re-find shared proclock object");
 
@@ -3576,7 +3583,6 @@ PostPrepare_Locks(FullTransactionId fxid)
 	LOCALLOCK  *locallock;
 	LOCK	   *lock;
 	PROCLOCK   *proclock;
-	PROCLOCKTAG proclocktag;
 	int			partition;
 
 	/* Can't prepare a lock group follower. */
@@ -3694,47 +3700,29 @@ PostPrepare_Locks(FullTransactionId fxid)
 			if (proclock->releaseMask == 0)
 				continue;
 
-			/* Else we should be releasing all locks */
-			if (proclock->releaseMask != proclock->holdMask)
-				elog(PANIC, "we seem to have dropped a bit somewhere");
-
-			/*
-			 * We cannot simply modify proclock->tag.myProc to reassign
-			 * ownership of the lock, because that's part of the hash key and
-			 * the proclock would then be in the wrong hash chain.  Instead
-			 * use hash_update_hash_key.  (We used to create a new hash entry,
-			 * but that risks out-of-memory failure if other processes are
-			 * busy making proclocks too.)	We must unlink the proclock from
-			 * our procLink chain and put it into the new proc's chain, too.
-			 *
-			 * Note: the updated proclock hash key will still belong to the
-			 * same hash partition, cf proclock_hash().  So the partition lock
-			 * we already hold is sufficient for this.
-			 */
-			dlist_delete(&proclock->procLink);
+		/* Else we should be releasing all locks */
+		if (proclock->releaseMask != proclock->holdMask)
+			elog(PANIC, "we seem to have dropped a bit somewhere");
 
-			/*
-			 * Create the new hash key for the proclock.
-			 */
-			proclocktag.myLock = lock;
-			proclocktag.myProc = newproc;
+		/*
+		 * Unlink the proclock from our procLink chain and update its tag
+		 * to point to the new proc, then re-link it into the new proc's
+		 * chain.  Note that we're already holding the correct partition
+		 * lock since PROCLOCK partitioning matches LOCK partitioning.
+		 */
+		dlist_delete(&proclock->procLink);
 
-			/*
-			 * Update groupLeader pointer to point to the new proc.  (We'd
-			 * better not be a member of somebody else's lock group!)
-			 */
-			Assert(proclock->groupLeader == proclock->tag.myProc);
-			proclock->groupLeader = newproc;
+		/*
+		 * Update groupLeader pointer to point to the new proc.  (We'd
+		 * better not be a member of somebody else's lock group!)
+		 */
+		Assert(proclock->groupLeader == proclock->tag.myProc);
+		proclock->groupLeader = newproc;
 
-			/*
-			 * Update the proclock.  We should not find any existing entry for
-			 * the same hash key, since there can be only one entry for any
-			 * given lock with my own proc.
-			 */
-			if (!hash_update_hash_key(LockMethodProcLockHash,
-									  proclock,
-									  &proclocktag))
-				elog(PANIC, "duplicate entry found while reassigning a prepared transaction's locks");
+		/*
+		 * Update the proclock tag to point to the new proc.
+		 */
+		proclock->tag.myProc = newproc;
 
 			/* Re-link into the new proc's proclock list */
 			dlist_push_tail(&newproc->myProcLocks[partition], &proclock->procLink);
@@ -3762,9 +3750,10 @@ LockManagerShmemSize(void)
 	max_table_size = NLOCKENTS();
 	size = add_size(size, hash_estimate_size(max_table_size, sizeof(LOCK)));
 
-	/* proclock hash table */
+	/* proclock array and free lists */
 	max_table_size *= 2;
-	size = add_size(size, hash_estimate_size(max_table_size, sizeof(PROCLOCK)));
+	size = add_size(size, mul_size(max_table_size, sizeof(PROCLOCK)));
+	size = add_size(size, mul_size(NUM_LOCK_PARTITIONS, sizeof(dlist_head)));
 
 	/*
 	 * Since NLOCKENTS is only an estimate, add 10% safety margin.
@@ -3793,6 +3782,7 @@ LockData *
 GetLockStatusData(void)
 {
 	LockData   *data;
+	LOCK	   *lock;
 	PROCLOCK   *proclock;
 	HASH_SEQ_STATUS seqstat;
 	int			els;
@@ -3920,40 +3910,53 @@ GetLockStatusData(void)
 	for (i = 0; i < NUM_LOCK_PARTITIONS; i++)
 		LWLockAcquire(LockHashPartitionLockByIndex(i), LW_SHARED);
 
-	/* Now we can safely count the number of proclocks */
-	data->nelements = el + hash_get_num_entries(LockMethodProcLockHash);
-	if (data->nelements > els)
+	/*
+	 * Now scan all locks to count and collect proclock data.
+	 * We iterate through the LOCK hash table and for each lock,
+	 * iterate through its procLocks list.
+	 */
+	hash_seq_init(&seqstat, LockMethodLockHash);
+
+	while ((lock = (LOCK *) hash_seq_search(&seqstat)))
 	{
-		els = data->nelements;
-		data->locks = (LockInstanceData *)
-			repalloc(data->locks, sizeof(LockInstanceData) * els);
-	}
+		dlist_iter	iter;
 
-	/* Now scan the tables to copy the data */
-	hash_seq_init(&seqstat, LockMethodProcLockHash);
+		dlist_foreach(iter, &lock->procLocks)
+		{
+			PGPROC	   *proc;
+			LockInstanceData *instance;
 
-	while ((proclock = (PROCLOCK *) hash_seq_search(&seqstat)))
-	{
-		PGPROC	   *proc = proclock->tag.myProc;
-		LOCK	   *lock = proclock->tag.myLock;
-		LockInstanceData *instance = &data->locks[el];
+			proclock = dlist_container(PROCLOCK, lockLink, iter.cur);
+			proc = proclock->tag.myProc;
 
-		memcpy(&instance->locktag, &lock->tag, sizeof(LOCKTAG));
-		instance->holdMask = proclock->holdMask;
-		if (proc->waitLock == proclock->tag.myLock)
-			instance->waitLockMode = proc->waitLockMode;
-		else
-			instance->waitLockMode = NoLock;
-		instance->vxid.procNumber = proc->vxid.procNumber;
-		instance->vxid.localTransactionId = proc->vxid.lxid;
-		instance->pid = proc->pid;
-		instance->leaderPid = proclock->groupLeader->pid;
-		instance->fastpath = false;
-		instance->waitStart = (TimestampTz) pg_atomic_read_u64(&proc->waitStart);
+			/* Enlarge array if needed */
+			if (el >= els)
+			{
+				els += MaxBackends;
+				data->locks = (LockInstanceData *)
+					repalloc(data->locks, sizeof(LockInstanceData) * els);
+			}
+
+			instance = &data->locks[el];
+			memcpy(&instance->locktag, &lock->tag, sizeof(LOCKTAG));
+			instance->holdMask = proclock->holdMask;
+			if (proc->waitLock == proclock->tag.myLock)
+				instance->waitLockMode = proc->waitLockMode;
+			else
+				instance->waitLockMode = NoLock;
+			instance->vxid.procNumber = proc->vxid.procNumber;
+			instance->vxid.localTransactionId = proc->vxid.lxid;
+			instance->pid = proc->pid;
+			instance->leaderPid = proclock->groupLeader->pid;
+			instance->fastpath = false;
+			instance->waitStart = (TimestampTz) pg_atomic_read_u64(&proc->waitStart);
 
-		el++;
+			el++;
+		}
 	}
 
+	data->nelements = el;
+
 	/*
 	 * And release locks.  We do this in reverse order for two reasons: (1)
 	 * Anyone else who needs more than one of the locks will be trying to lock
@@ -4171,11 +4174,11 @@ xl_standby_lock *
 GetRunningTransactionLocks(int *nlocks)
 {
 	xl_standby_lock *accessExclusiveLocks;
+	LOCK	   *lock;
 	PROCLOCK   *proclock;
 	HASH_SEQ_STATUS seqstat;
 	int			i;
 	int			index;
-	int			els;
 
 	/*
 	 * Acquire lock on the entire shared lock data structure.
@@ -4185,17 +4188,18 @@ GetRunningTransactionLocks(int *nlocks)
 	for (i = 0; i < NUM_LOCK_PARTITIONS; i++)
 		LWLockAcquire(LockHashPartitionLockByIndex(i), LW_SHARED);
 
-	/* Now we can safely count the number of proclocks */
-	els = hash_get_num_entries(LockMethodProcLockHash);
-
 	/*
-	 * Allocating enough space for all locks in the lock table is overkill,
-	 * but it's more convenient and faster than having to enlarge the array.
+	 * Allocating enough space for MaxProcLocks is overkill, but it's more
+	 * convenient and faster than having to enlarge the array.
 	 */
-	accessExclusiveLocks = palloc(els * sizeof(xl_standby_lock));
+	accessExclusiveLocks = palloc(MaxProcLocks * sizeof(xl_standby_lock));
 
-	/* Now scan the tables to copy the data */
-	hash_seq_init(&seqstat, LockMethodProcLockHash);
+	/*
+	 * Scan all locks to find AccessExclusiveLocks on relations.
+	 * We iterate through the LOCK hash table and for each lock,
+	 * iterate through its procLocks list.
+	 */
+	hash_seq_init(&seqstat, LockMethodLockHash);
 
 	/*
 	 * If lock is a currently granted AccessExclusiveLock then it will have
@@ -4205,35 +4209,43 @@ GetRunningTransactionLocks(int *nlocks)
 	 * non-exclusive lock types.
 	 */
 	index = 0;
-	while ((proclock = (PROCLOCK *) hash_seq_search(&seqstat)))
+	while ((lock = (LOCK *) hash_seq_search(&seqstat)))
 	{
-		/* make sure this definition matches the one used in LockAcquire */
-		if ((proclock->holdMask & LOCKBIT_ON(AccessExclusiveLock)) &&
-			proclock->tag.myLock->tag.locktag_type == LOCKTAG_RELATION)
+		dlist_iter	iter;
+
+		dlist_foreach(iter, &lock->procLocks)
 		{
-			PGPROC	   *proc = proclock->tag.myProc;
-			LOCK	   *lock = proclock->tag.myLock;
-			TransactionId xid = proc->xid;
+			PGPROC	   *proc;
 
-			/*
-			 * Don't record locks for transactions if we know they have
-			 * already issued their WAL record for commit but not yet released
-			 * lock. It is still possible that we see locks held by already
-			 * complete transactions, if they haven't yet zeroed their xids.
-			 */
-			if (!TransactionIdIsValid(xid))
-				continue;
+			proclock = dlist_container(PROCLOCK, lockLink, iter.cur);
+
+			/* make sure this definition matches the one used in LockAcquire */
+			if ((proclock->holdMask & LOCKBIT_ON(AccessExclusiveLock)) &&
+				lock->tag.locktag_type == LOCKTAG_RELATION)
+			{
+				TransactionId xid;
+
+				proc = proclock->tag.myProc;
+				xid = proc->xid;
+
+				/*
+				 * Don't record locks for transactions if we know they have
+				 * already issued their WAL record for commit but not yet released
+				 * lock. It is still possible that we see locks held by already
+				 * complete transactions, if they haven't yet zeroed their xids.
+				 */
+				if (!TransactionIdIsValid(xid))
+					continue;
 
-			accessExclusiveLocks[index].xid = xid;
-			accessExclusiveLocks[index].dbOid = lock->tag.locktag_field1;
-			accessExclusiveLocks[index].relOid = lock->tag.locktag_field2;
+				accessExclusiveLocks[index].xid = xid;
+				accessExclusiveLocks[index].dbOid = lock->tag.locktag_field1;
+				accessExclusiveLocks[index].relOid = lock->tag.locktag_field2;
 
-			index++;
+				index++;
+			}
 		}
 	}
 
-	Assert(index <= els);
-
 	/*
 	 * And release locks.  We do this in reverse order for two reasons: (1)
 	 * Anyone else who needs more than one of the locks will be trying to lock
@@ -4309,17 +4321,19 @@ DumpAllLocks(void)
 	if (proc && proc->waitLock)
 		LOCK_PRINT("DumpAllLocks: waiting on", proc->waitLock, 0);
 
-	hash_seq_init(&status, LockMethodProcLockHash);
+	/* Iterate through all locks and their proclocks */
+	hash_seq_init(&status, LockMethodLockHash);
 
-	while ((proclock = (PROCLOCK *) hash_seq_search(&status)) != NULL)
+	while ((lock = (LOCK *) hash_seq_search(&status)) != NULL)
 	{
-		PROCLOCK_PRINT("DumpAllLocks", proclock);
+		dlist_iter	iter;
 
-		lock = proclock->tag.myLock;
-		if (lock)
+		dlist_foreach(iter, &lock->procLocks)
+		{
+			proclock = dlist_container(PROCLOCK, lockLink, iter.cur);
+			PROCLOCK_PRINT("DumpAllLocks", proclock);
 			LOCK_PRINT("DumpAllLocks", lock, 0);
-		else
-			elog(LOG, "DumpAllLocks: proclock->tag.myLock = NULL");
+		}
 	}
 }
 #endif							/* LOCK_DEBUG */
@@ -4364,10 +4378,8 @@ lock_twophase_recover(FullTransactionId fxid, uint16 info,
 	LOCKMETHODID lockmethodid;
 	LOCK	   *lock;
 	PROCLOCK   *proclock;
-	PROCLOCKTAG proclocktag;
 	bool		found;
 	uint32		hashcode;
-	uint32		proclock_hashcode;
 	int			partition;
 	LWLock	   *partitionLock;
 	LockMethod	lockMethodTable;
@@ -4428,52 +4440,61 @@ lock_twophase_recover(FullTransactionId fxid, uint16 info,
 	}
 
 	/*
-	 * Create the hash key for the proclock table.
-	 */
-	proclocktag.myLock = lock;
-	proclocktag.myProc = proc;
-
-	proclock_hashcode = ProcLockHashCode(&proclocktag, hashcode);
-
-	/*
-	 * Find or create a proclock entry with this tag
+	 * Find or create a proclock entry for this lock and proc.
+	 * Search the lock's procLocks list to see if we already have one.
 	 */
-	proclock = (PROCLOCK *) hash_search_with_hash_value(LockMethodProcLockHash,
-														&proclocktag,
-														proclock_hashcode,
-														HASH_ENTER_NULL,
-														&found);
-	if (!proclock)
+	proclock = NULL;
+	found = false;
 	{
-		/* Oops, not enough shmem for the proclock */
-		if (lock->nRequested == 0)
+		dlist_iter	iter;
+
+		dlist_foreach(iter, &lock->procLocks)
 		{
-			/*
-			 * There are no other requestors of this lock, so garbage-collect
-			 * the lock object.  We *must* do this to avoid a permanent leak
-			 * of shared memory, because there won't be anything to cause
-			 * anyone to release the lock object later.
-			 */
-			Assert(dlist_is_empty(&lock->procLocks));
-			if (!hash_search_with_hash_value(LockMethodLockHash,
-											 &(lock->tag),
-											 hashcode,
-											 HASH_REMOVE,
-											 NULL))
-				elog(PANIC, "lock table corrupted");
+			PROCLOCK   *existing;
+
+			existing = dlist_container(PROCLOCK, lockLink, iter.cur);
+			if (existing->tag.myProc == proc)
+			{
+				proclock = existing;
+				found = true;
+				break;
+			}
 		}
-		LWLockRelease(partitionLock);
-		ereport(ERROR,
-				(errcode(ERRCODE_OUT_OF_MEMORY),
-				 errmsg("out of shared memory"),
-				 errhint("You might need to increase \"%s\".", "max_locks_per_transaction")));
 	}
 
-	/*
-	 * If new, initialize the new entry
-	 */
 	if (!found)
 	{
+		/* Allocate a new PROCLOCK from the free list */
+		proclock = ProcLockAlloc(partition);
+		if (!proclock)
+		{
+			/* Oops, not enough shmem for the proclock */
+			if (lock->nRequested == 0)
+			{
+				/*
+				 * There are no other requestors of this lock, so garbage-collect
+				 * the lock object.  We *must* do this to avoid a permanent leak
+				 * of shared memory, because there won't be anything to cause
+				 * anyone to release the lock object later.
+				 */
+				Assert(dlist_is_empty(&lock->procLocks));
+				if (!hash_search_with_hash_value(LockMethodLockHash,
+												 &(lock->tag),
+												 hashcode,
+												 HASH_REMOVE,
+												 NULL))
+					elog(PANIC, "lock table corrupted");
+			}
+			LWLockRelease(partitionLock);
+			ereport(ERROR,
+					(errcode(ERRCODE_OUT_OF_MEMORY),
+					 errmsg("out of shared memory"),
+					 errhint("You might need to increase \"%s\".", "max_locks_per_transaction")));
+		}
+
+		/* Initialize the new entry */
+		proclock->tag.myLock = lock;
+		proclock->tag.myProc = proc;
 		Assert(proc->lockGroupLeader == NULL);
 		proclock->groupLeader = proc;
 		proclock->holdMask = 0;
diff --git a/src/include/storage/lock.h b/src/include/storage/lock.h
index 826cf28fdbd..6d22bb91233 100644
--- a/src/include/storage/lock.h
+++ b/src/include/storage/lock.h
@@ -333,11 +333,10 @@ typedef struct LOCK
  * information for each such holder (or would-be holder).  This is kept in
  * a PROCLOCK struct.
  *
- * PROCLOCKTAG is the key information needed to look up a PROCLOCK item in the
- * proclock hashtable.  A PROCLOCKTAG value uniquely identifies the combination
- * of a lockable object and a holder/waiter for that object.  (We can use
- * pointers here because the PROCLOCKTAG need only be unique for the lifespan
- * of the PROCLOCK, and it will never outlive the lock or the proc.)
+ * PROCLOCKTAG uniquely identifies the combination of a lockable object
+ * and a holder/waiter for that object.  (We can use pointers here because
+ * the PROCLOCKTAG need only be unique for the lifespan of the PROCLOCK,
+ * and it will never outlive the lock or the proc.)
  *
  * Internally to a backend, it is possible for the same lock to be held
  * for different purposes: the backend tracks transaction locks separately
-- 
2.51.2

#2Heikki Linnakangas
hlinnaka@iki.fi
In reply to: Andrey Borodin (#1)
Re: Refactor PROCLOCK hash table into partitioned list allocator

On 30/12/2025 14:37, Andrey Borodin wrote:

Hi hackers,

Following up on the Discord discussion about the PROCLOCK hash table being
a "weird allocator" that we never actually use for lookups - I took a stab at
replacing it with a simpler partitioned free list approach as was suggested.
I was doing this mostly to educate myself on Lock Manager internals.

The current implementation uses LockMethodProcLockHash purely as an allocator.
We never do hash lookups by key; we only allocate entries, link them to the lock's
procLocks list, and free them later. Using a full hash table for this adds
unnecessary complexity and maybe even overhead (I did not measure this).

The attached patch replaces this with:
- ProcLockArray: A fixed-size array of all PROCLOCK structs (allocated at startup)
- ProcLockFreeList: Partitioned free lists, one per lock partition to reduce contention
- ProcLockAlloc/Free: Simple push/pop operations on the free lists
- PROCLOCK lookup: Linear traversal of lock->procLocks (see LockRefindAndRelease()
and FastPathGetRelationLockEntry())

The last point bothers me most. It seems like this traversals are expected to be short.
But I'm not 100% sure.

Hmm, yeah the last point contradicts the premise that the hash table is
used purely as an allocator. It *is* used for lookups, and you're
replacing them with linear scans. That doesn't seem like an improvement.

- Heikki

#3Matthias van de Meent
boekewurm+postgres@gmail.com
In reply to: Heikki Linnakangas (#2)
Re: Refactor PROCLOCK hash table into partitioned list allocator

On Tue, 6 Jan 2026 at 15:24, Heikki Linnakangas <hlinnaka@iki.fi> wrote:

On 30/12/2025 14:37, Andrey Borodin wrote:

Hi hackers,

Following up on the Discord discussion about the PROCLOCK hash table being
a "weird allocator" that we never actually use for lookups - I took a stab at
replacing it with a simpler partitioned free list approach as was suggested.
I was doing this mostly to educate myself on Lock Manager internals.

The current implementation uses LockMethodProcLockHash purely as an allocator.
We never do hash lookups by key; we only allocate entries, link them to the lock's
procLocks list, and free them later. Using a full hash table for this adds
unnecessary complexity and maybe even overhead (I did not measure this).

The attached patch replaces this with:
- ProcLockArray: A fixed-size array of all PROCLOCK structs (allocated at startup)
- ProcLockFreeList: Partitioned free lists, one per lock partition to reduce contention
- ProcLockAlloc/Free: Simple push/pop operations on the free lists
- PROCLOCK lookup: Linear traversal of lock->procLocks (see LockRefindAndRelease()
and FastPathGetRelationLockEntry())

The last point bothers me most. It seems like this traversals are expected to be short.
But I'm not 100% sure.

Hmm, yeah the last point contradicts the premise that the hash table is
used purely as an allocator. It *is* used for lookups, and you're
replacing them with linear scans. That doesn't seem like an improvement.

There are 2 types of PROCLOCK lookup used in LockRefindAndRelease and
FastPathGetRelationLockEntry:
- An active backend's PROCLOCK entries (in both LRAR and FPGRLE).
- Prepared transaction's PROCLOCK entries (only in LRAR, called from
lock_twophase_postcommit).

For the backend's PROCLOCK entry lookup, we can use a backend-local
hash table, which only keeps track of where this backend's entries
are.

For prepared transactions, I don't see any code that would indicate
more than one lock being released through this code
(lock_twophase_postcommit only releases one lock), which to me
indicates there is no risk of O(N^2)-related performance sinks. In the
case that there are more locks in the 2PC's PROCLOCK list, we could
"just" make sure to put the lock we're releasing in transaction wind
down at the head of the list; as that would also keep the lookup O(1).

Kind regards,

Matthias van de Meent
Databricks (https://www.databricks.com)

#4Japin Li
japinli@hotmail.com
In reply to: Heikki Linnakangas (#2)
Re: Refactor PROCLOCK hash table into partitioned list allocator

On Tue, 06 Jan 2026 at 16:23, Heikki Linnakangas <hlinnaka@iki.fi> wrote:

On 30/12/2025 14:37, Andrey Borodin wrote:

Hi hackers,
Following up on the Discord discussion about the PROCLOCK hash table
being
a "weird allocator" that we never actually use for lookups - I took a stab at
replacing it with a simpler partitioned free list approach as was suggested.
I was doing this mostly to educate myself on Lock Manager internals.
The current implementation uses LockMethodProcLockHash purely as an
allocator.
We never do hash lookups by key; we only allocate entries, link them to the lock's
procLocks list, and free them later. Using a full hash table for this adds
unnecessary complexity and maybe even overhead (I did not measure this).
The attached patch replaces this with:
- ProcLockArray: A fixed-size array of all PROCLOCK structs (allocated at startup)
- ProcLockFreeList: Partitioned free lists, one per lock partition to reduce contention
- ProcLockAlloc/Free: Simple push/pop operations on the free lists
- PROCLOCK lookup: Linear traversal of lock->procLocks (see LockRefindAndRelease()
and FastPathGetRelationLockEntry())
The last point bothers me most. It seems like this traversals are
expected to be short.
But I'm not 100% sure.

Hmm, yeah the last point contradicts the premise that the hash table
is used purely as an allocator. It *is* used for lookups, and you're
replacing them with linear scans. That doesn't seem like an
improvement.

- Heikki

I tested the patch on a Loongson 3C6000/D system with 128 vCPUs using
BenchmarkSQL 5.0 (100 warehouses, 100 clients).

Here are the results:

| | tpmC | tpmTotal |
|---------|-----------|-----------|
| master | 248199.09 | 551387.46 |
| | 243660.35 | 541902.31 |
| | 244418.30 | 542867.57 |
| patched | 247330.65 | 549949.25 |
| | 242953.79 | 539620.65 |
| | 237883.19 | 528491.66 |

Not sure if this is useful, but throwing it out there.

--
Regards,
Japin Li
ChengDu WenWu Information Technology Co., Ltd.

#5Heikki Linnakangas
hlinnaka@iki.fi
In reply to: Matthias van de Meent (#3)
Re: Refactor PROCLOCK hash table into partitioned list allocator

On 06/01/2026 16:58, Matthias van de Meent wrote:

On Tue, 6 Jan 2026 at 15:24, Heikki Linnakangas <hlinnaka@iki.fi> wrote:

On 30/12/2025 14:37, Andrey Borodin wrote:

Hi hackers,

Following up on the Discord discussion about the PROCLOCK hash table being
a "weird allocator" that we never actually use for lookups - I took a stab at
replacing it with a simpler partitioned free list approach as was suggested.
I was doing this mostly to educate myself on Lock Manager internals.

The current implementation uses LockMethodProcLockHash purely as an allocator.
We never do hash lookups by key; we only allocate entries, link them to the lock's
procLocks list, and free them later. Using a full hash table for this adds
unnecessary complexity and maybe even overhead (I did not measure this).

The attached patch replaces this with:
- ProcLockArray: A fixed-size array of all PROCLOCK structs (allocated at startup)
- ProcLockFreeList: Partitioned free lists, one per lock partition to reduce contention
- ProcLockAlloc/Free: Simple push/pop operations on the free lists
- PROCLOCK lookup: Linear traversal of lock->procLocks (see LockRefindAndRelease()
and FastPathGetRelationLockEntry())

The last point bothers me most. It seems like this traversals are expected to be short.
But I'm not 100% sure.

Hmm, yeah the last point contradicts the premise that the hash table is
used purely as an allocator. It *is* used for lookups, and you're
replacing them with linear scans. That doesn't seem like an improvement.

There are 2 types of PROCLOCK lookup used in LockRefindAndRelease and
FastPathGetRelationLockEntry:
- An active backend's PROCLOCK entries (in both LRAR and FPGRLE).
- Prepared transaction's PROCLOCK entries (only in LRAR, called from
lock_twophase_postcommit).

There are also lookups in SetupLockInTable and in LockRelease.

For the backend's PROCLOCK entry lookup, we can use a backend-local
hash table, which only keeps track of where this backend's entries
are.

Hmm, good point. In fact we already have that: there's a pointer to the
current process's PROCLOCK entry in LOCALLOCK already. Can we use that
to avoid the linear scans? There's this LockAcquireExtended:

/*
* Find or create lock and proclock entries with this tag
*
* Note: if the locallock object already existed, it might have a pointer
* to the lock already ... but we should not assume that that pointer is
* valid, since a lock object with zero hold and request counts can go
* away anytime. So we have to use SetupLockInTable() to recompute the
* lock and proclock pointers, even if they're already set.
*/
proclock = SetupLockInTable(lockMethodTable, MyProc, locktag,
hashcode, lockmode);

So that comment suggests that the 'proclock' pointer cannot be trusted
here. I don't remember how all this works, so I'm not sure if that is a
show-stopper or if we could somehow leverage the 'proclock' pointer in
LOCALLOCK anyway.

- Heikki