diff --git a/src/backend/access/transam/clog.c b/src/backend/access/transam/clog.c
index 3a58f1e..10c25cf 100644
--- a/src/backend/access/transam/clog.c
+++ b/src/backend/access/transam/clog.c
@@ -457,7 +457,8 @@ CLOGShmemInit(void)
 {
 	ClogCtl->PagePrecedes = CLOGPagePrecedes;
 	SimpleLruInit(ClogCtl, "CLOG Ctl", CLOGShmemBuffers(), CLOG_LSNS_PER_PAGE,
-				  CLogControlLock, "pg_clog");
+				  CLogControlLock, "pg_clog",
+				  "CLogBufferLocks");
 }
 
 /*
diff --git a/src/backend/access/transam/commit_ts.c b/src/backend/access/transam/commit_ts.c
index 33136e3..00f92e4 100644
--- a/src/backend/access/transam/commit_ts.c
+++ b/src/backend/access/transam/commit_ts.c
@@ -465,7 +465,8 @@ CommitTsShmemInit(void)
 
 	CommitTsCtl->PagePrecedes = CommitTsPagePrecedes;
 	SimpleLruInit(CommitTsCtl, "CommitTs Ctl", CommitTsShmemBuffers(), 0,
-				  CommitTsControlLock, "pg_commit_ts");
+				  CommitTsControlLock, "pg_commit_ts",
+				  "CommitTSBufferLocks");
 
 	commitTsShared = ShmemInitStruct("CommitTs shared",
 									 sizeof(CommitTimestampShared),
diff --git a/src/backend/access/transam/multixact.c b/src/backend/access/transam/multixact.c
index 1933a87..b905c59 100644
--- a/src/backend/access/transam/multixact.c
+++ b/src/backend/access/transam/multixact.c
@@ -1842,10 +1842,12 @@ MultiXactShmemInit(void)
 
 	SimpleLruInit(MultiXactOffsetCtl,
 				  "MultiXactOffset Ctl", NUM_MXACTOFFSET_BUFFERS, 0,
-				  MultiXactOffsetControlLock, "pg_multixact/offsets");
+				  MultiXactOffsetControlLock, "pg_multixact/offsets",
+				  "MultiXactOffsetBufferLocks");
 	SimpleLruInit(MultiXactMemberCtl,
 				  "MultiXactMember Ctl", NUM_MXACTMEMBER_BUFFERS, 0,
-				  MultiXactMemberControlLock, "pg_multixact/members");
+				  MultiXactMemberControlLock, "pg_multixact/members",
+				  "MultiXactMemberBufferLocks");
 
 	/* Initialize our shared state struct */
 	MultiXactState = ShmemInitStruct("Shared MultiXact State",
diff --git a/src/backend/access/transam/slru.c b/src/backend/access/transam/slru.c
index 5fcea11..4bf3fee 100644
--- a/src/backend/access/transam/slru.c
+++ b/src/backend/access/transam/slru.c
@@ -156,15 +156,20 @@ SimpleLruShmemSize(int nslots, int nlsns)
 	if (nlsns > 0)
 		sz += MAXALIGN(nslots * nlsns * sizeof(XLogRecPtr));	/* group_lsn[] */
 
+	/* size of lwlocks */
+	sz = add_size(sz, LWLockTrancheShmemSize(nslots));
+
 	return BUFFERALIGN(sz) + BLCKSZ * nslots;
 }
 
 void
 SimpleLruInit(SlruCtl ctl, const char *name, int nslots, int nlsns,
-			  LWLock *ctllock, const char *subdir)
+			  LWLock *ctllock, const char *subdir,
+			  const char *lwlocks_tranche)
 {
-	SlruShared	shared;
-	bool		found;
+	SlruShared    shared;
+	bool          found;
+	LWLockPadded *lwlock_array;
 
 	shared = (SlruShared) ShmemInitStruct(name,
 										  SimpleLruShmemSize(nslots, nlsns),
@@ -212,13 +217,18 @@ SimpleLruInit(SlruCtl ctl, const char *name, int nslots, int nlsns,
 		}
 
 		ptr += BUFFERALIGN(offset);
+
+		/* Create tranche and lwlocks required for slots */
+		LWLockCreateTranche(lwlocks_tranche, nslots, &lwlock_array);
+
+		/* Initialize slots */
 		for (slotno = 0; slotno < nslots; slotno++)
 		{
 			shared->page_buffer[slotno] = ptr;
 			shared->page_status[slotno] = SLRU_PAGE_EMPTY;
 			shared->page_dirty[slotno] = false;
 			shared->page_lru_count[slotno] = 0;
-			shared->buffer_locks[slotno] = LWLockAssign();
+			shared->buffer_locks[slotno] = &lwlock_array[slotno].lock;
 			ptr += BLCKSZ;
 		}
 	}
diff --git a/src/backend/access/transam/subtrans.c b/src/backend/access/transam/subtrans.c
index 6b70982..eed2fd9 100644
--- a/src/backend/access/transam/subtrans.c
+++ b/src/backend/access/transam/subtrans.c
@@ -179,7 +179,8 @@ SUBTRANSShmemInit(void)
 {
 	SubTransCtl->PagePrecedes = SubTransPagePrecedes;
 	SimpleLruInit(SubTransCtl, "SUBTRANS Ctl", NUM_SUBTRANS_BUFFERS, 0,
-				  SubtransControlLock, "pg_subtrans");
+				  SubtransControlLock, "pg_subtrans",
+				  "SubtransBufferLocks");
 	/* Override default assumption that writes should be fsync'd */
 	SubTransCtl->do_fsync = false;
 }
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 127bc58..8716b25 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -4653,8 +4653,6 @@ XLOGShmemInit(void)
 
 		/* Initialize local copy of WALInsertLocks and register the tranche */
 		WALInsertLocks = XLogCtl->Insert.WALInsertLocks;
-		LWLockRegisterTranche(XLogCtl->Insert.WALInsertLockTrancheId,
-							  &XLogCtl->Insert.WALInsertLockTranche);
 		return;
 	}
 	memset(XLogCtl, 0, sizeof(XLogCtlData));
