Re: [HACKERS] WAL logging problem in 9.4.3?

Поиск
Список
Период
Сортировка
От Kyotaro Horiguchi
Тема Re: [HACKERS] WAL logging problem in 9.4.3?
Дата
Msg-id 20190821.163238.176512239.horikyota.ntt@gmail.com
обсуждение исходный текст
Ответ на Re: [HACKERS] WAL logging problem in 9.4.3?  (Kyotaro Horiguchi <horikyota.ntt@gmail.com>)
Ответы Re: [HACKERS] WAL logging problem in 9.4.3?  (Noah Misch <noah@leadboat.com>)
Список pgsql-hackers
Hello. New version is attached.

At Mon, 19 Aug 2019 18:59:59 +0900 (Tokyo Standard Time), Kyotaro Horiguchi <horikyota.ntt@gmail.com> wrote in
<20190819.185959.118543656.horikyota.ntt@gmail.com>
> Thank you for taking time.
> 
> At Sat, 17 Aug 2019 20:52:30 -0700, Noah Misch <noah@leadboat.com> wrote in
<20190818035230.GB3021338@rfd.leadboat.com>
> > For two-phase commit, PrepareTransaction() needs to execute pending syncs.
 
Now TwoPhaseFileHeader has two new members for pending syncs. It
is useless on wal-replay, but that is needed for commit-prepared.

