aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--src/backend/catalog/heap.c55
-rw-r--r--src/backend/catalog/index.c125
-rw-r--r--src/backend/catalog/toasting.c10
-rw-r--r--src/backend/commands/tablecmds.c6
-rw-r--r--src/backend/storage/smgr/smgr.c29
-rw-r--r--src/backend/utils/cache/inval.c130
-rw-r--r--src/backend/utils/cache/relcache.c123
-rw-r--r--src/include/catalog/index.h4
-rw-r--r--src/include/utils/inval.h5
-rw-r--r--src/include/utils/relcache.h10
10 files changed, 250 insertions, 247 deletions
diff --git a/src/backend/catalog/heap.c b/src/backend/catalog/heap.c
index f86747e149e..c344b8e01cb 100644
--- a/src/backend/catalog/heap.c
+++ b/src/backend/catalog/heap.c
@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/catalog/heap.c,v 1.368 2010/01/28 23:21:11 petere Exp $
+ * $PostgreSQL: pgsql/src/backend/catalog/heap.c,v 1.369 2010/02/03 01:14:16 tgl Exp $
*
*
* INTERFACE ROUTINES
@@ -70,6 +70,10 @@
#include "utils/tqual.h"
+/* Kluge for upgrade-in-place support */
+Oid binary_upgrade_next_heap_relfilenode = InvalidOid;
+Oid binary_upgrade_next_toast_relfilenode = InvalidOid;
+
static void AddNewRelationTuple(Relation pg_class_desc,
Relation new_rel_desc,
Oid new_rel_oid,
@@ -98,9 +102,6 @@ static Node *cookConstraint(ParseState *pstate,
char *relname);
static List *insert_ordered_unique_oid(List *list, Oid datum);
-Oid binary_upgrade_next_heap_relfilenode = InvalidOid;
-Oid binary_upgrade_next_toast_relfilenode = InvalidOid;
-
/* ----------------------------------------------------------------
* XXX UGLY HARD CODED BADNESS FOLLOWS XXX
@@ -955,29 +956,31 @@ heap_create_with_catalog(const char *relname,
errmsg("only shared relations can be placed in pg_global tablespace")));
}
- if ((relkind == RELKIND_RELATION || relkind == RELKIND_SEQUENCE ||
- relkind == RELKIND_VIEW || relkind == RELKIND_COMPOSITE_TYPE) &&
- OidIsValid(binary_upgrade_next_heap_relfilenode))
- {
- relid = binary_upgrade_next_heap_relfilenode;
- binary_upgrade_next_heap_relfilenode = InvalidOid;
- }
- else if (relkind == RELKIND_TOASTVALUE &&
- OidIsValid(binary_upgrade_next_toast_relfilenode))
- {
- relid = binary_upgrade_next_toast_relfilenode;
- binary_upgrade_next_toast_relfilenode = InvalidOid;
- }
- else if (!OidIsValid(relid))
+ /*
+ * Allocate an OID for the relation, unless we were told what to use.
+ *
+ * The OID will be the relfilenode as well, so make sure it doesn't
+ * collide with either pg_class OIDs or existing physical files.
+ */
+ if (!OidIsValid(relid))
{
- /*
- * Allocate an OID for the relation, unless we were told what to use.
- *
- * The OID will be the relfilenode as well, so make sure it doesn't
- * collide with either pg_class OIDs or existing physical files.
- */
- relid = GetNewRelFileNode(reltablespace, shared_relation,
- pg_class_desc);
+ /* Use binary-upgrade overrides if applicable */
+ if (OidIsValid(binary_upgrade_next_heap_relfilenode) &&
+ (relkind == RELKIND_RELATION || relkind == RELKIND_SEQUENCE ||
+ relkind == RELKIND_VIEW || relkind == RELKIND_COMPOSITE_TYPE))
+ {
+ relid = binary_upgrade_next_heap_relfilenode;
+ binary_upgrade_next_heap_relfilenode = InvalidOid;
+ }
+ else if (OidIsValid(binary_upgrade_next_toast_relfilenode) &&
+ relkind == RELKIND_TOASTVALUE)
+ {
+ relid = binary_upgrade_next_toast_relfilenode;
+ binary_upgrade_next_toast_relfilenode = InvalidOid;
+ }
+ else
+ relid = GetNewRelFileNode(reltablespace, shared_relation,
+ pg_class_desc);
}
/*
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index ed70f973296..c6b6e76933f 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/catalog/index.c,v 1.331 2010/01/22 16:40:18 rhaas Exp $
+ * $PostgreSQL: pgsql/src/backend/catalog/index.c,v 1.332 2010/02/03 01:14:16 tgl Exp $
*
*
* INTERFACE ROUTINES
@@ -69,6 +69,9 @@
#include "utils/tqual.h"
+/* Kluge for upgrade-in-place support */
+Oid binary_upgrade_next_index_relfilenode = InvalidOid;
+
/* state info for validate_index bulkdelete callback */
typedef struct
{
@@ -79,9 +82,6 @@ typedef struct
tups_inserted;
} v_i_state;
-/* For simple relation creation, this is the toast index relfilenode */
-Oid binary_upgrade_next_index_relfilenode = InvalidOid;
-
/* non-export function prototypes */
static TupleDesc ConstructTupleDescriptor(Relation heapRelation,
IndexInfo *indexInfo,
@@ -642,21 +642,23 @@ index_create(Oid heapRelationId,
accessMethodObjectId,
classObjectId);
- if (OidIsValid(binary_upgrade_next_index_relfilenode))
- {
- indexRelationId = binary_upgrade_next_index_relfilenode;
- binary_upgrade_next_index_relfilenode = InvalidOid;
- }
- else if (!OidIsValid(indexRelationId))
+ /*
+ * Allocate an OID for the index, unless we were told what to use.
+ *
+ * The OID will be the relfilenode as well, so make sure it doesn't
+ * collide with either pg_class OIDs or existing physical files.
+ */
+ if (!OidIsValid(indexRelationId))
{
- /*
- * Allocate an OID for the index, unless we were told what to use.
- *
- * The OID will be the relfilenode as well, so make sure it doesn't
- * collide with either pg_class OIDs or existing physical files.
- */
- indexRelationId = GetNewRelFileNode(tableSpaceId, shared_relation,
- pg_class);
+ /* Use binary-upgrade override if applicable */
+ if (OidIsValid(binary_upgrade_next_index_relfilenode))
+ {
+ indexRelationId = binary_upgrade_next_index_relfilenode;
+ binary_upgrade_next_index_relfilenode = InvalidOid;
+ }
+ else
+ indexRelationId = GetNewRelFileNode(tableSpaceId, shared_relation,
+ pg_class);
}
/*
@@ -1391,87 +1393,6 @@ index_update_stats(Relation rel,
heap_close(pg_class, RowExclusiveLock);
}
-/*
- * setNewRelfilenode - assign a new relfilenode value to the relation
- *
- * Caller must already hold exclusive lock on the relation.
- *
- * The relation is marked with relfrozenxid=freezeXid (InvalidTransactionId
- * must be passed for indexes)
- */
-void
-setNewRelfilenode(Relation relation, TransactionId freezeXid)
-{
- Oid newrelfilenode;
- RelFileNode newrnode;
- Relation pg_class;
- HeapTuple tuple;
- Form_pg_class rd_rel;
-
- /* Can't change relfilenode for nailed tables (indexes ok though) */
- Assert(!relation->rd_isnailed ||
- relation->rd_rel->relkind == RELKIND_INDEX);
- /* Can't change for shared tables or indexes */
- Assert(!relation->rd_rel->relisshared);
- /* Indexes must have Invalid frozenxid; other relations must not */
- Assert((relation->rd_rel->relkind == RELKIND_INDEX &&
- freezeXid == InvalidTransactionId) ||
- TransactionIdIsNormal(freezeXid));
-
- /* Allocate a new relfilenode */
- newrelfilenode = GetNewRelFileNode(relation->rd_rel->reltablespace,
- relation->rd_rel->relisshared,
- NULL);
-
- /*
- * Find the pg_class tuple for the given relation. This is not used
- * during bootstrap, so okay to use heap_update always.
- */
- pg_class = heap_open(RelationRelationId, RowExclusiveLock);
-
- tuple = SearchSysCacheCopy(RELOID,
- ObjectIdGetDatum(RelationGetRelid(relation)),
- 0, 0, 0);
- if (!HeapTupleIsValid(tuple))
- elog(ERROR, "could not find tuple for relation %u",
- RelationGetRelid(relation));
- rd_rel = (Form_pg_class) GETSTRUCT(tuple);
-
- /*
- * ... and create storage for corresponding forks in the new relfilenode.
- *
- * NOTE: any conflict in relfilenode value will be caught here
- */
- newrnode = relation->rd_node;
- newrnode.relNode = newrelfilenode;
-
- /*
- * Create the main fork, like heap_create() does, and drop the old
- * storage.
- */
- RelationCreateStorage(newrnode, relation->rd_istemp);
- smgrclosenode(newrnode);
- RelationDropStorage(relation);
-
- /* update the pg_class row */
- rd_rel->relfilenode = newrelfilenode;
- rd_rel->relpages = 0; /* it's empty until further notice */
- rd_rel->reltuples = 0;
- rd_rel->relfrozenxid = freezeXid;
- simple_heap_update(pg_class, &tuple->t_self, tuple);
- CatalogUpdateIndexes(pg_class, tuple);
-
- heap_freetuple(tuple);
-
- heap_close(pg_class, RowExclusiveLock);
-
- /* Make sure the relfilenode change is visible */
- CommandCounterIncrement();
-
- /* Mark the rel as having a new relfilenode in current transaction */
- RelationCacheMarkNewRelfilenode(relation);
-}
-
/*
* index_build - invoke access-method-specific index build procedure
@@ -2562,7 +2483,7 @@ reindex_index(Oid indexId)
/*
* We'll build a new physical relation for the index.
*/
- setNewRelfilenode(iRel, InvalidTransactionId);
+ RelationSetNewRelfilenode(iRel, InvalidTransactionId);
}
/* Initialize the index and rebuild */
@@ -2660,8 +2581,8 @@ reindex_relation(Oid relid, bool toast_too)
* yet because all of this is transaction-safe. If we fail partway
* through, the updated rows are dead and it doesn't matter whether they
* have index entries. Also, a new pg_class index will be created with an
- * entry for its own pg_class row because we do setNewRelfilenode() before
- * we do index_build().
+ * entry for its own pg_class row because we do RelationSetNewRelfilenode()
+ * before we do index_build().
*
* Note that we also clear pg_class's rd_oidindex until the loop is done,
* so that that index can't be accessed either. This means we cannot
diff --git a/src/backend/catalog/toasting.c b/src/backend/catalog/toasting.c
index 58890aa6a0a..ca70f19bf32 100644
--- a/src/backend/catalog/toasting.c
+++ b/src/backend/catalog/toasting.c
@@ -8,7 +8,7 @@
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/catalog/toasting.c,v 1.28 2010/01/28 23:21:11 petere Exp $
+ * $PostgreSQL: pgsql/src/backend/catalog/toasting.c,v 1.29 2010/02/03 01:14:16 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -31,9 +31,11 @@
#include "utils/builtins.h"
#include "utils/syscache.h"
-Oid binary_upgrade_next_pg_type_toast_oid = InvalidOid;
+/* Kluges for upgrade-in-place support */
extern Oid binary_upgrade_next_toast_relfilenode;
+Oid binary_upgrade_next_pg_type_toast_oid = InvalidOid;
+
static bool create_toast_table(Relation rel, Oid toastOid, Oid toastIndexOid,
Datum reloptions);
static bool needs_toast_table(Relation rel);
@@ -145,7 +147,9 @@ create_toast_table(Relation rel, Oid toastOid, Oid toastIndexOid, Datum reloptio
/*
* Check to see whether the table actually needs a TOAST table.
- * If the relfilenode is specified, force toast file creation.
+ *
+ * If an update-in-place relfilenode is specified, force toast file
+ * creation even if it seems not to need one.
*/
if (!needs_toast_table(rel) &&
!OidIsValid(binary_upgrade_next_toast_relfilenode))
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 0e0b0de9f95..cec3b7fc58e 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/commands/tablecmds.c,v 1.321 2010/02/01 19:28:56 rhaas Exp $
+ * $PostgreSQL: pgsql/src/backend/commands/tablecmds.c,v 1.322 2010/02/03 01:14:16 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -996,7 +996,7 @@ ExecuteTruncate(TruncateStmt *stmt)
* as the relfilenode value. The old storage file is scheduled for
* deletion at commit.
*/
- setNewRelfilenode(rel, RecentXmin);
+ RelationSetNewRelfilenode(rel, RecentXmin);
heap_relid = RelationGetRelid(rel);
toast_relid = rel->rd_rel->reltoastrelid;
@@ -1007,7 +1007,7 @@ ExecuteTruncate(TruncateStmt *stmt)
if (OidIsValid(toast_relid))
{
rel = relation_open(toast_relid, AccessExclusiveLock);
- setNewRelfilenode(rel, RecentXmin);
+ RelationSetNewRelfilenode(rel, RecentXmin);
heap_close(rel, NoLock);
}
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index 5497d03cc5a..958be2433fb 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -11,7 +11,7 @@
*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/storage/smgr/smgr.c,v 1.118 2010/01/02 16:57:52 momjian Exp $
+ * $PostgreSQL: pgsql/src/backend/storage/smgr/smgr.c,v 1.119 2010/02/03 01:14:17 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -24,6 +24,7 @@
#include "storage/ipc.h"
#include "storage/smgr.h"
#include "utils/hsearch.h"
+#include "utils/inval.h"
/*
@@ -351,13 +352,21 @@ smgr_internal_unlink(RelFileNode rnode, ForkNumber forknum,
*/
/*
- * And delete the physical files.
+ * Delete the physical file(s).
*
* Note: smgr_unlink must treat deletion failure as a WARNING, not an
* ERROR, because we've already decided to commit or abort the current
* xact.
*/
(*(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);
}
/*
@@ -437,6 +446,8 @@ smgrnblocks(SMgrRelation reln, ForkNumber forknum)
/*
* smgrtruncate() -- Truncate supplied relation to the specified number
* of blocks
+ *
+ * The truncation is done immediately, so this can't be rolled back.
*/
void
smgrtruncate(SMgrRelation reln, ForkNumber forknum, BlockNumber nblocks,
@@ -448,9 +459,21 @@ smgrtruncate(SMgrRelation reln, ForkNumber forknum, BlockNumber nblocks,
*/
DropRelFileNodeBuffers(reln->smgr_rnode, forknum, isTemp, nblocks);
- /* Do the truncation */
+ /*
+ * 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
+ * references they may have for this rel. This is useful because they
+ * might have open file pointers to segments that got removed. (The inval
+ * message will come back to our backend, too, causing a
+ * probably-unnecessary smgr flush. But we don't expect that this is
+ * a performance-critical path.)
+ */
+ CacheInvalidateSmgr(reln->smgr_rnode);
}
/*
diff --git a/src/backend/utils/cache/inval.c b/src/backend/utils/cache/inval.c
index 04935ffd546..99aad752bb3 100644
--- a/src/backend/utils/cache/inval.c
+++ b/src/backend/utils/cache/inval.c
@@ -53,14 +53,14 @@
*
* Also, whenever we see an operation on a pg_class or pg_attribute tuple,
* we register a relcache flush operation for the relation described by that
- * tuple. pg_class updates trigger an smgr flush operation as well.
+ * tuple.
*
- * We keep the relcache and smgr flush requests in lists separate from the
- * catcache tuple flush requests. This allows us to issue all the pending
- * catcache flushes before we issue relcache flushes, which saves us from
- * loading a catcache tuple during relcache load only to flush it again
- * right away. Also, we avoid queuing multiple relcache flush requests for
- * the same relation, since a relcache flush is relatively expensive to do.
+ * We keep the relcache flush requests in lists separate from the catcache
+ * tuple flush requests. This allows us to issue all the pending catcache
+ * flushes before we issue relcache flushes, which saves us from loading
+ * a catcache tuple during relcache load only to flush it again right away.
+ * Also, we avoid queuing multiple relcache flush requests for the same
+ * relation, since a relcache flush is relatively expensive to do.
* (XXX is it worth testing likewise for duplicate catcache flush entries?
* Probably not.)
*
@@ -80,7 +80,7 @@
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/utils/cache/inval.c,v 1.92 2010/01/09 16:49:27 sriggs Exp $
+ * $PostgreSQL: pgsql/src/backend/utils/cache/inval.c,v 1.93 2010/02/03 01:14:17 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -116,7 +116,7 @@ typedef struct InvalidationChunk
typedef struct InvalidationListHeader
{
InvalidationChunk *cclist; /* list of chunks holding catcache msgs */
- InvalidationChunk *rclist; /* list of chunks holding relcache/smgr msgs */
+ InvalidationChunk *rclist; /* list of chunks holding relcache msgs */
} InvalidationListHeader;
/*----------------
@@ -304,7 +304,7 @@ AppendInvalidationMessageList(InvalidationChunk **destHdr,
* Invalidation set support functions
*
* These routines understand about the division of a logical invalidation
- * list into separate physical lists for catcache and relcache/smgr entries.
+ * list into separate physical lists for catcache and relcache entries.
* ----------------------------------------------------------------
*/
@@ -349,27 +349,6 @@ AddRelcacheInvalidationMessage(InvalidationListHeader *hdr,
}
/*
- * Add an smgr inval entry
- */
-static void
-AddSmgrInvalidationMessage(InvalidationListHeader *hdr,
- RelFileNode rnode)
-{
- SharedInvalidationMessage msg;
-
- /* Don't add a duplicate item */
- ProcessMessageList(hdr->rclist,
- if (msg->sm.id == SHAREDINVALSMGR_ID &&
- RelFileNodeEquals(msg->sm.rnode, rnode))
- return);
-
- /* OK, add the item */
- msg.sm.id = SHAREDINVALSMGR_ID;
- msg.sm.rnode = rnode;
- AddInvalidationMessage(&hdr->rclist, &msg);
-}
-
-/*
* Append one list of invalidation messages to another, resetting
* the source list to empty.
*/
@@ -455,23 +434,6 @@ RegisterRelcacheInvalidation(Oid dbId, Oid relId)
}
/*
- * RegisterSmgrInvalidation
- *
- * As above, but register an smgr invalidation event.
- */
-static void
-RegisterSmgrInvalidation(RelFileNode rnode)
-{
- AddSmgrInvalidationMessage(&transInvalInfo->CurrentCmdInvalidMsgs,
- rnode);
-
- /*
- * As above, just in case there is not an associated catalog change.
- */
- (void) GetCurrentCommandId(true);
-}
-
-/*
* LocalExecuteInvalidationMessage
*
* Process a single invalidation message (which could be of any type).
@@ -606,35 +568,12 @@ PrepareForTupleInvalidation(Relation relation, HeapTuple tuple)
if (tupleRelId == RelationRelationId)
{
Form_pg_class classtup = (Form_pg_class) GETSTRUCT(tuple);
- RelFileNode rnode;
relationId = HeapTupleGetOid(tuple);
if (classtup->relisshared)
databaseId = InvalidOid;
else
databaseId = MyDatabaseId;
-
- /*
- * We need to send out an smgr inval as well as a relcache inval. This
- * is needed because other backends might possibly possess smgr cache
- * but not relcache entries for the target relation.
- *
- * Note: during a pg_class row update that assigns a new relfilenode
- * or reltablespace value, we will be called on both the old and new
- * tuples, and thus will broadcast invalidation messages showing both
- * the old and new RelFileNode values. This ensures that other
- * backends will close smgr references to the old file.
- *
- * XXX possible future cleanup: it might be better to trigger smgr
- * flushes explicitly, rather than indirectly from pg_class updates.
- */
- if (classtup->reltablespace)
- rnode.spcNode = classtup->reltablespace;
- else
- rnode.spcNode = MyDatabaseTableSpace;
- rnode.dbNode = databaseId;
- rnode.relNode = classtup->relfilenode;
- RegisterSmgrInvalidation(rnode);
}
else if (tupleRelId == AttributeRelationId)
{
@@ -902,7 +841,7 @@ xactGetCommittedInvalidationMessages(SharedInvalidationMessage **msgs,
*/
void
ProcessCommittedInvalidationMessages(SharedInvalidationMessage *msgs,
- int nmsgs, bool RelcacheInitFileInval)
+ int nmsgs, bool RelcacheInitFileInval)
{
Oid dboid = 0;
bool invalidate_global = false;
@@ -1251,10 +1190,6 @@ CacheInvalidateHeapTuple(Relation relation, HeapTuple tuple)
* This is used in places that need to force relcache rebuild but aren't
* changing any of the tuples recognized as contributors to the relcache
* entry by PrepareForTupleInvalidation. (An example is dropping an index.)
- * We assume in particular that relfilenode/reltablespace aren't changing
- * (so the rd_node value is still good).
- *
- * XXX most callers of this probably don't need to force an smgr flush.
*/
void
CacheInvalidateRelcache(Relation relation)
@@ -1269,7 +1204,6 @@ CacheInvalidateRelcache(Relation relation)
databaseId = MyDatabaseId;
RegisterRelcacheInvalidation(databaseId, relationId);
- RegisterSmgrInvalidation(relation->rd_node);
}
/*
@@ -1282,22 +1216,13 @@ CacheInvalidateRelcacheByTuple(HeapTuple classTuple)
Form_pg_class classtup = (Form_pg_class) GETSTRUCT(classTuple);
Oid databaseId;
Oid relationId;
- RelFileNode rnode;
relationId = HeapTupleGetOid(classTuple);
if (classtup->relisshared)
databaseId = InvalidOid;
else
databaseId = MyDatabaseId;
- if (classtup->reltablespace)
- rnode.spcNode = classtup->reltablespace;
- else
- rnode.spcNode = MyDatabaseTableSpace;
- rnode.dbNode = databaseId;
- rnode.relNode = classtup->relfilenode;
-
RegisterRelcacheInvalidation(databaseId, relationId);
- RegisterSmgrInvalidation(rnode);
}
/*
@@ -1320,6 +1245,39 @@ CacheInvalidateRelcacheByRelid(Oid relid)
ReleaseSysCache(tup);
}
+
+/*
+ * CacheInvalidateSmgr
+ * Register invalidation of smgr references to a physical relation.
+ *
+ * Sending this type of invalidation msg forces other backends to close open
+ * smgr entries for the rel. This should be done to flush dangling open-file
+ * references when the physical rel is being dropped or truncated. Because
+ * these are nontransactional (i.e., not-rollback-able) operations, we just
+ * send the inval message immediately without any queuing.
+ *
+ * Note: in most cases there will have been a relcache flush issued against
+ * the rel at the logical level. We need a separate smgr-level flush because
+ * it is possible for backends to have open smgr entries for rels they don't
+ * have a relcache entry for, e.g. because the only thing they ever did with
+ * the rel is write out dirty shared buffers.
+ *
+ * Note: because these messages are nontransactional, they won't be captured
+ * in commit/abort WAL entries. Instead, calls to CacheInvalidateSmgr()
+ * should happen in low-level smgr.c routines, which are executed while
+ * replaying WAL as well as when creating it.
+ */
+void
+CacheInvalidateSmgr(RelFileNode rnode)
+{
+ SharedInvalidationMessage msg;
+
+ msg.sm.id = SHAREDINVALSMGR_ID;
+ msg.sm.rnode = rnode;
+ SendSharedInvalidMessages(&msg, 1);
+}
+
+
/*
* CacheRegisterSyscacheCallback
* Register the specified function to be called for all future
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index 30a32d17f9a..ba09331aba9 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/utils/cache/relcache.c,v 1.300 2010/01/13 23:07:08 tgl Exp $
+ * $PostgreSQL: pgsql/src/backend/utils/cache/relcache.c,v 1.301 2010/02/03 01:14:17 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -33,6 +33,7 @@
#include "access/genam.h"
#include "access/reloptions.h"
#include "access/sysattr.h"
+#include "access/transam.h"
#include "access/xact.h"
#include "catalog/catalog.h"
#include "catalog/index.h"
@@ -53,6 +54,7 @@
#include "catalog/pg_trigger.h"
#include "catalog/pg_type.h"
#include "catalog/schemapg.h"
+#include "catalog/storage.h"
#include "commands/trigger.h"
#include "miscadmin.h"
#include "optimizer/clauses.h"
@@ -2377,22 +2379,6 @@ AtEOSubXact_RelationCache(bool isCommit, SubTransactionId mySubid,
}
}
-/*
- * RelationCacheMarkNewRelfilenode
- *
- * Mark the rel as having been given a new relfilenode in the current
- * (sub) transaction. This is a hint that can be used to optimize
- * later operations on the rel in the same transaction.
- */
-void
-RelationCacheMarkNewRelfilenode(Relation rel)
-{
- /* Mark it... */
- rel->rd_newRelfilenodeSubid = GetCurrentSubTransactionId();
- /* ... and now we have eoxact cleanup work to do */
- need_eoxact_work = true;
-}
-
/*
* RelationBuildLocalRelation
@@ -2562,6 +2548,109 @@ RelationBuildLocalRelation(const char *relname,
return rel;
}
+
+/*
+ * RelationSetNewRelfilenode
+ *
+ * Assign a new relfilenode (physical file name) to the relation.
+ *
+ * This allows a full rewrite of the relation to be done with transactional
+ * safety (since the filenode assignment can be rolled back). Note however
+ * that there is no simple way to access the relation's old data for the
+ * remainder of the current transaction. This limits the usefulness to cases
+ * such as TRUNCATE or rebuilding an index from scratch.
+ *
+ * Caller must already hold exclusive lock on the relation.
+ *
+ * The relation is marked with relfrozenxid = freezeXid (InvalidTransactionId
+ * must be passed for indexes). This should be a lower bound on the XIDs
+ * that will be put into the new relation contents.
+ */
+void
+RelationSetNewRelfilenode(Relation relation, TransactionId freezeXid)
+{
+ Oid newrelfilenode;
+ RelFileNode newrnode;
+ Relation pg_class;
+ HeapTuple tuple;
+ Form_pg_class classform;
+
+ /* Can't change relfilenode for nailed tables (indexes ok though) */
+ Assert(!relation->rd_isnailed ||
+ relation->rd_rel->relkind == RELKIND_INDEX);
+ /* Can't change for shared tables or indexes */
+ Assert(!relation->rd_rel->relisshared);
+ /* Indexes must have Invalid frozenxid; other relations must not */
+ Assert((relation->rd_rel->relkind == RELKIND_INDEX &&
+ freezeXid == InvalidTransactionId) ||
+ TransactionIdIsNormal(freezeXid));
+
+ /* Allocate a new relfilenode */
+ newrelfilenode = GetNewRelFileNode(relation->rd_rel->reltablespace,
+ relation->rd_rel->relisshared,
+ NULL);
+
+ /*
+ * Find the pg_class tuple for the given relation. This is not used
+ * during bootstrap, so okay to use heap_update always.
+ */
+ pg_class = heap_open(RelationRelationId, RowExclusiveLock);
+
+ tuple = SearchSysCacheCopy(RELOID,
+ ObjectIdGetDatum(RelationGetRelid(relation)),
+ 0, 0, 0);
+ if (!HeapTupleIsValid(tuple))
+ elog(ERROR, "could not find tuple for relation %u",
+ RelationGetRelid(relation));
+ classform = (Form_pg_class) GETSTRUCT(tuple);
+
+ /*
+ * Create storage for the main fork of the new relfilenode.
+ *
+ * NOTE: any conflict in relfilenode value will be caught here, if
+ * GetNewRelFileNode messes up for any reason.
+ */
+ newrnode = relation->rd_node;
+ newrnode.relNode = newrelfilenode;
+ RelationCreateStorage(newrnode, relation->rd_istemp);
+ smgrclosenode(newrnode);
+
+ /*
+ * Schedule unlinking of the old storage at transaction commit.
+ */
+ RelationDropStorage(relation);
+
+ /*
+ * Now update the pg_class row.
+ */
+ classform->relfilenode = newrelfilenode;
+ classform->relpages = 0; /* it's empty until further notice */
+ classform->reltuples = 0;
+ classform->relfrozenxid = freezeXid;
+ simple_heap_update(pg_class, &tuple->t_self, tuple);
+ CatalogUpdateIndexes(pg_class, tuple);
+
+ heap_freetuple(tuple);
+
+ heap_close(pg_class, RowExclusiveLock);
+
+ /*
+ * Make the pg_class row change visible. This will cause the relcache
+ * entry to get updated, too.
+ */
+ CommandCounterIncrement();
+
+ /*
+ * Mark the rel as having been given a new relfilenode in the current
+ * (sub) transaction. This is a hint that can be used to optimize
+ * later operations on the rel in the same transaction.
+ */
+ relation->rd_newRelfilenodeSubid = GetCurrentSubTransactionId();
+ /* ... and now we have eoxact cleanup work to do */
+ need_eoxact_work = true;
+}
+
+
/*
* RelationCacheInitialize
*
diff --git a/src/include/catalog/index.h b/src/include/catalog/index.h
index 28fbd3319eb..bdb1c71a734 100644
--- a/src/include/catalog/index.h
+++ b/src/include/catalog/index.h
@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $PostgreSQL: pgsql/src/include/catalog/index.h,v 1.80 2010/01/02 16:58:01 momjian Exp $
+ * $PostgreSQL: pgsql/src/include/catalog/index.h,v 1.81 2010/02/03 01:14:17 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -56,8 +56,6 @@ extern void FormIndexDatum(IndexInfo *indexInfo,
Datum *values,
bool *isnull);
-extern void setNewRelfilenode(Relation relation, TransactionId freezeXid);
-
extern void index_build(Relation heapRelation,
Relation indexRelation,
IndexInfo *indexInfo,
diff --git a/src/include/utils/inval.h b/src/include/utils/inval.h
index 5da5fbc18b8..dc35160ffef 100644
--- a/src/include/utils/inval.h
+++ b/src/include/utils/inval.h
@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $PostgreSQL: pgsql/src/include/utils/inval.h,v 1.46 2010/01/02 16:58:10 momjian Exp $
+ * $PostgreSQL: pgsql/src/include/utils/inval.h,v 1.47 2010/02/03 01:14:17 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -15,6 +15,7 @@
#define INVAL_H
#include "access/htup.h"
+#include "storage/relfilenode.h"
#include "utils/relcache.h"
@@ -50,6 +51,8 @@ extern void CacheInvalidateRelcacheByTuple(HeapTuple classTuple);
extern void CacheInvalidateRelcacheByRelid(Oid relid);
+extern void CacheInvalidateSmgr(RelFileNode rnode);
+
extern void CacheRegisterSyscacheCallback(int cacheid,
SyscacheCallbackFunction func,
Datum arg);
diff --git a/src/include/utils/relcache.h b/src/include/utils/relcache.h
index 249a37c849f..2e48250cbf3 100644
--- a/src/include/utils/relcache.h
+++ b/src/include/utils/relcache.h
@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $PostgreSQL: pgsql/src/include/utils/relcache.h,v 1.66 2010/01/02 16:58:10 momjian Exp $
+ * $PostgreSQL: pgsql/src/include/utils/relcache.h,v 1.67 2010/02/03 01:14:17 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -71,6 +71,12 @@ extern Relation RelationBuildLocalRelation(const char *relname,
bool shared_relation);
/*
+ * Routine to manage assignment of new relfilenode to a relation
+ */
+extern void RelationSetNewRelfilenode(Relation relation,
+ TransactionId freezeXid);
+
+/*
* Routines for flushing/rebuilding relcache entries in various scenarios
*/
extern void RelationForgetRelation(Oid rid);
@@ -83,8 +89,6 @@ extern void AtEOXact_RelationCache(bool isCommit);
extern void AtEOSubXact_RelationCache(bool isCommit, SubTransactionId mySubid,
SubTransactionId parentSubid);
-extern void RelationCacheMarkNewRelfilenode(Relation rel);
-
/*
* Routines to help manage rebuilding of relcache init files
*/