*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/storage/buffer/bufmgr.c,v 1.161 2004/04/19 23:27:17 tgl Exp $
+ * $PostgreSQL: pgsql/src/backend/storage/buffer/bufmgr.c,v 1.162 2004/04/21 18:06:30 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.
+ * ReadBuffer() -- find or create a buffer holding the requested page,
+ * and pin it so that no one can destroy it while this process
+ * is using it.
*
* ReleaseBuffer() -- unpin the buffer
*
*
* WriteBuffer() -- WriteNoReleaseBuffer() + ReleaseBuffer()
*
- * BufferSync() -- flush all dirty buffers in the buffer pool.
+ * BufferSync() -- flush all (or some) dirty buffers in the buffer pool.
*
* InitBufferPool() -- Init the buffer module.
*
* bypass bufmgr */
long NDirectFileWrite; /* e.g., I/O in psort and hashjoin. */
-/*
- * 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 void PinBuffer(BufferDesc *buf);
static void UnpinBuffer(BufferDesc *buf);
static void WaitIO(BufferDesc *buf);
static void StartBufferIO(BufferDesc *buf, bool forInput);
-static void TerminateBufferIO(BufferDesc *buf);
+static void TerminateBufferIO(BufferDesc *buf, int err_flag);
static void ContinueBufferIO(BufferDesc *buf, bool forInput);
static void buffer_write_error_callback(void *arg);
static Buffer ReadBufferInternal(Relation reln, BlockNumber blockNum,
bool bufferLockHeld);
static BufferDesc *BufferAlloc(Relation reln, BlockNumber blockNum,
bool *foundPtr);
-static void BufferReplace(BufferDesc *bufHdr);
+static void FlushBuffer(BufferDesc *buf, SMgrRelation reln);
static void write_buffer(Buffer buffer, bool unpin);
* relation at the same time!)
*
* Returns: the buffer number for the buffer containing
- * the block read, or NULL on an error. If successful,
- * the returned buffer has been pinned.
+ * the block read. The returned buffer has been pinned.
+ * Does not return on error --- elog's instead.
*
* Assume when this function is called, that reln has been
* opened already.
pgstat_count_buffer_read(&reln->pgstat_info, reln);
/* Substitute proper block number if caller asked for P_NEW */
if (isExtend)
- {
blockNum = reln->rd_nblocks;
- reln->rd_nblocks++;
- }
+
bufHdr = LocalBufferAlloc(reln, blockNum, &found);
if (found)
- {
LocalBufferHitCount++;
- pgstat_count_buffer_hit(&reln->pgstat_info, reln);
- }
}
else
{
{
/* must be sure we have accurate file length! */
blockNum = reln->rd_nblocks = smgrnblocks(reln->rd_smgr);
- reln->rd_nblocks++;
}
/*
LWLockAcquire(BufMgrLock, LW_EXCLUSIVE);
bufHdr = BufferAlloc(reln, blockNum, &found);
if (found)
- {
BufferHitCount++;
- pgstat_count_buffer_hit(&reln->pgstat_info, reln);
- }
}
/* At this point we do NOT hold the bufmgr lock. */
- if (!bufHdr)
- return InvalidBuffer;
-
- /* if it's already in the buffer pool, we're done */
+ /* if it was already in the buffer pool, we're done */
if (found)
{
- /* That is, we're done if we expected to be able to find it ... */
- if (!isExtend)
- return BufferDescriptorGetBuffer(bufHdr);
+ /* Just need to update stats before we exit */
+ pgstat_count_buffer_hit(&reln->pgstat_info, reln);
- /*
- * If we found a buffer when we were expecting to extend the
- * relation, the implication is that a buffer was already created
- * for the next page position, but then smgrextend failed to write
- * the page. We'd better try the smgrextend again. But since
- * BufferAlloc won't have done StartBufferIO, we must do that
- * first.
- */
- if (!isLocalBuf)
- {
- LWLockAcquire(BufMgrLock, LW_EXCLUSIVE);
- StartBufferIO(bufHdr, false);
- LWLockRelease(BufMgrLock);
- }
+ if (VacuumCostActive)
+ VacuumCostBalance += VacuumCostPageHit;
+
+ return BufferDescriptorGetBuffer(bufHdr);
}
/*
- * if we have gotten to this point, the relation must be open in the smgr.
+ * if we have gotten to this point, we have allocated a buffer for the
+ * page but its contents are not yet valid. IO_IN_PROGRESS is set for
+ * it, if it's a shared buffer.
+ *
+ * Note: if smgrextend fails, we will end up with a buffer that is
+ * allocated but not marked BM_VALID. P_NEW will still select the same
+ * block number (because the relation didn't get any longer on disk)
+ * and so future attempts to extend the relation will find the same
+ * buffer (if it's not been recycled) but come right back here to try
+ * smgrextend again.
*/
+ Assert(!(bufHdr->flags & BM_VALID));
+
if (isExtend)
{
/* new buffers are zero-filled */
MemSet((char *) MAKE_PTR(bufHdr->data), 0, BLCKSZ);
smgrextend(reln->rd_smgr, blockNum, (char *) MAKE_PTR(bufHdr->data));
+ /* After successful extend, increment relation length */
+ reln->rd_nblocks++;
}
else
{
if (isLocalBuf)
{
- /* No shared buffer state to update... */
- return BufferDescriptorGetBuffer(bufHdr);
+ /* Only need to adjust flags */
+ bufHdr->flags |= BM_VALID;
}
+ else
+ {
+ /* lock buffer manager again to update IO IN PROGRESS */
+ LWLockAcquire(BufMgrLock, LW_EXCLUSIVE);
- /* lock buffer manager again to update IO IN PROGRESS */
- LWLockAcquire(BufMgrLock, LW_EXCLUSIVE);
+ /* IO Succeeded, so mark data valid */
+ bufHdr->flags |= BM_VALID;
- /* 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, 0);
- /* If anyone was waiting for IO to complete, wake them up now */
- TerminateBufferIO(bufHdr);
+ LWLockRelease(BufMgrLock);
+ }
- LWLockRelease(BufMgrLock);
+ if (VacuumCostActive)
+ VacuumCostBalance += VacuumCostPageMiss;
return BufferDescriptorGetBuffer(bufHdr);
}
/*
- * BufferAlloc -- Get a buffer from the buffer pool but don't
- * read it. If successful, the returned buffer is pinned.
+ * BufferAlloc -- subroutine for ReadBuffer. Handles lookup of a shared
+ * buffer. If no buffer exists already, selects a replacement
+ * victim and evicts the old page, but does NOT read in new page.
*
- * Returns: descriptor for buffer
+ * The returned buffer is pinned and is already marked as holding the
+ * desired page. If it already did have the desired page, *foundPtr is
+ * set TRUE. Otherwise, *foundPtr is set FALSE and the buffer is marked
+ * as IO_IN_PROGRESS; ReadBuffer will now need to do I/O to fill it.
+ *
+ * *foundPtr is actually redundant with the buffer's BM_VALID flag, but
+ * we keep it for simplicity in ReadBuffer.
*
* BufMgrLock must be held at entry. When this routine returns,
* the BufMgrLock is guaranteed NOT to be held.
BlockNumber blockNum,
bool *foundPtr)
{
+ BufferTag newTag; /* identity of requested block */
BufferDesc *buf,
*buf2;
- BufferTag newTag; /* identity of requested block */
int cdb_found_index,
cdb_replace_index;
- bool inProgress; /* buffer undergoing IO */
+ bool inProgress; /* did we already do StartBufferIO? */
/* create a tag so we can lookup the buffer */
- INIT_BUFFERTAG(&newTag, reln, blockNum);
+ INIT_BUFFERTAG(newTag, reln, blockNum);
/* see if the block is in the buffer pool already */
buf = StrategyBufferLookup(&newTag, false, &cdb_found_index);
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.
+ * Found it. Now, pin the buffer so no one can steal it from the
+ * buffer pool, and check to see if someone else is still reading
+ * data into the buffer. (Formerly, we'd always block here if
+ * IO_IN_PROGRESS is set, but there's no need to wait when someone
+ * is writing rather than reading.)
*/
+ *foundPtr = TRUE;
PinBuffer(buf);
- inProgress = (buf->flags & BM_IO_IN_PROGRESS);
- *foundPtr = TRUE;
- if (inProgress) /* confirm end of IO */
- {
- WaitIO(buf);
- inProgress = (buf->flags & BM_IO_IN_PROGRESS);
- }
- if (BUFFER_IS_BROKEN(buf))
+ if (!(buf->flags & BM_VALID))
{
- /*
- * 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;
- StartBufferIO(buf, true);
+ if (buf->flags & BM_IO_IN_PROGRESS)
+ {
+ /* someone else is reading it, wait for them */
+ WaitIO(buf);
+ }
+ if (!(buf->flags & BM_VALID))
+ {
+ /*
+ * If we get here, previous attempts to read the buffer
+ * must have failed ... but we shall bravely try again.
+ */
+ *foundPtr = FALSE;
+ StartBufferIO(buf, true);
+ }
}
LWLockRelease(BufMgrLock);
- /*
- * Do the cost accounting for vacuum
- */
- if (VacuumCostActive)
- VacuumCostBalance += VacuumCostPageHit;
-
return buf;
}
/*
* 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 BufMgrLock while doing the IOs.
+ * it to disk. Remember to unlock BufMgrLock while doing the IO.
*/
inProgress = FALSE;
- for (buf = NULL; buf == NULL;)
+ do
{
buf = StrategyGetBuffer(&cdb_replace_index);
buf->refcount = 1;
PrivateRefCount[BufferDescriptorGetBuffer(buf) - 1] = 1;
- if (buf->flags & BM_DIRTY || buf->cntxDirty)
+ if ((buf->flags & BM_VALID) &&
+ (buf->flags & BM_DIRTY || buf->cntxDirty))
{
/*
- * skip write error buffers
+ * Set BM_IO_IN_PROGRESS to show the buffer is being written.
+ * It cannot already be set because the buffer would be pinned
+ * if someone were writing it.
+ *
+ * Note: it's okay to grab the io_in_progress lock while holding
+ * BufMgrLock. 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 the lock.
*/
- if ((buf->flags & BM_IO_ERROR) != 0)
- {
- UnpinBuffer(buf);
- buf = NULL;
- continue;
- }
+ StartBufferIO(buf, false);
- /*
- * 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.
- */
- BufferReplace(buf);
-
- /*
- * BM_JUST_DIRTIED cleared by BufferReplace and shouldn't
- * be set by anyone. - vadim 01/17/97
- */
- if (buf->flags & BM_JUST_DIRTIED)
- {
- elog(PANIC, "content of block %u of %u/%u changed while flushing",
- buf->tag.blockNum,
- buf->tag.rnode.tblNode, buf->tag.rnode.relNode);
- }
-
- buf->flags &= ~BM_DIRTY;
- 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.
+ * while doing the I/O.
*/
- if (buf && buf->refcount > 1)
- {
- inProgress = FALSE;
- buf->flags &= ~BM_IO_IN_PROGRESS;
- TerminateBufferIO(buf);
- UnpinBuffer(buf);
- buf = NULL;
- }
+ FlushBuffer(buf, NULL);
/*
* Somebody could have allocated another buffer for the same
- * block we are about to read in. (While we flush out the
+ * 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
+ * we haven't yet inserted the new tag into the buffer table.
+ * So we need to check here. -ay 3/95
+ *
+ * Another reason we have to do this is to update cdb_found_index,
+ * since the CDB could have disappeared from B1/B2 list while
+ * we were writing.
*/
buf2 = StrategyBufferLookup(&newTag, true, &cdb_found_index);
if (buf2 != NULL)
{
/*
- * Found it. Someone has already done what we're about to
+ * Found it. Someone has already done what we were about to
* do. We'll just handle this as if it were found in the
- * buffer pool in the first place.
+ * buffer pool in the first place. First, give up the
+ * buffer we were planning to use.
*/
- if (buf != NULL)
- {
- buf->flags &= ~BM_IO_IN_PROGRESS;
- TerminateBufferIO(buf);
- /* give up old buffer since we don't need it any more */
- UnpinBuffer(buf);
- }
+ TerminateBufferIO(buf, 0);
+ UnpinBuffer(buf);
- PinBuffer(buf2);
- inProgress = (buf2->flags & BM_IO_IN_PROGRESS);
+ buf = buf2;
+
+ /* remaining code should match code at top of routine */
*foundPtr = TRUE;
- if (inProgress)
- {
- WaitIO(buf2);
- inProgress = (buf2->flags & BM_IO_IN_PROGRESS);
- }
- if (BUFFER_IS_BROKEN(buf2))
+ PinBuffer(buf);
+
+ if (!(buf->flags & BM_VALID))
{
- *foundPtr = FALSE;
- StartBufferIO(buf2, true);
+ if (buf->flags & BM_IO_IN_PROGRESS)
+ {
+ /* someone else is reading it, wait for them */
+ WaitIO(buf);
+ }
+ if (!(buf->flags & BM_VALID))
+ {
+ /*
+ * If we get here, previous attempts to read the buffer
+ * must have failed ... but we shall bravely try again.
+ */
+ *foundPtr = FALSE;
+ StartBufferIO(buf, true);
+ }
}
LWLockRelease(BufMgrLock);
- /*
- * Do the cost accounting for vacuum. (XXX perhaps better
- * to consider this a miss? We didn't have to do the read,
- * but we did have to write ...)
- */
- if (VacuumCostActive)
- VacuumCostBalance += VacuumCostPageHit;
+ return buf;
+ }
- return buf2;
+ /*
+ * Somebody could have pinned the buffer while we were doing
+ * the I/O and had given up the BufMgrLock. If so, we can't
+ * recycle this buffer --- we need to clear the I/O flags,
+ * remove our pin and choose a new victim buffer. Similarly,
+ * we have to start over if somebody re-dirtied the buffer.
+ */
+ if (buf->refcount > 1 || buf->flags & BM_DIRTY || buf->cntxDirty)
+ {
+ TerminateBufferIO(buf, 0);
+ UnpinBuffer(buf);
+ inProgress = FALSE;
+ buf = NULL;
}
}
- }
+ } while (buf == NULL);
/*
* At this point we should have the sole pin on a non-dirty buffer and
/*
* Tell the buffer replacement strategy that we are replacing the
- * buffer content. Then rename the buffer.
+ * buffer content. Then rename the buffer. Clearing BM_VALID here
+ * is necessary, clearing the dirtybits is just paranoia.
*/
StrategyReplaceBuffer(buf, &newTag, cdb_found_index, cdb_replace_index);
buf->tag = newTag;
+ buf->flags &= ~(BM_VALID | BM_DIRTY | BM_JUST_DIRTIED | BM_IO_ERROR);
+ buf->cntxDirty = false;
/*
* 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.
+ * so no one fiddles with them until the read completes. We may have
+ * already marked it, in which case we just flip from write to read
+ * status.
*/
if (!inProgress)
StartBufferIO(buf, true);
LWLockRelease(BufMgrLock);
- /*
- * Do the cost accounting for vacuum
- */
- if (VacuumCostActive)
- VacuumCostBalance += VacuumCostPageMiss;
-
return buf;
}
bufHdr = &BufferDescriptors[buffer - 1];
+ Assert(PrivateRefCount[buffer - 1] > 0);
+
LWLockAcquire(BufMgrLock, LW_EXCLUSIVE);
Assert(bufHdr->refcount > 0);
/*
- * If the buffer is not dirty yet, do vacuum cost accounting.
+ * If the buffer was not dirty already, do vacuum cost accounting.
*/
if (!(bufHdr->flags & BM_DIRTY) && VacuumCostActive)
VacuumCostBalance += VacuumCostPageDirty;
BufferTag *buftags;
int num_buffer_dirty;
int i;
- ErrorContextCallback errcontext;
/*
* Get a list of all currently dirty buffers and how many there are.
if (maxpages > 0 && num_buffer_dirty > maxpages)
num_buffer_dirty = maxpages;
- /* Setup error traceback support for ereport() */
- errcontext.callback = buffer_write_error_callback;
- errcontext.arg = NULL;
- errcontext.previous = error_context_stack;
- error_context_stack = &errcontext;
-
/*
* Loop over buffers to be written. Note the BufMgrLock is held at
- * loop top, but is released and reacquired intraloop, so we aren't
- * holding it long.
+ * loop top, but is released and reacquired within FlushBuffer,
+ * so we aren't holding it long.
*/
for (i = 0; i < num_buffer_dirty; i++)
{
BufferDesc *bufHdr = dirty_buffers[i];
- Buffer buffer;
- XLogRecPtr recptr;
- SMgrRelation reln;
-
- errcontext.arg = bufHdr;
/*
* Check it is still the same page and still needs writing.
*/
if (!(bufHdr->flags & BM_VALID))
continue;
- if (!BUFFERTAGS_EQUAL(&bufHdr->tag, &buftags[i]))
+ if (!BUFFERTAGS_EQUAL(bufHdr->tag, buftags[i]))
continue;
- if (!(bufHdr->flags & BM_DIRTY) && !(bufHdr->cntxDirty))
+ if (!(bufHdr->flags & BM_DIRTY || bufHdr->cntxDirty))
continue;
/*
/* Still need writing? */
if (!(bufHdr->flags & BM_VALID))
continue;
- if (!BUFFERTAGS_EQUAL(&bufHdr->tag, &buftags[i]))
+ if (!BUFFERTAGS_EQUAL(bufHdr->tag, buftags[i]))
continue;
- if (!(bufHdr->flags & BM_DIRTY) && !(bufHdr->cntxDirty))
+ if (!(bufHdr->flags & BM_DIRTY || bufHdr->cntxDirty))
continue;
}
* avoid conflicts with FlushRelationBuffers.
*/
PinBuffer(bufHdr);
- StartBufferIO(bufHdr, false); /* output IO start */
-
- /* Release BufMgrLock while doing xlog work */
- LWLockRelease(BufMgrLock);
-
- buffer = BufferDescriptorGetBuffer(bufHdr);
-
- /*
- * Protect buffer content against concurrent update
- */
- LockBuffer(buffer, BUFFER_LOCK_SHARE);
-
- /*
- * 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 have been able to write it while we were busy with
- * locking and log flushing because we set the IO flag.
- *
- * Before we issue the actual write command, clear the just-dirtied
- * flag. This lets us recognize concurrent changes (note that only
- * hint-bit changes are possible since we hold the buffer shlock).
- */
- LWLockAcquire(BufMgrLock, LW_EXCLUSIVE);
- Assert(bufHdr->flags & BM_DIRTY || bufHdr->cntxDirty);
- bufHdr->flags &= ~BM_JUST_DIRTIED;
- LWLockRelease(BufMgrLock);
-
- /* Find smgr relation for buffer */
- reln = smgropen(bufHdr->tag.rnode);
-
- /* And write... */
- smgrwrite(reln,
- bufHdr->tag.blockNum,
- (char *) MAKE_PTR(bufHdr->data));
-
- /*
- * 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;
+ StartBufferIO(bufHdr, false);
- /*
- * Release the per-buffer readlock, reacquire BufMgrLock.
- */
- LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
+ FlushBuffer(bufHdr, NULL);
- LWLockAcquire(BufMgrLock, LW_EXCLUSIVE);
-
- bufHdr->flags &= ~BM_IO_IN_PROGRESS; /* mark IO finished */
- TerminateBufferIO(bufHdr); /* Sync IO finished */
- BufferFlushCount++;
-
- /*
- * 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;
+ TerminateBufferIO(bufHdr, 0);
UnpinBuffer(bufHdr);
}
LWLockRelease(BufMgrLock);
- /* Pop the error context stack */
- error_context_stack = errcontext.previous;
-
pfree(dirty_buffers);
pfree(buftags);
}
/*
- * BufferReplace
+ * FlushBuffer
+ * Physically write out a shared buffer.
*
- * Write out the buffer corresponding to 'bufHdr'.
+ * NOTE: this actually just passes the buffer contents to the kernel; the
+ * real write to disk won't happen until the kernel feels like it. This
+ * is okay from our point of view since we can redo the changes from WAL.
+ * However, we will need to force the changes to disk via sync/fsync
+ * before we can checkpoint WAL.
*
- * BufMgrLock must be held at entry, and the buffer must be pinned.
+ * BufMgrLock must be held at entry, and the buffer must be pinned. The
+ * caller is also responsible for doing StartBufferIO/TerminateBufferIO.
+ *
+ * If the caller has an smgr reference for the buffer's relation, pass it
+ * as the second parameter. If not, pass NULL. (Do not open relation
+ * while holding BufMgrLock!)
*/
static void
-BufferReplace(BufferDesc *bufHdr)
+FlushBuffer(BufferDesc *buf, SMgrRelation reln)
{
- SMgrRelation reln;
+ Buffer buffer;
XLogRecPtr recptr;
ErrorContextCallback errcontext;
+ /* Transpose cntxDirty into flags while holding BufMgrLock */
+ buf->cntxDirty = false;
+ buf->flags |= BM_DIRTY;
+
/* To check if block content changed while flushing. - vadim 01/17/97 */
- bufHdr->flags &= ~BM_JUST_DIRTIED;
+ buf->flags &= ~BM_JUST_DIRTIED;
+ /* Release BufMgrLock while doing xlog work */
LWLockRelease(BufMgrLock);
/* Setup error traceback support for ereport() */
errcontext.callback = buffer_write_error_callback;
- errcontext.arg = bufHdr;
+ errcontext.arg = buf;
errcontext.previous = error_context_stack;
error_context_stack = &errcontext;
+ /* Find smgr relation for buffer while holding minimal locks */
+ if (reln == NULL)
+ reln = smgropen(buf->tag.rnode);
+
+ buffer = BufferDescriptorGetBuffer(buf);
+
/*
- * No need to lock buffer context - no one should be able to end
- * ReadBuffer
+ * Protect buffer content against concurrent update. (Note that
+ * hint-bit updates can still occur while the write is in progress,
+ * but we assume that that will not invalidate the data written.)
*/
- recptr = BufferGetLSN(bufHdr);
- XLogFlush(recptr);
+ LockBuffer(buffer, BUFFER_LOCK_SHARE);
- /* Find smgr relation for buffer */
- reln = smgropen(bufHdr->tag.rnode);
+ /*
+ * Force XLOG flush for buffer' LSN. This implements the basic WAL
+ * rule that log updates must hit disk before any of the data-file
+ * changes they describe do.
+ */
+ recptr = BufferGetLSN(buf);
+ XLogFlush(recptr);
- /* And write... */
+ /*
+ * Now it's safe to write buffer to disk. Note that no one else
+ * should have been able to write it while we were busy with
+ * locking and log flushing because caller has set the IO flag.
+ *
+ * It would be better to clear BM_JUST_DIRTIED right here, but we'd
+ * have to reacquire the BufMgrLock and it doesn't seem worth it.
+ */
smgrwrite(reln,
- bufHdr->tag.blockNum,
- (char *) MAKE_PTR(bufHdr->data));
+ buf->tag.blockNum,
+ (char *) MAKE_PTR(buf->data));
/* Pop the error context stack */
error_context_stack = errcontext.previous;
+ /*
+ * Release the per-buffer readlock, reacquire BufMgrLock.
+ */
+ LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
+
LWLockAcquire(BufMgrLock, LW_EXCLUSIVE);
BufferFlushCount++;
+
+ /*
+ * 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 (!(buf->flags & BM_JUST_DIRTIED))
+ buf->flags &= ~BM_DIRTY;
}
/*
* 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).
+ * This should also ensure that no one is busy dirtying these blocks.
*
* Formerly, we considered it an error condition if we found dirty
* buffers here. However, since BufferSync no longer forces out all
* 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
+ * Returns: 0 - Ok, -1 - FAILED TO CLEAR DIRTY BIT, -2 - PINNED
*
* XXX currently it sequentially searches the buffer pool, should be
* changed to more clever ways of searching.
{
int i;
BufferDesc *bufHdr;
- XLogRecPtr recptr;
- ErrorContextCallback errcontext;
-
- /* Setup error traceback support for ereport() */
- errcontext.callback = buffer_write_error_callback;
- errcontext.arg = NULL;
- errcontext.previous = error_context_stack;
- error_context_stack = &errcontext;
if (rel->rd_istemp)
{
for (i = 0; i < NLocBuffer; i++)
{
bufHdr = &LocalBufferDescriptors[i];
- errcontext.arg = bufHdr;
if (RelFileNodeEquals(bufHdr->tag.rnode, rel->rd_node))
{
- if (bufHdr->flags & BM_DIRTY || bufHdr->cntxDirty)
+ if ((bufHdr->flags & BM_VALID) &&
+ (bufHdr->flags & BM_DIRTY || bufHdr->cntxDirty))
{
- /* Open it at the smgr level if not already done */
+ ErrorContextCallback errcontext;
+
+ /* Setup error traceback support for ereport() */
+ errcontext.callback = buffer_write_error_callback;
+ errcontext.arg = bufHdr;
+ errcontext.previous = error_context_stack;
+ error_context_stack = &errcontext;
+
+ /* Open rel at the smgr level if not already done */
if (rel->rd_smgr == NULL)
rel->rd_smgr = smgropen(rel->rd_node);
smgrwrite(rel->rd_smgr,
bufHdr->tag.blockNum,
(char *) MAKE_PTR(bufHdr->data));
+
bufHdr->flags &= ~(BM_DIRTY | BM_JUST_DIRTIED);
bufHdr->cntxDirty = false;
+
+ /* Pop the error context stack */
+ error_context_stack = errcontext.previous;
}
if (LocalRefCount[i] > 0)
{
- error_context_stack = errcontext.previous;
elog(WARNING, "FlushRelationBuffers(\"%s\" (local), %u): block %u is referenced (%ld)",
RelationGetRelationName(rel), firstDelBlock,
bufHdr->tag.blockNum, LocalRefCount[i]);
}
}
- /* Pop the error context stack */
- error_context_stack = errcontext.previous;
-
return 0;
}
for (i = 0; i < NBuffers; i++)
{
bufHdr = &BufferDescriptors[i];
- errcontext.arg = bufHdr;
if (RelFileNodeEquals(bufHdr->tag.rnode, rel->rd_node))
{
- if (bufHdr->flags & BM_DIRTY || bufHdr->cntxDirty)
+ if ((bufHdr->flags & BM_VALID) &&
+ (bufHdr->flags & BM_DIRTY || bufHdr->cntxDirty))
{
PinBuffer(bufHdr);
+ /* Someone else might be flushing buffer */
if (bufHdr->flags & BM_IO_IN_PROGRESS)
WaitIO(bufHdr);
- LWLockRelease(BufMgrLock);
-
- /*
- * Force XLOG flush for buffer' LSN
- */
- recptr = BufferGetLSN(bufHdr);
- XLogFlush(recptr);
-
- /*
- * Now it's safe to write buffer to disk
- */
-
- LWLockAcquire(BufMgrLock, LW_EXCLUSIVE);
- if (bufHdr->flags & BM_IO_IN_PROGRESS)
- WaitIO(bufHdr);
-
+ /* Still dirty? */
if (bufHdr->flags & BM_DIRTY || bufHdr->cntxDirty)
{
- bufHdr->flags &= ~BM_JUST_DIRTIED;
- StartBufferIO(bufHdr, false); /* output IO start */
-
- LWLockRelease(BufMgrLock);
-
- /* Open it at the smgr level if not already done */
- if (rel->rd_smgr == NULL)
- rel->rd_smgr = smgropen(rel->rd_node);
-
- smgrwrite(rel->rd_smgr,
- bufHdr->tag.blockNum,
- (char *) MAKE_PTR(bufHdr->data));
-
- BufferFlushCount++;
+ StartBufferIO(bufHdr, false);
- LWLockAcquire(BufMgrLock, LW_EXCLUSIVE);
- bufHdr->flags &= ~BM_IO_IN_PROGRESS;
- TerminateBufferIO(bufHdr);
- Assert(!(bufHdr->flags & BM_JUST_DIRTIED));
- bufHdr->flags &= ~BM_DIRTY;
+ FlushBuffer(bufHdr, rel->rd_smgr);
- /*
- * Note that it's safe to change cntxDirty here
- * because of we protect it from upper writers by
- * AccessExclusiveLock and from other bufmgr routines
- * by BM_IO_IN_PROGRESS
- */
- bufHdr->cntxDirty = false;
+ TerminateBufferIO(bufHdr, 0);
}
UnpinBuffer(bufHdr);
+ if (bufHdr->flags & BM_DIRTY || bufHdr->cntxDirty)
+ {
+ LWLockRelease(BufMgrLock);
+ elog(WARNING, "FlushRelationBuffers(\"%s\", %u): block %u was re-dirtied",
+ RelationGetRelationName(rel), firstDelBlock,
+ bufHdr->tag.blockNum);
+ return -1;
+ }
}
if (bufHdr->refcount != 0)
{
LWLockRelease(BufMgrLock);
- error_context_stack = errcontext.previous;
elog(WARNING, "FlushRelationBuffers(\"%s\", %u): block %u is referenced (private %ld, global %d)",
RelationGetRelationName(rel), firstDelBlock,
bufHdr->tag.blockNum,
LWLockRelease(BufMgrLock);
- /* Pop the error context stack */
- error_context_stack = errcontext.previous;
-
return 0;
}
* ReleaseBuffer -- remove the pin on a buffer without
* marking it dirty.
*/
-int
+void
ReleaseBuffer(Buffer buffer)
{
BufferDesc *bufHdr;
{
Assert(LocalRefCount[-buffer - 1] > 0);
LocalRefCount[-buffer - 1]--;
- return STATUS_OK;
+ return;
}
if (BAD_BUFFER_ID(buffer))
- return STATUS_ERROR;
+ elog(ERROR, "bad buffer id: %d", buffer);
bufHdr = &BufferDescriptors[buffer - 1];
Assert(PrivateRefCount[buffer - 1] > 0);
+
if (PrivateRefCount[buffer - 1] > 1)
PrivateRefCount[buffer - 1]--;
else
UnpinBuffer(bufHdr);
LWLockRelease(BufMgrLock);
}
-
- return STATUS_OK;
}
#ifdef NOT_USED
* (Assumptions)
* My process is executing IO for the buffer
* BufMgrLock is held
+ * BM_IO_IN_PROGRESS mask is set for the buffer
* The buffer is Pinned
*
+ * err_flag must be 0 for successful completion and BM_IO_ERROR for failure.
+ *
* Because BufMgrLock is held, we are already in an interrupt holdoff here,
* and do not need another.
*/
static void
-TerminateBufferIO(BufferDesc *buf)
+TerminateBufferIO(BufferDesc *buf, int err_flag)
{
Assert(buf == InProgressBuf);
+ Assert(buf->flags & BM_IO_IN_PROGRESS);
+ buf->flags &= ~(BM_IO_IN_PROGRESS | BM_IO_ERROR);
+ buf->flags |= err_flag;
+
LWLockRelease(buf->io_in_progress_lock);
+
InProgressBuf = NULL;
}
/*
* Clean up any active buffer I/O after an error.
- * BufMgrLock isn't held when this function is called.
+ * BufMgrLock isn't held when this function is called,
+ * but we haven't yet released buffer pins, so the buffer is still pinned.
*
* If I/O was in progress, we always set BM_IO_ERROR.
*/
Assert(buf->flags & BM_IO_IN_PROGRESS);
if (IsForInput)
{
- Assert(!(buf->flags & BM_DIRTY) && !(buf->cntxDirty));
- /* Don't think that buffer is valid */
- StrategyInvalidateBuffer(buf);
+ Assert(!(buf->flags & BM_DIRTY || buf->cntxDirty));
+ /* We'd better not think buffer is valid yet */
+ Assert(!(buf->flags & BM_VALID));
}
else
{
}
buf->flags |= BM_DIRTY;
}
- buf->flags |= BM_IO_ERROR;
- buf->flags &= ~BM_IO_IN_PROGRESS;
- TerminateBufferIO(buf);
+ TerminateBufferIO(buf, BM_IO_ERROR);
LWLockRelease(BufMgrLock);
}
}