Clean up code associated with updating pg_class statistics columns

(relpages/reltuples).  To do this, create formal support in heapam.c for
"overwrite" tuple updates (including xlog replay capability) and use that
instead of the ad-hoc overwrites we'd been using in VACUUM and CREATE INDEX.
Take the responsibility for updating stats during CREATE INDEX out of the
individual index AMs, and do it where it belongs, in catalog/index.c.  Aside
from being more modular, this avoids having to update the same tuple twice in
some paths through CREATE INDEX.  It's probably not measurably faster, but
for sure it's a lot cleaner than before.
This commit is contained in:
Tom Lane 2006-05-10 23:18:39 +00:00
parent c1f39437d0
commit 3fdeb189e9
17 changed files with 519 additions and 404 deletions

View File

@ -1,4 +1,4 @@
<!-- $PostgreSQL: pgsql/doc/src/sgml/indexam.sgml,v 2.10 2006/05/02 22:25:09 tgl Exp $ --> <!-- $PostgreSQL: pgsql/doc/src/sgml/indexam.sgml,v 2.11 2006/05/10 23:18:38 tgl Exp $ -->
<chapter id="indexam"> <chapter id="indexam">
<title>Index Access Method Interface Definition</title> <title>Index Access Method Interface Definition</title>
@ -144,7 +144,7 @@
<para> <para>
<programlisting> <programlisting>
void IndexBuildResult *
ambuild (Relation heapRelation, ambuild (Relation heapRelation,
Relation indexRelation, Relation indexRelation,
IndexInfo *indexInfo); IndexInfo *indexInfo);
@ -155,6 +155,8 @@ ambuild (Relation heapRelation,
in the table. Ordinarily the <function>ambuild</> function will call in the table. Ordinarily the <function>ambuild</> function will call
<function>IndexBuildHeapScan()</> to scan the table for existing tuples <function>IndexBuildHeapScan()</> to scan the table for existing tuples
and compute the keys that need to be inserted into the index. and compute the keys that need to be inserted into the index.
The function must return a palloc'd struct containing statistics about
the new index.
</para> </para>
<para> <para>

View File

@ -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/access/gin/gininsert.c,v 1.1 2006/05/02 11:28:54 teodor Exp $ * $PostgreSQL: pgsql/src/backend/access/gin/gininsert.c,v 1.2 2006/05/10 23:18:38 tgl Exp $
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
@ -242,6 +242,7 @@ ginbuild(PG_FUNCTION_ARGS) {
Relation heap = (Relation) PG_GETARG_POINTER(0); Relation heap = (Relation) PG_GETARG_POINTER(0);
Relation index = (Relation) PG_GETARG_POINTER(1); Relation index = (Relation) PG_GETARG_POINTER(1);
IndexInfo *indexInfo = (IndexInfo *) PG_GETARG_POINTER(2); IndexInfo *indexInfo = (IndexInfo *) PG_GETARG_POINTER(2);
IndexBuildResult *result;
double reltuples; double reltuples;
GinBuildState buildstate; GinBuildState buildstate;
Buffer buffer; Buffer buffer;
@ -310,10 +311,15 @@ ginbuild(PG_FUNCTION_ARGS) {
MemoryContextDelete(buildstate.tmpCtx); MemoryContextDelete(buildstate.tmpCtx);
/* since we just counted the # of tuples, may as well update stats */ /*
IndexCloseAndUpdateStats(heap, reltuples, index, buildstate.indtuples); * Return statistics
*/
result = (IndexBuildResult *) palloc(sizeof(IndexBuildResult));
PG_RETURN_VOID(); result->heap_tuples = reltuples;
result->index_tuples = buildstate.indtuples;
PG_RETURN_POINTER(result);
} }
/* /*

View File

@ -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/access/gist/gist.c,v 1.133 2006/05/10 09:19:54 teodor Exp $ * $PostgreSQL: pgsql/src/backend/access/gist/gist.c,v 1.134 2006/05/10 23:18:38 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
@ -89,6 +89,7 @@ gistbuild(PG_FUNCTION_ARGS)
Relation heap = (Relation) PG_GETARG_POINTER(0); Relation heap = (Relation) PG_GETARG_POINTER(0);
Relation index = (Relation) PG_GETARG_POINTER(1); Relation index = (Relation) PG_GETARG_POINTER(1);
IndexInfo *indexInfo = (IndexInfo *) PG_GETARG_POINTER(2); IndexInfo *indexInfo = (IndexInfo *) PG_GETARG_POINTER(2);
IndexBuildResult *result;
double reltuples; double reltuples;
GISTBuildState buildstate; GISTBuildState buildstate;
Buffer buffer; Buffer buffer;
@ -154,12 +155,17 @@ gistbuild(PG_FUNCTION_ARGS)
/* okay, all heap tuples are indexed */ /* okay, all heap tuples are indexed */
MemoryContextDelete(buildstate.tmpCtx); MemoryContextDelete(buildstate.tmpCtx);
/* since we just counted the # of tuples, may as well update stats */
IndexCloseAndUpdateStats(heap, reltuples, index, buildstate.indtuples);
freeGISTstate(&buildstate.giststate); freeGISTstate(&buildstate.giststate);
PG_RETURN_VOID(); /*
* Return statistics
*/
result = (IndexBuildResult *) palloc(sizeof(IndexBuildResult));
result->heap_tuples = reltuples;
result->index_tuples = buildstate.indtuples;
PG_RETURN_POINTER(result);
} }
/* /*

View File

@ -8,7 +8,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/access/hash/hash.c,v 1.89 2006/05/02 22:25:10 tgl Exp $ * $PostgreSQL: pgsql/src/backend/access/hash/hash.c,v 1.90 2006/05/10 23:18:38 tgl Exp $
* *
* NOTES * NOTES
* This file contains only the public interface routines. * This file contains only the public interface routines.
@ -51,6 +51,7 @@ hashbuild(PG_FUNCTION_ARGS)
Relation heap = (Relation) PG_GETARG_POINTER(0); Relation heap = (Relation) PG_GETARG_POINTER(0);
Relation index = (Relation) PG_GETARG_POINTER(1); Relation index = (Relation) PG_GETARG_POINTER(1);
IndexInfo *indexInfo = (IndexInfo *) PG_GETARG_POINTER(2); IndexInfo *indexInfo = (IndexInfo *) PG_GETARG_POINTER(2);
IndexBuildResult *result;
double reltuples; double reltuples;
HashBuildState buildstate; HashBuildState buildstate;
@ -72,10 +73,15 @@ hashbuild(PG_FUNCTION_ARGS)
reltuples = IndexBuildHeapScan(heap, index, indexInfo, reltuples = IndexBuildHeapScan(heap, index, indexInfo,
hashbuildCallback, (void *) &buildstate); hashbuildCallback, (void *) &buildstate);
/* since we just counted the # of tuples, may as well update stats */ /*
IndexCloseAndUpdateStats(heap, reltuples, index, buildstate.indtuples); * Return statistics
*/
result = (IndexBuildResult *) palloc(sizeof(IndexBuildResult));
PG_RETURN_VOID(); result->heap_tuples = reltuples;
result->index_tuples = buildstate.indtuples;
PG_RETURN_POINTER(result);
} }
/* /*

View File

@ -8,7 +8,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/access/heap/heapam.c,v 1.211 2006/03/31 23:32:05 tgl Exp $ * $PostgreSQL: pgsql/src/backend/access/heap/heapam.c,v 1.212 2006/05/10 23:18:39 tgl Exp $
* *
* *
* INTERFACE ROUTINES * INTERFACE ROUTINES
@ -2673,6 +2673,97 @@ l3:
return HeapTupleMayBeUpdated; return HeapTupleMayBeUpdated;
} }
/*
* heap_inplace_update - update a tuple "in place" (ie, overwrite it)
*
* Overwriting violates both MVCC and transactional safety, so the uses
* of this function in Postgres are extremely limited. Nonetheless we
* find some places to use it.
*
* The tuple cannot change size, and therefore it's reasonable to assume
* that its null bitmap (if any) doesn't change either. So we just
* overwrite the data portion of the tuple without touching the null
* bitmap or any of the header fields.
*
* tuple is an in-memory tuple structure containing the data to be written
* over the target tuple. Also, tuple->t_self identifies the target tuple.
*/
void
heap_inplace_update(Relation relation, HeapTuple tuple)
{
Buffer buffer;
Page page;
OffsetNumber offnum;
ItemId lp = NULL;
HeapTupleHeader htup;
uint32 oldlen;
uint32 newlen;
buffer = ReadBuffer(relation, ItemPointerGetBlockNumber(&(tuple->t_self)));
LockBuffer(buffer, BUFFER_LOCK_EXCLUSIVE);
page = (Page) BufferGetPage(buffer);
offnum = ItemPointerGetOffsetNumber(&(tuple->t_self));
if (PageGetMaxOffsetNumber(page) >= offnum)
lp = PageGetItemId(page, offnum);
if (PageGetMaxOffsetNumber(page) < offnum || !ItemIdIsUsed(lp))
elog(ERROR, "heap_inplace_update: invalid lp");
htup = (HeapTupleHeader) PageGetItem(page, lp);
oldlen = ItemIdGetLength(lp) - htup->t_hoff;
newlen = tuple->t_len - tuple->t_data->t_hoff;
if (oldlen != newlen || htup->t_hoff != tuple->t_data->t_hoff)
elog(ERROR, "heap_inplace_update: wrong tuple length");
/* NO EREPORT(ERROR) from here till changes are logged */
START_CRIT_SECTION();
memcpy((char *) htup + htup->t_hoff,
(char *) tuple->t_data + tuple->t_data->t_hoff,
newlen);
MarkBufferDirty(buffer);
/* XLOG stuff */
if (!relation->rd_istemp)
{
xl_heap_inplace xlrec;
XLogRecPtr recptr;
XLogRecData rdata[2];
xlrec.target.node = relation->rd_node;
xlrec.target.tid = tuple->t_self;
rdata[0].data = (char *) &xlrec;
rdata[0].len = SizeOfHeapInplace;
rdata[0].buffer = InvalidBuffer;
rdata[0].next = &(rdata[1]);
rdata[1].data = (char *) htup + htup->t_hoff;
rdata[1].len = newlen;
rdata[1].buffer = buffer;
rdata[1].buffer_std = true;
rdata[1].next = NULL;
recptr = XLogInsert(RM_HEAP_ID, XLOG_HEAP_INPLACE, rdata);
PageSetLSN(page, recptr);
PageSetTLI(page, ThisTimeLineID);
}
END_CRIT_SECTION();
UnlockReleaseBuffer(buffer);
/* Send out shared cache inval if necessary */
if (!IsBootstrapProcessingMode())
CacheInvalidateHeapTuple(relation, tuple);
}
/* ---------------- /* ----------------
* heap_markpos - mark scan position * heap_markpos - mark scan position
* ---------------- * ----------------
@ -3329,6 +3420,59 @@ heap_xlog_lock(XLogRecPtr lsn, XLogRecord *record)
UnlockReleaseBuffer(buffer); UnlockReleaseBuffer(buffer);
} }
static void
heap_xlog_inplace(XLogRecPtr lsn, XLogRecord *record)
{
xl_heap_inplace *xlrec = (xl_heap_inplace *) XLogRecGetData(record);
Relation reln = XLogOpenRelation(xlrec->target.node);
Buffer buffer;
Page page;
OffsetNumber offnum;
ItemId lp = NULL;
HeapTupleHeader htup;
uint32 oldlen;
uint32 newlen;
if (record->xl_info & XLR_BKP_BLOCK_1)
return;
buffer = XLogReadBuffer(reln,
ItemPointerGetBlockNumber(&(xlrec->target.tid)),
false);
if (!BufferIsValid(buffer))
return;
page = (Page) BufferGetPage(buffer);
if (XLByteLE(lsn, PageGetLSN(page))) /* changes are applied */
{
UnlockReleaseBuffer(buffer);
return;
}
offnum = ItemPointerGetOffsetNumber(&(xlrec->target.tid));
if (PageGetMaxOffsetNumber(page) >= offnum)
lp = PageGetItemId(page, offnum);
if (PageGetMaxOffsetNumber(page) < offnum || !ItemIdIsUsed(lp))
elog(PANIC, "heap_inplace_redo: invalid lp");
htup = (HeapTupleHeader) PageGetItem(page, lp);
oldlen = ItemIdGetLength(lp) - htup->t_hoff;
newlen = record->xl_len - SizeOfHeapInplace;
if (oldlen != newlen)
elog(PANIC, "heap_inplace_redo: wrong tuple length");
memcpy((char *) htup + htup->t_hoff,
(char *) xlrec + SizeOfHeapInplace,
newlen);
PageSetLSN(page, lsn);
PageSetTLI(page, ThisTimeLineID);
MarkBufferDirty(buffer);
UnlockReleaseBuffer(buffer);
}
void void
heap_redo(XLogRecPtr lsn, XLogRecord *record) heap_redo(XLogRecPtr lsn, XLogRecord *record)
{ {
@ -3349,6 +3493,8 @@ heap_redo(XLogRecPtr lsn, XLogRecord *record)
heap_xlog_newpage(lsn, record); heap_xlog_newpage(lsn, record);
else if (info == XLOG_HEAP_LOCK) else if (info == XLOG_HEAP_LOCK)
heap_xlog_lock(lsn, record); heap_xlog_lock(lsn, record);
else if (info == XLOG_HEAP_INPLACE)
heap_xlog_inplace(lsn, record);
else else
elog(PANIC, "heap_redo: unknown op code %u", info); elog(PANIC, "heap_redo: unknown op code %u", info);
} }
@ -3442,6 +3588,13 @@ heap_desc(StringInfo buf, uint8 xl_info, char *rec)
appendStringInfo(buf, "%u ", xlrec->locking_xid); appendStringInfo(buf, "%u ", xlrec->locking_xid);
out_target(buf, &(xlrec->target)); out_target(buf, &(xlrec->target));
} }
else if (info == XLOG_HEAP_INPLACE)
{
xl_heap_inplace *xlrec = (xl_heap_inplace *) rec;
appendStringInfo(buf, "inplace: ");
out_target(buf, &(xlrec->target));
}
else else
appendStringInfo(buf, "UNKNOWN"); appendStringInfo(buf, "UNKNOWN");
} }

