postgresql/src/backend/storage/buffer/bufmgr.c

2304 lines
57 KiB
C
Raw Normal View History

/*-------------------------------------------------------------------------
*
* bufmgr.c
* buffer manager interface routines
*
* Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/storage/buffer/bufmgr.c,v 1.112 2001/06/09 18:16:57 tgl Exp $
*
*-------------------------------------------------------------------------
*/
/*
*
* BufferAlloc() -- lookup a buffer in the buffer table. If
* it isn't there add it, but do not read data into memory.
* This is used when we are about to reinitialize the
* buffer so don't care what the current disk contents are.
* BufferAlloc() also pins the new buffer in memory.
*
* ReadBuffer() -- like BufferAlloc() but reads the data
* on a buffer cache miss.
*
* ReleaseBuffer() -- unpin the buffer
*
* WriteNoReleaseBuffer() -- mark the buffer contents as "dirty"
* but don't unpin. The disk IO is delayed until buffer
* replacement.
*
* WriteBuffer() -- WriteNoReleaseBuffer() + ReleaseBuffer()
*
* BufferSync() -- flush all dirty buffers in the buffer pool.
*
* InitBufferPool() -- Init the buffer module.
*
* See other files:
* freelist.c -- chooses victim for buffer replacement
* buf_table.c -- manages the buffer lookup table
*/
#include "postgres.h"
2000-10-28 18:21:00 +02:00
1997-07-28 02:57:08 +02:00
#include <sys/types.h>
#include <sys/file.h>
#include <math.h>
#include <signal.h>
1999-07-16 07:00:38 +02:00
#include "executor/execdebug.h"
#include "miscadmin.h"
#include "storage/buf_internals.h"
#include "storage/bufmgr.h"
1997-09-18 16:21:02 +02:00
#include "storage/s_lock.h"
#include "storage/smgr.h"
1996-11-08 07:02:30 +01:00
#include "utils/relcache.h"
#include "catalog/pg_database.h"
2000-11-30 09:46:26 +01:00
#define BufferGetLSN(bufHdr) \
(*((XLogRecPtr*)MAKE_PTR((bufHdr)->data)))
extern SPINLOCK BufMgrLock;
extern long int ReadBufferCount;
extern long int ReadLocalBufferCount;
extern long int BufferHitCount;
extern long int LocalBufferHitCount;
extern long int BufferFlushCount;
extern long int LocalBufferFlushCount;
/*
* It's used to avoid disk writes for read-only transactions
* (i.e. when no one shared buffer was changed by transaction).
* We set it to true in WriteBuffer/WriteNoReleaseBuffer when
* marking shared buffer as dirty. We set it to false in xact.c
* after transaction is committed/aborted.
*/
bool SharedBufferChanged = false;
1997-09-08 22:59:27 +02:00
static void WaitIO(BufferDesc *buf, SPINLOCK spinlock);
static void StartBufferIO(BufferDesc *buf, bool forInput);
static void TerminateBufferIO(BufferDesc *buf);
static void ContinueBufferIO(BufferDesc *buf, bool forInput);
extern void AbortBufferIO(void);
/*
* Macro : BUFFER_IS_BROKEN
* Note that write error doesn't mean the buffer broken
*/
#define BUFFER_IS_BROKEN(buf) ((buf->flags & BM_IO_ERROR) && !(buf->flags & BM_DIRTY))
static Buffer ReadBufferInternal(Relation reln, BlockNumber blockNum,
bool isExtend, bool bufferLockHeld);
static BufferDesc *BufferAlloc(Relation reln, BlockNumber blockNum,
bool *foundPtr);
static int ReleaseBufferWithBufferLock(Buffer buffer);
static int BufferReplace(BufferDesc *bufHdr);
1999-05-25 18:15:34 +02:00
void PrintBufferDescs(void);
/*
* ReadBuffer -- returns a buffer containing the requested
* block of the requested relation. If the blknum
* requested is P_NEW, extend the relation file and
* allocate a new block.
*
* Returns: the buffer number for the buffer containing
* the block read, or NULL on an error. If successful,
* the returned buffer has been pinned.
*
* Assume when this function is called, that reln has been
* opened already.
*/
#undef ReadBuffer /* conflicts with macro when BUFMGR_DEBUG
* defined */
/*
1999-05-25 18:15:34 +02:00
* ReadBuffer
*/
Buffer
ReadBuffer(Relation reln, BlockNumber blockNum)
{
return ReadBufferInternal(reln, blockNum, false, false);
}
/*
* ReadBufferInternal -- internal version of ReadBuffer with more options
*
* isExtend: if true, assume that we are extending the file and the caller
* is passing the current EOF block number (ie, caller already called
* smgrnblocks()).
*
* bufferLockHeld: if true, caller already acquired the bufmgr spinlock.
* (This is assumed never to be true if dealing with a local buffer!)
*/
static Buffer
ReadBufferInternal(Relation reln, BlockNumber blockNum,
bool isExtend, bool bufferLockHeld)
{
BufferDesc *bufHdr;
int status;
bool found;
bool isLocalBuf;
isLocalBuf = reln->rd_myxactonly;
if (isLocalBuf)
{
ReadLocalBufferCount++;
/* Substitute proper block number if caller asked for P_NEW */
if (blockNum == P_NEW)
{
blockNum = reln->rd_nblocks;
reln->rd_nblocks++;
isExtend = true;
}
bufHdr = LocalBufferAlloc(reln, blockNum, &found);
if (found)
LocalBufferHitCount++;
}
else
{
ReadBufferCount++;
/* Substitute proper block number if caller asked for P_NEW */
if (blockNum == P_NEW)
{
blockNum = smgrnblocks(DEFAULT_SMGR, reln);
isExtend = true;
}
/*
* lookup the buffer. IO_IN_PROGRESS is set if the requested
* block is not currently in memory.
*/
if (!bufferLockHeld)
SpinAcquire(BufMgrLock);
bufHdr = BufferAlloc(reln, blockNum, &found);
if (found)
BufferHitCount++;
}
/* At this point we do NOT hold the bufmgr spinlock. */
if (!bufHdr)
1998-09-01 05:29:17 +02:00
return InvalidBuffer;
2000-01-05 19:23:54 +01:00
/* if it's already in the buffer pool, we're done */
if (found)
{
/*
* Could have found && isExtend if a buffer was already created for
* the next page position, but then smgrextend failed to write
* the page. Must fall through and try to extend file again.
*/
if (!isExtend)
return BufferDescriptorGetBuffer(bufHdr);
}
/*
* if we have gotten to this point, the reln pointer must be ok and
* the relation file must be open.
*/
if (isExtend)
{
/* new buffers are zero-filled */
1997-09-18 22:22:58 +02:00
MemSet((char *) MAKE_PTR(bufHdr->data), 0, BLCKSZ);
status = smgrextend(DEFAULT_SMGR, reln, blockNum,
(char *) MAKE_PTR(bufHdr->data));
}
else
{
status = smgrread(DEFAULT_SMGR, reln, blockNum,
(char *) MAKE_PTR(bufHdr->data));
}
if (isLocalBuf)
{
/* No shared buffer state to update... */
if (status == SM_FAIL)
{
bufHdr->flags |= BM_IO_ERROR;
return InvalidBuffer;
}
1998-09-01 05:29:17 +02:00
return BufferDescriptorGetBuffer(bufHdr);
}
/* lock buffer manager again to update IO IN PROGRESS */
SpinAcquire(BufMgrLock);
if (status == SM_FAIL)
{
/* IO Failed. cleanup the data structures and go home */
if (!BufTableDelete(bufHdr))
{
SpinRelease(BufMgrLock);
elog(FATAL, "BufRead: buffer table broken after IO error\n");
}
/* remember that BufferAlloc() pinned the buffer */
UnpinBuffer(bufHdr);
/*
* Have to reset the flag so that anyone waiting for the buffer
* can tell that the contents are invalid.
*/
bufHdr->flags |= BM_IO_ERROR;
bufHdr->flags &= ~BM_IO_IN_PROGRESS;
}
else
{
/* IO Succeeded. clear the flags, finish buffer update */
bufHdr->flags &= ~(BM_IO_ERROR | BM_IO_IN_PROGRESS);
}
/* If anyone was waiting for IO to complete, wake them up now */
TerminateBufferIO(bufHdr);
SpinRelease(BufMgrLock);
if (status == SM_FAIL)
1998-09-01 05:29:17 +02:00
return InvalidBuffer;
1998-09-01 05:29:17 +02:00
return BufferDescriptorGetBuffer(bufHdr);
}
/*
* BufferAlloc -- Get a buffer from the buffer pool but don't
* read it. If successful, the returned buffer is pinned.
*
* Returns: descriptor for buffer
*
* BufMgrLock must be held at entry. When this routine returns,
* the BufMgrLock is guaranteed NOT to be held.
*/
static BufferDesc *
BufferAlloc(Relation reln,
BlockNumber blockNum,
bool *foundPtr)
{
BufferDesc *buf,
*buf2;
BufferTag newTag; /* identity of requested block */
bool inProgress; /* buffer undergoing IO */
/* create a new tag so we can lookup the buffer */
/* assume that the relation is already open */
INIT_BUFFERTAG(&newTag, reln, blockNum);
/* see if the block is in the buffer pool already */
buf = BufTableLookup(&newTag);
if (buf != NULL)
{
/*
* Found it. Now, (a) pin the buffer so no one steals it from the
* buffer pool, (b) check IO_IN_PROGRESS, someone may be faulting
* the buffer into the buffer pool.
*/
PinBuffer(buf);
inProgress = (buf->flags & BM_IO_IN_PROGRESS);
*foundPtr = TRUE;
if (inProgress) /* confirm end of IO */
{
WaitIO(buf, BufMgrLock);
inProgress = (buf->flags & BM_IO_IN_PROGRESS);
}
if (BUFFER_IS_BROKEN(buf))
{
/*
* I couldn't understand the following old comment. If there's
* no IO for the buffer and the buffer is BROKEN,it should be
* read again. So start a new buffer IO here.
*
* wierd race condition:
*
* We were waiting for someone else to read the buffer. While we
* were waiting, the reader boof'd in some way, so the
* contents of the buffer are still invalid. By saying that
* we didn't find it, we can make the caller reinitialize the
* buffer. If two processes are waiting for this block, both
* will read the block. The second one to finish may
* overwrite any updates made by the first. (Assume higher
* level synchronization prevents this from happening).
*
* This is never going to happen, don't worry about it.
*/
*foundPtr = FALSE;
}
#ifdef BMTRACE
_bm_trace((reln->rd_rel->relisshared ? 0 : MyDatabaseId), RelationGetRelid(reln), blockNum, BufferDescriptorGetBuffer(buf), BMT_ALLOCFND);
#endif /* BMTRACE */
if (!(*foundPtr))
StartBufferIO(buf, true);
SpinRelease(BufMgrLock);
1998-09-01 05:29:17 +02:00
return buf;
}
*foundPtr = FALSE;
/*
* Didn't find it in the buffer pool. We'll have to initialize a new
* buffer. First, grab one from the free list. If it's dirty, flush
* it to disk. Remember to unlock BufMgr spinlock while doing the IOs.
*/
inProgress = FALSE;
for (buf = (BufferDesc *) NULL; buf == (BufferDesc *) NULL;)
{
buf = GetFreeBuffer();
/* GetFreeBuffer will abort if it can't find a free buffer */
Assert(buf);
/*
* There should be exactly one pin on the buffer after it is
* allocated -- ours. If it had a pin it wouldn't have been on
* the free list. No one else could have pinned it between
* GetFreeBuffer and here because we have the BufMgrLock.
*/
Assert(buf->refcount == 0);
buf->refcount = 1;
PrivateRefCount[BufferDescriptorGetBuffer(buf) - 1] = 1;
2000-11-30 09:46:26 +01:00
if (buf->flags & BM_DIRTY || buf->cntxDirty)
{
bool smok;
/*
2001-03-22 05:01:46 +01:00
* skip write error buffers
*/
if ((buf->flags & BM_IO_ERROR) != 0)
{
UnpinBuffer(buf);
buf = (BufferDesc *) NULL;
continue;
}
2001-03-22 05:01:46 +01:00
/*
* Set BM_IO_IN_PROGRESS to keep anyone from doing anything
* with the contents of the buffer while we write it out. We
* don't really care if they try to read it, but if they can
* complete a BufferAlloc on it they can then scribble into
* it, and we'd really like to avoid that while we are
* flushing the buffer. Setting this flag should block them
* in WaitIO until we're done.
*/
inProgress = TRUE;
/*
* All code paths that acquire this lock pin the buffer first;
* since no one had it pinned (it just came off the free
* list), no one else can have this lock.
*/
StartBufferIO(buf, false);
/*
* Write the buffer out, being careful to release BufMgrLock
* before starting the I/O.
*/
smok = BufferReplace(buf);
if (smok == FALSE)
{
elog(NOTICE, "BufferAlloc: cannot write block %u for %s/%s",
buf->tag.blockNum, buf->blind.dbname, buf->blind.relname);
inProgress = FALSE;
buf->flags |= BM_IO_ERROR;
buf->flags &= ~BM_IO_IN_PROGRESS;
TerminateBufferIO(buf);
PrivateRefCount[BufferDescriptorGetBuffer(buf) - 1] = 0;
Assert(buf->refcount > 0);
buf->refcount--;
if (buf->refcount == 0)
{
AddBufferToFreelist(buf);
buf->flags |= BM_FREE;
}
buf = (BufferDesc *) NULL;
}
else
{
2001-03-22 05:01:46 +01:00
/*
* BM_JUST_DIRTIED cleared by BufferReplace and shouldn't
* be setted by anyone. - vadim 01/17/97
*/
if (buf->flags & BM_JUST_DIRTIED)
{
2000-11-30 09:46:26 +01:00
elog(STOP, "BufferAlloc: content of block %u (%s) changed while flushing",
buf->tag.blockNum, buf->blind.relname);
}
else
buf->flags &= ~BM_DIRTY;
2000-11-30 09:46:26 +01:00
buf->cntxDirty = false;
}
/*
* Somebody could have pinned the buffer while we were doing
* the I/O and had given up the BufMgrLock (though they would
* be waiting for us to clear the BM_IO_IN_PROGRESS flag).
* That's why this is a loop -- if so, we need to clear the
* I/O flags, remove our pin and start all over again.
*
* People may be making buffers free at any time, so there's no
* reason to think that we have an immediate disaster on our
* hands.
*/
if (buf && buf->refcount > 1)
{
inProgress = FALSE;
buf->flags &= ~BM_IO_IN_PROGRESS;
TerminateBufferIO(buf);
PrivateRefCount[BufferDescriptorGetBuffer(buf) - 1] = 0;
buf->refcount--;
buf = (BufferDesc *) NULL;
}
/*
* Somebody could have allocated another buffer for the same
* block we are about to read in. (While we flush out the
* dirty buffer, we don't hold the lock and someone could have
* allocated another buffer for the same block. The problem is
* we haven't gotten around to insert the new tag into the
* buffer table. So we need to check here. -ay 3/95
*/
buf2 = BufTableLookup(&newTag);
if (buf2 != NULL)
{
/*
* Found it. Someone has already done what we're about to
* do. We'll just handle this as if it were found in the
* buffer pool in the first place.
*/
if (buf != NULL)
{
buf->flags &= ~BM_IO_IN_PROGRESS;
TerminateBufferIO(buf);
/* give up the buffer since we don't need it any more */
PrivateRefCount[BufferDescriptorGetBuffer(buf) - 1] = 0;
Assert(buf->refcount > 0);
buf->refcount--;
if (buf->refcount == 0)
{
AddBufferToFreelist(buf);
buf->flags |= BM_FREE;
}
}
PinBuffer(buf2);
inProgress = (buf2->flags & BM_IO_IN_PROGRESS);
*foundPtr = TRUE;
if (inProgress)
{
WaitIO(buf2, BufMgrLock);
inProgress = (buf2->flags & BM_IO_IN_PROGRESS);
}
if (BUFFER_IS_BROKEN(buf2))
*foundPtr = FALSE;
if (!(*foundPtr))
StartBufferIO(buf2, true);
SpinRelease(BufMgrLock);
1998-09-01 05:29:17 +02:00
return buf2;
}
}
}
/*
* At this point we should have the sole pin on a non-dirty buffer and
* we may or may not already have the BM_IO_IN_PROGRESS flag set.
*/
/*
* Change the name of the buffer in the lookup table:
*
* Need to update the lookup table before the read starts. If someone
* comes along looking for the buffer while we are reading it in, we
* don't want them to allocate a new buffer. For the same reason, we
* didn't want to erase the buf table entry for the buffer we were
* writing back until now, either.
*/
if (!BufTableDelete(buf))
{
SpinRelease(BufMgrLock);
elog(FATAL, "buffer wasn't in the buffer table\n");
}
/* record the database name and relation name for this buffer */
2000-10-21 17:43:36 +02:00
strcpy(buf->blind.dbname, (DatabaseName) ? DatabaseName : "Recovery");
strcpy(buf->blind.relname, RelationGetPhysicalRelationName(reln));
INIT_BUFFERTAG(&(buf->tag), reln, blockNum);
if (!BufTableInsert(buf))
{
SpinRelease(BufMgrLock);
elog(FATAL, "Buffer in lookup table twice \n");
}
/*
* Buffer contents are currently invalid. Have to mark IO IN PROGRESS
* so no one fiddles with them until the read completes. If this
* routine has been called simply to allocate a buffer, no io will be
* attempted, so the flag isnt set.
*/
if (!inProgress)
StartBufferIO(buf, true);
else
ContinueBufferIO(buf, true);
#ifdef BMTRACE
_bm_trace((reln->rd_rel->relisshared ? 0 : MyDatabaseId), RelationGetRelid(reln), blockNum, BufferDescriptorGetBuffer(buf), BMT_ALLOCNOTFND);
#endif /* BMTRACE */
SpinRelease(BufMgrLock);
1998-09-01 05:29:17 +02:00
return buf;
}
/*
* WriteBuffer
*
* Marks buffer contents as dirty (actual write happens later).
*
* Assume that buffer is pinned. Assume that reln is
* valid.
*
* Side Effects:
* Pin count is decremented.
*/
#undef WriteBuffer
int
WriteBuffer(Buffer buffer)
{
BufferDesc *bufHdr;
if (BufferIsLocal(buffer))
return WriteLocalBuffer(buffer, TRUE);
if (BAD_BUFFER_ID(buffer))
return FALSE;
bufHdr = &BufferDescriptors[buffer - 1];
SharedBufferChanged = true;
SpinAcquire(BufMgrLock);
Assert(bufHdr->refcount > 0);
2000-11-30 09:46:26 +01:00
bufHdr->flags |= (BM_DIRTY | BM_JUST_DIRTIED);
2000-11-30 09:46:26 +01:00
UnpinBuffer(bufHdr);
SpinRelease(BufMgrLock);
1998-09-01 05:29:17 +02:00
return TRUE;
}
/*
* WriteNoReleaseBuffer -- like WriteBuffer, but do not unpin the buffer
* when the operation is complete.
*/
int
WriteNoReleaseBuffer(Buffer buffer)
{
BufferDesc *bufHdr;
if (BufferIsLocal(buffer))
return WriteLocalBuffer(buffer, FALSE);
if (BAD_BUFFER_ID(buffer))
return STATUS_ERROR;
bufHdr = &BufferDescriptors[buffer - 1];
SharedBufferChanged = true;
SpinAcquire(BufMgrLock);
Assert(bufHdr->refcount > 0);
2000-11-30 09:46:26 +01:00
bufHdr->flags |= (BM_DIRTY | BM_JUST_DIRTIED);
2000-11-30 09:46:26 +01:00
SpinRelease(BufMgrLock);
1998-09-01 05:29:17 +02:00
return STATUS_OK;
}
#undef ReleaseAndReadBuffer
/*
* ReleaseAndReadBuffer -- combine ReleaseBuffer() and ReadBuffer()
* to save a spinlock release/acquire.
*
* An additional frammish of this routine is that the caller may perform
* file extension (as if blockNum = P_NEW) by passing the actual current
* EOF block number as blockNum and setting isExtend true. This hack
* allows us to avoid calling smgrnblocks() again when the caller has
* already done it.
*
* Note: it is OK to pass buffer = InvalidBuffer, indicating that no old
* buffer actually needs to be released. This case is the same as ReadBuffer
* except for the isExtend option.
*
* Also, if the passed buffer is valid and already contains the desired block
* number, we simply return it without ever acquiring the spinlock at all.
* Since the passed buffer must be pinned, it's OK to examine its block
* number without getting the lock first.
*/
Buffer
ReleaseAndReadBuffer(Buffer buffer,
Relation relation,
BlockNumber blockNum,
bool isExtend)
{
BufferDesc *bufHdr;
if (BufferIsValid(buffer))
{
if (BufferIsLocal(buffer))
{
Assert(LocalRefCount[-buffer - 1] > 0);
bufHdr = &LocalBufferDescriptors[-buffer - 1];
if (bufHdr->tag.blockNum == blockNum &&
RelFileNodeEquals(bufHdr->tag.rnode, relation->rd_node))
return buffer;
LocalRefCount[-buffer - 1]--;
}
else
{
Assert(PrivateRefCount[buffer - 1] > 0);
bufHdr = &BufferDescriptors[buffer - 1];
if (bufHdr->tag.blockNum == blockNum &&
RelFileNodeEquals(bufHdr->tag.rnode, relation->rd_node))
return buffer;
if (PrivateRefCount[buffer - 1] > 1)
PrivateRefCount[buffer - 1]--;
else
{
SpinAcquire(BufMgrLock);
PrivateRefCount[buffer - 1] = 0;
Assert(bufHdr->refcount > 0);
bufHdr->refcount--;
if (bufHdr->refcount == 0)
{
AddBufferToFreelist(bufHdr);
bufHdr->flags |= BM_FREE;
}
return ReadBufferInternal(relation, blockNum,
isExtend, true);
}
From: Dan McGuirk <mcguirk@indirect.com> Reply-To: hackers@hub.org, Dan McGuirk <mcguirk@indirect.com> To: hackers@hub.org Subject: [HACKERS] tmin writeback optimization I was doing some profiling of the backend, and noticed that during a certain benchmark I was running somewhere between 30% and 75% of the backend's CPU time was being spent in calls to TransactionIdDidCommit() from HeapTupleSatisfiesNow() or HeapTupleSatisfiesItself() to determine that changed rows' transactions had in fact been committed even though the rows' tmin values had not yet been set. When a query looks at a given row, it needs to figure out whether the transaction that changed the row has been committed and hence it should pay attention to the row, or whether on the other hand the transaction is still in progress or has been aborted and hence the row should be ignored. If a tmin value is set, it is known definitively that the row's transaction has been committed. However, if tmin is not set, the transaction referred to in xmin must be looked up in pg_log, and this is what the backend was spending a lot of time doing during my benchmark. So, implementing a method suggested by Vadim, I created the following patch that, the first time a query finds a committed row whose tmin value is not set, sets it, and marks the buffer where the row is stored as dirty. (It works for tmax, too.) This doesn't result in the boost in real time performance I was hoping for, however it does decrease backend CPU usage by up to two-thirds in certain situations, so it could be rather beneficial in high-concurrency settings.
1997-03-28 08:06:53 +01:00
}
}
return ReadBufferInternal(relation, blockNum,
isExtend, false);
}
/*
2000-11-30 09:46:26 +01:00
* BufferSync -- Write all dirty buffers in the pool.
*
2000-11-30 09:46:26 +01:00
* This is called at checkpoint time and write out all dirty buffers.
*/
2000-11-30 09:46:26 +01:00
void
BufferSync()
{
int i;
BufferDesc *bufHdr;
2000-11-30 09:46:26 +01:00
Buffer buffer;
int status;
2001-03-22 05:01:46 +01:00
RelFileNode rnode;
2000-11-30 09:46:26 +01:00
XLogRecPtr recptr;
Relation reln = NULL;
for (i = 0, bufHdr = BufferDescriptors; i < NBuffers; i++, bufHdr++)
{
SpinAcquire(BufMgrLock);
2000-11-30 09:46:26 +01:00
if (!(bufHdr->flags & BM_VALID))
{
2000-11-30 09:46:26 +01:00
SpinRelease(BufMgrLock);
continue;
}
2000-11-30 09:46:26 +01:00
/*
* We can check bufHdr->cntxDirty here *without* holding any lock
* on buffer context as long as we set this flag in access methods
* *before* logging changes with XLogInsert(): if someone will set
* cntxDirty just after our check we don't worry because of our
* checkpoint.redo points before log record for upcoming changes
* and so we are not required to write such dirty buffer.
*/
if (!(bufHdr->flags & BM_DIRTY) && !(bufHdr->cntxDirty))
{
SpinRelease(BufMgrLock);
continue;
}
/*
2001-03-22 05:01:46 +01:00
* IO synchronization. Note that we do it with unpinned buffer to
* avoid conflicts with FlushRelationBuffers.
2000-11-30 09:46:26 +01:00
*/
if (bufHdr->flags & BM_IO_IN_PROGRESS)
{
2000-11-30 09:46:26 +01:00
WaitIO(bufHdr, BufMgrLock);
if (!(bufHdr->flags & BM_VALID) ||
(!(bufHdr->flags & BM_DIRTY) && !(bufHdr->cntxDirty)))
{
SpinRelease(BufMgrLock);
continue;
}
}
/*
2001-03-22 05:01:46 +01:00
* Here: no one doing IO for this buffer and it's dirty. Pin
* buffer now and set IO state for it *before* acquiring shlock to
* avoid conflicts with FlushRelationBuffers.
*/
PinBuffer(bufHdr);
2001-03-22 05:01:46 +01:00
StartBufferIO(bufHdr, false); /* output IO start */
2000-11-30 09:46:26 +01:00
buffer = BufferDescriptorGetBuffer(bufHdr);
rnode = bufHdr->tag.rnode;
2000-11-30 09:46:26 +01:00
SpinRelease(BufMgrLock);
2000-11-30 09:46:26 +01:00
/*
* Try to find relation for buffer
*/
reln = RelationNodeCacheGetRelation(rnode);
2000-11-30 09:46:26 +01:00
/*
* Protect buffer content against concurrent update
*/
LockBuffer(buffer, BUFFER_LOCK_SHARE);
2000-11-30 09:46:26 +01:00
/*
* Force XLOG flush for buffer' LSN
*/
recptr = BufferGetLSN(bufHdr);
XLogFlush(recptr);
/*
* Now it's safe to write buffer to disk. Note that no one else
* should not be able to write it while we were busy with locking
* and log flushing because of we setted IO flag.
*/
2000-11-30 09:46:26 +01:00
SpinAcquire(BufMgrLock);
Assert(bufHdr->flags & BM_DIRTY || bufHdr->cntxDirty);
bufHdr->flags &= ~BM_JUST_DIRTIED;
SpinRelease(BufMgrLock);
2000-11-30 09:46:26 +01:00
if (reln == (Relation) NULL)
{
status = smgrblindwrt(DEFAULT_SMGR,
2001-03-22 05:01:46 +01:00
bufHdr->tag.rnode,
bufHdr->tag.blockNum,
(char *) MAKE_PTR(bufHdr->data),
true); /* must fsync */
}
else
{
status = smgrwrite(DEFAULT_SMGR, reln,
2001-03-22 05:01:46 +01:00
bufHdr->tag.blockNum,
(char *) MAKE_PTR(bufHdr->data));
}
if (status == SM_FAIL) /* disk failure ?! */
elog(STOP, "BufferSync: cannot write %u for %s",
bufHdr->tag.blockNum, bufHdr->blind.relname);
2000-11-30 09:46:26 +01:00
/*
2001-03-22 05:01:46 +01:00
* Note that it's safe to change cntxDirty here because of we
* protect it from upper writers by share lock and from other
* bufmgr routines by BM_IO_IN_PROGRESS
*/
bufHdr->cntxDirty = false;
2000-11-30 09:46:26 +01:00
/*
* Release the per-buffer readlock, reacquire BufMgrLock.
*/
LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
BufferFlushCount++;
SpinAcquire(BufMgrLock);
2000-11-30 09:46:26 +01:00
bufHdr->flags &= ~BM_IO_IN_PROGRESS; /* mark IO finished */
2001-03-22 05:01:46 +01:00
TerminateBufferIO(bufHdr); /* Sync IO finished */
2000-11-30 09:46:26 +01:00
/*
2001-03-22 05:01:46 +01:00
* If this buffer was marked by someone as DIRTY while we were
* flushing it out we must not clear DIRTY flag - vadim 01/17/97
*/
if (!(bufHdr->flags & BM_JUST_DIRTIED))
bufHdr->flags &= ~BM_DIRTY;
UnpinBuffer(bufHdr);
SpinRelease(BufMgrLock);
2000-11-30 09:46:26 +01:00
/* drop refcnt obtained by RelationNodeCacheGetRelation */
if (reln != (Relation) NULL)
RelationDecrementReferenceCount(reln);
}
2000-11-30 09:46:26 +01:00
}
/*
* WaitIO -- Block until the IO_IN_PROGRESS flag on 'buf' is cleared.
*
* Should be entered with buffer manager spinlock held; releases it before
* waiting and re-acquires it afterwards.
*/
static void
1997-09-08 22:59:27 +02:00
WaitIO(BufferDesc *buf, SPINLOCK spinlock)
{
/*
* Changed to wait until there's no IO - Inoue 01/13/2000
*/
while ((buf->flags & BM_IO_IN_PROGRESS) != 0)
{
SpinRelease(spinlock);
HOLD_INTERRUPTS(); /* don't want to die() holding the lock... */
S_LOCK(&(buf->io_in_progress_lock));
S_UNLOCK(&(buf->io_in_progress_lock));
RESUME_INTERRUPTS();
SpinAcquire(spinlock);
}
}
long NDirectFileRead; /* some I/O's are direct file access.
* bypass bufmgr */
long NDirectFileWrite; /* e.g., I/O in psort and hashjoin. */
void
PrintBufferUsage(FILE *statfp)
{
float hitrate;
float localhitrate;
if (ReadBufferCount == 0)
hitrate = 0.0;
else
hitrate = (float) BufferHitCount *100.0 / ReadBufferCount;
if (ReadLocalBufferCount == 0)
localhitrate = 0.0;
else
localhitrate = (float) LocalBufferHitCount *100.0 / ReadLocalBufferCount;
fprintf(statfp, "!\tShared blocks: %10ld read, %10ld written, buffer hit rate = %.2f%%\n",
ReadBufferCount - BufferHitCount, BufferFlushCount, hitrate);
fprintf(statfp, "!\tLocal blocks: %10ld read, %10ld written, buffer hit rate = %.2f%%\n",
ReadLocalBufferCount - LocalBufferHitCount, LocalBufferFlushCount, localhitrate);
fprintf(statfp, "!\tDirect blocks: %10ld read, %10ld written\n",
NDirectFileRead, NDirectFileWrite);
}
void
ResetBufferUsage()
{
BufferHitCount = 0;
ReadBufferCount = 0;
BufferFlushCount = 0;
LocalBufferHitCount = 0;
ReadLocalBufferCount = 0;
LocalBufferFlushCount = 0;
NDirectFileRead = 0;
NDirectFileWrite = 0;
}
/* ----------------------------------------------
* ResetBufferPool
*
* This routine is supposed to be called when a transaction aborts.
* It will release all the buffer pins held by the transaction.
* Currently, we also call it during commit if BufferPoolCheckLeak
* detected a problem --- in that case, isCommit is TRUE, and we
* only clean up buffer pin counts.
*
* ----------------------------------------------
*/
void
ResetBufferPool(bool isCommit)
{
int i;
for (i = 0; i < NBuffers; i++)
{
if (PrivateRefCount[i] != 0)
{
BufferDesc *buf = &BufferDescriptors[i];
SpinAcquire(BufMgrLock);
PrivateRefCount[i] = 0;
Assert(buf->refcount > 0);
buf->refcount--;
if (buf->refcount == 0)
{
AddBufferToFreelist(buf);
buf->flags |= BM_FREE;
}
SpinRelease(BufMgrLock);
}
}
ResetLocalBufferPool();
if (!isCommit)
smgrabort();
}
/* -----------------------------------------------
* BufferPoolCheckLeak
*
* check if there is buffer leak
*
* -----------------------------------------------
*/
int
BufferPoolCheckLeak()
{
int i;
int result = 0;
for (i = 1; i <= NBuffers; i++)
{
if (PrivateRefCount[i - 1] != 0)
{
BufferDesc *buf = &(BufferDescriptors[i - 1]);
elog(NOTICE,
"Buffer Leak: [%03d] (freeNext=%d, freePrev=%d, \
relname=%s, blockNum=%d, flags=0x%x, refcount=%d %ld)",
i - 1, buf->freeNext, buf->freePrev,
buf->blind.relname, buf->tag.blockNum, buf->flags,
buf->refcount, PrivateRefCount[i - 1]);
result = 1;
}
}
return result;
}
/* ------------------------------------------------
2000-11-30 09:46:26 +01:00
* FlushBufferPool
*
2000-11-30 09:46:26 +01:00
* Flush all dirty blocks in buffer pool to disk
* at the checkpoint time
* ------------------------------------------------
*/
void
1999-09-28 13:41:09 +02:00
FlushBufferPool(void)
{
1999-09-28 13:41:09 +02:00
BufferSync();
2000-11-30 09:46:26 +01:00
smgrsync();
}
/*
* At the commit time we have to flush local buffer pool only
*/
void
BufmgrCommit(void)
{
LocalBufferSync();
2001-03-22 05:01:46 +01:00
2000-11-30 09:46:26 +01:00
/*
* All files created in current transaction will be fsync-ed
*/
1999-09-28 13:41:09 +02:00
smgrcommit();
}
/*
1999-05-25 18:15:34 +02:00
* BufferGetBlockNumber
* Returns the block number associated with a buffer.
*
* Note:
* Assumes that the buffer is valid and pinned, else the
* value may be obsolete immediately...
*/
BlockNumber
BufferGetBlockNumber(Buffer buffer)
{
Assert(BufferIsValid(buffer));
if (BufferIsLocal(buffer))
1998-09-01 05:29:17 +02:00
return LocalBufferDescriptors[-buffer - 1].tag.blockNum;
else
1998-09-01 05:29:17 +02:00
return BufferDescriptors[buffer - 1].tag.blockNum;
}
/*
* BufferReplace
*
* Write out the buffer corresponding to 'bufHdr'
*
* BufMgrLock must be held at entry, and the buffer must be pinned.
*/
static int
BufferReplace(BufferDesc *bufHdr)
{
Relation reln;
2000-11-30 09:46:26 +01:00
XLogRecPtr recptr;
int status;
/* To check if block content changed while flushing. - vadim 01/17/97 */
bufHdr->flags &= ~BM_JUST_DIRTIED;
SpinRelease(BufMgrLock);
/*
2001-03-22 05:01:46 +01:00
* No need to lock buffer context - no one should be able to end
* ReadBuffer
*/
2000-11-30 09:46:26 +01:00
recptr = BufferGetLSN(bufHdr);
XLogFlush(recptr);
reln = RelationNodeCacheGetRelation(bufHdr->tag.rnode);
if (reln != (Relation) NULL)
{
status = smgrwrite(DEFAULT_SMGR, reln, bufHdr->tag.blockNum,
(char *) MAKE_PTR(bufHdr->data));
}
else
{
status = smgrblindwrt(DEFAULT_SMGR, bufHdr->tag.rnode,
bufHdr->tag.blockNum,
(char *) MAKE_PTR(bufHdr->data),
false); /* no fsync */
}
/* drop relcache refcnt incremented by RelationNodeCacheGetRelation */
if (reln != (Relation) NULL)
RelationDecrementReferenceCount(reln);
2000-11-30 09:46:26 +01:00
SpinAcquire(BufMgrLock);
if (status == SM_FAIL)
1998-09-01 05:29:17 +02:00
return FALSE;
BufferFlushCount++;
1998-09-01 05:29:17 +02:00
return TRUE;
}
/*
1999-05-25 18:15:34 +02:00
* RelationGetNumberOfBlocks
* Determines the current number of pages in the relation.
* Side effect: relation->rd_nblocks is updated.
*
* Note:
* XXX may fail for huge relations.
* XXX should be elsewhere.
* XXX maybe should be hidden
*/
BlockNumber
RelationGetNumberOfBlocks(Relation relation)
{
/*
* relation->rd_nblocks should be accurate already if the relation
* is myxactonly. (XXX how safe is that really?) Don't call smgr
* on a view, either.
*/
if (relation->rd_rel->relkind == RELKIND_VIEW)
relation->rd_nblocks = 0;
else if (!relation->rd_myxactonly)
relation->rd_nblocks = smgrnblocks(DEFAULT_SMGR, relation);
return relation->rd_nblocks;
}
/* ---------------------------------------------------------------------
* DropRelationBuffers
*
* This function removes all the buffered pages for a relation
* from the buffer pool. Dirty pages are simply dropped, without
2001-03-22 05:01:46 +01:00
* bothering to write them out first. This is NOT rollback-able,
* and so should be used only with extreme caution!
*
* We assume that the caller holds an exclusive lock on the relation,
* which should assure that no new buffers will be acquired for the rel
* meanwhile.
*
* XXX currently it sequentially searches the buffer pool, should be
* changed to more clever ways of searching.
* --------------------------------------------------------------------
*/
void
DropRelationBuffers(Relation rel)
{
int i;
BufferDesc *bufHdr;
if (rel->rd_myxactonly)
{
for (i = 0; i < NLocBuffer; i++)
{
bufHdr = &LocalBufferDescriptors[i];
if (RelFileNodeEquals(bufHdr->tag.rnode, rel->rd_node))
{
bufHdr->flags &= ~(BM_DIRTY | BM_JUST_DIRTIED);
2000-11-30 09:46:26 +01:00
bufHdr->cntxDirty = false;
LocalRefCount[i] = 0;
bufHdr->tag.rnode.relNode = InvalidOid;
}
}
return;
}
SpinAcquire(BufMgrLock);
for (i = 1; i <= NBuffers; i++)
{
bufHdr = &BufferDescriptors[i - 1];
recheck:
if (RelFileNodeEquals(bufHdr->tag.rnode, rel->rd_node))
{
/*
* If there is I/O in progress, better wait till it's done;
* don't want to delete the relation out from under someone
* who's just trying to flush the buffer!
*/
if (bufHdr->flags & BM_IO_IN_PROGRESS)
{
WaitIO(bufHdr, BufMgrLock);
/*
* By now, the buffer very possibly belongs to some other
* rel, so check again before proceeding.
*/
goto recheck;
}
/* Now we can do what we came for */
bufHdr->flags &= ~(BM_DIRTY | BM_JUST_DIRTIED);
2000-11-30 09:46:26 +01:00
bufHdr->cntxDirty = false;
/*
* Release any refcount we may have.
*
* This is very probably dead code, and if it isn't then it's
* probably wrong. I added the Assert to find out --- tgl
* 11/99.
*/
if (!(bufHdr->flags & BM_FREE))
{
/* Assert checks that buffer will actually get freed! */
Assert(PrivateRefCount[i - 1] == 1 &&
bufHdr->refcount == 1);
ReleaseBufferWithBufferLock(i);
}
2001-03-22 05:01:46 +01:00
/*
* And mark the buffer as no longer occupied by this rel.
*/
BufTableDelete(bufHdr);
}
}
SpinRelease(BufMgrLock);
}
/* ---------------------------------------------------------------------
* DropRelFileNodeBuffers
*
* This is the same as DropRelationBuffers, except that the target
* relation is specified by RelFileNode.
*
2001-03-22 05:01:46 +01:00
* This is NOT rollback-able. One legitimate use is to clear the
* buffer cache of buffers for a relation that is being deleted
* during transaction abort.
* --------------------------------------------------------------------
*/
void
DropRelFileNodeBuffers(RelFileNode rnode)
{
int i;
BufferDesc *bufHdr;
/* We have to search both local and shared buffers... */
for (i = 0; i < NLocBuffer; i++)
{
bufHdr = &LocalBufferDescriptors[i];
if (RelFileNodeEquals(bufHdr->tag.rnode, rnode))
{
bufHdr->flags &= ~(BM_DIRTY | BM_JUST_DIRTIED);
2000-11-30 09:46:26 +01:00
bufHdr->cntxDirty = false;
LocalRefCount[i] = 0;
bufHdr->tag.rnode.relNode = InvalidOid;
}
}
SpinAcquire(BufMgrLock);
for (i = 1; i <= NBuffers; i++)
{
bufHdr = &BufferDescriptors[i - 1];
recheck:
if (RelFileNodeEquals(bufHdr->tag.rnode, rnode))
{
/*
* If there is I/O in progress, better wait till it's done;
* don't want to delete the relation out from under someone
* who's just trying to flush the buffer!
*/
if (bufHdr->flags & BM_IO_IN_PROGRESS)
{
WaitIO(bufHdr, BufMgrLock);
/*
* By now, the buffer very possibly belongs to some other
* rel, so check again before proceeding.
*/
goto recheck;
}
/* Now we can do what we came for */
bufHdr->flags &= ~(BM_DIRTY | BM_JUST_DIRTIED);
2000-11-30 09:46:26 +01:00
bufHdr->cntxDirty = false;
/*
* Release any refcount we may have.
*
* This is very probably dead code, and if it isn't then it's
* probably wrong. I added the Assert to find out --- tgl
* 11/99.
*/
if (!(bufHdr->flags & BM_FREE))
{
/* Assert checks that buffer will actually get freed! */
Assert(PrivateRefCount[i - 1] == 1 &&
bufHdr->refcount == 1);
ReleaseBufferWithBufferLock(i);
}
2001-03-22 05:01:46 +01:00
/*
* And mark the buffer as no longer occupied by this rel.
*/
BufTableDelete(bufHdr);
}
}
SpinRelease(BufMgrLock);
}
/* ---------------------------------------------------------------------
* DropBuffers
*
* This function removes all the buffers in the buffer cache for a
* particular database. Dirty pages are simply dropped, without
2001-03-22 05:01:46 +01:00
* bothering to write them out first. This is used when we destroy a
* database, to avoid trying to flush data to disk when the directory
* tree no longer exists. Implementation is pretty similar to
* DropRelationBuffers() which is for destroying just one relation.
* --------------------------------------------------------------------
*/
void
DropBuffers(Oid dbid)
{
int i;
BufferDesc *bufHdr;
SpinAcquire(BufMgrLock);
for (i = 1; i <= NBuffers; i++)
{
bufHdr = &BufferDescriptors[i - 1];
recheck:
2001-03-22 05:01:46 +01:00
/*
2001-03-22 05:01:46 +01:00
* We know that currently database OID is tblNode but this
* probably will be changed in future and this func will be used
* to drop tablespace buffers.
*/
if (bufHdr->tag.rnode.tblNode == dbid)
{
/*
* If there is I/O in progress, better wait till it's done;
* don't want to delete the database out from under someone
* who's just trying to flush the buffer!
*/
if (bufHdr->flags & BM_IO_IN_PROGRESS)
{
WaitIO(bufHdr, BufMgrLock);
/*
* By now, the buffer very possibly belongs to some other
* DB, so check again before proceeding.
*/
goto recheck;
}
/* Now we can do what we came for */
bufHdr->flags &= ~(BM_DIRTY | BM_JUST_DIRTIED);
2000-11-30 09:46:26 +01:00
bufHdr->cntxDirty = false;
/*
* The thing should be free, if caller has checked that no
* backends are running in that database.
*/
Assert(bufHdr->flags & BM_FREE);
2001-03-22 05:01:46 +01:00
/*
* And mark the buffer as no longer occupied by this page.
*/
BufTableDelete(bufHdr);
}
}
SpinRelease(BufMgrLock);
}
/* -----------------------------------------------------------------
* PrintBufferDescs
*
* this function prints all the buffer descriptors, for debugging
* use only.
* -----------------------------------------------------------------
*/
void
PrintBufferDescs()
{
int i;
BufferDesc *buf = BufferDescriptors;
if (IsUnderPostmaster)
{
SpinAcquire(BufMgrLock);
for (i = 0; i < NBuffers; ++i, ++buf)
{
elog(DEBUG, "[%02d] (freeNext=%d, freePrev=%d, relname=%s, \
blockNum=%d, flags=0x%x, refcount=%d %ld)",
i, buf->freeNext, buf->freePrev,
buf->blind.relname, buf->tag.blockNum, buf->flags,
buf->refcount, PrivateRefCount[i]);
}
SpinRelease(BufMgrLock);
}
else
{
/* interactive backend */
for (i = 0; i < NBuffers; ++i, ++buf)
{
printf("[%-2d] (%s, %d) flags=0x%x, refcnt=%d %ld)\n",
2001-03-22 05:01:46 +01:00
i, buf->blind.relname, buf->tag.blockNum,
buf->flags, buf->refcount, PrivateRefCount[i]);
}
}
}
void
PrintPinnedBufs()
{
int i;
BufferDesc *buf = BufferDescriptors;
SpinAcquire(BufMgrLock);
for (i = 0; i < NBuffers; ++i, ++buf)
{
if (PrivateRefCount[i] > 0)
elog(NOTICE, "[%02d] (freeNext=%d, freePrev=%d, relname=%s, \
blockNum=%d, flags=0x%x, refcount=%d %ld)\n",
i, buf->freeNext, buf->freePrev, buf->blind.relname,
buf->tag.blockNum, buf->flags,
buf->refcount, PrivateRefCount[i]);
}
SpinRelease(BufMgrLock);
}
/*
* BufferPoolBlowaway
*
* this routine is solely for the purpose of experiments -- sometimes
* you may want to blowaway whatever is left from the past in buffer
* pool and start measuring some performance with a clean empty buffer
* pool.
*/
#ifdef NOT_USED
void
BufferPoolBlowaway()
{
int i;
BufferSync();
for (i = 1; i <= NBuffers; i++)
{
if (BufferIsValid(i))
{
while (BufferIsValid(i))
ReleaseBuffer(i);
}
BufTableDelete(&BufferDescriptors[i - 1]);
}
}
#endif
/* ---------------------------------------------------------------------
* FlushRelationBuffers
*
* This function writes all dirty pages of a relation out to disk.
* Furthermore, pages that have blocknumber >= firstDelBlock are
2001-03-22 05:01:46 +01:00
* actually removed from the buffer pool. An error code is returned
* if we fail to dump a dirty buffer or if we find one of
* the target pages is pinned into the cache.
*
* This is called by DROP TABLE to clear buffers for the relation
* from the buffer pool. Note that we must write dirty buffers,
* rather than just dropping the changes, because our transaction
* might abort later on; we want to roll back safely in that case.
*
* This is also called by VACUUM before truncating the relation to the
* given number of blocks. It might seem unnecessary for VACUUM to
* write dirty pages before firstDelBlock, since VACUUM should already
* have committed its changes. However, it is possible for there still
* to be dirty pages: if some page had unwritten on-row tuple status
* updates from a prior transaction, and VACUUM had no additional
* changes to make to that page, then VACUUM won't have written it.
* This is harmless in most cases but will break pg_upgrade, which
* relies on VACUUM to ensure that *all* tuples have correct on-row
* status. So, we check and flush all dirty pages of the rel
* regardless of block number.
*
* In all cases, the caller should be holding AccessExclusiveLock on
* the target relation to ensure that no other backend is busy reading
* more blocks of the relation (or might do so before we commit).
*
* Formerly, we considered it an error condition if we found dirty
* buffers here. However, since BufferSync no longer forces out all
* dirty buffers at every xact commit, it's possible for dirty buffers
* to still be present in the cache due to failure of an earlier
* transaction. So, must flush dirty buffers without complaint.
*
* Returns: 0 - Ok, -1 - FAILED TO WRITE DIRTY BUFFER, -2 - PINNED
*
* XXX currently it sequentially searches the buffer pool, should be
* changed to more clever ways of searching.
* --------------------------------------------------------------------
*/
int
FlushRelationBuffers(Relation rel, BlockNumber firstDelBlock)
{
int i;
BufferDesc *bufHdr;
2000-11-30 09:46:26 +01:00
XLogRecPtr recptr;
int status;
if (rel->rd_myxactonly)
{
for (i = 0; i < NLocBuffer; i++)
{
bufHdr = &LocalBufferDescriptors[i];
if (RelFileNodeEquals(bufHdr->tag.rnode, rel->rd_node))
{
2000-11-30 09:46:26 +01:00
if (bufHdr->flags & BM_DIRTY || bufHdr->cntxDirty)
{
2001-03-22 05:01:46 +01:00
status = smgrwrite(DEFAULT_SMGR, rel,
bufHdr->tag.blockNum,
(char *) MAKE_PTR(bufHdr->data));
2000-11-30 09:46:26 +01:00
if (status == SM_FAIL)
{
elog(NOTICE, "FlushRelationBuffers(%s (local), %u): block %u is dirty, could not flush it",
RelationGetRelationName(rel), firstDelBlock,
bufHdr->tag.blockNum);
2001-03-22 05:01:46 +01:00
return (-1);
}
2000-11-30 09:46:26 +01:00
bufHdr->flags &= ~(BM_DIRTY | BM_JUST_DIRTIED);
bufHdr->cntxDirty = false;
}
if (LocalRefCount[i] > 0)
{
elog(NOTICE, "FlushRelationBuffers(%s (local), %u): block %u is referenced (%ld)",
RelationGetRelationName(rel), firstDelBlock,
bufHdr->tag.blockNum, LocalRefCount[i]);
2001-03-22 05:01:46 +01:00
return (-2);
}
if (bufHdr->tag.blockNum >= firstDelBlock)
bufHdr->tag.rnode.relNode = InvalidOid;
}
}
1998-09-01 05:29:17 +02:00
return 0;
}
SpinAcquire(BufMgrLock);
for (i = 0; i < NBuffers; i++)
{
bufHdr = &BufferDescriptors[i];
if (RelFileNodeEquals(bufHdr->tag.rnode, rel->rd_node))
{
2000-11-30 09:46:26 +01:00
if (bufHdr->flags & BM_DIRTY || bufHdr->cntxDirty)
{
PinBuffer(bufHdr);
2000-11-30 09:46:26 +01:00
if (bufHdr->flags & BM_IO_IN_PROGRESS)
WaitIO(bufHdr, BufMgrLock);
SpinRelease(BufMgrLock);
2000-11-30 09:46:26 +01:00
/*
* Force XLOG flush for buffer' LSN
*/
recptr = BufferGetLSN(bufHdr);
XLogFlush(recptr);
/*
* Now it's safe to write buffer to disk
*/
SpinAcquire(BufMgrLock);
if (bufHdr->flags & BM_IO_IN_PROGRESS)
WaitIO(bufHdr, BufMgrLock);
if (bufHdr->flags & BM_DIRTY || bufHdr->cntxDirty)
{
2000-11-30 09:46:26 +01:00
bufHdr->flags &= ~BM_JUST_DIRTIED;
StartBufferIO(bufHdr, false); /* output IO start */
SpinRelease(BufMgrLock);
2000-11-30 09:46:26 +01:00
status = smgrwrite(DEFAULT_SMGR, rel,
2001-03-22 05:01:46 +01:00
bufHdr->tag.blockNum,
(char *) MAKE_PTR(bufHdr->data));
2000-11-30 09:46:26 +01:00
2001-03-22 05:01:46 +01:00
if (status == SM_FAIL) /* disk failure ?! */
2000-11-30 09:46:26 +01:00
elog(STOP, "FlushRelationBuffers: cannot write %u for %s",
bufHdr->tag.blockNum, bufHdr->blind.relname);
BufferFlushCount++;
SpinAcquire(BufMgrLock);
bufHdr->flags &= ~BM_IO_IN_PROGRESS;
TerminateBufferIO(bufHdr);
Assert(!(bufHdr->flags & BM_JUST_DIRTIED));
bufHdr->flags &= ~BM_DIRTY;
2001-03-22 05:01:46 +01:00
2000-11-30 09:46:26 +01:00
/*
2001-03-22 05:01:46 +01:00
* Note that it's safe to change cntxDirty here
* because of we protect it from upper writers by
2000-11-30 09:46:26 +01:00
* AccessExclusiveLock and from other bufmgr routines
* by BM_IO_IN_PROGRESS
*/
bufHdr->cntxDirty = false;
}
UnpinBuffer(bufHdr);
}
if (!(bufHdr->flags & BM_FREE))
{
SpinRelease(BufMgrLock);
elog(NOTICE, "FlushRelationBuffers(%s, %u): block %u is referenced (private %ld, global %d)",
RelationGetRelationName(rel), firstDelBlock,
bufHdr->tag.blockNum,
PrivateRefCount[i], bufHdr->refcount);
1998-09-01 05:29:17 +02:00
return -2;
}
if (bufHdr->tag.blockNum >= firstDelBlock)
BufTableDelete(bufHdr);
}
}
SpinRelease(BufMgrLock);
1998-09-01 05:29:17 +02:00
return 0;
}
#undef ReleaseBuffer
/*
* ReleaseBuffer -- remove the pin on a buffer without
* marking it dirty.
*
*/
int
ReleaseBuffer(Buffer buffer)
{
BufferDesc *bufHdr;
if (BufferIsLocal(buffer))
{
Assert(LocalRefCount[-buffer - 1] > 0);
LocalRefCount[-buffer - 1]--;
1998-09-01 05:29:17 +02:00
return STATUS_OK;
}
if (BAD_BUFFER_ID(buffer))
1998-09-01 05:29:17 +02:00
return STATUS_ERROR;
bufHdr = &BufferDescriptors[buffer - 1];
Assert(PrivateRefCount[buffer - 1] > 0);
if (PrivateRefCount[buffer - 1] > 1)
PrivateRefCount[buffer - 1]--;
else
{
SpinAcquire(BufMgrLock);
PrivateRefCount[buffer - 1] = 0;
Assert(bufHdr->refcount > 0);
bufHdr->refcount--;
if (bufHdr->refcount == 0)
{
AddBufferToFreelist(bufHdr);
bufHdr->flags |= BM_FREE;
}
SpinRelease(BufMgrLock);
From: Dan McGuirk <mcguirk@indirect.com> Reply-To: hackers@hub.org, Dan McGuirk <mcguirk@indirect.com> To: hackers@hub.org Subject: [HACKERS] tmin writeback optimization I was doing some profiling of the backend, and noticed that during a certain benchmark I was running somewhere between 30% and 75% of the backend's CPU time was being spent in calls to TransactionIdDidCommit() from HeapTupleSatisfiesNow() or HeapTupleSatisfiesItself() to determine that changed rows' transactions had in fact been committed even though the rows' tmin values had not yet been set. When a query looks at a given row, it needs to figure out whether the transaction that changed the row has been committed and hence it should pay attention to the row, or whether on the other hand the transaction is still in progress or has been aborted and hence the row should be ignored. If a tmin value is set, it is known definitively that the row's transaction has been committed. However, if tmin is not set, the transaction referred to in xmin must be looked up in pg_log, and this is what the backend was spending a lot of time doing during my benchmark. So, implementing a method suggested by Vadim, I created the following patch that, the first time a query finds a committed row whose tmin value is not set, sets it, and marks the buffer where the row is stored as dirty. (It works for tmax, too.) This doesn't result in the boost in real time performance I was hoping for, however it does decrease backend CPU usage by up to two-thirds in certain situations, so it could be rather beneficial in high-concurrency settings.
1997-03-28 08:06:53 +01:00
}
1998-09-01 05:29:17 +02:00
return STATUS_OK;
}
/*
* ReleaseBufferWithBufferLock
* Same as ReleaseBuffer except we hold the lock
*/
static int
ReleaseBufferWithBufferLock(Buffer buffer)
{
BufferDesc *bufHdr;
if (BufferIsLocal(buffer))
{
Assert(LocalRefCount[-buffer - 1] > 0);
LocalRefCount[-buffer - 1]--;
return STATUS_OK;
}
if (BAD_BUFFER_ID(buffer))
return STATUS_ERROR;
bufHdr = &BufferDescriptors[buffer - 1];
Assert(PrivateRefCount[buffer - 1] > 0);
if (PrivateRefCount[buffer - 1] > 1)
PrivateRefCount[buffer - 1]--;
else
{
PrivateRefCount[buffer - 1] = 0;
Assert(bufHdr->refcount > 0);
bufHdr->refcount--;
if (bufHdr->refcount == 0)
{
AddBufferToFreelist(bufHdr);
bufHdr->flags |= BM_FREE;
}
}
return STATUS_OK;
}
#ifdef NOT_USED
void
IncrBufferRefCount_Debug(char *file, int line, Buffer buffer)
{
IncrBufferRefCount(buffer);
if (ShowPinTrace && !BufferIsLocal(buffer) && is_userbuffer(buffer))
{
BufferDesc *buf = &BufferDescriptors[buffer - 1];
fprintf(stderr, "PIN(Incr) %d relname = %s, blockNum = %d, \
refcount = %ld, file: %s, line: %d\n",
buffer, buf->blind.relname, buf->tag.blockNum,
PrivateRefCount[buffer - 1], file, line);
}
}
#endif
#ifdef NOT_USED
void
ReleaseBuffer_Debug(char *file, int line, Buffer buffer)
{
ReleaseBuffer(buffer);
if (ShowPinTrace && !BufferIsLocal(buffer) && is_userbuffer(buffer))
{
BufferDesc *buf = &BufferDescriptors[buffer - 1];
fprintf(stderr, "UNPIN(Rel) %d relname = %s, blockNum = %d, \
refcount = %ld, file: %s, line: %d\n",
buffer, buf->blind.relname, buf->tag.blockNum,
PrivateRefCount[buffer - 1], file, line);
}
}
#endif
#ifdef NOT_USED
int
ReleaseAndReadBuffer_Debug(char *file,
int line,
Buffer buffer,
Relation relation,
BlockNumber blockNum,
bool isExtend)
{
bool bufferValid;
Buffer b;
bufferValid = BufferIsValid(buffer);
b = ReleaseAndReadBuffer(buffer, relation, blockNum, isExtend);
if (ShowPinTrace && bufferValid && BufferIsLocal(buffer)
&& is_userbuffer(buffer))
{
BufferDesc *buf = &BufferDescriptors[buffer - 1];
fprintf(stderr, "UNPIN(Rel&Rd) %d relname = %s, blockNum = %d, \
refcount = %ld, file: %s, line: %d\n",
buffer, buf->blind.relname, buf->tag.blockNum,
PrivateRefCount[buffer - 1], file, line);
}
if (ShowPinTrace && BufferIsLocal(buffer) && is_userbuffer(buffer))
{
BufferDesc *buf = &BufferDescriptors[b - 1];
fprintf(stderr, "PIN(Rel&Rd) %d relname = %s, blockNum = %d, \
refcount = %ld, file: %s, line: %d\n",
b, buf->blind.relname, buf->tag.blockNum,
PrivateRefCount[b - 1], file, line);
}
return b;
}
#endif
#ifdef BMTRACE
/*
* trace allocations and deallocations in a circular buffer in
* shared memory. check the buffer before doing the allocation,
* and die if there's anything fishy.
*/
_bm_trace(Oid dbId, Oid relId, int blkNo, int bufNo, int allocType)
{
long start,
cur;
bmtrace *tb;
start = *CurTraceBuf;
if (start > 0)
cur = start - 1;
else
cur = BMT_LIMIT - 1;
for (;;)
{
tb = &TraceBuf[cur];
if (tb->bmt_op != BMT_NOTUSED)
{
if (tb->bmt_buf == bufNo)
{
if ((tb->bmt_op == BMT_DEALLOC)
|| (tb->bmt_dbid == dbId && tb->bmt_relid == relId
&& tb->bmt_blkno == blkNo))
goto okay;
/* die holding the buffer lock */
_bm_die(dbId, relId, blkNo, bufNo, allocType, start, cur);
}
}
if (cur == start)
goto okay;
if (cur == 0)
cur = BMT_LIMIT - 1;
else
cur--;
}
okay:
tb = &TraceBuf[start];
1998-01-25 06:15:15 +01:00
tb->bmt_pid = MyProcPid;
tb->bmt_buf = bufNo;
tb->bmt_dbid = dbId;
tb->bmt_relid = relId;
tb->bmt_blkno = blkNo;
tb->bmt_op = allocType;
*CurTraceBuf = (start + 1) % BMT_LIMIT;
}
_bm_die(Oid dbId, Oid relId, int blkNo, int bufNo,
int allocType, long start, long cur)
{
FILE *fp;
bmtrace *tb;
int i;
tb = &TraceBuf[cur];
if ((fp = AllocateFile("/tmp/death_notice", "w")) == NULL)
elog(FATAL, "buffer alloc trace error and can't open log file");
fprintf(fp, "buffer alloc trace detected the following error:\n\n");
fprintf(fp, " buffer %d being %s inconsistently with a previous %s\n\n",
bufNo, (allocType == BMT_DEALLOC ? "deallocated" : "allocated"),
(tb->bmt_op == BMT_DEALLOC ? "deallocation" : "allocation"));
fprintf(fp, "the trace buffer contains:\n");
i = start;
for (;;)
{
tb = &TraceBuf[i];
if (tb->bmt_op != BMT_NOTUSED)
{
fprintf(fp, " [%3d]%spid %d buf %2d for <%d,%u,%d> ",
i, (i == cur ? " ---> " : "\t"),
tb->bmt_pid, tb->bmt_buf,
tb->bmt_dbid, tb->bmt_relid, tb->bmt_blkno);
switch (tb->bmt_op)
{
case BMT_ALLOCFND:
fprintf(fp, "allocate (found)\n");
break;
case BMT_ALLOCNOTFND:
fprintf(fp, "allocate (not found)\n");
break;
case BMT_DEALLOC:
fprintf(fp, "deallocate\n");
break;
default:
fprintf(fp, "unknown op type %d\n", tb->bmt_op);
break;
}
}
i = (i + 1) % BMT_LIMIT;
if (i == start)
break;
}
fprintf(fp, "\noperation causing error:\n");
fprintf(fp, "\tpid %d buf %d for <%d,%u,%d> ",
getpid(), bufNo, dbId, relId, blkNo);
switch (allocType)
{
case BMT_ALLOCFND:
fprintf(fp, "allocate (found)\n");
break;
case BMT_ALLOCNOTFND:
fprintf(fp, "allocate (not found)\n");
break;
case BMT_DEALLOC:
fprintf(fp, "deallocate\n");
break;
default:
fprintf(fp, "unknown op type %d\n", allocType);
break;
}
FreeFile(fp);
kill(getpid(), SIGILL);
}
#endif /* BMTRACE */
/*
* SetBufferCommitInfoNeedsSave
*
* Mark a buffer dirty when we have updated tuple commit-status bits in it.
*
* This is similar to WriteNoReleaseBuffer, except that we do not set
* SharedBufferChanged or BufferDirtiedByMe, because we have not made a
* critical change that has to be flushed to disk before xact commit --- the
* status-bit update could be redone by someone else just as easily. The
* buffer will be marked dirty, but it will not be written to disk until
* there is another reason to write it.
*
* This routine might get called many times on the same page, if we are making
* the first scan after commit of an xact that added/deleted many tuples.
* So, be as quick as we can if the buffer is already dirty.
*/
void
SetBufferCommitInfoNeedsSave(Buffer buffer)
From: Dan McGuirk <mcguirk@indirect.com> Reply-To: hackers@hub.org, Dan McGuirk <mcguirk@indirect.com> To: hackers@hub.org Subject: [HACKERS] tmin writeback optimization I was doing some profiling of the backend, and noticed that during a certain benchmark I was running somewhere between 30% and 75% of the backend's CPU time was being spent in calls to TransactionIdDidCommit() from HeapTupleSatisfiesNow() or HeapTupleSatisfiesItself() to determine that changed rows' transactions had in fact been committed even though the rows' tmin values had not yet been set. When a query looks at a given row, it needs to figure out whether the transaction that changed the row has been committed and hence it should pay attention to the row, or whether on the other hand the transaction is still in progress or has been aborted and hence the row should be ignored. If a tmin value is set, it is known definitively that the row's transaction has been committed. However, if tmin is not set, the transaction referred to in xmin must be looked up in pg_log, and this is what the backend was spending a lot of time doing during my benchmark. So, implementing a method suggested by Vadim, I created the following patch that, the first time a query finds a committed row whose tmin value is not set, sets it, and marks the buffer where the row is stored as dirty. (It works for tmax, too.) This doesn't result in the boost in real time performance I was hoping for, however it does decrease backend CPU usage by up to two-thirds in certain situations, so it could be rather beneficial in high-concurrency settings.
1997-03-28 08:06:53 +01:00
{
BufferDesc *bufHdr;
if (BufferIsLocal(buffer))
return;
if (BAD_BUFFER_ID(buffer))
return;
bufHdr = &BufferDescriptors[buffer - 1];
if ((bufHdr->flags & (BM_DIRTY | BM_JUST_DIRTIED)) !=
(BM_DIRTY | BM_JUST_DIRTIED))
{
SpinAcquire(BufMgrLock);
Assert(bufHdr->refcount > 0);
bufHdr->flags |= (BM_DIRTY | BM_JUST_DIRTIED);
SpinRelease(BufMgrLock);
}
From: Dan McGuirk <mcguirk@indirect.com> Reply-To: hackers@hub.org, Dan McGuirk <mcguirk@indirect.com> To: hackers@hub.org Subject: [HACKERS] tmin writeback optimization I was doing some profiling of the backend, and noticed that during a certain benchmark I was running somewhere between 30% and 75% of the backend's CPU time was being spent in calls to TransactionIdDidCommit() from HeapTupleSatisfiesNow() or HeapTupleSatisfiesItself() to determine that changed rows' transactions had in fact been committed even though the rows' tmin values had not yet been set. When a query looks at a given row, it needs to figure out whether the transaction that changed the row has been committed and hence it should pay attention to the row, or whether on the other hand the transaction is still in progress or has been aborted and hence the row should be ignored. If a tmin value is set, it is known definitively that the row's transaction has been committed. However, if tmin is not set, the transaction referred to in xmin must be looked up in pg_log, and this is what the backend was spending a lot of time doing during my benchmark. So, implementing a method suggested by Vadim, I created the following patch that, the first time a query finds a committed row whose tmin value is not set, sets it, and marks the buffer where the row is stored as dirty. (It works for tmax, too.) This doesn't result in the boost in real time performance I was hoping for, however it does decrease backend CPU usage by up to two-thirds in certain situations, so it could be rather beneficial in high-concurrency settings.
1997-03-28 08:06:53 +01:00
}
void
UnlockBuffers(void)
{
BufferDesc *buf;
int i;
for (i = 0; i < NBuffers; i++)
{
if (BufferLocks[i] == 0)
continue;
1999-05-25 18:15:34 +02:00
Assert(BufferIsValid(i + 1));
buf = &(BufferDescriptors[i]);
HOLD_INTERRUPTS(); /* don't want to die() holding the lock... */
S_LOCK(&(buf->cntx_lock));
if (BufferLocks[i] & BL_R_LOCK)
{
Assert(buf->r_locks > 0);
(buf->r_locks)--;
}
if (BufferLocks[i] & BL_RI_LOCK)
{
/*
* Someone else could remove our RI lock when acquiring W
* lock. This is possible if we came here from elog(ERROR)
* from IpcSemaphore{Lock|Unlock}(WaitCLSemId). And so we
* don't do Assert(buf->ri_lock) here.
*/
buf->ri_lock = false;
}
if (BufferLocks[i] & BL_W_LOCK)
{
Assert(buf->w_lock);
buf->w_lock = false;
}
S_UNLOCK(&(buf->cntx_lock));
BufferLocks[i] = 0;
RESUME_INTERRUPTS();
}
}
/* Max time to wait to acquire a buffer read or write lock */
#define BUFFER_LOCK_TIMEOUT (10*60*1000000) /* 10 minutes */
void
1999-05-25 18:15:34 +02:00
LockBuffer(Buffer buffer, int mode)
{
BufferDesc *buf;
bits8 *buflock;
Assert(BufferIsValid(buffer));
if (BufferIsLocal(buffer))
return;
1999-05-25 18:15:34 +02:00
buf = &(BufferDescriptors[buffer - 1]);
buflock = &(BufferLocks[buffer - 1]);
HOLD_INTERRUPTS(); /* don't want to die() holding the lock... */
1999-05-25 18:15:34 +02:00
S_LOCK(&(buf->cntx_lock));
if (mode == BUFFER_LOCK_UNLOCK)
{
if (*buflock & BL_R_LOCK)
{
Assert(buf->r_locks > 0);
Assert(!(buf->w_lock));
Assert(!(*buflock & (BL_W_LOCK | BL_RI_LOCK)));
(buf->r_locks)--;
*buflock &= ~BL_R_LOCK;
}
else if (*buflock & BL_W_LOCK)
{
Assert(buf->w_lock);
Assert(buf->r_locks == 0);
Assert(!(*buflock & (BL_R_LOCK | BL_RI_LOCK)));
buf->w_lock = false;
*buflock &= ~BL_W_LOCK;
}
else
{
S_UNLOCK(&(buf->cntx_lock));
RESUME_INTERRUPTS();
elog(ERROR, "UNLockBuffer: buffer %d is not locked", buffer);
}
}
else if (mode == BUFFER_LOCK_SHARE)
{
unsigned i = 0;
Assert(!(*buflock & (BL_R_LOCK | BL_W_LOCK | BL_RI_LOCK)));
while (buf->ri_lock || buf->w_lock)
{
S_UNLOCK(&(buf->cntx_lock));
RESUME_INTERRUPTS();
S_LOCK_SLEEP(&(buf->cntx_lock), i++, BUFFER_LOCK_TIMEOUT);
HOLD_INTERRUPTS();
S_LOCK(&(buf->cntx_lock));
}
(buf->r_locks)++;
*buflock |= BL_R_LOCK;
}
else if (mode == BUFFER_LOCK_EXCLUSIVE)
{
unsigned i = 0;
1999-05-25 18:15:34 +02:00
Assert(!(*buflock & (BL_R_LOCK | BL_W_LOCK | BL_RI_LOCK)));
while (buf->r_locks > 0 || buf->w_lock)
{
if (buf->r_locks > 3 || (*buflock & BL_RI_LOCK))
{
/*
* Our RI lock might be removed by concurrent W lock
* acquiring (see what we do with RI locks below when our
* own W acquiring succeeded) and so we set RI lock again
* if we already did this.
*/
*buflock |= BL_RI_LOCK;
buf->ri_lock = true;
}
S_UNLOCK(&(buf->cntx_lock));
RESUME_INTERRUPTS();
S_LOCK_SLEEP(&(buf->cntx_lock), i++, BUFFER_LOCK_TIMEOUT);
HOLD_INTERRUPTS();
S_LOCK(&(buf->cntx_lock));
}
buf->w_lock = true;
*buflock |= BL_W_LOCK;
2000-11-30 09:46:26 +01:00
/*
2001-03-22 05:01:46 +01:00
* This is not the best place to set cntxDirty flag (eg indices do
* not always change buffer they lock in excl mode). But please
* remember that it's critical to set cntxDirty *before* logging
* changes with XLogInsert() - see comments in BufferSync().
*/
2000-11-30 09:46:26 +01:00
buf->cntxDirty = true;
if (*buflock & BL_RI_LOCK)
{
/*
* It's possible to remove RI locks acquired by another W
* lockers here, but they'll take care about it.
*/
buf->ri_lock = false;
*buflock &= ~BL_RI_LOCK;
}
}
else
{
S_UNLOCK(&(buf->cntx_lock));
RESUME_INTERRUPTS();
elog(ERROR, "LockBuffer: unknown lock mode %d", mode);
}
1999-05-25 18:15:34 +02:00
S_UNLOCK(&(buf->cntx_lock));
RESUME_INTERRUPTS();
}
/*
* Functions for IO error handling
*
* Note : We assume that nested buffer IO never occur.
* i.e at most one io_in_progress spinlock is held
* per proc.
*/
static BufferDesc *InProgressBuf = (BufferDesc *) NULL;
static bool IsForInput;
/*
* Function:StartBufferIO
* (Assumptions)
* My process is executing no IO
* BufMgrLock is held
* BM_IO_IN_PROGRESS mask is not set for the buffer
* The buffer is Pinned
*
* Because BufMgrLock is held, we are already in an interrupt holdoff here,
* and do not need another.
*/
static void
StartBufferIO(BufferDesc *buf, bool forInput)
{
Assert(!InProgressBuf);
Assert(!(buf->flags & BM_IO_IN_PROGRESS));
buf->flags |= BM_IO_IN_PROGRESS;
/*
* There used to be
*
* Assert(S_LOCK_FREE(&(buf->io_in_progress_lock)));
*
* here, but that's wrong because of the way WaitIO works: someone else
* waiting for the I/O to complete will succeed in grabbing the lock
* for a few instructions, and if we context-swap back to here the
* Assert could fail. Tiny window for failure, but I've seen it
* happen -- tgl
*/
S_LOCK(&(buf->io_in_progress_lock));
InProgressBuf = buf;
IsForInput = forInput;
}
/*
* Function:TerminateBufferIO
* (Assumptions)
* My process is executing IO for the buffer
* BufMgrLock is held
* The buffer is Pinned
*
* Because BufMgrLock is held, we are already in an interrupt holdoff here,
* and do not need another.
*/
static void
TerminateBufferIO(BufferDesc *buf)
{
Assert(buf == InProgressBuf);
S_UNLOCK(&(buf->io_in_progress_lock));
InProgressBuf = (BufferDesc *) 0;
}
/*
* Function:ContinueBufferIO
* (Assumptions)
* My process is executing IO for the buffer
* BufMgrLock is held
* The buffer is Pinned
*
* Because BufMgrLock is held, we are already in an interrupt holdoff here,
* and do not need another.
*/
static void
ContinueBufferIO(BufferDesc *buf, bool forInput)
{
Assert(buf == InProgressBuf);
Assert(buf->flags & BM_IO_IN_PROGRESS);
IsForInput = forInput;
}
#ifdef NOT_USED
void
InitBufferIO(void)
{
InProgressBuf = (BufferDesc *) 0;
}
2001-03-22 05:01:46 +01:00
#endif
/*
* Clean up any active buffer I/O after an error.
* This function is called from ProcReleaseSpins().
* BufMgrLock isn't held when this function is called.
*
* If I/O was in progress, we always set BM_IO_ERROR.
*/
void
AbortBufferIO(void)
{
BufferDesc *buf = InProgressBuf;
if (buf)
{
SpinAcquire(BufMgrLock);
Assert(buf->flags & BM_IO_IN_PROGRESS);
if (IsForInput)
2000-11-30 09:46:26 +01:00
Assert(!(buf->flags & BM_DIRTY) && !(buf->cntxDirty));
else
{
2000-11-30 09:46:26 +01:00
Assert(buf->flags & BM_DIRTY || buf->cntxDirty);
/* Issue notice if this is not the first failure... */
2000-11-30 09:46:26 +01:00
if (buf->flags & BM_IO_ERROR)
{
elog(NOTICE, "write error may be permanent: cannot write block %u for %s/%s",
buf->tag.blockNum, buf->blind.dbname, buf->blind.relname);
}
buf->flags |= BM_DIRTY;
}
buf->flags |= BM_IO_ERROR;
buf->flags &= ~BM_IO_IN_PROGRESS;
TerminateBufferIO(buf);
SpinRelease(BufMgrLock);
}
}
/*
* Cleanup buffer or mark it for cleanup. Buffer may be cleaned
* up if it's pinned only once.
*
* NOTE: buffer must be excl locked.
*/
void
2001-03-22 05:01:46 +01:00
MarkBufferForCleanup(Buffer buffer, void (*CleanupFunc) (Buffer))
{
BufferDesc *bufHdr = &BufferDescriptors[buffer - 1];
Assert(PrivateRefCount[buffer - 1] > 0);
if (PrivateRefCount[buffer - 1] > 1)
{
LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
PrivateRefCount[buffer - 1]--;
SpinAcquire(BufMgrLock);
Assert(bufHdr->refcount > 0);
bufHdr->flags |= (BM_DIRTY | BM_JUST_DIRTIED);
bufHdr->CleanupFunc = CleanupFunc;
SpinRelease(BufMgrLock);
return;
}
SpinAcquire(BufMgrLock);
Assert(bufHdr->refcount > 0);
if (bufHdr->refcount == 1)
{
SpinRelease(BufMgrLock);
CleanupFunc(buffer);
CleanupFunc = NULL;
}
else
SpinRelease(BufMgrLock);
LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
SpinAcquire(BufMgrLock);
PrivateRefCount[buffer - 1] = 0;
Assert(bufHdr->refcount > 0);
bufHdr->flags |= (BM_DIRTY | BM_JUST_DIRTIED);
bufHdr->CleanupFunc = CleanupFunc;
bufHdr->refcount--;
if (bufHdr->refcount == 0)
{
AddBufferToFreelist(bufHdr);
bufHdr->flags |= BM_FREE;
}
SpinRelease(BufMgrLock);
return;
}
RelFileNode
BufferGetFileNode(Buffer buffer)
{
BufferDesc *bufHdr;
if (BufferIsLocal(buffer))
bufHdr = &(LocalBufferDescriptors[-buffer - 1]);
else
bufHdr = &BufferDescriptors[buffer - 1];
2001-03-22 05:01:46 +01:00
return (bufHdr->tag.rnode);
}