* 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"
if (SRF_IS_FIRSTCALL())
{
- uint32 i;
+ int i;
volatile BufferDesc *bufHdr;
funcctx = SRF_FIRSTCALL_INIT();
funcctx->max_calls = NBuffers;
funcctx->user_fctx = fctx;
-
/* Allocate NBuffers worth of BufferCachePagesRec records. */
fctx->record = (BufferCachePagesRec *) palloc(sizeof(BufferCachePagesRec) * NBuffers);
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. */
}
/* Release Buffer map. */
- LWLockRelease(BufMappingLock);
+ for (i = 0; i < NUM_BUFFER_PARTITIONS; i++)
+ LWLockRelease(FirstBufMappingLock + i);
}
funcctx = SRF_PERCALL_SETUP();
/* Get the saved state */
fctx = funcctx->user_fctx;
-
if (funcctx->call_cntr < funcctx->max_calls)
{
uint32 i = funcctx->call_cntr;
-$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
--------------------------------------
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
* 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
*
*
* 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 $
*
*-------------------------------------------------------------------------
*/
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;
* 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;
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;
* 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");
*
*
* 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 $
*
*-------------------------------------------------------------------------
*/
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;
/* 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)
{
/*
valid = PinBuffer(buf);
/* Can release the mapping lock as soon as we've pinned it */
- LWLockRelease(BufMappingLock);
+ LWLockRelease(newPartitionLock);
*foundPtr = TRUE;
/*
* 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 (;;)
}
/*
- * 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.
* 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)
{
* 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];
valid = PinBuffer(buf);
/* Can release the mapping lock as soon as we've pinned it */
- LWLockRelease(BufMappingLock);
+ LWLockRelease(newPartitionLock);
*foundPtr = TRUE;
* 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 */ );
}
* 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;
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
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 */
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);
if (!BUFFERTAGS_EQUAL(buf->tag, oldTag))
{
UnlockBufHdr(buf);
- LWLockRelease(BufMappingLock);
+ LWLockRelease(oldPartitionLock);
return;
}
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");
* 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;
*/
HOLD_INTERRUPTS();
- LWLockRelease(BufMappingLock);
+ LWLockRelease(oldPartitionLock);
/*
* Insert the buffer at the head of the list of free buffers.
* 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 $
*
*-------------------------------------------------------------------------
*/
*/
/* 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;
/*
* 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;
}
* 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 $
*
*-------------------------------------------------------------------------
*/
(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.
*
/* 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,
* 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 $
*
*-------------------------------------------------------------------------
*/
#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;
* 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
*/
typedef enum LWLockId
{
- BufMappingLock,
BufFreelistLock,
ShmemIndexLock,
OidGenLock,
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;