提交 cbe9d6be 编写于 作者: T Tom Lane

Fix up rickety handling of relation-truncation interlocks.

Move rd_targblock, rd_fsm_nblocks, and rd_vm_nblocks from relcache to the smgr
relation entries, so that they will get reset to InvalidBlockNumber whenever
an smgr-level flush happens.  Because we now send smgr invalidation messages
immediately (not at end of transaction) when a relation truncation occurs,
this ensures that other backends will reset their values before they next
access the relation.  We no longer need the unreliable assumption that a
VACUUM that's doing a truncation will hold its AccessExclusive lock until
commit --- in fact, we can intentionally release that lock as soon as we've
completed the truncation.  This patch therefore reverts (most of) Alvaro's
patch of 2009-11-10, as well as my marginal hacking on it yesterday.  We can
also get rid of assorted no-longer-needed relcache flushes, which are far more
expensive than an smgr flush because they kill a lot more state.

In passing this patch fixes smgr_redo's failure to perform visibility-map
truncation, and cleans up some rather dubious assumptions in freespace.c and
visibilitymap.c about when rd_fsm_nblocks and rd_vm_nblocks can be out of
date.
上级 79647eed
...@@ -8,7 +8,7 @@ ...@@ -8,7 +8,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/access/heap/hio.c,v 1.77 2010/01/02 16:57:34 momjian Exp $ * $PostgreSQL: pgsql/src/backend/access/heap/hio.c,v 1.78 2010/02/09 21:43:29 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -20,6 +20,7 @@ ...@@ -20,6 +20,7 @@
#include "storage/bufmgr.h" #include "storage/bufmgr.h"
#include "storage/freespace.h" #include "storage/freespace.h"
#include "storage/lmgr.h" #include "storage/lmgr.h"
#include "storage/smgr.h"
/* /*
...@@ -126,7 +127,7 @@ ReadBufferBI(Relation relation, BlockNumber targetBlock, ...@@ -126,7 +127,7 @@ ReadBufferBI(Relation relation, BlockNumber targetBlock,
* *
* HEAP_INSERT_SKIP_FSM is also useful for non-WAL-logged additions to a * HEAP_INSERT_SKIP_FSM is also useful for non-WAL-logged additions to a
* relation, if the caller holds exclusive lock and is careful to invalidate * relation, if the caller holds exclusive lock and is careful to invalidate
* relation->rd_targblock before the first insertion --- that ensures that * relation's smgr_targblock before the first insertion --- that ensures that
* all insertions will occur into newly added pages and not be intermixed * all insertions will occur into newly added pages and not be intermixed
* with tuples from other transactions. That way, a crash can't risk losing * with tuples from other transactions. That way, a crash can't risk losing
* any committed data of other transactions. (See heap_insert's comments * any committed data of other transactions. (See heap_insert's comments
...@@ -206,7 +207,7 @@ RelationGetBufferForTuple(Relation relation, Size len, ...@@ -206,7 +207,7 @@ RelationGetBufferForTuple(Relation relation, Size len,
else if (bistate && bistate->current_buf != InvalidBuffer) else if (bistate && bistate->current_buf != InvalidBuffer)
targetBlock = BufferGetBlockNumber(bistate->current_buf); targetBlock = BufferGetBlockNumber(bistate->current_buf);
else else
targetBlock = relation->rd_targblock; targetBlock = RelationGetTargetBlock(relation);
if (targetBlock == InvalidBlockNumber && use_fsm) if (targetBlock == InvalidBlockNumber && use_fsm)
{ {
...@@ -273,7 +274,7 @@ RelationGetBufferForTuple(Relation relation, Size len, ...@@ -273,7 +274,7 @@ RelationGetBufferForTuple(Relation relation, Size len,
if (len + saveFreeSpace <= pageFreeSpace) if (len + saveFreeSpace <= pageFreeSpace)
{ {
/* use this page as future insert target, too */ /* use this page as future insert target, too */
relation->rd_targblock = targetBlock; RelationSetTargetBlock(relation, targetBlock);
return buffer; return buffer;
} }
...@@ -377,7 +378,7 @@ RelationGetBufferForTuple(Relation relation, Size len, ...@@ -377,7 +378,7 @@ RelationGetBufferForTuple(Relation relation, Size len,
* current backend to make more insertions or not, which is probably a * current backend to make more insertions or not, which is probably a
* good bet most of the time. So for now, don't add it to FSM yet. * good bet most of the time. So for now, don't add it to FSM yet.
*/ */
relation->rd_targblock = BufferGetBlockNumber(buffer); RelationSetTargetBlock(relation, BufferGetBlockNumber(buffer));
return buffer; return buffer;
} }
...@@ -8,7 +8,7 @@ ...@@ -8,7 +8,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/access/heap/visibilitymap.c,v 1.7 2010/01/02 16:57:35 momjian Exp $ * $PostgreSQL: pgsql/src/backend/access/heap/visibilitymap.c,v 1.8 2010/02/09 21:43:29 tgl Exp $
* *
* INTERFACE ROUTINES * INTERFACE ROUTINES
* visibilitymap_clear - clear a bit in the visibility map * visibilitymap_clear - clear a bit in the visibility map
...@@ -94,7 +94,7 @@ ...@@ -94,7 +94,7 @@
#include "storage/bufpage.h" #include "storage/bufpage.h"
#include "storage/lmgr.h" #include "storage/lmgr.h"
#include "storage/smgr.h" #include "storage/smgr.h"
#include "utils/inval.h"
/*#define TRACE_VISIBILITYMAP */ /*#define TRACE_VISIBILITYMAP */
...@@ -291,7 +291,13 @@ visibilitymap_test(Relation rel, BlockNumber heapBlk, Buffer *buf) ...@@ -291,7 +291,13 @@ visibilitymap_test(Relation rel, BlockNumber heapBlk, Buffer *buf)
} }
/* /*
* visibilitymap_test - truncate the visibility map * visibilitymap_truncate - truncate the visibility map
*
* The caller must hold AccessExclusiveLock on the relation, to ensure that
* other backends receive the smgr invalidation event that this function sends
* before they access the VM again.
*
* nheapblocks is the new size of the heap.
*/ */
void void
visibilitymap_truncate(Relation rel, BlockNumber nheapblocks) visibilitymap_truncate(Relation rel, BlockNumber nheapblocks)
...@@ -307,6 +313,8 @@ visibilitymap_truncate(Relation rel, BlockNumber nheapblocks) ...@@ -307,6 +313,8 @@ visibilitymap_truncate(Relation rel, BlockNumber nheapblocks)
elog(DEBUG1, "vm_truncate %s %d", RelationGetRelationName(rel), nheapblocks); elog(DEBUG1, "vm_truncate %s %d", RelationGetRelationName(rel), nheapblocks);
#endif #endif
RelationOpenSmgr(rel);
/* /*
* If no visibility map has been created yet for this relation, there's * If no visibility map has been created yet for this relation, there's
* nothing to truncate. * nothing to truncate.
...@@ -358,23 +366,25 @@ visibilitymap_truncate(Relation rel, BlockNumber nheapblocks) ...@@ -358,23 +366,25 @@ visibilitymap_truncate(Relation rel, BlockNumber nheapblocks)
else else
newnblocks = truncBlock; newnblocks = truncBlock;
if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM) < newnblocks) if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM) <= newnblocks)
{ {
/* nothing to do, the file was already smaller than requested size */ /* nothing to do, the file was already smaller than requested size */
return; return;
} }
/* Truncate the unused VM pages, and send smgr inval message */
smgrtruncate(rel->rd_smgr, VISIBILITYMAP_FORKNUM, newnblocks, smgrtruncate(rel->rd_smgr, VISIBILITYMAP_FORKNUM, newnblocks,
rel->rd_istemp); rel->rd_istemp);
/* /*
* Need to invalidate the relcache entry, because rd_vm_nblocks seen by * We might as well update the local smgr_vm_nblocks setting.
* other backends is no longer valid. * smgrtruncate sent an smgr cache inval message, which will cause
* other backends to invalidate their copy of smgr_vm_nblocks, and
* this one too at the next command boundary. But this ensures it
* isn't outright wrong until then.
*/ */
if (!InRecovery) if (rel->rd_smgr)
CacheInvalidateRelcache(rel); rel->rd_smgr->smgr_vm_nblocks = newnblocks;
rel->rd_vm_nblocks = newnblocks;
} }
/* /*
...@@ -391,21 +401,23 @@ vm_readbuf(Relation rel, BlockNumber blkno, bool extend) ...@@ -391,21 +401,23 @@ vm_readbuf(Relation rel, BlockNumber blkno, bool extend)
RelationOpenSmgr(rel); RelationOpenSmgr(rel);
/* /*
* The current size of the visibility map fork is kept in relcache, to * If we haven't cached the size of the visibility map fork yet, check it
* avoid reading beyond EOF. If we haven't cached the size of the map yet, * first. Also recheck if the requested block seems to be past end, since
* do that first. * our cached value might be stale. (We send smgr inval messages on
* truncation, but not on extension.)
*/ */
if (rel->rd_vm_nblocks == InvalidBlockNumber) if (rel->rd_smgr->smgr_vm_nblocks == InvalidBlockNumber ||
blkno >= rel->rd_smgr->smgr_vm_nblocks)
{ {
if (smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM)) if (smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM))
rel->rd_vm_nblocks = smgrnblocks(rel->rd_smgr, rel->rd_smgr->smgr_vm_nblocks = smgrnblocks(rel->rd_smgr,
VISIBILITYMAP_FORKNUM); VISIBILITYMAP_FORKNUM);
else else
rel->rd_vm_nblocks = 0; rel->rd_smgr->smgr_vm_nblocks = 0;
} }
/* Handle requests beyond EOF */ /* Handle requests beyond EOF */
if (blkno >= rel->rd_vm_nblocks) if (blkno >= rel->rd_smgr->smgr_vm_nblocks)
{ {
if (extend) if (extend)
vm_extend(rel, blkno + 1); vm_extend(rel, blkno + 1);
...@@ -446,19 +458,23 @@ vm_extend(Relation rel, BlockNumber vm_nblocks) ...@@ -446,19 +458,23 @@ vm_extend(Relation rel, BlockNumber vm_nblocks)
* separate lock tag type for it. * separate lock tag type for it.
* *
* Note that another backend might have extended or created the relation * Note that another backend might have extended or created the relation
* before we get the lock. * by the time we get the lock.
*/ */
LockRelationForExtension(rel, ExclusiveLock); LockRelationForExtension(rel, ExclusiveLock);
/* Create the file first if it doesn't exist */ /* Might have to re-open if a cache flush happened */
if ((rel->rd_vm_nblocks == 0 || rel->rd_vm_nblocks == InvalidBlockNumber) RelationOpenSmgr(rel);
&& !smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM))
{ /*
* Create the file first if it doesn't exist. If smgr_vm_nblocks
* is positive then it must exist, no need for an smgrexists call.
*/
if ((rel->rd_smgr->smgr_vm_nblocks == 0 ||
rel->rd_smgr->smgr_vm_nblocks == InvalidBlockNumber) &&
!smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM))
smgrcreate(rel->rd_smgr, VISIBILITYMAP_FORKNUM, false); smgrcreate(rel->rd_smgr, VISIBILITYMAP_FORKNUM, false);
vm_nblocks_now = 0;
} vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
else
vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
while (vm_nblocks_now < vm_nblocks) while (vm_nblocks_now < vm_nblocks)
{ {
...@@ -467,12 +483,10 @@ vm_extend(Relation rel, BlockNumber vm_nblocks) ...@@ -467,12 +483,10 @@ vm_extend(Relation rel, BlockNumber vm_nblocks)
vm_nblocks_now++; vm_nblocks_now++;
} }
/* Update local cache with the up-to-date size */
rel->rd_smgr->smgr_vm_nblocks = vm_nblocks_now;
UnlockRelationForExtension(rel, ExclusiveLock); UnlockRelationForExtension(rel, ExclusiveLock);
pfree(pg); pfree(pg);
/* Update the relcache with the up-to-date size */
if (!InRecovery)
CacheInvalidateRelcache(rel);
rel->rd_vm_nblocks = vm_nblocks_now;
} }
...@@ -11,7 +11,7 @@ ...@@ -11,7 +11,7 @@
* Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* $PostgreSQL: pgsql/src/backend/access/transam/xlogutils.c,v 1.69 2010/01/02 16:57:35 momjian Exp $ * $PostgreSQL: pgsql/src/backend/access/transam/xlogutils.c,v 1.70 2010/02/09 21:43:29 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -393,9 +393,6 @@ CreateFakeRelcacheEntry(RelFileNode rnode) ...@@ -393,9 +393,6 @@ CreateFakeRelcacheEntry(RelFileNode rnode)
rel->rd_lockInfo.lockRelId.dbId = rnode.dbNode; rel->rd_lockInfo.lockRelId.dbId = rnode.dbNode;
rel->rd_lockInfo.lockRelId.relId = rnode.relNode; rel->rd_lockInfo.lockRelId.relId = rnode.relNode;
rel->rd_targblock = InvalidBlockNumber;
rel->rd_fsm_nblocks = InvalidBlockNumber;
rel->rd_vm_nblocks = InvalidBlockNumber;
rel->rd_smgr = NULL; rel->rd_smgr = NULL;
return rel; return rel;
......
...@@ -8,7 +8,7 @@ ...@@ -8,7 +8,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/catalog/storage.c,v 1.9 2010/02/08 19:59:49 heikki Exp $ * $PostgreSQL: pgsql/src/backend/catalog/storage.c,v 1.10 2010/02/09 21:43:30 tgl Exp $
* *
* NOTES * NOTES
* Some of this code used to be in storage/smgr/smgr.c, and the * Some of this code used to be in storage/smgr/smgr.c, and the
...@@ -226,8 +226,12 @@ RelationTruncate(Relation rel, BlockNumber nblocks) ...@@ -226,8 +226,12 @@ RelationTruncate(Relation rel, BlockNumber nblocks)
/* Open it at the smgr level if not already done */ /* Open it at the smgr level if not already done */
RelationOpenSmgr(rel); RelationOpenSmgr(rel);
/* Make sure rd_targblock isn't pointing somewhere past end */ /*
rel->rd_targblock = InvalidBlockNumber; * Make sure smgr_targblock etc aren't pointing somewhere past new end
*/
rel->rd_smgr->smgr_targblock = InvalidBlockNumber;
rel->rd_smgr->smgr_fsm_nblocks = InvalidBlockNumber;
rel->rd_smgr->smgr_vm_nblocks = InvalidBlockNumber;
/* Truncate the FSM first if it exists */ /* Truncate the FSM first if it exists */
fsm = smgrexists(rel->rd_smgr, FSM_FORKNUM); fsm = smgrexists(rel->rd_smgr, FSM_FORKNUM);
...@@ -459,6 +463,7 @@ smgr_redo(XLogRecPtr lsn, XLogRecord *record) ...@@ -459,6 +463,7 @@ smgr_redo(XLogRecPtr lsn, XLogRecord *record)
{ {
xl_smgr_truncate *xlrec = (xl_smgr_truncate *) XLogRecGetData(record); xl_smgr_truncate *xlrec = (xl_smgr_truncate *) XLogRecGetData(record);
SMgrRelation reln; SMgrRelation reln;
Relation rel;
reln = smgropen(xlrec->rnode); reln = smgropen(xlrec->rnode);
...@@ -475,14 +480,15 @@ smgr_redo(XLogRecPtr lsn, XLogRecord *record) ...@@ -475,14 +480,15 @@ smgr_redo(XLogRecPtr lsn, XLogRecord *record)
/* Also tell xlogutils.c about it */ /* Also tell xlogutils.c about it */
XLogTruncateRelation(xlrec->rnode, MAIN_FORKNUM, xlrec->blkno); XLogTruncateRelation(xlrec->rnode, MAIN_FORKNUM, xlrec->blkno);
/* Truncate FSM too */ /* Truncate FSM and VM too */
if (smgrexists(reln, FSM_FORKNUM)) rel = CreateFakeRelcacheEntry(xlrec->rnode);
{
Relation rel = CreateFakeRelcacheEntry(xlrec->rnode);
if (smgrexists(reln, FSM_FORKNUM))
FreeSpaceMapTruncateRel(rel, xlrec->blkno); FreeSpaceMapTruncateRel(rel, xlrec->blkno);
FreeFakeRelcacheEntry(rel); if (smgrexists(reln, VISIBILITYMAP_FORKNUM))
} visibilitymap_truncate(rel, xlrec->blkno);
FreeFakeRelcacheEntry(rel);
} }
else else
elog(PANIC, "smgr_redo: unknown op code %u", info); elog(PANIC, "smgr_redo: unknown op code %u", info);
......
...@@ -11,7 +11,7 @@ ...@@ -11,7 +11,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/commands/cluster.c,v 1.199 2010/02/07 22:40:33 tgl Exp $ * $PostgreSQL: pgsql/src/backend/commands/cluster.c,v 1.200 2010/02/09 21:43:30 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -38,6 +38,7 @@ ...@@ -38,6 +38,7 @@
#include "miscadmin.h" #include "miscadmin.h"
#include "storage/bufmgr.h" #include "storage/bufmgr.h"
#include "storage/procarray.h" #include "storage/procarray.h"
#include "storage/smgr.h"
#include "utils/acl.h" #include "utils/acl.h"
#include "utils/fmgroids.h" #include "utils/fmgroids.h"
#include "utils/inval.h" #include "utils/inval.h"
...@@ -559,16 +560,12 @@ mark_index_clustered(Relation rel, Oid indexOid) ...@@ -559,16 +560,12 @@ mark_index_clustered(Relation rel, Oid indexOid)
indexForm->indisclustered = false; indexForm->indisclustered = false;
simple_heap_update(pg_index, &indexTuple->t_self, indexTuple); simple_heap_update(pg_index, &indexTuple->t_self, indexTuple);
CatalogUpdateIndexes(pg_index, indexTuple); CatalogUpdateIndexes(pg_index, indexTuple);
/* Ensure we see the update in the index's relcache entry */
CacheInvalidateRelcacheByRelid(thisIndexOid);
} }
else if (thisIndexOid == indexOid) else if (thisIndexOid == indexOid)
{ {
indexForm->indisclustered = true; indexForm->indisclustered = true;
simple_heap_update(pg_index, &indexTuple->t_self, indexTuple); simple_heap_update(pg_index, &indexTuple->t_self, indexTuple);
CatalogUpdateIndexes(pg_index, indexTuple); CatalogUpdateIndexes(pg_index, indexTuple);
/* Ensure we see the update in the index's relcache entry */
CacheInvalidateRelcacheByRelid(thisIndexOid);
} }
heap_freetuple(indexTuple); heap_freetuple(indexTuple);
} }
...@@ -819,8 +816,8 @@ copy_heap_data(Oid OIDNewHeap, Oid OIDOldHeap, Oid OIDOldIndex, ...@@ -819,8 +816,8 @@ copy_heap_data(Oid OIDNewHeap, Oid OIDOldHeap, Oid OIDOldIndex,
XLogReportUnloggedStatement(reason); XLogReportUnloggedStatement(reason);
} }
/* use_wal off requires rd_targblock be initially invalid */ /* use_wal off requires smgr_targblock be initially invalid */
Assert(NewHeap->rd_targblock == InvalidBlockNumber); Assert(RelationGetTargetBlock(NewHeap) == InvalidBlockNumber);
/* /*
* If both tables have TOAST tables, perform toast swap by content. It is * If both tables have TOAST tables, perform toast swap by content. It is
......
...@@ -8,7 +8,7 @@ ...@@ -8,7 +8,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/commands/sequence.c,v 1.164 2010/01/02 16:57:37 momjian Exp $ * $PostgreSQL: pgsql/src/backend/commands/sequence.c,v 1.165 2010/02/09 21:43:30 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -29,6 +29,7 @@ ...@@ -29,6 +29,7 @@
#include "storage/bufmgr.h" #include "storage/bufmgr.h"
#include "storage/lmgr.h" #include "storage/lmgr.h"
#include "storage/proc.h" #include "storage/proc.h"
#include "storage/smgr.h"
#include "utils/acl.h" #include "utils/acl.h"
#include "utils/builtins.h" #include "utils/builtins.h"
#include "utils/lsyscache.h" #include "utils/lsyscache.h"
...@@ -220,7 +221,7 @@ DefineSequence(CreateSeqStmt *seq) ...@@ -220,7 +221,7 @@ DefineSequence(CreateSeqStmt *seq)
sm->magic = SEQ_MAGIC; sm->magic = SEQ_MAGIC;
/* hack: ensure heap_insert will insert on the just-created page */ /* hack: ensure heap_insert will insert on the just-created page */
rel->rd_targblock = 0; RelationSetTargetBlock(rel, 0);
/* Now form & insert sequence tuple */ /* Now form & insert sequence tuple */
tuple = heap_form_tuple(tupDesc, value, null); tuple = heap_form_tuple(tupDesc, value, null);
......
...@@ -14,7 +14,7 @@ ...@@ -14,7 +14,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/commands/vacuum.c,v 1.406 2010/02/08 16:50:21 tgl Exp $ * $PostgreSQL: pgsql/src/backend/commands/vacuum.c,v 1.407 2010/02/09 21:43:30 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -789,7 +789,6 @@ vacuum_rel(Oid relid, VacuumStmt *vacstmt, bool do_toast, bool for_wraparound, ...@@ -789,7 +789,6 @@ vacuum_rel(Oid relid, VacuumStmt *vacstmt, bool do_toast, bool for_wraparound,
Oid save_userid; Oid save_userid;
int save_sec_context; int save_sec_context;
int save_nestlevel; int save_nestlevel;
bool heldoff;
if (scanned_all) if (scanned_all)
*scanned_all = false; *scanned_all = false;
...@@ -970,10 +969,9 @@ vacuum_rel(Oid relid, VacuumStmt *vacstmt, bool do_toast, bool for_wraparound, ...@@ -970,10 +969,9 @@ vacuum_rel(Oid relid, VacuumStmt *vacstmt, bool do_toast, bool for_wraparound,
cluster_rel(relid, InvalidOid, false, cluster_rel(relid, InvalidOid, false,
(vacstmt->options & VACOPT_VERBOSE) != 0, (vacstmt->options & VACOPT_VERBOSE) != 0,
vacstmt->freeze_min_age, vacstmt->freeze_table_age); vacstmt->freeze_min_age, vacstmt->freeze_table_age);
heldoff = false;
} }
else else
heldoff = lazy_vacuum_rel(onerel, vacstmt, vac_strategy, scanned_all); lazy_vacuum_rel(onerel, vacstmt, vac_strategy, scanned_all);
/* Roll back any GUC changes executed by index functions */ /* Roll back any GUC changes executed by index functions */
AtEOXact_GUC(false, save_nestlevel); AtEOXact_GUC(false, save_nestlevel);
...@@ -991,10 +989,6 @@ vacuum_rel(Oid relid, VacuumStmt *vacstmt, bool do_toast, bool for_wraparound, ...@@ -991,10 +989,6 @@ vacuum_rel(Oid relid, VacuumStmt *vacstmt, bool do_toast, bool for_wraparound,
PopActiveSnapshot(); PopActiveSnapshot();
CommitTransactionCommand(); CommitTransactionCommand();
/* now we can allow interrupts again, if disabled */
if (heldoff)
RESUME_INTERRUPTS();
/* /*
* If the relation has a secondary toast rel, vacuum that too while we * If the relation has a secondary toast rel, vacuum that too while we
* still hold the session lock on the master table. Note however that * still hold the session lock on the master table. Note however that
......
...@@ -29,7 +29,7 @@ ...@@ -29,7 +29,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/commands/vacuumlazy.c,v 1.130 2010/02/09 00:28:30 tgl Exp $ * $PostgreSQL: pgsql/src/backend/commands/vacuumlazy.c,v 1.131 2010/02/09 21:43:30 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -50,7 +50,6 @@ ...@@ -50,7 +50,6 @@
#include "storage/bufmgr.h" #include "storage/bufmgr.h"
#include "storage/freespace.h" #include "storage/freespace.h"
#include "storage/lmgr.h" #include "storage/lmgr.h"
#include "utils/inval.h"
#include "utils/lsyscache.h" #include "utils/lsyscache.h"
#include "utils/memutils.h" #include "utils/memutils.h"
#include "utils/pg_rusage.h" #include "utils/pg_rusage.h"
...@@ -123,7 +122,7 @@ static void lazy_cleanup_index(Relation indrel, ...@@ -123,7 +122,7 @@ static void lazy_cleanup_index(Relation indrel,
LVRelStats *vacrelstats); LVRelStats *vacrelstats);
static int lazy_vacuum_page(Relation onerel, BlockNumber blkno, Buffer buffer, static int lazy_vacuum_page(Relation onerel, BlockNumber blkno, Buffer buffer,
int tupindex, LVRelStats *vacrelstats); int tupindex, LVRelStats *vacrelstats);
static bool lazy_truncate_heap(Relation onerel, LVRelStats *vacrelstats); static void lazy_truncate_heap(Relation onerel, LVRelStats *vacrelstats);
static BlockNumber count_nondeletable_pages(Relation onerel, static BlockNumber count_nondeletable_pages(Relation onerel,
LVRelStats *vacrelstats); LVRelStats *vacrelstats);
static void lazy_space_alloc(LVRelStats *vacrelstats, BlockNumber relblocks); static void lazy_space_alloc(LVRelStats *vacrelstats, BlockNumber relblocks);
...@@ -141,11 +140,8 @@ static int vac_cmp_itemptr(const void *left, const void *right); ...@@ -141,11 +140,8 @@ static int vac_cmp_itemptr(const void *left, const void *right);
* *
* At entry, we have already established a transaction and opened * At entry, we have already established a transaction and opened
* and locked the relation. * and locked the relation.
*
* The return value indicates whether this function has held off
* interrupts -- if true, caller must RESUME_INTERRUPTS() after commit.
*/ */
bool void
lazy_vacuum_rel(Relation onerel, VacuumStmt *vacstmt, lazy_vacuum_rel(Relation onerel, VacuumStmt *vacstmt,
BufferAccessStrategy bstrategy, bool *scanned_all) BufferAccessStrategy bstrategy, bool *scanned_all)
{ {
...@@ -157,7 +153,6 @@ lazy_vacuum_rel(Relation onerel, VacuumStmt *vacstmt, ...@@ -157,7 +153,6 @@ lazy_vacuum_rel(Relation onerel, VacuumStmt *vacstmt,
TimestampTz starttime = 0; TimestampTz starttime = 0;
bool scan_all; bool scan_all;
TransactionId freezeTableLimit; TransactionId freezeTableLimit;
bool heldoff = false;
pg_rusage_init(&ru0); pg_rusage_init(&ru0);
...@@ -194,16 +189,9 @@ lazy_vacuum_rel(Relation onerel, VacuumStmt *vacstmt, ...@@ -194,16 +189,9 @@ lazy_vacuum_rel(Relation onerel, VacuumStmt *vacstmt,
/* Done with indexes */ /* Done with indexes */
vac_close_indexes(nindexes, Irel, NoLock); vac_close_indexes(nindexes, Irel, NoLock);
/* Vacuum the Free Space Map */
FreeSpaceMapVacuum(onerel);
/* /*
* Optionally truncate the relation. * Optionally truncate the relation.
* *
* NB: there should be as little code as possible after this point,
* to minimize the chance of failure as well as the time spent ignoring
* cancel/die interrupts.
*
* Don't even think about it unless we have a shot at releasing a goodly * Don't even think about it unless we have a shot at releasing a goodly
* number of pages. Otherwise, the time taken isn't worth it. * number of pages. Otherwise, the time taken isn't worth it.
*/ */
...@@ -211,7 +199,10 @@ lazy_vacuum_rel(Relation onerel, VacuumStmt *vacstmt, ...@@ -211,7 +199,10 @@ lazy_vacuum_rel(Relation onerel, VacuumStmt *vacstmt,
if (possibly_freeable > 0 && if (possibly_freeable > 0 &&
(possibly_freeable >= REL_TRUNCATE_MINIMUM || (possibly_freeable >= REL_TRUNCATE_MINIMUM ||
possibly_freeable >= vacrelstats->rel_pages / REL_TRUNCATE_FRACTION)) possibly_freeable >= vacrelstats->rel_pages / REL_TRUNCATE_FRACTION))
heldoff = lazy_truncate_heap(onerel, vacrelstats); lazy_truncate_heap(onerel, vacrelstats);
/* Vacuum the Free Space Map */
FreeSpaceMapVacuum(onerel);
/* /*
* Update statistics in pg_class. But only if we didn't skip any pages; * Update statistics in pg_class. But only if we didn't skip any pages;
...@@ -255,8 +246,6 @@ lazy_vacuum_rel(Relation onerel, VacuumStmt *vacstmt, ...@@ -255,8 +246,6 @@ lazy_vacuum_rel(Relation onerel, VacuumStmt *vacstmt,
if (scanned_all) if (scanned_all)
*scanned_all = vacrelstats->scanned_all; *scanned_all = vacrelstats->scanned_all;
return heldoff;
} }
/* /*
...@@ -996,11 +985,8 @@ lazy_cleanup_index(Relation indrel, ...@@ -996,11 +985,8 @@ lazy_cleanup_index(Relation indrel,
/* /*
* lazy_truncate_heap - try to truncate off any empty pages at the end * lazy_truncate_heap - try to truncate off any empty pages at the end
*
* The return value indicates whether this function has held off
* interrupts -- if true, caller must RESUME_INTERRUPTS() after commit.
*/ */
static bool static void
lazy_truncate_heap(Relation onerel, LVRelStats *vacrelstats) lazy_truncate_heap(Relation onerel, LVRelStats *vacrelstats)
{ {
BlockNumber old_rel_pages = vacrelstats->rel_pages; BlockNumber old_rel_pages = vacrelstats->rel_pages;
...@@ -1016,7 +1002,7 @@ lazy_truncate_heap(Relation onerel, LVRelStats *vacrelstats) ...@@ -1016,7 +1002,7 @@ lazy_truncate_heap(Relation onerel, LVRelStats *vacrelstats)
* possible considering we already hold a lower-grade lock). * possible considering we already hold a lower-grade lock).
*/ */
if (!ConditionalLockRelation(onerel, AccessExclusiveLock)) if (!ConditionalLockRelation(onerel, AccessExclusiveLock))
return false; return;
/* /*
* Now that we have exclusive lock, look to see if the rel has grown * Now that we have exclusive lock, look to see if the rel has grown
...@@ -1029,7 +1015,7 @@ lazy_truncate_heap(Relation onerel, LVRelStats *vacrelstats) ...@@ -1029,7 +1015,7 @@ lazy_truncate_heap(Relation onerel, LVRelStats *vacrelstats)
/* might as well use the latest news when we update pg_class stats */ /* might as well use the latest news when we update pg_class stats */
vacrelstats->rel_pages = new_rel_pages; vacrelstats->rel_pages = new_rel_pages;
UnlockRelation(onerel, AccessExclusiveLock); UnlockRelation(onerel, AccessExclusiveLock);
return false; return;
} }
/* /*
...@@ -1044,34 +1030,22 @@ lazy_truncate_heap(Relation onerel, LVRelStats *vacrelstats) ...@@ -1044,34 +1030,22 @@ lazy_truncate_heap(Relation onerel, LVRelStats *vacrelstats)
{ {
/* can't do anything after all */ /* can't do anything after all */
UnlockRelation(onerel, AccessExclusiveLock); UnlockRelation(onerel, AccessExclusiveLock);
return false; return;
} }
/* /*
* Prevent cancel/die interrupts from now till commit. Once we have * Okay to truncate.
* truncated, it is essential that we send the sinval message before
* releasing exclusive lock on the relation; both of which will
* happen during commit. Other backends must receive the sinval
* message to reset their rd_targblock values before they can safely
* write to the table again. While we can't positively guarantee
* no error before commit, we can at least prevent cancel interrupts.
*
* XXX it would be better if we had a way to send the inval message
* nontransactionally; an error after the truncate will mean that the
* message is lost. Note however that turning this all into a critical
* section would not be an improvement. Making it critical would mean
* that an error forces PANIC, whereas losing the sinval will at worst
* cause unexpected nonfatal errors in other sessions.
*/ */
HOLD_INTERRUPTS(); RelationTruncate(onerel, new_rel_pages);
/* force relcache inval so all backends reset their rd_targblock */
CacheInvalidateRelcache(onerel);
/* /*
* Okay to truncate. Do as little as possible between here and commit. * We can release the exclusive lock as soon as we have truncated. Other
* backends can't safely access the relation until they have processed the
* smgr invalidation that smgrtruncate sent out ... but that should happen
* as part of standard invalidation processing once they acquire lock on
* the relation.
*/ */
RelationTruncate(onerel, new_rel_pages); UnlockRelation(onerel, AccessExclusiveLock);
/* update statistics */ /* update statistics */
vacrelstats->rel_pages = new_rel_pages; vacrelstats->rel_pages = new_rel_pages;
...@@ -1083,8 +1057,6 @@ lazy_truncate_heap(Relation onerel, LVRelStats *vacrelstats) ...@@ -1083,8 +1057,6 @@ lazy_truncate_heap(Relation onerel, LVRelStats *vacrelstats)
old_rel_pages, new_rel_pages), old_rel_pages, new_rel_pages),
errdetail("%s.", errdetail("%s.",
pg_rusage_show(&ru0)))); pg_rusage_show(&ru0))));
return true; /* interrupts are held off */
} }
/* /*
......
...@@ -26,7 +26,7 @@ ...@@ -26,7 +26,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/executor/execMain.c,v 1.345 2010/02/07 20:48:10 tgl Exp $ * $PostgreSQL: pgsql/src/backend/executor/execMain.c,v 1.346 2010/02/09 21:43:30 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -49,6 +49,7 @@ ...@@ -49,6 +49,7 @@
#include "parser/parsetree.h" #include "parser/parsetree.h"
#include "storage/bufmgr.h" #include "storage/bufmgr.h"
#include "storage/lmgr.h" #include "storage/lmgr.h"
#include "storage/smgr.h"
#include "utils/acl.h" #include "utils/acl.h"
#include "utils/lsyscache.h" #include "utils/lsyscache.h"
#include "utils/memutils.h" #include "utils/memutils.h"
...@@ -2222,8 +2223,8 @@ OpenIntoRel(QueryDesc *queryDesc) ...@@ -2222,8 +2223,8 @@ OpenIntoRel(QueryDesc *queryDesc)
(XLogIsNeeded() ? 0 : HEAP_INSERT_SKIP_WAL); (XLogIsNeeded() ? 0 : HEAP_INSERT_SKIP_WAL);
myState->bistate = GetBulkInsertState(); myState->bistate = GetBulkInsertState();
/* Not using WAL requires rd_targblock be initially invalid */ /* Not using WAL requires smgr_targblock be initially invalid */
Assert(intoRelationDesc->rd_targblock == InvalidBlockNumber); Assert(RelationGetTargetBlock(intoRelationDesc) == InvalidBlockNumber);
} }
/* /*
......
...@@ -8,7 +8,7 @@ ...@@ -8,7 +8,7 @@
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/storage/freespace/freespace.c,v 1.75 2010/02/09 00:28:57 tgl Exp $ * $PostgreSQL: pgsql/src/backend/storage/freespace/freespace.c,v 1.76 2010/02/09 21:43:30 tgl Exp $
* *
* *
* NOTES: * NOTES:
...@@ -25,16 +25,16 @@ ...@@ -25,16 +25,16 @@
#include "access/htup.h" #include "access/htup.h"
#include "access/xlogutils.h" #include "access/xlogutils.h"
#include "storage/bufpage.h" #include "miscadmin.h"
#include "storage/bufmgr.h" #include "storage/bufmgr.h"
#include "storage/bufpage.h"
#include "storage/freespace.h" #include "storage/freespace.h"
#include "storage/fsm_internals.h" #include "storage/fsm_internals.h"
#include "storage/lmgr.h" #include "storage/lmgr.h"
#include "storage/lwlock.h" #include "storage/lwlock.h"
#include "storage/smgr.h" #include "storage/smgr.h"
#include "utils/rel.h" #include "utils/rel.h"
#include "utils/inval.h"
#include "miscadmin.h"
/* /*
* We use just one byte to store the amount of free space on a page, so we * We use just one byte to store the amount of free space on a page, so we
...@@ -251,9 +251,9 @@ GetRecordedFreeSpace(Relation rel, BlockNumber heapBlk) ...@@ -251,9 +251,9 @@ GetRecordedFreeSpace(Relation rel, BlockNumber heapBlk)
/* /*
* FreeSpaceMapTruncateRel - adjust for truncation of a relation. * FreeSpaceMapTruncateRel - adjust for truncation of a relation.
* *
* The caller must hold AccessExclusiveLock on the relation, to ensure * The caller must hold AccessExclusiveLock on the relation, to ensure that
* that other backends receive the relcache invalidation event that this * other backends receive the smgr invalidation event that this function sends
* function sends, before accessing the FSM again. * before they access the FSM again.
* *
* nblocks is the new size of the heap. * nblocks is the new size of the heap.
*/ */
...@@ -302,17 +302,18 @@ FreeSpaceMapTruncateRel(Relation rel, BlockNumber nblocks) ...@@ -302,17 +302,18 @@ FreeSpaceMapTruncateRel(Relation rel, BlockNumber nblocks)
return; /* nothing to do; the FSM was already smaller */ return; /* nothing to do; the FSM was already smaller */
} }
/* Truncate the unused FSM pages */ /* Truncate the unused FSM pages, and send smgr inval message */
smgrtruncate(rel->rd_smgr, FSM_FORKNUM, new_nfsmblocks, rel->rd_istemp); smgrtruncate(rel->rd_smgr, FSM_FORKNUM, new_nfsmblocks, rel->rd_istemp);
/* /*
* Need to invalidate the relcache entry, because rd_fsm_nblocks seen by * We might as well update the local smgr_fsm_nblocks setting.
* other backends is no longer valid. * smgrtruncate sent an smgr cache inval message, which will cause
* other backends to invalidate their copy of smgr_fsm_nblocks, and
* this one too at the next command boundary. But this ensures it
* isn't outright wrong until then.
*/ */
if (!InRecovery) if (rel->rd_smgr)
CacheInvalidateRelcache(rel); rel->rd_smgr->smgr_fsm_nblocks = new_nfsmblocks;
rel->rd_fsm_nblocks = new_nfsmblocks;
} }
/* /*
...@@ -506,17 +507,24 @@ fsm_readbuf(Relation rel, FSMAddress addr, bool extend) ...@@ -506,17 +507,24 @@ fsm_readbuf(Relation rel, FSMAddress addr, bool extend)
RelationOpenSmgr(rel); RelationOpenSmgr(rel);
/* If we haven't cached the size of the FSM yet, check it first */ /*
if (rel->rd_fsm_nblocks == InvalidBlockNumber) * If we haven't cached the size of the FSM yet, check it first. Also
* recheck if the requested block seems to be past end, since our
* cached value might be stale. (We send smgr inval messages on
* truncation, but not on extension.)
*/
if (rel->rd_smgr->smgr_fsm_nblocks == InvalidBlockNumber ||
blkno >= rel->rd_smgr->smgr_fsm_nblocks)
{ {
if (smgrexists(rel->rd_smgr, FSM_FORKNUM)) if (smgrexists(rel->rd_smgr, FSM_FORKNUM))
rel->rd_fsm_nblocks = smgrnblocks(rel->rd_smgr, FSM_FORKNUM); rel->rd_smgr->smgr_fsm_nblocks = smgrnblocks(rel->rd_smgr,
FSM_FORKNUM);
else else
rel->rd_fsm_nblocks = 0; rel->rd_smgr->smgr_fsm_nblocks = 0;
} }
/* Handle requests beyond EOF */ /* Handle requests beyond EOF */
if (blkno >= rel->rd_fsm_nblocks) if (blkno >= rel->rd_smgr->smgr_fsm_nblocks)
{ {
if (extend) if (extend)
fsm_extend(rel, blkno + 1); fsm_extend(rel, blkno + 1);
...@@ -559,19 +567,23 @@ fsm_extend(Relation rel, BlockNumber fsm_nblocks) ...@@ -559,19 +567,23 @@ fsm_extend(Relation rel, BlockNumber fsm_nblocks)
* it. * it.
* *
* Note that another backend might have extended or created the relation * Note that another backend might have extended or created the relation
* before we get the lock. * by the time we get the lock.
*/ */
LockRelationForExtension(rel, ExclusiveLock); LockRelationForExtension(rel, ExclusiveLock);
/* Create the FSM file first if it doesn't exist */ /* Might have to re-open if a cache flush happened */
if ((rel->rd_fsm_nblocks == 0 || rel->rd_fsm_nblocks == InvalidBlockNumber) RelationOpenSmgr(rel);
&& !smgrexists(rel->rd_smgr, FSM_FORKNUM))
{ /*
* Create the FSM file first if it doesn't exist. If smgr_fsm_nblocks
* is positive then it must exist, no need for an smgrexists call.
*/
if ((rel->rd_smgr->smgr_fsm_nblocks == 0 ||
rel->rd_smgr->smgr_fsm_nblocks == InvalidBlockNumber) &&
!smgrexists(rel->rd_smgr, FSM_FORKNUM))
smgrcreate(rel->rd_smgr, FSM_FORKNUM, false); smgrcreate(rel->rd_smgr, FSM_FORKNUM, false);
fsm_nblocks_now = 0;
} fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
else
fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
while (fsm_nblocks_now < fsm_nblocks) while (fsm_nblocks_now < fsm_nblocks)
{ {
...@@ -580,14 +592,12 @@ fsm_extend(Relation rel, BlockNumber fsm_nblocks) ...@@ -580,14 +592,12 @@ fsm_extend(Relation rel, BlockNumber fsm_nblocks)
fsm_nblocks_now++; fsm_nblocks_now++;
} }
/* Update local cache with the up-to-date size */
rel->rd_smgr->smgr_fsm_nblocks = fsm_nblocks_now;
UnlockRelationForExtension(rel, ExclusiveLock); UnlockRelationForExtension(rel, ExclusiveLock);
pfree(pg); pfree(pg);
/* Update the relcache with the up-to-date size */
if (!InRecovery)
CacheInvalidateRelcache(rel);
rel->rd_fsm_nblocks = fsm_nblocks_now;
} }
/* /*
......
...@@ -11,7 +11,7 @@ ...@@ -11,7 +11,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/storage/smgr/smgr.c,v 1.119 2010/02/03 01:14:17 tgl Exp $ * $PostgreSQL: pgsql/src/backend/storage/smgr/smgr.c,v 1.120 2010/02/09 21:43:30 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -161,6 +161,9 @@ smgropen(RelFileNode rnode) ...@@ -161,6 +161,9 @@ smgropen(RelFileNode rnode)
/* hash_search already filled in the lookup key */ /* hash_search already filled in the lookup key */
reln->smgr_owner = NULL; reln->smgr_owner = NULL;
reln->smgr_targblock = InvalidBlockNumber;
reln->smgr_fsm_nblocks = InvalidBlockNumber;
reln->smgr_vm_nblocks = InvalidBlockNumber;
reln->smgr_which = 0; /* we only have md.c at present */ reln->smgr_which = 0; /* we only have md.c at present */
/* mark it not open */ /* mark it not open */
...@@ -351,6 +354,16 @@ smgr_internal_unlink(RelFileNode rnode, ForkNumber forknum, ...@@ -351,6 +354,16 @@ smgr_internal_unlink(RelFileNode rnode, ForkNumber forknum,
* anyway). * anyway).
*/ */
/*
* Send a shared-inval message to force other backends to close any
* dangling smgr references they may have for this rel. We should do
* this before starting the actual unlinking, in case we fail partway
* through that step. Note that the sinval message will eventually come
* back to this backend, too, and thereby provide a backstop that we
* closed our own smgr rel.
*/
CacheInvalidateSmgr(rnode);
/* /*
* Delete the physical file(s). * Delete the physical file(s).
* *
...@@ -359,14 +372,6 @@ smgr_internal_unlink(RelFileNode rnode, ForkNumber forknum, ...@@ -359,14 +372,6 @@ smgr_internal_unlink(RelFileNode rnode, ForkNumber forknum,
* xact. * xact.
*/ */
(*(smgrsw[which].smgr_unlink)) (rnode, forknum, isRedo); (*(smgrsw[which].smgr_unlink)) (rnode, forknum, isRedo);
/*
* Lastly, send a shared-inval message to force other backends to close
* any dangling smgr references they may have for this rel. We do this
* last because the sinval will eventually come back to this backend, too,
* and thereby provide a backstop that we closed our own smgr rel.
*/
CacheInvalidateSmgr(rnode);
} }
/* /*
...@@ -459,21 +464,23 @@ smgrtruncate(SMgrRelation reln, ForkNumber forknum, BlockNumber nblocks, ...@@ -459,21 +464,23 @@ smgrtruncate(SMgrRelation reln, ForkNumber forknum, BlockNumber nblocks,
*/ */
DropRelFileNodeBuffers(reln->smgr_rnode, forknum, isTemp, nblocks); DropRelFileNodeBuffers(reln->smgr_rnode, forknum, isTemp, nblocks);
/*
* Do the truncation.
*/
(*(smgrsw[reln->smgr_which].smgr_truncate)) (reln, forknum, nblocks,
isTemp);
/* /*
* Send a shared-inval message to force other backends to close any smgr * Send a shared-inval message to force other backends to close any smgr
* references they may have for this rel. This is useful because they * references they may have for this rel. This is useful because they
* might have open file pointers to segments that got removed. (The inval * might have open file pointers to segments that got removed, and/or
* smgr_targblock variables pointing past the new rel end. (The inval
* message will come back to our backend, too, causing a * message will come back to our backend, too, causing a
* probably-unnecessary smgr flush. But we don't expect that this is * probably-unnecessary local smgr flush. But we don't expect that this
* a performance-critical path.) * is a performance-critical path.) As in the unlink code, we want to
* be sure the message is sent before we start changing things on-disk.
*/ */
CacheInvalidateSmgr(reln->smgr_rnode); CacheInvalidateSmgr(reln->smgr_rnode);
/*
* Do the truncation.
*/
(*(smgrsw[reln->smgr_which].smgr_truncate)) (reln, forknum, nblocks,
isTemp);
} }
/* /*
......
...@@ -8,7 +8,7 @@ ...@@ -8,7 +8,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/utils/cache/relcache.c,v 1.304 2010/02/08 05:53:55 tgl Exp $ * $PostgreSQL: pgsql/src/backend/utils/cache/relcache.c,v 1.305 2010/02/09 21:43:30 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -328,13 +328,6 @@ AllocateRelationDesc(Form_pg_class relp) ...@@ -328,13 +328,6 @@ AllocateRelationDesc(Form_pg_class relp)
*/ */
relation = (Relation) palloc0(sizeof(RelationData)); relation = (Relation) palloc0(sizeof(RelationData));
/*
* clear fields of reldesc that should initialize to something non-zero
*/
relation->rd_targblock = InvalidBlockNumber;
relation->rd_fsm_nblocks = InvalidBlockNumber;
relation->rd_vm_nblocks = InvalidBlockNumber;
/* make sure relation is marked as having no open file yet */ /* make sure relation is marked as having no open file yet */
relation->rd_smgr = NULL; relation->rd_smgr = NULL;
...@@ -1413,9 +1406,6 @@ formrdesc(const char *relationName, Oid relationReltype, ...@@ -1413,9 +1406,6 @@ formrdesc(const char *relationName, Oid relationReltype,
* allocate new relation desc, clear all fields of reldesc * allocate new relation desc, clear all fields of reldesc
*/ */
relation = (Relation) palloc0(sizeof(RelationData)); relation = (Relation) palloc0(sizeof(RelationData));
relation->rd_targblock = InvalidBlockNumber;
relation->rd_fsm_nblocks = InvalidBlockNumber;
relation->rd_vm_nblocks = InvalidBlockNumber;
/* make sure relation is marked as having no open file yet */ /* make sure relation is marked as having no open file yet */
relation->rd_smgr = NULL; relation->rd_smgr = NULL;
...@@ -1714,14 +1704,7 @@ RelationReloadIndexInfo(Relation relation) ...@@ -1714,14 +1704,7 @@ RelationReloadIndexInfo(Relation relation)
/* Should be closed at smgr level */ /* Should be closed at smgr level */
Assert(relation->rd_smgr == NULL); Assert(relation->rd_smgr == NULL);
/* /* Must free any AM cached data upon relcache flush */
* Must reset targblock, fsm_nblocks and vm_nblocks in case rel was
* truncated
*/
relation->rd_targblock = InvalidBlockNumber;
relation->rd_fsm_nblocks = InvalidBlockNumber;
relation->rd_vm_nblocks = InvalidBlockNumber;
/* Must free any AM cached data, too */
if (relation->rd_amcache) if (relation->rd_amcache)
pfree(relation->rd_amcache); pfree(relation->rd_amcache);
relation->rd_amcache = NULL; relation->rd_amcache = NULL;
...@@ -1867,11 +1850,8 @@ RelationClearRelation(Relation relation, bool rebuild) ...@@ -1867,11 +1850,8 @@ RelationClearRelation(Relation relation, bool rebuild)
/* /*
* Never, never ever blow away a nailed-in system relation, because we'd * Never, never ever blow away a nailed-in system relation, because we'd
* be unable to recover. However, we must reset rd_targblock, in case we * be unable to recover. However, we must redo RelationInitPhysicalAddr
* got called because of a relation cache flush that was triggered by * in case it is a mapped relation whose mapping changed.
* VACUUM. Likewise reset the fsm and vm size info. Also, redo
* RelationInitPhysicalAddr in case it is a mapped relation whose mapping
* changed.
* *
* If it's a nailed index, then we need to re-read the pg_class row to see * If it's a nailed index, then we need to re-read the pg_class row to see
* if its relfilenode changed. We can't necessarily do that here, because * if its relfilenode changed. We can't necessarily do that here, because
...@@ -1882,10 +1862,6 @@ RelationClearRelation(Relation relation, bool rebuild) ...@@ -1882,10 +1862,6 @@ RelationClearRelation(Relation relation, bool rebuild)
*/ */
if (relation->rd_isnailed) if (relation->rd_isnailed)
{ {
relation->rd_targblock = InvalidBlockNumber;
relation->rd_fsm_nblocks = InvalidBlockNumber;
relation->rd_vm_nblocks = InvalidBlockNumber;
/* We must recalculate physical address in case it changed */
RelationInitPhysicalAddr(relation); RelationInitPhysicalAddr(relation);
if (relation->rd_rel->relkind == RELKIND_INDEX) if (relation->rd_rel->relkind == RELKIND_INDEX)
...@@ -2502,10 +2478,6 @@ RelationBuildLocalRelation(const char *relname, ...@@ -2502,10 +2478,6 @@ RelationBuildLocalRelation(const char *relname,
*/ */
rel = (Relation) palloc0(sizeof(RelationData)); rel = (Relation) palloc0(sizeof(RelationData));
rel->rd_targblock = InvalidBlockNumber;
rel->rd_fsm_nblocks = InvalidBlockNumber;
rel->rd_vm_nblocks = InvalidBlockNumber;
/* make sure relation is marked as having no open file yet */ /* make sure relation is marked as having no open file yet */
rel->rd_smgr = NULL; rel->rd_smgr = NULL;
...@@ -4169,9 +4141,6 @@ load_relcache_init_file(bool shared) ...@@ -4169,9 +4141,6 @@ load_relcache_init_file(bool shared)
* Reset transient-state fields in the relcache entry * Reset transient-state fields in the relcache entry
*/ */
rel->rd_smgr = NULL; rel->rd_smgr = NULL;
rel->rd_targblock = InvalidBlockNumber;
rel->rd_fsm_nblocks = InvalidBlockNumber;
rel->rd_vm_nblocks = InvalidBlockNumber;
if (rel->rd_isnailed) if (rel->rd_isnailed)
rel->rd_refcnt = 1; rel->rd_refcnt = 1;
else else
......
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* $PostgreSQL: pgsql/src/include/commands/vacuum.h,v 1.88 2010/02/08 04:33:54 tgl Exp $ * $PostgreSQL: pgsql/src/include/commands/vacuum.h,v 1.89 2010/02/09 21:43:30 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -145,7 +145,7 @@ extern void vac_update_datfrozenxid(void); ...@@ -145,7 +145,7 @@ extern void vac_update_datfrozenxid(void);
extern void vacuum_delay_point(void); extern void vacuum_delay_point(void);
/* in commands/vacuumlazy.c */ /* in commands/vacuumlazy.c */
extern bool lazy_vacuum_rel(Relation onerel, VacuumStmt *vacstmt, extern void lazy_vacuum_rel(Relation onerel, VacuumStmt *vacstmt,
BufferAccessStrategy bstrategy, bool *scanned_all); BufferAccessStrategy bstrategy, bool *scanned_all);
/* in commands/analyze.c */ /* in commands/analyze.c */
......
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* $PostgreSQL: pgsql/src/include/storage/smgr.h,v 1.69 2010/01/02 16:58:08 momjian Exp $ * $PostgreSQL: pgsql/src/include/storage/smgr.h,v 1.70 2010/02/09 21:43:30 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -43,6 +43,17 @@ typedef struct SMgrRelationData ...@@ -43,6 +43,17 @@ typedef struct SMgrRelationData
/* pointer to owning pointer, or NULL if none */ /* pointer to owning pointer, or NULL if none */
struct SMgrRelationData **smgr_owner; struct SMgrRelationData **smgr_owner;
/*
* These next three fields are not actually used or manipulated by smgr,
* except that they are reset to InvalidBlockNumber upon a cache flush
* event (in particular, upon truncation of the relation). Higher levels
* store cached state here so that it will be reset when truncation
* happens. In all three cases, InvalidBlockNumber means "unknown".
*/
BlockNumber smgr_targblock; /* current insertion target block */
BlockNumber smgr_fsm_nblocks; /* last known size of fsm fork */
BlockNumber smgr_vm_nblocks; /* last known size of vm fork */
/* additional public fields may someday exist here */ /* additional public fields may someday exist here */
/* /*
......
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* $PostgreSQL: pgsql/src/include/utils/rel.h,v 1.122 2010/02/07 20:48:13 tgl Exp $ * $PostgreSQL: pgsql/src/include/utils/rel.h,v 1.123 2010/02/09 21:43:30 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -125,8 +125,6 @@ typedef struct RelationData ...@@ -125,8 +125,6 @@ typedef struct RelationData
RelFileNode rd_node; /* relation physical identifier */ RelFileNode rd_node; /* relation physical identifier */
/* use "struct" here to avoid needing to include smgr.h: */ /* use "struct" here to avoid needing to include smgr.h: */
struct SMgrRelationData *rd_smgr; /* cached file handle, or NULL */ struct SMgrRelationData *rd_smgr; /* cached file handle, or NULL */
BlockNumber rd_targblock; /* current insertion target block, or
* InvalidBlockNumber */
int rd_refcnt; /* reference count */ int rd_refcnt; /* reference count */
bool rd_istemp; /* rel is a temporary relation */ bool rd_istemp; /* rel is a temporary relation */
bool rd_islocaltemp; /* rel is a temp rel of this session */ bool rd_islocaltemp; /* rel is a temp rel of this session */
...@@ -212,13 +210,6 @@ typedef struct RelationData ...@@ -212,13 +210,6 @@ typedef struct RelationData
*/ */
Oid rd_toastoid; /* Real TOAST table's OID, or InvalidOid */ Oid rd_toastoid; /* Real TOAST table's OID, or InvalidOid */
/*
* sizes of the free space and visibility map forks, or InvalidBlockNumber
* if not known yet
*/
BlockNumber rd_fsm_nblocks;
BlockNumber rd_vm_nblocks;
/* use "struct" here to avoid needing to include pgstat.h: */ /* use "struct" here to avoid needing to include pgstat.h: */
struct PgStat_TableStatus *pgstat_info; /* statistics collection area */ struct PgStat_TableStatus *pgstat_info; /* statistics collection area */
} RelationData; } RelationData;
...@@ -374,6 +365,26 @@ typedef struct StdRdOptions ...@@ -374,6 +365,26 @@ typedef struct StdRdOptions
} \ } \
} while (0) } while (0)
/*
* RelationGetTargetBlock
* Fetch relation's current insertion target block.
*
* Returns InvalidBlockNumber if there is no current target block. Note
* that the target block status is discarded on any smgr-level invalidation.
*/
#define RelationGetTargetBlock(relation) \
( (relation)->rd_smgr != NULL ? (relation)->rd_smgr->smgr_targblock : InvalidBlockNumber )
/*
* RelationSetTargetBlock
* Set relation's current insertion target block.
*/
#define RelationSetTargetBlock(relation, targblock) \
do { \
RelationOpenSmgr(relation); \
(relation)->rd_smgr->smgr_targblock = (targblock); \
} while (0)
/* /*
* RELATION_IS_LOCAL * RELATION_IS_LOCAL
* If a rel is either temp or newly created in the current transaction, * If a rel is either temp or newly created in the current transaction,
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册