View File

@ -12,21 +12,18 @@
* 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/access/nbtree/nbtree.c,v 1.148 2006/05/08 00:00:10 tgl Exp $ * $PostgreSQL: pgsql/src/backend/access/nbtree/nbtree.c,v 1.149 2006/05/10 23:18:39 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
#include "postgres.h" #include "postgres.h"
#include "access/genam.h" #include "access/genam.h"
#include "access/heapam.h"
#include "access/nbtree.h" #include "access/nbtree.h"
#include "catalog/index.h" #include "catalog/index.h"
#include "commands/vacuum.h" #include "commands/vacuum.h"
#include "miscadmin.h"
#include "storage/freespace.h" #include "storage/freespace.h"
#include "storage/smgr.h" #include "storage/lmgr.h"
#include "utils/inval.h"
#include "utils/memutils.h" #include "utils/memutils.h"
@ -84,6 +81,7 @@ btbuild(PG_FUNCTION_ARGS)
Relation heap = (Relation) PG_GETARG_POINTER(0); Relation heap = (Relation) PG_GETARG_POINTER(0);
Relation index = (Relation) PG_GETARG_POINTER(1); Relation index = (Relation) PG_GETARG_POINTER(1);
IndexInfo *indexInfo = (IndexInfo *) PG_GETARG_POINTER(2); IndexInfo *indexInfo = (IndexInfo *) PG_GETARG_POINTER(2);
IndexBuildResult *result;
double reltuples; double reltuples;
BTBuildState buildstate; BTBuildState buildstate;
@ -149,18 +147,20 @@ btbuild(PG_FUNCTION_ARGS)
/* /*
* If we are reindexing a pre-existing index, it is critical to send out * If we are reindexing a pre-existing index, it is critical to send out
* a relcache invalidation SI message to ensure all backends re-read the * a relcache invalidation SI message to ensure all backends re-read the
* index metapage. In most circumstances the update-stats operation will * index metapage. We expect that the caller will ensure that happens
* cause that to happen, but at the moment there are corner cases where * (typically as a side effect of updating index stats, but it must
* no pg_class update will occur, so force an inval here. XXX FIXME: * happen even if the stats don't change!)
* the upper levels of CREATE INDEX should handle the stats update as
* well as guaranteeing relcache inval.
*/ */
CacheInvalidateRelcache(index);
/* since we just counted the # of tuples, may as well update stats */ /*
IndexCloseAndUpdateStats(heap, reltuples, index, buildstate.indtuples); * Return statistics
*/
result = (IndexBuildResult *) palloc(sizeof(IndexBuildResult));
PG_RETURN_VOID(); result->heap_tuples = reltuples;
result->index_tuples = buildstate.indtuples;
PG_RETURN_POINTER(result);
} }
/* /*

View File

@ -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/bootstrap/bootstrap.c,v 1.214 2006/04/04 19:35:33 tgl Exp $ * $PostgreSQL: pgsql/src/backend/bootstrap/bootstrap.c,v 1.215 2006/05/10 23:18:39 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
@ -1173,11 +1173,12 @@ AddStr(char *str, int strlength, int mderef)
* index_register() -- record an index that has been set up for building * index_register() -- record an index that has been set up for building
* later. * later.
* *
* At bootstrap time, we define a bunch of indices on system catalogs. * At bootstrap time, we define a bunch of indexes on system catalogs.
* We postpone actually building the indices until just before we're * We postpone actually building the indexes until just before we're
* finished with initialization, however. This is because more classes * finished with initialization, however. This is because the indexes
* and indices may be defined, and we want to be sure that all of them * themselves have catalog entries, and those have to be included in the
* are present in the index. * indexes on those catalogs. Doing it in two phases is the simplest
* way of making sure the indexes have the right contents at the end.
*/ */
void void
index_register(Oid heap, index_register(Oid heap,
@ -1189,7 +1190,7 @@ index_register(Oid heap,
/* /*
* XXX mao 10/31/92 -- don't gc index reldescs, associated info at * XXX mao 10/31/92 -- don't gc index reldescs, associated info at
* bootstrap time. we'll declare the indices now, but want to create them * bootstrap time. we'll declare the indexes now, but want to create them
* later. * later.
*/ */
@ -1223,6 +1224,10 @@ index_register(Oid heap,
MemoryContextSwitchTo(oldcxt); MemoryContextSwitchTo(oldcxt);
} }
/*
* build_indices -- fill in all the indexes registered earlier
*/
void void
build_indices(void) build_indices(void)
{ {
@ -1233,13 +1238,10 @@ build_indices(void)
heap = heap_open(ILHead->il_heap, NoLock); heap = heap_open(ILHead->il_heap, NoLock);
ind = index_open(ILHead->il_ind); ind = index_open(ILHead->il_ind);
index_build(heap, ind, ILHead->il_info);
/* index_build(heap, ind, ILHead->il_info, false, false);
* In normal processing mode, index_build would close the heap and
* index, but in bootstrap mode it will not.
*/
/* XXX Probably we ought to close the heap and index here? */ index_close(ind);
heap_close(heap, NoLock);
} }
} }

