diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 5d1c902..12d07b1 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -2481,21 +2481,14 @@ include_dir 'conf.d'
levels. This parameter can only be set at server start.
- In minimal level, WAL-logging of some bulk
- operations can be safely skipped, which can make those
- operations much faster (see ).
- Operations in which this optimization can be applied include:
-
- CREATE TABLE AS
- CREATE INDEX
- CLUSTER
- COPY into tables that were created or truncated in the same
- transaction
-
- But minimal WAL does not contain enough information to reconstruct the
- data from a base backup and the WAL logs, so replica or
- higher must be used to enable WAL archiving
- () and streaming replication.
+ In minimal level, no information is logged for
+ tables or indexes for the remainder of a transaction that creates or
+ truncates them. This can make bulk operations much faster (see
+ ). But minimal WAL does not contain
+ enough information to reconstruct the data from a base backup and the
+ WAL logs, so replica or higher must be used to
+ enable WAL archiving () and
+ streaming replication.
In logical level, the same information is logged as
@@ -2887,6 +2880,26 @@ include_dir 'conf.d'
+
+ wal_skip_threshold (integer)
+
+ wal_skip_threshold configuration parameter
+
+
+
+
+ When wal_level is minimal and a
+ transaction commits after creating or rewriting a permanent table,
+ materialized view, or index, this setting determines how to persist
+ the new data. If the data is smaller than this setting, write it to
+ the WAL log; otherwise, use an fsync of the data file. Depending on
+ the properties of your storage, raising or lowering this value might
+ help if such commits are slowing concurrent transactions. The default
+ is two megabytes (2MB).
+
+
+
+
commit_delay (integer)
diff --git a/doc/src/sgml/perform.sgml b/doc/src/sgml/perform.sgml
index 0f61b09..12fda69 100644
--- a/doc/src/sgml/perform.sgml
+++ b/doc/src/sgml/perform.sgml
@@ -1606,8 +1606,8 @@ SELECT * FROM x, y, a, b, c WHERE something AND somethingelse;
needs to be written, because in case of an error, the files
containing the newly loaded data will be removed anyway.
However, this consideration only applies when
- is minimal for
- non-partitioned tables as all commands must write WAL otherwise.
+ is minimal
+ as all commands must write WAL otherwise.
@@ -1707,42 +1707,13 @@ SELECT * FROM x, y, a, b, c WHERE something AND somethingelse;
- Aside from avoiding the time for the archiver or WAL sender to
- process the WAL data,
- doing this will actually make certain commands faster, because they
- are designed not to write WAL at all if wal_level
- is minimal. (They can guarantee crash safety more cheaply
- by doing an fsync at the end than by writing WAL.)
- This applies to the following commands:
-
-
-
- CREATE TABLE AS SELECT
-
-
-
-
- CREATE INDEX (and variants such as
- ALTER TABLE ADD PRIMARY KEY)
-
-
-
-
- ALTER TABLE SET TABLESPACE
-
-
-
-
- CLUSTER
-
-
-
-
- COPY FROM, when the target table has been
- created or truncated earlier in the same transaction
-
-
-
+ Aside from avoiding the time for the archiver or WAL sender to process the
+ WAL data, doing this will actually make certain commands faster, because
+ they do not to write WAL at all if wal_level
+ is minimal and the current subtransaction (or top-level
+ transaction) created or truncated the table or index they change. (They
+ can guarantee crash safety more cheaply by doing
+ an fsync at the end than by writing WAL.)
diff --git a/src/backend/access/common/toast_internals.c b/src/backend/access/common/toast_internals.c
index a971242..784028a 100644
--- a/src/backend/access/common/toast_internals.c
+++ b/src/backend/access/common/toast_internals.c
@@ -528,8 +528,8 @@ toast_get_valid_index(Oid toastoid, LOCKMODE lock)
validIndexOid = RelationGetRelid(toastidxs[validIndex]);
/* Close the toast relation and all its indexes */
- toast_close_indexes(toastidxs, num_indexes, lock);
- table_close(toastrel, lock);
+ toast_close_indexes(toastidxs, num_indexes, NoLock);
+ table_close(toastrel, NoLock);
return validIndexOid;
}
diff --git a/src/backend/access/gist/gistutil.c b/src/backend/access/gist/gistutil.c
index 553a6d6..e7a1e21 100644
--- a/src/backend/access/gist/gistutil.c
+++ b/src/backend/access/gist/gistutil.c
@@ -1004,23 +1004,44 @@ gistproperty(Oid index_oid, int attno,
}
/*
- * Temporary and unlogged GiST indexes are not WAL-logged, but we need LSNs
- * to detect concurrent page splits anyway. This function provides a fake
- * sequence of LSNs for that purpose.
+ * Some indexes are not WAL-logged, but we need LSNs to detect concurrent page
+ * splits anyway. This function provides a fake sequence of LSNs for that
+ * purpose.
*/
XLogRecPtr
gistGetFakeLSN(Relation rel)
{
- static XLogRecPtr counter = FirstNormalUnloggedLSN;
-
if (rel->rd_rel->relpersistence == RELPERSISTENCE_TEMP)
{
/*
* Temporary relations are only accessible in our session, so a simple
* backend-local counter will do.
*/
+ static XLogRecPtr counter = FirstNormalUnloggedLSN;
+
return counter++;
}
+ else if (rel->rd_rel->relpersistence == RELPERSISTENCE_PERMANENT)
+ {
+ /*
+ * WAL-logging on this relation will start after commit, so its LSNs
+ * must be distinct numbers smaller than the LSN at the next commit.
+ * Emit a dummy WAL record if insert-LSN hasn't advanced after the
+ * last call.
+ */
+ static XLogRecPtr lastlsn = InvalidXLogRecPtr;
+ XLogRecPtr currlsn = GetXLogInsertRecPtr();
+
+ /* Shouldn't be called for WAL-logging relations */
+ Assert(!RelationNeedsWAL(rel));
+
+ /* No need for an actual record if we already have a distinct LSN */
+ if (!XLogRecPtrIsInvalid(lastlsn) && lastlsn == currlsn)
+ currlsn = gistXLogAssignLSN();
+
+ lastlsn = currlsn;
+ return currlsn;
+ }
else
{
/*
diff --git a/src/backend/access/gist/gistxlog.c b/src/backend/access/gist/gistxlog.c
index 3b28f54..53db3a8 100644
--- a/src/backend/access/gist/gistxlog.c
+++ b/src/backend/access/gist/gistxlog.c
@@ -449,6 +449,9 @@ gist_redo(XLogReaderState *record)
case XLOG_GIST_PAGE_DELETE:
gistRedoPageDelete(record);
break;
+ case XLOG_GIST_ASSIGN_LSN:
+ /* nop. See gistGetFakeLSN(). */
+ break;
default:
elog(PANIC, "gist_redo: unknown op code %u", info);
}
@@ -593,6 +596,24 @@ gistXLogPageDelete(Buffer buffer, FullTransactionId xid,
}
/*
+ * Write an empty XLOG record to assign a distinct LSN.
+ */
+XLogRecPtr
+gistXLogAssignLSN(void)
+{
+ int dummy = 0;
+
+ /*
+ * Records other than SWITCH_WAL must have content. We use an integer 0 to
+ * follow the restriction.
+ */
+ XLogBeginInsert();
+ XLogSetRecordFlags(XLOG_MARK_UNIMPORTANT);
+ XLogRegisterData((char *) &dummy, sizeof(dummy));
+ return XLogInsert(RM_GIST_ID, XLOG_GIST_ASSIGN_LSN);
+}
+
+/*
* Write XLOG record about reuse of a deleted page.
*/
void
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 0128bb3..be19c34 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -21,7 +21,6 @@
* heap_multi_insert - insert multiple tuples into a relation
* heap_delete - delete a tuple from a relation
* heap_update - replace a tuple in a relation with another tuple
- * heap_sync - sync heap, for when no WAL has been written
*
* NOTES
* This file contains the heap_ routines which implement
@@ -1936,7 +1935,7 @@ heap_insert(Relation relation, HeapTuple tup, CommandId cid,
MarkBufferDirty(buffer);
/* XLOG stuff */
- if (!(options & HEAP_INSERT_SKIP_WAL) && RelationNeedsWAL(relation))
+ if (RelationNeedsWAL(relation))
{
xl_heap_insert xlrec;
xl_heap_header xlhdr;
@@ -2119,7 +2118,7 @@ heap_multi_insert(Relation relation, TupleTableSlot **slots, int ntuples,
/* currently not needed (thus unsupported) for heap_multi_insert() */
AssertArg(!(options & HEAP_INSERT_NO_LOGICAL));
- needwal = !(options & HEAP_INSERT_SKIP_WAL) && RelationNeedsWAL(relation);
+ needwal = RelationNeedsWAL(relation);
saveFreeSpace = RelationGetTargetPageFreeSpace(relation,
HEAP_DEFAULT_FILLFACTOR);
@@ -8921,46 +8920,6 @@ heap2_redo(XLogReaderState *record)
}
/*
- * heap_sync - sync a heap, for use when no WAL has been written
- *
- * This forces the heap contents (including TOAST heap if any) down to disk.
- * If we skipped using WAL, and WAL is otherwise needed, we must force the
- * relation down to disk before it's safe to commit the transaction. This
- * requires writing out any dirty buffers and then doing a forced fsync.
- *
- * Indexes are not touched. (Currently, index operations associated with
- * the commands that use this are WAL-logged and so do not need fsync.
- * That behavior might change someday, but in any case it's likely that
- * any fsync decisions required would be per-index and hence not appropriate
- * to be done here.)
- */
-void
-heap_sync(Relation rel)
-{
- /* non-WAL-logged tables never need fsync */
- if (!RelationNeedsWAL(rel))
- return;
-
- /* main heap */
- FlushRelationBuffers(rel);
- /* FlushRelationBuffers will have opened rd_smgr */
- smgrimmedsync(rel->rd_smgr, MAIN_FORKNUM);
-
- /* FSM is not critical, don't bother syncing it */
-
- /* toast heap, if any */
- if (OidIsValid(rel->rd_rel->reltoastrelid))
- {
- Relation toastrel;
-
- toastrel = table_open(rel->rd_rel->reltoastrelid, AccessShareLock);
- FlushRelationBuffers(toastrel);
- smgrimmedsync(toastrel->rd_smgr, MAIN_FORKNUM);
- table_close(toastrel, AccessShareLock);
- }
-}
-
-/*
* Mask a heap page before performing consistency checks on it.
*/
void
diff --git a/src/backend/access/heap/heapam_handler.c b/src/backend/access/heap/heapam_handler.c
index 92073fe..07fe717 100644
--- a/src/backend/access/heap/heapam_handler.c
+++ b/src/backend/access/heap/heapam_handler.c
@@ -555,17 +555,6 @@ tuple_lock_retry:
return result;
}
-static void
-heapam_finish_bulk_insert(Relation relation, int options)
-{
- /*
- * If we skipped writing WAL, then we need to sync the heap (but not
- * indexes since those use WAL anyway / don't go through tableam)
- */
- if (options & HEAP_INSERT_SKIP_WAL)
- heap_sync(relation);
-}
-
/* ------------------------------------------------------------------------
* DDL related callbacks for heap AM.
@@ -698,7 +687,6 @@ heapam_relation_copy_for_cluster(Relation OldHeap, Relation NewHeap,
IndexScanDesc indexScan;
TableScanDesc tableScan;
HeapScanDesc heapScan;
- bool use_wal;
bool is_system_catalog;
Tuplesortstate *tuplesort;
TupleDesc oldTupDesc = RelationGetDescr(OldHeap);
@@ -713,12 +701,9 @@ heapam_relation_copy_for_cluster(Relation OldHeap, Relation NewHeap,
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.
+ * Valid smgr_targblock implies something already wrote to the relation.
+ * This may be harmless, but this function hasn't planned for it.
*/
- use_wal = XLogIsNeeded() && RelationNeedsWAL(NewHeap);
-
- /* use_wal off requires smgr_targblock be initially invalid */
Assert(RelationGetTargetBlock(NewHeap) == InvalidBlockNumber);
/* Preallocate values/isnull arrays */
@@ -728,7 +713,7 @@ heapam_relation_copy_for_cluster(Relation OldHeap, Relation NewHeap,
/* Initialize the rewrite operation */
rwstate = begin_heap_rewrite(OldHeap, NewHeap, OldestXmin, *xid_cutoff,
- *multi_cutoff, use_wal);
+ *multi_cutoff);
/* Set up sorting if wanted */
@@ -2515,7 +2500,6 @@ static const TableAmRoutine heapam_methods = {
.tuple_delete = heapam_tuple_delete,
.tuple_update = heapam_tuple_update,
.tuple_lock = heapam_tuple_lock,
- .finish_bulk_insert = heapam_finish_bulk_insert,
.tuple_fetch_row_version = heapam_fetch_row_version,
.tuple_get_latest_tid = heap_get_latest_tid,
diff --git a/src/backend/access/heap/rewriteheap.c b/src/backend/access/heap/rewriteheap.c
index d285b1f..3e56483 100644
--- a/src/backend/access/heap/rewriteheap.c
+++ b/src/backend/access/heap/rewriteheap.c
@@ -136,7 +136,6 @@ typedef struct RewriteStateData
Page rs_buffer; /* page currently being built */
BlockNumber rs_blockno; /* block where page will go */
bool rs_buffer_valid; /* T if any tuples in buffer */
- bool rs_use_wal; /* must we WAL-log inserts? */
bool rs_logical_rewrite; /* do we need to do logical rewriting */
TransactionId rs_oldest_xmin; /* oldest xmin used by caller to determine
* tuple visibility */
@@ -230,15 +229,13 @@ static void logical_end_heap_rewrite(RewriteState state);
* oldest_xmin xid used by the caller to determine which tuples are dead
* freeze_xid xid before which tuples will be frozen
* cutoff_multi multixact before which multis will be removed
- * use_wal should the inserts to the new heap be WAL-logged?
*
* Returns an opaque RewriteState, allocated in current memory context,
* to be used in subsequent calls to the other functions.
*/
RewriteState
begin_heap_rewrite(Relation old_heap, Relation new_heap, TransactionId oldest_xmin,
- TransactionId freeze_xid, MultiXactId cutoff_multi,
- bool use_wal)
+ TransactionId freeze_xid, MultiXactId cutoff_multi)
{
RewriteState state;
MemoryContext rw_cxt;
@@ -263,7 +260,6 @@ begin_heap_rewrite(Relation old_heap, Relation new_heap, TransactionId oldest_xm
/* new_heap needn't be empty, just locked */
state->rs_blockno = RelationGetNumberOfBlocks(new_heap);
state->rs_buffer_valid = false;
- state->rs_use_wal = use_wal;
state->rs_oldest_xmin = oldest_xmin;
state->rs_freeze_xid = freeze_xid;
state->rs_cutoff_multi = cutoff_multi;
@@ -322,7 +318,7 @@ end_heap_rewrite(RewriteState state)
/* Write the last page, if any */
if (state->rs_buffer_valid)
{
- if (state->rs_use_wal)
+ if (RelationNeedsWAL(state->rs_new_rel))
log_newpage(&state->rs_new_rel->rd_node,
MAIN_FORKNUM,
state->rs_blockno,
@@ -337,18 +333,14 @@ end_heap_rewrite(RewriteState state)
}
/*
- * If the rel is WAL-logged, must fsync before commit. We use heap_sync
- * to ensure that the toast table gets fsync'd too.
- *
- * It's obvious that we must do this when not WAL-logging. It's less
- * obvious that we have to do it even if we did WAL-log the pages. The
+ * When we WAL-logged rel pages, we must nonetheless fsync them. The
* reason is the same as in storage.c's RelationCopyStorage(): we're
* writing data that's not in shared buffers, and so a CHECKPOINT
* occurring during the rewriteheap operation won't have fsync'd data we
* wrote before the checkpoint.
*/
if (RelationNeedsWAL(state->rs_new_rel))
- heap_sync(state->rs_new_rel);
+ smgrimmedsync(state->rs_new_rel->rd_smgr, MAIN_FORKNUM);
logical_end_heap_rewrite(state);
@@ -646,9 +638,6 @@ raw_heap_insert(RewriteState state, HeapTuple tup)
{
int options = HEAP_INSERT_SKIP_FSM;
- if (!state->rs_use_wal)
- options |= HEAP_INSERT_SKIP_WAL;
-
/*
* While rewriting the heap for VACUUM FULL / CLUSTER, make sure data
* for the TOAST table are not logically decoded. The main heap is
@@ -687,7 +676,7 @@ raw_heap_insert(RewriteState state, HeapTuple tup)
/* Doesn't fit, so write out the existing page */
/* XLOG stuff */
- if (state->rs_use_wal)
+ if (RelationNeedsWAL(state->rs_new_rel))
log_newpage(&state->rs_new_rel->rd_node,
MAIN_FORKNUM,
state->rs_blockno,
diff --git a/src/backend/access/nbtree/nbtsort.c b/src/backend/access/nbtree/nbtsort.c
index c8110a1..f419e92 100644
--- a/src/backend/access/nbtree/nbtsort.c
+++ b/src/backend/access/nbtree/nbtsort.c
@@ -31,18 +31,6 @@
* them. They will need to be re-read into shared buffers on first use after
* the build finishes.
*
- * Since the index will never be used unless it is completely built,
- * from a crash-recovery point of view there is no need to WAL-log the
- * steps of the build. After completing the index build, we can just sync
- * the whole file to disk using smgrimmedsync() before exiting this module.
- * This can be seen to be sufficient for crash recovery by considering that
- * it's effectively equivalent to what would happen if a CHECKPOINT occurred
- * just after the index build. However, it is clearly not sufficient if the
- * DBA is using the WAL log for PITR or replication purposes, since another
- * machine would not be able to reconstruct the index from WAL. Therefore,
- * we log the completed index pages to WAL if and only if WAL archiving is
- * active.
- *
* This code isn't concerned about the FSM at all. The caller is responsible
* for initializing that.
*
@@ -563,12 +551,7 @@ _bt_leafbuild(BTSpool *btspool, BTSpool *btspool2)
wstate.heap = btspool->heap;
wstate.index = btspool->index;
wstate.inskey = _bt_mkscankey(wstate.index, NULL);
-
- /*
- * We need to log index creation in WAL iff WAL archiving/streaming is
- * enabled UNLESS the index isn't WAL-logged anyway.
- */
- wstate.btws_use_wal = XLogIsNeeded() && RelationNeedsWAL(wstate.index);
+ wstate.btws_use_wal = RelationNeedsWAL(wstate.index);
/* reserve the metapage */
wstate.btws_pages_alloced = BTREE_METAPAGE + 1;
@@ -1265,21 +1248,15 @@ _bt_load(BTWriteState *wstate, BTSpool *btspool, BTSpool *btspool2)
_bt_uppershutdown(wstate, state);
/*
- * If the index is WAL-logged, we must fsync it down to disk before it's
- * safe to commit the transaction. (For a non-WAL-logged index we don't
- * care since the index will be uninteresting after a crash anyway.)
- *
- * It's obvious that we must do this when not WAL-logging the build. It's
- * less obvious that we have to do it even if we did WAL-log the index
- * pages. The reason is that since we're building outside shared buffers,
- * a CHECKPOINT occurring during the build has no way to flush the
- * previously written data to disk (indeed it won't know the index 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.
+ * When we WAL-logged index pages, we must nonetheless fsync index files.
+ * Since we're building outside shared buffers, a CHECKPOINT occurring
+ * during the build has no way to flush the previously written data to
+ * disk (indeed it won't know the index 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 (RelationNeedsWAL(wstate->index))
+ if (wstate->btws_use_wal)
{
RelationOpenSmgr(wstate->index);
smgrimmedsync(wstate->index->rd_smgr, MAIN_FORKNUM);
diff --git a/src/backend/access/rmgrdesc/gistdesc.c b/src/backend/access/rmgrdesc/gistdesc.c
index eccb6fd..48cda40 100644
--- a/src/backend/access/rmgrdesc/gistdesc.c
+++ b/src/backend/access/rmgrdesc/gistdesc.c
@@ -80,6 +80,9 @@ gist_desc(StringInfo buf, XLogReaderState *record)
case XLOG_GIST_PAGE_DELETE:
out_gistxlogPageDelete(buf, (gistxlogPageDelete *) rec);
break;
+ case XLOG_GIST_ASSIGN_LSN:
+ /* No details to write out */
+ break;
}
}
@@ -104,6 +107,8 @@ gist_identify(uint8 info)
break;
case XLOG_GIST_PAGE_DELETE:
id = "PAGE_DELETE";
+ case XLOG_GIST_ASSIGN_LSN:
+ id = "ASSIGN_LSN";
break;
}
diff --git a/src/backend/access/transam/README b/src/backend/access/transam/README
index b5a2cb2..eb9aac5 100644
--- a/src/backend/access/transam/README
+++ b/src/backend/access/transam/README
@@ -717,6 +717,38 @@ then restart recovery. This is part of the reason for not writing a WAL
entry until we've successfully done the original action.
+Skipping WAL for New RelFileNode
+--------------------------------
+
+Under wal_level=minimal, if a change modifies a relfilenode that ROLLBACK
+would unlink, in-tree access methods write no WAL for that change. Code that
+writes WAL without calling RelationNeedsWAL() must check for this case. This
+skipping is mandatory. If a WAL-writing change preceded a WAL-skipping change
+for the same block, REDO could overwrite the WAL-skipping change. If a
+WAL-writing change followed a WAL-skipping change for the same block, a
+related problem would arise. When a WAL record contains no full-page image,
+REDO expects the page to match its contents from just before record insertion.
+A WAL-skipping change may not reach disk at all, violating REDO's expectation
+under full_page_writes=off. For any access method, CommitTransaction() writes
+and fsyncs affected blocks before recording the commit.
+
+Prefer to do the same in future access methods. However, two other approaches
+can work. First, an access method can irreversibly transition a given fork
+from WAL-skipping to WAL-writing by calling FlushRelationBuffers() and
+smgrimmedsync(). Second, an access method can opt to write WAL
+unconditionally for permanent relations. Under these approaches, the access
+method callbacks must not call functions that react to RelationNeedsWAL().
+
+This applies only to WAL records whose replay would modify bytes stored in the
+new relfilenode. It does not apply to other records about the relfilenode,
+such as XLOG_SMGR_CREATE. Because it operates at the level of individual
+relfilenodes, RelationNeedsWAL() can differ for tightly-coupled relations.
+Consider "CREATE TABLE t (); BEGIN; ALTER TABLE t ADD c text; ..." in which
+ALTER TABLE adds a TOAST relation. The TOAST relation will skip WAL, while
+the table owning it will not. ALTER TABLE SET TABLESPACE will cause a table
+to skip WAL, but that won't affect its indexes.
+
+
Asynchronous Commit
-------------------
@@ -820,13 +852,12 @@ Changes to a temp table are not WAL-logged, hence could reach disk in
advance of T1's commit, but we don't care since temp table contents don't
survive crashes anyway.
-Database writes made via any of the paths we have introduced to avoid WAL
-overhead for bulk updates are also safe. In these cases it's entirely
-possible for the data to reach disk before T1's commit, because T1 will
-fsync it down to disk without any sort of interlock, as soon as it finishes
-the bulk update. However, all these paths are designed to write data that
-no other transaction can see until after T1 commits. The situation is thus
-not different from ordinary WAL-logged updates.
+Database writes that skip WAL for new relfilenodes are also safe. In these
+cases it's entirely possible for the data to reach disk before T1's commit,
+because T1 will fsync it down to disk without any sort of interlock. However,
+all these paths are designed to write data that no other transaction can see
+until after T1 commits. The situation is thus not different from ordinary
+WAL-logged updates.
Transaction Emulation during Recovery
-------------------------------------
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 5353b6a..5268253 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -2109,6 +2109,13 @@ CommitTransaction(void)
*/
PreCommit_on_commit_actions();
+ /*
+ * Synchronize files that are created and not WAL-logged during this
+ * transaction. This must happen before AtEOXact_RelationMap(), so that we
+ * don't see committed-but-broken files after a crash.
+ */
+ smgrDoPendingSyncs(true);
+
/* close large objects before lower-level cleanup */
AtEOXact_LargeObject(true);
@@ -2342,6 +2349,13 @@ PrepareTransaction(void)
*/
PreCommit_on_commit_actions();
+ /*
+ * Synchronize files that are created and not WAL-logged during this
+ * transaction. This must happen before EndPrepare(), so that we don't see
+ * committed-but-broken files after a crash and COMMIT PREPARED.
+ */
+ smgrDoPendingSyncs(true);
+
/* close large objects before lower-level cleanup */
AtEOXact_LargeObject(true);
@@ -2660,6 +2674,7 @@ AbortTransaction(void)
*/
AfterTriggerEndXact(false); /* 'false' means it's abort */
AtAbort_Portals();
+ smgrDoPendingSyncs(false);
AtEOXact_LargeObject(false);
AtAbort_Notify();
AtEOXact_RelationMap(false, is_parallel_worker);
diff --git a/src/backend/access/transam/xloginsert.c b/src/backend/access/transam/xloginsert.c
index aa9dca0..dda1dea 100644
--- a/src/backend/access/transam/xloginsert.c
+++ b/src/backend/access/transam/xloginsert.c
@@ -1043,8 +1043,13 @@ log_newpage_range(Relation rel, ForkNumber forkNum,
BlockNumber startblk, BlockNumber endblk,
bool page_std)
{
+ int flags;
BlockNumber blkno;
+ flags = REGBUF_FORCE_IMAGE;
+ if (page_std)
+ flags |= REGBUF_STANDARD;
+
/*
* Iterate over all the pages in the range. They are collected into
* batches of XLR_MAX_BLOCK_ID pages, and a single WAL-record is written
@@ -1066,7 +1071,8 @@ log_newpage_range(Relation rel, ForkNumber forkNum,
nbufs = 0;
while (nbufs < XLR_MAX_BLOCK_ID && blkno < endblk)
{
- Buffer buf = ReadBuffer(rel, blkno);
+ Buffer buf = ReadBufferExtended(rel, forkNum, blkno,
+ RBM_NORMAL, NULL);
LockBuffer(buf, BUFFER_LOCK_EXCLUSIVE);
@@ -1088,7 +1094,7 @@ log_newpage_range(Relation rel, ForkNumber forkNum,
START_CRIT_SECTION();
for (i = 0; i < nbufs; i++)
{
- XLogRegisterBuffer(i, bufpack[i], REGBUF_FORCE_IMAGE | REGBUF_STANDARD);
+ XLogRegisterBuffer(i, bufpack[i], flags);
MarkBufferDirty(bufpack[i]);
}
diff --git a/src/backend/access/transam/xlogutils.c b/src/backend/access/transam/xlogutils.c
index 14efbf3..a490ce9 100644
--- a/src/backend/access/transam/xlogutils.c
+++ b/src/backend/access/transam/xlogutils.c
@@ -544,6 +544,8 @@ typedef FakeRelCacheEntryData *FakeRelCacheEntry;
* fields related to physical storage, like rd_rel, are initialized, so the
* fake entry is only usable in low-level operations like ReadBuffer().
*
+ * This is also used for syncing WAL-skipped files.
+ *
* Caller must free the returned entry with FreeFakeRelcacheEntry().
*/
Relation
@@ -552,18 +554,20 @@ CreateFakeRelcacheEntry(RelFileNode rnode)
FakeRelCacheEntry fakeentry;
Relation rel;
- Assert(InRecovery);
-
/* Allocate the Relation struct and all related space in one block. */
fakeentry = palloc0(sizeof(FakeRelCacheEntryData));
rel = (Relation) fakeentry;
rel->rd_rel = &fakeentry->pgc;
rel->rd_node = rnode;
- /* We will never be working with temp rels during recovery */
+
+ /*
+ * We will never be working with temp rels during recovery or while
+ * syncing WAL-skipped files.
+ */
rel->rd_backend = InvalidBackendId;
- /* It must be a permanent table if we're in recovery. */
+ /* It must be a permanent table here */
rel->rd_rel->relpersistence = RELPERSISTENCE_PERMANENT;
/* We don't know the name of the relation; use relfilenode instead */
@@ -572,9 +576,9 @@ CreateFakeRelcacheEntry(RelFileNode rnode)
/*
* We set up the lockRelId in case anything tries to lock the dummy
* relation. Note that this is fairly bogus since relNode may be
- * different from the relation's OID. It shouldn't really matter though,
- * since we are presumably running by ourselves and can't have any lock
- * conflicts ...
+ * different from the relation's OID. It shouldn't really matter though.
+ * In recovery, we are running by ourselves and can't have any lock
+ * conflicts. While syncing, we already hold AccessExclusiveLock.
*/
rel->rd_lockInfo.lockRelId.dbId = rnode.dbNode;
rel->rd_lockInfo.lockRelId.relId = rnode.relNode;
diff --git a/src/backend/catalog/heap.c b/src/backend/catalog/heap.c
index 8404904..3e13457 100644
--- a/src/backend/catalog/heap.c
+++ b/src/backend/catalog/heap.c
@@ -440,6 +440,10 @@ heap_create(const char *relname,
break;
}
}
+ else
+ {
+ rel->rd_createSubid = InvalidSubTransactionId;
+ }
return rel;
}
diff --git a/src/backend/catalog/storage.c b/src/backend/catalog/storage.c
index 056ea3d..3147ad1 100644
--- a/src/backend/catalog/storage.c
+++ b/src/backend/catalog/storage.c
@@ -29,9 +29,13 @@
#include "miscadmin.h"
#include "storage/freespace.h"
#include "storage/smgr.h"
+#include "utils/hsearch.h"
#include "utils/memutils.h"
#include "utils/rel.h"
+/* GUC variables */
+int wal_skip_threshold = 2048; /* in kilobytes */
+
/*
* We keep a list of all relations (represented as RelFileNode values)
* that have been created or deleted in the current transaction. When
@@ -61,7 +65,14 @@ typedef struct PendingRelDelete
struct PendingRelDelete *next; /* linked-list link */
} PendingRelDelete;
+typedef struct pendingSync
+{
+ RelFileNode rnode;
+ BlockNumber max_truncated;
+} pendingSync;
+
static PendingRelDelete *pendingDeletes = NULL; /* head of linked list */
+HTAB *pendingSyncHash = NULL;
/*
* RelationCreateStorage
@@ -117,6 +128,35 @@ RelationCreateStorage(RelFileNode rnode, char relpersistence)
pending->next = pendingDeletes;
pendingDeletes = pending;
+ /*
+ * If the relation needs at-commit sync, we also need to track the maximum
+ * unsynced truncated block; see smgrDoPendingSyncs().
+ */
+ if (relpersistence == RELPERSISTENCE_PERMANENT && !XLogIsNeeded())
+ {
+ pendingSync *pending;
+ bool found;
+
+ /* we sync only permanent relations */
+ Assert(backend == InvalidBackendId);
+
+ if (!pendingSyncHash)
+ {
+ HASHCTL ctl;
+
+ ctl.keysize = sizeof(RelFileNode);
+ ctl.entrysize = sizeof(pendingSync);
+ ctl.hcxt = TopTransactionContext;
+ pendingSyncHash =
+ hash_create("max truncated block hash",
+ 16, &ctl, HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
+ }
+
+ pending = hash_search(pendingSyncHash, &rnode, HASH_ENTER, &found);
+ Assert(!found);
+ pending->max_truncated = 0;
+ }
+
return srel;
}
@@ -216,6 +256,8 @@ RelationPreserveStorage(RelFileNode rnode, bool atCommit)
prev = pending;
}
}
+
+ /* FIXME what to do about pending syncs? */
}
/*
@@ -275,6 +317,8 @@ RelationTruncate(Relation rel, BlockNumber nblocks)
}
}
+ RelationPreTruncate(rel);
+
/*
* We WAL-log the truncation before actually truncating, which means
* trouble if the truncation fails. If we then crash, the WAL replay
@@ -326,6 +370,34 @@ RelationTruncate(Relation rel, BlockNumber nblocks)
}
/*
+ * RelationPreTruncate
+ * Perform AM-independent work before a physical truncation.
+ *
+ * If an access method's relation_nontransactional_truncate does not call
+ * RelationTruncate(), it must call this before decreasing the table size.
+ */
+void
+RelationPreTruncate(Relation rel)
+{
+ pendingSync *pending;
+
+ if (!pendingSyncHash)
+ return;
+ RelationOpenSmgr(rel);
+
+ /* Record largest maybe-unsynced block of files under tracking */
+ pending = hash_search(pendingSyncHash, &(rel->rd_smgr->smgr_rnode.node),
+ HASH_FIND, NULL);
+ if (pending)
+ {
+ BlockNumber nblocks = smgrnblocks(rel->rd_smgr, MAIN_FORKNUM);
+
+ if (pending->max_truncated < nblocks)
+ pending->max_truncated = nblocks;
+ }
+}
+
+/*
* Copy a fork's data, block by block.
*
* Note that this requires that there is no dirty data in shared buffers. If
@@ -355,7 +427,9 @@ RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
/*
* We need to log the copied data in WAL iff WAL archiving/streaming is
- * enabled AND it's a permanent relation.
+ * enabled AND it's a permanent relation. This gives the same answer as
+ * "RelationNeedsWAL(rel) || copying_initfork", because we know the
+ * current operation created a new relfilenode.
*/
use_wal = XLogIsNeeded() &&
(relpersistence == RELPERSISTENCE_PERMANENT || copying_initfork);
@@ -397,25 +471,40 @@ RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
}
/*
- * 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.
+ * When we WAL-logged rel pages, we must nonetheless fsync them. 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)
+ if (use_wal || copying_initfork)
smgrimmedsync(dst, forkNum);
}
/*
+ * RelFileNodeSkippingWAL - check if a BM_PERMANENT relfilenode is using WAL
+ *
+ * Changes of certain relfilenodes must not write WAL; see "Skipping WAL for
+ * New RelFileNode" in src/backend/access/transam/README. Though it is
+ * known from Relation efficiently, this function is intended for the code
+ * paths not having access to Relation.
+ */
+bool
+RelFileNodeSkippingWAL(RelFileNode rnode)
+{
+ if (XLogIsNeeded())
+ return false; /* no permanent relfilenode skips WAL */
+
+ if (!pendingSyncHash ||
+ hash_search(pendingSyncHash, &rnode, HASH_FIND, NULL) == NULL)
+ return false;
+
+ return true;
+}
+
+/*
* smgrDoPendingDeletes() -- Take care of relation deletes at end of xact.
*
* This also runs when aborting a subxact; we want to clean up a failed
@@ -493,6 +582,135 @@ smgrDoPendingDeletes(bool isCommit)
}
/*
+ * smgrDoPendingSyncs() -- Take care of relation syncs at end of xact.
+ */
+void
+smgrDoPendingSyncs(bool isCommit)
+{
+ PendingRelDelete *pending;
+ int nrels = 0,
+ maxrels = 0;
+ SMgrRelation *srels = NULL;
+ HASH_SEQ_STATUS scan;
+ pendingSync *pendingsync;
+
+ if (XLogIsNeeded())
+ return; /* no relation can use this */
+
+ Assert(GetCurrentTransactionNestLevel() == 1);
+
+ if (!pendingSyncHash)
+ return; /* no relation needs sync */
+
+ /* Just throw away all pending syncs if any at rollback */
+ if (!isCommit)
+ {
+ pendingSyncHash = NULL;
+ return;
+ }
+
+ AssertPendingSyncs_RelationCache();
+
+ /* Skip syncing nodes that smgrDoPendingDeletes() will delete. */
+ for (pending = pendingDeletes; pending != NULL; pending = pending->next)
+ {
+ if (!pending->atCommit)
+ continue;
+
+ (void) hash_search(pendingSyncHash, (void *) &pending->relnode,
+ HASH_REMOVE, NULL);
+ }
+
+ hash_seq_init(&scan, pendingSyncHash);
+ while ((pendingsync = (pendingSync *) hash_seq_search(&scan)))
+ {
+ ForkNumber fork;
+ BlockNumber nblocks[MAX_FORKNUM + 1];
+ BlockNumber total_blocks = 0;
+ SMgrRelation srel;
+
+ srel = smgropen(pendingsync->rnode, InvalidBackendId);
+
+ /*
+ * We emit newpage WAL records for smaller relations.
+ *
+ * Small WAL records have a chance to be emitted along with other
+ * backends' WAL records. We emit WAL records instead of syncing for
+ * files that are smaller than a certain threshold, expecting faster
+ * commit. The threshold is defined by the GUC wal_skip_threshold.
+ */
+ for (fork = 0; fork <= MAX_FORKNUM; fork++)
+ {
+ if (smgrexists(srel, fork))
+ {
+ BlockNumber n = smgrnblocks(srel, fork);
+
+ /* we shouldn't come here for unlogged relations */
+ Assert(fork != INIT_FORKNUM);
+
+ nblocks[fork] = n;
+ total_blocks += n;
+ }
+ else
+ nblocks[fork] = InvalidBlockNumber;
+ }
+
+ /*
+ * Sync file or emit WAL records for its contents. Do file sync if
+ * the size is larger than the threshold or truncates may have removed
+ * blocks beyond the current size.
+ */
+ if (total_blocks * BLCKSZ / 1024 >= wal_skip_threshold ||
+ nblocks[MAIN_FORKNUM] < pendingsync->max_truncated)
+ {
+ /* allocate the initial array, or extend it, if needed */
+ if (maxrels == 0)
+ {
+ maxrels = 8;
+ srels = palloc(sizeof(SMgrRelation) * maxrels);
+ }
+ else if (maxrels <= nrels)
+ {
+ maxrels *= 2;
+ srels = repalloc(srels, sizeof(SMgrRelation) * maxrels);
+ }
+
+ srels[nrels++] = srel;
+ }
+ else
+ {
+ /* Emit WAL records for all blocks. The file is small enough. */
+ for (fork = 0; fork <= MAX_FORKNUM; fork++)
+ {
+ int n = nblocks[fork];
+ Relation rel;
+
+ if (!BlockNumberIsValid(n))
+ continue;
+
+ /*
+ * Emit WAL for the whole file. Unfortunately we don't know
+ * what kind of a page this is, so we have to log the full
+ * page including any unused space. ReadBufferExtended()
+ * counts some pgstat events; unfortunately, we discard them.
+ */
+ rel = CreateFakeRelcacheEntry(srel->smgr_rnode.node);
+ log_newpage_range(rel, fork, 0, n, false);
+ FreeFakeRelcacheEntry(rel);
+ }
+ }
+ }
+
+ pendingSyncHash = NULL;
+
+ if (nrels > 0)
+ {
+ smgrdosyncall(srels, nrels);
+ pfree(srels);
+ }
+}
+
+/*
* smgrGetPendingDeletes() -- Get a list of non-temp relations to be deleted.
*
* The return value is the number of relations scheduled for termination.
diff --git a/src/backend/commands/cluster.c b/src/backend/commands/cluster.c
index b8c349f..4b79068 100644
--- a/src/backend/commands/cluster.c
+++ b/src/backend/commands/cluster.c
@@ -1014,6 +1014,7 @@ swap_relation_files(Oid r1, Oid r2, bool target_is_pg_class,
relfilenode2;
Oid swaptemp;
char swptmpchr;
+ Relation rel1;
/* We need writable copies of both pg_class tuples. */
relRelation = table_open(RelationRelationId, RowExclusiveLock);
@@ -1174,6 +1175,15 @@ swap_relation_files(Oid r1, Oid r2, bool target_is_pg_class,
}
/*
+ * Recognize that rel1's relfilenode (swapped from rel2) is new in this
+ * subtransaction. Since the next step for rel2 is deletion, don't bother
+ * recording the newness of its relfilenode.
+ */
+ rel1 = relation_open(r1, NoLock);
+ RelationAssumeNewRelfilenode(rel1);
+ relation_close(rel1, NoLock);
+
+ /*
* Post alter hook for modified relations. The change to r2 is always
* internal, but r1 depends on the invocation context.
*/
@@ -1489,7 +1499,7 @@ finish_heap_swap(Oid OIDOldHeap, Oid OIDNewHeap,
/* Get the associated valid index to be renamed */
toastidx = toast_get_valid_index(newrel->rd_rel->reltoastrelid,
- AccessShareLock);
+ NoLock);
/* rename the toast table ... */
snprintf(NewToastName, NAMEDATALEN, "pg_toast_%u",
diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index 42a147b..607e255 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -2711,63 +2711,15 @@ CopyFrom(CopyState cstate)
RelationGetRelationName(cstate->rel))));
}
- /*----------
- * Check to see if we can avoid writing WAL
- *
- * If archive logging/streaming is not enabled *and* either
- * - table was created in same transaction as this COPY
- * - data is being written to relfilenode created in this transaction
- * then we can skip writing WAL. It's safe because if the transaction
- * doesn't commit, we'll discard the table (or the new relfilenode file).
- * If it does commit, we'll have done the table_finish_bulk_insert() at
- * the bottom of this routine first.
- *
- * As mentioned in comments in utils/rel.h, the in-same-transaction test
- * is not always set correctly, since in rare cases rd_newRelfilenodeSubid
- * can be cleared before the end of the transaction. The exact case is
- * when a relation sets a new relfilenode twice in same transaction, yet
- * the second one fails in an aborted subtransaction, e.g.
- *
- * BEGIN;
- * TRUNCATE t;
- * SAVEPOINT save;
- * TRUNCATE t;
- * ROLLBACK TO save;
- * COPY ...
- *
- * Also, if the target file is new-in-transaction, we assume that checking
- * FSM for free space is a waste of time, even if we must use WAL because
- * of archiving. This could possibly be wrong, but it's unlikely.
- *
- * The comments for table_tuple_insert and RelationGetBufferForTuple
- * specify that skipping WAL logging is only safe if we ensure that our
- * tuples do not go into pages containing tuples from any other
- * transactions --- but this must be the case if we have a new table or
- * new relfilenode, so we need no additional work to enforce that.
- *
- * We currently don't support this optimization if the COPY target is a
- * partitioned table as we currently only lazily initialize partition
- * information when routing the first tuple to the partition. We cannot
- * know at this stage if we can perform this optimization. It should be
- * possible to improve on this, but it does mean maintaining heap insert
- * option flags per partition and setting them when we first open the
- * partition.
- *
- * This optimization is not supported for relation types which do not
- * have any physical storage, with foreign tables and views using
- * INSTEAD OF triggers entering in this category. Partitioned tables
- * are not supported as per the description above.
- *----------
+ /*
+ * If the target file is new-in-transaction, we assume that checking FSM
+ * for free space is a waste of time. This could possibly be wrong, but
+ * it's unlikely.
*/
- /* createSubid is creation check, newRelfilenodeSubid is truncation check */
if (RELKIND_HAS_STORAGE(cstate->rel->rd_rel->relkind) &&
(cstate->rel->rd_createSubid != InvalidSubTransactionId ||
- cstate->rel->rd_newRelfilenodeSubid != InvalidSubTransactionId))
- {
+ cstate->rel->rd_firstRelfilenodeSubid != InvalidSubTransactionId))
ti_options |= TABLE_INSERT_SKIP_FSM;
- if (!XLogIsNeeded())
- ti_options |= TABLE_INSERT_SKIP_WAL;
- }
/*
* Optimize if new relfilenode was created in this subxact or one of its
diff --git a/src/backend/commands/createas.c b/src/backend/commands/createas.c
index 2bf7083..20225dc 100644
--- a/src/backend/commands/createas.c
+++ b/src/backend/commands/createas.c
@@ -552,16 +552,13 @@ intorel_startup(DestReceiver *self, int operation, TupleDesc typeinfo)
myState->rel = intoRelationDesc;
myState->reladdr = intoRelationAddr;
myState->output_cid = GetCurrentCommandId(true);
+ myState->ti_options = TABLE_INSERT_SKIP_FSM;
+ myState->bistate = GetBulkInsertState();
/*
- * We can skip WAL-logging the insertions, unless PITR or streaming
- * replication is in use. We can skip the FSM in any case.
+ * Valid smgr_targblock implies something already wrote to the relation.
+ * This may be harmless, but this function hasn't planned for it.
*/
- myState->ti_options = TABLE_INSERT_SKIP_FSM |
- (XLogIsNeeded() ? 0 : TABLE_INSERT_SKIP_WAL);
- myState->bistate = GetBulkInsertState();
-
- /* Not using WAL requires smgr_targblock be initially invalid */
Assert(RelationGetTargetBlock(intoRelationDesc) == InvalidBlockNumber);
}
diff --git a/src/backend/commands/matview.c b/src/backend/commands/matview.c
index 537d0e8..ae809c9 100644
--- a/src/backend/commands/matview.c
+++ b/src/backend/commands/matview.c
@@ -457,17 +457,13 @@ transientrel_startup(DestReceiver *self, int operation, TupleDesc typeinfo)
*/
myState->transientrel = transientrel;
myState->output_cid = GetCurrentCommandId(true);
-
- /*
- * We can skip WAL-logging the insertions, unless PITR or streaming
- * replication is in use. We can skip the FSM in any case.
- */
myState->ti_options = TABLE_INSERT_SKIP_FSM | TABLE_INSERT_FROZEN;
- if (!XLogIsNeeded())
- myState->ti_options |= TABLE_INSERT_SKIP_WAL;
myState->bistate = GetBulkInsertState();
- /* Not using WAL requires smgr_targblock be initially invalid */
+ /*
+ * Valid smgr_targblock implies something already wrote to the relation.
+ * This may be harmless, but this function hasn't planned for it.
+ */
Assert(RelationGetTargetBlock(transientrel) == InvalidBlockNumber);
}
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index e8e004e..e97c144 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -4766,19 +4766,14 @@ ATRewriteTable(AlteredTableInfo *tab, Oid OIDNewHeap, LOCKMODE lockmode)
newrel = NULL;
/*
- * Prepare a BulkInsertState and options for table_tuple_insert. Because
- * we're building a new heap, we can skip WAL-logging and fsync it to disk
- * at the end instead (unless WAL-logging is required for archiving or
- * streaming replication). The FSM is empty too, so don't bother using it.
+ * Prepare a BulkInsertState and options for table_tuple_insert. The FSM
+ * is empty, so don't bother using it.
*/
if (newrel)
{
mycid = GetCurrentCommandId(true);
bistate = GetBulkInsertState();
-
ti_options = TABLE_INSERT_SKIP_FSM;
- if (!XLogIsNeeded())
- ti_options |= TABLE_INSERT_SKIP_WAL;
}
else
{
@@ -12432,6 +12427,8 @@ ATExecSetTableSpace(Oid tableOid, Oid newTableSpace, LOCKMODE lockmode)
table_close(pg_class, RowExclusiveLock);
+ RelationAssumeNewRelfilenode(rel);
+
relation_close(rel, NoLock);
/* Make sure the reltablespace change is visible */
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 1f10a97..aff7234 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -66,7 +66,7 @@
#define BUF_WRITTEN 0x01
#define BUF_REUSABLE 0x02
-#define DROP_RELS_BSEARCH_THRESHOLD 20
+#define RELS_BSEARCH_THRESHOLD 20
typedef struct PrivateRefCountEntry
{
@@ -105,6 +105,19 @@ typedef struct CkptTsStatus
int index;
} CkptTsStatus;
+/*
+ * Type for array used to sort SMgrRelations
+ *
+ * FlushRelationsAllBuffers shares the same comparator function with
+ * DropRelFileNodesAllBuffers. Pointer to this struct and RelFileNode must be
+ * compatible.
+ */
+typedef struct SMgrSortArray
+{
+ RelFileNode rnode; /* This must be the first member */
+ SMgrRelation srel;
+} SMgrSortArray;
+
/* GUC variables */
bool zero_damaged_pages = false;
int bgwriter_lru_maxpages = 100;
@@ -3043,7 +3056,7 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
* an exactly determined value, as it depends on many factors (CPU and RAM
* speeds, amount of shared buffers etc.).
*/
- use_bsearch = n > DROP_RELS_BSEARCH_THRESHOLD;
+ use_bsearch = n > RELS_BSEARCH_THRESHOLD;
/* sort the list of rnodes if necessary */
if (use_bsearch)
@@ -3294,6 +3307,104 @@ FlushRelationBuffers(Relation rel)
}
/* ---------------------------------------------------------------------
+ * FlushRelationsAllBuffers
+ *
+ * This function flushes out of the buffer pool all the pages of all
+ * forks of the specified smgr relations. It's equivalent to calling
+ * FlushRelationBuffers once per fork per relation. The relations are
+ * assumed not to use local buffers.
+ * --------------------------------------------------------------------
+ */
+void
+FlushRelationsAllBuffers(SMgrRelation *smgrs, int nrels)
+{
+ int i;
+ SMgrSortArray *srels;
+ bool use_bsearch;
+
+ if (nrels == 0)
+ return;
+
+ /* fill-in array for qsort */
+ srels = palloc(sizeof(SMgrSortArray) * nrels);
+
+ for (i = 0; i < nrels; i++)
+ {
+ Assert(!RelFileNodeBackendIsTemp(smgrs[i]->smgr_rnode));
+
+ srels[i].rnode = smgrs[i]->smgr_rnode.node;
+ srels[i].srel = smgrs[i];
+ }
+
+ /*
+ * Save the bsearch overhead for low number of relations to sync. See
+ * DropRelFileNodesAllBuffers for details.
+ */
+ use_bsearch = nrels > RELS_BSEARCH_THRESHOLD;
+
+ /* sort the list of SMgrRelations if necessary */
+ if (use_bsearch)
+ pg_qsort(srels, nrels, sizeof(SMgrSortArray), rnode_comparator);
+
+ /* Make sure we can handle the pin inside the loop */
+ ResourceOwnerEnlargeBuffers(CurrentResourceOwner);
+
+ for (i = 0; i < NBuffers; i++)
+ {
+ SMgrSortArray *srelent = NULL;
+ BufferDesc *bufHdr = GetBufferDescriptor(i);
+ uint32 buf_state;
+
+ /*
+ * As in DropRelFileNodeBuffers, an unlocked precheck should be safe
+ * and saves some cycles.
+ */
+
+ if (!use_bsearch)
+ {
+ int j;
+
+ for (j = 0; j < nrels; j++)
+ {
+ if (RelFileNodeEquals(bufHdr->tag.rnode, srels[j].rnode))
+ {
+ srelent = &srels[j];
+ break;
+ }
+ }
+
+ }
+ else
+ {
+ srelent = bsearch((const void *) &(bufHdr->tag.rnode),
+ srels, nrels, sizeof(SMgrSortArray),
+ rnode_comparator);
+ }
+
+ /* buffer doesn't belong to any of the given relfilenodes; skip it */
+ if (srelent == NULL)
+ continue;
+
+ ReservePrivateRefCountEntry();
+
+ buf_state = LockBufHdr(bufHdr);
+ if (RelFileNodeEquals(bufHdr->tag.rnode, srelent->rnode) &&
+ (buf_state & (BM_VALID | BM_DIRTY)) == (BM_VALID | BM_DIRTY))
+ {
+ PinBuffer_Locked(bufHdr);
+ LWLockAcquire(BufferDescriptorGetContentLock(bufHdr), LW_SHARED);
+ FlushBuffer(bufHdr, srelent->srel);
+ LWLockRelease(BufferDescriptorGetContentLock(bufHdr));
+ UnpinBuffer(bufHdr, true);
+ }
+ else
+ UnlockBufHdr(bufHdr, buf_state);
+ }
+
+ pfree(srels);
+}
+
+/* ---------------------------------------------------------------------
* FlushDatabaseBuffers
*
* This function writes all dirty pages of a database out to disk
@@ -3494,13 +3605,15 @@ MarkBufferDirtyHint(Buffer buffer, bool buffer_std)
(pg_atomic_read_u32(&bufHdr->state) & BM_PERMANENT))
{
/*
- * If we're in recovery we cannot dirty a page because of a hint.
- * We can set the hint, just not dirty the page as a result so the
- * hint is lost when we evict the page or shutdown.
+ * If we must not write WAL, due to a relfilenode-specific
+ * condition or being in recovery, don't dirty the page. We can
+ * set the hint, just not dirty the page as a result so the hint
+ * is lost when we evict the page or shutdown.
*
* See src/backend/storage/page/README for longer discussion.
*/
- if (RecoveryInProgress())
+ if (RecoveryInProgress() ||
+ RelFileNodeSkippingWAL(bufHdr->tag.rnode))
return;
/*
diff --git a/src/backend/storage/lmgr/lock.c b/src/backend/storage/lmgr/lock.c
index 9089733..574d8b5 100644
--- a/src/backend/storage/lmgr/lock.c
+++ b/src/backend/storage/lmgr/lock.c
@@ -587,6 +587,18 @@ LockHeldByMe(const LOCKTAG *locktag, LOCKMODE lockmode)
return (locallock && locallock->nLocks > 0);
}
+#ifdef USE_ASSERT_CHECKING
+/*
+ * GetLockMethodLocalHash -- return the hash of local locks, for modules that
+ * evaluate assertions based on all locks held.
+ */
+HTAB *
+GetLockMethodLocalHash(void)
+{
+ return LockMethodLocalHash;
+}
+#endif
+
/*
* LockHasWaiters -- look up 'locktag' and check if releasing this
* lock would wake up other processes waiting for it.
diff --git a/src/backend/storage/smgr/md.c b/src/backend/storage/smgr/md.c
index 82442db..c47c956 100644
--- a/src/backend/storage/smgr/md.c
+++ b/src/backend/storage/smgr/md.c
@@ -248,11 +248,10 @@ mdcreate(SMgrRelation reln, ForkNumber forkNum, bool isRedo)
* During replay, we would delete the file and then recreate it, which is fine
* if the contents of the file were repopulated by subsequent WAL entries.
* But if we didn't WAL-log insertions, but instead relied on fsyncing the
- * file after populating it (as for instance CLUSTER and CREATE INDEX do),
- * the contents of the file would be lost forever. By leaving the empty file
- * until after the next checkpoint, we prevent reassignment of the relfilenode
- * number until it's safe, because relfilenode assignment skips over any
- * existing file.
+ * file after populating it (as we do at wal_level=minimal), the contents of
+ * the file would be lost forever. By leaving the empty file until after the
+ * next checkpoint, we prevent reassignment of the relfilenode number until
+ * it's safe, because relfilenode assignment skips over any existing file.
*
* We do not need to go through this dance for temp relations, though, because
* we never make WAL entries for temp rels, and so a temp rel poses no threat
@@ -885,12 +884,18 @@ mdtruncate(SMgrRelation reln, ForkNumber forknum, BlockNumber nblocks)
* mdimmedsync() -- Immediately sync a relation to stable storage.
*
* Note that only writes already issued are synced; this routine knows
- * nothing of dirty buffers that may exist inside the buffer manager.
+ * nothing of dirty buffers that may exist inside the buffer manager. We
+ * sync active and inactive segments; smgrDoPendingSyncs() relies on this.
+ * Consider a relation skipping WAL. Suppose a checkpoint syncs blocks of
+ * some segment, then mdtruncate() renders that segment inactive. If we
+ * crash before the next checkpoint syncs the newly-inactive segment, that
+ * segment may survive recovery, reintroducing unwanted data into the table.
*/
void
mdimmedsync(SMgrRelation reln, ForkNumber forknum)
{
int segno;
+ int min_inactive_seg;
/*
* NOTE: mdnblocks makes sure we have opened all active segments, so that
@@ -898,7 +903,16 @@ mdimmedsync(SMgrRelation reln, ForkNumber forknum)
*/
mdnblocks(reln, forknum);
- segno = reln->md_num_open_segs[forknum];
+ min_inactive_seg = segno = reln->md_num_open_segs[forknum];
+
+ /*
+ * Temporarily open inactive segments, then close them after sync. There
+ * may be some inactive segments left opened after fsync() error, but that
+ * is harmless. We don't bother to clean them up and take a risk of
+ * further trouble. The next mdclose() will soon close them.
+ */
+ while (_mdfd_openseg(reln, forknum, segno, 0) != NULL)
+ segno++;
while (segno > 0)
{
@@ -909,6 +923,14 @@ mdimmedsync(SMgrRelation reln, ForkNumber forknum)
(errcode_for_file_access(),
errmsg("could not fsync file \"%s\": %m",
FilePathName(v->mdfd_vfd))));
+
+ /* Close inactive segments immediately */
+ if (segno > min_inactive_seg)
+ {
+ FileClose(v->mdfd_vfd);
+ _fdvec_resize(reln, forknum, segno - 1);
+ }
+
segno--;
}
}
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index b50c69b..904ea6f 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -389,6 +389,41 @@ smgrdounlink(SMgrRelation reln, bool isRedo)
}
/*
+ * smgrdosyncall() -- Immediately sync all forks of all given relations
+ *
+ * All forks of all given relations are synced out to the store.
+ *
+ * This is equivalent to FlushRelationBuffers() for each smgr relation,
+ * then calling smgrimmedsync() for all forks of each relation, but it's
+ * significantly quicker so should be preferred when possible.
+ */
+void
+smgrdosyncall(SMgrRelation *rels, int nrels)
+{
+ int i = 0;
+ ForkNumber forknum;
+
+ if (nrels == 0)
+ return;
+
+ FlushRelationsAllBuffers(rels, nrels);
+
+ /*
+ * Sync the physical file(s).
+ */
+ for (i = 0; i < nrels; i++)
+ {
+ int which = rels[i]->smgr_which;
+
+ for (forknum = 0; forknum <= MAX_FORKNUM; forknum++)
+ {
+ if (smgrsw[which].smgr_exists(rels[i], forknum))
+ smgrsw[which].smgr_immedsync(rels[i], forknum);
+ }
+ }
+}
+
+/*
* smgrdounlinkall() -- Immediately unlink all forks of all given relations
*
* All forks of all given relations are removed from the store. This
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index 50f8912..449fe17 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -262,6 +262,9 @@ static void RelationReloadIndexInfo(Relation relation);
static void RelationReloadNailed(Relation relation);
static void RelationFlushRelation(Relation relation);
static void RememberToFreeTupleDescAtEOX(TupleDesc td);
+#ifdef USE_ASSERT_CHECKING
+static void AssertPendingSyncConsistency(Relation relation);
+#endif
static void AtEOXact_cleanup(Relation relation, bool isCommit);
static void AtEOSubXact_cleanup(Relation relation, bool isCommit,
SubTransactionId mySubid, SubTransactionId parentSubid);
@@ -1095,6 +1098,7 @@ RelationBuildDesc(Oid targetRelId, bool insertIt)
relation->rd_isnailed = false;
relation->rd_createSubid = InvalidSubTransactionId;
relation->rd_newRelfilenodeSubid = InvalidSubTransactionId;
+ relation->rd_firstRelfilenodeSubid = InvalidSubTransactionId;
switch (relation->rd_rel->relpersistence)
{
case RELPERSISTENCE_UNLOGGED:
@@ -1828,6 +1832,7 @@ formrdesc(const char *relationName, Oid relationReltype,
relation->rd_isnailed = true;
relation->rd_createSubid = InvalidSubTransactionId;
relation->rd_newRelfilenodeSubid = InvalidSubTransactionId;
+ relation->rd_firstRelfilenodeSubid = InvalidSubTransactionId;
relation->rd_backend = InvalidBackendId;
relation->rd_islocaltemp = false;
@@ -2035,6 +2040,7 @@ RelationIdGetRelation(Oid relationId)
rd = RelationBuildDesc(relationId, true);
if (RelationIsValid(rd))
RelationIncrementReferenceCount(rd);
+
return rd;
}
@@ -2093,7 +2099,7 @@ RelationClose(Relation relation)
#ifdef RELCACHE_FORCE_RELEASE
if (RelationHasReferenceCountZero(relation) &&
relation->rd_createSubid == InvalidSubTransactionId &&
- relation->rd_newRelfilenodeSubid == InvalidSubTransactionId)
+ relation->rd_firstRelfilenodeSubid == InvalidSubTransactionId)
RelationClearRelation(relation, false);
#endif
}
@@ -2509,13 +2515,13 @@ RelationClearRelation(Relation relation, bool rebuild)
* problem.
*
* When rebuilding an open relcache entry, we must preserve ref count,
- * rd_createSubid/rd_newRelfilenodeSubid, and rd_toastoid state. Also
- * attempt to preserve the pg_class entry (rd_rel), tupledesc,
- * rewrite-rule, partition key, and partition descriptor substructures
- * in place, because various places assume that these structures won't
- * move while they are working with an open relcache entry. (Note:
- * the refcount mechanism for tupledescs might someday allow us to
- * remove this hack for the tupledesc.)
+ * rd_*Subid, and rd_toastoid state. Also attempt to preserve the
+ * pg_class entry (rd_rel), tupledesc, rewrite-rule, partition key,
+ * and partition descriptor substructures in place, because various
+ * places assume that these structures won't move while they are
+ * working with an open relcache entry. (Note: the refcount
+ * mechanism for tupledescs might someday allow us to remove this hack
+ * for the tupledesc.)
*
* Note that this process does not touch CurrentResourceOwner; which
* is good because whatever ref counts the entry may have do not
@@ -2599,6 +2605,7 @@ RelationClearRelation(Relation relation, bool rebuild)
/* creation sub-XIDs must be preserved */
SWAPFIELD(SubTransactionId, rd_createSubid);
SWAPFIELD(SubTransactionId, rd_newRelfilenodeSubid);
+ SWAPFIELD(SubTransactionId, rd_firstRelfilenodeSubid);
/* un-swap rd_rel pointers, swap contents instead */
SWAPFIELD(Form_pg_class, rd_rel);
/* ... but actually, we don't have to update newrel->rd_rel */
@@ -2666,12 +2673,12 @@ static void
RelationFlushRelation(Relation relation)
{
if (relation->rd_createSubid != InvalidSubTransactionId ||
- relation->rd_newRelfilenodeSubid != InvalidSubTransactionId)
+ relation->rd_firstRelfilenodeSubid != InvalidSubTransactionId)
{
/*
* New relcache entries are always rebuilt, not flushed; else we'd
- * forget the "new" status of the relation, which is a useful
- * optimization to have. Ditto for the new-relfilenode status.
+ * forget the "new" status of the relation. Ditto for the
+ * new-relfilenode status.
*
* The rel could have zero refcnt here, so temporarily increment the
* refcnt to ensure it's safe to rebuild it. We can assume that the
@@ -2751,11 +2758,10 @@ RelationCacheInvalidateEntry(Oid relationId)
* relation cache and re-read relation mapping data.
*
* This is currently used only to recover from SI message buffer overflow,
- * so we do not touch new-in-transaction relations; they cannot be targets
- * of cross-backend SI updates (and our own updates now go through a
- * separate linked list that isn't limited by the SI message buffer size).
- * Likewise, we need not discard new-relfilenode-in-transaction hints,
- * since any invalidation of those would be a local event.
+ * so we do not touch relations having new-in-transaction relfilenodes; they
+ * cannot be targets of cross-backend SI updates (and our own updates now go
+ * through a separate linked list that isn't limited by the SI message
+ * buffer size).
*
* We do this in two phases: the first pass deletes deletable items, and
* the second one rebuilds the rebuildable items. This is essential for
@@ -2806,7 +2812,7 @@ RelationCacheInvalidate(void)
* pending invalidations.
*/
if (relation->rd_createSubid != InvalidSubTransactionId ||
- relation->rd_newRelfilenodeSubid != InvalidSubTransactionId)
+ relation->rd_firstRelfilenodeSubid != InvalidSubTransactionId)
continue;
relcacheInvalsReceived++;
@@ -2918,6 +2924,78 @@ RememberToFreeTupleDescAtEOX(TupleDesc td)
EOXactTupleDescArray[NextEOXactTupleDescNum++] = td;
}
+#ifdef USE_ASSERT_CHECKING
+static void
+AssertPendingSyncConsistency(Relation relation)
+{
+ bool relcache_verdict =
+ relation->rd_rel->relpersistence == RELPERSISTENCE_PERMANENT &&
+ ((relation->rd_createSubid != InvalidSubTransactionId &&
+ RELKIND_HAS_STORAGE(relation->rd_rel->relkind)) ||
+ relation->rd_firstRelfilenodeSubid != InvalidSubTransactionId);
+
+ Assert(relcache_verdict == RelFileNodeSkippingWAL(relation->rd_node));
+}
+
+/*
+ * AssertPendingSyncs_RelationCache
+ *
+ * Assert that relcache.c and storage.c agree on whether to skip WAL.
+ *
+ * This consistently detects relcache.c skipping WAL while storage.c is not
+ * skipping WAL. It often fails to detect the reverse error, because
+ * invalidation will have destroyed the relcache entry. It will detect the
+ * reverse error if something opens the relation after the DDL.
+ */
+void
+AssertPendingSyncs_RelationCache(void)
+{
+ HASH_SEQ_STATUS status;
+ LOCALLOCK *locallock;
+ Relation *rels;
+ int maxrels;
+ int nrels;
+ RelIdCacheEnt *idhentry;
+ int i;
+
+ /* open every relation that this transaction has locked */
+ PushActiveSnapshot(GetTransactionSnapshot());
+ maxrels = 1;
+ rels = palloc(maxrels * sizeof(*rels));
+ nrels = 0;
+ hash_seq_init(&status, GetLockMethodLocalHash());
+ while ((locallock = (LOCALLOCK *) hash_seq_search(&status)) != NULL)
+ {
+ Oid relid;
+ Relation r;
+
+ if (locallock->nLocks <= 0)
+ continue;
+ if ((LockTagType) locallock->tag.lock.locktag_type !=
+ LOCKTAG_RELATION)
+ continue;
+ relid = ObjectIdGetDatum(locallock->tag.lock.locktag_field2);
+ r = RelationIdGetRelation(relid);
+ if (r == NULL)
+ continue;
+ if (nrels >= maxrels)
+ {
+ maxrels *= 2;
+ rels = repalloc(rels, maxrels * sizeof(*rels));
+ }
+ rels[nrels++] = r;
+ }
+
+ hash_seq_init(&status, RelationIdCache);
+ while ((idhentry = (RelIdCacheEnt *) hash_seq_search(&status)) != NULL)
+ AssertPendingSyncConsistency(idhentry->reldesc);
+
+ for (i = 0; i < nrels; i++)
+ RelationClose(rels[i]);
+ PopActiveSnapshot();
+}
+#endif
+
/*
* AtEOXact_RelationCache
*
@@ -3029,10 +3107,7 @@ AtEOXact_cleanup(Relation relation, bool isCommit)
*
* During commit, reset the flag to zero, since we are now out of the
* creating transaction. During abort, simply delete the relcache entry
- * --- it isn't interesting any longer. (NOTE: if we have forgotten the
- * new-ness of a new relation due to a forced cache flush, the entry will
- * get deleted anyway by shared-cache-inval processing of the aborted
- * pg_class insertion.)
+ * --- it isn't interesting any longer.
*/
if (relation->rd_createSubid != InvalidSubTransactionId)
{
@@ -3060,9 +3135,10 @@ AtEOXact_cleanup(Relation relation, bool isCommit)
}
/*
- * Likewise, reset the hint about the relfilenode being new.
+ * Likewise, reset any record of the relfilenode being new.
*/
relation->rd_newRelfilenodeSubid = InvalidSubTransactionId;
+ relation->rd_firstRelfilenodeSubid = InvalidSubTransactionId;
}
/*
@@ -3154,7 +3230,7 @@ AtEOSubXact_cleanup(Relation relation, bool isCommit,
}
/*
- * Likewise, update or drop any new-relfilenode-in-subtransaction hint.
+ * Likewise, update or drop any new-relfilenode-in-subtransaction record.
*/
if (relation->rd_newRelfilenodeSubid == mySubid)
{
@@ -3163,6 +3239,14 @@ AtEOSubXact_cleanup(Relation relation, bool isCommit,
else
relation->rd_newRelfilenodeSubid = InvalidSubTransactionId;
}
+
+ if (relation->rd_firstRelfilenodeSubid == mySubid)
+ {
+ if (isCommit)
+ relation->rd_firstRelfilenodeSubid = parentSubid;
+ else
+ relation->rd_firstRelfilenodeSubid = InvalidSubTransactionId;
+ }
}
@@ -3252,6 +3336,7 @@ RelationBuildLocalRelation(const char *relname,
/* it's being created in this transaction */
rel->rd_createSubid = GetCurrentSubTransactionId();
rel->rd_newRelfilenodeSubid = InvalidSubTransactionId;
+ rel->rd_firstRelfilenodeSubid = InvalidSubTransactionId;
/*
* create a new tuple descriptor from the one passed in. We do this
@@ -3549,14 +3634,29 @@ RelationSetNewRelfilenode(Relation relation, char persistence)
*/
CommandCounterIncrement();
- /*
- * Mark the rel as having been given a new relfilenode in the current
- * (sub) transaction. This is a hint that can be used to optimize later
- * operations on the rel in the same transaction.
- */
+ RelationAssumeNewRelfilenode(relation);
+}
+
+/*
+ * RelationAssumeNewRelfilenode
+ *
+ * Code that modifies pg_class.reltablespace or pg_class.relfilenode must call
+ * this. The call shall precede any code that might insert WAL records whose
+ * replay would modify bytes in the new RelFileNode, and the call shall follow
+ * any WAL modifying bytes in the prior RelFileNode. See struct RelationData.
+ * Ideally, call this as near as possible to the CommandCounterIncrement()
+ * that makes the pg_class change visible (before it or after it); that
+ * minimizes the chance of future development adding a forbidden WAL insertion
+ * between RelationAssumeNewRelfilenode() and CommandCounterIncrement().
+ */
+void
+RelationAssumeNewRelfilenode(Relation relation)
+{
relation->rd_newRelfilenodeSubid = GetCurrentSubTransactionId();
+ if (relation->rd_firstRelfilenodeSubid == InvalidSubTransactionId)
+ relation->rd_firstRelfilenodeSubid = relation->rd_newRelfilenodeSubid;
- /* Flag relation as needing eoxact cleanup (to remove the hint) */
+ /* Flag relation as needing eoxact cleanup (to clear these fields) */
EOXactListAdd(relation);
}
@@ -5642,6 +5742,7 @@ load_relcache_init_file(bool shared)
rel->rd_fkeylist = NIL;
rel->rd_createSubid = InvalidSubTransactionId;
rel->rd_newRelfilenodeSubid = InvalidSubTransactionId;
+ rel->rd_firstRelfilenodeSubid = InvalidSubTransactionId;
rel->rd_amcache = NULL;
MemSet(&rel->pgstat_info, 0, sizeof(rel->pgstat_info));
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 8d951ce..f436a0c 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -36,6 +36,7 @@
#include "access/xlog_internal.h"
#include "catalog/namespace.h"
#include "catalog/pg_authid.h"
+#include "catalog/storage.h"
#include "commands/async.h"
#include "commands/prepare.h"
#include "commands/trigger.h"
@@ -2662,6 +2663,18 @@ static struct config_int ConfigureNamesInt[] =
},
{
+ {"wal_skip_threshold", PGC_USERSET, RESOURCES_DISK,
+ gettext_noop("Size of new file to fsync instead of writing WAL."),
+ NULL,
+ GUC_UNIT_KB
+ },
+ &wal_skip_threshold,
+ 2048,
+ 0, MAX_KILOBYTES,
+ NULL, NULL, NULL
+ },
+
+ {
{"max_wal_senders", PGC_POSTMASTER, REPLICATION_SENDING,
gettext_noop("Sets the maximum number of simultaneously running WAL sender processes."),
NULL
diff --git a/src/include/access/gist_private.h b/src/include/access/gist_private.h
index a409975..3455dd2 100644
--- a/src/include/access/gist_private.h
+++ b/src/include/access/gist_private.h
@@ -455,6 +455,8 @@ extern XLogRecPtr gistXLogSplit(bool page_is_leaf,
BlockNumber origrlink, GistNSN oldnsn,
Buffer leftchild, bool markfollowright);
+extern XLogRecPtr gistXLogAssignLSN(void);
+
/* gistget.c */
extern bool gistgettuple(IndexScanDesc scan, ScanDirection dir);
extern int64 gistgetbitmap(IndexScanDesc scan, TIDBitmap *tbm);
diff --git a/src/include/access/gistxlog.h b/src/include/access/gistxlog.h
index e44922d..596a881 100644
--- a/src/include/access/gistxlog.h
+++ b/src/include/access/gistxlog.h
@@ -26,6 +26,7 @@
/* #define XLOG_GIST_INSERT_COMPLETE 0x40 */ /* not used anymore */
/* #define XLOG_GIST_CREATE_INDEX 0x50 */ /* not used anymore */
#define XLOG_GIST_PAGE_DELETE 0x60
+#define XLOG_GIST_ASSIGN_LSN 0x70 /* nop, assign new LSN */
/*
* Backup Blk 0: updated page.
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index 858bcb6..22916e8 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -29,7 +29,6 @@
/* "options" flag bits for heap_insert */
-#define HEAP_INSERT_SKIP_WAL TABLE_INSERT_SKIP_WAL
#define HEAP_INSERT_SKIP_FSM TABLE_INSERT_SKIP_FSM
#define HEAP_INSERT_FROZEN TABLE_INSERT_FROZEN
#define HEAP_INSERT_NO_LOGICAL TABLE_INSERT_NO_LOGICAL
@@ -166,8 +165,6 @@ extern void simple_heap_delete(Relation relation, ItemPointer tid);
extern void simple_heap_update(Relation relation, ItemPointer otid,
HeapTuple tup);
-extern void heap_sync(Relation relation);
-
extern TransactionId heap_compute_xid_horizon_for_tuples(Relation rel,
ItemPointerData *items,
int nitems);
diff --git a/src/include/access/rewriteheap.h b/src/include/access/rewriteheap.h
index 8056253..7f9736e 100644
--- a/src/include/access/rewriteheap.h
+++ b/src/include/access/rewriteheap.h
@@ -23,7 +23,7 @@ typedef struct RewriteStateData *RewriteState;
extern RewriteState begin_heap_rewrite(Relation OldHeap, Relation NewHeap,
TransactionId OldestXmin, TransactionId FreezeXid,
- MultiXactId MultiXactCutoff, bool use_wal);
+ MultiXactId MultiXactCutoff);
extern void end_heap_rewrite(RewriteState state);
extern void rewrite_heap_tuple(RewriteState state, HeapTuple oldTuple,
HeapTuple newTuple);
diff --git a/src/include/access/tableam.h b/src/include/access/tableam.h
index 6402291..be4dd52 100644
--- a/src/include/access/tableam.h
+++ b/src/include/access/tableam.h
@@ -127,7 +127,7 @@ typedef struct TM_FailureData
} TM_FailureData;
/* "options" flag bits for table_tuple_insert */
-#define TABLE_INSERT_SKIP_WAL 0x0001
+/* TABLE_INSERT_SKIP_WAL was 0x0001; RelationNeedsWAL() now governs */
#define TABLE_INSERT_SKIP_FSM 0x0002
#define TABLE_INSERT_FROZEN 0x0004
#define TABLE_INSERT_NO_LOGICAL 0x0008
@@ -409,9 +409,8 @@ typedef struct TableAmRoutine
/*
* Perform operations necessary to complete insertions made via
- * tuple_insert and multi_insert with a BulkInsertState specified. This
- * may for example be used to flush the relation, when the
- * TABLE_INSERT_SKIP_WAL option was used.
+ * tuple_insert and multi_insert with a BulkInsertState specified. In-tree
+ * access methods ceased to use this.
*
* Typically callers of tuple_insert and multi_insert will just pass all
* the flags that apply to them, and each AM has to decide which of them
@@ -1087,10 +1086,6 @@ table_compute_xid_horizon_for_tuples(Relation rel,
* The options bitmask allows the caller to specify options that may change the
* behaviour of the AM. The AM will ignore options that it does not support.
*
- * If the TABLE_INSERT_SKIP_WAL option is specified, the new tuple doesn't
- * need to be logged to WAL, even for a non-temp relation. It is the AMs
- * choice whether this optimization is supported.
- *
* If the TABLE_INSERT_SKIP_FSM option is specified, AMs are free to not reuse
* free space in the relation. This can save some cycles when we know the
* relation is new and doesn't contain useful amounts of free space.
@@ -1310,9 +1305,7 @@ table_tuple_lock(Relation rel, ItemPointer tid, Snapshot snapshot,
/*
* Perform operations necessary to complete insertions made via
- * tuple_insert and multi_insert with a BulkInsertState specified. This
- * e.g. may e.g. used to flush the relation when inserting with
- * TABLE_INSERT_SKIP_WAL specified.
+ * tuple_insert and multi_insert with a BulkInsertState specified.
*/
static inline void
table_finish_bulk_insert(Relation rel, int options)
diff --git a/src/include/catalog/storage.h b/src/include/catalog/storage.h
index 3579d3f..d58030d 100644
--- a/src/include/catalog/storage.h
+++ b/src/include/catalog/storage.h
@@ -19,18 +19,24 @@
#include "storage/smgr.h"
#include "utils/relcache.h"
+/* GUC variables */
+extern int wal_skip_threshold;
+
extern SMgrRelation RelationCreateStorage(RelFileNode rnode, char relpersistence);
extern void RelationDropStorage(Relation rel);
extern void RelationPreserveStorage(RelFileNode rnode, bool atCommit);
+extern void RelationPreTruncate(Relation rel);
extern void RelationTruncate(Relation rel, BlockNumber nblocks);
extern void RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
ForkNumber forkNum, char relpersistence);
+extern bool RelFileNodeSkippingWAL(RelFileNode rnode);
/*
* These functions used to be in storage/smgr/smgr.c, which explains the
* naming
*/
extern void smgrDoPendingDeletes(bool isCommit);
+extern void smgrDoPendingSyncs(bool isCommit);
extern int smgrGetPendingDeletes(bool forCommit, RelFileNode **ptr);
extern void AtSubCommit_smgr(void);
extern void AtSubAbort_smgr(void);
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index 17b97f7..8e21181 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -49,6 +49,9 @@ typedef enum
/* forward declared, to avoid having to expose buf_internals.h here */
struct WritebackContext;
+/* forward declared, to avoid including smgr.h here */
+struct SMgrRelationData;
+
/* in globals.c ... this duplicates miscadmin.h */
extern PGDLLIMPORT int NBuffers;
@@ -189,6 +192,7 @@ extern BlockNumber RelationGetNumberOfBlocksInFork(Relation relation,
ForkNumber forkNum);
extern void FlushOneBuffer(Buffer buffer);
extern void FlushRelationBuffers(Relation rel);
+extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
extern void FlushDatabaseBuffers(Oid dbid);
extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
int nforks, BlockNumber *firstDelBlock);
diff --git a/src/include/storage/lock.h b/src/include/storage/lock.h
index e8b50fe..493deb0 100644
--- a/src/include/storage/lock.h
+++ b/src/include/storage/lock.h
@@ -544,6 +544,9 @@ extern void LockReleaseSession(LOCKMETHODID lockmethodid);
extern void LockReleaseCurrentOwner(LOCALLOCK **locallocks, int nlocks);
extern void LockReassignCurrentOwner(LOCALLOCK **locallocks, int nlocks);
extern bool LockHeldByMe(const LOCKTAG *locktag, LOCKMODE lockmode);
+#ifdef USE_ASSERT_CHECKING
+extern HTAB *GetLockMethodLocalHash(void);
+#endif
extern bool LockHasWaiters(const LOCKTAG *locktag,
LOCKMODE lockmode, bool sessionLock);
extern VirtualTransactionId *GetLockConflicts(const LOCKTAG *locktag,
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index 1543d8d..31a5ecd 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -89,6 +89,7 @@ extern void smgrcloseall(void);
extern void smgrclosenode(RelFileNodeBackend rnode);
extern void smgrcreate(SMgrRelation reln, ForkNumber forknum, bool isRedo);
extern void smgrdounlink(SMgrRelation reln, bool isRedo);
+extern void smgrdosyncall(SMgrRelation *rels, int nrels);
extern void smgrdounlinkall(SMgrRelation *rels, int nrels, bool isRedo);
extern void smgrextend(SMgrRelation reln, ForkNumber forknum,
BlockNumber blocknum, char *buffer, bool skipFsync);
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 31d8a1a..9db3d23 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -63,22 +63,40 @@ typedef struct RelationData
* rd_replidindex) */
bool rd_statvalid; /* is rd_statlist valid? */
- /*
+ /*----------
* rd_createSubid is the ID of the highest subtransaction the rel has
- * survived into; or zero if the rel was not created in the current top
- * transaction. This can be now be relied on, whereas previously it could
- * be "forgotten" in earlier releases. Likewise, rd_newRelfilenodeSubid is
- * the ID of the highest subtransaction the relfilenode change has
- * survived into, or zero if not changed in the current transaction (or we
- * have forgotten changing it). rd_newRelfilenodeSubid can be forgotten
- * when a relation has multiple new relfilenodes within a single
- * transaction, with one of them occurring in a subsequently aborted
- * subtransaction, e.g. BEGIN; TRUNCATE t; SAVEPOINT save; TRUNCATE t;
- * ROLLBACK TO save; -- rd_newRelfilenodeSubid is now forgotten
+ * survived into or zero if the rel was not created in the current top
+ * transaction. rd_firstRelfilenodeSubid is the ID of the highest
+ * subtransaction an rd_node change has survived into or zero if rd_node
+ * matches the value it had at the start of the current top transaction.
+ * (Rolling back the subtransaction that rd_firstRelfilenodeSubid denotes
+ * would restore rd_node to the value it had at the start of the current
+ * top transaction. Rolling back any lower subtransaction would not.)
+ * Their accuracy is critical to RelationNeedsWAL().
+ *
+ * rd_newRelfilenodeSubid is the ID of the highest subtransaction the
+ * most-recent relfilenode change has survived into or zero if not changed
+ * in the current transaction (or we have forgotten changing it). This
+ * field is accurate when non-zero, but it can be zero when a relation has
+ * multiple new relfilenodes within a single transaction, with one of them
+ * occurring in a subsequently aborted subtransaction, e.g.
+ * BEGIN;
+ * TRUNCATE t;
+ * SAVEPOINT save;
+ * TRUNCATE t;
+ * ROLLBACK TO save;
+ * -- rd_newRelfilenodeSubid is now forgotten
+ *
+ * These fields are read-only outside relcache.c. Other files trigger
+ * rd_node changes by updating pg_class.reltablespace and/or
+ * pg_class.relfilenode. They must call RelationAssumeNewRelfilenode() to
+ * update these fields.
*/
SubTransactionId rd_createSubid; /* rel was created in current xact */
- SubTransactionId rd_newRelfilenodeSubid; /* new relfilenode assigned in
- * current xact */
+ SubTransactionId rd_newRelfilenodeSubid; /* highest subxact changing
+ * rd_node to current value */
+ SubTransactionId rd_firstRelfilenodeSubid; /* highest subxact changing
+ * rd_node to any value */
Form_pg_class rd_rel; /* RELATION tuple */
TupleDesc rd_att; /* tuple descriptor */
@@ -520,9 +538,16 @@ typedef struct ViewOptions
/*
* RelationNeedsWAL
* True if relation needs WAL.
- */
-#define RelationNeedsWAL(relation) \
- ((relation)->rd_rel->relpersistence == RELPERSISTENCE_PERMANENT)
+ *
+ * Returns false if wal_level = minimal and this relation is created or
+ * truncated in the current transaction. See "Skipping WAL for New
+ * RelFileNode" in src/backend/access/transam/README.
+ */
+#define RelationNeedsWAL(relation) \
+ ((relation)->rd_rel->relpersistence == RELPERSISTENCE_PERMANENT && \
+ (XLogIsNeeded() || \
+ (relation->rd_createSubid == InvalidSubTransactionId && \
+ relation->rd_firstRelfilenodeSubid == InvalidSubTransactionId)))
/*
* RelationUsesLocalBuffers
diff --git a/src/include/utils/relcache.h b/src/include/utils/relcache.h
index 90487b2..66e247d 100644
--- a/src/include/utils/relcache.h
+++ b/src/include/utils/relcache.h
@@ -106,9 +106,10 @@ extern Relation RelationBuildLocalRelation(const char *relname,
char relkind);
/*
- * Routine to manage assignment of new relfilenode to a relation
+ * Routines to manage assignment of new relfilenode to a relation
*/
extern void RelationSetNewRelfilenode(Relation relation, char persistence);
+extern void RelationAssumeNewRelfilenode(Relation relation);
/*
* Routines for flushing/rebuilding relcache entries in various scenarios
@@ -121,6 +122,11 @@ extern void RelationCacheInvalidate(void);
extern void RelationCloseSmgrByOid(Oid relationId);
+#ifdef USE_ASSERT_CHECKING
+extern void AssertPendingSyncs_RelationCache(void);
+#else
+#define AssertPendingSyncs_RelationCache() do {} while (0)
+#endif
extern void AtEOXact_RelationCache(bool isCommit);
extern void AtEOSubXact_RelationCache(bool isCommit, SubTransactionId mySubid,
SubTransactionId parentSubid);
diff --git a/src/test/recovery/t/018_wal_optimize.pl b/src/test/recovery/t/018_wal_optimize.pl
new file mode 100644
index 0000000..78d81e1
--- /dev/null
+++ b/src/test/recovery/t/018_wal_optimize.pl
@@ -0,0 +1,374 @@
+# Test WAL replay when some operation has skipped WAL.
+#
+# These tests exercise code that once violated the mandate described in
+# src/backend/access/transam/README section "Skipping WAL for New
+# RelFileNode". The tests work by committing some transactions, initiating an
+# immediate shutdown, and confirming that the expected data survives recovery.
+# For many years, individual commands made the decision to skip WAL, hence the
+# frequent appearance of COPY in these tests.
+use strict;
+use warnings;
+
+use PostgresNode;
+use TestLib;
+use Test::More tests => 34;
+
+sub check_orphan_relfilenodes
+{
+ my ($node, $test_name) = @_;
+
+ my $db_oid = $node->safe_psql('postgres',
+ "SELECT oid FROM pg_database WHERE datname = 'postgres'");
+ my $prefix = "base/$db_oid/";
+ my $filepaths_referenced = $node->safe_psql(
+ 'postgres', "
+ SELECT pg_relation_filepath(oid) FROM pg_class
+ WHERE reltablespace = 0 AND relpersistence <> 't' AND
+ pg_relation_filepath(oid) IS NOT NULL;");
+ is_deeply(
+ [
+ sort(map { "$prefix$_" }
+ grep(/^[0-9]+$/, slurp_dir($node->data_dir . "/$prefix")))
+ ],
+ [ sort split /\n/, $filepaths_referenced ],
+ $test_name);
+ return;
+}
+
+# We run this same test suite for both wal_level=minimal and replica.
+sub run_wal_optimize
+{
+ my $wal_level = shift;
+
+ my $node = get_new_node("node_$wal_level");
+ $node->init;
+ $node->append_conf(
+ 'postgresql.conf', qq(
+wal_level = $wal_level
+max_prepared_transactions = 1
+wal_log_hints = on
+wal_skip_threshold = 0
+#wal_debug = on
+));
+ $node->start;
+
+ # Setup
+ my $tablespace_dir = $node->basedir . '/tablespace_other';
+ mkdir($tablespace_dir);
+ $tablespace_dir = TestLib::perl2host($tablespace_dir);
+ $node->safe_psql('postgres',
+ "CREATE TABLESPACE other LOCATION '$tablespace_dir';");
+
+ # Test direct truncation optimization. No tuples.
+ $node->safe_psql(
+ 'postgres', "
+ BEGIN;
+ CREATE TABLE trunc (id serial PRIMARY KEY);
+ TRUNCATE trunc;
+ COMMIT;");
+ $node->stop('immediate');
+ $node->start;
+ my $result = $node->safe_psql('postgres', "SELECT count(*) FROM trunc;");
+ is($result, qq(0), "wal_level = $wal_level, TRUNCATE with empty table");
+
+ # Test truncation with inserted tuples within the same transaction.
+ # Tuples inserted after the truncation should be seen.
+ $node->safe_psql(
+ 'postgres', "
+ BEGIN;
+ CREATE TABLE trunc_ins (id serial PRIMARY KEY);
+ INSERT INTO trunc_ins VALUES (DEFAULT);
+ TRUNCATE trunc_ins;
+ INSERT INTO trunc_ins VALUES (DEFAULT);
+ COMMIT;");
+ $node->stop('immediate');
+ $node->start;
+ $result = $node->safe_psql('postgres', "SELECT count(*) FROM trunc_ins;");
+ is($result, qq(1), "wal_level = $wal_level, TRUNCATE INSERT");
+
+ # Same for prepared transaction.
+ # Tuples inserted after the truncation should be seen.
+ $node->safe_psql(
+ 'postgres', "
+ BEGIN;
+ CREATE TABLE twophase (id serial PRIMARY KEY);
+ INSERT INTO twophase VALUES (DEFAULT);
+ TRUNCATE twophase;
+ INSERT INTO twophase VALUES (DEFAULT);
+ PREPARE TRANSACTION 't';
+ COMMIT PREPARED 't';");
+ $node->stop('immediate');
+ $node->start;
+ $result = $node->safe_psql('postgres', "SELECT count(*) FROM twophase;");
+ is($result, qq(1), "wal_level = $wal_level, TRUNCATE INSERT PREPARE");
+
+ # Same with writing WAL at end of xact, instead of syncing.
+ # Tuples inserted after the truncation should be seen.
+ $node->safe_psql(
+ 'postgres', "
+ SET wal_skip_threshold = '1TB';
+ BEGIN;
+ CREATE TABLE noskip (id serial PRIMARY KEY);
+ INSERT INTO noskip VALUES (DEFAULT);
+ TRUNCATE noskip;
+ INSERT INTO noskip VALUES (DEFAULT);
+ COMMIT;");
+ $node->stop('immediate');
+ $node->start;
+ $result = $node->safe_psql('postgres', "SELECT count(*) FROM noskip;");
+ is($result, qq(1),
+ "wal_level = $wal_level, TRUNCATE with end-of-xact WAL");
+
+ # Data file for COPY query in subsequent tests
+ my $basedir = $node->basedir;
+ my $copy_file = "$basedir/copy_data.txt";
+ TestLib::append_to_file(
+ $copy_file, qq(20000,30000
+20001,30001
+20002,30002));
+
+ # Test truncation with inserted tuples using both INSERT and COPY. Tuples
+ # inserted after the truncation should be seen.
+ $node->safe_psql(
+ 'postgres', "
+ BEGIN;
+ CREATE TABLE ins_trunc (id serial PRIMARY KEY, id2 int);
+ INSERT INTO ins_trunc VALUES (DEFAULT, generate_series(1,10000));
+ TRUNCATE ins_trunc;
+ INSERT INTO ins_trunc (id, id2) VALUES (DEFAULT, 10000);
+ COPY ins_trunc FROM '$copy_file' DELIMITER ',';
+ INSERT INTO ins_trunc (id, id2) VALUES (DEFAULT, 10000);
+ COMMIT;");
+ $node->stop('immediate');
+ $node->start;
+ $result = $node->safe_psql('postgres', "SELECT count(*) FROM ins_trunc;");
+ is($result, qq(5), "wal_level = $wal_level, TRUNCATE COPY INSERT");
+
+ # Test truncation with inserted tuples using COPY. Tuples copied after
+ # the truncation should be seen.
+ $node->safe_psql(
+ 'postgres', "
+ BEGIN;
+ CREATE TABLE trunc_copy (id serial PRIMARY KEY, id2 int);
+ INSERT INTO trunc_copy VALUES (DEFAULT, generate_series(1,3000));
+ TRUNCATE trunc_copy;
+ COPY trunc_copy FROM '$copy_file' DELIMITER ',';
+ COMMIT;");
+ $node->stop('immediate');
+ $node->start;
+ $result =
+ $node->safe_psql('postgres', "SELECT count(*) FROM trunc_copy;");
+ is($result, qq(3), "wal_level = $wal_level, TRUNCATE COPY");
+
+ # Like previous test, but rollback SET TABLESPACE in a subtransaction.
+ $node->safe_psql(
+ 'postgres', "
+ BEGIN;
+ CREATE TABLE spc_abort (id serial PRIMARY KEY, id2 int);
+ INSERT INTO spc_abort VALUES (DEFAULT, generate_series(1,3000));
+ TRUNCATE spc_abort;
+ SAVEPOINT s;
+ ALTER TABLE spc_abort SET TABLESPACE other; ROLLBACK TO s;
+ COPY spc_abort FROM '$copy_file' DELIMITER ',';
+ COMMIT;");
+ $node->stop('immediate');
+ $node->start;
+ $result = $node->safe_psql('postgres', "SELECT count(*) FROM spc_abort;");
+ is($result, qq(3),
+ "wal_level = $wal_level, SET TABLESPACE abort subtransaction");
+
+ # in different subtransaction patterns
+ $node->safe_psql(
+ 'postgres', "
+ BEGIN;
+ CREATE TABLE spc_commit (id serial PRIMARY KEY, id2 int);
+ INSERT INTO spc_commit VALUES (DEFAULT, generate_series(1,3000));
+ TRUNCATE spc_commit;
+ SAVEPOINT s; ALTER TABLE spc_commit SET TABLESPACE other; RELEASE s;
+ COPY spc_commit FROM '$copy_file' DELIMITER ',';
+ COMMIT;");
+ $node->stop('immediate');
+ $node->start;
+ $result =
+ $node->safe_psql('postgres', "SELECT count(*) FROM spc_commit;");
+ is($result, qq(3),
+ "wal_level = $wal_level, SET TABLESPACE commit subtransaction");
+
+ $node->safe_psql(
+ 'postgres', "
+ BEGIN;
+ CREATE TABLE spc_nest (id serial PRIMARY KEY, id2 int);
+ INSERT INTO spc_nest VALUES (DEFAULT, generate_series(1,3000));
+ TRUNCATE spc_nest;
+ SAVEPOINT s;
+ ALTER TABLE spc_nest SET TABLESPACE other;
+ SAVEPOINT s2;
+ ALTER TABLE spc_nest SET TABLESPACE pg_default;
+ ROLLBACK TO s2;
+ SAVEPOINT s2;
+ ALTER TABLE spc_nest SET TABLESPACE pg_default;
+ RELEASE s2;
+ ROLLBACK TO s;
+ COPY spc_nest FROM '$copy_file' DELIMITER ',';
+ COMMIT;");
+ $node->stop('immediate');
+ $node->start;
+ $result = $node->safe_psql('postgres', "SELECT count(*) FROM spc_nest;");
+ is($result, qq(3),
+ "wal_level = $wal_level, SET TABLESPACE nested subtransaction");
+
+ $node->safe_psql(
+ 'postgres', "
+ CREATE TABLE spc_hint (id int);
+ INSERT INTO spc_hint VALUES (1);
+ BEGIN;
+ ALTER TABLE spc_hint SET TABLESPACE other;
+ CHECKPOINT;
+ SELECT * FROM spc_hint; -- set hint bit
+ INSERT INTO spc_hint VALUES (2);
+ COMMIT;");
+ $node->stop('immediate');
+ $node->start;
+ $result = $node->safe_psql('postgres', "SELECT count(*) FROM spc_hint;");
+ is($result, qq(2), "wal_level = $wal_level, SET TABLESPACE, hint bit");
+
+ $node->safe_psql(
+ 'postgres', "
+ BEGIN;
+ CREATE TABLE idx_hint (c int PRIMARY KEY);
+ SAVEPOINT q; INSERT INTO idx_hint VALUES (1); ROLLBACK TO q;
+ CHECKPOINT;
+ INSERT INTO idx_hint VALUES (1); -- set index hint bit
+ INSERT INTO idx_hint VALUES (2);
+ COMMIT;");
+ $node->stop('immediate');
+ $node->start;
+ $result = $node->psql('postgres',);
+ my ($ret, $stdout, $stderr) =
+ $node->psql('postgres', "INSERT INTO idx_hint VALUES (2);");
+ is($ret, qq(3), "wal_level = $wal_level, unique index LP_DEAD");
+ like(
+ $stderr,
+ qr/violates unique/,
+ "wal_level = $wal_level, unique index LP_DEAD message");
+
+ # UPDATE touches two buffers for one row.
+ $node->safe_psql(
+ 'postgres', "
+ BEGIN;
+ CREATE TABLE upd (id serial PRIMARY KEY, id2 int);
+ INSERT INTO upd (id, id2) VALUES (DEFAULT, generate_series(1,10000));
+ COPY upd FROM '$copy_file' DELIMITER ',';
+ UPDATE upd SET id2 = id2 + 1;
+ DELETE FROM upd;
+ COMMIT;");
+ $node->stop('immediate');
+ $node->start;
+ $result = $node->safe_psql('postgres', "SELECT count(*) FROM upd;");
+ is($result, qq(0),
+ "wal_level = $wal_level, UPDATE touches two buffers for one row");
+
+ # Test consistency of COPY with INSERT for table created in the same
+ # transaction.
+ $node->safe_psql(
+ 'postgres', "
+ BEGIN;
+ CREATE TABLE ins_copy (id serial PRIMARY KEY, id2 int);
+ INSERT INTO ins_copy VALUES (DEFAULT, 1);
+ COPY ins_copy FROM '$copy_file' DELIMITER ',';
+ COMMIT;");
+ $node->stop('immediate');
+ $node->start;
+ $result = $node->safe_psql('postgres', "SELECT count(*) FROM ins_copy;");
+ is($result, qq(4), "wal_level = $wal_level, INSERT COPY");
+
+ # Test consistency of COPY that inserts more to the same table using
+ # triggers. If the INSERTS from the trigger go to the same block data
+ # is copied to, and the INSERTs are WAL-logged, WAL replay will fail when
+ # it tries to replay the WAL record but the "before" image doesn't match,
+ # because not all changes were WAL-logged.
+ $node->safe_psql(
+ 'postgres', "
+ BEGIN;
+ CREATE TABLE ins_trig (id serial PRIMARY KEY, id2 text);
+ CREATE FUNCTION ins_trig_before_row_trig() RETURNS trigger
+ LANGUAGE plpgsql as \$\$
+ BEGIN
+ IF new.id2 NOT LIKE 'triggered%' THEN
+ INSERT INTO ins_trig
+ VALUES (DEFAULT, 'triggered row before' || NEW.id2);
+ END IF;
+ RETURN NEW;
+ END; \$\$;
+ CREATE FUNCTION ins_trig_after_row_trig() RETURNS trigger
+ LANGUAGE plpgsql as \$\$
+ BEGIN
+ IF new.id2 NOT LIKE 'triggered%' THEN
+ INSERT INTO ins_trig
+ VALUES (DEFAULT, 'triggered row after' || OLD.id2);
+ END IF;
+ RETURN NEW;
+ END; \$\$;
+ CREATE TRIGGER ins_trig_before_row_insert
+ BEFORE INSERT ON ins_trig
+ FOR EACH ROW EXECUTE PROCEDURE ins_trig_before_row_trig();
+ CREATE TRIGGER ins_trig_after_row_insert
+ AFTER INSERT ON ins_trig
+ FOR EACH ROW EXECUTE PROCEDURE ins_trig_after_row_trig();
+ COPY ins_trig FROM '$copy_file' DELIMITER ',';
+ COMMIT;");
+ $node->stop('immediate');
+ $node->start;
+ $result = $node->safe_psql('postgres', "SELECT count(*) FROM ins_trig;");
+ is($result, qq(9), "wal_level = $wal_level, COPY with INSERT triggers");
+
+ # Test consistency of INSERT, COPY and TRUNCATE in same transaction block
+ # with TRUNCATE triggers.
+ $node->safe_psql(
+ 'postgres', "
+ BEGIN;
+ CREATE TABLE trunc_trig (id serial PRIMARY KEY, id2 text);
+ CREATE FUNCTION trunc_trig_before_stat_trig() RETURNS trigger
+ LANGUAGE plpgsql as \$\$
+ BEGIN
+ INSERT INTO trunc_trig VALUES (DEFAULT, 'triggered stat before');
+ RETURN NULL;
+ END; \$\$;
+ CREATE FUNCTION trunc_trig_after_stat_trig() RETURNS trigger
+ LANGUAGE plpgsql as \$\$
+ BEGIN
+ INSERT INTO trunc_trig VALUES (DEFAULT, 'triggered stat before');
+ RETURN NULL;
+ END; \$\$;
+ CREATE TRIGGER trunc_trig_before_stat_truncate
+ BEFORE TRUNCATE ON trunc_trig
+ FOR EACH STATEMENT EXECUTE PROCEDURE trunc_trig_before_stat_trig();
+ CREATE TRIGGER trunc_trig_after_stat_truncate
+ AFTER TRUNCATE ON trunc_trig
+ FOR EACH STATEMENT EXECUTE PROCEDURE trunc_trig_after_stat_trig();
+ INSERT INTO trunc_trig VALUES (DEFAULT, 1);
+ TRUNCATE trunc_trig;
+ COPY trunc_trig FROM '$copy_file' DELIMITER ',';
+ COMMIT;");
+ $node->stop('immediate');
+ $node->start;
+ $result =
+ $node->safe_psql('postgres', "SELECT count(*) FROM trunc_trig;");
+ is($result, qq(4),
+ "wal_level = $wal_level, TRUNCATE COPY with TRUNCATE triggers");
+
+ # Test redo of temp table creation.
+ $node->safe_psql(
+ 'postgres', "
+ CREATE TEMP TABLE temp (id serial PRIMARY KEY, id2 text);");
+ $node->stop('immediate');
+ $node->start;
+ check_orphan_relfilenodes($node,
+ "wal_level = $wal_level, no orphan relfilenode remains");
+
+ return;
+}
+
+# Run same test suite for multiple wal_level values.
+run_wal_optimize("minimal");
+run_wal_optimize("replica");
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index b492c60..3ac009f 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -1982,6 +1982,12 @@ select * from another;
(3 rows)
drop table another;
+-- Create an index that skips WAL, then perform a SET DATA TYPE that skips
+-- rewriting the index.
+begin;
+create table skip_wal_skip_rewrite_index (c varchar(10) primary key);
+alter table skip_wal_skip_rewrite_index alter c type varchar(20);
+commit;
-- table's row type
create table tab1 (a int, b text);
create table tab2 (x int, y tab1);
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index abe7be3..0420fa4 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -1358,6 +1358,13 @@ select * from another;
drop table another;
+-- Create an index that skips WAL, then perform a SET DATA TYPE that skips
+-- rewriting the index.
+begin;
+create table skip_wal_skip_rewrite_index (c varchar(10) primary key);
+alter table skip_wal_skip_rewrite_index alter c type varchar(20);
+commit;
+
-- table's row type
create table tab1 (a int, b text);
create table tab2 (x int, y tab1);