diff --git a/contrib/pg_buffercache/pg_buffercache_pages.c b/contrib/pg_buffercache/pg_buffercache_pages.c
index a2811eb001b32f1d5a6131e3d62bb101f37b7813..2d7392884eb068ca9daf6f7a48f51829ba5b43be 100644
--- a/contrib/pg_buffercache/pg_buffercache_pages.c
+++ b/contrib/pg_buffercache/pg_buffercache_pages.c
@@ -3,7 +3,7 @@
  * pg_buffercache_pages.c
  *	  display some contents of the buffer cache
  *
- *	  $PostgreSQL: pgsql/contrib/pg_buffercache/pg_buffercache_pages.c,v 1.7 2006/05/30 22:12:13 tgl Exp $
+ *	  $PostgreSQL: pgsql/contrib/pg_buffercache/pg_buffercache_pages.c,v 1.8 2006/07/23 03:07:57 tgl Exp $
  *-------------------------------------------------------------------------
  */
 #include "postgres.h"
@@ -74,7 +74,7 @@ pg_buffercache_pages(PG_FUNCTION_ARGS)
 
 	if (SRF_IS_FIRSTCALL())
 	{
-		uint32		i;
+		int		i;
 		volatile BufferDesc *bufHdr;
 
 		funcctx = SRF_FIRSTCALL_INIT();
@@ -108,7 +108,6 @@ pg_buffercache_pages(PG_FUNCTION_ARGS)
 		funcctx->max_calls = NBuffers;
 		funcctx->user_fctx = fctx;
 
-
 		/* Allocate NBuffers worth of BufferCachePagesRec records. */
 		fctx->record = (BufferCachePagesRec *) palloc(sizeof(BufferCachePagesRec) * NBuffers);
 
@@ -120,17 +119,21 @@ pg_buffercache_pages(PG_FUNCTION_ARGS)
 		fctx->values[4] = (char *) palloc(3 * sizeof(uint32) + 1);
 		fctx->values[5] = (char *) palloc(2);
 
-
 		/* Return to original context when allocating transient memory */
 		MemoryContextSwitchTo(oldcontext);
 
-
 		/*
-		 * Lock Buffer map and scan though all the buffers, saving the
-		 * relevant fields in the fctx->record structure.
+		 * To get a consistent picture of the buffer state, we must lock
+		 * all partitions of the buffer map.  Needless to say, this is
+		 * horrible for concurrency...
 		 */
-		LWLockAcquire(BufMappingLock, LW_SHARED);
+		for (i = 0; i < NUM_BUFFER_PARTITIONS; i++)
+			LWLockAcquire(FirstBufMappingLock + i, LW_SHARED);
 
+		/*
+		 * Scan though all the buffers, saving the relevant fields in the
+		 * fctx->record structure.
+		 */
 		for (i = 0, bufHdr = BufferDescriptors; i < NBuffers; i++, bufHdr++)
 		{
 			/* Lock each buffer header before inspecting. */
@@ -157,7 +160,8 @@ pg_buffercache_pages(PG_FUNCTION_ARGS)
 		}
 
 		/* Release Buffer map. */
-		LWLockRelease(BufMappingLock);
+		for (i = 0; i < NUM_BUFFER_PARTITIONS; i++)
+			LWLockRelease(FirstBufMappingLock + i);
 	}
 
 	funcctx = SRF_PERCALL_SETUP();
@@ -165,7 +169,6 @@ pg_buffercache_pages(PG_FUNCTION_ARGS)
 	/* Get the saved state */
 	fctx = funcctx->user_fctx;
 