View File

@ -8,7 +8,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/catalog/heap.c,v 1.298 2006/04/30 01:08:06 momjian Exp $ * $PostgreSQL: pgsql/src/backend/catalog/heap.c,v 1.299 2006/05/10 23:18:39 tgl Exp $
* *
* *
* INTERFACE ROUTINES * INTERFACE ROUTINES
@ -1921,17 +1921,14 @@ RelationTruncateIndexes(Oid heapId)
RelationTruncate(currentIndex, 0); RelationTruncate(currentIndex, 0);
/* Initialize the index and rebuild */ /* Initialize the index and rebuild */
index_build(heapRelation, currentIndex, indexInfo); /* Note: we do not need to re-establish pkey or toast settings */
index_build(heapRelation, currentIndex, indexInfo, false, false);
/* /* We're done with this index */
* index_build will close both the heap and index relations (but not index_close(currentIndex);
* give up the locks we hold on them). We're done with this index,
* but we must re-open the heap rel.
*/
heapRelation = heap_open(heapId, NoLock);
} }
/* Finish by closing the heap rel again */ /* And now done with the heap; but keep lock until commit */
heap_close(heapRelation, NoLock); heap_close(heapRelation, NoLock);
} }

View File

@ -8,7 +8,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/catalog/index.c,v 1.265 2006/03/31 23:32:06 tgl Exp $ * $PostgreSQL: pgsql/src/backend/catalog/index.c,v 1.266 2006/05/10 23:18:39 tgl Exp $
* *
* *
* INTERFACE ROUTINES * INTERFACE ROUTINES
@ -61,8 +61,9 @@ static void UpdateIndexRelation(Oid indexoid, Oid heapoid,
IndexInfo *indexInfo, IndexInfo *indexInfo,
Oid *classOids, Oid *classOids,
bool primary); bool primary);
static void index_update_stats(Relation rel, bool hasindex, bool isprimary,
Oid reltoastidxid, double reltuples);
static Oid IndexGetRelation(Oid indexId); static Oid IndexGetRelation(Oid indexId);
static void UpdateStats(Oid relid, double reltuples);
/* /*
@ -437,15 +438,26 @@ UpdateIndexRelation(Oid indexoid,
} }
/* ---------------------------------------------------------------- /*
* index_create * index_create
* *
* indexRelationId is normally InvalidOid to let this routine * heapRelationId: OID of table to build index on
* generate an OID for the index. During bootstrap it may be * indexRelationName: what it say
* indexRelationId: normally, pass InvalidOid to let this routine
* generate an OID for the index. During bootstrap this may be
* nonzero to specify a preselected OID. * nonzero to specify a preselected OID.
* indexInfo: same info executor uses to insert into the index
* accessMethodObjectId: OID of index AM to use
* tableSpaceId: OID of tablespace to use
* classObjectId: array of index opclass OIDs, one per index column
* isprimary: index is a PRIMARY KEY
* istoast: index is a toast table's index
* isconstraint: index is owned by a PRIMARY KEY or UNIQUE constraint
* allow_system_table_mods: allow table to be a system catalog
* skip_build: true to skip the index_build() step for the moment; caller
* must do it later (typically via reindex_index())
* *
* Returns OID of the created index. * Returns OID of the created index.
* ----------------------------------------------------------------
*/ */
Oid Oid
index_create(Oid heapRelationId, index_create(Oid heapRelationId,
@ -455,7 +467,8 @@ index_create(Oid heapRelationId,
Oid accessMethodObjectId, Oid accessMethodObjectId,
Oid tableSpaceId, Oid tableSpaceId,
Oid *classObjectId, Oid *classObjectId,
bool primary, bool isprimary,
bool istoast,
bool isconstraint, bool isconstraint,
bool allow_system_table_mods, bool allow_system_table_mods,
bool skip_build) bool skip_build)
@ -595,7 +608,7 @@ index_create(Oid heapRelationId,
* ---------------- * ----------------
*/ */
UpdateIndexRelation(indexRelationId, heapRelationId, indexInfo, UpdateIndexRelation(indexRelationId, heapRelationId, indexInfo,
classObjectId, primary); classObjectId, isprimary);
/* /*
* Register constraint and dependencies for the index. * Register constraint and dependencies for the index.
@ -625,7 +638,7 @@ index_create(Oid heapRelationId,
char constraintType; char constraintType;
Oid conOid; Oid conOid;
if (primary) if (isprimary)
constraintType = CONSTRAINT_PRIMARY; constraintType = CONSTRAINT_PRIMARY;
else if (indexInfo->ii_Unique) else if (indexInfo->ii_Unique)
constraintType = CONSTRAINT_UNIQUE; constraintType = CONSTRAINT_UNIQUE;
@ -736,28 +749,40 @@ index_create(Oid heapRelationId,
* Similarly, if the caller specified skip_build then filling the index is * Similarly, if the caller specified skip_build then filling the index is
* delayed till later (ALTER TABLE can save work in some cases with this). * delayed till later (ALTER TABLE can save work in some cases with this).
* Otherwise, we call the AM routine that constructs the index. * Otherwise, we call the AM routine that constructs the index.
*
* In normal processing mode, the heap and index relations are closed, but
* we continue to hold the ShareLock on the heap and the exclusive lock on
* the index that we acquired above, until end of transaction.
*/ */
if (IsBootstrapProcessingMode()) if (IsBootstrapProcessingMode())
{ {
index_register(heapRelationId, indexRelationId, indexInfo); index_register(heapRelationId, indexRelationId, indexInfo);
/* XXX shouldn't we close the heap and index rels here? */
} }
else if (skip_build) else if (skip_build)
{ {
/* caller is responsible for filling the index later on */ /*
relation_close(indexRelation, NoLock); * Caller is responsible for filling the index later on. However,
heap_close(heapRelation, NoLock); * we'd better make sure that the heap relation is correctly marked
* as having an index.
*/
index_update_stats(heapRelation,
true,
isprimary,
InvalidOid,
heapRelation->rd_rel->reltuples);
/* Make the above update visible */
CommandCounterIncrement();
} }
else else
{ {
index_build(heapRelation, indexRelation, indexInfo); index_build(heapRelation, indexRelation, indexInfo,
/* index_build closes the passed rels */ isprimary, istoast);
} }
/*
* Close the heap and index; but we keep the ShareLock on the heap and
* the exclusive lock on the index that we acquired above, until end of
* transaction.
*/
index_close(indexRelation);
heap_close(heapRelation, NoLock);
return indexRelationId; return indexRelationId;
} }
@ -983,38 +1008,59 @@ FormIndexDatum(IndexInfo *indexInfo,
} }
/* ---------------- /*
* set relhasindex of relation's pg_class entry * index_update_stats --- update pg_class entry after CREATE INDEX
* *
* If isprimary is TRUE, we are defining a primary index, so also set * This routine updates the pg_class row of either an index or its parent
* relhaspkey to TRUE. Otherwise, leave relhaspkey alone. * relation after CREATE INDEX. Its rather bizarre API is designed to
* ensure we can do all the necessary work in just one update.
* *
* If reltoastidxid is not InvalidOid, also set reltoastidxid to that value. * hasindex: set relhasindex to this value
* This is only used for TOAST relations. * isprimary: if true, set relhaspkey true; else no change
* reltoastidxid: if not InvalidOid, set reltoastidxid to this value;
* else no change
* reltuples: set reltuples to this value
*
* relpages is also updated (using RelationGetNumberOfBlocks()).
* *
* NOTE: an important side-effect of this operation is that an SI invalidation * NOTE: an important side-effect of this operation is that an SI invalidation
* message is sent out to all backends --- including me --- causing relcache * message is sent out to all backends --- including me --- causing relcache
* entries to be flushed or updated with the new hasindex data. This must * entries to be flushed or updated with the new data. This must happen even
* happen even if we find that no change is needed in the pg_class row. * if we find that no change is needed in the pg_class row. When updating
* ---------------- * a heap entry, this ensures that other backends find out about the new
* index. When updating an index, it's important because some index AMs
* expect a relcache flush to occur after REINDEX.
*/ */
void static void
setRelhasindex(Oid relid, bool hasindex, bool isprimary, Oid reltoastidxid) index_update_stats(Relation rel, bool hasindex, bool isprimary,
Oid reltoastidxid, double reltuples)
{ {
BlockNumber relpages = RelationGetNumberOfBlocks(rel);
Oid relid = RelationGetRelid(rel);
Relation pg_class; Relation pg_class;
HeapTuple tuple; HeapTuple tuple;
Form_pg_class classtuple; Form_pg_class rd_rel;
bool dirty = false; bool in_place_upd;
HeapScanDesc pg_class_scan = NULL; bool dirty;
/* /*
* Find the tuple to update in pg_class. In bootstrap mode we can't use * Find the tuple to update in pg_class. Normally we make a copy of the
* heap_update, so cheat and overwrite the tuple in-place. In normal * tuple using the syscache, modify it, and apply heap_update. But in
* processing, make a copy to scribble on. * bootstrap mode we can't use heap_update, so we use a nontransactional
* update, ie, overwrite the tuple in-place.
*
* We also must use an in-place update if reindexing pg_class itself,
* because the target index may presently not be part of the set of
* indexes that CatalogUpdateIndexes would update (see reindex_relation).
*/ */
pg_class = heap_open(RelationRelationId, RowExclusiveLock); pg_class = heap_open(RelationRelationId, RowExclusiveLock);
if (!IsBootstrapProcessingMode()) in_place_upd = IsBootstrapProcessingMode() ||
ReindexIsProcessingHeap(RelationRelationId);
restart:
if (!in_place_upd)
{ {
tuple = SearchSysCacheCopy(RELOID, tuple = SearchSysCacheCopy(RELOID,
ObjectIdGetDatum(relid), ObjectIdGetDatum(relid),
@ -1022,6 +1068,8 @@ setRelhasindex(Oid relid, bool hasindex, bool isprimary, Oid reltoastidxid)
} }
else else
{ {
/* don't assume syscache will work */
HeapScanDesc pg_class_scan;
ScanKeyData key[1]; ScanKeyData key[1];
ScanKeyInit(&key[0], ScanKeyInit(&key[0],
@ -1031,65 +1079,110 @@ setRelhasindex(Oid relid, bool hasindex, bool isprimary, Oid reltoastidxid)
pg_class_scan = heap_beginscan(pg_class, SnapshotNow, 1, key); pg_class_scan = heap_beginscan(pg_class, SnapshotNow, 1, key);
tuple = heap_getnext(pg_class_scan, ForwardScanDirection); tuple = heap_getnext(pg_class_scan, ForwardScanDirection);
tuple = heap_copytuple(tuple);
heap_endscan(pg_class_scan);
} }
if (!HeapTupleIsValid(tuple)) if (!HeapTupleIsValid(tuple))
elog(ERROR, "could not find tuple for relation %u", relid); elog(ERROR, "could not find tuple for relation %u", relid);
classtuple = (Form_pg_class) GETSTRUCT(tuple); rd_rel = (Form_pg_class) GETSTRUCT(tuple);
/* Apply required updates */ /* Apply required updates, if any, to copied tuple */
if (pg_class_scan) dirty = false;
LockBuffer(pg_class_scan->rs_cbuf, BUFFER_LOCK_EXCLUSIVE); if (rd_rel->relhasindex != hasindex)
if (classtuple->relhasindex != hasindex)
{ {
classtuple->relhasindex = hasindex; rd_rel->relhasindex = hasindex;
dirty = true; dirty = true;
} }
if (isprimary) if (isprimary)
{ {
if (!classtuple->relhaspkey) if (!rd_rel->relhaspkey)
{ {
classtuple->relhaspkey = true; rd_rel->relhaspkey = true;
dirty = true; dirty = true;
} }
} }
if (OidIsValid(reltoastidxid)) if (OidIsValid(reltoastidxid))
{ {
Assert(classtuple->relkind == RELKIND_TOASTVALUE); Assert(rd_rel->relkind == RELKIND_TOASTVALUE);
if (classtuple->reltoastidxid != reltoastidxid) if (rd_rel->reltoastidxid != reltoastidxid)
{ {
classtuple->reltoastidxid = reltoastidxid; rd_rel->reltoastidxid = reltoastidxid;
dirty = true; dirty = true;
} }
} }
if (rd_rel->reltuples != (float4) reltuples)
if (pg_class_scan)
{ {
MarkBufferDirty(pg_class_scan->rs_cbuf); rd_rel->reltuples = (float4) reltuples;
LockBuffer(pg_class_scan->rs_cbuf, BUFFER_LOCK_UNLOCK); dirty = true;
/* Send out shared cache inval if necessary */
if (!IsBootstrapProcessingMode())
CacheInvalidateHeapTuple(pg_class, tuple);
} }
else if (dirty) if (rd_rel->relpages != (int32) relpages)
{ {
simple_heap_update(pg_class, &tuple->t_self, tuple); rd_rel->relpages = (int32) relpages;
dirty = true;
}
/*
* If anything changed, write out the tuple
*/
if (dirty)
{
if (in_place_upd)
{
heap_inplace_update(pg_class, tuple);
}
else
{
/*
* Because PG allows concurrent CREATE INDEX commands, it's
* possible that someone else tries to update the pg_class
* row at about the same time we do. Hence, instead of using
* simple_heap_update(), we must use full heap_update() and
* cope with HeapTupleUpdated result. If we see that, just
* go back and try the whole update again.
*/
HTSU_Result result;
ItemPointerData update_ctid;
TransactionId update_xmax;
result = heap_update(pg_class, &tuple->t_self, tuple,
&update_ctid, &update_xmax,
GetCurrentCommandId(), InvalidSnapshot,
true /* wait for commit */ );
switch (result)
{
case HeapTupleSelfUpdated:
/* Tuple was already updated in current command? */
elog(ERROR, "tuple already updated by self");
break;
case HeapTupleMayBeUpdated:
/* done successfully */
break;
case HeapTupleUpdated:
heap_freetuple(tuple);
/* Must do CCI so we can see the updated tuple */
CommandCounterIncrement();
goto restart;
default:
elog(ERROR, "unrecognized heap_update status: %u", result);
break;
}
/* Keep the catalog indexes up to date */ /* Keep the catalog indexes up to date */
CatalogUpdateIndexes(pg_class, tuple); CatalogUpdateIndexes(pg_class, tuple);
} }
}
else else
{ {
/* no need to change tuple, but force relcache rebuild anyway */ /* no need to change tuple, but force relcache inval anyway */
CacheInvalidateRelcacheByTuple(tuple); CacheInvalidateRelcacheByTuple(tuple);
} }
if (!pg_class_scan)
heap_freetuple(tuple); heap_freetuple(tuple);
else
heap_endscan(pg_class_scan);
heap_close(pg_class, RowExclusiveLock); heap_close(pg_class, RowExclusiveLock);
} }
@ -1163,177 +1256,31 @@ setNewRelfilenode(Relation relation)
} }
/*
* This is invoked by the various index AMs once they have finished
* constructing an index. Constructing an index involves counting the
* number of tuples in both the relation and the index, so we take
* advantage of the opportunity to update pg_class to ensure that the
* planner takes advantage of the index we just created. But, only
* update statistics during normal index definitions, not for indices
* on system catalogs created during bootstrap processing. We must
* close the relations before updating statistics to guarantee that
* the relcache entries are flushed when we increment the command
* counter in UpdateStats(). But we do not release any locks on the
* relations; those will be held until end of transaction.
*/
void
IndexCloseAndUpdateStats(Relation heap, double heapTuples,
Relation index, double indexTuples)
{
Oid hrelid = RelationGetRelid(heap);
Oid irelid = RelationGetRelid(index);
if (!IsNormalProcessingMode())
return;
heap_close(heap, NoLock);
index_close(index);
UpdateStats(hrelid, heapTuples);
UpdateStats(irelid, indexTuples);
}
/* ----------------
* UpdateStats
*
* Update pg_class' relpages and reltuples statistics for the given relation
* (which can be either a table or an index). Note that this is not used
* in the context of VACUUM, only CREATE INDEX.
* ----------------
*/
static void
UpdateStats(Oid relid, double reltuples)
{
Relation whichRel;
Relation pg_class;
HeapTuple tuple;
BlockNumber relpages;
Form_pg_class rd_rel;
HeapScanDesc pg_class_scan = NULL;
bool in_place_upd;
/*
* This routine handles updates for both the heap and index relation
* statistics. In order to guarantee that we're able to *see* the index
* relation tuple, we bump the command counter id here. The index
* relation tuple was created in the current transaction.
*/
CommandCounterIncrement();
/*
* CommandCounterIncrement() flushes invalid cache entries, including
* those for the heap and index relations for which we're updating
* statistics. Now that the cache is flushed, it's safe to open the
* relation again. We need the relation open in order to figure out how
* many blocks it contains.
*/
/*
* Grabbing lock here is probably redundant ...
*/
whichRel = relation_open(relid, ShareLock);
/*
* Find the tuple to update in pg_class. Normally we make a copy of the
* tuple using the syscache, modify it, and apply heap_update. But in
* bootstrap mode we can't use heap_update, so we cheat and overwrite the
* tuple in-place. (Note: as of PG 8.0 this isn't called during
* bootstrap, but leave the code here for possible future use.)
*
* We also must cheat if reindexing pg_class itself, because the target
* index may presently not be part of the set of indexes that
* CatalogUpdateIndexes would update (see reindex_relation). In this case
* the stats updates will not be WAL-logged and so could be lost in a
* crash. This seems OK considering VACUUM does the same thing.
*/
pg_class = heap_open(RelationRelationId, RowExclusiveLock);
in_place_upd = IsBootstrapProcessingMode() ||
ReindexIsProcessingHeap(RelationRelationId);
if (!in_place_upd)
{
tuple = SearchSysCacheCopy(RELOID,
ObjectIdGetDatum(relid),
0, 0, 0);
}
else
{
ScanKeyData key[1];
ScanKeyInit(&key[0],
ObjectIdAttributeNumber,
BTEqualStrategyNumber, F_OIDEQ,
ObjectIdGetDatum(relid));
pg_class_scan = heap_beginscan(pg_class, SnapshotNow, 1, key);
tuple = heap_getnext(pg_class_scan, ForwardScanDirection);
}
if (!HeapTupleIsValid(tuple))
elog(ERROR, "could not find tuple for relation %u", relid);
rd_rel = (Form_pg_class) GETSTRUCT(tuple);
/*
* Update statistics in pg_class, if they changed. (Avoiding an
* unnecessary update is not just a tiny performance improvement; it also
* reduces the window wherein concurrent CREATE INDEX commands may
* conflict.)
*/
relpages = RelationGetNumberOfBlocks(whichRel);
if (rd_rel->relpages != (int32) relpages ||
rd_rel->reltuples != (float4) reltuples)
{
if (in_place_upd)
{
/* Bootstrap or reindex case: overwrite fields in place. */
LockBuffer(pg_class_scan->rs_cbuf, BUFFER_LOCK_EXCLUSIVE);
rd_rel->relpages = (int32) relpages;
rd_rel->reltuples = (float4) reltuples;
MarkBufferDirty(pg_class_scan->rs_cbuf);
LockBuffer(pg_class_scan->rs_cbuf, BUFFER_LOCK_UNLOCK);
if (!IsBootstrapProcessingMode())
CacheInvalidateHeapTuple(pg_class, tuple);
}
else
{
/* During normal processing, must work harder. */
rd_rel->relpages = (int32) relpages;
rd_rel->reltuples = (float4) reltuples;
simple_heap_update(pg_class, &tuple->t_self, tuple);
CatalogUpdateIndexes(pg_class, tuple);
}
}
if (in_place_upd)
heap_endscan(pg_class_scan);
else
heap_freetuple(tuple);
/*
* We shouldn't have to do this, but we do... Modify the reldesc in place
* with the new values so that the cache contains the latest copy. (XXX
* is this really still necessary? The relcache will get fixed at next
* CommandCounterIncrement, so why bother here?)
*/
whichRel->rd_rel->relpages = (int32) relpages;
whichRel->rd_rel->reltuples = (float4) reltuples;
heap_close(pg_class, RowExclusiveLock);
relation_close(whichRel, NoLock);
}
/* /*
* index_build - invoke access-method-specific index build procedure * index_build - invoke access-method-specific index build procedure
*
* On entry, the index's catalog entries are valid, and its physical disk
* file has been created but is empty. We call the AM-specific build
* procedure to fill in the index contents. We then update the pg_class
* entries of the index and heap relation as needed, using statistics
* returned by ambuild as well as data passed by the caller.
*
* Note: when reindexing an existing index, isprimary and istoast can be
* false; the index is already properly marked and need not be re-marked.
*
* Note: before Postgres 8.2, the passed-in heap and index Relations
* were automatically closed by this routine. This is no longer the case.
* The caller opened 'em, and the caller should close 'em.
*/ */
void void
index_build(Relation heapRelation, index_build(Relation heapRelation,
Relation indexRelation, Relation indexRelation,
IndexInfo *indexInfo) IndexInfo *indexInfo,
bool isprimary,
bool istoast)
{ {
RegProcedure procedure; RegProcedure procedure;
IndexBuildResult *stats;
/* /*
* sanity checks * sanity checks
@ -1347,10 +1294,30 @@ index_build(Relation heapRelation,
/* /*
* Call the access method's build procedure * Call the access method's build procedure
*/ */
OidFunctionCall3(procedure, stats = (IndexBuildResult *)
DatumGetPointer(OidFunctionCall3(procedure,
PointerGetDatum(heapRelation), PointerGetDatum(heapRelation),
PointerGetDatum(indexRelation), PointerGetDatum(indexRelation),
PointerGetDatum(indexInfo)); PointerGetDatum(indexInfo)));
Assert(PointerIsValid(stats));
/*
* Update heap and index pg_class rows
*/
index_update_stats(heapRelation,
true,
isprimary,
istoast ? RelationGetRelid(indexRelation) : InvalidOid,
stats->heap_tuples);
index_update_stats(indexRelation,
false,
false,
InvalidOid,
stats->index_tuples);
/* Make the updated versions visible */
CommandCounterIncrement();
} }
@ -1674,12 +1641,8 @@ reindex_index(Oid indexId)
} }
/* Initialize the index and rebuild */ /* Initialize the index and rebuild */
index_build(heapRelation, iRel, indexInfo); /* Note: we do not need to re-establish pkey or toast settings */
index_build(heapRelation, iRel, indexInfo, false, false);
/*
* index_build will close both the heap and index relations (but not
* give up the locks we hold on them). So we're done.
*/
} }
PG_CATCH(); PG_CATCH();
{ {
@ -1689,6 +1652,10 @@ reindex_index(Oid indexId)
} }
PG_END_TRY(); PG_END_TRY();
ResetReindexProcessing(); ResetReindexProcessing();
/* Close rels, but keep locks */
index_close(iRel);
heap_close(heapRelation, NoLock);
} }
/* /*

View File

@ -8,7 +8,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/commands/indexcmds.c,v 1.138 2006/03/05 15:58:24 momjian Exp $ * $PostgreSQL: pgsql/src/backend/commands/indexcmds.c,v 1.139 2006/05/10 23:18:39 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
@ -397,17 +397,8 @@ DefineIndex(RangeVar *heapRelation,
index_create(relationId, indexRelationName, indexRelationId, index_create(relationId, indexRelationName, indexRelationId,
indexInfo, accessMethodId, tablespaceId, classObjectId, indexInfo, accessMethodId, tablespaceId, classObjectId,
primary, isconstraint, primary, false, isconstraint,
allowSystemTableMods, skip_build); allowSystemTableMods, skip_build);
/*
* We update the relation's pg_class tuple even if it already has
* relhasindex = true. This is needed to cause a shared-cache-inval
* message to be sent for the pg_class tuple, which will cause other
* backends to flush their relcache entries and in particular their cached
* lists of the indexes for this relation.
*/
setRelhasindex(relationId, true, primary, InvalidOid);
} }