@@ -4679,11 +4677,12 @@ XLOGShmemInit(void)
 
 	XLogCtl->Insert.WALInsertLockTrancheId = LWLockNewTrancheId();
 
-	XLogCtl->Insert.WALInsertLockTranche.name = "WALInsertLocks";
+	strcpy(XLogCtl->Insert.WALInsertLockTranche.name, "WALInsertLocks");
 	XLogCtl->Insert.WALInsertLockTranche.array_base = WALInsertLocks;
 	XLogCtl->Insert.WALInsertLockTranche.array_stride = sizeof(WALInsertLockPadded);
 
-	LWLockRegisterTranche(XLogCtl->Insert.WALInsertLockTrancheId, &XLogCtl->Insert.WALInsertLockTranche);
+	LWLockRegisterTranche(XLogCtl->Insert.WALInsertLockTrancheId,
+		&XLogCtl->Insert.WALInsertLockTranche);
 	for (i = 0; i < NUM_XLOGINSERT_LOCKS; i++)
 	{
 		LWLockInitialize(&WALInsertLocks[i].l.lock,
diff --git a/src/backend/commands/async.c b/src/backend/commands/async.c
index 3b71174..7b5f00e 100644
--- a/src/backend/commands/async.c
+++ b/src/backend/commands/async.c
@@ -469,7 +469,7 @@ AsyncShmemInit(void)
 	 */
 	AsyncCtl->PagePrecedes = asyncQueuePagePrecedes;
 	SimpleLruInit(AsyncCtl, "Async Ctl", NUM_ASYNC_BUFFERS, 0,
-				  AsyncCtlLock, "pg_notify");
+				  AsyncCtlLock, "pg_notify", "AsyncBufferLocks");
 	/* Override default assumption that writes should be fsync'd */
 	AsyncCtl->do_fsync = false;
 
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index 1818f7c..c7a7512 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -481,7 +481,8 @@ typedef struct
 #ifndef HAVE_SPINLOCKS
 	PGSemaphore SpinlockSemaArray;
 #endif
-	LWLockPadded *MainLWLockArray;
+	LWLockTranche **LWLockTrancheArray;
+	LWLockPadded   *MainLWLockArray;
 	slock_t    *ProcStructLock;
 	PROC_HDR   *ProcGlobal;
 	PGPROC	   *AuxiliaryProcs;
@@ -5715,6 +5716,7 @@ save_backend_variables(BackendParameters *param, Port *port,
 #ifndef HAVE_SPINLOCKS
 	param->SpinlockSemaArray = SpinlockSemaArray;
 #endif
+	param->LWLockTrancheArray = LWLockTrancheArray;
 	param->MainLWLockArray = MainLWLockArray;
 	param->ProcStructLock = ProcStructLock;
 	param->ProcGlobal = ProcGlobal;
@@ -5946,6 +5948,7 @@ restore_backend_variables(BackendParameters *param, Port *port)
 #ifndef HAVE_SPINLOCKS
 	SpinlockSemaArray = param->SpinlockSemaArray;
 #endif
+	LWLockTrancheArray = param->LWLockTrancheArray;
 	MainLWLockArray = param->MainLWLockArray;
 	ProcStructLock = param->ProcStructLock;
 	ProcGlobal = param->ProcGlobal;
diff --git a/src/backend/replication/logical/origin.c b/src/backend/replication/logical/origin.c
index c219590..3f41813 100644
--- a/src/backend/replication/logical/origin.c
+++ b/src/backend/replication/logical/origin.c
@@ -474,7 +474,7 @@ ReplicationOriginShmemInit(void)
 		int			i;
 
 		replication_states_ctl->tranche_id = LWLockNewTrancheId();
-		replication_states_ctl->tranche.name = "ReplicationOrigins";
+		strcpy(replication_states_ctl->tranche.name, "ReplicationOrigins");
 		replication_states_ctl->tranche.array_base =
 			&replication_states[0].lock;
 		replication_states_ctl->tranche.array_stride =
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index c66619c..6eb8f97 100644
--- a/src/backend/replication/slot.c
+++ b/src/backend/replication/slot.c
@@ -119,6 +119,9 @@ ReplicationSlotsShmemSize(void)
 	size = add_size(size,
 					mul_size(max_replication_slots, sizeof(ReplicationSlot)));
 
+	/* size of lwlocks */
+	size = add_size(size, LWLockTrancheShmemSize(max_replication_slots));
+
 	return size;
 }
 
@@ -128,7 +131,8 @@ ReplicationSlotsShmemSize(void)
 void
 ReplicationSlotsShmemInit(void)
 {
-	bool		found;
+	bool          found;
+	LWLockPadded *lwlocks_array;
 
 	if (max_replication_slots == 0)
 		return;
@@ -144,13 +148,17 @@ ReplicationSlotsShmemInit(void)
 		/* First time through, so initialize */
 		MemSet(ReplicationSlotCtl, 0, ReplicationSlotsShmemSize());
 
+		/* Create lwlocks */
+		LWLockCreateTranche("ReplicationSlotLocks", max_replication_slots,
+			&lwlocks_array);
+
 		for (i = 0; i < max_replication_slots; i++)
 		{
 			ReplicationSlot *slot = &ReplicationSlotCtl->replication_slots[i];
 
 			/* everything else is zeroed by the memset above */
 			SpinLockInit(&slot->mutex);
-			slot->io_in_progress_lock = LWLockAssign();
+			slot->io_in_progress_lock = &lwlocks_array[i].lock;
 		}
 	}
 }
diff --git a/src/backend/storage/buffer/buf_init.c b/src/backend/storage/buffer/buf_init.c
index 3ae2848..40cf33f 100644
--- a/src/backend/storage/buffer/buf_init.c
+++ b/src/backend/storage/buffer/buf_init.c
@@ -18,8 +18,9 @@
 #include "storage/buf_internals.h"
 
 