-
 	if (funcctx->call_cntr < funcctx->max_calls)
 	{
 		uint32		i = funcctx->call_cntr;
diff --git a/src/backend/storage/buffer/README b/src/backend/storage/buffer/README
index 3b6c72182fb54e91cf853724c5167759fe478bf6..afdea2af747277f68e1ab9c7f607e295a01c4c2a 100644
--- a/src/backend/storage/buffer/README
+++ b/src/backend/storage/buffer/README
@@ -1,4 +1,4 @@
-$PostgreSQL: pgsql/src/backend/storage/buffer/README,v 1.10 2006/06/08 14:58:33 tgl Exp $
+$PostgreSQL: pgsql/src/backend/storage/buffer/README,v 1.11 2006/07/23 03:07:58 tgl Exp $
 
 Notes about shared buffer access rules
 --------------------------------------
@@ -114,6 +114,14 @@ operation that needs exclusive lock is reading in a page that was not
 in shared buffers already, which will require at least a kernel call
 and usually a wait for I/O, so it will be slow anyway.
 
+* As of PG 8.2, the BufMappingLock has been split into NUM_BUFFER_PARTITIONS
+separate locks, each guarding a portion of the buffer tag space.  This allows
+further reduction of contention in the normal code paths.  The partition
+that a particular buffer tag belongs to is determined from the low-order
+bits of the tag's hash value.  The rules stated above apply to each partition
+independently.  If it is necessary to lock more than one partition at a time,
+they must be locked in partition-number order to avoid risk of deadlock.
+
 * A separate system-wide LWLock, the BufFreelistLock, provides mutual
 exclusion for operations that access the buffer free list or select
 buffers for replacement.  This is always taken in exclusive mode since
diff --git a/src/backend/storage/buffer/buf_table.c b/src/backend/storage/buffer/buf_table.c
index 59d1fd170517fb1d2a2b8bf8c32212fd8b307b80..864826c1b4347afd05cc89cc84277bac7500ad51 100644
--- a/src/backend/storage/buffer/buf_table.c
+++ b/src/backend/storage/buffer/buf_table.c
@@ -4,8 +4,10 @@
  *	  routines for mapping BufferTags to buffer indexes.
  *
  * Note: the routines in this file do no locking of their own.	The caller
- * must hold a suitable lock on the BufMappingLock, as specified in the
- * comments.
+ * must hold a suitable lock on the appropriate BufMappingLock, as specified
+ * in the comments.  We can't do the locking inside these functions because
+ * in most cases the caller needs to adjust the buffer header contents
+ * before the lock is released (see notes in README).
  *
  *
  * Portions Copyright (c) 1996-2006, PostgreSQL Global Development Group
@@ -13,7 +15,7 @@
  *
  *
  * IDENTIFICATION
- *	  $PostgreSQL: pgsql/src/backend/storage/buffer/buf_table.c,v 1.46 2006/07/14 16:59:19 tgl Exp $
+ *	  $PostgreSQL: pgsql/src/backend/storage/buffer/buf_table.c,v 1.47 2006/07/23 03:07:58 tgl Exp $
  *
  *-------------------------------------------------------------------------
  */
@@ -58,29 +60,49 @@ InitBufTable(int size)
 	info.keysize = sizeof(BufferTag);
 	info.entrysize = sizeof(BufferLookupEnt);
 	info.hash = tag_hash;
+	info.num_partitions = NUM_BUFFER_PARTITIONS;
 
 	SharedBufHash = ShmemInitHash("Shared Buffer Lookup Table",
 								  size, size,
 								  &info,
-								  HASH_ELEM | HASH_FUNCTION);
+								  HASH_ELEM | HASH_FUNCTION | HASH_PARTITION);
 
 	if (!SharedBufHash)
 		elog(FATAL, "could not initialize shared buffer hash table");
 }
 
+/*
+ * BufTableHashCode
+ *		Compute the hash code associated with a BufferTag
+ *
+ * This must be passed to the lookup/insert/delete routines along with the
+ * tag.  We do it like this because the callers need to know the hash code
+ * in order to determine which buffer partition to lock, and we don't want
+ * to do the hash computation twice (hash_any is a bit slow).
+ */
+uint32
+BufTableHashCode(BufferTag *tagPtr)
+{
+	return get_hash_value(SharedBufHash, (void *) tagPtr);
+}
+
 /*
  * BufTableLookup
  *		Lookup the given BufferTag; return buffer ID, or -1 if not found
  *
- * Caller must hold at least share lock on BufMappingLock
+ * Caller must hold at least share lock on BufMappingLock for tag's partition
  */
 int
