aboutsummaryrefslogtreecommitdiff
path: root/src/backend/storage/buffer/bufmgr.c
diff options
context:
space:
mode:
authorVadim B. Mikheev <vadim4o@yahoo.com>2000-11-30 08:46:26 +0000
committerVadim B. Mikheev <vadim4o@yahoo.com>2000-11-30 08:46:26 +0000
commit81c8c244b26011a071c89b43a38bba7039226019 (patch)
treea0602e39901d870d1fe4275a96c70a8450710882 /src/backend/storage/buffer/bufmgr.c
parentb16516b887f058782d67c90103148544f8adbd8f (diff)
downloadpostgresql-81c8c244b26011a071c89b43a38bba7039226019.tar.gz
postgresql-81c8c244b26011a071c89b43a38bba7039226019.zip
No more #ifdef XLOG.
Diffstat (limited to 'src/backend/storage/buffer/bufmgr.c')
-rw-r--r--src/backend/storage/buffer/bufmgr.c731
1 files changed, 199 insertions, 532 deletions
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index a25d4d9a55b..9400da38058 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -1,6 +1,6 @@
/*-------------------------------------------------------------------------
*
- * bufmgr.c
+ * xlog_bufmgr.c
* buffer manager interface routines
*
* Portions Copyright (c) 1996-2000, PostgreSQL, Inc
@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
- * $Header: /cvsroot/pgsql/src/backend/storage/buffer/bufmgr.c,v 1.96 2000/11/30 01:39:07 tgl Exp $
+ * $Header: /cvsroot/pgsql/src/backend/storage/buffer/bufmgr.c,v 1.97 2000/11/30 08:46:23 vadim Exp $
*
*-------------------------------------------------------------------------
*/
@@ -31,9 +31,6 @@
*
* WriteBuffer() -- WriteNoReleaseBuffer() + ReleaseBuffer()
*
- * FlushBuffer() -- Write buffer immediately. Can unpin, or not,
- * depending on parameter.
- *
* BufferSync() -- flush all dirty buffers in the buffer pool.
*
* InitBufferPool() -- Init the buffer module.
@@ -42,13 +39,8 @@
* freelist.c -- chooses victim for buffer replacement
* buf_table.c -- manages the buffer lookup table
*/
-
#include "postgres.h"
-#ifdef XLOG
-#include "xlog_bufmgr.c"
-#else
-
#include <sys/types.h>
#include <sys/file.h>
#include <math.h>
@@ -61,10 +53,11 @@
#include "storage/s_lock.h"
#include "storage/smgr.h"
#include "utils/relcache.h"
-
-#ifdef XLOG
#include "catalog/pg_database.h"
-#endif
+
+#define BufferGetLSN(bufHdr) \
+ (*((XLogRecPtr*)MAKE_PTR((bufHdr)->data)))
+
extern SPINLOCK BufMgrLock;
extern long int ReadBufferCount;
@@ -99,9 +92,6 @@ static Buffer ReadBufferWithBufferLock(Relation relation, BlockNumber blockNum,
bool bufferLockHeld);
static BufferDesc *BufferAlloc(Relation reln, BlockNumber blockNum,
bool *foundPtr, bool bufferLockHeld);
-static void SetBufferDirtiedByMe(Buffer buffer, BufferDesc *bufHdr);
-static void ClearBufferDirtiedByMe(Buffer buffer, BufferDesc *bufHdr);
-static void BufferSync(void);
static int BufferReplace(BufferDesc *bufHdr);
void PrintBufferDescs(void);
@@ -170,48 +160,6 @@ ReadBuffer(Relation reln, BlockNumber blockNum)
}
/*
- * is_userbuffer
- *
- * XXX caller must have already acquired BufMgrLock
- */
-#ifdef NOT_USED
-static bool
-is_userbuffer(Buffer buffer)
-{
- BufferDesc *buf = &BufferDescriptors[buffer - 1];
-
- if (IsSystemRelationName(buf->blind.relname))
- return false;
- return true;
-}
-
-#endif
-
-#ifdef NOT_USED
-Buffer
-ReadBuffer_Debug(char *file,
- int line,
- Relation reln,
- BlockNumber blockNum)
-{
- Buffer buffer;
-
- buffer = ReadBufferWithBufferLock(reln, blockNum, false);
- if (ShowPinTrace && !BufferIsLocal(buffer) && is_userbuffer(buffer))
- {
- BufferDesc *buf = &BufferDescriptors[buffer - 1];
-
- fprintf(stderr, "PIN(RD) %ld relname = %s, blockNum = %d, \
-refcount = %ld, file: %s, line: %d\n",
- buffer, buf->blind.relname, buf->tag.blockNum,
- PrivateRefCount[buffer - 1], file, line);
- }
- return buffer;
-}
-
-#endif
-
-/*
* ReadBufferWithBufferLock -- does the work of
* ReadBuffer() but with the possibility that
* the buffer lock has already been held. this
@@ -447,7 +395,7 @@ BufferAlloc(Relation reln,
buf->refcount = 1;
PrivateRefCount[BufferDescriptorGetBuffer(buf) - 1] = 1;
- if (buf->flags & BM_DIRTY)
+ if (buf->flags & BM_DIRTY || buf->cntxDirty)
{
bool smok;
@@ -505,18 +453,18 @@ BufferAlloc(Relation reln,
}
else
{
-
/*
* BM_JUST_DIRTIED cleared by BufferReplace and shouldn't
* be setted by anyone. - vadim 01/17/97
*/
if (buf->flags & BM_JUST_DIRTIED)
{
- elog(FATAL, "BufferAlloc: content of block %u (%s) changed while flushing",
+ elog(STOP, "BufferAlloc: content of block %u (%s) changed while flushing",
buf->tag.blockNum, buf->blind.relname);
}
else
buf->flags &= ~BM_DIRTY;
+ buf->cntxDirty = false;
}
/*
@@ -676,131 +624,15 @@ WriteBuffer(Buffer buffer)
SpinAcquire(BufMgrLock);
Assert(bufHdr->refcount > 0);
+
bufHdr->flags |= (BM_DIRTY | BM_JUST_DIRTIED);
- SetBufferDirtiedByMe(buffer, bufHdr);
+
UnpinBuffer(bufHdr);
SpinRelease(BufMgrLock);
return TRUE;
}
-#ifdef NOT_USED
-void
-WriteBuffer_Debug(char *file, int line, Buffer buffer)
-{
- WriteBuffer(buffer);
- if (ShowPinTrace && BufferIsLocal(buffer) && is_userbuffer(buffer))
- {
- BufferDesc *buf;
-
- buf = &BufferDescriptors[buffer - 1];
- fprintf(stderr, "UNPIN(WR) %ld relname = %s, blockNum = %d, \
-refcount = %ld, file: %s, line: %d\n",
- buffer, buf->blind.relname, buf->tag.blockNum,
- PrivateRefCount[buffer - 1], file, line);
- }
-}
-
-#endif
-
-/*
- * FlushBuffer -- like WriteBuffer, but write the page immediately,
- * rather than just marking it dirty. On success return, the buffer will
- * no longer be dirty.
- *
- * 'buffer' is known to be dirty/pinned, so there should not be a
- * problem reading the BufferDesc members without the BufMgrLock
- * (nobody should be able to change tags out from under us).
- *
- * If 'sync' is true, a synchronous write is wanted (wait for buffer to hit
- * the disk). Otherwise it's sufficient to issue the kernel write call.
- *
- * Unpin buffer if 'release' is true.
- */
-int
-FlushBuffer(Buffer buffer, bool sync, bool release)
-{
- BufferDesc *bufHdr;
- Relation bufrel;
- int status;
-
- if (BufferIsLocal(buffer))
- return FlushLocalBuffer(buffer, sync, release) ? STATUS_OK : STATUS_ERROR;
-
- if (BAD_BUFFER_ID(buffer))
- return STATUS_ERROR;
-
- Assert(PrivateRefCount[buffer - 1] > 0); /* else caller didn't pin */
-
- bufHdr = &BufferDescriptors[buffer - 1];
-
- bufrel = RelationNodeCacheGetRelation(bufHdr->tag.rnode);
-
- Assert(bufrel != (Relation) NULL);
-
- SharedBufferChanged = true;
-
- /* To check if block content changed while flushing. - vadim 01/17/97 */
- SpinAcquire(BufMgrLock);
- WaitIO(bufHdr, BufMgrLock); /* confirm end of IO */
- bufHdr->flags &= ~BM_JUST_DIRTIED;
- StartBufferIO(bufHdr, false); /* output IO start */
-
- SpinRelease(BufMgrLock);
-
- /*
- * Grab a read lock on the buffer to ensure that no
- * other backend changes its contents while we write it;
- * see comments in BufferSync().
- */
- LockBuffer(BufferDescriptorGetBuffer(bufHdr), BUFFER_LOCK_SHARE);
-
- if (sync)
- status = smgrflush(DEFAULT_SMGR, bufrel, bufHdr->tag.blockNum,
- (char *) MAKE_PTR(bufHdr->data));
- else
- status = smgrwrite(DEFAULT_SMGR, bufrel, bufHdr->tag.blockNum,
- (char *) MAKE_PTR(bufHdr->data));
-
- LockBuffer(BufferDescriptorGetBuffer(bufHdr), BUFFER_LOCK_UNLOCK);
-
- /* drop relcache refcnt incremented by RelationNodeCacheGetRelation */
- RelationDecrementReferenceCount(bufrel);
-
- if (status == SM_FAIL)
- {
- elog(ERROR, "FlushBuffer: cannot flush block %u of the relation %s",
- bufHdr->tag.blockNum, bufHdr->blind.relname);
- return STATUS_ERROR;
- }
- BufferFlushCount++;
-
- SpinAcquire(BufMgrLock);
- bufHdr->flags &= ~BM_IO_IN_PROGRESS; /* mark IO finished */
- TerminateBufferIO(bufHdr); /* output IO finished */
-
- /*
- * If this buffer was marked by someone as DIRTY while we were
- * flushing it out we must not clear shared DIRTY flag - vadim
- * 01/17/97
- *
- * ... but we can clear BufferDirtiedByMe anyway - tgl 3/31/00
- */
- if (bufHdr->flags & BM_JUST_DIRTIED)
- {
- elog(NOTICE, "FlushBuffer: content of block %u (%s) changed while flushing",
- bufHdr->tag.blockNum, bufHdr->blind.relname);
- }
- else
- bufHdr->flags &= ~BM_DIRTY;
- ClearBufferDirtiedByMe(buffer, bufHdr);
- if (release)
- UnpinBuffer(bufHdr);
- SpinRelease(BufMgrLock);
-
- return STATUS_OK;
-}
-
/*
* WriteNoReleaseBuffer -- like WriteBuffer, but do not unpin the buffer
* when the operation is complete.
@@ -822,8 +654,9 @@ WriteNoReleaseBuffer(Buffer buffer)
SpinAcquire(BufMgrLock);
Assert(bufHdr->refcount > 0);
+
bufHdr->flags |= (BM_DIRTY | BM_JUST_DIRTIED);
- SetBufferDirtiedByMe(buffer, bufHdr);
+
SpinRelease(BufMgrLock);
return STATUS_OK;
@@ -876,307 +709,138 @@ ReleaseAndReadBuffer(Buffer buffer,
}
/*
- * SetBufferDirtiedByMe -- mark a shared buffer as being dirtied by this xact
- *
- * This flag essentially remembers that we need to write and fsync this buffer
- * before we can commit the transaction. The write might end up getting done
- * by another backend, but we must do the fsync ourselves (else we could
- * commit before the data actually reaches disk). We do not issue fsync
- * instantly upon write; the storage manager keeps track of which files need
- * to be fsync'd before commit can occur. A key aspect of this data structure
- * is that we will be able to notify the storage manager that an fsync is
- * needed even after another backend has done the physical write and replaced
- * the buffer contents with something else!
+ * BufferSync -- Write all dirty buffers in the pool.
*
- * NB: we must be holding the bufmgr lock at entry, and the buffer must be
- * pinned so that no other backend can take it away from us.
+ * This is called at checkpoint time and write out all dirty buffers.
*/
-static void
-SetBufferDirtiedByMe(Buffer buffer, BufferDesc *bufHdr)
-{
- BufferTag *tagLastDirtied = &BufferTagLastDirtied[buffer - 1];
- Relation reln;
- int status;
-
- /*
- * If the flag is already set, check to see whether the buffertag is
- * the same. If not, some other backend already wrote the buffer data
- * that we dirtied. We must tell the storage manager to make an fsync
- * pending on that file before we can overwrite the old tag value.
- */
- if (BufferDirtiedByMe[buffer - 1])
- {
- if (RelFileNodeEquals(bufHdr->tag.rnode, tagLastDirtied->rnode) &&
- bufHdr->tag.blockNum == tagLastDirtied->blockNum)
- return; /* Same tag already dirtied, so no work */
-
-#ifndef OPTIMIZE_SINGLE
- SpinRelease(BufMgrLock);
-#endif /* OPTIMIZE_SINGLE */
-
- reln = RelationNodeCacheGetRelation(tagLastDirtied->rnode);
-
- if (reln == (Relation) NULL)
- {
- status = smgrblindmarkdirty(DEFAULT_SMGR,
- tagLastDirtied->rnode,
- tagLastDirtied->blockNum);
- }
- else
- {
- Assert(RelFileNodeEquals(tagLastDirtied->rnode, reln->rd_node));
- status = smgrmarkdirty(DEFAULT_SMGR, reln,
- tagLastDirtied->blockNum);
-
- /*
- * drop relcache refcnt incremented by
- * RelationNodeCacheGetRelation
- */
- RelationDecrementReferenceCount(reln);
- }
- if (status == SM_FAIL)
- {
- elog(ERROR, "SetBufferDirtiedByMe: cannot mark %u for %s",
- tagLastDirtied->blockNum,
- BufferBlindLastDirtied[buffer - 1].relname);
- }
-
-#ifndef OPTIMIZE_SINGLE
- SpinAcquire(BufMgrLock);
-#endif /* OPTIMIZE_SINGLE */
-
- }
-
- *tagLastDirtied = bufHdr->tag;
- BufferBlindLastDirtied[buffer - 1] = bufHdr->blind;
- BufferDirtiedByMe[buffer - 1] = true;
-}
-
-/*
- * ClearBufferDirtiedByMe -- mark a shared buffer as no longer needing fsync
- *
- * If we write out a buffer ourselves, then the storage manager will set its
- * needs-fsync flag for that file automatically, and so we can clear our own
- * flag that says it needs to be done later.
- *
- * NB: we must be holding the bufmgr lock at entry.
- */
-static void
-ClearBufferDirtiedByMe(Buffer buffer, BufferDesc *bufHdr)
-{
- BufferTag *tagLastDirtied = &BufferTagLastDirtied[buffer - 1];
-
- /*
- * Do *not* clear the flag if it refers to some other buffertag than
- * the data we just wrote. This is unlikely, but possible if some
- * other backend replaced the buffer contents since we set our flag.
- */
- if (RelFileNodeEquals(bufHdr->tag.rnode, tagLastDirtied->rnode) &&
- bufHdr->tag.blockNum == tagLastDirtied->blockNum)
- BufferDirtiedByMe[buffer - 1] = false;
-}
-
-/*
- * BufferSync -- Flush all dirty buffers in the pool.
- *
- * This is called at transaction commit time. We find all buffers
- * that have been dirtied by the current xact and flush them to disk.
- * We do *not* flush dirty buffers that have been dirtied by other xacts.
- * (This is a substantial change from pre-7.0 behavior.)
- */
-static void
+void
BufferSync()
{
int i;
BufferDesc *bufHdr;
+ Buffer buffer;
int status;
- Relation reln;
- bool didwrite;
+ RelFileNode rnode;
+ XLogRecPtr recptr;
+ Relation reln = NULL;
for (i = 0, bufHdr = BufferDescriptors; i < NBuffers; i++, bufHdr++)
{
- /* Ignore buffers that were not dirtied by me */
- if (!BufferDirtiedByMe[i])
- continue;
SpinAcquire(BufMgrLock);
- /*
- * We only need to write if the buffer is still dirty and still
- * contains the same disk page that it contained when we dirtied
- * it. Otherwise, someone else has already written our changes for
- * us, and we need only fsync.
- *
- * (NOTE: it's still possible to do an unnecessary write, if other
- * xacts have written and then re-dirtied the page since our last
- * change to it. But that should be pretty uncommon, and there's
- * no easy way to detect it anyway.)
- */
- reln = NULL;
- didwrite = false;
- if ((bufHdr->flags & BM_VALID) && (bufHdr->flags & BM_DIRTY))
+ if (!(bufHdr->flags & BM_VALID))
{
- if (RelFileNodeEquals(bufHdr->tag.rnode, BufferTagLastDirtied[i].rnode) &&
- bufHdr->tag.blockNum == BufferTagLastDirtied[i].blockNum)
- {
- /*
- * Try to find relation for buf. This could fail, if the
- * rel has been flushed from the relcache since we dirtied
- * the page. That should be uncommon, so paying the extra
- * cost of a blind write when it happens seems OK.
- */
- if (!InRecovery)
- reln = RelationNodeCacheGetRelation(bufHdr->tag.rnode);
-
- /*
- * We have to pin buffer to keep anyone from stealing it
- * from the buffer pool while we are flushing it or
- * waiting in WaitIO. It's bad for GetFreeBuffer in
- * BufferAlloc, but there is no other way to prevent
- * writing into disk block data from some other buffer,
- * getting smgr status of some other block and clearing
- * BM_DIRTY of ... - VAdim 09/16/96
- */
- PinBuffer(bufHdr);
- if (bufHdr->flags & BM_IO_IN_PROGRESS)
- {
- WaitIO(bufHdr, BufMgrLock);
- UnpinBuffer(bufHdr);
- if (bufHdr->flags & BM_IO_ERROR)
- {
- elog(ERROR, "BufferSync: write error %u for %s",
- bufHdr->tag.blockNum, bufHdr->blind.relname);
- }
- }
- else
- {
-
- /*
- * To check if block content changed while flushing
- * (see below). - vadim 01/17/97
- */
- WaitIO(bufHdr, BufMgrLock); /* confirm end of IO */
- bufHdr->flags &= ~BM_JUST_DIRTIED;
- StartBufferIO(bufHdr, false); /* output IO start */
-
- SpinRelease(BufMgrLock);
-
- /*
- * Grab a read lock on the buffer to ensure that no
- * other backend changes its contents while we write it;
- * otherwise we could write a non-self-consistent page
- * image to disk, which'd be bad news if the other
- * transaction aborts before writing its changes.
- *
- * Note that we still need the BM_JUST_DIRTIED mechanism
- * in case someone dirties the buffer just before we
- * grab this lock or just after we release it.
- */
- LockBuffer(BufferDescriptorGetBuffer(bufHdr),
- BUFFER_LOCK_SHARE);
+ SpinRelease(BufMgrLock);
+ continue;
+ }
- /*
- * If we didn't have the reldesc in our local cache,
- * write this page out using the 'blind write' storage
- * manager routine. If we did find it, use the
- * standard interface.
- */
- if (reln == (Relation) NULL)
- {
- status = smgrblindwrt(DEFAULT_SMGR,
- bufHdr->tag.rnode,
- bufHdr->tag.blockNum,
- (char *) MAKE_PTR(bufHdr->data),
- true); /* must fsync */
- }
- else
- {
- status = smgrwrite(DEFAULT_SMGR, reln,
- bufHdr->tag.blockNum,
- (char *) MAKE_PTR(bufHdr->data));
- }
+ /*
+ * Pin buffer and ensure that no one reads it from disk
+ */
+ PinBuffer(bufHdr);
+ /* Synchronize with BufferAlloc */
+ if (bufHdr->flags & BM_IO_IN_PROGRESS)
+ WaitIO(bufHdr, BufMgrLock);
- /*
- * Release the per-buffer readlock, reacquire BufMgrLock.
- */
- LockBuffer(BufferDescriptorGetBuffer(bufHdr),
- BUFFER_LOCK_UNLOCK);
+ buffer = BufferDescriptorGetBuffer(bufHdr);
+ rnode = bufHdr->tag.rnode;
- SpinAcquire(BufMgrLock);
+ SpinRelease(BufMgrLock);
- UnpinBuffer(bufHdr);
- if (status == SM_FAIL)
- {
- bufHdr->flags |= BM_IO_ERROR;
- elog(ERROR, "BufferSync: cannot write %u for %s",
- bufHdr->tag.blockNum, bufHdr->blind.relname);
- }
- bufHdr->flags &= ~BM_IO_IN_PROGRESS; /* mark IO finished */
- TerminateBufferIO(bufHdr); /* Sync IO finished */
- BufferFlushCount++;
- didwrite = true;
+ /*
+ * Try to find relation for buffer
+ */
+ reln = RelationNodeCacheGetRelation(rnode);
- /*
- * 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
- *
- * but it is OK to clear BufferDirtiedByMe - tgl 3/31/00
- */
- if (!(bufHdr->flags & BM_JUST_DIRTIED))
- bufHdr->flags &= ~BM_DIRTY;
- }
+ /*
+ * Protect buffer content against concurrent update
+ */
+ LockBuffer(buffer, BUFFER_LOCK_SHARE);
- /* drop refcnt obtained by RelationNodeCacheGetRelation */
- if (reln != (Relation) NULL)
- RelationDecrementReferenceCount(reln);
- }
- }
+ /*
+ * Force XLOG flush for buffer' LSN
+ */
+ recptr = BufferGetLSN(bufHdr);
+ XLogFlush(recptr);
/*
- * If we did not write the buffer (because someone else did), we
- * must still fsync the file containing it, to ensure that the
- * write is down to disk before we commit.
+ * Now it's safe to write buffer to disk
+ * (if needed at all -:))
*/
- if (!didwrite)
+
+ SpinAcquire(BufMgrLock);
+ if (bufHdr->flags & BM_IO_IN_PROGRESS)
+ WaitIO(bufHdr, BufMgrLock);
+
+ if (bufHdr->flags & BM_DIRTY || bufHdr->cntxDirty)
{
-#ifndef OPTIMIZE_SINGLE
+ bufHdr->flags &= ~BM_JUST_DIRTIED;
+ StartBufferIO(bufHdr, false); /* output IO start */
+
SpinRelease(BufMgrLock);
-#endif /* OPTIMIZE_SINGLE */
- reln = RelationNodeCacheGetRelation(BufferTagLastDirtied[i].rnode);
if (reln == (Relation) NULL)
{
- status = smgrblindmarkdirty(DEFAULT_SMGR,
- BufferTagLastDirtied[i].rnode,
- BufferTagLastDirtied[i].blockNum);
+ status = smgrblindwrt(DEFAULT_SMGR,
+ bufHdr->tag.rnode,
+ bufHdr->tag.blockNum,
+ (char *) MAKE_PTR(bufHdr->data),
+ true); /* must fsync */
}
else
{
- status = smgrmarkdirty(DEFAULT_SMGR, reln,
- BufferTagLastDirtied[i].blockNum);
+ status = smgrwrite(DEFAULT_SMGR, reln,
+ bufHdr->tag.blockNum,
+ (char *) MAKE_PTR(bufHdr->data));
+ }
- /*
- * drop relcache refcnt incremented by
- * RelationNodeCacheGetRelation
- */
- RelationDecrementReferenceCount(reln);
+ if (status == SM_FAIL) /* disk failure ?! */
+ elog(STOP, "BufferSync: cannot write %u for %s",
+ bufHdr->tag.blockNum, bufHdr->blind.relname);
+
+ /*
+ * 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;
+
+ /*
+ * Release the per-buffer readlock, reacquire BufMgrLock.
+ */
+ LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
+ BufferFlushCount++;
- }
-#ifndef OPTIMIZE_SINGLE
SpinAcquire(BufMgrLock);
-#endif /* OPTIMIZE_SINGLE */
+
+ bufHdr->flags &= ~BM_IO_IN_PROGRESS; /* mark IO finished */
+ TerminateBufferIO(bufHdr); /* Sync IO finished */
+
+ /*
+ * 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;
}
+ else
+ LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
- BufferDirtiedByMe[i] = false;
+ UnpinBuffer(bufHdr);
SpinRelease(BufMgrLock);
+
+ /* drop refcnt obtained by RelationNodeCacheGetRelation */
+ if (reln != (Relation) NULL)
+ {
+ RelationDecrementReferenceCount(reln);
+ reln = NULL;
+ }
}
-#ifndef XLOG
- LocalBufferSync();
-#endif
-}
+}
/*
* WaitIO -- Block until the IO_IN_PROGRESS flag on 'buf' is cleared.
@@ -1278,9 +942,6 @@ ResetBufferPool(bool isCommit)
SpinRelease(BufMgrLock);
}
PrivateRefCount[i] = 0;
-
- if (!isCommit)
- BufferDirtiedByMe[i] = false;
}
ResetLocalBufferPool();
@@ -1321,16 +982,29 @@ relname=%s, blockNum=%d, flags=0x%x, refcount=%d %ld)",
}
/* ------------------------------------------------
- * FlushBufferPool
- *
- * flush all dirty blocks in buffer pool to disk
+ * FlushBufferPool
*
+ * Flush all dirty blocks in buffer pool to disk
+ * at the checkpoint time
* ------------------------------------------------
*/
void
FlushBufferPool(void)
{
BufferSync();
+ smgrsync();
+}
+
+/*
+ * At the commit time we have to flush local buffer pool only
+ */
+void
+BufmgrCommit(void)
+{
+ LocalBufferSync();
+ /*
+ * All files created in current transaction will be fsync-ed
+ */
smgrcommit();
}
@@ -1358,35 +1032,28 @@ BufferGetBlockNumber(Buffer buffer)
*
* Write out the buffer corresponding to 'bufHdr'
*
- * This routine used to flush the data to disk (ie, force immediate fsync)
- * but that's no longer necessary because BufferSync is smarter than before.
- *
* BufMgrLock must be held at entry, and the buffer must be pinned.
*/
static int
BufferReplace(BufferDesc *bufHdr)
{
Relation reln;
+ XLogRecPtr recptr;
int status;
- /*
- * first try to find the reldesc in the cache, if no luck, don't
- * bother to build the reldesc from scratch, just do a blind write.
- */
-
- reln = RelationNodeCacheGetRelation(bufHdr->tag.rnode);
-
/* To check if block content changed while flushing. - vadim 01/17/97 */
bufHdr->flags &= ~BM_JUST_DIRTIED;
SpinRelease(BufMgrLock);
/*
- * Grab a read lock on the buffer to ensure that no
- * other backend changes its contents while we write it;
- * see comments in BufferSync().
+ * No need to lock buffer context - no one should be able to
+ * end ReadBuffer
*/
- LockBuffer(BufferDescriptorGetBuffer(bufHdr), BUFFER_LOCK_SHARE);
+ recptr = BufferGetLSN(bufHdr);
+ XLogFlush(recptr);
+
+ reln = RelationNodeCacheGetRelation(bufHdr->tag.rnode);
if (reln != (Relation) NULL)
{
@@ -1401,25 +1068,15 @@ BufferReplace(BufferDesc *bufHdr)
false); /* no fsync */
}
- LockBuffer(BufferDescriptorGetBuffer(bufHdr), BUFFER_LOCK_UNLOCK);
-
- SpinAcquire(BufMgrLock);
-
/* drop relcache refcnt incremented by RelationNodeCacheGetRelation */
if (reln != (Relation) NULL)
RelationDecrementReferenceCount(reln);
+ SpinAcquire(BufMgrLock);
+
if (status == SM_FAIL)
return FALSE;
- /*
- * If we had marked this buffer as needing to be fsync'd, we can
- * forget about that, because it's now the storage manager's
- * responsibility (but only if we called smgrwrite, not smgrblindwrt).
- */
- if (reln != (Relation) NULL)
- ClearBufferDirtiedByMe(BufferDescriptorGetBuffer(bufHdr), bufHdr);
-
BufferFlushCount++;
return TRUE;
@@ -1438,7 +1095,8 @@ BlockNumber
RelationGetNumberOfBlocks(Relation relation)
{
return ((relation->rd_myxactonly) ? relation->rd_nblocks :
- smgrnblocks(DEFAULT_SMGR, relation));
+ ((relation->rd_rel->relkind == RELKIND_VIEW) ? 0 :
+ smgrnblocks(DEFAULT_SMGR, relation)));
}
/* ---------------------------------------------------------------------
@@ -1471,6 +1129,7 @@ DropRelationBuffers(Relation rel)
if (RelFileNodeEquals(bufHdr->tag.rnode, rel->rd_node))
{
bufHdr->flags &= ~(BM_DIRTY | BM_JUST_DIRTIED);
+ bufHdr->cntxDirty = false;
LocalRefCount[i] = 0;
bufHdr->tag.rnode.relNode = InvalidOid;
}
@@ -1503,6 +1162,7 @@ recheck:
}
/* Now we can do what we came for */
bufHdr->flags &= ~(BM_DIRTY | BM_JUST_DIRTIED);
+ bufHdr->cntxDirty = false;
/*
* Release any refcount we may have.
@@ -1526,20 +1186,6 @@ recheck:
*/
BufTableDelete(bufHdr);
}
-
- /*
- * Also check to see if BufferDirtiedByMe info for this buffer
- * refers to the target relation, and clear it if so. This is
- * independent of whether the current contents of the buffer
- * belong to the target relation!
- *
- * NOTE: we have no way to clear BufferDirtiedByMe info in other
- * backends, but hopefully there are none with that bit set for
- * this rel, since we hold exclusive lock on this rel.
- */
- if (RelFileNodeEquals(rel->rd_node,
- BufferTagLastDirtied[i - 1].rnode))
- BufferDirtiedByMe[i - 1] = false;
}
SpinRelease(BufMgrLock);
@@ -1570,6 +1216,7 @@ DropRelFileNodeBuffers(RelFileNode rnode)
if (RelFileNodeEquals(bufHdr->tag.rnode, rnode))
{
bufHdr->flags &= ~(BM_DIRTY | BM_JUST_DIRTIED);
+ bufHdr->cntxDirty = false;
LocalRefCount[i] = 0;
bufHdr->tag.rnode.relNode = InvalidOid;
}
@@ -1600,6 +1247,7 @@ recheck:
}
/* Now we can do what we came for */
bufHdr->flags &= ~(BM_DIRTY | BM_JUST_DIRTIED);
+ bufHdr->cntxDirty = false;
/*
* Release any refcount we may have.
@@ -1623,20 +1271,6 @@ recheck:
*/
BufTableDelete(bufHdr);
}
-
- /*
- * Also check to see if BufferDirtiedByMe info for this buffer
- * refers to the target relation, and clear it if so. This is
- * independent of whether the current contents of the buffer
- * belong to the target relation!
- *
- * NOTE: we have no way to clear BufferDirtiedByMe info in other
- * backends, but hopefully there are none with that bit set for
- * this rel, since we hold exclusive lock on this rel.
- */
- if (RelFileNodeEquals(rnode,
- BufferTagLastDirtied[i - 1].rnode))
- BufferDirtiedByMe[i - 1] = false;
}
SpinRelease(BufMgrLock);
@@ -1689,6 +1323,7 @@ recheck:
}
/* Now we can do what we came for */
bufHdr->flags &= ~(BM_DIRTY | BM_JUST_DIRTIED);
+ bufHdr->cntxDirty = false;
/*
* The thing should be free, if caller has checked that no
@@ -1700,17 +1335,6 @@ recheck:
*/
BufTableDelete(bufHdr);
}
- /*
- * Also check to see if BufferDirtiedByMe info for this buffer
- * refers to the target database, and clear it if so. This is
- * independent of whether the current contents of the buffer
- * belong to the target database!
- *
- * (Actually, this is probably unnecessary, since I shouldn't have
- * ever dirtied pages of the target database, but...)
- */
- if (BufferTagLastDirtied[i - 1].rnode.tblNode == dbid)
- BufferDirtiedByMe[i - 1] = false;
}
SpinRelease(BufMgrLock);
}
@@ -1847,6 +1471,8 @@ FlushRelationBuffers(Relation rel, BlockNumber firstDelBlock)
{
int i;
BufferDesc *bufHdr;
+ XLogRecPtr recptr;
+ int status;
if (rel->rd_myxactonly)
{
@@ -1855,22 +1481,27 @@ FlushRelationBuffers(Relation rel, BlockNumber firstDelBlock)
bufHdr = &LocalBufferDescriptors[i];
if (RelFileNodeEquals(bufHdr->tag.rnode, rel->rd_node))
{
- if (bufHdr->flags & BM_DIRTY)
+ if (bufHdr->flags & BM_DIRTY || bufHdr->cntxDirty)
{
- if (FlushBuffer(-i - 1, false, false) != STATUS_OK)
+ status = smgrwrite(DEFAULT_SMGR, rel,
+ bufHdr->tag.blockNum,
+ (char *) MAKE_PTR(bufHdr->data));
+ if (status == SM_FAIL)
{
elog(NOTICE, "FlushRelationBuffers(%s (local), %u): block %u is dirty, could not flush it",
RelationGetRelationName(rel), firstDelBlock,
bufHdr->tag.blockNum);
- return -1;
+ return(-1);
}
+ 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]);
- return -2;
+ return(-2);
}
if (bufHdr->tag.blockNum >= firstDelBlock)
{
@@ -1887,22 +1518,57 @@ FlushRelationBuffers(Relation rel, BlockNumber firstDelBlock)
bufHdr = &BufferDescriptors[i];
if (RelFileNodeEquals(bufHdr->tag.rnode, rel->rd_node))
{
- if (bufHdr->flags & BM_DIRTY)
+ if (bufHdr->flags & BM_DIRTY || bufHdr->cntxDirty)
{
PinBuffer(bufHdr);
+ if (bufHdr->flags & BM_IO_IN_PROGRESS)
+ WaitIO(bufHdr, BufMgrLock);
SpinRelease(BufMgrLock);
- if (FlushBuffer(i + 1, false, false) != STATUS_OK)
+
+ /*
+ * 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)
{
- SpinAcquire(BufMgrLock);
- UnpinBuffer(bufHdr);
+ bufHdr->flags &= ~BM_JUST_DIRTIED;
+ StartBufferIO(bufHdr, false); /* output IO start */
+
SpinRelease(BufMgrLock);
- elog(NOTICE, "FlushRelationBuffers(%s, %u): block %u is dirty (private %ld, global %d), could not flush it",
- RelationGetRelationName(rel), firstDelBlock,
- bufHdr->tag.blockNum,
- PrivateRefCount[i], bufHdr->refcount);
- return -1;
+
+ status = smgrwrite(DEFAULT_SMGR, rel,
+ bufHdr->tag.blockNum,
+ (char *) MAKE_PTR(bufHdr->data));
+
+ if (status == SM_FAIL) /* disk failure ?! */
+ 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;
+ /*
+ * 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;
}
- SpinAcquire(BufMgrLock);
UnpinBuffer(bufHdr);
}
if (!(bufHdr->flags & BM_FREE))
@@ -2341,6 +2007,9 @@ LockBuffer(Buffer buffer, int mode)
}
buf->w_lock = true;
*buflock |= BL_W_LOCK;
+
+ buf->cntxDirty = true;
+
if (*buflock & BL_RI_LOCK)
{
@@ -2458,11 +2127,11 @@ AbortBufferIO(void)
Assert(buf->flags & BM_IO_IN_PROGRESS);
SpinAcquire(BufMgrLock);
if (IsForInput)
- Assert(!(buf->flags & BM_DIRTY));
+ Assert(!(buf->flags & BM_DIRTY) && !(buf->cntxDirty));
else
{
- Assert((buf->flags & BM_DIRTY) != 0);
- if ((buf->flags & BM_IO_ERROR) != 0)
+ Assert(buf->flags & BM_DIRTY || buf->cntxDirty);
+ 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);
@@ -2528,5 +2197,3 @@ MarkBufferForCleanup(Buffer buffer, void (*CleanupFunc)(Buffer))
SpinRelease(BufMgrLock);
return;
}
-
-#endif /* ! XLOG */