-BufferDescPadded *BufferDescriptors;
-char	   *BufferBlocks;
+BufferDescPadded  *BufferDescriptors;
+char              *BufferBlocks;
+LWLockPadded      *BufferLWLockArray;
 
 
 /*
@@ -64,8 +65,9 @@ char	   *BufferBlocks;
 void
 InitBufferPool(void)
 {
-	bool		foundBufs,
-				foundDescs;
+	bool          foundBufs;
+	bool          foundDescs;
+	LWLockPadded *lwlocks_array;
 
 	/* Align descriptors to a cacheline boundary. */
 	BufferDescriptors = (BufferDescPadded *) CACHELINEALIGN(
@@ -77,6 +79,10 @@ InitBufferPool(void)
 		ShmemInitStruct("Buffer Blocks",
 						NBuffers * (Size) BLCKSZ, &foundBufs);
 
+	/* Init LWLocks for buffer headers */
+	LWLockCreateTranche("BufferMgrLocks", 2 * NBuffers,
+		&lwlocks_array);
+
 	if (foundDescs || foundBufs)
 	{
 		/* both should be present or neither */
@@ -110,14 +116,18 @@ InitBufferPool(void)
 			 */
 			buf->freeNext = i + 1;
 
-			buf->io_in_progress_lock = LWLockAssign();
-			buf->content_lock = LWLockAssign();
+			buf->io_in_progress_lock = &lwlocks_array[i * 2].lock;
+			buf->content_lock = &lwlocks_array[i * 2 + 1].lock;
 		}
 
 		/* Correct last entry of linked list */
 		GetBufferDescriptor(NBuffers - 1)->freeNext = FREENEXT_END_OF_LIST;
 	}
 
+	/* Init bufmgr LWLocks */
+	LWLockCreateTranche("BufferLWLocks", NUM_BUFFER_PARTITIONS,
+		&BufferLWLockArray);
+
 	/* Init other shared buffer-management stuff */
 	StrategyInitialize(!foundDescs);
 }
@@ -144,5 +154,9 @@ BufferShmemSize(void)
 	/* size of stuff controlled by freelist.c */
 	size = add_size(size, StrategyShmemSize());
 
+	/* size of LWLock structures required for buffers */
+	size = add_size(size, LWLockTrancheShmemSize(NUM_BUFFER_PARTITIONS));
+	size = add_size(size, LWLockTrancheShmemSize(2 * NBuffers));
+
 	return size;
 }
diff --git a/src/backend/storage/lmgr/lock.c b/src/backend/storage/lmgr/lock.c
index 1eb2d4b..80c7955 100644
--- a/src/backend/storage/lmgr/lock.c
+++ b/src/backend/storage/lmgr/lock.c
@@ -48,13 +48,15 @@
 #include "utils/resowner_private.h"
 
 
+/* LWLocks tranche and array */
+LWLockPadded		    *LockMgrLWLockArray;
+
 /* This configuration variable is used to set the lock table size */
 int			max_locks_per_xact; /* set by guc.c */
 
 #define NLOCKENTS() \
 	mul_size(max_locks_per_xact, add_size(MaxBackends, max_prepared_xacts))
 
-
 /*
  * Data structures defining the semantics of the standard lock methods.
  *
@@ -446,6 +448,10 @@ InitLocks(void)
 									  16,
 									  &info,
 									  HASH_ELEM | HASH_BLOBS);
+
+	/* Init LWLocks tranche and array */
+	LWLockCreateTranche("LockMgrLWLocks", NUM_LOCK_PARTITIONS,
+		&LockMgrLWLockArray);
 }
 
 
@@ -3282,6 +3288,9 @@ LockShmemSize(void)
 	 */
 	size = add_size(size, size / 10);
 
+	/* Lock Manager LWLock structures */
+	size = add_size(size, LWLockTrancheShmemSize(NUM_LOCK_PARTITIONS));
+
 	return size;
 }
 
diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index 687ed63..1c172cd 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -110,11 +110,9 @@ extern slock_t *ShmemLock;
 #define LW_SHARED_MASK				((uint32)(1 << 23))
 
 /*
- * This is indexed by tranche ID and stores metadata for all tranches known
- * to the current backend.
+ * This is indexed by tranche ID and stores metadata for all tranches
  */
-static LWLockTranche **LWLockTrancheArray = NULL;
-static int	LWLockTranchesAllocated = 0;
+LWLockTranche **LWLockTrancheArray = NULL;
 
 #define T_NAME(lock) \
 	(LWLockTrancheArray[(lock)->tranche]->name)
@@ -129,7 +127,9 @@ static int	LWLockTranchesAllocated = 0;
  * where we have special measures to pass it down).
  */
 LWLockPadded *MainLWLockArray = NULL;
-static LWLockTranche MainLWLockTranche;
+
+/* Points to the array of user defined LWLocks in shared memory */
+static LWLockPadded *userdef_lwlocks_array = NULL;
 
 /*
  * We use this structure to keep track of locked LWLocks for release
@@ -316,64 +316,66 @@ get_lwlock_stats_entry(LWLock *lock)
 }
 #endif   /* LWLOCK_STATS */
 
