#include "access/genam.h"
#include "access/heapam.h"
+#include "access/multixact.h"
+#include "access/rewriteheap.h"
#include "access/tableam.h"
#include "access/xact.h"
#include "catalog/catalog.h"
#include "catalog/index.h"
+#include "catalog/storage.h"
+#include "catalog/storage_xlog.h"
+#include "commands/progress.h"
#include "executor/executor.h"
+#include "pgstat.h"
#include "storage/bufmgr.h"
#include "storage/bufpage.h"
+#include "storage/bufmgr.h"
#include "storage/lmgr.h"
+#include "storage/predicate.h"
#include "storage/procarray.h"
+#include "storage/smgr.h"
#include "utils/builtins.h"
+#include "utils/rel.h"
+
+
+static void
+reform_and_rewrite_tuple(HeapTuple tuple,
+ Relation OldHeap, Relation NewHeap,
+ Datum *values, bool *isnull, RewriteState rwstate);
static const TableAmRoutine heapam_methods;
* ------------------------------------------------------------------------
*/
+static void
+heapam_relation_set_new_filenode(Relation rel, char persistence,
+ TransactionId *freezeXid,
+ MultiXactId *minmulti)
+{
+ /*
+ * Initialize to the minimum XID that could put tuples in the table. We
+ * know that no xacts older than RecentXmin are still running, so that
+ * will do.
+ */
+ *freezeXid = RecentXmin;
+
+ /*
+ * Similarly, initialize the minimum Multixact to the first value that
+ * could possibly be stored in tuples in the table. Running transactions
+ * could reuse values from their local cache, so we are careful to
+ * consider all currently running multis.
+ *
+ * XXX this could be refined further, but is it worth the hassle?
+ */
+ *minmulti = GetOldestMultiXactId();
+
+ RelationCreateStorage(rel->rd_node, persistence);
+
+ /*
+ * If required, set up an init fork for an unlogged table so that it can
+ * be correctly reinitialized on restart. An immediate sync is required
+ * even if the page has been logged, because the write did not go through
+ * shared_buffers and therefore a concurrent checkpoint may have moved the
+ * redo pointer past our xlog record. Recovery may as well remove it
+ * while replaying, for example, XLOG_DBASE_CREATE or XLOG_TBLSPC_CREATE
+ * record. Therefore, logging is necessary even if wal_level=minimal.
+ */
+ if (rel->rd_rel->relpersistence == RELPERSISTENCE_UNLOGGED)
+ {
+ Assert(rel->rd_rel->relkind == RELKIND_RELATION ||
+ rel->rd_rel->relkind == RELKIND_MATVIEW ||
+ rel->rd_rel->relkind == RELKIND_TOASTVALUE);
+ RelationOpenSmgr(rel);
+ smgrcreate(rel->rd_smgr, INIT_FORKNUM, false);
+ log_smgrcreate(&rel->rd_smgr->smgr_rnode.node, INIT_FORKNUM);
+ smgrimmedsync(rel->rd_smgr, INIT_FORKNUM);
+ }
+}
+
+static void
+heapam_relation_nontransactional_truncate(Relation rel)
+{
+ RelationTruncate(rel, 0);
+}
+
+static void
+heapam_relation_copy_data(Relation rel, RelFileNode newrnode)
+{
+ SMgrRelation dstrel;
+
+ dstrel = smgropen(newrnode, rel->rd_backend);
+ RelationOpenSmgr(rel);
+
+ /*
+ * Create and copy all forks of the relation, and schedule unlinking of
+ * old physical files.
+ *
+ * NOTE: any conflict in relfilenode value will be caught in
+ * RelationCreateStorage().
+ */
+ RelationCreateStorage(newrnode, rel->rd_rel->relpersistence);
+
+ /* copy main fork */
+ RelationCopyStorage(rel->rd_smgr, dstrel, MAIN_FORKNUM,
+ rel->rd_rel->relpersistence);
+
+ /* copy those extra forks that exist */
+ for (ForkNumber forkNum = MAIN_FORKNUM + 1;
+ forkNum <= MAX_FORKNUM; forkNum++)
+ {
+ if (smgrexists(rel->rd_smgr, forkNum))
+ {
+ smgrcreate(dstrel, forkNum, false);
+
+ /*
+ * WAL log creation if the relation is persistent, or this is the
+ * init fork of an unlogged relation.
+ */
+ if (rel->rd_rel->relpersistence == RELPERSISTENCE_PERMANENT ||
+ (rel->rd_rel->relpersistence == RELPERSISTENCE_UNLOGGED &&
+ forkNum == INIT_FORKNUM))
+ log_smgrcreate(&newrnode, forkNum);
+ RelationCopyStorage(rel->rd_smgr, dstrel, forkNum,
+ rel->rd_rel->relpersistence);
+ }
+ }
+
+
+ /* drop old relation, and close new one */
+ RelationDropStorage(rel);
+ smgrclose(dstrel);
+}
+
+static void
+heapam_relation_copy_for_cluster(Relation OldHeap, Relation NewHeap,
+ Relation OldIndex, bool use_sort,
+ TransactionId OldestXmin,
+ TransactionId FreezeXid,
+ MultiXactId MultiXactCutoff,
+ double *num_tuples,
+ double *tups_vacuumed,
+ double *tups_recently_dead)
+{
+ RewriteState rwstate;
+ IndexScanDesc indexScan;
+ TableScanDesc tableScan;
+ HeapScanDesc heapScan;
+ bool use_wal;
+ bool is_system_catalog;
+ Tuplesortstate *tuplesort;
+ TupleDesc oldTupDesc = RelationGetDescr(OldHeap);
+ TupleDesc newTupDesc = RelationGetDescr(NewHeap);
+ TupleTableSlot *slot;
+ int natts;
+ Datum *values;
+ bool *isnull;
+ BufferHeapTupleTableSlot *hslot;
+
+ /* Remember if it's a system catalog */
+ is_system_catalog = IsSystemRelation(OldHeap);
+
+ /*
+ * We need to log the copied data in WAL iff WAL archiving/streaming is
+ * enabled AND it's a WAL-logged rel.
+ */
+ use_wal = XLogIsNeeded() && RelationNeedsWAL(NewHeap);
+
+ /* use_wal off requires smgr_targblock be initially invalid */
+ Assert(RelationGetTargetBlock(NewHeap) == InvalidBlockNumber);
+
+ /* Preallocate values/isnull arrays */
+ natts = newTupDesc->natts;
+ values = (Datum *) palloc(natts * sizeof(Datum));
+ isnull = (bool *) palloc(natts * sizeof(bool));
+
+ /* Initialize the rewrite operation */
+ rwstate = begin_heap_rewrite(OldHeap, NewHeap, OldestXmin, FreezeXid,
+ MultiXactCutoff, use_wal);
+
+
+ /* Set up sorting if wanted */
+ if (use_sort)
+ tuplesort = tuplesort_begin_cluster(oldTupDesc, OldIndex,
+ maintenance_work_mem,
+ NULL, false);
+ else
+ tuplesort = NULL;
+
+ /*
+ * Prepare to scan the OldHeap. To ensure we see recently-dead tuples
+ * that still need to be copied, we scan with SnapshotAny and use
+ * HeapTupleSatisfiesVacuum for the visibility test.
+ */
+ if (OldIndex != NULL && !use_sort)
+ {
+ const int ci_index[] = {
+ PROGRESS_CLUSTER_PHASE,
+ PROGRESS_CLUSTER_INDEX_RELID
+ };
+ int64 ci_val[2];
+
+ /* Set phase and OIDOldIndex to columns */
+ ci_val[0] = PROGRESS_CLUSTER_PHASE_INDEX_SCAN_HEAP;
+ ci_val[1] = RelationGetRelid(OldIndex);
+ pgstat_progress_update_multi_param(2, ci_index, ci_val);
+
+ tableScan = NULL;
+ heapScan = NULL;
+ indexScan = index_beginscan(OldHeap, OldIndex, SnapshotAny, 0, 0);
+ index_rescan(indexScan, NULL, 0, NULL, 0);
+ }
+ else
+ {
+ /* In scan-and-sort mode and also VACUUM FULL, set phase */
+ pgstat_progress_update_param(PROGRESS_CLUSTER_PHASE,
+ PROGRESS_CLUSTER_PHASE_SEQ_SCAN_HEAP);
+
+ tableScan = table_beginscan(OldHeap, SnapshotAny, 0, (ScanKey) NULL);
+ heapScan = (HeapScanDesc) tableScan;
+ indexScan = NULL;
+
+ /* Set total heap blocks */
+ pgstat_progress_update_param(PROGRESS_CLUSTER_TOTAL_HEAP_BLKS,
+ heapScan->rs_nblocks);
+ }
+
+ slot = table_slot_create(OldHeap, NULL);
+ hslot = (BufferHeapTupleTableSlot *) slot;
+
+ /*
+ * Scan through the OldHeap, either in OldIndex order or sequentially;
+ * copy each tuple into the NewHeap, or transiently to the tuplesort
+ * module. Note that we don't bother sorting dead tuples (they won't get
+ * to the new table anyway).
+ */
+ for (;;)
+ {
+ HeapTuple tuple;
+ Buffer buf;
+ bool isdead;
+
+ CHECK_FOR_INTERRUPTS();
+
+ if (indexScan != NULL)
+ {
+ if (!index_getnext_slot(indexScan, ForwardScanDirection, slot))
+ break;
+
+ /* Since we used no scan keys, should never need to recheck */
+ if (indexScan->xs_recheck)
+ elog(ERROR, "CLUSTER does not support lossy index conditions");
+ }
+ else
+ {
+ if (!table_scan_getnextslot(tableScan, ForwardScanDirection, slot))
+ break;
+
+ /* In scan-and-sort mode and also VACUUM FULL, set heap blocks scanned */
+ pgstat_progress_update_param(PROGRESS_CLUSTER_HEAP_BLKS_SCANNED,
+ heapScan->rs_cblock + 1);
+ }
+
+ tuple = ExecFetchSlotHeapTuple(slot, false, NULL);
+ buf = hslot->buffer;
+
+ LockBuffer(buf, BUFFER_LOCK_SHARE);
+
+ switch (HeapTupleSatisfiesVacuum(tuple, OldestXmin, buf))
+ {
+ case HEAPTUPLE_DEAD:
+ /* Definitely dead */
+ isdead = true;
+ break;
+ case HEAPTUPLE_RECENTLY_DEAD:
+ *tups_recently_dead += 1;
+ /* fall through */
+ case HEAPTUPLE_LIVE:
+ /* Live or recently dead, must copy it */
+ isdead = false;
+ break;
+ case HEAPTUPLE_INSERT_IN_PROGRESS:
+
+ /*
+ * Since we hold exclusive lock on the relation, normally the
+ * only way to see this is if it was inserted earlier in our
+ * own transaction. However, it can happen in system
+ * catalogs, since we tend to release write lock before commit
+ * there. Give a warning if neither case applies; but in any
+ * case we had better copy it.
+ */
+ if (!is_system_catalog &&
+ !TransactionIdIsCurrentTransactionId(HeapTupleHeaderGetXmin(tuple->t_data)))
+ elog(WARNING, "concurrent insert in progress within table \"%s\"",
+ RelationGetRelationName(OldHeap));
+ /* treat as live */
+ isdead = false;
+ break;
+ case HEAPTUPLE_DELETE_IN_PROGRESS:
+
+ /*
+ * Similar situation to INSERT_IN_PROGRESS case.
+ */
+ if (!is_system_catalog &&
+ !TransactionIdIsCurrentTransactionId(HeapTupleHeaderGetUpdateXid(tuple->t_data)))
+ elog(WARNING, "concurrent delete in progress within table \"%s\"",
+ RelationGetRelationName(OldHeap));
+ /* treat as recently dead */
+ *tups_recently_dead += 1;
+ isdead = false;
+ break;
+ default:
+ elog(ERROR, "unexpected HeapTupleSatisfiesVacuum result");
+ isdead = false; /* keep compiler quiet */
+ break;
+ }
+
+ LockBuffer(buf, BUFFER_LOCK_UNLOCK);
+
+ if (isdead)
+ {
+ *tups_vacuumed += 1;
+ /* heap rewrite module still needs to see it... */
+ if (rewrite_heap_dead_tuple(rwstate, tuple))
+ {
+ /* A previous recently-dead tuple is now known dead */
+ *tups_vacuumed += 1;
+ *tups_recently_dead -= 1;
+ }
+ continue;
+ }
+
+ *num_tuples += 1;
+ if (tuplesort != NULL)
+ {
+ tuplesort_putheaptuple(tuplesort, tuple);
+
+ /* In scan-and-sort mode, report increase in number of tuples scanned */
+ pgstat_progress_update_param(PROGRESS_CLUSTER_HEAP_TUPLES_SCANNED,
+ *num_tuples);
+ }
+ else
+ {
+ const int ct_index[] = {
+ PROGRESS_CLUSTER_HEAP_TUPLES_SCANNED,
+ PROGRESS_CLUSTER_HEAP_TUPLES_WRITTEN
+ };
+ int64 ct_val[2];
+
+ reform_and_rewrite_tuple(tuple, OldHeap, NewHeap,
+ values, isnull, rwstate);
+
+ /*
+ * In indexscan mode and also VACUUM FULL, report increase in
+ * number of tuples scanned and written
+ */
+ ct_val[0] = *num_tuples;
+ ct_val[1] = *num_tuples;
+ pgstat_progress_update_multi_param(2, ct_index, ct_val);
+ }
+ }
+
+ if (indexScan != NULL)
+ index_endscan(indexScan);
+ if (tableScan != NULL)
+ table_endscan(tableScan);
+ if (slot)
+ ExecDropSingleTupleTableSlot(slot);
+
+ /*
+ * In scan-and-sort mode, complete the sort, then read out all live tuples
+ * from the tuplestore and write them to the new relation.
+ */
+ if (tuplesort != NULL)
+ {
+ double n_tuples = 0;
+ /* Report that we are now sorting tuples */
+ pgstat_progress_update_param(PROGRESS_CLUSTER_PHASE,
+ PROGRESS_CLUSTER_PHASE_SORT_TUPLES);
+
+ tuplesort_performsort(tuplesort);
+
+ /* Report that we are now writing new heap */
+ pgstat_progress_update_param(PROGRESS_CLUSTER_PHASE,
+ PROGRESS_CLUSTER_PHASE_WRITE_NEW_HEAP);
+
+ for (;;)
+ {
+ HeapTuple tuple;
+
+ CHECK_FOR_INTERRUPTS();
+
+ tuple = tuplesort_getheaptuple(tuplesort, true);
+ if (tuple == NULL)
+ break;
+
+ n_tuples += 1;
+ reform_and_rewrite_tuple(tuple,
+ OldHeap, NewHeap,
+ values, isnull,
+ rwstate);
+ /* Report n_tuples */
+ pgstat_progress_update_param(PROGRESS_CLUSTER_HEAP_TUPLES_WRITTEN,
+ n_tuples);
+ }
+
+ tuplesort_end(tuplesort);
+ }
+
+ /* Write out any remaining tuples, and fsync if needed */
+ end_heap_rewrite(rwstate);
+
+ /* Clean up */
+ pfree(values);
+ pfree(isnull);
+}
+
static double
heapam_index_build_range_scan(Relation heapRelation,
Relation indexRelation,
}
+/* ----------------------------------------------------------------------------
+ * Helper functions for the above.
+ * ----------------------------------------------------------------------------
+ */
+
+/*
+ * Reconstruct and rewrite the given tuple
+ *
+ * We cannot simply copy the tuple as-is, for several reasons:
+ *
+ * 1. We'd like to squeeze out the values of any dropped columns, both
+ * to save space and to ensure we have no corner-case failures. (It's
+ * possible for example that the new table hasn't got a TOAST table
+ * and so is unable to store any large values of dropped cols.)
+ *
+ * 2. The tuple might not even be legal for the new table; this is
+ * currently only known to happen as an after-effect of ALTER TABLE
+ * SET WITHOUT OIDS.
+ *
+ * So, we must reconstruct the tuple from component Datums.
+ */
+static void
+reform_and_rewrite_tuple(HeapTuple tuple,
+ Relation OldHeap, Relation NewHeap,
+ Datum *values, bool *isnull, RewriteState rwstate)
+{
+ TupleDesc oldTupDesc = RelationGetDescr(OldHeap);
+ TupleDesc newTupDesc = RelationGetDescr(NewHeap);
+ HeapTuple copiedTuple;
+ int i;
+
+ heap_deform_tuple(tuple, oldTupDesc, values, isnull);
+
+ /* Be sure to null out any dropped columns */
+ for (i = 0; i < newTupDesc->natts; i++)
+ {
+ if (TupleDescAttr(newTupDesc, i)->attisdropped)
+ isnull[i] = true;
+ }
+
+ copiedTuple = heap_form_tuple(newTupDesc, values, isnull);
+
+ /* The heap rewrite module does the rest */
+ rewrite_heap_tuple(rwstate, tuple, copiedTuple);
+
+ heap_freetuple(copiedTuple);
+}
+
+
/* ------------------------------------------------------------------------
* Definition of the heap table access method.
* ------------------------------------------------------------------------
.tuple_satisfies_snapshot = heapam_tuple_satisfies_snapshot,
.compute_xid_horizon_for_tuples = heap_compute_xid_horizon_for_tuples,
+ .relation_set_new_filenode = heapam_relation_set_new_filenode,
+ .relation_nontransactional_truncate = heapam_relation_nontransactional_truncate,
+ .relation_copy_data = heapam_relation_copy_data,
+ .relation_copy_for_cluster = heapam_relation_copy_for_cluster,
.index_build_range_scan = heapam_index_build_range_scan,
.index_validate_scan = heapam_index_validate_scan,
};
if ($4)
{
+ TransactionId relfrozenxid;
+ MultiXactId relminmxid;
+
if (boot_reldesc)
{
elog(DEBUG4, "create bootstrap: warning, open relation exists, closing first");
RELPERSISTENCE_PERMANENT,
shared_relation,
mapped_relation,
- true);
+ true,
+ &relfrozenxid,
+ &relminmxid);
elog(DEBUG4, "bootstrap relation created");
}
else
#include "access/relation.h"
#include "access/sysattr.h"
#include "access/table.h"
+#include "access/tableam.h"
#include "access/transam.h"
#include "access/xact.h"
#include "access/xlog.h"
Oid reloftype,
Oid relowner,
char relkind,
+ TransactionId relfrozenxid,
+ TransactionId relminmxid,
Datum relacl,
Datum reloptions);
static ObjectAddress AddNewRelationType(const char *typeName,
char relpersistence,
bool shared_relation,
bool mapped_relation,
- bool allow_system_table_mods)
+ bool allow_system_table_mods,
+ TransactionId *relfrozenxid,
+ MultiXactId *relminmxid)
{
bool create_storage;
Relation rel;
get_namespace_name(relnamespace), relname),
errdetail("System catalog modifications are currently disallowed.")));
+ *relfrozenxid = InvalidTransactionId;
+ *relminmxid = InvalidMultiXactId;
+
/* Handle reltablespace for specific relkinds. */
switch (relkind)
{
/*
* Have the storage manager create the relation's disk file, if needed.
*
- * We only create the main fork here, other forks will be created on
- * demand.
+ * For relations the callback creates both the main and the init fork, for
+ * indexes only the main fork is created. The other forks will be created
+ * on demand.
*/
if (create_storage)
{
RelationOpenSmgr(rel);
- RelationCreateStorage(rel->rd_node, relpersistence);
+
+ switch (rel->rd_rel->relkind)
+ {
+ case RELKIND_VIEW:
+ case RELKIND_COMPOSITE_TYPE:
+ case RELKIND_FOREIGN_TABLE:
+ case RELKIND_PARTITIONED_TABLE:
+ case RELKIND_PARTITIONED_INDEX:
+ Assert(false);
+ break;
+
+ case RELKIND_INDEX:
+ case RELKIND_SEQUENCE:
+ RelationCreateStorage(rel->rd_node, relpersistence);
+ break;
+
+ case RELKIND_RELATION:
+ case RELKIND_TOASTVALUE:
+ case RELKIND_MATVIEW:
+ table_relation_set_new_filenode(rel, relpersistence,
+ relfrozenxid, relminmxid);
+ break;
+ }
}
return rel;
Oid reloftype,
Oid relowner,
char relkind,
+ TransactionId relfrozenxid,
+ TransactionId relminmxid,
Datum relacl,
Datum reloptions)
{
break;
}
- /* Initialize relfrozenxid and relminmxid */
- if (relkind == RELKIND_RELATION ||
- relkind == RELKIND_MATVIEW ||
- relkind == RELKIND_TOASTVALUE)
- {
- /*
- * Initialize to the minimum XID that could put tuples in the table.
- * We know that no xacts older than RecentXmin are still running, so
- * that will do.
- */
- new_rel_reltup->relfrozenxid = RecentXmin;
-
- /*
- * Similarly, initialize the minimum Multixact to the first value that
- * could possibly be stored in tuples in the table. Running
- * transactions could reuse values from their local cache, so we are
- * careful to consider all currently running multis.
- *
- * XXX this could be refined further, but is it worth the hassle?
- */
- new_rel_reltup->relminmxid = GetOldestMultiXactId();
- }
- else
- {
- /*
- * Other relation types will not contain XIDs, so set relfrozenxid to
- * InvalidTransactionId. (Note: a sequence does contain a tuple, but
- * we force its xmin to be FrozenTransactionId always; see
- * commands/sequence.c.)
- */
- new_rel_reltup->relfrozenxid = InvalidTransactionId;
- new_rel_reltup->relminmxid = InvalidMultiXactId;
- }
-
+ new_rel_reltup->relfrozenxid = relfrozenxid;
+ new_rel_reltup->relminmxid = relminmxid;
new_rel_reltup->relowner = relowner;
new_rel_reltup->reltype = new_type_oid;
new_rel_reltup->reloftype = reloftype;
Oid new_type_oid;
ObjectAddress new_type_addr;
Oid new_array_oid = InvalidOid;
+ TransactionId relfrozenxid;
+ MultiXactId relminmxid;
pg_class_desc = table_open(RelationRelationId, RowExclusiveLock);
relpersistence,
shared_relation,
mapped_relation,
- allow_system_table_mods);
+ allow_system_table_mods,
+ &relfrozenxid,
+ &relminmxid);
Assert(relid == RelationGetRelid(new_rel_desc));
reloftypeid,
ownerid,
relkind,
+ relfrozenxid,
+ relminmxid,
PointerGetDatum(relacl),
reloptions);
if (oncommit != ONCOMMIT_NOOP)
register_on_commit_action(relid, oncommit);
- /*
- * Unlogged objects need an init fork, except for partitioned tables which
- * have no storage at all.
- */
- if (relpersistence == RELPERSISTENCE_UNLOGGED &&
- relkind != RELKIND_PARTITIONED_TABLE)
- heap_create_init_fork(new_rel_desc);
-
/*
* ok, the relation has been cataloged, so close our relations and return
* the OID of the newly created relation.
return relid;
}
-/*
- * Set up an init fork for an unlogged table so that it can be correctly
- * reinitialized on restart. An immediate sync is required even if the
- * page has been logged, because the write did not go through
- * shared_buffers and therefore a concurrent checkpoint may have moved
- * the redo pointer past our xlog record. Recovery may as well remove it
- * while replaying, for example, XLOG_DBASE_CREATE or XLOG_TBLSPC_CREATE
- * record. Therefore, logging is necessary even if wal_level=minimal.
- */
-void
-heap_create_init_fork(Relation rel)
-{
- Assert(rel->rd_rel->relkind == RELKIND_RELATION ||
- rel->rd_rel->relkind == RELKIND_MATVIEW ||
- rel->rd_rel->relkind == RELKIND_TOASTVALUE);
- RelationOpenSmgr(rel);
- smgrcreate(rel->rd_smgr, INIT_FORKNUM, false);
- log_smgrcreate(&rel->rd_smgr->smgr_rnode.node, INIT_FORKNUM);
- smgrimmedsync(rel->rd_smgr, INIT_FORKNUM);
-}
-
/*
* RelationRemoveInheritance
*
if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
return;
- /* Truncate the actual file (and discard buffers) */
- RelationTruncate(rel, 0);
+ /* Truncate the underlying relation */
+ table_relation_nontransactional_truncate(rel);
/* If the relation has indexes, truncate the indexes too */
RelationTruncateIndexes(rel);
{
Relation toastrel = table_open(toastrelid, AccessExclusiveLock);
- RelationTruncate(toastrel, 0);
+ table_relation_nontransactional_truncate(toastrel);
RelationTruncateIndexes(toastrel);
/* keep the lock... */
table_close(toastrel, NoLock);
bool concurrent = (flags & INDEX_CREATE_CONCURRENT) != 0;
bool partitioned = (flags & INDEX_CREATE_PARTITIONED) != 0;
char relkind;
+ TransactionId relfrozenxid;
+ MultiXactId relminmxid;
/* constraint flags can only be set when a constraint is requested */
Assert((constr_flags == 0) ||
relpersistence,
shared_relation,
mapped_relation,
- allow_system_table_mods);
+ allow_system_table_mods,
+ &relfrozenxid,
+ &relminmxid);
+ Assert(relfrozenxid == InvalidTransactionId);
+ Assert(relminmxid == InvalidMultiXactId);
Assert(indexRelationId == RelationGetRelid(indexRelation));
/*
}
/* We'll build a new physical relation for the index */
- RelationSetNewRelfilenode(iRel, persistence, InvalidTransactionId,
- InvalidMultiXactId);
+ RelationSetNewRelfilenode(iRel, persistence);
/* Initialize the index and rebuild */
/* Note: we do not need to re-establish pkey setting */
#include "postgres.h"
+#include "miscadmin.h"
+
#include "access/visibilitymap.h"
#include "access/xact.h"
#include "access/xlog.h"
smgrtruncate(rel->rd_smgr, MAIN_FORKNUM, nblocks);
}
+/*
+ * Copy a fork's data, block by block.
+ */
+void
+RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
+ ForkNumber forkNum, char relpersistence)
+{
+ PGAlignedBlock buf;
+ Page page;
+ bool use_wal;
+ bool copying_initfork;
+ BlockNumber nblocks;
+ BlockNumber blkno;
+
+ page = (Page) buf.data;
+
+ /*
+ * The init fork for an unlogged relation in many respects has to be
+ * treated the same as normal relation, changes need to be WAL logged and
+ * it needs to be synced to disk.
+ */
+ copying_initfork = relpersistence == RELPERSISTENCE_UNLOGGED &&
+ forkNum == INIT_FORKNUM;
+
+ /*
+ * We need to log the copied data in WAL iff WAL archiving/streaming is
+ * enabled AND it's a permanent relation.
+ */
+ use_wal = XLogIsNeeded() &&
+ (relpersistence == RELPERSISTENCE_PERMANENT || copying_initfork);
+
+ nblocks = smgrnblocks(src, forkNum);
+
+ for (blkno = 0; blkno < nblocks; blkno++)
+ {
+ /* If we got a cancel signal during the copy of the data, quit */
+ CHECK_FOR_INTERRUPTS();
+
+ smgrread(src, forkNum, blkno, buf.data);
+
+ if (!PageIsVerified(page, blkno))
+ ereport(ERROR,
+ (errcode(ERRCODE_DATA_CORRUPTED),
+ errmsg("invalid page in block %u of relation %s",
+ blkno,
+ relpathbackend(src->smgr_rnode.node,
+ src->smgr_rnode.backend,
+ forkNum))));
+
+ /*
+ * WAL-log the copied page. Unfortunately we don't know what kind of a
+ * page this is, so we have to log the full page including any unused
+ * space.
+ */
+ if (use_wal)
+ log_newpage(&dst->smgr_rnode.node, forkNum, blkno, page, false);
+
+ PageSetChecksumInplace(page, blkno);
+
+ /*
+ * Now write the page. We say isTemp = true even if it's not a temp
+ * rel, because there's no need for smgr to schedule an fsync for this
+ * write; we'll do it ourselves below.
+ */
+ smgrextend(dst, forkNum, blkno, buf.data, true);
+ }
+
+ /*
+ * If the rel is WAL-logged, must fsync before commit. We use heap_sync
+ * to ensure that the toast table gets fsync'd too. (For a temp or
+ * unlogged rel we don't care since the data will be gone after a crash
+ * anyway.)
+ *
+ * It's obvious that we must do this when not WAL-logging the copy. It's
+ * less obvious that we have to do it even if we did WAL-log the copied
+ * pages. The reason is that since we're copying outside shared buffers, a
+ * CHECKPOINT occurring during the copy has no way to flush the previously
+ * written data to disk (indeed it won't know the new rel even exists). A
+ * crash later on would replay WAL from the checkpoint, therefore it
+ * wouldn't replay our earlier WAL entries. If we do not fsync those pages
+ * here, they might still not be on disk when the crash occurs.
+ */
+ if (relpersistence == RELPERSISTENCE_PERMANENT || copying_initfork)
+ smgrimmedsync(dst, forkNum);
+}
+
/*
* smgrDoPendingDeletes() -- Take care of relation deletes at end of xact.
*
#include "access/heapam.h"
#include "access/multixact.h"
#include "access/relscan.h"
-#include "access/rewriteheap.h"
#include "access/tableam.h"
#include "access/transam.h"
#include "access/tuptoaster.h"
#include "storage/bufmgr.h"
#include "storage/lmgr.h"
#include "storage/predicate.h"
-#include "storage/smgr.h"
#include "utils/acl.h"
#include "utils/fmgroids.h"
#include "utils/inval.h"
static void rebuild_relation(Relation OldHeap, Oid indexOid, bool verbose);
-static void copy_heap_data(Oid OIDNewHeap, Oid OIDOldHeap, Oid OIDOldIndex,
+static void copy_table_data(Oid OIDNewHeap, Oid OIDOldHeap, Oid OIDOldIndex,
bool verbose, bool *pSwapToastByContent,
TransactionId *pFreezeXid, MultiXactId *pCutoffMulti);
static List *get_tables_to_cluster(MemoryContext cluster_context);
-static void reform_and_rewrite_tuple(HeapTuple tuple,
- TupleDesc oldTupDesc, TupleDesc newTupDesc,
- Datum *values, bool *isnull,
- RewriteState rwstate);
/*---------------------------------------------------------------------------
AccessExclusiveLock);
/* Copy the heap data into the new table in the desired order */
- copy_heap_data(OIDNewHeap, tableOid, indexOid, verbose,
+ copy_table_data(OIDNewHeap, tableOid, indexOid, verbose,
&swap_toast_by_content, &frozenXid, &cutoffMulti);
/*
}
/*
- * Do the physical copying of heap data.
+ * Do the physical copying of table data.
*
* There are three output parameters:
* *pSwapToastByContent is set true if toast tables must be swapped by content.
* *pCutoffMulti receives the MultiXactId used as a cutoff point.
*/
static void
-copy_heap_data(Oid OIDNewHeap, Oid OIDOldHeap, Oid OIDOldIndex, bool verbose,
- bool *pSwapToastByContent, TransactionId *pFreezeXid,
- MultiXactId *pCutoffMulti)
+copy_table_data(Oid OIDNewHeap, Oid OIDOldHeap, Oid OIDOldIndex, bool verbose,
+ bool *pSwapToastByContent, TransactionId *pFreezeXid,
+ MultiXactId *pCutoffMulti)
{
Relation NewHeap,
OldHeap,
Relation relRelation;
HeapTuple reltup;
Form_pg_class relform;
- TupleDesc oldTupDesc;
- TupleDesc newTupDesc;
- int natts;
- Datum *values;
- bool *isnull;
- IndexScanDesc indexScan;
- TableScanDesc tableScan;
- HeapScanDesc heapScan;
- bool use_wal;
- bool is_system_catalog;
+ TupleDesc oldTupDesc PG_USED_FOR_ASSERTS_ONLY;
+ TupleDesc newTupDesc PG_USED_FOR_ASSERTS_ONLY;
TransactionId OldestXmin;
TransactionId FreezeXid;
MultiXactId MultiXactCutoff;
- RewriteState rwstate;
bool use_sort;
- Tuplesortstate *tuplesort;
double num_tuples = 0,
tups_vacuumed = 0,
tups_recently_dead = 0;
BlockNumber num_pages;
int elevel = verbose ? INFO : DEBUG2;
PGRUsage ru0;
- TupleTableSlot *slot;
- BufferHeapTupleTableSlot *hslot;
pg_rusage_init(&ru0);
newTupDesc = RelationGetDescr(NewHeap);
Assert(newTupDesc->natts == oldTupDesc->natts);
- /* Preallocate values/isnull arrays */
- natts = newTupDesc->natts;
- values = (Datum *) palloc(natts * sizeof(Datum));
- isnull = (bool *) palloc(natts * sizeof(bool));
-
/*
* If the OldHeap has a toast table, get lock on the toast table to keep
* it from being vacuumed. This is needed because autovacuum processes
if (OldHeap->rd_rel->reltoastrelid)
LockRelationOid(OldHeap->rd_rel->reltoastrelid, AccessExclusiveLock);
- /*
- * We need to log the copied data in WAL iff WAL archiving/streaming is
- * enabled AND it's a WAL-logged rel.
- */
- use_wal = XLogIsNeeded() && RelationNeedsWAL(NewHeap);
-
- /* use_wal off requires smgr_targblock be initially invalid */
- Assert(RelationGetTargetBlock(NewHeap) == InvalidBlockNumber);
-
/*
* If both tables have TOAST tables, perform toast swap by content. It is
* possible that the old table has a toast table but the new one doesn't,
*pFreezeXid = FreezeXid;
*pCutoffMulti = MultiXactCutoff;
- /* Remember if it's a system catalog */
- is_system_catalog = IsSystemRelation(OldHeap);
-
- /* Initialize the rewrite operation */
- rwstate = begin_heap_rewrite(OldHeap, NewHeap, OldestXmin, FreezeXid,
- MultiXactCutoff, use_wal);
-
/*
* Decide whether to use an indexscan or seqscan-and-optional-sort to scan
* the OldHeap. We know how to use a sort to duplicate the ordering of a
else
use_sort = false;
- /* Set up sorting if wanted */
- if (use_sort)
- tuplesort = tuplesort_begin_cluster(oldTupDesc, OldIndex,
- maintenance_work_mem,
- NULL, false);
- else
- tuplesort = NULL;
-
- /*
- * Prepare to scan the OldHeap. To ensure we see recently-dead tuples
- * that still need to be copied, we scan with SnapshotAny and use
- * HeapTupleSatisfiesVacuum for the visibility test.
- */
- if (OldIndex != NULL && !use_sort)
- {
- const int ci_index[] = {
- PROGRESS_CLUSTER_PHASE,
- PROGRESS_CLUSTER_INDEX_RELID
- };
- int64 ci_val[2];
-
- /* Set phase and OIDOldIndex to columns */
- ci_val[0] = PROGRESS_CLUSTER_PHASE_INDEX_SCAN_HEAP;
- ci_val[1] = OIDOldIndex;
- pgstat_progress_update_multi_param(2, ci_index, ci_val);
-
- tableScan = NULL;
- heapScan = NULL;
- indexScan = index_beginscan(OldHeap, OldIndex, SnapshotAny, 0, 0);
- index_rescan(indexScan, NULL, 0, NULL, 0);
- }
- else
- {
- /* In scan-and-sort mode and also VACUUM FULL, set phase */
- pgstat_progress_update_param(PROGRESS_CLUSTER_PHASE,
- PROGRESS_CLUSTER_PHASE_SEQ_SCAN_HEAP);
-
- tableScan = table_beginscan(OldHeap, SnapshotAny, 0, (ScanKey) NULL);
- heapScan = (HeapScanDesc) tableScan;
- indexScan = NULL;
-
- /* Set total heap blocks */
- pgstat_progress_update_param(PROGRESS_CLUSTER_TOTAL_HEAP_BLKS,
- heapScan->rs_nblocks);
- }
-
- slot = table_slot_create(OldHeap, NULL);
- hslot = (BufferHeapTupleTableSlot *) slot;
-
/* Log what we're doing */
- if (indexScan != NULL)
+ if (OldIndex != NULL && !use_sort)
ereport(elevel,
(errmsg("clustering \"%s.%s\" using index scan on \"%s\"",
get_namespace_name(RelationGetNamespace(OldHeap)),
RelationGetRelationName(OldHeap),
RelationGetRelationName(OldIndex))));
- else if (tuplesort != NULL)
+ else if (use_sort)
ereport(elevel,
(errmsg("clustering \"%s.%s\" using sequential scan and sort",
get_namespace_name(RelationGetNamespace(OldHeap)),
RelationGetRelationName(OldHeap))));
/*
- * Scan through the OldHeap, either in OldIndex order or sequentially;
- * copy each tuple into the NewHeap, or transiently to the tuplesort
- * module. Note that we don't bother sorting dead tuples (they won't get
- * to the new table anyway).
+ * Hand of the actual copying to AM specific function, the generic code
+ * cannot know how to deal with visibility across AMs.
*/
- for (;;)
- {
- HeapTuple tuple;
- Buffer buf;
- bool isdead;
-
- CHECK_FOR_INTERRUPTS();
-
- if (indexScan != NULL)
- {
- if (!index_getnext_slot(indexScan, ForwardScanDirection, slot))
- break;
-
- /* Since we used no scan keys, should never need to recheck */
- if (indexScan->xs_recheck)
- elog(ERROR, "CLUSTER does not support lossy index conditions");
-
- tuple = hslot->base.tuple;
- buf = hslot->buffer;
- }
- else
- {
- tuple = heap_getnext(tableScan, ForwardScanDirection);
- if (tuple == NULL)
- break;
-
- buf = heapScan->rs_cbuf;
-
- /* In scan-and-sort mode and also VACUUM FULL, set heap blocks scanned */
- pgstat_progress_update_param(PROGRESS_CLUSTER_HEAP_BLKS_SCANNED,
- heapScan->rs_cblock + 1);
- }
-
- LockBuffer(buf, BUFFER_LOCK_SHARE);
-
- switch (HeapTupleSatisfiesVacuum(tuple, OldestXmin, buf))
- {
- case HEAPTUPLE_DEAD:
- /* Definitely dead */
- isdead = true;
- break;
- case HEAPTUPLE_RECENTLY_DEAD:
- tups_recently_dead += 1;
- /* fall through */
- case HEAPTUPLE_LIVE:
- /* Live or recently dead, must copy it */
- isdead = false;
- break;
- case HEAPTUPLE_INSERT_IN_PROGRESS:
-
- /*
- * Since we hold exclusive lock on the relation, normally the
- * only way to see this is if it was inserted earlier in our
- * own transaction. However, it can happen in system
- * catalogs, since we tend to release write lock before commit
- * there. Give a warning if neither case applies; but in any
- * case we had better copy it.
- */
- if (!is_system_catalog &&
- !TransactionIdIsCurrentTransactionId(HeapTupleHeaderGetXmin(tuple->t_data)))
- elog(WARNING, "concurrent insert in progress within table \"%s\"",
- RelationGetRelationName(OldHeap));
- /* treat as live */
- isdead = false;
- break;
- case HEAPTUPLE_DELETE_IN_PROGRESS:
-
- /*
- * Similar situation to INSERT_IN_PROGRESS case.
- */
- if (!is_system_catalog &&
- !TransactionIdIsCurrentTransactionId(HeapTupleHeaderGetUpdateXid(tuple->t_data)))
- elog(WARNING, "concurrent delete in progress within table \"%s\"",
- RelationGetRelationName(OldHeap));
- /* treat as recently dead */
- tups_recently_dead += 1;
- isdead = false;
- break;
- default:
- elog(ERROR, "unexpected HeapTupleSatisfiesVacuum result");
- isdead = false; /* keep compiler quiet */
- break;
- }
-
- LockBuffer(buf, BUFFER_LOCK_UNLOCK);
-
- if (isdead)
- {
- tups_vacuumed += 1;
- /* heap rewrite module still needs to see it... */
- if (rewrite_heap_dead_tuple(rwstate, tuple))
- {
- /* A previous recently-dead tuple is now known dead */
- tups_vacuumed += 1;
- tups_recently_dead -= 1;
- }
- continue;
- }
-
- num_tuples += 1;
- if (tuplesort != NULL)
- {
- tuplesort_putheaptuple(tuplesort, tuple);
-
- /* In scan-and-sort mode, report increase in number of tuples scanned */
- pgstat_progress_update_param(PROGRESS_CLUSTER_HEAP_TUPLES_SCANNED,
- num_tuples);
- }
- else
- {
- const int ct_index[] = {
- PROGRESS_CLUSTER_HEAP_TUPLES_SCANNED,
- PROGRESS_CLUSTER_HEAP_TUPLES_WRITTEN
- };
- int64 ct_val[2];
-
- reform_and_rewrite_tuple(tuple,
- oldTupDesc, newTupDesc,
- values, isnull,
- rwstate);
-
- /* In indexscan mode and also VACUUM FULL, report increase in number of tuples scanned and written */
- ct_val[0] = num_tuples;
- ct_val[1] = num_tuples;
- pgstat_progress_update_multi_param(2, ct_index, ct_val);
- }
- }
-
- if (indexScan != NULL)
- index_endscan(indexScan);
- if (heapScan != NULL)
- table_endscan(tableScan);
- if (slot)
- ExecDropSingleTupleTableSlot(slot);
-
- /*
- * In scan-and-sort mode, complete the sort, then read out all live tuples
- * from the tuplestore and write them to the new relation.
- */
- if (tuplesort != NULL)
- {
- double n_tuples = 0;
- /* Report that we are now sorting tuples */
- pgstat_progress_update_param(PROGRESS_CLUSTER_PHASE,
- PROGRESS_CLUSTER_PHASE_SORT_TUPLES);
-
- tuplesort_performsort(tuplesort);
-
- /* Report that we are now writing new heap */
- pgstat_progress_update_param(PROGRESS_CLUSTER_PHASE,
- PROGRESS_CLUSTER_PHASE_WRITE_NEW_HEAP);
-
- for (;;)
- {
- HeapTuple tuple;
-
- CHECK_FOR_INTERRUPTS();
-
- tuple = tuplesort_getheaptuple(tuplesort, true);
- if (tuple == NULL)
- break;
-
- n_tuples += 1;
- reform_and_rewrite_tuple(tuple,
- oldTupDesc, newTupDesc,
- values, isnull,
- rwstate);
- /* Report n_tuples */
- pgstat_progress_update_param(PROGRESS_CLUSTER_HEAP_TUPLES_WRITTEN,
- n_tuples);
- }
-
- tuplesort_end(tuplesort);
- }
-
- /* Write out any remaining tuples, and fsync if needed */
- end_heap_rewrite(rwstate);
+ table_relation_copy_for_cluster(OldHeap, NewHeap, OldIndex, use_sort,
+ OldestXmin, FreezeXid, MultiXactCutoff,
+ &num_tuples, &tups_vacuumed,
+ &tups_recently_dead);
/* Reset rd_toastoid just to be tidy --- it shouldn't be looked at again */
NewHeap->rd_toastoid = InvalidOid;
tups_recently_dead,
pg_rusage_show(&ru0))));
- /* Clean up */
- pfree(values);
- pfree(isnull);
-
if (OldIndex != NULL)
index_close(OldIndex, NoLock);
table_close(OldHeap, NoLock);
return rvs;
}
-
-
-/*
- * Reconstruct and rewrite the given tuple
- *
- * We cannot simply copy the tuple as-is, for several reasons:
- *
- * 1. We'd like to squeeze out the values of any dropped columns, both
- * to save space and to ensure we have no corner-case failures. (It's
- * possible for example that the new table hasn't got a TOAST table
- * and so is unable to store any large values of dropped cols.)
- *
- * 2. The tuple might not even be legal for the new table; this is
- * currently only known to happen as an after-effect of ALTER TABLE
- * SET WITHOUT OIDS (in an older version, via pg_upgrade).
- *
- * So, we must reconstruct the tuple from component Datums.
- */
-static void
-reform_and_rewrite_tuple(HeapTuple tuple,
- TupleDesc oldTupDesc, TupleDesc newTupDesc,
- Datum *values, bool *isnull,
- RewriteState rwstate)
-{
- HeapTuple copiedTuple;
- int i;
-
- heap_deform_tuple(tuple, oldTupDesc, values, isnull);
-
- /* Be sure to null out any dropped columns */
- for (i = 0; i < newTupDesc->natts; i++)
- {
- if (TupleDescAttr(newTupDesc, i)->attisdropped)
- isnull[i] = true;
- }
-
- copiedTuple = heap_form_tuple(newTupDesc, values, isnull);
-
- /* The heap rewrite module does the rest */
- rewrite_heap_tuple(rwstate, tuple, copiedTuple);
-
- heap_freetuple(copiedTuple);
-}
seq->log_cnt = 0;
/*
- * Create a new storage file for the sequence. We want to keep the
- * sequence's relfrozenxid at 0, since it won't contain any unfrozen XIDs.
- * Same with relminmxid, since a sequence will never contain multixacts.
+ * Create a new storage file for the sequence.
*/
- RelationSetNewRelfilenode(seq_rel, seq_rel->rd_rel->relpersistence,
- InvalidTransactionId, InvalidMultiXactId);
+ RelationSetNewRelfilenode(seq_rel, seq_rel->rd_rel->relpersistence);
+
+ /*
+ * Ensure sequence's relfrozenxid is at 0, since it won't contain any
+ * unfrozen XIDs. Same with relminmxid, since a sequence will never
+ * contain multixacts.
+ */
+ Assert(seq_rel->rd_rel->relfrozenxid == InvalidTransactionId);
+ Assert(seq_rel->rd_rel->relminmxid == InvalidMultiXactId);
/*
* Insert the modified tuple into the new storage file.
/*
* Create a new storage file for the sequence, making the state
- * changes transactional. We want to keep the sequence's relfrozenxid
- * at 0, since it won't contain any unfrozen XIDs. Same with
- * relminmxid, since a sequence will never contain multixacts.
+ * changes transactional.
+ */
+ RelationSetNewRelfilenode(seqrel, seqrel->rd_rel->relpersistence);
+
+ /*
+ * Ensure sequence's relfrozenxid is at 0, since it won't contain any
+ * unfrozen XIDs. Same with relminmxid, since a sequence will never
+ * contain multixacts.
*/
- RelationSetNewRelfilenode(seqrel, seqrel->rd_rel->relpersistence,
- InvalidTransactionId, InvalidMultiXactId);
+ Assert(seqrel->rd_rel->relfrozenxid == InvalidTransactionId);
+ Assert(seqrel->rd_rel->relminmxid == InvalidMultiXactId);
/*
* Insert the modified tuple into the new storage file.
#include "access/multixact.h"
#include "access/reloptions.h"
#include "access/relscan.h"
+#include "access/tableam.h"
#include "access/sysattr.h"
#include "access/tableam.h"
#include "access/tupconvert.h"
static void ATExecDisableRowSecurity(Relation rel);
static void ATExecForceNoForceRowSecurity(Relation rel, bool force_rls);
-static void copy_relation_data(SMgrRelation rel, SMgrRelation dst,
- ForkNumber forkNum, char relpersistence);
+static void index_copy_data(Relation rel, RelFileNode newrnode);
static const char *storage_name(char c);
static void RangeVarCallbackForDropRelation(const RangeVar *rel, Oid relOid,
{
Oid heap_relid;
Oid toast_relid;
- MultiXactId minmulti;
/*
* This effectively deletes all rows in the table, and may be done
*/
CheckTableForSerializableConflictIn(rel);
- minmulti = GetOldestMultiXactId();
-
/*
* Need the full transaction-safe pushups.
*
* as the relfilenode value. The old storage file is scheduled for
* deletion at commit.
*/
- RelationSetNewRelfilenode(rel, rel->rd_rel->relpersistence,
- RecentXmin, minmulti);
- if (rel->rd_rel->relpersistence == RELPERSISTENCE_UNLOGGED)
- heap_create_init_fork(rel);
+ RelationSetNewRelfilenode(rel, rel->rd_rel->relpersistence);
heap_relid = RelationGetRelid(rel);
{
Relation toastrel = relation_open(toast_relid,
AccessExclusiveLock);
-
RelationSetNewRelfilenode(toastrel,
- toastrel->rd_rel->relpersistence,
- RecentXmin, minmulti);
- if (toastrel->rd_rel->relpersistence == RELPERSISTENCE_UNLOGGED)
- heap_create_init_fork(toastrel);
+ toastrel->rd_rel->relpersistence);
table_close(toastrel, NoLock);
}
/* Write the tuple out to the new relation */
if (newrel)
- {
- HeapTuple tuple;
-
- tuple = ExecFetchSlotHeapTuple(newslot, true, NULL);
- heap_insert(newrel, tuple, mycid, hi_options, bistate);
- ItemPointerCopy(&tuple->t_self, &newslot->tts_tid);
- }
+ table_insert(newrel, insertslot, mycid, hi_options, bistate);
ResetExprContext(econtext);
Oid reltoastrelid;
Oid newrelfilenode;
RelFileNode newrnode;
- SMgrRelation dstrel;
Relation pg_class;
HeapTuple tuple;
Form_pg_class rd_rel;
- ForkNumber forkNum;
List *reltoastidxids = NIL;
ListCell *lc;
newrnode = rel->rd_node;
newrnode.relNode = newrelfilenode;
newrnode.spcNode = newTableSpace;
- dstrel = smgropen(newrnode, rel->rd_backend);
-
- RelationOpenSmgr(rel);
-
- /*
- * Create and copy all forks of the relation, and schedule unlinking of
- * old physical files.
- *
- * NOTE: any conflict in relfilenode value will be caught in
- * RelationCreateStorage().
- */
- RelationCreateStorage(newrnode, rel->rd_rel->relpersistence);
-
- /* copy main fork */
- copy_relation_data(rel->rd_smgr, dstrel, MAIN_FORKNUM,
- rel->rd_rel->relpersistence);
- /* copy those extra forks that exist */
- for (forkNum = MAIN_FORKNUM + 1; forkNum <= MAX_FORKNUM; forkNum++)
+ /* hand off to AM to actually create the new filenode and copy the data */
+ if (rel->rd_rel->relkind == RELKIND_INDEX)
{
- if (smgrexists(rel->rd_smgr, forkNum))
- {
- smgrcreate(dstrel, forkNum, false);
-
- /*
- * WAL log creation if the relation is persistent, or this is the
- * init fork of an unlogged relation.
- */
- if (rel->rd_rel->relpersistence == RELPERSISTENCE_PERMANENT ||
- (rel->rd_rel->relpersistence == RELPERSISTENCE_UNLOGGED &&
- forkNum == INIT_FORKNUM))
- log_smgrcreate(&newrnode, forkNum);
- copy_relation_data(rel->rd_smgr, dstrel, forkNum,
- rel->rd_rel->relpersistence);
- }
+ index_copy_data(rel, newrnode);
+ }
+ else
+ {
+ Assert(rel->rd_rel->relkind == RELKIND_RELATION ||
+ rel->rd_rel->relkind == RELKIND_MATVIEW ||
+ rel->rd_rel->relkind == RELKIND_TOASTVALUE);
+ table_relation_copy_data(rel, newrnode);
}
-
- /* drop old relation, and close new one */
- RelationDropStorage(rel);
- smgrclose(dstrel);
/* update the pg_class row */
rd_rel->reltablespace = (newTableSpace == MyDatabaseTableSpace) ? InvalidOid : newTableSpace;
return new_tablespaceoid;
}
-/*
- * Copy data, block by block
- */
static void
-copy_relation_data(SMgrRelation src, SMgrRelation dst,
- ForkNumber forkNum, char relpersistence)
+index_copy_data(Relation rel, RelFileNode newrnode)
{
- PGAlignedBlock buf;
- Page page;
- bool use_wal;
- bool copying_initfork;
- BlockNumber nblocks;
- BlockNumber blkno;
-
- page = (Page) buf.data;
+ SMgrRelation dstrel;
- /*
- * The init fork for an unlogged relation in many respects has to be
- * treated the same as normal relation, changes need to be WAL logged and
- * it needs to be synced to disk.
- */
- copying_initfork = relpersistence == RELPERSISTENCE_UNLOGGED &&
- forkNum == INIT_FORKNUM;
+ dstrel = smgropen(newrnode, rel->rd_backend);
+ RelationOpenSmgr(rel);
/*
- * We need to log the copied data in WAL iff WAL archiving/streaming is
- * enabled AND it's a permanent relation.
+ * Create and copy all forks of the relation, and schedule unlinking of
+ * old physical files.
+ *
+ * NOTE: any conflict in relfilenode value will be caught in
+ * RelationCreateStorage().
*/
- use_wal = XLogIsNeeded() &&
- (relpersistence == RELPERSISTENCE_PERMANENT || copying_initfork);
+ RelationCreateStorage(newrnode, rel->rd_rel->relpersistence);
- nblocks = smgrnblocks(src, forkNum);
+ /* copy main fork */
+ RelationCopyStorage(rel->rd_smgr, dstrel, MAIN_FORKNUM,
+ rel->rd_rel->relpersistence);
- for (blkno = 0; blkno < nblocks; blkno++)
+ /* copy those extra forks that exist */
+ for (ForkNumber forkNum = MAIN_FORKNUM + 1;
+ forkNum <= MAX_FORKNUM; forkNum++)
{
- /* If we got a cancel signal during the copy of the data, quit */
- CHECK_FOR_INTERRUPTS();
-
- smgrread(src, forkNum, blkno, buf.data);
-
- if (!PageIsVerified(page, blkno))
- ereport(ERROR,
- (errcode(ERRCODE_DATA_CORRUPTED),
- errmsg("invalid page in block %u of relation %s",
- blkno,
- relpathbackend(src->smgr_rnode.node,
- src->smgr_rnode.backend,
- forkNum))));
-
- /*
- * WAL-log the copied page. Unfortunately we don't know what kind of a
- * page this is, so we have to log the full page including any unused
- * space.
- */
- if (use_wal)
- log_newpage(&dst->smgr_rnode.node, forkNum, blkno, page, false);
-
- PageSetChecksumInplace(page, blkno);
+ if (smgrexists(rel->rd_smgr, forkNum))
+ {
+ smgrcreate(dstrel, forkNum, false);
- /*
- * Now write the page. We say isTemp = true even if it's not a temp
- * rel, because there's no need for smgr to schedule an fsync for this
- * write; we'll do it ourselves below.
- */
- smgrextend(dst, forkNum, blkno, buf.data, true);
+ /*
+ * WAL log creation if the relation is persistent, or this is the
+ * init fork of an unlogged relation.
+ */
+ if (rel->rd_rel->relpersistence == RELPERSISTENCE_PERMANENT ||
+ (rel->rd_rel->relpersistence == RELPERSISTENCE_UNLOGGED &&
+ forkNum == INIT_FORKNUM))
+ log_smgrcreate(&newrnode, forkNum);
+ RelationCopyStorage(rel->rd_smgr, dstrel, forkNum,
+ rel->rd_rel->relpersistence);
+ }
}
- /*
- * If the rel is WAL-logged, must fsync before commit. We use heap_sync
- * to ensure that the toast table gets fsync'd too. (For a temp or
- * unlogged rel we don't care since the data will be gone after a crash
- * anyway.)
- *
- * It's obvious that we must do this when not WAL-logging the copy. It's
- * less obvious that we have to do it even if we did WAL-log the copied
- * pages. The reason is that since we're copying outside shared buffers, a
- * CHECKPOINT occurring during the copy has no way to flush the previously
- * written data to disk (indeed it won't know the new rel even exists). A
- * crash later on would replay WAL from the checkpoint, therefore it
- * wouldn't replay our earlier WAL entries. If we do not fsync those pages
- * here, they might still not be on disk when the crash occurs.
- */
- if (relpersistence == RELPERSISTENCE_PERMANENT || copying_initfork)
- smgrimmedsync(dst, forkNum);
+ /* drop old relation, and close new one */
+ RelationDropStorage(rel);
+ smgrclose(dstrel);
}
/*
* 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 and sequences). This should be a lower bound on
- * the XIDs that will be put into the new relation contents.
- *
- * The new filenode's persistence is set to the given value. This is useful
- * for the cases that are changing the relation's persistence; other callers
- * need to pass the original relpersistence value.
*/
void
-RelationSetNewRelfilenode(Relation relation, char persistence,
- TransactionId freezeXid, MultiXactId minmulti)
+RelationSetNewRelfilenode(Relation relation, char persistence)
{
Oid newrelfilenode;
- RelFileNodeBackend newrnode;
Relation pg_class;
HeapTuple tuple;
Form_pg_class classform;
-
- /* Indexes, sequences must have Invalid frozenxid; other rels must not */
- Assert((relation->rd_rel->relkind == RELKIND_INDEX ||
- relation->rd_rel->relkind == RELKIND_SEQUENCE) ?
- freezeXid == InvalidTransactionId :
- TransactionIdIsNormal(freezeXid));
- Assert(TransactionIdIsNormal(freezeXid) == MultiXactIdIsValid(minmulti));
+ MultiXactId minmulti = InvalidMultiXactId;
+ TransactionId freezeXid = InvalidTransactionId;
/* Allocate a new relfilenode */
newrelfilenode = GetNewRelFileNode(relation->rd_rel->reltablespace, NULL,
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.node = relation->rd_node;
- newrnode.node.relNode = newrelfilenode;
- newrnode.backend = relation->rd_backend;
- RelationCreateStorage(newrnode.node, persistence);
- smgrclosenode(newrnode);
-
/*
* Schedule unlinking of the old storage at transaction commit.
*/
RelationMapUpdateMap(RelationGetRelid(relation),
newrelfilenode,
relation->rd_rel->relisshared,
- false);
+ true);
else
+ {
+ relation->rd_rel->relfilenode = newrelfilenode;
classform->relfilenode = newrelfilenode;
+ }
+
+ RelationInitPhysicalAddr(relation);
+
+ /*
+ * Create storage for the main fork of the new relfilenode. If it's
+ * table-like object, call into table AM to do so, which'll also create
+ * the table's init fork.
+ *
+ * NOTE: any conflict in relfilenode value will be caught here, if
+ * GetNewRelFileNode messes up for any reason.
+ */
+
+ /*
+ * Create storage for relation.
+ */
+ switch (relation->rd_rel->relkind)
+ {
+ /* shouldn't be called for these */
+ case RELKIND_VIEW:
+ case RELKIND_COMPOSITE_TYPE:
+ case RELKIND_FOREIGN_TABLE:
+ case RELKIND_PARTITIONED_TABLE:
+ case RELKIND_PARTITIONED_INDEX:
+ elog(ERROR, "should not have storage");
+ break;
+
+ case RELKIND_INDEX:
+ case RELKIND_SEQUENCE:
+ RelationCreateStorage(relation->rd_node, persistence);
+ RelationOpenSmgr(relation);
+ break;
+
+ case RELKIND_RELATION:
+ case RELKIND_TOASTVALUE:
+ case RELKIND_MATVIEW:
+ table_relation_set_new_filenode(relation, persistence,
+ &freezeXid, &minmulti);
+ break;
+ }
/* These changes are safe even for a mapped relation */
if (relation->rd_rel->relkind != RELKIND_SEQUENCE)
* ------------------------------------------------------------------------
*/
+ /*
+ * This callback needs to create a new relation filenode for `rel`, with
+ * appropriate durability behaviour for `persistence`.
+ *
+ * On output *freezeXid, *minmulti should be set to the values appropriate
+ * for pg_class.{relfrozenxid, relminmxid} have to be set to. For AMs that
+ * don't need those fields to be filled they can be set to
+ * InvalidTransactionId, InvalidMultiXactId respectively.
+ *
+ * See also table_relation_set_new_filenode().
+ */
+ void (*relation_set_new_filenode) (Relation rel,
+ char persistence,
+ TransactionId *freezeXid,
+ MultiXactId *minmulti);
+
+ /*
+ * This callback needs to remove all contents from `rel`'s current
+ * relfilenode. No provisions for transactional behaviour need to be
+ * made. Often this can be implemented by truncating the underlying
+ * storage to its minimal size.
+ *
+ * See also table_relation_nontransactional_truncate().
+ */
+ void (*relation_nontransactional_truncate) (Relation rel);
+
+ /*
+ * See table_relation_copy_data().
+ *
+ * This can typically be implemented by directly copying the underlying
+ * storage, unless it contains references to the tablespace internally.
+ */
+ void (*relation_copy_data) (Relation rel, RelFileNode newrnode);
+
+ /* See table_relation_copy_for_cluster() */
+ void (*relation_copy_for_cluster) (Relation NewHeap, Relation OldHeap, Relation OldIndex,
+ bool use_sort,
+ TransactionId OldestXmin, TransactionId FreezeXid, MultiXactId MultiXactCutoff,
+ double *num_tuples, double *tups_vacuumed, double *tups_recently_dead);
+
/* see table_index_build_range_scan for reference about parameters */
double (*index_build_range_scan) (Relation heap_rel,
Relation index_rel,
* ------------------------------------------------------------------------
*/
+/*
+ * Create a new relation filenode for `rel`, with persistence set to
+ * `persistence`.
+ *
+ * This is used both during relation creation and various DDL operations to
+ * create a new relfilenode that can be filled from scratch.
+ *
+ * *freezeXid, *minmulti are set to the xid / multixact horizon for the table
+ * that pg_class.{relfrozenxid, relminmxid} have to be set to.
+ */
+static inline void
+table_relation_set_new_filenode(Relation rel, char persistence,
+ TransactionId *freezeXid,
+ MultiXactId *minmulti)
+{
+ rel->rd_tableam->relation_set_new_filenode(rel, persistence,
+ freezeXid, minmulti);
+}
+
+/*
+ * Remove all table contents from `rel`, in a non-transactional manner.
+ * Non-transactional meaning that there's no need to support rollbacks. This
+ * commonly only is used to perform truncations for relfilenodes created in the
+ * current transaction.
+ */
+static inline void
+table_relation_nontransactional_truncate(Relation rel)
+{
+ rel->rd_tableam->relation_nontransactional_truncate(rel);
+}
+
+/*
+ * Copy data from `rel` into the new relfilenode `newrnode`. The new
+ * relfilenode may not have storage associated before this function is
+ * called. This is only supposed to be used for low level operations like
+ * changing a relation's tablespace.
+ */
+static inline void
+table_relation_copy_data(Relation rel, RelFileNode newrnode)
+{
+ rel->rd_tableam->relation_copy_data(rel, newrnode);
+}
+
+/*
+ * Copy data from `OldHeap` into `NewHeap`, as part of a CLUSTER or VACUUM
+ * FULL.
+ *
+ * If `use_sort` is true, the table contents are sorted appropriate for
+ * `OldIndex`; if use_sort is false and OldIndex is not InvalidOid, the data
+ * is copied in that index's order; if use_sort is false and OidIndex is
+ * InvalidOid, no sorting is performed.
+ *
+ * OldestXmin, FreezeXid, MultiXactCutoff need to currently valid values for
+ * the table.
+ *
+ * *num_tuples, *tups_vacuumed, *tups_recently_dead will contain statistics
+ * computed while copying for the relation. Not all might make sense for every
+ * AM.
+ */
+static inline void
+table_relation_copy_for_cluster(Relation OldHeap, Relation NewHeap,
+ Relation OldIndex,
+ bool use_sort,
+ TransactionId OldestXmin,
+ TransactionId FreezeXid,
+ MultiXactId MultiXactCutoff,
+ double *num_tuples,
+ double *tups_vacuumed,
+ double *tups_recently_dead)
+{
+ OldHeap->rd_tableam->relation_copy_for_cluster(OldHeap, NewHeap, OldIndex,
+ use_sort, OldestXmin,
+ FreezeXid, MultiXactCutoff,
+ num_tuples, tups_vacuumed,
+ tups_recently_dead);
+}
+
/*
* table_index_build_range_scan - scan the table to find tuples to be indexed
*
char relpersistence,
bool shared_relation,
bool mapped_relation,
- bool allow_system_table_mods);
+ bool allow_system_table_mods,
+ TransactionId *relfrozenxid,
+ MultiXactId *relminmxid);
extern Oid heap_create_with_catalog(const char *relname,
Oid relnamespace,
Oid relrewrite,
ObjectAddress *typaddress);
-extern void heap_create_init_fork(Relation rel);
-
extern void heap_drop_with_catalog(Oid relid);
extern void heap_truncate(List *relids);
#include "storage/block.h"
#include "storage/relfilenode.h"
+#include "storage/smgr.h"
#include "utils/relcache.h"
extern void RelationCreateStorage(RelFileNode rnode, char relpersistence);
extern void RelationDropStorage(Relation rel);
extern void RelationPreserveStorage(RelFileNode rnode, bool atCommit);
extern void RelationTruncate(Relation rel, BlockNumber nblocks);
+extern void RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
+ ForkNumber forkNum, char relpersistence);
/*
* These functions used to be in storage/smgr/smgr.c, which explains the
/*
* Routine to manage assignment of new relfilenode to a relation
*/
-extern void RelationSetNewRelfilenode(Relation relation, char persistence,
- TransactionId freezeXid, MultiXactId minmulti);
+extern void RelationSetNewRelfilenode(Relation relation, char persistence);
/*
* Routines for flushing/rebuilding relcache entries in various scenarios