-BufTableLookup(BufferTag *tagPtr)
+BufTableLookup(BufferTag *tagPtr, uint32 hashcode)
 {
 	BufferLookupEnt *result;
 
 	result = (BufferLookupEnt *)
-		hash_search(SharedBufHash, (void *) tagPtr, HASH_FIND, NULL);
+		hash_search_with_hash_value(SharedBufHash,
+									(void *) tagPtr,
+									hashcode,
+									HASH_FIND,
+									NULL);
 
 	if (!result)
 		return -1;
@@ -96,10 +118,10 @@ BufTableLookup(BufferTag *tagPtr)
  * Returns -1 on successful insertion.	If a conflicting entry exists
  * already, returns the buffer ID in that entry.
  *
- * Caller must hold write lock on BufMappingLock
+ * Caller must hold exclusive lock on BufMappingLock for tag's partition
  */
 int
-BufTableInsert(BufferTag *tagPtr, int buf_id)
+BufTableInsert(BufferTag *tagPtr, uint32 hashcode, int buf_id)
 {
 	BufferLookupEnt *result;
 	bool		found;
@@ -108,7 +130,11 @@ BufTableInsert(BufferTag *tagPtr, int buf_id)
 	Assert(tagPtr->blockNum != P_NEW);	/* invalid tag */
 
 	result = (BufferLookupEnt *)
-		hash_search(SharedBufHash, (void *) tagPtr, HASH_ENTER, &found);
+		hash_search_with_hash_value(SharedBufHash,
+									(void *) tagPtr,
+									hashcode,
+									HASH_ENTER,
+									&found);
 
 	if (found)					/* found something already in the table */
 		return result->id;
@@ -122,15 +148,19 @@ BufTableInsert(BufferTag *tagPtr, int buf_id)
  * BufTableDelete
  *		Delete the hashtable entry for given tag (which must exist)
  *
- * Caller must hold write lock on BufMappingLock
+ * Caller must hold exclusive lock on BufMappingLock for tag's partition
  */
 void
-BufTableDelete(BufferTag *tagPtr)
+BufTableDelete(BufferTag *tagPtr, uint32 hashcode)
 {
 	BufferLookupEnt *result;
 
 	result = (BufferLookupEnt *)
-		hash_search(SharedBufHash, (void *) tagPtr, HASH_REMOVE, NULL);
+		hash_search_with_hash_value(SharedBufHash,
+									(void *) tagPtr,
+									hashcode,
+									HASH_REMOVE,
+									NULL);
 
 	if (!result)				/* shouldn't happen */
 		elog(ERROR, "shared buffer hash table corrupted");
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 3841d7aa60d886ba7a2f008484abe0ba9e33d1d5..fd68bb91e77394dc2ec75abf50bbabed1f4ef3fc 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -8,7 +8,7 @@
  *
  *
  * IDENTIFICATION
- *	  $PostgreSQL: pgsql/src/backend/storage/buffer/bufmgr.c,v 1.208 2006/07/14 14:52:22 momjian Exp $
+ *	  $PostgreSQL: pgsql/src/backend/storage/buffer/bufmgr.c,v 1.209 2006/07/23 03:07:58 tgl Exp $
  *
  *-------------------------------------------------------------------------
  */
@@ -293,7 +293,11 @@ BufferAlloc(Relation reln,
 			bool *foundPtr)
 {
 	BufferTag	newTag;			/* identity of requested block */
-	BufferTag	oldTag;
+	uint32		newHash;		/* hash value for newTag */
+	LWLockId	newPartitionLock;	/* buffer partition lock for it */
+	BufferTag	oldTag;			/* previous identity of selected buffer */
+	uint32		oldHash;		/* hash value for oldTag */
+	LWLockId	oldPartitionLock;	/* buffer partition lock for it */
 	BufFlags	oldFlags;
 	int			buf_id;
 	volatile BufferDesc *buf;
@@ -302,9 +306,13 @@ BufferAlloc(Relation reln,
 	/* create a tag so we can lookup the buffer */
 	INIT_BUFFERTAG(newTag, reln, blockNum);
 
+	/* determine its hash code and partition lock ID */
+	newHash = BufTableHashCode(&newTag);
+	newPartitionLock = BufMappingPartitionLock(newHash);
+
 	/* see if the block is in the buffer pool already */
-	LWLockAcquire(BufMappingLock, LW_SHARED);
-	buf_id = BufTableLookup(&newTag);
+	LWLockAcquire(newPartitionLock, LW_SHARED);
+	buf_id = BufTableLookup(&newTag, newHash);
 	if (buf_id >= 0)
 	{
 		/*
@@ -317,7 +325,7 @@ BufferAlloc(Relation reln,
 		valid = PinBuffer(buf);
 
 		/* Can release the mapping lock as soon as we've pinned it */
-		LWLockRelease(BufMappingLock);
+		LWLockRelease(newPartitionLock);
 
 		*foundPtr = TRUE;
 
@@ -345,9 +353,9 @@ BufferAlloc(Relation reln,
 
 	/*
 	 * Didn't find it in the buffer pool.  We'll have to initialize a new
-	 * buffer.	Remember to unlock BufMappingLock while doing the work.
+	 * buffer.	Remember to unlock the mapping lock while doing the work.
 	 */
-	LWLockRelease(BufMappingLock);
+	LWLockRelease(newPartitionLock);
 
 	/* Loop here in case we have to try another victim buffer */
 	for (;;)
@@ -412,10 +420,48 @@ BufferAlloc(Relation reln,
 		}
 
 		/*
-		 * Acquire exclusive mapping lock in preparation for changing the
-		 * buffer's association.
+		 * To change the association of a valid buffer, we'll need to have
+		 * exclusive lock on both the old and new mapping partitions.
 		 */
-		LWLockAcquire(BufMappingLock, LW_EXCLUSIVE);
+		if (oldFlags & BM_TAG_VALID)
+		{
+			/*
+			 * Need to compute the old tag's hashcode and partition lock ID.
+			 * XXX is it worth storing the hashcode in BufferDesc so we need
+			 * not recompute it here?  Probably not.
+			 */
+			oldTag = buf->tag;
+			oldHash = BufTableHashCode(&oldTag);
+			oldPartitionLock = BufMappingPartitionLock(oldHash);
+
+			/*
+			 * Must lock the lower-numbered partition first to avoid
+			 * deadlocks.
+			 */
+			if (oldPartitionLock < newPartitionLock)
+			{
+				LWLockAcquire(oldPartitionLock, LW_EXCLUSIVE);
+				LWLockAcquire(newPartitionLock, LW_EXCLUSIVE);
+			}
+			else if (oldPartitionLock > newPartitionLock)
+			{
+				LWLockAcquire(newPartitionLock, LW_EXCLUSIVE);
+				LWLockAcquire(oldPartitionLock, LW_EXCLUSIVE);
+			}
+			else
+			{
+				/* only one partition, only one lock */
+				LWLockAcquire(newPartitionLock, LW_EXCLUSIVE);
+			}
+		}
+		else
+		{
+			/* if it wasn't valid, we need only the new partition */
+			LWLockAcquire(newPartitionLock, LW_EXCLUSIVE);
+			/* these just keep the compiler quiet about uninit variables */
+			oldHash = 0;
+			oldPartitionLock = 0;
+		}
 
 		/*
 		 * Try to make a hashtable entry for the buffer under its new tag.
@@ -424,7 +470,7 @@ BufferAlloc(Relation reln,
 		 * Note that we have not yet removed the hashtable entry for the old
 		 * tag.
 		 */
-		buf_id = BufTableInsert(&newTag, buf->buf_id);
+		buf_id = BufTableInsert(&newTag, newHash, buf->buf_id);
 
 		if (buf_id >= 0)
 		{
@@ -433,10 +479,15 @@ BufferAlloc(Relation reln,
 			 * do. We'll just handle this as if it were found in the buffer
 			 * pool in the first place.  First, give up the buffer we were
 			 * planning to use.  Don't allow it to be thrown in the free list
-			 * (we don't want to hold both global locks at once).
+			 * (we don't want to hold freelist and mapping locks at once).
 			 */
 			UnpinBuffer(buf, true, false);
 
+			/* Can give up that buffer's mapping partition lock now */
+			if ((oldFlags & BM_TAG_VALID) &&
+				oldPartitionLock != newPartitionLock)
+				LWLockRelease(oldPartitionLock);
+
 			/* remaining code should match code at top of routine */
 
 			buf = &BufferDescriptors[buf_id];
@@ -444,7 +495,7 @@ BufferAlloc(Relation reln,
 			valid = PinBuffer(buf);
 
 			/* Can release the mapping lock as soon as we've pinned it */
-			LWLockRelease(BufMappingLock);
+			LWLockRelease(newPartitionLock);
 
 			*foundPtr = TRUE;
 
@@ -481,12 +532,16 @@ BufferAlloc(Relation reln,
 		 * recycle this buffer; we must undo everything we've done and start
 		 * over with a new victim buffer.
 		 */
-		if (buf->refcount == 1 && !(buf->flags & BM_DIRTY))
+		oldFlags = buf->flags;
+		if (buf->refcount == 1 && !(oldFlags & BM_DIRTY))
 			break;
 
 		UnlockBufHdr(buf);
-		BufTableDelete(&newTag);
-		LWLockRelease(BufMappingLock);
+		BufTableDelete(&newTag, newHash);
+		if ((oldFlags & BM_TAG_VALID) &&
+			oldPartitionLock != newPartitionLock)
+			LWLockRelease(oldPartitionLock);
+		LWLockRelease(newPartitionLock);
 		UnpinBuffer(buf, true, false /* evidently recently used */ );
 	}
 
@@ -497,8 +552,6 @@ BufferAlloc(Relation reln,
 	 * paranoia.  We also clear the usage_count since any recency of use of
 	 * the old content is no longer relevant.
 	 */
-	oldTag = buf->tag;
-	oldFlags = buf->flags;
 	buf->tag = newTag;
 	buf->flags &= ~(BM_VALID | BM_DIRTY | BM_JUST_DIRTIED | BM_IO_ERROR);
 	buf->flags |= BM_TAG_VALID;
@@ -507,9 +560,13 @@ BufferAlloc(Relation reln,
 	UnlockBufHdr(buf);
 
 	if (oldFlags & BM_TAG_VALID)
-		BufTableDelete(&oldTag);
+	{
+		BufTableDelete(&oldTag, oldHash);
+		if (oldPartitionLock != newPartitionLock)
+			LWLockRelease(oldPartitionLock);
+	}
 
-	LWLockRelease(BufMappingLock);
+	LWLockRelease(newPartitionLock);
 
 	/*
 	 * Buffer contents are currently invalid.  Try to get the io_in_progress
@@ -545,6 +602,8 @@ static void
 InvalidateBuffer(volatile BufferDesc *buf)
 {
 	BufferTag	oldTag;
+	uint32		oldHash;		/* hash value for oldTag */
+	LWLockId	oldPartitionLock;	/* buffer partition lock for it */
 	BufFlags	oldFlags;
 
 	/* Save the original buffer tag before dropping the spinlock */
@@ -552,13 +611,21 @@ InvalidateBuffer(volatile BufferDesc *buf)
 
 	UnlockBufHdr(buf);
 
+	/*
+	 * Need to compute the old tag's hashcode and partition lock ID.
+	 * XXX is it worth storing the hashcode in BufferDesc so we need
+	 * not recompute it here?  Probably not.
+	 */
+	oldHash = BufTableHashCode(&oldTag);
+	oldPartitionLock = BufMappingPartitionLock(oldHash);
+
 retry:
 
 	/*
 	 * Acquire exclusive mapping lock in preparation for changing the buffer's
 	 * association.
 	 */
-	LWLockAcquire(BufMappingLock, LW_EXCLUSIVE);
+	LWLockAcquire(oldPartitionLock, LW_EXCLUSIVE);
 
 	/* Re-lock the buffer header */
 	LockBufHdr(buf);
@@ -567,7 +634,7 @@ retry:
 	if (!BUFFERTAGS_EQUAL(buf->tag, oldTag))
 	{
 		UnlockBufHdr(buf);
-		LWLockRelease(BufMappingLock);
+		LWLockRelease(oldPartitionLock);
 		return;
 	}
 
@@ -583,7 +650,7 @@ retry:
 	if (buf->refcount != 0)
 	{
 		UnlockBufHdr(buf);
-		LWLockRelease(BufMappingLock);
+		LWLockRelease(oldPartitionLock);
 		/* safety check: should definitely not be our *own* pin */
 		if (PrivateRefCount[buf->buf_id] != 0)
 			elog(ERROR, "buffer is pinned in InvalidateBuffer");
@@ -606,7 +673,7 @@ retry:
 	 * Remove the buffer from the lookup hashtable, if it was in there.
 	 */
 	if (oldFlags & BM_TAG_VALID)
-		BufTableDelete(&oldTag);
+		BufTableDelete(&oldTag, oldHash);
 
 	/*
 	 * Avoid accepting a cancel interrupt when we release the mapping lock;
@@ -616,7 +683,7 @@ retry:
 	 */
 	HOLD_INTERRUPTS();
 
-	LWLockRelease(BufMappingLock);
+	LWLockRelease(oldPartitionLock);
 
 	/*
 	 * Insert the buffer at the head of the list of free buffers.
diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index 0005162b07734d043d4fe0509dc0dbe5fc846a16..077bec4a60f5f04fd33dddc9ef71f0df0469dc35 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -15,7 +15,7 @@
  * Portions Copyright (c) 1994, Regents of the University of California
  *
  * IDENTIFICATION
- *	  $PostgreSQL: pgsql/src/backend/storage/lmgr/lwlock.c,v 1.40 2006/07/14 14:52:23 momjian Exp $
+ *	  $PostgreSQL: pgsql/src/backend/storage/lmgr/lwlock.c,v 1.41 2006/07/23 03:07:58 tgl Exp $
  *
  *-------------------------------------------------------------------------
  */
@@ -157,10 +157,7 @@ NumLWLocks(void)
 	 */
 
 	/* Predefined LWLocks */
-	numLocks = (int) FirstLockMgrLock;
-
-	/* lock.c gets the ones starting at FirstLockMgrLock */
-	numLocks += NUM_LOCK_PARTITIONS;
+	numLocks = (int) NumFixedLWLocks;
 
 	/* bufmgr.c needs two for each shared buffer */
 	numLocks += 2 * NBuffers;
@@ -239,11 +236,10 @@ CreateLWLocks(void)
 
 	/*
 	 * Initialize the dynamic-allocation counter, which is stored just before
-	 * the first LWLock.  The LWLocks used by lock.c are not dynamically
-	 * allocated, it just assumes it has them.
+	 * the first LWLock.
 	 */
 	LWLockCounter = (int *) ((char *) LWLockArray - 2 * sizeof(int));
-	LWLockCounter[0] = (int) FirstLockMgrLock + NUM_LOCK_PARTITIONS;
+	LWLockCounter[0] = (int) NumFixedLWLocks;
 	LWLockCounter[1] = numLocks;
 }
 
diff --git a/src/include/storage/buf_internals.h b/src/include/storage/buf_internals.h
index 4effa0d740f9b4eba79d281deca91e4f040a14a1..18239176b8ec56bf80c3b9adad54e85b46b6cac7 100644
--- a/src/include/storage/buf_internals.h
+++ b/src/include/storage/buf_internals.h
@@ -8,7 +8,7 @@
  * Portions Copyright (c) 1996-2006, PostgreSQL Global Development Group
  * Portions Copyright (c) 1994, Regents of the University of California
  *
- * $PostgreSQL: pgsql/src/include/storage/buf_internals.h,v 1.86 2006/03/31 23:32:07 tgl Exp $
+ * $PostgreSQL: pgsql/src/include/storage/buf_internals.h,v 1.87 2006/07/23 03:07:58 tgl Exp $
  *
  *-------------------------------------------------------------------------
  */
@@ -86,6 +86,17 @@ typedef struct buftag
 	(a).blockNum == (b).blockNum \
 )
 
+/*
+ * The shared buffer mapping table is partitioned to reduce contention.
+ * To determine which partition lock a given tag requires, compute the tag's
+ * hash code with BufTableHashCode(), then apply BufMappingPartitionLock().
+ * NB: NUM_BUFFER_PARTITIONS must be a power of 2!
+ */
+#define BufTableHashPartition(hashcode) \
+	((hashcode) % NUM_BUFFER_PARTITIONS)
+#define BufMappingPartitionLock(hashcode) \
+	((LWLockId) (FirstBufMappingLock + BufTableHashPartition(hashcode)))
+
 /*
  *	BufferDesc -- shared descriptor/state data for a single shared buffer.
  *
@@ -182,9 +193,10 @@ extern void StrategyInitialize(bool init);
 /* buf_table.c */
 extern Size BufTableShmemSize(int size);
 extern void InitBufTable(int size);
-extern int	BufTableLookup(BufferTag *tagPtr);
-extern int	BufTableInsert(BufferTag *tagPtr, int buf_id);
-extern void BufTableDelete(BufferTag *tagPtr);
+extern uint32 BufTableHashCode(BufferTag *tagPtr);
+extern int	BufTableLookup(BufferTag *tagPtr, uint32 hashcode);
+extern int	BufTableInsert(BufferTag *tagPtr, uint32 hashcode, int buf_id);
+extern void BufTableDelete(BufferTag *tagPtr, uint32 hashcode);
 
 /* localbuf.c */
 extern BufferDesc *LocalBufferAlloc(Relation reln, BlockNumber blockNum,
diff --git a/src/include/storage/lock.h b/src/include/storage/lock.h
index 136299c060dca8292e9cc5db46300f5c5d7ba7d9..d471293b702ad639f5443b298d6d8295d77ab61d 100644
--- a/src/include/storage/lock.h
+++ b/src/include/storage/lock.h
@@ -7,7 +7,7 @@
  * Portions Copyright (c) 1996-2006, PostgreSQL Global Development Group
  * Portions Copyright (c) 1994, Regents of the University of California
  *
- * $PostgreSQL: pgsql/src/include/storage/lock.h,v 1.94 2006/03/05 15:58:59 momjian Exp $
+ * $PostgreSQL: pgsql/src/include/storage/lock.h,v 1.95 2006/07/23 03:07:58 tgl Exp $
  *
  *-------------------------------------------------------------------------
  */
@@ -19,23 +19,15 @@
 #include "storage/shmem.h"
 
 
-/*
- * Number of partitions the shared lock tables are divided into.
- *
- * See LockTagToPartition() if you change this.
- */
-#define NUM_LOCK_PARTITIONS  16
+/* struct PGPROC is declared in proc.h, but must forward-reference it */
+typedef struct PGPROC PGPROC;
 
-/* originally in procq.h */
 typedef struct PROC_QUEUE
 {
 	SHM_QUEUE	links;			/* head of list of PGPROC objects */
 	int			size;			/* number of entries in list */
 } PROC_QUEUE;
 
-/* struct PGPROC is declared in proc.h, but must forward-reference it */
-typedef struct PGPROC PGPROC;
-
 /* GUC variables */
 extern int	max_locks_per_xact;
 
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index 47742c02c590d38aaa2f5950cee5005a9559b34a..2810326597ea81e5da3a625b62580ca1f9c2d89a 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -7,13 +7,25 @@
  * Portions Copyright (c) 1996-2006, PostgreSQL Global Development Group
  * Portions Copyright (c) 1994, Regents of the University of California
  *
- * $PostgreSQL: pgsql/src/include/storage/lwlock.h,v 1.28 2006/05/08 00:00:17 tgl Exp $
+ * $PostgreSQL: pgsql/src/include/storage/lwlock.h,v 1.29 2006/07/23 03:07:58 tgl Exp $
  *
  *-------------------------------------------------------------------------
  */
 #ifndef LWLOCK_H
 #define LWLOCK_H
 
+/*
+ * It's a bit odd to declare NUM_BUFFER_PARTITIONS and NUM_LOCK_PARTITIONS
+ * here, but we need them to set up enum LWLockId correctly, 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  16
+
+/* Number of partitions the shared lock tables are divided into */
+#define NUM_LOCK_PARTITIONS  16
+
 /*
  * We have a number of predefined LWLocks, plus a bunch of LWLocks that are
  * dynamically assigned (e.g., for shared buffers).  The LWLock structures
@@ -25,7 +37,6 @@
  */
 typedef enum LWLockId
 {
-	BufMappingLock,
 	BufFreelistLock,
 	ShmemIndexLock,
 	OidGenLock,
@@ -48,7 +59,11 @@ typedef enum LWLockId
 	TwoPhaseStateLock,
 	TablespaceCreateLock,
 	BtreeVacuumLock,
-	FirstLockMgrLock,			/* must be last except for MaxDynamicLWLock */
+	FirstBufMappingLock,
+	FirstLockMgrLock = FirstBufMappingLock + NUM_BUFFER_PARTITIONS,
+
+	/* must be last except for MaxDynamicLWLock: */
+	NumFixedLWLocks = FirstLockMgrLock + NUM_LOCK_PARTITIONS,
 
 	MaxDynamicLWLock = 1000000000
 } LWLockId;