+#define INIT_LWLOCK_NAME(lock) \
+	strcpy(LWLockTrancheArray[T_ID(lock)]->name, #lock);
 
 /*
- * Compute number of LWLocks to allocate in the main array.
+ * Fill name in individual LWLock tranches. Must be called
+ * after tranches creation
  */
-static int
-NumLWLocks(void)
+static void
+InitLWLockNames(void)
 {
-	int			numLocks;
-
-	/*
-	 * Possibly this logic should be spread out among the affected modules,
-	 * the same way that shmem space estimation is done.  But for now, there
-	 * are few enough users of LWLocks that we can get away with just keeping
-	 * the knowledge here.
-	 */
-
-	/* Predefined LWLocks */
-	numLocks = NUM_FIXED_LWLOCKS;
-
-	/* bufmgr.c needs two for each shared buffer */
-	numLocks += 2 * NBuffers;
-
-	/* proc.c needs one for each backend or auxiliary process */
-	numLocks += MaxBackends + NUM_AUXILIARY_PROCS;
-
-	/* clog.c needs one per CLOG buffer */
-	numLocks += CLOGShmemBuffers();
-
-	/* commit_ts.c needs one per CommitTs buffer */
-	numLocks += CommitTsShmemBuffers();
-
-	/* subtrans.c needs one per SubTrans buffer */
-	numLocks += NUM_SUBTRANS_BUFFERS;
-
-	/* multixact.c needs two SLRU areas */
-	numLocks += NUM_MXACTOFFSET_BUFFERS + NUM_MXACTMEMBER_BUFFERS;
-
-	/* async.c needs one per Async buffer */
-	numLocks += NUM_ASYNC_BUFFERS;
-
-	/* predicate.c needs one per old serializable xid buffer */
-	numLocks += NUM_OLDSERXID_BUFFERS;
-
-	/* slot.c needs one for each slot */
-	numLocks += max_replication_slots;
-
-	/*
-	 * Add any requested by loadable modules; for backwards-compatibility
-	 * reasons, allocate at least NUM_USER_DEFINED_LWLOCKS of them even if
-	 * there are no explicit requests.
-	 */
-	lock_addin_request_allowed = false;
-	numLocks += Max(lock_addin_request, NUM_USER_DEFINED_LWLOCKS);
-
-	return numLocks;
+	int i;
+
+	INIT_LWLOCK_NAME(ShmemIndexLock);
+	INIT_LWLOCK_NAME(OidGenLock);
+	INIT_LWLOCK_NAME(XidGenLock);
+	INIT_LWLOCK_NAME(ProcArrayLock);
+	INIT_LWLOCK_NAME(SInvalReadLock);
+	INIT_LWLOCK_NAME(SInvalWriteLock);
+	INIT_LWLOCK_NAME(WALBufMappingLock);
+	INIT_LWLOCK_NAME(WALWriteLock);
+	INIT_LWLOCK_NAME(ControlFileLock);
+	INIT_LWLOCK_NAME(CheckpointLock);
+	INIT_LWLOCK_NAME(CLogControlLock);
+	INIT_LWLOCK_NAME(SubtransControlLock);
+	INIT_LWLOCK_NAME(MultiXactGenLock);
+	INIT_LWLOCK_NAME(MultiXactOffsetControlLock);
+	INIT_LWLOCK_NAME(MultiXactMemberControlLock);
+	INIT_LWLOCK_NAME(RelCacheInitLock);
+	INIT_LWLOCK_NAME(CheckpointerCommLock);
+	INIT_LWLOCK_NAME(TwoPhaseStateLock);
+	INIT_LWLOCK_NAME(TablespaceCreateLock);
+	INIT_LWLOCK_NAME(BtreeVacuumLock);
+	INIT_LWLOCK_NAME(AddinShmemInitLock);
+	INIT_LWLOCK_NAME(AutovacuumLock);
+	INIT_LWLOCK_NAME(AutovacuumScheduleLock);
+	INIT_LWLOCK_NAME(SyncScanLock);
+	INIT_LWLOCK_NAME(RelationMappingLock);
+	INIT_LWLOCK_NAME(AsyncCtlLock);
+	INIT_LWLOCK_NAME(AsyncQueueLock);
+	INIT_LWLOCK_NAME(SerializableXactHashLock);
+	INIT_LWLOCK_NAME(SerializableFinishedListLock);
+	INIT_LWLOCK_NAME(SerializablePredicateLockListLock);
+	INIT_LWLOCK_NAME(OldSerXidLock);
+	INIT_LWLOCK_NAME(SyncRepLock);
+	INIT_LWLOCK_NAME(BackgroundWorkerLock);
+	INIT_LWLOCK_NAME(DynamicSharedMemoryControlLock);
+	INIT_LWLOCK_NAME(AutoFileLock);
+	INIT_LWLOCK_NAME(ReplicationSlotAllocationLock);
+	INIT_LWLOCK_NAME(ReplicationSlotControlLock);
+	INIT_LWLOCK_NAME(CommitTsControlLock);
+	INIT_LWLOCK_NAME(CommitTsLock);
+	INIT_LWLOCK_NAME(ReplicationOriginLock);
+
+	/* Check that every individual LWLock has name */
+	for (i = 1; i < NUM_INDIVIDUAL_LWLOCKS; i++)
+		if (strncmp(LWLOCK_TRANCHE_NAME(i), "", 2) == 0)
+			elog(ERROR, "Individual LWLock at %d index still hasn't a name. "\
+				"It must be registered in InitLWLockNames function", i);
 }
 
-
 /*
  * RequestAddinLWLocks
  *		Request that extra LWLocks be allocated for use by
@@ -393,21 +395,59 @@ RequestAddinLWLocks(int n)
 	lock_addin_request += n;
 }
 
+/*
+ * Return number of user defined LWLocks
+ */
+static int
+NumUserDefinedLWLocks()
+{
+	/*
+	 * Add any requested by loadable modules; for backwards-compatibility
+	 * reasons, allocate at least NUM_USER_DEFINED_LWLOCKS of them even if
+	 * there are no explicit requests.
+	 */
+
+	lock_addin_request_allowed = false;
+	return Max(lock_addin_request, NUM_USER_DEFINED_LWLOCKS);
+}
+
+/*
+ * Compute shmem space for LWLock counters, individual LWLocks array
+ * and tranches
+ */
+static Size
+LWLocksIndividualShmemSize(void)
+{
+	Size		size;
+
+	/* Space for dynamic allocation counter, plus room for alignment. */
+	size = 3 * sizeof(int) + LWLOCK_PADDED_SIZE;
+
+	/* Space for individual LWLock tranches */
+	size = add_size(size,
+		mul_size(sizeof(LWLockTranche), NUM_INDIVIDUAL_LWLOCKS));
+
+	/* Space for individual LWLocks */
+	size = add_size(size,
+		mul_size(sizeof(LWLockPadded), NUM_INDIVIDUAL_LWLOCKS));
+
+	return size;
+}
 
 /*
- * Compute shmem space needed for LWLocks.
+ * Compute shmem space needed by LWLocks initialization
  */
 Size
 LWLockShmemSize(void)
 {
-	Size		size;
-	int			numLocks = NumLWLocks();
+	Size size = LWLocksIndividualShmemSize();
 
-	/* Space for the LWLock array. */
-	size = mul_size(numLocks, sizeof(LWLockPadded));
+	/* Space for tranches array */
+	size = add_size(size,
+		mul_size(sizeof(LWLockTranche **), NUM_LWLOCK_TRANCHES));
 
-	/* Space for dynamic allocation counter, plus room for alignment. */
-	size = add_size(size, 3 * sizeof(int) + LWLOCK_PADDED_SIZE);
+	/* Space for user defined lwlocks */
+	size = add_size(size, LWLockTrancheShmemSize(NumUserDefinedLWLocks()));
 
 	return size;
 }
@@ -415,7 +455,7 @@ LWLockShmemSize(void)
 
 /*
  * Allocate shmem space for the main LWLock array and initialize it.  We also
- * register the main tranch here.
+ * register the tranches for individual LWLocks here.
  */
 void
 CreateLWLocks(void)
@@ -425,12 +465,16 @@ CreateLWLocks(void)
 
 	if (!IsUnderPostmaster)
 	{
-		int			numLocks = NumLWLocks();
-		Size		spaceLocks = LWLockShmemSize();
-		LWLockPadded *lock;
-		int		   *LWLockCounter;
-		char	   *ptr;
-		int			id;
+		int           *LWLockCounter;
+		int            i;
+		char          *ptr;
+		Size           spaceLocks = LWLocksIndividualShmemSize();
+		int            numUserLocks = NumUserDefinedLWLocks();
+		LWLockTranche *tranche;
+
+		/* Init LWLock tranches array */
+		LWLockTrancheArray = (LWLockTranche **)ShmemAlloc(
+			mul_size(sizeof(LWLockTranche**), NUM_LWLOCK_TRANCHES));
 
 		/* Allocate space */
 		ptr = (char *) ShmemAlloc(spaceLocks);
@@ -443,35 +487,48 @@ CreateLWLocks(void)
 
 		MainLWLockArray = (LWLockPadded *) ptr;
 
-		/* Initialize all LWLocks in main array */
-		for (id = 0, lock = MainLWLockArray; id < numLocks; id++, lock++)
-			LWLockInitialize(&lock->lock, 0);
+		/* Now points to individual LWlock tranches */
+		ptr += mul_size(sizeof(LWLockPadded), NUM_INDIVIDUAL_LWLOCKS);
 
 		/*
 		 * Initialize the dynamic-allocation counters, which are stored just
 		 * before the first LWLock.  LWLockCounter[0] is the allocation
 		 * counter for lwlocks, LWLockCounter[1] is the maximum number that
-		 * can be allocated from the main array, and LWLockCounter[2] is the
-		 * allocation counter for tranches.
+		 * can be allocated from the user defined lwlocks array,
+		 * and LWLockCounter[2] is the allocation counter for tranches.
 		 */
 		LWLockCounter = (int *) ((char *) MainLWLockArray - 3 * sizeof(int));
-		LWLockCounter[0] = NUM_FIXED_LWLOCKS;
-		LWLockCounter[1] = numLocks;
-		LWLockCounter[2] = 1;	/* 0 is the main array */
-	}
+		LWLockCounter[0] = 0;
+		LWLockCounter[1] = numUserLocks;
+		LWLockCounter[2] = 0;
 
-	if (LWLockTrancheArray == NULL)
-	{
-		LWLockTranchesAllocated = 16;
-		LWLockTrancheArray = (LWLockTranche **)
-			MemoryContextAlloc(TopMemoryContext,
-						  LWLockTranchesAllocated * sizeof(LWLockTranche *));
-	}
+		tranche = (LWLockTranche *) ptr;
+
+		/* Create tranches for individual LWLocks */
+		for (i = 0; i < NUM_INDIVIDUAL_LWLOCKS; i++, tranche++)
+		{
+			int id = LWLockNewTrancheId();
+
+			/*
+			 * We need to be sure that generated id is equal to index
+			 * for individual LWLocks
+			 */
+			Assert(id == i);
 
-	MainLWLockTranche.name = "main";
-	MainLWLockTranche.array_base = MainLWLockArray;
-	MainLWLockTranche.array_stride = sizeof(LWLockPadded);
-	LWLockRegisterTranche(0, &MainLWLockTranche);
+			tranche->array_base = MainLWLockArray;
+			tranche->array_stride = sizeof(LWLockPadded);
+			MemSet(tranche->name, 0, LWLOCK_MAX_TRANCHE_NAME);
+
+			/* Initialize individual LWLock */
+			LWLockInitialize(&MainLWLockArray[i].lock, id);
+
+			/* Register new tranche in tranches array */
+			LWLockRegisterTranche(id, tranche);
+		}
+
+		/* Fill individual LWLock names */
+		InitLWLockNames();
+	}
 }
 
 /*
@@ -500,13 +557,20 @@ LWLockAssign(void)
 	int		   *LWLockCounter;
 
 	LWLockCounter = (int *) ((char *) MainLWLockArray - 3 * sizeof(int));
+
+	/* Create tranche for user defined LWLocks */
+	if (userdef_lwlocks_array == NULL)
+		LWLockCreateTranche("UserDefinedLocks", LWLockCounter[1],
+			&userdef_lwlocks_array);
+
 	SpinLockAcquire(ShmemLock);
 	if (LWLockCounter[0] >= LWLockCounter[1])
 	{
 		SpinLockRelease(ShmemLock);
 		elog(ERROR, "no more LWLocks available");
 	}