View File

@ -8,7 +8,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/commands/tablecmds.c,v 1.183 2006/04/30 01:08:07 momjian Exp $ * $PostgreSQL: pgsql/src/backend/commands/tablecmds.c,v 1.184 2006/05/10 23:18:39 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
@ -6046,14 +6046,7 @@ AlterTableCreateToastTable(Oid relOid, bool silent)
BTREE_AM_OID, BTREE_AM_OID,
rel->rd_rel->reltablespace, rel->rd_rel->reltablespace,
classObjectId, classObjectId,
true, false, true, false); true, true, false, true, false);
/*
* Update toast rel's pg_class entry to show that it has an index. The
* index OID is stored into the reltoastidxid field for easy access by the
* tuple toaster.
*/
setRelhasindex(toast_relid, true, true, toast_idxid);
/* /*
* Store the toast table's OID in the parent relation's pg_class row * Store the toast table's OID in the parent relation's pg_class row

View File

@ -13,7 +13,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/commands/vacuum.c,v 1.329 2006/05/03 22:45:26 tgl Exp $ * $PostgreSQL: pgsql/src/backend/commands/vacuum.c,v 1.330 2006/05/10 23:18:39 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
@ -673,9 +673,10 @@ vacuum_set_xid_limits(VacuumStmt *vacstmt, bool sharedRel,
* doing ANALYZE, but we always update these stats. This routine works * doing ANALYZE, but we always update these stats. This routine works
* for both index and heap relation entries in pg_class. * for both index and heap relation entries in pg_class.
* *
* We violate no-overwrite semantics here by storing new values for the * We violate transaction semantics here by overwriting the rel's
* statistics columns directly into the pg_class tuple that's already on * existing pg_class tuple with the new values. This is reasonably
* the page. The reason for this is that if we updated these tuples in * safe since the new values are correct whether or not this transaction
* commits. The reason for this is that if we updated these tuples in
* the usual way, vacuuming pg_class itself wouldn't work very well --- * the usual way, vacuuming pg_class itself wouldn't work very well ---
* by the time we got done with a vacuum cycle, most of the tuples in * by the time we got done with a vacuum cycle, most of the tuples in
* pg_class would've been obsoleted. Of course, this only works for * pg_class would've been obsoleted. Of course, this only works for
@ -689,59 +690,57 @@ vac_update_relstats(Oid relid, BlockNumber num_pages, double num_tuples,
bool hasindex) bool hasindex)
{ {
Relation rd; Relation rd;
HeapTupleData rtup;
HeapTuple ctup; HeapTuple ctup;
Form_pg_class pgcform; Form_pg_class pgcform;
Buffer buffer; bool dirty;
/*
* update number of tuples and number of pages in pg_class
*/
rd = heap_open(RelationRelationId, RowExclusiveLock); rd = heap_open(RelationRelationId, RowExclusiveLock);
ctup = SearchSysCache(RELOID, /* Fetch a copy of the tuple to scribble on */
ctup = SearchSysCacheCopy(RELOID,
ObjectIdGetDatum(relid), ObjectIdGetDatum(relid),
0, 0, 0); 0, 0, 0);
if (!HeapTupleIsValid(ctup)) if (!HeapTupleIsValid(ctup))
elog(ERROR, "pg_class entry for relid %u vanished during vacuuming", elog(ERROR, "pg_class entry for relid %u vanished during vacuuming",
relid); relid);
pgcform = (Form_pg_class) GETSTRUCT(ctup);
/* get the buffer cache tuple */ /* Apply required updates, if any, to copied tuple */
rtup.t_self = ctup->t_self;
ReleaseSysCache(ctup);
if (!heap_fetch(rd, SnapshotNow, &rtup, &buffer, false, NULL))
elog(ERROR, "pg_class entry for relid %u vanished during vacuuming",
relid);
/* ensure no one else does this at the same time */ dirty = false;
LockBuffer(buffer, BUFFER_LOCK_EXCLUSIVE); if (pgcform->relpages != (int32) num_pages)
{
/* overwrite the existing statistics in the tuple */
pgcform = (Form_pg_class) GETSTRUCT(&rtup);
pgcform->relpages = (int32) num_pages; pgcform->relpages = (int32) num_pages;
dirty = true;
}
if (pgcform->reltuples != (float4) num_tuples)
{
pgcform->reltuples = (float4) num_tuples; pgcform->reltuples = (float4) num_tuples;
dirty = true;
}
if (pgcform->relhasindex != hasindex)
{
pgcform->relhasindex = hasindex; pgcform->relhasindex = hasindex;
dirty = true;
}
/* /*
* If we have discovered that there are no indexes, then there's no * If we have discovered that there are no indexes, then there's no
* primary key either. This could be done more thoroughly... * primary key either. This could be done more thoroughly...
*/ */
if (!hasindex) if (!hasindex)
{
if (pgcform->relhaspkey)
{
pgcform->relhaspkey = false; pgcform->relhaspkey = false;
dirty = true;
MarkBufferDirty(buffer); }
}
LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
/* /*
* Invalidate the tuple in the catcaches; this also arranges to flush the * If anything changed, write out the tuple
* relation's relcache entry. (If we fail to commit for some reason, no
* flush will occur, but no great harm is done since there are no
* noncritical state updates here.)
*/ */
CacheInvalidateHeapTuple(rd, &rtup); if (dirty)
heap_inplace_update(rd, ctup);
ReleaseBuffer(buffer);
heap_close(rd, RowExclusiveLock); heap_close(rd, RowExclusiveLock);
} }
@ -753,10 +752,11 @@ vac_update_relstats(Oid relid, BlockNumber num_pages, double num_tuples,
* Update the whole-database statistics that are kept in its pg_database * Update the whole-database statistics that are kept in its pg_database
* row, and the flat-file copy of pg_database. * row, and the flat-file copy of pg_database.
* *
* We violate no-overwrite semantics here by storing new values for the * We violate transaction semantics here by overwriting the database's
* statistics columns directly into the tuple that's already on the page. * existing pg_database tuple with the new values. This is reasonably
* As with vac_update_relstats, this avoids leaving dead tuples behind * safe since the new values are correct whether or not this transaction
* after a VACUUM. * commits. As with vac_update_relstats, this avoids leaving dead tuples
* behind after a VACUUM.
* *
* This routine is shared by full and lazy VACUUM. Note that it is only * This routine is shared by full and lazy VACUUM. Note that it is only
* applied after a database-wide VACUUM operation. * applied after a database-wide VACUUM operation.
@ -767,49 +767,24 @@ vac_update_dbstats(Oid dbid,
TransactionId frozenXID) TransactionId frozenXID)
{ {
Relation relation; Relation relation;
ScanKeyData entry[1];
SysScanDesc scan;
HeapTuple tuple; HeapTuple tuple;
Buffer buf;
Form_pg_database dbform; Form_pg_database dbform;
relation = heap_open(DatabaseRelationId, RowExclusiveLock); relation = heap_open(DatabaseRelationId, RowExclusiveLock);
ScanKeyInit(&entry[0], /* Fetch a copy of the tuple to scribble on */
ObjectIdAttributeNumber, tuple = SearchSysCacheCopy(DATABASEOID,
BTEqualStrategyNumber, F_OIDEQ, ObjectIdGetDatum(dbid),
ObjectIdGetDatum(dbid)); 0, 0, 0);
scan = systable_beginscan(relation, DatabaseOidIndexId, true,
SnapshotNow, 1, entry);
tuple = systable_getnext(scan);
if (!HeapTupleIsValid(tuple)) if (!HeapTupleIsValid(tuple))
elog(ERROR, "could not find tuple for database %u", dbid); elog(ERROR, "could not find tuple for database %u", dbid);
if (scan->irel)
buf = scan->iscan->xs_cbuf;
else
buf = scan->scan->rs_cbuf;
/* ensure no one else does this at the same time */
LockBuffer(buf, BUFFER_LOCK_EXCLUSIVE);
dbform = (Form_pg_database) GETSTRUCT(tuple); dbform = (Form_pg_database) GETSTRUCT(tuple);
/* overwrite the existing statistics in the tuple */ /* overwrite the existing statistics in the tuple */
dbform->datvacuumxid = vacuumXID; dbform->datvacuumxid = vacuumXID;
dbform->datfrozenxid = frozenXID; dbform->datfrozenxid = frozenXID;
MarkBufferDirty(buf); heap_inplace_update(relation, tuple);
LockBuffer(buf, BUFFER_LOCK_UNLOCK);
/* invalidate the tuple in the cache so we'll see the change in cache */
CacheInvalidateHeapTuple(relation, tuple);
systable_endscan(scan);
heap_close(relation, RowExclusiveLock); heap_close(relation, RowExclusiveLock);

View File

@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2006, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2006, 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/access/genam.h,v 1.59 2006/05/02 22:25:10 tgl Exp $ * $PostgreSQL: pgsql/src/include/access/genam.h,v 1.60 2006/05/10 23:18:39 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
@ -20,6 +20,15 @@
#include "nodes/primnodes.h" #include "nodes/primnodes.h"
/*
* Struct for statistics returned by ambuild
*/
typedef struct IndexBuildResult
{
double heap_tuples; /* # of tuples seen in parent table */
double index_tuples; /* # of tuples inserted into index */
} IndexBuildResult;
/* /*
* Struct for input arguments passed to ambulkdelete and amvacuumcleanup * Struct for input arguments passed to ambulkdelete and amvacuumcleanup
* *

View File

@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2006, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2006, 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/access/heapam.h,v 1.110 2006/04/24 22:24:58 momjian Exp $ * $PostgreSQL: pgsql/src/include/access/heapam.h,v 1.111 2006/05/10 23:18:39 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
@ -168,6 +168,7 @@ extern HTSU_Result heap_lock_tuple(Relation relation, HeapTuple tuple,
Buffer *buffer, ItemPointer ctid, Buffer *buffer, ItemPointer ctid,
TransactionId *update_xmax, CommandId cid, TransactionId *update_xmax, CommandId cid,
LockTupleMode mode, bool nowait); LockTupleMode mode, bool nowait);
extern void heap_inplace_update(Relation relation, HeapTuple tuple);
extern Oid simple_heap_insert(Relation relation, HeapTuple tup); extern Oid simple_heap_insert(Relation relation, HeapTuple tup);
extern void simple_heap_delete(Relation relation, ItemPointer tid); extern void simple_heap_delete(Relation relation, ItemPointer tid);

View File

@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2006, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2006, 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/access/htup.h,v 1.81 2006/03/05 15:58:53 momjian Exp $ * $PostgreSQL: pgsql/src/include/access/htup.h,v 1.82 2006/05/10 23:18:39 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
@ -444,7 +444,7 @@ typedef HeapTupleData *HeapTuple;
#define XLOG_HEAP_CLEAN 0x40 #define XLOG_HEAP_CLEAN 0x40
#define XLOG_HEAP_NEWPAGE 0x50 #define XLOG_HEAP_NEWPAGE 0x50
#define XLOG_HEAP_LOCK 0x60 #define XLOG_HEAP_LOCK 0x60
/* opcode 0x70 still free */ #define XLOG_HEAP_INPLACE 0x70
#define XLOG_HEAP_OPMASK 0x70 #define XLOG_HEAP_OPMASK 0x70
/* /*
* When we insert 1st item on new page in INSERT/UPDATE * When we insert 1st item on new page in INSERT/UPDATE
@ -545,4 +545,13 @@ typedef struct xl_heap_lock
#define SizeOfHeapLock (offsetof(xl_heap_lock, shared_lock) + sizeof(bool)) #define SizeOfHeapLock (offsetof(xl_heap_lock, shared_lock) + sizeof(bool))
/* This is what we need to know about in-place update */
typedef struct xl_heap_inplace
{
xl_heaptid target; /* updated tuple id */
/* TUPLE DATA FOLLOWS AT END OF STRUCT */
} xl_heap_inplace;
#define SizeOfHeapInplace (offsetof(xl_heap_inplace, target) + SizeOfHeapTid)
#endif /* HTUP_H */ #endif /* HTUP_H */

View File

@ -1,13 +1,13 @@
/*------------------------------------------------------------------------- /*-------------------------------------------------------------------------
* *
* index.h * index.h
* prototypes for index.c. * prototypes for catalog/index.c.
* *
* *
* Portions Copyright (c) 1996-2006, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2006, 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/catalog/index.h,v 1.65 2006/03/05 15:58:54 momjian Exp $ * $PostgreSQL: pgsql/src/include/catalog/index.h,v 1.66 2006/05/10 23:18:39 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
@ -37,7 +37,8 @@ extern Oid index_create(Oid heapRelationId,
Oid accessMethodObjectId, Oid accessMethodObjectId,
Oid tableSpaceId, Oid tableSpaceId,
Oid *classObjectId, Oid *classObjectId,
bool primary, bool isprimary,
bool istoast,
bool isconstraint, bool isconstraint,
bool allow_system_table_mods, bool allow_system_table_mods,
bool skip_build); bool skip_build);
@ -52,16 +53,13 @@ extern void FormIndexDatum(IndexInfo *indexInfo,
Datum *values, Datum *values,
bool *isnull); bool *isnull);
extern void IndexCloseAndUpdateStats(Relation heap, double heapTuples,
Relation index, double indexTuples);
extern void setRelhasindex(Oid relid, bool hasindex,
bool isprimary, Oid reltoastidxid);
extern void setNewRelfilenode(Relation relation); extern void setNewRelfilenode(Relation relation);
extern void index_build(Relation heapRelation, Relation indexRelation, extern void index_build(Relation heapRelation,
IndexInfo *indexInfo); Relation indexRelation,
IndexInfo *indexInfo,
bool isprimary,
bool istoast);
extern double IndexBuildHeapScan(Relation heapRelation, extern double IndexBuildHeapScan(Relation heapRelation,
Relation indexRelation, Relation indexRelation,

View File

@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2006, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2006, 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/catalog/pg_proc.h,v 1.410 2006/05/02 22:25:10 tgl Exp $ * $PostgreSQL: pgsql/src/include/catalog/pg_proc.h,v 1.411 2006/05/10 23:18:39 tgl Exp $
* *
* NOTES * NOTES
* The script catalog/genbki.sh reads this file and generates .bki * The script catalog/genbki.sh reads this file and generates .bki
@ -672,7 +672,7 @@ DATA(insert OID = 336 ( btmarkpos PGNSP PGUID 12 f f t f v 1 2278 "2281" _n
DESCR("btree(internal)"); DESCR("btree(internal)");
DATA(insert OID = 337 ( btrestrpos PGNSP PGUID 12 f f t f v 1 2278 "2281" _null_ _null_ _null_ btrestrpos - _null_ )); DATA(insert OID = 337 ( btrestrpos PGNSP PGUID 12 f f t f v 1 2278 "2281" _null_ _null_ _null_ btrestrpos - _null_ ));
DESCR("btree(internal)"); DESCR("btree(internal)");
DATA(insert OID = 338 ( btbuild PGNSP PGUID 12 f f t f v 3 2278 "2281 2281 2281" _null_ _null_ _null_ btbuild - _null_ )); DATA(insert OID = 338 ( btbuild PGNSP PGUID 12 f f t f v 3 2281 "2281 2281 2281" _null_ _null_ _null_ btbuild - _null_ ));
DESCR("btree(internal)"); DESCR("btree(internal)");
DATA(insert OID = 332 ( btbulkdelete PGNSP PGUID 12 f f t f v 4 2281 "2281 2281 2281 2281" _null_ _null_ _null_ btbulkdelete - _null_ )); DATA(insert OID = 332 ( btbulkdelete PGNSP PGUID 12 f f t f v 4 2281 "2281 2281 2281 2281" _null_ _null_ _null_ btbulkdelete - _null_ ));
DESCR("btree(internal)"); DESCR("btree(internal)");
@ -789,7 +789,7 @@ DATA(insert OID = 446 ( hashmarkpos PGNSP PGUID 12 f f t f v 1 2278 "2281" _
DESCR("hash(internal)"); DESCR("hash(internal)");
DATA(insert OID = 447 ( hashrestrpos PGNSP PGUID 12 f f t f v 1 2278 "2281" _null_ _null_ _null_ hashrestrpos - _null_ )); DATA(insert OID = 447 ( hashrestrpos PGNSP PGUID 12 f f t f v 1 2278 "2281" _null_ _null_ _null_ hashrestrpos - _null_ ));
DESCR("hash(internal)"); DESCR("hash(internal)");
DATA(insert OID = 448 ( hashbuild PGNSP PGUID 12 f f t f v 3 2278 "2281 2281 2281" _null_ _null_ _null_ hashbuild - _null_ )); DATA(insert OID = 448 ( hashbuild PGNSP PGUID 12 f f t f v 3 2281 "2281 2281 2281" _null_ _null_ _null_ hashbuild - _null_ ));
DESCR("hash(internal)"); DESCR("hash(internal)");
DATA(insert OID = 442 ( hashbulkdelete PGNSP PGUID 12 f f t f v 4 2281 "2281 2281 2281 2281" _null_ _null_ _null_ hashbulkdelete - _null_ )); DATA(insert OID = 442 ( hashbulkdelete PGNSP PGUID 12 f f t f v 4 2281 "2281 2281 2281 2281" _null_ _null_ _null_ hashbulkdelete - _null_ ));
DESCR("hash(internal)"); DESCR("hash(internal)");
@ -1055,7 +1055,7 @@ DATA(insert OID = 780 ( gistmarkpos PGNSP PGUID 12 f f t f v 1 2278 "2281" _
DESCR("gist(internal)"); DESCR("gist(internal)");
DATA(insert OID = 781 ( gistrestrpos PGNSP PGUID 12 f f t f v 1 2278 "2281" _null_ _null_ _null_ gistrestrpos - _null_ )); DATA(insert OID = 781 ( gistrestrpos PGNSP PGUID 12 f f t f v 1 2278 "2281" _null_ _null_ _null_ gistrestrpos - _null_ ));
DESCR("gist(internal)"); DESCR("gist(internal)");
DATA(insert OID = 782 ( gistbuild PGNSP PGUID 12 f f t f v 3 2278 "2281 2281 2281" _null_ _null_ _null_ gistbuild - _null_ )); DATA(insert OID = 782 ( gistbuild PGNSP PGUID 12 f f t f v 3 2281 "2281 2281 2281" _null_ _null_ _null_ gistbuild - _null_ ));
DESCR("gist(internal)"); DESCR("gist(internal)");
DATA(insert OID = 776 ( gistbulkdelete PGNSP PGUID 12 f f t f v 4 2281 "2281 2281 2281 2281" _null_ _null_ _null_ gistbulkdelete - _null_ )); DATA(insert OID = 776 ( gistbulkdelete PGNSP PGUID 12 f f t f v 4 2281 "2281 2281 2281 2281" _null_ _null_ _null_ gistbulkdelete - _null_ ));
DESCR("gist(internal)"); DESCR("gist(internal)");
@ -3833,7 +3833,7 @@ DATA(insert OID = 2736 ( ginmarkpos PGNSP PGUID 12 f f t f v 1 2278 "2281" _
DESCR("gin(internal)"); DESCR("gin(internal)");
DATA(insert OID = 2737 ( ginrestrpos PGNSP PGUID 12 f f t f v 1 2278 "2281" _null_ _null_ _null_ ginrestrpos - _null_ )); DATA(insert OID = 2737 ( ginrestrpos PGNSP PGUID 12 f f t f v 1 2278 "2281" _null_ _null_ _null_ ginrestrpos - _null_ ));
DESCR("gin(internal)"); DESCR("gin(internal)");
DATA(insert OID = 2738 ( ginbuild PGNSP PGUID 12 f f t f v 3 2278 "2281 2281 2281" _null_ _null_ _null_ ginbuild - _null_ )); DATA(insert OID = 2738 ( ginbuild PGNSP PGUID 12 f f t f v 3 2281 "2281 2281 2281" _null_ _null_ _null_ ginbuild - _null_ ));
DESCR("gin(internal)"); DESCR("gin(internal)");
DATA(insert OID = 2739 ( ginbulkdelete PGNSP PGUID 12 f f t f v 4 2281 "2281 2281 2281 2281" _null_ _null_ _null_ ginbulkdelete - _null_ )); DATA(insert OID = 2739 ( ginbulkdelete PGNSP PGUID 12 f f t f v 4 2281 "2281 2281 2281 2281" _null_ _null_ _null_ ginbulkdelete - _null_ ));
DESCR("gin(internal)"); DESCR("gin(internal)");