> > On Thu, Jul 25, 2019 at 10:39:36AM +0900, Kyotaro Horiguchi wrote:
> > > --- a/src/backend/access/heap/heapam_handler.c
> > > +++ b/src/backend/access/heap/heapam_handler.c
> > > @@ -715,12 +702,6 @@ heapam_relation_copy_for_cluster(Relation OldHeap, Relation NewHeap,    
> ...
> > > -    use_wal = XLogIsNeeded() && RelationNeedsWAL(NewHeap);
> > > -
> > >      /* use_wal off requires smgr_targblock be initially invalid */
> > >      Assert(RelationGetTargetBlock(NewHeap) == InvalidBlockNumber);
> > 
> > Since you're deleting the use_wal variable, update that last comment.

Oops! Rewrote it.

> > > --- a/src/backend/catalog/storage.c
> > > +++ b/src/backend/catalog/storage.c
> > > @@ -428,21 +450,34 @@ smgrDoPendingDeletes(bool isCommit)
> ...
> > > +                    smgrimmedsync(srel, MAIN_FORKNUM);
> > 
> > This should sync all forks, not just MAIN_FORKNUM.  Code that writes WAL for
> > FSM_FORKNUM and VISIBILITYMAP_FORKNUM checks RelationNeedsWAL().  There may be
> > no bug today, but it's conceptually wrong to make RelationNeedsWAL() return
> > false due to this code, use RelationNeedsWAL() for multiple forks, and then
> > not actually sync all forks.
> 
> I agree that all forks needs syncing, but FSM and VM are checking
> RelationNeedsWAL(modified). To make sure, are you suggesting to
> sync all forks instead of emitting WAL for them, or suggesting
> that VM and FSM to emit WALs even when the modified
> RelationNeedsWAL returns false (+ sync all forks)?

All forks are synced and have no WALs emitted (as before) in the
attached version 19. FSM and VM are not changed.

> > The https://postgr.es/m/559FA0BA.3080808@iki.fi design had another component
> > not appearing here.  It said, "Instead, at COMMIT, we'd fsync() the relation,
> > or if it's smaller than some threshold, WAL-log the contents of the whole file
> > at that point."  Please write the part to WAL-log the contents of small files
> > instead of syncing them.
> 
> I'm not sure the point of the behavior. I suppose that the "log"
> is a sequence of new_page records. It also needs to be synced and
> it is always larger than the file to be synced. I can't think of
> an appropriate threshold without the point.

This is not included in this version. I'll continue to consider
this.

> > > --- a/src/backend/commands/copy.c
> > > +++ b/src/backend/commands/copy.c
> > > @@ -2725,28 +2722,9 @@ CopyFrom(CopyState cstate)
> > >       * 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 ...
> > 
> > The comment material being deleted is still correct, so don't delete it.

The code is changed to use rd_firstRelfilenodeSubid instead of
rd_firstRelfilenodeSubid which has the issue mentioned in the
deleted section. So this is right but irrelevant to the code
here. The same thing is written in the comment in RelationData.

(In short, not reverted)

> > Moreover, the code managing rd_firstRelfilenodeSubid has a similar bug.  The
> > attached patch adds an assertion that RelationNeedsWAL() and the
> > pendingDeletes array have the same opinion about the relfilenode, and it
> > expands a test case to fail that assertion.
..
> > In general, to add a field like this, run "git grep -n 'rd_.*Subid'" and audit
> > all the lines printed.  Many bits of code need to look at all three,
> > e.g. RelationClose().

I forgot to maintain rd_firstRelfilenode in many places and the
assertion failure no longer happens after I fixed it. Opposite to
my previous mail, of course useless pending entries are removed
at subtransction abort and no needless syncs happen in that
meaning. But another type of useless sync was seen with the
previous version 18.

(In short fixed.)


> >  This field needs to be 100% reliable.  In other words,
> > it must equal InvalidSubTransactionId if and only if the relfilenode matches
> > the relfilenode that would be in place if the top transaction rolled back.

Sorry, I confused this with another similar behavior of the
previous version 18, where files are synced even if it is to be
removed immediately at commit. In this version
smgrDoPendingOperations doesn't sync to-be-deleted files.

While checking this, I found that smgrDoPendingDeletes is making
unnecessary call to smgrclose() which lead server to crash while
deleting files. I removed it.


Please find the new version attached.

Changes:

- Rebased to f8cf524da1.

- Fixed prepare transaction. test2a catches this.
  (twophase.c)

- Fixed a comment in heapam_relation_copy_for_cluster.

- All forks are synced. (smgrDoPendingDeletes/Operations, SyncRelationFiles)

- Fixed handling of rd_firstRelfilenodeSubid.
  (RelationBuildLocalRelation, RelationSetNewRelfilenode,
   load_relcache_init_file) 

- Prevent to-be-deleted files from syncing. (smgrDoPendingDeletes/Operations)

- Fixed a crash bug caused by smgrclose() in smgrDoPendingOperations.

Minor changes:

- Renamed: PendingRelOps => PendingRelOp
- Type changed: bool PendingRelOp.dosync => PendingOpType PendingRelOp.op

regards.

-- 
Kyotaro Horiguchi
NTT Open Source Software Center
From b4144d7e1f1fb22f4387e3af9d37a29b68c9795f Mon Sep 17 00:00:00 2001
From: Kyotaro Horiguchi <horiguchi.kyotaro@lab.ntt.co.jp>
Date: Thu, 11 Oct 2018 10:03:21 +0900
Subject: [PATCH 1/3] TAP test for copy-truncation optimization.

---
 src/test/recovery/t/018_wal_optimize.pl | 312 ++++++++++++++++++++++++++++++++
 1 file changed, 312 insertions(+)
 create mode 100644 src/test/recovery/t/018_wal_optimize.pl

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 0000000000..b041121745
--- /dev/null
+++ b/src/test/recovery/t/018_wal_optimize.pl
@@ -0,0 +1,312 @@
+# Test WAL replay for optimized TRUNCATE and COPY records
+#
+# WAL truncation is optimized in some cases with TRUNCATE and COPY queries
+# which sometimes interact badly with the other optimizations in line with
+# several setting values of wal_level, particularly when using "minimal" or
+# "replica".  The optimization may be enabled or disabled depending on the
+# scenarios dealt here, and should never result in any type of failures or
+# data loss.
+use strict;
+use warnings;
+
+use PostgresNode;
+use TestLib;
+use Test::More tests => 26;
+
+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;
+}
+
+# Wrapper routine tunable for wal_level.
+sub run_wal_optimize
+{
+    my $wal_level = shift;
+
+    # Primary needs to have wal_level = minimal here
+    my $node = get_new_node("node_$wal_level");
+    $node->init;
+    $node->append_conf('postgresql.conf', qq(
+wal_level = $wal_level
+max_prepared_transactions = 1
+));
+    $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 test1 (id serial PRIMARY KEY);
+        TRUNCATE test1;
+        COMMIT;");
+
+    $node->stop('immediate');
+    $node->start;
+
+    my $result = $node->safe_psql('postgres', "SELECT count(*) FROM test1;");
+    is($result, qq(0),
+       "wal_level = $wal_level, optimized truncation 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 test2 (id serial PRIMARY KEY);
+        INSERT INTO test2 VALUES (DEFAULT);
+        TRUNCATE test2;
+        INSERT INTO test2 VALUES (DEFAULT);
+        COMMIT;");
+
+    $node->stop('immediate');
+    $node->start;
+
+    $result = $node->safe_psql('postgres', "SELECT count(*) FROM test2;");
+    is($result, qq(1),
+       "wal_level = $wal_level, optimized truncation with inserted table");
+
+
+    # Same for prepared transaction
+    # Tuples inserted after the truncation should be seen.
+    $node->safe_psql('postgres', "
+        BEGIN;
+        CREATE TABLE test2a (id serial PRIMARY KEY);
+        INSERT INTO test2a VALUES (DEFAULT);
+        TRUNCATE test2a;
+        INSERT INTO test2a VALUES (DEFAULT);
+        PREPARE TRANSACTION 't';
+        COMMIT PREPARED 't';");
+
+    $node->stop('immediate');
+    $node->start;
+
+    $result = $node->safe_psql('postgres', "SELECT count(*) FROM test2a;");
+    is($result, qq(1),
+       "wal_level = $wal_level, optimized truncation with prepared transaction");
+
+
+    # Data file for COPY query in follow-up 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 COPY.  Tuples copied after the
+    # truncation should be seen.
+    $node->safe_psql('postgres', "
+        BEGIN;
+        CREATE TABLE test3 (id serial PRIMARY KEY, id2 int);
+        INSERT INTO test3 (id, id2) VALUES (DEFAULT, generate_series(1,3000));
+        TRUNCATE test3;
+        COPY test3 FROM '$copy_file' DELIMITER ',';
+        COMMIT;");
+    $node->stop('immediate');
+    $node->start;
+    $result = $node->safe_psql('postgres', "SELECT count(*) FROM test3;");
+    is($result, qq(3),
+       "wal_level = $wal_level, optimized truncation with copied table");
+
+    # Like previous test, but rollback SET TABLESPACE in a subtransaction.
+    $node->safe_psql('postgres', "
+        BEGIN;
+        CREATE TABLE test3a (id serial PRIMARY KEY, id2 int);
+        INSERT INTO test3a (id, id2) VALUES (DEFAULT, generate_series(1,3000));
+        TRUNCATE test3a;
+        SAVEPOINT s; ALTER TABLE test3a SET TABLESPACE other; ROLLBACK TO s;
+        COPY test3a FROM '$copy_file' DELIMITER ',';
+        COMMIT;");
+    $node->stop('immediate');
+    $node->start;
+    $result = $node->safe_psql('postgres', "SELECT count(*) FROM test3a;");
+    is($result, qq(3),
+       "wal_level = $wal_level, SET TABLESPACE in subtransaction");
+
+    # in different subtransaction patterns
+    $node->safe_psql('postgres', "
+        BEGIN;
+        CREATE TABLE test3a2 (id serial PRIMARY KEY, id2 int);
+        INSERT INTO test3a2 (id, id2) VALUES (DEFAULT, generate_series(1,3000));
+        TRUNCATE test3a2;
+        SAVEPOINT s; ALTER TABLE test3a SET TABLESPACE other; RELEASE s;
+        COPY test3a2 FROM '$copy_file' DELIMITER ',';
+        COMMIT;");
+    $node->stop('immediate');
+    $node->start;
+    $result = $node->safe_psql('postgres', "SELECT count(*) FROM test3a;");
+    is($result, qq(3),
+       "wal_level = $wal_level, SET TABLESPACE in subtransaction");
+
+    $node->safe_psql('postgres', "
+        BEGIN;
+        CREATE TABLE test3a3 (id serial PRIMARY KEY, id2 int);
+        INSERT INTO test3a3 (id, id2) VALUES (DEFAULT, generate_series(1,3000));
+        TRUNCATE test3a3;
+        SAVEPOINT s;
+            ALTER TABLE test3a3 SET TABLESPACE other;
+            SAVEPOINT s2;
+                ALTER TABLE test3a3 SET TABLESPACE pg_default;
+            ROLLBACK TO s2;
+            SAVEPOINT s2;
+                ALTER TABLE test3a3 SET TABLESPACE pg_default;
+            RELEASE s2;
+        ROLLBACK TO s;
+        COPY test3a3 FROM '$copy_file' DELIMITER ',';
+        COMMIT;");
+    $node->stop('immediate');
+    $node->start;
+    $result = $node->safe_psql('postgres', "SELECT count(*) FROM test3a;");
+    is($result, qq(3),
+       "wal_level = $wal_level, SET TABLESPACE in subtransaction");
+
+    # UPDATE touches two buffers; one is BufferNeedsWAL(); the other is not.
+    $node->safe_psql('postgres', "
+        BEGIN;
+        CREATE TABLE test3b (id serial PRIMARY KEY, id2 int);
+        INSERT INTO test3b (id, id2) VALUES (DEFAULT, generate_series(1,10000));
+        COPY test3b FROM '$copy_file' DELIMITER ',';  -- set sync_above
+        UPDATE test3b SET id2 = id2 + 1;
+        DELETE FROM test3b;
+        COMMIT;");
+    $node->stop('immediate');
+    $node->start;
+    $result = $node->safe_psql('postgres', "SELECT count(*) FROM test3b;");
+    is($result, qq(0),
+       "wal_level = $wal_level, UPDATE of logged page extends relation");
+
+    # 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 test4 (id serial PRIMARY KEY, id2 int);
+        INSERT INTO test4 (id, id2) VALUES (DEFAULT, generate_series(1,10000));
+        TRUNCATE test4;
+        INSERT INTO test4 (id, id2) VALUES (DEFAULT, 10000);
+        COPY test4 FROM '$copy_file' DELIMITER ',';
+        INSERT INTO test4 (id, id2) VALUES (DEFAULT, 10000);
+        COMMIT;");
+
+    $node->stop('immediate');
+    $node->start;
+    $result = $node->safe_psql('postgres', "SELECT count(*) FROM test4;");
+    is($result, qq(5),
+       "wal_level = $wal_level, optimized truncation with inserted/copied table");
+
+    # Test consistency of COPY with INSERT for table created in the same
+    # transaction.
+    $node->safe_psql('postgres', "
+        BEGIN;
+        CREATE TABLE test5 (id serial PRIMARY KEY, id2 int);
+        INSERT INTO test5 VALUES (DEFAULT, 1);
+        COPY test5 FROM '$copy_file' DELIMITER ',';
+        COMMIT;");
+    $node->stop('immediate');
+    $node->start;
+    $result = $node->safe_psql('postgres', "SELECT count(*) FROM test5;");
+    is($result, qq(4),
+       "wal_level = $wal_level, replay of optimized copy with inserted table");
+
+    # 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 test6 (id serial PRIMARY KEY, id2 text);
+        CREATE FUNCTION test6_before_row_trig() RETURNS trigger
+          LANGUAGE plpgsql as \$\$
+          BEGIN
+            IF new.id2 NOT LIKE 'triggered%' THEN
+              INSERT INTO test6 VALUES (DEFAULT, 'triggered row before' || NEW.id2);
+            END IF;
+            RETURN NEW;
+          END; \$\$;
+        CREATE FUNCTION test6_after_row_trig() RETURNS trigger
+          LANGUAGE plpgsql as \$\$
+          BEGIN
+            IF new.id2 NOT LIKE 'triggered%' THEN
+              INSERT INTO test6 VALUES (DEFAULT, 'triggered row after' || OLD.id2);
+            END IF;
+            RETURN NEW;
+          END; \$\$;
+        CREATE TRIGGER test6_before_row_insert
+          BEFORE INSERT ON test6
+          FOR EACH ROW EXECUTE PROCEDURE test6_before_row_trig();
+        CREATE TRIGGER test6_after_row_insert
+          AFTER INSERT ON test6
+          FOR EACH ROW EXECUTE PROCEDURE test6_after_row_trig();
+        COPY test6 FROM '$copy_file' DELIMITER ',';
+        COMMIT;");
+    $node->stop('immediate');
+    $node->start;
+    $result = $node->safe_psql('postgres', "SELECT count(*) FROM test6;");
+    is($result, qq(9),
+       "wal_level = $wal_level, replay of optimized copy with before trigger");
+
+    # Test consistency of INSERT, COPY and TRUNCATE in same transaction block
+    # with TRUNCATE triggers.
+    $node->safe_psql('postgres', "
+        BEGIN;
+        CREATE TABLE test7 (id serial PRIMARY KEY, id2 text);
+        CREATE FUNCTION test7_before_stat_trig() RETURNS trigger
+          LANGUAGE plpgsql as \$\$
+          BEGIN
+            INSERT INTO test7 VALUES (DEFAULT, 'triggered stat before');
+            RETURN NULL;
+          END; \$\$;
+        CREATE FUNCTION test7_after_stat_trig() RETURNS trigger
+          LANGUAGE plpgsql as \$\$
+          BEGIN
+            INSERT INTO test7 VALUES (DEFAULT, 'triggered stat before');
+            RETURN NULL;
+          END; \$\$;
+        CREATE TRIGGER test7_before_stat_truncate
+          BEFORE TRUNCATE ON test7
+          FOR EACH STATEMENT EXECUTE PROCEDURE test7_before_stat_trig();
+        CREATE TRIGGER test7_after_stat_truncate
+          AFTER TRUNCATE ON test7
+          FOR EACH STATEMENT EXECUTE PROCEDURE test7_after_stat_trig();
+        INSERT INTO test7 VALUES (DEFAULT, 1);
+        TRUNCATE test7;
+        COPY test7 FROM '$copy_file' DELIMITER ',';
+        COMMIT;");
+    $node->stop('immediate');
+    $node->start;
+    $result = $node->safe_psql('postgres', "SELECT count(*) FROM test7;");
+    is($result, qq(4),
+       "wal_level = $wal_level, replay of optimized copy with before trigger");
+
+    # Test redo of temp table creation.
+    $node->safe_psql('postgres', "
+        CREATE TEMP TABLE test8 (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");
-- 
2.16.3

From d62a337281024c1f9df09596e62724057b02cdfb Mon Sep 17 00:00:00 2001
From: Kyotaro Horiguchi <horiguchi.kyotaro@lab.ntt.co.jp>
Date: Wed, 21 Aug 2019 13:57:00 +0900
Subject: [PATCH 2/3] Fix WAL skipping feature

WAL-skipping operations mixed with WAL-logged operations can lead to
database corruption after a crash. This patch changes the WAL-skipping
feature so that no data modification is WAL-logged at all then sync
such relations at commit.
---
 src/backend/access/heap/heapam.c         |   4 +-
 src/backend/access/heap/heapam_handler.c |  22 +----
 src/backend/access/heap/rewriteheap.c    |  13 +--
 src/backend/access/transam/twophase.c    |  23 ++++-
 src/backend/catalog/storage.c            | 158 ++++++++++++++++++++++++++-----
 src/backend/commands/cluster.c           |  24 +++++
 src/backend/commands/copy.c              |  39 ++------
 src/backend/commands/createas.c          |   5 +-
 src/backend/commands/matview.c           |   4 -
 src/backend/commands/tablecmds.c         |  10 +-
 src/backend/storage/buffer/bufmgr.c      |  33 +++++--
 src/backend/storage/smgr/md.c            |  30 ++++++
 src/backend/utils/cache/relcache.c       |  28 ++++--
 src/include/access/heapam.h              |   1 -
 src/include/access/rewriteheap.h         |   2 +-
 src/include/access/tableam.h             |  40 +-------
 src/include/catalog/storage.h            |   8 ++
 src/include/storage/bufmgr.h             |   1 +
 src/include/storage/md.h                 |   1 +
 src/include/utils/rel.h                  |  17 +++-
 20 files changed, 300 insertions(+), 163 deletions(-)

diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index cb811d345a..ef18b61c55 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1936,7 +1936,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 +2119,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);
 
diff --git a/src/backend/access/heap/heapam_handler.c b/src/backend/access/heap/heapam_handler.c
index f1ff01e8cb..27f414a361 100644
--- a/src/backend/access/heap/heapam_handler.c
+++ b/src/backend/access/heap/heapam_handler.c
@@ -558,18 +558,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.
  * ------------------------------------------------------------------------
@@ -701,7 +689,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);
@@ -716,12 +703,8 @@ 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.
+     * smgr_targblock must be initially invalid if we are to skip WAL logging
      */
-    use_wal = XLogIsNeeded() && RelationNeedsWAL(NewHeap);
-
-    /* use_wal off requires smgr_targblock be initially invalid */
     Assert(RelationGetTargetBlock(NewHeap) == InvalidBlockNumber);
 
     /* Preallocate values/isnull arrays */
@@ -731,7 +714,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 */
@@ -2519,7 +2502,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 a17508a82f..9e0d7295af 100644
--- a/src/backend/access/heap/rewriteheap.c
+++ b/src/backend/access/heap/rewriteheap.c
@@ -144,7 +144,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 */
@@ -238,15 +237,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;
@@ -271,7 +268,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;
@@ -330,7 +326,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,
@@ -654,9 +650,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
@@ -695,7 +688,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/transam/twophase.c b/src/backend/access/transam/twophase.c
index 477709bbc2..e3512fc415 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -921,6 +921,7 @@ typedef struct TwoPhaseFileHeader
     Oid            owner;            /* user running the transaction */
     int32        nsubxacts;        /* number of following subxact XIDs */
     int32        ncommitrels;    /* number of delete-on-commit rels */
+    int32        npendsyncrels;    /* number of sync-on-commit rels */
     int32        nabortrels;        /* number of delete-on-abort rels */
     int32        ninvalmsgs;        /* number of cache invalidation messages */
     bool        initfileinval;    /* does relcache init file need invalidation? */
@@ -1009,6 +1010,7 @@ StartPrepare(GlobalTransaction gxact)
     TwoPhaseFileHeader hdr;
     TransactionId *children;
     RelFileNode *commitrels;
+    RelFileNode *pendsyncrels;
     RelFileNode *abortrels;
     SharedInvalidationMessage *invalmsgs;
 
@@ -1034,6 +1036,7 @@ StartPrepare(GlobalTransaction gxact)
     hdr.owner = gxact->owner;
     hdr.nsubxacts = xactGetCommittedChildren(&children);
     hdr.ncommitrels = smgrGetPendingDeletes(true, &commitrels);
+    hdr.npendsyncrels = smgrGetPendingSyncs(true, &pendsyncrels);
     hdr.nabortrels = smgrGetPendingDeletes(false, &abortrels);
     hdr.ninvalmsgs = xactGetCommittedInvalidationMessages(&invalmsgs,
                                                           &hdr.initfileinval);
@@ -1057,6 +1060,11 @@ StartPrepare(GlobalTransaction gxact)
         save_state_data(commitrels, hdr.ncommitrels * sizeof(RelFileNode));
         pfree(commitrels);
     }
+    if (hdr.npendsyncrels > 0)
+    {
+        save_state_data(pendsyncrels, hdr.npendsyncrels * sizeof(RelFileNode));
+        pfree(pendsyncrels);
+    }
     if (hdr.nabortrels > 0)
     {
         save_state_data(abortrels, hdr.nabortrels * sizeof(RelFileNode));
@@ -1464,6 +1472,7 @@ FinishPreparedTransaction(const char *gid, bool isCommit)
     TransactionId latestXid;
     TransactionId *children;
     RelFileNode *commitrels;
+    RelFileNode *pendsyncrels;
     RelFileNode *abortrels;
     RelFileNode *delrels;
     int            ndelrels;
@@ -1499,6 +1508,8 @@ FinishPreparedTransaction(const char *gid, bool isCommit)
     children = (TransactionId *) bufptr;
     bufptr += MAXALIGN(hdr->nsubxacts * sizeof(TransactionId));
     commitrels = (RelFileNode *) bufptr;
+    bufptr += MAXALIGN(hdr->npendsyncrels * sizeof(RelFileNode));
+    pendsyncrels = (RelFileNode *) bufptr;
     bufptr += MAXALIGN(hdr->ncommitrels * sizeof(RelFileNode));
     abortrels = (RelFileNode *) bufptr;
     bufptr += MAXALIGN(hdr->nabortrels * sizeof(RelFileNode));
@@ -1544,9 +1555,9 @@ FinishPreparedTransaction(const char *gid, bool isCommit)
     gxact->valid = false;
 
     /*
-     * We have to remove any files that were supposed to be dropped. For
-     * consistency with the regular xact.c code paths, must do this before
-     * releasing locks, so do it before running the callbacks.
+     * We have to sync or remove any files that were supposed to be done
+     * so. For consistency with the regular xact.c code paths, must do this
+     * before releasing locks, so do it before running the callbacks.
      *
      * NB: this code knows that we couldn't be dropping any temp rels ...
      */
@@ -1554,11 +1565,17 @@ FinishPreparedTransaction(const char *gid, bool isCommit)
     {
         delrels = commitrels;
         ndelrels = hdr->ncommitrels;
+
+        /* Make sure files supposed to be synced are synced */
+        SyncRelationFiles(pendsyncrels, hdr->npendsyncrels);
     }
     else
     {
         delrels = abortrels;
         ndelrels = hdr->nabortrels;
+
+        /* We don't have an at-abort pending sync */
+        Assert(pendsyncrels == 0);
     }
 
     /* Make sure files supposed to be dropped are dropped */
diff --git a/src/backend/catalog/storage.c b/src/backend/catalog/storage.c
index 3cc886f7fe..354a74c27c 100644
--- a/src/backend/catalog/storage.c
+++ b/src/backend/catalog/storage.c
@@ -30,6 +30,7 @@
 #include "catalog/storage_xlog.h"
 #include "storage/freespace.h"
 #include "storage/smgr.h"
+#include "utils/hsearch.h"
 #include "utils/memutils.h"
 #include "utils/rel.h"
 
@@ -53,11 +54,13 @@
  * but I'm being paranoid.
  */
 
+/* entry type of pendingDeletes */
 typedef struct PendingRelDelete
 {
     RelFileNode relnode;        /* relation that may need to be deleted */
     BackendId    backend;        /* InvalidBackendId if not a temp rel */
-    bool        atCommit;        /* T=delete at commit; F=delete at abort */
+    bool        atCommit;        /* T=work at commit; F=work at abort */
+    PendingOpType    op;            /* type of operation to do */
     int            nestLevel;        /* xact nesting level of request */
     struct PendingRelDelete *next;    /* linked-list link */
 } PendingRelDelete;
@@ -114,10 +117,29 @@ RelationCreateStorage(RelFileNode rnode, char relpersistence)
     pending->relnode = rnode;
     pending->backend = backend;
     pending->atCommit = false;    /* delete if abort */
+    pending->op = PENDING_DELETE;
     pending->nestLevel = GetCurrentTransactionNestLevel();
     pending->next = pendingDeletes;
     pendingDeletes = pending;
 
+    /*
+     * We are going to skip WAL-logging for storage of persistent relations
+     * created in the current transaction when wal_level = minimal. The
+     * relation needs to be synced at commit.
+     */
+    if (relpersistence == RELPERSISTENCE_PERMANENT && !XLogIsNeeded())
+    {
+        pending = (PendingRelDelete *)
+            MemoryContextAlloc(TopMemoryContext, sizeof(PendingRelDelete));
+        pending->relnode = rnode;
+        pending->backend = backend;
+        pending->atCommit = true;
+        pending->op = PENDING_SYNC;
+        pending->nestLevel = GetCurrentTransactionNestLevel();
+        pending->next = pendingDeletes;
+        pendingDeletes = pending;
+    }
+
     return srel;
 }
 
@@ -155,6 +177,7 @@ RelationDropStorage(Relation rel)
     pending->relnode = rel->rd_node;
     pending->backend = rel->rd_backend;
     pending->atCommit = true;    /* delete if commit */
+    pending->op = PENDING_DELETE;
     pending->nestLevel = GetCurrentTransactionNestLevel();
     pending->next = pendingDeletes;
     pendingDeletes = pending;
@@ -201,7 +224,8 @@ RelationPreserveStorage(RelFileNode rnode, bool atCommit)
     {
         next = pending->next;
         if (RelFileNodeEquals(rnode, pending->relnode)
-            && pending->atCommit == atCommit)
+            && pending->atCommit == atCommit
+            && pending->op == PENDING_DELETE)
         {
             /* unlink and delete list entry */
             if (prev)
@@ -406,6 +430,7 @@ smgrDoPendingDeletes(bool isCommit)
                 i = 0,
                 maxrels = 0;
     SMgrRelation *srels = NULL;
+    struct HTAB *synchash = NULL;
 
     prev = NULL;
     for (pending = pendingDeletes; pending != NULL; pending = next)
@@ -428,21 +453,50 @@ smgrDoPendingDeletes(bool isCommit)
             {
                 SMgrRelation srel;
 
-                srel = smgropen(pending->relnode, pending->backend);
-
-                /* allocate the initial array, or extend it, if needed */
-                if (maxrels == 0)
+                if (pending->op == PENDING_SYNC)
                 {
-                    maxrels = 8;
-                    srels = palloc(sizeof(SMgrRelation) * maxrels);
-                }
-                else if (maxrels <= nrels)
-                {
-                    maxrels *= 2;
-                    srels = repalloc(srels, sizeof(SMgrRelation) * maxrels);
-                }
+                    /* We don't have abort-time pending syncs */
+                    Assert(isCommit);
 
-                srels[nrels++] = srel;
+                    /* Create hash if not yet */
+                    if (synchash == NULL)
+                    {
+                        HASHCTL hash_ctl;
+
+                        memset(&hash_ctl, 0, sizeof(hash_ctl));
+                        hash_ctl.keysize = sizeof(SMgrRelation*);
+                        hash_ctl.entrysize = sizeof(SMgrRelation*);
+                        hash_ctl.hcxt = CurrentMemoryContext;
+                        synchash =
+                            hash_create("pending sync hash", 8,
+                                        &hash_ctl,
+                                        HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
+                    }
+
+                    /* Collect pending syncs */
+                    srel = smgropen(pending->relnode, pending->backend);
+                    (void) hash_search(synchash, (void *) &srel,
+                                       HASH_ENTER, NULL);
+                }
+                else
+                {
+                    /* Collect pending deletions */
+                    srel = smgropen(pending->relnode, pending->backend);
+
+                    /* 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;
+                }
             }
             /* must explicitly free the list entry */
             pfree(pending);
@@ -450,6 +504,43 @@ smgrDoPendingDeletes(bool isCommit)
         }
     }
 
+    /* Sync only files that are not to be removed. */
+    if (synchash)
+    {
+        HASH_SEQ_STATUS hstat;
+        SMgrRelation *psrel;
+
+        /* remove to-be-removed files from synchash */
+        if (nrels > 0)
+        {
+            int i;
+            bool found;
+
+            for (i = 0 ; i < nrels ; i++)
+                (void) hash_search(synchash, (void *) &(srels[i]),
+                                   HASH_REMOVE, &found);
+        }
+
+        /* sync survuvied files */
+        hash_seq_init(&hstat, synchash);
+        while ((psrel = (SMgrRelation *) hash_seq_search(&hstat)) != NULL)
+        {
+            ForkNumber fork;
+
+            /* Perform pending sync of WAL-skipped relation */
+            FlushRelationBuffersWithoutRelcache((*psrel)->smgr_rnode.node,
+                                                false);
+            for (fork = 0; fork <= MAX_FORKNUM; fork++)
+            {
+                if (smgrexists(*psrel, fork))
+                    smgrimmedsync(*psrel, fork);
+            }
+        }
+
+        hash_destroy(synchash);
+        synchash = NULL;
+    }
+
     if (nrels > 0)
     {
         smgrdounlinkall(srels, nrels, false);
@@ -462,11 +553,12 @@ smgrDoPendingDeletes(bool isCommit)
 }
 
 /*
- * smgrGetPendingDeletes() -- Get a list of non-temp relations to be deleted.
+ * smgrGetPendingOperations() -- Get a list of non-temp relations to be
+ *                                 deleted or synced.
  *
- * The return value is the number of relations scheduled for termination.
- * *ptr is set to point to a freshly-palloc'd array of RelFileNodes.
- * If there are no relations to be deleted, *ptr is set to NULL.
+ * The return value is the number of relations scheduled for the operation
+ * specified by op. *ptr is set to point to a freshly-palloc'd array of
+ * RelFileNodes.  If there are no matching relations, *ptr is set to NULL.
  *
  * Only non-temporary relations are included in the returned list.  This is OK
  * because the list is used only in contexts where temporary relations don't
@@ -475,11 +567,11 @@ smgrDoPendingDeletes(bool isCommit)
  * (and all temporary files will be zapped if we restart anyway, so no need
  * for redo to do it also).
  *
- * Note that the list does not include anything scheduled for termination
- * by upper-level transactions.
+ * Note that the list does not include anything scheduled by upper-level
+ * transactions.
  */
-int
-smgrGetPendingDeletes(bool forCommit, RelFileNode **ptr)
+static inline int
+smgrGetPendingOperations(PendingOpType op, bool forCommit, RelFileNode **ptr)
 {
     int            nestLevel = GetCurrentTransactionNestLevel();
     int            nrels;
@@ -490,7 +582,8 @@ smgrGetPendingDeletes(bool forCommit, RelFileNode **ptr)
     for (pending = pendingDeletes; pending != NULL; pending = pending->next)
     {
         if (pending->nestLevel >= nestLevel && pending->atCommit == forCommit
-            && pending->backend == InvalidBackendId)
+            && pending->backend == InvalidBackendId
+            && pending->op == op)
             nrels++;
     }
     if (nrels == 0)
@@ -503,7 +596,8 @@ smgrGetPendingDeletes(bool forCommit, RelFileNode **ptr)
     for (pending = pendingDeletes; pending != NULL; pending = pending->next)
     {
         if (pending->nestLevel >= nestLevel && pending->atCommit == forCommit
-            && pending->backend == InvalidBackendId)
+            && pending->backend == InvalidBackendId
+            && pending->op == op)
         {
             *rptr = pending->relnode;
             rptr++;
@@ -512,6 +606,20 @@ smgrGetPendingDeletes(bool forCommit, RelFileNode **ptr)
     return nrels;
 }
 
+/* Returns list of pending deletes, see smgrGetPendingOperations for details */
+int
+smgrGetPendingDeletes(bool forCommit, RelFileNode **ptr)
+{
+    return smgrGetPendingOperations(PENDING_DELETE, forCommit, ptr);
+}
+
+/* Returns list of pending syncs, see smgrGetPendingOperations for details */
+int
+smgrGetPendingSyncs(bool forCommit, RelFileNode **ptr)
+{
+    return smgrGetPendingOperations(PENDING_SYNC, forCommit, ptr);
+}
+
 /*
  *    PostPrepare_smgr -- Clean up after a successful PREPARE
  *
diff --git a/src/backend/commands/cluster.c b/src/backend/commands/cluster.c
index 28985a07ec..f665ee8358 100644
--- a/src/backend/commands/cluster.c
+++ b/src/backend/commands/cluster.c
@@ -1034,12 +1034,36 @@ swap_relation_files(Oid r1, Oid r2, bool target_is_pg_class,
 
     if (OidIsValid(relfilenode1) && OidIsValid(relfilenode2))
     {
+        Relation rel1;
+        Relation rel2;
+
         /*
          * Normal non-mapped relations: swap relfilenodes, reltablespaces,
          * relpersistence
          */
         Assert(!target_is_pg_class);
 
+        /* Update creation subid hints of relcache */
+        rel1 = relation_open(r1, ExclusiveLock);
+        rel2 = relation_open(r2, ExclusiveLock);
+
+        /*
+         * New relation's relfilenode is created in the current transaction
+         * and used as old ralation's new relfilenode. So its
+         * newRelfilenodeSubid as new relation's createSubid. We don't fix
+         * rel2 since it would be deleted soon.
+         */
+        Assert(rel2->rd_createSubid != InvalidSubTransactionId);
+        rel1->rd_newRelfilenodeSubid = rel2->rd_createSubid;
+
+        /* record the first relfilenode change in the current transaction */
+        if (rel1->rd_firstRelfilenodeSubid == InvalidSubTransactionId)
+            rel1->rd_firstRelfilenodeSubid = GetCurrentSubTransactionId();
+
+        relation_close(rel1, ExclusiveLock);
+        relation_close(rel2, ExclusiveLock);
+
+        /* swap relfilenodes, reltablespaces, relpersistence */
         swaptemp = relform1->relfilenode;
         relform1->relfilenode = relform2->relfilenode;
         relform2->relfilenode = swaptemp;
diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index 3aeef30b28..3ce04f7efc 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -2534,9 +2534,6 @@ CopyMultiInsertBufferCleanup(CopyMultiInsertInfo *miinfo,
     for (i = 0; i < MAX_BUFFERED_TUPLES && buffer->slots[i] != NULL; i++)
         ExecDropSingleTupleTableSlot(buffer->slots[i]);
 
-    table_finish_bulk_insert(buffer->resultRelInfo->ri_RelationDesc,
-                             miinfo->ti_options);
-
     pfree(buffer);
 }
 
@@ -2725,28 +2722,9 @@ CopyFrom(CopyState cstate)
      * 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.
+     * 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.
      *
      * We currently don't support this optimization if the COPY target is a
      * partitioned table as we currently only lazily initialize partition
@@ -2762,15 +2740,14 @@ CopyFrom(CopyState cstate)
      * are not supported as per the description above.
      *----------
      */
-    /* createSubid is creation check, newRelfilenodeSubid is truncation check */
+    /*
+     * createSubid is creation check, firstRelfilenodeSubid is truncation and
+     * cluster check. Partitioned table doesn't have storage.
+     */
     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 b7d220699f..8a91d946e3 100644
--- a/src/backend/commands/createas.c
+++ b/src/backend/commands/createas.c
@@ -558,8 +558,7 @@ intorel_startup(DestReceiver *self, int operation, TupleDesc typeinfo)
      * 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 |
-        (XLogIsNeeded() ? 0 : TABLE_INSERT_SKIP_WAL);
+    myState->ti_options = TABLE_INSERT_SKIP_FSM;
     myState->bistate = GetBulkInsertState();
 
     /* Not using WAL requires smgr_targblock be initially invalid */
@@ -604,8 +603,6 @@ intorel_shutdown(DestReceiver *self)
 
     FreeBulkInsertState(myState->bistate);
 
-    table_finish_bulk_insert(myState->rel, myState->ti_options);
-
     /* close rel, but keep lock until commit */
     table_close(myState->rel, NoLock);
     myState->rel = NULL;
diff --git a/src/backend/commands/matview.c b/src/backend/commands/matview.c
index 537d0e8cef..1c854dcebf 100644
--- a/src/backend/commands/matview.c
+++ b/src/backend/commands/matview.c
@@ -463,8 +463,6 @@ transientrel_startup(DestReceiver *self, int operation, TupleDesc typeinfo)
      * 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 */
@@ -509,8 +507,6 @@ transientrel_shutdown(DestReceiver *self)
 
     FreeBulkInsertState(myState->bistate);
 
-    table_finish_bulk_insert(myState->transientrel, myState->ti_options);
-
     /* close transientrel, but keep lock until commit */
     table_close(myState->transientrel, NoLock);
     myState->transientrel = NULL;
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index cceefbdd49..2468b178cb 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -4762,9 +4762,9 @@ ATRewriteTable(AlteredTableInfo *tab, Oid OIDNewHeap, LOCKMODE lockmode)
 
     /*
      * 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.
+     * we're building a new heap, the underlying table AM can skip WAL-logging
+     * and smgr will sync the relation to disk at the end of the current
+     * transaction instead. The FSM is empty too, so don't bother using it.
      */
     if (newrel)
     {
@@ -4772,8 +4772,6 @@ ATRewriteTable(AlteredTableInfo *tab, Oid OIDNewHeap, LOCKMODE lockmode)
         bistate = GetBulkInsertState();
 
         ti_options = TABLE_INSERT_SKIP_FSM;
-        if (!XLogIsNeeded())
-            ti_options |= TABLE_INSERT_SKIP_WAL;
     }
     else
     {
@@ -5058,8 +5056,6 @@ ATRewriteTable(AlteredTableInfo *tab, Oid OIDNewHeap, LOCKMODE lockmode)
     {
         FreeBulkInsertState(bistate);
 
-        table_finish_bulk_insert(newrel, ti_options);
-
         table_close(newrel, NoLock);
     }
 }
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 6f3a402854..41ff6da9d9 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -171,6 +171,7 @@ static HTAB *PrivateRefCountHash = NULL;
 static int32 PrivateRefCountOverflowed = 0;
 static uint32 PrivateRefCountClock = 0;
 static PrivateRefCountEntry *ReservedRefCountEntry = NULL;
+static void FlushRelationBuffers_common(SMgrRelation smgr, bool islocal);
 
 static void ReservePrivateRefCountEntry(void);
 static PrivateRefCountEntry *NewPrivateRefCountEntry(Buffer buffer);
@@ -3191,20 +3192,32 @@ PrintPinnedBufs(void)
 void
 FlushRelationBuffers(Relation rel)
 {
-    int            i;
-    BufferDesc *bufHdr;
-
-    /* Open rel at the smgr level if not already done */
     RelationOpenSmgr(rel);
 
-    if (RelationUsesLocalBuffers(rel))
+    FlushRelationBuffers_common(rel->rd_smgr, RelationUsesLocalBuffers(rel));
+}
+
+void
+FlushRelationBuffersWithoutRelcache(RelFileNode rnode, bool islocal)
+{
+    FlushRelationBuffers_common(smgropen(rnode, InvalidBackendId), islocal);
+}
+
+static void
+FlushRelationBuffers_common(SMgrRelation smgr, bool islocal)
+{
+    RelFileNode rnode = smgr->smgr_rnode.node;
+    int i;
+    BufferDesc *bufHdr;
+
+    if (islocal)
     {
         for (i = 0; i < NLocBuffer; i++)
         {
             uint32        buf_state;
 
             bufHdr = GetLocalBufferDescriptor(i);
-            if (RelFileNodeEquals(bufHdr->tag.rnode, rel->rd_node) &&
+            if (RelFileNodeEquals(bufHdr->tag.rnode, rnode) &&
                 ((buf_state = pg_atomic_read_u32(&bufHdr->state)) &
                  (BM_VALID | BM_DIRTY)) == (BM_VALID | BM_DIRTY))
             {
@@ -3221,7 +3234,7 @@ FlushRelationBuffers(Relation rel)
 
                 PageSetChecksumInplace(localpage, bufHdr->tag.blockNum);
 
-                smgrwrite(rel->rd_smgr,
+                smgrwrite(smgr,
                           bufHdr->tag.forkNum,
                           bufHdr->tag.blockNum,
                           localpage,
@@ -3251,18 +3264,18 @@ FlushRelationBuffers(Relation rel)
          * As in DropRelFileNodeBuffers, an unlocked precheck should be safe
          * and saves some cycles.
          */
-        if (!RelFileNodeEquals(bufHdr->tag.rnode, rel->rd_node))
+        if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode))
             continue;
 
         ReservePrivateRefCountEntry();
 
         buf_state = LockBufHdr(bufHdr);
-        if (RelFileNodeEquals(bufHdr->tag.rnode, rel->rd_node) &&
+        if (RelFileNodeEquals(bufHdr->tag.rnode, rnode) &&
             (buf_state & (BM_VALID | BM_DIRTY)) == (BM_VALID | BM_DIRTY))
         {
             PinBuffer_Locked(bufHdr);
             LWLockAcquire(BufferDescriptorGetContentLock(bufHdr), LW_SHARED);
-            FlushBuffer(bufHdr, rel->rd_smgr);
+            FlushBuffer(bufHdr, smgr);
             LWLockRelease(BufferDescriptorGetContentLock(bufHdr));
             UnpinBuffer(bufHdr, true);
         }
diff --git a/src/backend/storage/smgr/md.c b/src/backend/storage/smgr/md.c
index 07f3c93d3f..514c6098e6 100644
--- a/src/backend/storage/smgr/md.c
+++ b/src/backend/storage/smgr/md.c
@@ -994,6 +994,36 @@ ForgetDatabaseSyncRequests(Oid dbid)
     RegisterSyncRequest(&tag, SYNC_FILTER_REQUEST, true /* retryOnError */ );
 }
 
+/*
+ * SyncRelationFiles -- sync files of all given relations
+ *
+ * This function is assumed to be called only when skipping WAL-logging and
+ * emits no xlog records.
+ */
+void
+SyncRelationFiles(RelFileNode *syncrels, int nsyncrels)
+{
+    int            i;
+
+    for (i = 0; i < nsyncrels; i++)
+    {
+        SMgrRelation srel;
+        ForkNumber    fork;
+
+        /* sync all existing forks of the relation */
+        FlushRelationBuffersWithoutRelcache(syncrels[i], false);
+        srel = smgropen(syncrels[i], InvalidBackendId);
+
+        for (fork = 0; fork <= MAX_FORKNUM; fork++)
+        {
+            if (smgrexists(srel, fork))
+                smgrimmedsync(srel, fork);
+        }
+
+        smgrclose(srel);
+    }
+}
+
 /*
  * DropRelationFiles -- drop files of all given relations
  */
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index 248860758c..147babb6b5 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -1096,6 +1096,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:
@@ -1829,6 +1830,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;
 
@@ -2094,7 +2096,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
 }
@@ -2510,8 +2512,8 @@ 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,
+         * rd_createSubid/rd_new/firstRelfilenodeSubid, 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:
@@ -2600,6 +2602,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 */
@@ -2667,7 +2670,7 @@ 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
@@ -2807,7 +2810,7 @@ RelationCacheInvalidate(void)
          * pending invalidations.
          */
         if (relation->rd_createSubid != InvalidSubTransactionId ||
-            relation->rd_newRelfilenodeSubid != InvalidSubTransactionId)
+            relation->rd_firstRelfilenodeSubid != InvalidSubTransactionId)
             continue;
 
         relcacheInvalsReceived++;
@@ -3064,6 +3067,7 @@ AtEOXact_cleanup(Relation relation, bool isCommit)
      * Likewise, reset the hint about the relfilenode being new.
      */
     relation->rd_newRelfilenodeSubid = InvalidSubTransactionId;
+    relation->rd_firstRelfilenodeSubid = InvalidSubTransactionId;
 }
 
 /*
@@ -3155,7 +3159,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 hints.
      */
     if (relation->rd_newRelfilenodeSubid == mySubid)
     {
@@ -3164,6 +3168,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;
+    }
 }
 
 
@@ -3253,6 +3265,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
@@ -3556,6 +3569,8 @@ RelationSetNewRelfilenode(Relation relation, char persistence)
      * operations on the rel in the same transaction.
      */
     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) */
     EOXactListAdd(relation);
@@ -5592,6 +5607,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/include/access/heapam.h b/src/include/access/heapam.h
index 858bcb6bc9..80c2e1bafc 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
diff --git a/src/include/access/rewriteheap.h b/src/include/access/rewriteheap.h
index 8056253916..7f9736e294 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 7f81703b78..b652cd6cef 100644
--- a/src/include/access/tableam.h
+++ b/src/include/access/tableam.h
@@ -407,22 +407,6 @@ typedef struct TableAmRoutine
                                uint8 flags,
                                TM_FailureData *tmfd);
 
-    /*
-     * 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.
-     *
-     * 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
-     * make sense for it, and then only take actions in finish_bulk_insert for
-     * those flags, and ignore others.
-     *
-     * Optional callback.
-     */
-    void        (*finish_bulk_insert) (Relation rel, int options);
-
-
     /* ------------------------------------------------------------------------
      * DDL related functionality.
      * ------------------------------------------------------------------------
@@ -1087,10 +1071,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.
@@ -1112,8 +1092,7 @@ table_compute_xid_horizon_for_tuples(Relation rel,
  * heap's TOAST table, too, if the tuple requires any out-of-line data.
  *
  * The BulkInsertState object (if any; bistate can be NULL for default
- * behavior) is also just passed through to RelationGetBufferForTuple. If
- * `bistate` is provided, table_finish_bulk_insert() needs to be called.
+ * behavior) is also just passed through to RelationGetBufferForTuple.
  *
  * On return the slot's tts_tid and tts_tableOid are updated to reflect the
  * insertion. But note that any toasting of fields within the slot is NOT
@@ -1248,6 +1227,8 @@ table_tuple_delete(Relation rel, ItemPointer tid, CommandId cid,
  * update was done.  However, any TOAST changes in the new tuple's
  * data are not reflected into *newtup.
  *
+ * See table_insert about skipping WAL-logging feature.
+ *
  * In the failure cases, the routine fills *tmfd with the tuple's t_ctid,
  * t_xmax, and, if possible, t_cmax.  See comments for struct TM_FailureData
  * for additional info.
@@ -1308,21 +1289,6 @@ table_tuple_lock(Relation rel, ItemPointer tid, Snapshot snapshot,
                                        flags, tmfd);
 }
 
-/*
- * 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.
- */
-static inline void
-table_finish_bulk_insert(Relation rel, int options)
-{
-    /* optional callback */
-    if (rel->rd_tableam && rel->rd_tableam->finish_bulk_insert)
-        rel->rd_tableam->finish_bulk_insert(rel, options);
-}
-
-
 /* ------------------------------------------------------------------------
  * DDL related functionality.
  * ------------------------------------------------------------------------
diff --git a/src/include/catalog/storage.h b/src/include/catalog/storage.h
index 3579d3f3eb..1de6f1655c 100644
--- a/src/include/catalog/storage.h
+++ b/src/include/catalog/storage.h
@@ -19,6 +19,13 @@
 #include "storage/smgr.h"
 #include "utils/relcache.h"
 
+/* enum for operation type of PendingDelete entries */
+typedef enum PendingOpType
+{
+    PENDING_DELETE,
+    PENDING_SYNC
+} PendingOpType;
+
 extern SMgrRelation RelationCreateStorage(RelFileNode rnode, char relpersistence);
 extern void RelationDropStorage(Relation rel);
 extern void RelationPreserveStorage(RelFileNode rnode, bool atCommit);
@@ -32,6 +39,7 @@ extern void RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
  */
 extern void smgrDoPendingDeletes(bool isCommit);
 extern int    smgrGetPendingDeletes(bool forCommit, RelFileNode **ptr);
+extern int    smgrGetPendingSyncs(bool forCommit, RelFileNode **ptr);
 extern void AtSubCommit_smgr(void);
 extern void AtSubAbort_smgr(void);
 extern void PostPrepare_smgr(void);
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index 509f4b7ef1..ace5f5a2ae 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -189,6 +189,7 @@ extern BlockNumber RelationGetNumberOfBlocksInFork(Relation relation,
                                                    ForkNumber forkNum);
 extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
+extern void FlushRelationBuffersWithoutRelcache(RelFileNode rnode, bool islocal);
 extern void FlushDatabaseBuffers(Oid dbid);
 extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode,
                                    ForkNumber forkNum, BlockNumber firstDelBlock);
diff --git a/src/include/storage/md.h b/src/include/storage/md.h
index c0f05e23ff..2bb2947bdb 100644
--- a/src/include/storage/md.h
+++ b/src/include/storage/md.h
@@ -42,6 +42,7 @@ extern void mdtruncate(SMgrRelation reln, ForkNumber forknum,
 extern void mdimmedsync(SMgrRelation reln, ForkNumber forknum);
 
 extern void ForgetDatabaseSyncRequests(Oid dbid);
+extern void SyncRelationFiles(RelFileNode *syncrels, int nsyncrels);
 extern void DropRelationFiles(RelFileNode *delrels, int ndelrels, bool isRedo);
 
 /* md sync callbacks */
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index c5d36680a2..f372dc2086 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -75,10 +75,17 @@ typedef struct RelationData
      * 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
+     * rd_firstRelfilenodeSubid is the ID of the first subtransaction the
+     * relfilenode change has took place in the current transaction. Unlike
+     * newRelfilenodeSubid, this won't be accidentially forgotten. A valid OID
+     * means that the currently active relfilenode is transaction-local and we
+     * sync the relation at commit instead of WAL-logging.
      */
     SubTransactionId rd_createSubid;    /* rel was created in current xact */
     SubTransactionId rd_newRelfilenodeSubid;    /* new relfilenode assigned in
                                                  * current xact */
+    SubTransactionId rd_firstRelfilenodeSubid;    /* new relfilenode assigned
+                                                 * first in current xact */
 
     Form_pg_class rd_rel;        /* RELATION tuple */
     TupleDesc    rd_att;            /* tuple descriptor */
@@ -514,9 +521,15 @@ typedef struct ViewOptions
 /*
  * RelationNeedsWAL
  *        True if relation needs WAL.
+ *
+ * Returns false if wal_level = minimal and this relation is created or
+ * truncated in the current transaction.
  */
-#define RelationNeedsWAL(relation) \
-    ((relation)->rd_rel->relpersistence == RELPERSISTENCE_PERMANENT)
+#define RelationNeedsWAL(relation)                                        \
+    ((relation)->rd_rel->relpersistence == RELPERSISTENCE_PERMANENT &&    \
+     (XLogIsNeeded() ||                                                    \
+      (relation->rd_createSubid == InvalidSubTransactionId &&            \
+       relation->rd_firstRelfilenodeSubid == InvalidSubTransactionId)))
 
 /*
  * RelationUsesLocalBuffers
-- 
2.16.3

From 6f6b87ef06e26ad8222f5900f8e3b146d2f18cba Mon Sep 17 00:00:00 2001
From: Kyotaro Horiguchi <horiguchi.kyotaro@oss.ntt.co.jp>
Date: Wed, 29 May 2019 23:03:22 +0900
Subject: [PATCH 3/3] Rename smgrDoPendingDeletes to smgrDoPendingOperations

The function longer does only deletions but also syncs. Rename the
function to reflect that. smgrGetPendingDeletes is not renamed since it
does not change behavior.
---
 src/backend/access/transam/xact.c |  4 +-
 src/backend/catalog/storage.c     | 91 ++++++++++++++++++++-------------------
 src/include/catalog/storage.h     |  2 +-
 3 files changed, 49 insertions(+), 48 deletions(-)

diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index f594d33e7a..0123fb0f7f 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -2228,7 +2228,7 @@ CommitTransaction(void)
      * Other backends will observe the attendant catalog changes and not
      * attempt to access affected files.
      */
-    smgrDoPendingDeletes(true);
+    smgrDoPendingOperations(true);
 
     AtCommit_Notify();
     AtEOXact_GUC(true, 1);
@@ -2716,7 +2716,7 @@ AbortTransaction(void)
         ResourceOwnerRelease(TopTransactionResourceOwner,
                              RESOURCE_RELEASE_AFTER_LOCKS,
                              false, true);
-        smgrDoPendingDeletes(false);
+        smgrDoPendingOperations(false);
 
         AtEOXact_GUC(false, 1);
         AtEOXact_SPI(false);
diff --git a/src/backend/catalog/storage.c b/src/backend/catalog/storage.c
index 354a74c27c..544ef3aa55 100644
--- a/src/backend/catalog/storage.c
+++ b/src/backend/catalog/storage.c
@@ -54,18 +54,18 @@
  * but I'm being paranoid.
  */
 
-/* entry type of pendingDeletes */
-typedef struct PendingRelDelete
+/* entry type of pendingOperations */
+typedef struct PendingRelOp
 {
     RelFileNode relnode;        /* relation that may need to be deleted */
     BackendId    backend;        /* InvalidBackendId if not a temp rel */
     bool        atCommit;        /* T=work at commit; F=work at abort */
     PendingOpType    op;            /* type of operation to do */
     int            nestLevel;        /* xact nesting level of request */
-    struct PendingRelDelete *next;    /* linked-list link */
-} PendingRelDelete;
+    struct PendingRelOp *next;    /* linked-list link */
+} PendingRelOp;
 
-static PendingRelDelete *pendingDeletes = NULL; /* head of linked list */
+static PendingRelOp *pendingOperations = NULL; /* head of linked list */
 
 /*
  * RelationCreateStorage
@@ -81,7 +81,7 @@ static PendingRelDelete *pendingDeletes = NULL; /* head of linked list */
 SMgrRelation
 RelationCreateStorage(RelFileNode rnode, char relpersistence)
 {
-    PendingRelDelete *pending;
+    PendingRelOp *pending;
     SMgrRelation srel;
     BackendId    backend;
     bool        needs_wal;
@@ -112,15 +112,15 @@ RelationCreateStorage(RelFileNode rnode, char relpersistence)
         log_smgrcreate(&srel->smgr_rnode.node, MAIN_FORKNUM);
 
     /* Add the relation to the list of stuff to delete at abort */
-    pending = (PendingRelDelete *)
-        MemoryContextAlloc(TopMemoryContext, sizeof(PendingRelDelete));
+    pending = (PendingRelOp *)
+        MemoryContextAlloc(TopMemoryContext, sizeof(PendingRelOp));
     pending->relnode = rnode;
     pending->backend = backend;
     pending->atCommit = false;    /* delete if abort */
     pending->op = PENDING_DELETE;
     pending->nestLevel = GetCurrentTransactionNestLevel();
-    pending->next = pendingDeletes;
-    pendingDeletes = pending;
+    pending->next = pendingOperations;
+    pendingOperations = pending;
 
     /*
      * We are going to skip WAL-logging for storage of persistent relations
@@ -129,15 +129,15 @@ RelationCreateStorage(RelFileNode rnode, char relpersistence)
      */
     if (relpersistence == RELPERSISTENCE_PERMANENT && !XLogIsNeeded())
     {
-        pending = (PendingRelDelete *)
-            MemoryContextAlloc(TopMemoryContext, sizeof(PendingRelDelete));
+        pending = (PendingRelOp *)
+            MemoryContextAlloc(TopMemoryContext, sizeof(PendingRelOp));
         pending->relnode = rnode;
         pending->backend = backend;
         pending->atCommit = true;
         pending->op = PENDING_SYNC;
         pending->nestLevel = GetCurrentTransactionNestLevel();
-        pending->next = pendingDeletes;
-        pendingDeletes = pending;
+        pending->next = pendingOperations;
+        pendingOperations = pending;
     }
 
     return srel;
@@ -169,27 +169,27 @@ log_smgrcreate(const RelFileNode *rnode, ForkNumber forkNum)
 void
 RelationDropStorage(Relation rel)
 {
-    PendingRelDelete *pending;
+    PendingRelOp *pending;
 
     /* Add the relation to the list of stuff to delete at commit */
-    pending = (PendingRelDelete *)
-        MemoryContextAlloc(TopMemoryContext, sizeof(PendingRelDelete));
+    pending = (PendingRelOp *)
+        MemoryContextAlloc(TopMemoryContext, sizeof(PendingRelOp));
     pending->relnode = rel->rd_node;
     pending->backend = rel->rd_backend;
     pending->atCommit = true;    /* delete if commit */
     pending->op = PENDING_DELETE;
     pending->nestLevel = GetCurrentTransactionNestLevel();
-    pending->next = pendingDeletes;
-    pendingDeletes = pending;
+    pending->next = pendingOperations;
+    pendingOperations = pending;
 
     /*
      * NOTE: if the relation was created in this transaction, it will now be
      * present in the pending-delete list twice, once with atCommit true and
      * once with atCommit false.  Hence, it will be physically deleted at end
      * of xact in either case (and the other entry will be ignored by
-     * smgrDoPendingDeletes, so no error will occur).  We could instead remove
-     * the existing list entry and delete the physical file immediately, but
-     * for now I'll keep the logic simple.
+     * smgrDoPendingOperations, so no error will occur).  We could instead
+     * remove the existing list entry and delete the physical file
+     * immediately, but for now I'll keep the logic simple.
      */
 
     RelationCloseSmgr(rel);
@@ -215,12 +215,12 @@ RelationDropStorage(Relation rel)
 void
 RelationPreserveStorage(RelFileNode rnode, bool atCommit)
 {
-    PendingRelDelete *pending;
-    PendingRelDelete *prev;
-    PendingRelDelete *next;
+    PendingRelOp *pending;
+    PendingRelOp *prev;
+    PendingRelOp *next;
 
     prev = NULL;
-    for (pending = pendingDeletes; pending != NULL; pending = next)
+    for (pending = pendingOperations; pending != NULL; pending = next)
     {
         next = pending->next;
         if (RelFileNodeEquals(rnode, pending->relnode)
@@ -231,7 +231,7 @@ RelationPreserveStorage(RelFileNode rnode, bool atCommit)
             if (prev)
                 prev->next = next;
             else
-                pendingDeletes = next;
+                pendingOperations = next;
             pfree(pending);
             /* prev does not change */
         }
@@ -409,7 +409,8 @@ RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
 }
 
 /*
- *    smgrDoPendingDeletes() -- Take care of relation deletes at end of xact.
+ *    smgrDoPendingOperations() -- Take care of relation deletes and syncs at
+ *                                 end of xact.
  *
  * This also runs when aborting a subxact; we want to clean up a failed
  * subxact immediately.
@@ -420,12 +421,12 @@ RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
  * already recovered the physical storage.
  */
 void
-smgrDoPendingDeletes(bool isCommit)
+smgrDoPendingOperations(bool isCommit)
 {
     int            nestLevel = GetCurrentTransactionNestLevel();
-    PendingRelDelete *pending;
-    PendingRelDelete *prev;
-    PendingRelDelete *next;
+    PendingRelOp *pending;
+    PendingRelOp *prev;
+    PendingRelOp *next;
     int            nrels = 0,
                 i = 0,
                 maxrels = 0;
@@ -433,7 +434,7 @@ smgrDoPendingDeletes(bool isCommit)
     struct HTAB *synchash = NULL;
 
     prev = NULL;
-    for (pending = pendingDeletes; pending != NULL; pending = next)
+    for (pending = pendingOperations; pending != NULL; pending = next)
     {
         next = pending->next;
         if (pending->nestLevel < nestLevel)
@@ -447,7 +448,7 @@ smgrDoPendingDeletes(bool isCommit)
             if (prev)
                 prev->next = next;
             else
-                pendingDeletes = next;
+                pendingOperations = next;
             /* do deletion if called for */
             if (pending->atCommit == isCommit)
             {
@@ -576,10 +577,10 @@ smgrGetPendingOperations(PendingOpType op, bool forCommit, RelFileNode **ptr)
     int            nestLevel = GetCurrentTransactionNestLevel();
     int            nrels;
     RelFileNode *rptr;
-    PendingRelDelete *pending;
+    PendingRelOp *pending;
 
     nrels = 0;
-    for (pending = pendingDeletes; pending != NULL; pending = pending->next)
+    for (pending = pendingOperations; pending != NULL; pending = pending->next)
     {
         if (pending->nestLevel >= nestLevel && pending->atCommit == forCommit
             && pending->backend == InvalidBackendId
@@ -593,7 +594,7 @@ smgrGetPendingOperations(PendingOpType op, bool forCommit, RelFileNode **ptr)
     }
     rptr = (RelFileNode *) palloc(nrels * sizeof(RelFileNode));
     *ptr = rptr;
-    for (pending = pendingDeletes; pending != NULL; pending = pending->next)
+    for (pending = pendingOperations; pending != NULL; pending = pending->next)
     {
         if (pending->nestLevel >= nestLevel && pending->atCommit == forCommit
             && pending->backend == InvalidBackendId
@@ -630,13 +631,13 @@ smgrGetPendingSyncs(bool forCommit, RelFileNode **ptr)
 void
 PostPrepare_smgr(void)
 {
-    PendingRelDelete *pending;
-    PendingRelDelete *next;
+    PendingRelOp *pending;
+    PendingRelOp *next;
 
-    for (pending = pendingDeletes; pending != NULL; pending = next)
+    for (pending = pendingOperations; pending != NULL; pending = next)
     {
         next = pending->next;
-        pendingDeletes = next;
+        pendingOperations = next;
         /* must explicitly free the list entry */
         pfree(pending);
     }
@@ -646,15 +647,15 @@ PostPrepare_smgr(void)
 /*
  * AtSubCommit_smgr() --- Take care of subtransaction commit.
  *
- * Reassign all items in the pending-deletes list to the parent transaction.
+ * Reassign all items in the pending-operations list to the parent transaction.
  */
 void
 AtSubCommit_smgr(void)
 {
     int            nestLevel = GetCurrentTransactionNestLevel();
-    PendingRelDelete *pending;
+    PendingRelOp *pending;
 
-    for (pending = pendingDeletes; pending != NULL; pending = pending->next)
+    for (pending = pendingOperations; pending != NULL; pending = pending->next)
     {
         if (pending->nestLevel >= nestLevel)
             pending->nestLevel = nestLevel - 1;
@@ -671,7 +672,7 @@ AtSubCommit_smgr(void)
 void
 AtSubAbort_smgr(void)
 {
-    smgrDoPendingDeletes(false);
+    smgrDoPendingOperations(false);
 }
 
 void
diff --git a/src/include/catalog/storage.h b/src/include/catalog/storage.h
index 1de6f1655c..dcb3bc4b69 100644
--- a/src/include/catalog/storage.h
+++ b/src/include/catalog/storage.h
@@ -37,7 +37,7 @@ extern void RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
  * These functions used to be in storage/smgr/smgr.c, which explains the
  * naming
  */
-extern void smgrDoPendingDeletes(bool isCommit);
+extern void smgrDoPendingOperations(bool isCommit);
 extern int    smgrGetPendingDeletes(bool forCommit, RelFileNode **ptr);
 extern int    smgrGetPendingSyncs(bool forCommit, RelFileNode **ptr);
 extern void AtSubCommit_smgr(void);
-- 
2.16.3


В списке pgsql-hackers по дате отправления:

Предыдущее
От: Peter Eisentraut
Дата:
Сообщение: Re: ICU for global collation
Следующее
От: Michael Paquier
Дата:
Сообщение: Re: Fix typos and inconsistencies for HEAD (take 11)