-	result = &MainLWLockArray[LWLockCounter[0]++].lock;
+	result = &userdef_lwlocks_array[LWLockCounter[0]++].lock;
+
 	SpinLockRelease(ShmemLock);
 	return result;
 }
@@ -525,34 +589,105 @@ LWLockNewTrancheId(void)
 	result = LWLockCounter[2]++;
 	SpinLockRelease(ShmemLock);
 
+	if (result == NUM_LWLOCK_TRANCHES)
+		elog(ERROR, "LWLock tranches count exceeded. Consider increasing"\
+			" NUM_LWLOCK_TRANCHES value.");
+
 	return result;
 }
 
 /*
- * Register a tranche ID in the lookup table for the current process.  This
- * routine will save a pointer to the tranche object passed as an argument,
- * so that object should be allocated in a backend-lifetime context
- * (TopMemoryContext, static variable, or similar).
+ * Register a tranche ID in the lookup table in shared memory.
+ * Tranche object must be allocated in shared memory previously
  */
 void
 LWLockRegisterTranche(int tranche_id, LWLockTranche *tranche)
 {
 	Assert(LWLockTrancheArray != NULL);
+	Assert(ShmemAddrIsValid((void *)tranche));
+	LWLockTrancheArray[tranche_id] = tranche;
+}
+
+/*
+ * LWLockCreateTranche - create new tranche for LWLockPadded based LWLocks
+ *
+ * Space for LWLocks and LWLockTranche must be already acquired in
+ * shared memory.
+ * If LWLocks are not based on LWLockPadded then backend must register
+ * tranche itself with LWLockNewTrancheId and LWLockRegisterTranche functions
+ */
+void
+LWLockCreateTranche(const char *tranche_name, int locks_count,
+	LWLockPadded **array)
+{
+	LWLockTranche  *tranche;
+	int             tranche_id;
+	int             len;
+	int             i;
+	char           *tranche_key;
+	char           *prefix = "Tranche";
+	bool            found;
+
+	Assert(strlen(tranche_name) < LWLOCK_MAX_TRANCHE_NAME);
 
-	if (tranche_id >= LWLockTranchesAllocated)
+	/* Generate a tranche name */
+	len = strlen(tranche_name) + strlen(prefix) + 1;
+	tranche_key = (char *) palloc(len);
+	snprintf(tranche_key, len, "%s%s", prefix, tranche_name);
+
+	/* Allocate space in shared memory for tranche */
+	tranche = (LWLockTranche *)
+		ShmemInitStruct(tranche_key, sizeof(LWLockTranche), &found);
+
+	pfree(tranche_key);
+
+	if (tranche == NULL)
+		ereport(PANIC, (errcode(ERRCODE_OUT_OF_MEMORY),
+						errmsg("out of memory")));
+
+	/* Initialize tranche fields */
+	if (!found)
 	{
-		int			i = LWLockTranchesAllocated;
+		/* Allocate space in shared memory for lwlocks */
+		*array = (LWLockPadded *) ShmemAlloc(
+			mul_size(sizeof(LWLockPadded), locks_count));
+
+		if (*array == NULL)
+			ereport(PANIC, (errcode(ERRCODE_OUT_OF_MEMORY),
+							errmsg("out of memory")));
 
-		while (i <= tranche_id)
-			i *= 2;
+		/* Init tranche fields */
+		strcpy(tranche->name, tranche_name);
+		tranche->array_base = (void *)*array;
+		tranche->array_stride = sizeof(LWLockPadded);
 
-		LWLockTrancheArray = (LWLockTranche **)
-			repalloc(LWLockTrancheArray,
-					 i * sizeof(LWLockTranche *));
-		LWLockTranchesAllocated = i;
+		/* Generate id for new tranche */
+		tranche_id = LWLockNewTrancheId();
+
+		/* Register new tranche in tranche array */
+		LWLockRegisterTranche(tranche_id, tranche);
+
+		/* Initialize new LWLocks within created tranche */
+		for (i=0; i < locks_count; i++)
+			LWLockInitialize(&(*array)[i].lock, tranche_id);
 	}
+	else
+		*array = (LWLockPadded *)tranche->array_base;
+}
 
-	LWLockTrancheArray[tranche_id] = tranche;
+/*
+ * LWLockTrancheShmemSize - calculate size required in shared memory for
+ * for tranche with locks_count of LWLocks.
+ *
+ * This function used in pair with LWLockCreateTranche
+ */
+Size
+LWLockTrancheShmemSize(int locks_count)
+{
+	Size size;
+	size = MAXALIGN(sizeof(LWLockTranche));
+	size = add_size(size, mul_size(sizeof(LWLockPadded), locks_count));
+	return size;
 }
 
 /*
@@ -1195,6 +1330,7 @@ LWLockAcquireOrWait(LWLock *lock, LWLockMode mode)
 				Assert(nwaiters < MAX_BACKENDS);
 			}
 #endif
+
 			TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(T_NAME(lock), T_ID(lock), mode);
 
 			LOG_LWDEBUG("LWLockAcquireOrWait", lock, "awakened");
diff --git a/src/backend/storage/lmgr/predicate.c b/src/backend/storage/lmgr/predicate.c
index bad5618..af4bb25 100644
--- a/src/backend/storage/lmgr/predicate.c
+++ b/src/backend/storage/lmgr/predicate.c
@@ -239,13 +239,20 @@
  * apply one of these macros.
  * NB: NUM_PREDICATELOCK_PARTITIONS must be a power of 2!
  */
+
+/* Number of partitions the shared predicate lock tables are divided into */
+#define LOG2_NUM_PREDICATELOCK_PARTITIONS  4
+#define NUM_PREDICATELOCK_PARTITIONS  (1 << LOG2_NUM_PREDICATELOCK_PARTITIONS)
+
+/* LWLocks tranche and array */
+static LWLockPadded		*PredLWLockArray;
+
 #define PredicateLockHashPartition(hashcode) \
 	((hashcode) % NUM_PREDICATELOCK_PARTITIONS)
 #define PredicateLockHashPartitionLock(hashcode) \
-	(&MainLWLockArray[PREDICATELOCK_MANAGER_LWLOCK_OFFSET + \
-		PredicateLockHashPartition(hashcode)].lock)
+	(&PredLWLockArray[PredicateLockHashPartition(hashcode)].lock)
 #define PredicateLockHashPartitionLockByIndex(i) \
-	(&MainLWLockArray[PREDICATELOCK_MANAGER_LWLOCK_OFFSET + (i)].lock)
+	(&PredLWLockArray[i].lock)
 
 #define NPREDICATELOCKTARGETENTS() \
 	mul_size(max_predicate_locks_per_xact, add_size(MaxBackends, max_prepared_xacts))
@@ -395,6 +402,8 @@ static LWLock *ScratchPartitionLock;
  */
 static HTAB *LocalPredicateLockHash = NULL;
 
+
+
 /*
  * Keep a pointer to the currently-running serializable transaction (if any)
  * for quick reference. Also, remember if we have written anything that could
@@ -795,7 +804,8 @@ OldSerXidInit(void)
 	 */
 	OldSerXidSlruCtl->PagePrecedes = OldSerXidPagePrecedesLogically;
 	SimpleLruInit(OldSerXidSlruCtl, "OldSerXid SLRU Ctl",
-				  NUM_OLDSERXID_BUFFERS, 0, OldSerXidLock, "pg_serial");
+				  NUM_OLDSERXID_BUFFERS, 0, OldSerXidLock, "pg_serial",
+				  "OldSerXidBufferLocks");
 	/* Override default assumption that writes should be fsync'd */
 	OldSerXidSlruCtl->do_fsync = false;
 
@@ -1289,6 +1299,9 @@ InitPredicateLocks(void)
 	/* Pre-calculate the hash and partition lock of the scratch entry */
 	ScratchTargetTagHash = PredicateLockTargetTagHashCode(&ScratchTargetTag);
 	ScratchPartitionLock = PredicateLockHashPartitionLock(ScratchTargetTagHash);
+
+	LWLockCreateTranche("PredicateLWLocks", NUM_PREDICATELOCK_PARTITIONS,
+		&PredLWLockArray);
 }
 
 /*
@@ -1340,6 +1353,9 @@ PredicateLockShmemSize(void)
 	size = add_size(size, sizeof(OldSerXidControlData));
 	size = add_size(size, SimpleLruShmemSize(NUM_OLDSERXID_BUFFERS, 0));
 
+	/* LWLocks */
+	size = add_size(size, LWLockTrancheShmemSize(NUM_PREDICATELOCK_PARTITIONS));
+
 	return size;
 }
 
diff --git a/src/backend/storage/lmgr/proc.c b/src/backend/storage/lmgr/proc.c
index 2c2535b..cab98aa 100644
--- a/src/backend/storage/lmgr/proc.c
+++ b/src/backend/storage/lmgr/proc.c
@@ -114,6 +114,10 @@ ProcGlobalShmemSize(void)
 	size = add_size(size, mul_size(NUM_AUXILIARY_PROCS, sizeof(PGXACT)));
 	size = add_size(size, mul_size(max_prepared_xacts, sizeof(PGXACT)));
 
+	/* LWLocks */
+	size = add_size(size,
+		LWLockTrancheShmemSize(MaxBackends + NUM_AUXILIARY_PROCS));
+
 	return size;
 }
 
@@ -157,12 +161,13 @@ ProcGlobalSemas(void)
 void
 InitProcGlobal(void)
 {
-	PGPROC	   *procs;
-	PGXACT	   *pgxacts;
-	int			i,
-				j;
-	bool		found;
-	uint32		TotalProcs = MaxBackends + NUM_AUXILIARY_PROCS + max_prepared_xacts;
+	PGPROC       *procs;
+	PGXACT       *pgxacts;
+	LWLockPadded *lwlocks_array;
+	int           i, j;
+	bool          found;
+	uint32        TotalProcs = MaxBackends + NUM_AUXILIARY_PROCS
+		+ max_prepared_xacts;
 
 	/* Create the ProcGlobal shared structure */
 	ProcGlobal = (PROC_HDR *)
@@ -213,6 +218,10 @@ InitProcGlobal(void)
 	MemSet(pgxacts, 0, TotalProcs * sizeof(PGXACT));
 	ProcGlobal->allPgXact = pgxacts;
 
+	/* Create LWLocks */
+	LWLockCreateTranche("ProcessLocks", MaxBackends + NUM_AUXILIARY_PROCS,
+		&lwlocks_array);
+
 	for (i = 0; i < TotalProcs; i++)
 	{
 		/* Common initialization for all PGPROCs, regardless of type. */
@@ -226,7 +235,7 @@ InitProcGlobal(void)
 		{
 			PGSemaphoreCreate(&(procs[i].sem));
 			InitSharedLatch(&(procs[i].procLatch));
-			procs[i].backendLock = LWLockAssign();
+			procs[i].backendLock = &lwlocks_array[i].lock;
 		}
 		procs[i].pgprocno = i;
 
diff --git a/src/include/access/slru.h b/src/include/access/slru.h
index 9c7f019..d9e9582 100644
--- a/src/include/access/slru.h
+++ b/src/include/access/slru.h
@@ -136,7 +136,8 @@ typedef SlruCtlData *SlruCtl;
 
 extern Size SimpleLruShmemSize(int nslots, int nlsns);
 extern void SimpleLruInit(SlruCtl ctl, const char *name, int nslots, int nlsns,
-			  LWLock *ctllock, const char *subdir);
+			  LWLock *ctllock, const char *subdir,
+			  const char *lwlocks_tranche);
 extern int	SimpleLruZeroPage(SlruCtl ctl, int pageno);
 extern int SimpleLruReadPage(SlruCtl ctl, int pageno, bool write_ok,
 				  TransactionId xid);
diff --git a/src/include/storage/buf_internals.h b/src/include/storage/buf_internals.h
index 521ee1c..1c50d68 100644
--- a/src/include/storage/buf_internals.h
+++ b/src/include/storage/buf_internals.h
@@ -53,6 +53,10 @@ typedef bits16 BufFlags;
  */
 #define BM_MAX_USAGE_COUNT	5
 
+/* Number of partitions of the shared buffer mapping hashtable */
+#define NUM_BUFFER_PARTITIONS  128
+
+
 /*
  * Buffer tag identifies which disk block the buffer contains.
  *
@@ -104,10 +108,8 @@ typedef struct buftag
 #define BufTableHashPartition(hashcode) \
 	((hashcode) % NUM_BUFFER_PARTITIONS)
 #define BufMappingPartitionLock(hashcode) \
-	(&MainLWLockArray[BUFFER_MAPPING_LWLOCK_OFFSET + \
-		BufTableHashPartition(hashcode)].lock)
-#define BufMappingPartitionLockByIndex(i) \
-	(&MainLWLockArray[BUFFER_MAPPING_LWLOCK_OFFSET + (i)].lock)
+	(&BufferLWLockArray[BufTableHashPartition(hashcode)].lock)
+#define BufMappingPartitionLockByIndex(i) (&BufferLWLockArray[i].lock)
 
 /*
  *	BufferDesc -- shared descriptor/state data for a single shared buffer.
@@ -206,6 +208,7 @@ typedef union BufferDescPadded
 
 /* in buf_init.c */
 extern PGDLLIMPORT BufferDescPadded *BufferDescriptors;
+extern PGDLLIMPORT LWLockPadded *BufferLWLockArray;
 
 /* in localbuf.c */
 extern BufferDesc *LocalBufferDescriptors;
diff --git a/src/include/storage/lock.h b/src/include/storage/lock.h
index a9cd08c..9ca9350 100644
--- a/src/include/storage/lock.h
+++ b/src/include/storage/lock.h
@@ -464,13 +464,19 @@ typedef enum
  * hash code with LockTagHashCode(), then apply one of these macros.
  * NB: NUM_LOCK_PARTITIONS must be a power of 2!
  */
+
+/* Number of partitions the shared lock tables are divided into */
+#define LOG2_NUM_LOCK_PARTITIONS  4
+#define NUM_LOCK_PARTITIONS  (1 << LOG2_NUM_LOCK_PARTITIONS)
+
+extern PGDLLIMPORT LWLockPadded *LockMgrLWLockArray;
+
 #define LockHashPartition(hashcode) \
 	((hashcode) % NUM_LOCK_PARTITIONS)
 #define LockHashPartitionLock(hashcode) \
-	(&MainLWLockArray[LOCK_MANAGER_LWLOCK_OFFSET + \
-		LockHashPartition(hashcode)].lock)
+	(&LockMgrLWLockArray[LockHashPartition(hashcode)].lock)
 #define LockHashPartitionLockByIndex(i) \
-	(&MainLWLockArray[LOCK_MANAGER_LWLOCK_OFFSET + (i)].lock)
+	(&LockMgrLWLockArray[i].lock)
 
 /*
  * function prototypes
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index f2ff6a0..9ce73ee 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -37,13 +37,22 @@ struct PGPROC;
  * be an array of lwlocks, but rather some larger data structure that includes
  * one or more lwlocks per element.
  */
+
+#define NUM_LWLOCK_TRANCHES 64
+#define LWLOCK_MAX_TRANCHE_NAME 64
+
 typedef struct LWLockTranche
 {
-	const char *name;
+	char name[LWLOCK_MAX_TRANCHE_NAME];
 	void	   *array_base;
 	Size		array_stride;
 } LWLockTranche;
 
+extern PGDLLIMPORT LWLockTranche **LWLockTrancheArray;
+
+#define LWLOCK_TRANCHE_NAME(tranche_id) \
+	(LWLockTrancheArray[tranche_id]->name)
+
 /*
  * Code outside of lwlock.c should not manipulate the contents of this
  * structure directly, but we have to declare it here to allow LWLocks to be
@@ -89,6 +98,7 @@ typedef union LWLockPadded
 	LWLock		lock;
 	char		pad[LWLOCK_PADDED_SIZE];
 } LWLockPadded;
+
 extern PGDLLIMPORT LWLockPadded *MainLWLockArray;
 
 /*
@@ -96,7 +106,8 @@ extern PGDLLIMPORT LWLockPadded *MainLWLockArray;
  * defining macros here makes it much easier to keep track of these.  If you
  * add a lock, add it to the end to avoid renumbering the existing locks;
  * if you remove a lock, consider leaving a gap in the numbering sequence for
- * the benefit of DTrace and other external debugging scripts.
+ * the benefit of DTrace and other external debugging scripts; names for
+ * individual locks keeped in corresponding array, don't forget add it
  */
 /* 0 is available; was formerly BufFreelistLock */
 #define ShmemIndexLock				(&MainLWLockArray[1].lock)
@@ -142,32 +153,6 @@ extern PGDLLIMPORT LWLockPadded *MainLWLockArray;
 
 #define NUM_INDIVIDUAL_LWLOCKS		41
 
-/*
- * It's a bit odd to declare NUM_BUFFER_PARTITIONS and NUM_LOCK_PARTITIONS
- * here, but we need them to figure out offsets within MainLWLockArray, and
- * having this file include lock.h or bufmgr.h would be backwards.
- */
-
-/* Number of partitions of the shared buffer mapping hashtable */
-#define NUM_BUFFER_PARTITIONS  128
-
-/* Number of partitions the shared lock tables are divided into */
-#define LOG2_NUM_LOCK_PARTITIONS  4
-#define NUM_LOCK_PARTITIONS  (1 << LOG2_NUM_LOCK_PARTITIONS)
-
-/* Number of partitions the shared predicate lock tables are divided into */
-#define LOG2_NUM_PREDICATELOCK_PARTITIONS  4
-#define NUM_PREDICATELOCK_PARTITIONS  (1 << LOG2_NUM_PREDICATELOCK_PARTITIONS)
-
-/* Offsets for various chunks of preallocated lwlocks. */
-#define BUFFER_MAPPING_LWLOCK_OFFSET	NUM_INDIVIDUAL_LWLOCKS
-#define LOCK_MANAGER_LWLOCK_OFFSET		\
-	(BUFFER_MAPPING_LWLOCK_OFFSET + NUM_BUFFER_PARTITIONS)
-#define PREDICATELOCK_MANAGER_LWLOCK_OFFSET \
-	(LOCK_MANAGER_LWLOCK_OFFSET + NUM_LOCK_PARTITIONS)
-#define NUM_FIXED_LWLOCKS \
-	(PREDICATELOCK_MANAGER_LWLOCK_OFFSET + NUM_PREDICATELOCK_PARTITIONS)
-
 typedef enum LWLockMode
 {
 	LW_EXCLUSIVE,
@@ -177,7 +162,6 @@ typedef enum LWLockMode
 								 * to be used as LWLockAcquire argument */
 } LWLockMode;
 
-
 #ifdef LOCK_DEBUG
 extern bool Trace_lwlocks;
 #endif
@@ -222,6 +206,9 @@ extern LWLock *LWLockAssign(void);
 extern int	LWLockNewTrancheId(void);
 extern void LWLockRegisterTranche(int tranche_id, LWLockTranche *tranche);
 extern void LWLockInitialize(LWLock *lock, int tranche_id);
+extern void LWLockCreateTranche(const char *tranche_name, int locks_count,
+	LWLockPadded **array);
+extern Size LWLockTrancheShmemSize(int locks_count);
 
 /*
  * Prior to PostgreSQL 9.4, we used an enum type called LWLockId to refer
